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/04/11 09:47:31 UTC

[3/3] lucene-solr:branch_7x: LUCENE-8229: Add Weight.matches() to iterate over match positions

LUCENE-8229: Add Weight.matches() to iterate over match positions


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

Branch: refs/heads/branch_7x
Commit: 502fd4bf12b8860b8eea504a96ad1b49dd52938c
Parents: 2376b6c
Author: Alan Woodward <ro...@apache.org>
Authored: Wed Apr 11 09:43:27 2018 +0100
Committer: Alan Woodward <ro...@apache.org>
Committed: Wed Apr 11 09:51:07 2018 +0100

----------------------------------------------------------------------
 lucene/CHANGES.txt                              |   5 +
 .../org/apache/lucene/search/BooleanWeight.java |  35 ++
 .../lucene/search/ConstantScoreQuery.java       |   5 +
 .../search/DisjunctionMatchesIterator.java      | 171 +++++++
 .../lucene/search/DisjunctionMaxQuery.java      |  12 +
 .../lucene/search/DocValuesRewriteMethod.java   |  20 +-
 .../org/apache/lucene/search/FilterWeight.java  |   4 +
 .../lucene/search/IndexOrDocValuesQuery.java    |   6 +
 .../org/apache/lucene/search/LRUQueryCache.java |   5 +
 .../java/org/apache/lucene/search/Matches.java  | 146 ++++++
 .../apache/lucene/search/MatchesIterator.java   |  79 ++++
 .../MultiTermQueryConstantScoreWrapper.java     |  12 +
 .../org/apache/lucene/search/SynonymQuery.java  |  11 +
 .../apache/lucene/search/TermInSetQuery.java    |   9 +
 .../lucene/search/TermMatchesIterator.java      |  77 ++++
 .../org/apache/lucene/search/TermQuery.java     |  18 +
 .../java/org/apache/lucene/search/Weight.java   |  29 ++
 .../lucene/search/TestMatchesIterator.java      | 440 +++++++++++++++++++
 .../search/join/ToParentBlockJoinQuery.java     |  23 +
 .../queries/function/FunctionScoreQuery.java    |   6 +
 .../org/apache/lucene/search/CoveringQuery.java |  22 +
 .../apache/lucene/search/AssertingMatches.java  |  43 ++
 .../lucene/search/AssertingMatchesIterator.java |  77 ++++
 .../apache/lucene/search/AssertingWeight.java   |   8 +
 .../org/apache/lucene/search/CheckHits.java     | 340 ++++++++------
 .../org/apache/lucene/search/QueryUtils.java    |   2 +
 26 files changed, 1467 insertions(+), 138 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/502fd4bf/lucene/CHANGES.txt
----------------------------------------------------------------------
diff --git a/lucene/CHANGES.txt b/lucene/CHANGES.txt
index 7e47911..0c7026b 100644
--- a/lucene/CHANGES.txt
+++ b/lucene/CHANGES.txt
@@ -41,6 +41,11 @@ New Features
   soft deletes if the reader is opened form a directory. (Simon Willnauer,
   Mike McCandless, Uwe Schindler, Adrien Grand)
 
+* LUCENE-8229: Add a method Weight.matches(LeafReaderContext, doc) that returns
+  an iterator over matching positions for a given query and document.  This
+  allows exact hit extraction and will enable implementation of accurate 
+  highlighters. (Alan Woodward, Adrien Grand, David Smiley)
+
 Bug Fixes
 
 * LUCENE-8234: Fixed bug in how spatial relationship is computed for

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/502fd4bf/lucene/core/src/java/org/apache/lucene/search/BooleanWeight.java
----------------------------------------------------------------------
diff --git a/lucene/core/src/java/org/apache/lucene/search/BooleanWeight.java b/lucene/core/src/java/org/apache/lucene/search/BooleanWeight.java
index dbe3d17..96bfdac 100644
--- a/lucene/core/src/java/org/apache/lucene/search/BooleanWeight.java
+++ b/lucene/core/src/java/org/apache/lucene/search/BooleanWeight.java
@@ -114,6 +114,41 @@ final class BooleanWeight extends Weight {
     }
   }
 
+  @Override
+  public Matches matches(LeafReaderContext context, int doc) throws IOException {
+    final int minShouldMatch = query.getMinimumNumberShouldMatch();
+    List<Matches> matches = new ArrayList<>();
+    int shouldMatchCount = 0;
+    Iterator<Weight> wIt = weights.iterator();
+    Iterator<BooleanClause> cIt = query.clauses().iterator();
+    while (wIt.hasNext()) {
+      Weight w = wIt.next();
+      BooleanClause bc = cIt.next();
+      Matches m = w.matches(context, doc);
+      if (bc.isProhibited()) {
+        if (m != null) {
+          return null;
+        }
+      }
+      if (bc.isRequired()) {
+        if (m == null) {
+          return null;
+        }
+        matches.add(m);
+      }
+      if (bc.getOccur() == Occur.SHOULD) {
+        if (m != null) {
+          matches.add(m);
+          shouldMatchCount++;
+        }
+      }
+    }
+    if (shouldMatchCount < minShouldMatch) {
+      return null;
+    }
+    return Matches.fromSubMatches(matches);
+  }
+
   static BulkScorer disableScoring(final BulkScorer scorer) {
     return new BulkScorer() {
 

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/502fd4bf/lucene/core/src/java/org/apache/lucene/search/ConstantScoreQuery.java
----------------------------------------------------------------------
diff --git a/lucene/core/src/java/org/apache/lucene/search/ConstantScoreQuery.java b/lucene/core/src/java/org/apache/lucene/search/ConstantScoreQuery.java
index 90cc5b4..3ee4158 100644
--- a/lucene/core/src/java/org/apache/lucene/search/ConstantScoreQuery.java
+++ b/lucene/core/src/java/org/apache/lucene/search/ConstantScoreQuery.java
@@ -151,6 +151,11 @@ public final class ConstantScoreQuery extends Query {
         }
 
         @Override
+        public Matches matches(LeafReaderContext context, int doc) throws IOException {
+          return innerWeight.matches(context, doc);
+        }
+
+        @Override
         public Scorer scorer(LeafReaderContext context) throws IOException {
           ScorerSupplier scorerSupplier = scorerSupplier(context);
           if (scorerSupplier == null) {

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/502fd4bf/lucene/core/src/java/org/apache/lucene/search/DisjunctionMatchesIterator.java
----------------------------------------------------------------------
diff --git a/lucene/core/src/java/org/apache/lucene/search/DisjunctionMatchesIterator.java b/lucene/core/src/java/org/apache/lucene/search/DisjunctionMatchesIterator.java
new file mode 100644
index 0000000..37770d2
--- /dev/null
+++ b/lucene/core/src/java/org/apache/lucene/search/DisjunctionMatchesIterator.java
@@ -0,0 +1,171 @@
+/*
+ * 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 java.util.Objects;
+
+import org.apache.lucene.index.LeafReaderContext;
+import org.apache.lucene.index.PostingsEnum;
+import org.apache.lucene.index.Term;
+import org.apache.lucene.index.Terms;
+import org.apache.lucene.index.TermsEnum;
+import org.apache.lucene.util.BytesRef;
+import org.apache.lucene.util.BytesRefIterator;
+import org.apache.lucene.util.PriorityQueue;
+
+/**
+ * A {@link MatchesIterator} that combines matches from a set of sub-iterators
+ *
+ * Matches are sorted by their start positions, and then by their end positions, so that
+ * prefixes sort first.  Matches may overlap, or be duplicated if they appear in more
+ * than one of the sub-iterators.
+ */
+final class DisjunctionMatchesIterator implements MatchesIterator {
+
+  /**
+   * Create a {@link DisjunctionMatchesIterator} over a list of terms
+   *
+   * Only terms that have at least one match in the given document will be included
+   */
+  static MatchesIterator fromTerms(LeafReaderContext context, int doc, String field, List<Term> terms) throws IOException {
+    Objects.requireNonNull(field);
+    for (Term term : terms) {
+      if (Objects.equals(field, term.field()) == false) {
+        throw new IllegalArgumentException("Tried to generate iterator from terms in multiple fields: expected [" + field + "] but got [" + term.field() + "]");
+      }
+    }
+    return fromTermsEnum(context, doc, field, asBytesRefIterator(terms));
+  }
+
+  private static BytesRefIterator asBytesRefIterator(List<Term> terms) {
+    return new BytesRefIterator() {
+      int i = 0;
+      @Override
+      public BytesRef next() {
+        if (i >= terms.size())
+          return null;
+        return terms.get(i++).bytes();
+      }
+    };
+  }
+
+  /**
+   * Create a {@link DisjunctionMatchesIterator} over a list of terms extracted from a {@link BytesRefIterator}
+   *
+   * Only terms that have at least one match in the given document will be included
+   */
+  static MatchesIterator fromTermsEnum(LeafReaderContext context, int doc, String field, BytesRefIterator terms) throws IOException {
+    Objects.requireNonNull(field);
+    List<MatchesIterator> mis = new ArrayList<>();
+    Terms t = context.reader().terms(field);
+    if (t == null)
+      return null;
+    TermsEnum te = t.iterator();
+    PostingsEnum reuse = null;
+    for (BytesRef term = terms.next(); term != null; term = terms.next()) {
+      if (te.seekExact(term)) {
+        PostingsEnum pe = te.postings(reuse, PostingsEnum.OFFSETS);
+        if (pe.advance(doc) == doc) {
+          // TODO do we want to use the copied term here, or instead create a label that associates all of the TMIs with a single term?
+          mis.add(new TermMatchesIterator(BytesRef.deepCopyOf(term), pe));
+          reuse = null;
+        }
+        else {
+          reuse = pe;
+        }
+      }
+    }
+    if (mis.size() == 0)
+      return null;
+    if (mis.size() == 1)
+      return mis.get(0);
+    return new DisjunctionMatchesIterator(mis);
+  }
+
+  static MatchesIterator fromSubIterators(List<MatchesIterator> mis) throws IOException {
+    if (mis.size() == 0)
+      return null;
+    if (mis.size() == 1)
+      return mis.get(0);
+    return new DisjunctionMatchesIterator(mis);
+  }
+
+  private final PriorityQueue<MatchesIterator> queue;
+
+  private boolean started = false;
+
+  private DisjunctionMatchesIterator(List<MatchesIterator> matches) throws IOException {
+    queue = new PriorityQueue<MatchesIterator>(matches.size()){
+      @Override
+      protected boolean lessThan(MatchesIterator a, MatchesIterator b) {
+        return a.startPosition() < b.startPosition() ||
+            (a.startPosition() == b.startPosition() && a.endPosition() < b.endPosition()) ||
+            (a.startPosition() == b.startPosition() && a.endPosition() == b.endPosition() && a.term().compareTo(b.term()) < 0);
+      }
+    };
+    for (MatchesIterator mi : matches) {
+      if (mi.next()) {
+        queue.add(mi);
+      }
+    }
+  }
+
+  @Override
+  public boolean next() throws IOException {
+    if (started == false) {
+      return started = true;
+    }
+    if (queue.top().next() == false) {
+      queue.pop();
+    }
+    if (queue.size() > 0) {
+      queue.updateTop();
+      return true;
+    }
+    return false;
+  }
+
+  @Override
+  public int startPosition() {
+    return queue.top().startPosition();
+  }
+
+  @Override
+  public int endPosition() {
+    return queue.top().endPosition();
+  }
+
+  @Override
+  public int startOffset() throws IOException {
+    return queue.top().startOffset();
+  }
+
+  @Override
+  public int endOffset() throws IOException {
+    return queue.top().endOffset();
+  }
+
+  @Override
+  public BytesRef term() {
+    return queue.top().term();
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/502fd4bf/lucene/core/src/java/org/apache/lucene/search/DisjunctionMaxQuery.java
----------------------------------------------------------------------
diff --git a/lucene/core/src/java/org/apache/lucene/search/DisjunctionMaxQuery.java b/lucene/core/src/java/org/apache/lucene/search/DisjunctionMaxQuery.java
index 13237a2..29a57e3 100644
--- a/lucene/core/src/java/org/apache/lucene/search/DisjunctionMaxQuery.java
+++ b/lucene/core/src/java/org/apache/lucene/search/DisjunctionMaxQuery.java
@@ -115,6 +115,18 @@ public final class DisjunctionMaxQuery extends Query implements Iterable<Query>
       }
     }
 
+    @Override
+    public Matches matches(LeafReaderContext context, int doc) throws IOException {
+      List<Matches> mis = new ArrayList<>();
+      for (Weight weight : weights) {
+        Matches mi = weight.matches(context, doc);
+        if (mi != null) {
+          mis.add(mi);
+        }
+      }
+      return Matches.fromSubMatches(mis);
+    }
+
     /** Create the scorer used to score our associated DisjunctionMaxQuery */
     @Override
     public Scorer scorer(LeafReaderContext context) throws IOException {

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/502fd4bf/lucene/core/src/java/org/apache/lucene/search/DocValuesRewriteMethod.java
----------------------------------------------------------------------
diff --git a/lucene/core/src/java/org/apache/lucene/search/DocValuesRewriteMethod.java b/lucene/core/src/java/org/apache/lucene/search/DocValuesRewriteMethod.java
index 602864d..7034328 100644
--- a/lucene/core/src/java/org/apache/lucene/search/DocValuesRewriteMethod.java
+++ b/lucene/core/src/java/org/apache/lucene/search/DocValuesRewriteMethod.java
@@ -74,11 +74,16 @@ public final class DocValuesRewriteMethod extends MultiTermQuery.RewriteMethod {
     @Override
     public Weight createWeight(IndexSearcher searcher, boolean needsScores, float boost) throws IOException {
       return new ConstantScoreWeight(this, boost) {
+
         @Override
-        public Scorer scorer(LeafReaderContext context) throws IOException {
+        public Matches matches(LeafReaderContext context, int doc) throws IOException {
           final SortedSetDocValues fcsi = DocValues.getSortedSet(context.reader(), query.field);
-          TermsEnum termsEnum = query.getTermsEnum(new Terms() {
-            
+          return Matches.forField(query.field, () -> DisjunctionMatchesIterator.fromTermsEnum(context, doc, query.field, getTermsEnum(fcsi)));
+        }
+
+        private TermsEnum getTermsEnum(SortedSetDocValues fcsi) throws IOException {
+          return query.getTermsEnum(new Terms() {
+
             @Override
             public TermsEnum iterator() throws IOException {
               return fcsi.termsEnum();
@@ -118,13 +123,18 @@ public final class DocValuesRewriteMethod extends MultiTermQuery.RewriteMethod {
             public boolean hasPositions() {
               return false;
             }
-            
+
             @Override
             public boolean hasPayloads() {
               return false;
             }
           });
-          
+        }
+
+        @Override
+        public Scorer scorer(LeafReaderContext context) throws IOException {
+          final SortedSetDocValues fcsi = DocValues.getSortedSet(context.reader(), query.field);
+          TermsEnum termsEnum = getTermsEnum(fcsi);
           assert termsEnum != null;
           if (termsEnum.next() == null) {
             // no matching terms

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/502fd4bf/lucene/core/src/java/org/apache/lucene/search/FilterWeight.java
----------------------------------------------------------------------
diff --git a/lucene/core/src/java/org/apache/lucene/search/FilterWeight.java b/lucene/core/src/java/org/apache/lucene/search/FilterWeight.java
index 380ad40..5921915 100644
--- a/lucene/core/src/java/org/apache/lucene/search/FilterWeight.java
+++ b/lucene/core/src/java/org/apache/lucene/search/FilterWeight.java
@@ -75,4 +75,8 @@ public abstract class FilterWeight extends Weight {
     return in.scorer(context);
   }
 
+  @Override
+  public Matches matches(LeafReaderContext context, int doc) throws IOException {
+    return in.matches(context, doc);
+  }
 }

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/502fd4bf/lucene/core/src/java/org/apache/lucene/search/IndexOrDocValuesQuery.java
----------------------------------------------------------------------
diff --git a/lucene/core/src/java/org/apache/lucene/search/IndexOrDocValuesQuery.java b/lucene/core/src/java/org/apache/lucene/search/IndexOrDocValuesQuery.java
index 06fb9c2..944f7d3 100644
--- a/lucene/core/src/java/org/apache/lucene/search/IndexOrDocValuesQuery.java
+++ b/lucene/core/src/java/org/apache/lucene/search/IndexOrDocValuesQuery.java
@@ -120,6 +120,12 @@ public final class IndexOrDocValuesQuery extends Query {
       }
 
       @Override
+      public Matches matches(LeafReaderContext context, int doc) throws IOException {
+        // We need to check a single doc, so the dv query should perform better
+        return dvWeight.matches(context, doc);
+      }
+
+      @Override
       public Explanation explain(LeafReaderContext context, int doc) throws IOException {
         // We need to check a single doc, so the dv query should perform better
         return dvWeight.explain(context, doc);

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/502fd4bf/lucene/core/src/java/org/apache/lucene/search/LRUQueryCache.java
----------------------------------------------------------------------
diff --git a/lucene/core/src/java/org/apache/lucene/search/LRUQueryCache.java b/lucene/core/src/java/org/apache/lucene/search/LRUQueryCache.java
index 5c828d6..fcd84d0 100644
--- a/lucene/core/src/java/org/apache/lucene/search/LRUQueryCache.java
+++ b/lucene/core/src/java/org/apache/lucene/search/LRUQueryCache.java
@@ -676,6 +676,11 @@ public class LRUQueryCache implements QueryCache, Accountable {
       in.extractTerms(terms);
     }
 
+    @Override
+    public Matches matches(LeafReaderContext context, int doc) throws IOException {
+      return in.matches(context, doc);
+    }
+
     private boolean cacheEntryHasReasonableWorstCaseSize(int maxDoc) {
       // The worst-case (dense) is a bit set which needs one bit per document
       final long worstCaseRamUsage = maxDoc / 8;

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/502fd4bf/lucene/core/src/java/org/apache/lucene/search/Matches.java
----------------------------------------------------------------------
diff --git a/lucene/core/src/java/org/apache/lucene/search/Matches.java b/lucene/core/src/java/org/apache/lucene/search/Matches.java
new file mode 100644
index 0000000..3670563
--- /dev/null
+++ b/lucene/core/src/java/org/apache/lucene/search/Matches.java
@@ -0,0 +1,146 @@
+/*
+ * 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.Collections;
+import java.util.HashSet;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Objects;
+import java.util.Set;
+import java.util.stream.Collectors;
+
+/**
+ * Reports the positions and optionally offsets of all matching terms in a query
+ * for a single document
+ *
+ * To obtain a {@link MatchesIterator} for a particular field, call {@link #getMatches(String)}.
+ * Note that you can call {@link #getMatches(String)} multiple times to retrieve new
+ * iterators, but it is not thread-safe.
+ */
+public interface Matches extends Iterable<String> {
+
+  /**
+   * Returns a {@link MatchesIterator} over the matches for a single field,
+   * or {@code null} if there are no matches in that field.
+   */
+  MatchesIterator getMatches(String field) throws IOException;
+
+  /**
+   * Indicates a match with no term positions, for example on a Point or DocValues field,
+   * or a field indexed as docs and freqs only
+   */
+  Matches MATCH_WITH_NO_TERMS = new Matches() {
+    @Override
+    public Iterator<String> iterator() {
+      return Collections.emptyIterator();
+    }
+
+    @Override
+    public MatchesIterator getMatches(String field) {
+      return null;
+    }
+  };
+
+  /**
+   * Amalgamate a collection of {@link Matches} into a single object
+   */
+  static Matches fromSubMatches(List<Matches> subMatches) {
+    if (subMatches == null || subMatches.size() == 0) {
+      return null;
+    }
+    List<Matches> sm = subMatches.stream().filter(m -> m != MATCH_WITH_NO_TERMS).collect(Collectors.toList());
+    if (sm.size() == 0) {
+      return MATCH_WITH_NO_TERMS;
+    }
+    if (sm.size() == 1) {
+      return sm.get(0);
+    }
+    Set<String> fields = new HashSet<>();
+    for (Matches m : sm) {
+      for (String field : m) {
+        fields.add(field);
+      }
+    }
+    return new Matches() {
+      @Override
+      public MatchesIterator getMatches(String field) throws IOException {
+        List<MatchesIterator> subIterators = new ArrayList<>();
+        for (Matches m : sm) {
+          MatchesIterator it = m.getMatches(field);
+          if (it != null) {
+            subIterators.add(it);
+          }
+        }
+        return DisjunctionMatchesIterator.fromSubIterators(subIterators);
+      }
+
+      @Override
+      public Iterator<String> iterator() {
+        return fields.iterator();
+      }
+    };
+  }
+
+  /**
+   * A functional interface that supplies a {@link MatchesIterator}
+   */
+  @FunctionalInterface
+  interface MatchesIteratorSupplier {
+    /** Return a new {@link MatchesIterator} */
+    MatchesIterator get() throws IOException;
+  }
+
+  /**
+   * Create a Matches for a single field
+   */
+  static Matches forField(String field, MatchesIteratorSupplier mis) throws IOException {
+
+    // The indirection here, using a Supplier object rather than a MatchesIterator
+    // directly, is to allow for multiple calls to Matches.getMatches() to return
+    // new iterators.  We still need to call MatchesIteratorSupplier.get() eagerly
+    // to work out if we have a hit or not.
+
+    MatchesIterator mi = mis.get();
+    if (mi == null) {
+      return null;
+    }
+    return new Matches() {
+      boolean cached = true;
+      @Override
+      public MatchesIterator getMatches(String f) throws IOException {
+        if (Objects.equals(field, f) == false) {
+          return null;
+        }
+        if (cached == false) {
+          return mis.get();
+        }
+        cached = false;
+        return mi;
+      }
+
+      @Override
+      public Iterator<String> iterator() {
+        return Collections.singleton(field).iterator();
+      }
+    };
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/502fd4bf/lucene/core/src/java/org/apache/lucene/search/MatchesIterator.java
----------------------------------------------------------------------
diff --git a/lucene/core/src/java/org/apache/lucene/search/MatchesIterator.java b/lucene/core/src/java/org/apache/lucene/search/MatchesIterator.java
new file mode 100644
index 0000000..b874263
--- /dev/null
+++ b/lucene/core/src/java/org/apache/lucene/search/MatchesIterator.java
@@ -0,0 +1,79 @@
+/*
+ * 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.util.BytesRef;
+
+/**
+ * An iterator over match positions (and optionally offsets) for a single document and field
+ *
+ * To iterate over the matches, call {@link #next()} until it returns {@code false}, retrieving
+ * positions and/or offsets after each call.  You should not call the position or offset methods
+ * before {@link #next()} has been called, or after {@link #next()} has returned {@code false}.
+ *
+ * Matches are ordered by start position, and then by end position.  Match intervals may overlap.
+ *
+ * @see Weight#matches(LeafReaderContext, int)
+ */
+public interface MatchesIterator {
+
+  /**
+   * Advance the iterator to the next match position
+   * @return {@code true} if matches have not been exhausted
+   */
+  boolean next() throws IOException;
+
+  /**
+   * The start position of the current match
+   *
+   * Should only be called after {@link #next()} has returned {@code true}
+   */
+  int startPosition();
+
+  /**
+   * The end position of the current match
+   *
+   * Should only be called after {@link #next()} has returned {@code true}
+   */
+  int endPosition();
+
+  /**
+   * The starting offset of the current match, or {@code -1} if offsets are not available
+   *
+   * Should only be called after {@link #next()} has returned {@code true}
+   */
+  int startOffset() throws IOException;
+
+  /**
+   * The ending offset of the current match, or {@code -1} if offsets are not available
+   *
+   * Should only be called after {@link #next()} has returned {@code true}
+   */
+  int endOffset() throws IOException;
+
+  /**
+   * The underlying term of the current match
+   *
+   * Should only be called after {@link #next()} has returned {@code true}
+   */
+  BytesRef term();
+
+}

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/502fd4bf/lucene/core/src/java/org/apache/lucene/search/MultiTermQueryConstantScoreWrapper.java
----------------------------------------------------------------------
diff --git a/lucene/core/src/java/org/apache/lucene/search/MultiTermQueryConstantScoreWrapper.java b/lucene/core/src/java/org/apache/lucene/search/MultiTermQueryConstantScoreWrapper.java
index 705946f..161d96d 100644
--- a/lucene/core/src/java/org/apache/lucene/search/MultiTermQueryConstantScoreWrapper.java
+++ b/lucene/core/src/java/org/apache/lucene/search/MultiTermQueryConstantScoreWrapper.java
@@ -203,6 +203,18 @@ final class MultiTermQueryConstantScoreWrapper<Q extends MultiTermQuery> extends
       }
 
       @Override
+      public Matches matches(LeafReaderContext context, int doc) throws IOException {
+        final Terms terms = context.reader().terms(query.field);
+        if (terms == null) {
+          return null;
+        }
+        if (terms.hasPositions() == false) {
+          return super.matches(context, doc);
+        }
+        return Matches.forField(query.field, () -> DisjunctionMatchesIterator.fromTermsEnum(context, doc, query.field, query.getTermsEnum(terms)));
+      }
+
+      @Override
       public Scorer scorer(LeafReaderContext context) throws IOException {
         final WeightOrDocIdSet weightOrBitSet = rewrite(context);
         if (weightOrBitSet.weight != null) {

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/502fd4bf/lucene/core/src/java/org/apache/lucene/search/SynonymQuery.java
----------------------------------------------------------------------
diff --git a/lucene/core/src/java/org/apache/lucene/search/SynonymQuery.java b/lucene/core/src/java/org/apache/lucene/search/SynonymQuery.java
index ef01308..554633d 100644
--- a/lucene/core/src/java/org/apache/lucene/search/SynonymQuery.java
+++ b/lucene/core/src/java/org/apache/lucene/search/SynonymQuery.java
@@ -31,6 +31,7 @@ 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;
 import org.apache.lucene.search.similarities.Similarity.SimScorer;
@@ -157,6 +158,16 @@ public final class SynonymQuery extends Query {
     }
 
     @Override
+    public Matches matches(LeafReaderContext context, int doc) throws IOException {
+      String field = terms[0].field();
+      Terms terms = context.reader().terms(field);
+      if (terms == null || terms.hasPositions() == false) {
+        return super.matches(context, doc);
+      }
+      return Matches.forField(field, () -> DisjunctionMatchesIterator.fromTerms(context, doc, field, Arrays.asList(SynonymQuery.this.terms)));
+    }
+
+    @Override
     public Explanation explain(LeafReaderContext context, int doc) throws IOException {
       Scorer scorer = scorer(context);
       if (scorer != null) {

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/502fd4bf/lucene/core/src/java/org/apache/lucene/search/TermInSetQuery.java
----------------------------------------------------------------------
diff --git a/lucene/core/src/java/org/apache/lucene/search/TermInSetQuery.java b/lucene/core/src/java/org/apache/lucene/search/TermInSetQuery.java
index e65efe7..1f61f7f 100644
--- a/lucene/core/src/java/org/apache/lucene/search/TermInSetQuery.java
+++ b/lucene/core/src/java/org/apache/lucene/search/TermInSetQuery.java
@@ -220,6 +220,15 @@ public class TermInSetQuery extends Query implements Accountable {
         // order to protect highlighters
       }
 
+      @Override
+      public Matches matches(LeafReaderContext context, int doc) throws IOException {
+        Terms terms = context.reader().terms(field);
+        if (terms == null || terms.hasPositions() == false) {
+          return super.matches(context, doc);
+        }
+        return Matches.forField(field, () -> DisjunctionMatchesIterator.fromTermsEnum(context, doc, field, termData.iterator()));
+      }
+
       /**
        * On the given leaf context, try to either rewrite to a disjunction if
        * there are few matching terms, or build a bitset containing matching docs.

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/502fd4bf/lucene/core/src/java/org/apache/lucene/search/TermMatchesIterator.java
----------------------------------------------------------------------
diff --git a/lucene/core/src/java/org/apache/lucene/search/TermMatchesIterator.java b/lucene/core/src/java/org/apache/lucene/search/TermMatchesIterator.java
new file mode 100644
index 0000000..0516996
--- /dev/null
+++ b/lucene/core/src/java/org/apache/lucene/search/TermMatchesIterator.java
@@ -0,0 +1,77 @@
+/*
+ * 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.PostingsEnum;
+import org.apache.lucene.util.BytesRef;
+
+/**
+ * A {@link MatchesIterator} over a single term's postings list
+ */
+class TermMatchesIterator implements MatchesIterator {
+
+  private int upto;
+  private int pos;
+  private final PostingsEnum pe;
+  private final BytesRef term;
+
+  /**
+   * Create a new {@link TermMatchesIterator} for the given term and postings list
+   */
+  TermMatchesIterator(BytesRef term, PostingsEnum pe) throws IOException {
+    this.pe = pe;
+    this.upto = pe.freq();
+    this.term = term;
+  }
+
+  @Override
+  public boolean next() throws IOException {
+    if (upto-- > 0) {
+      pos = pe.nextPosition();
+      return true;
+    }
+    return false;
+  }
+
+  @Override
+  public int startPosition() {
+    return pos;
+  }
+
+  @Override
+  public int endPosition() {
+    return pos;
+  }
+
+  @Override
+  public int startOffset() throws IOException {
+    return pe.startOffset();
+  }
+
+  @Override
+  public int endOffset() throws IOException {
+    return pe.endOffset();
+  }
+
+  @Override
+  public BytesRef term() {
+    return term;
+  }
+}

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/502fd4bf/lucene/core/src/java/org/apache/lucene/search/TermQuery.java
----------------------------------------------------------------------
diff --git a/lucene/core/src/java/org/apache/lucene/search/TermQuery.java b/lucene/core/src/java/org/apache/lucene/search/TermQuery.java
index ab9e2be..8e635f2 100644
--- a/lucene/core/src/java/org/apache/lucene/search/TermQuery.java
+++ b/lucene/core/src/java/org/apache/lucene/search/TermQuery.java
@@ -80,6 +80,24 @@ public class TermQuery extends Query {
     }
 
     @Override
+    public Matches matches(LeafReaderContext context, int doc) throws IOException {
+      TermsEnum te = getTermsEnum(context);
+      if (te == null) {
+        return null;
+      }
+      if (context.reader().terms(term.field()).hasPositions() == false) {
+        return super.matches(context, doc);
+      }
+      return Matches.forField(term.field(), () -> {
+        PostingsEnum pe = te.postings(null, PostingsEnum.OFFSETS);
+        if (pe.advance(doc) != doc) {
+          return null;
+        }
+        return new TermMatchesIterator(term.bytes(), pe);
+      });
+    }
+
+    @Override
     public String toString() {
       return "weight(" + TermQuery.this + ")";
     }

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/502fd4bf/lucene/core/src/java/org/apache/lucene/search/Weight.java
----------------------------------------------------------------------
diff --git a/lucene/core/src/java/org/apache/lucene/search/Weight.java b/lucene/core/src/java/org/apache/lucene/search/Weight.java
index 3892d4f..13d058c 100644
--- a/lucene/core/src/java/org/apache/lucene/search/Weight.java
+++ b/lucene/core/src/java/org/apache/lucene/search/Weight.java
@@ -70,6 +70,35 @@ public abstract class Weight implements SegmentCacheable {
   public abstract void extractTerms(Set<Term> terms);
 
   /**
+   * Returns {@link Matches} for a specific document, or {@code null} if the document
+   * does not match the parent query
+   *
+   * A query match that contains no position information (for example, a Point or
+   * DocValues query) will return {@link Matches#MATCH_WITH_NO_TERMS}
+   *
+   * @param context the reader's context to create the {@link Matches} for
+   * @param doc     the document's id relative to the given context's reader
+   */
+  public Matches matches(LeafReaderContext context, int doc) throws IOException {
+    Scorer scorer = scorer(context);
+    if (scorer == null) {
+      return null;
+    }
+    final TwoPhaseIterator twoPhase = scorer.twoPhaseIterator();
+    if (twoPhase == null) {
+      if (scorer.iterator().advance(doc) != doc) {
+        return null;
+      }
+    }
+    else {
+      if (twoPhase.approximation().advance(doc) != doc || twoPhase.matches() == false) {
+        return null;
+      }
+    }
+    return Matches.MATCH_WITH_NO_TERMS;
+  }
+
+  /**
    * An explanation of the score computation for the named document.
    * 
    * @param context the readers context to create the {@link Explanation} for.

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/502fd4bf/lucene/core/src/test/org/apache/lucene/search/TestMatchesIterator.java
----------------------------------------------------------------------
diff --git a/lucene/core/src/test/org/apache/lucene/search/TestMatchesIterator.java b/lucene/core/src/test/org/apache/lucene/search/TestMatchesIterator.java
new file mode 100644
index 0000000..f617957
--- /dev/null
+++ b/lucene/core/src/test/org/apache/lucene/search/TestMatchesIterator.java
@@ -0,0 +1,440 @@
+/*
+ * 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.HashSet;
+import java.util.Set;
+
+import org.apache.lucene.analysis.MockAnalyzer;
+import org.apache.lucene.document.Document;
+import org.apache.lucene.document.FieldType;
+import org.apache.lucene.document.NumericDocValuesField;
+import org.apache.lucene.document.TextField;
+import org.apache.lucene.index.IndexOptions;
+import org.apache.lucene.index.IndexReader;
+import org.apache.lucene.index.LeafReaderContext;
+import org.apache.lucene.index.RandomIndexWriter;
+import org.apache.lucene.index.ReaderUtil;
+import org.apache.lucene.index.Term;
+import org.apache.lucene.store.Directory;
+import org.apache.lucene.util.LuceneTestCase;
+
+public class TestMatchesIterator extends LuceneTestCase {
+
+  protected IndexSearcher searcher;
+  protected Directory directory;
+  protected IndexReader reader;
+
+  private static final String FIELD_WITH_OFFSETS = "field_offsets";
+  private static final String FIELD_NO_OFFSETS = "field_no_offsets";
+  private static final String FIELD_DOCS_ONLY = "field_docs_only";
+  private static final String FIELD_FREQS = "field_freqs";
+
+  private static final FieldType OFFSETS = new FieldType(TextField.TYPE_STORED);
+  static {
+    OFFSETS.setIndexOptions(IndexOptions.DOCS_AND_FREQS_AND_POSITIONS_AND_OFFSETS);
+  }
+
+  private static final FieldType DOCS = new FieldType(TextField.TYPE_STORED);
+  static {
+    DOCS.setIndexOptions(IndexOptions.DOCS);
+  }
+
+  private static final FieldType DOCS_AND_FREQS = new FieldType(TextField.TYPE_STORED);
+  static {
+    DOCS_AND_FREQS.setIndexOptions(IndexOptions.DOCS_AND_FREQS);
+  }
+
+  @Override
+  public void tearDown() throws Exception {
+    reader.close();
+    directory.close();
+    super.tearDown();
+  }
+
+  @Override
+  public void setUp() throws Exception {
+    super.setUp();
+    directory = newDirectory();
+    RandomIndexWriter writer = new RandomIndexWriter(random(), directory,
+        newIndexWriterConfig(new MockAnalyzer(random())).setMergePolicy(newLogMergePolicy()));
+    for (int i = 0; i < docFields.length; i++) {
+      Document doc = new Document();
+      doc.add(newField(FIELD_WITH_OFFSETS, docFields[i], OFFSETS));
+      doc.add(newField(FIELD_NO_OFFSETS, docFields[i], TextField.TYPE_STORED));
+      doc.add(newField(FIELD_DOCS_ONLY, docFields[i], DOCS));
+      doc.add(newField(FIELD_FREQS, docFields[i], DOCS_AND_FREQS));
+      doc.add(new NumericDocValuesField("id", i));
+      doc.add(newField("id", Integer.toString(i), TextField.TYPE_STORED));
+      writer.addDocument(doc);
+    }
+    writer.forceMerge(1);
+    reader = writer.getReader();
+    writer.close();
+    searcher = newSearcher(getOnlyLeafReader(reader));
+  }
+
+  protected String[] docFields = {
+      "w1 w2 w3 w4 w5",
+      "w1 w3 w2 w3 zz",
+      "w1 xx w2 yy w4",
+      "w1 w2 w1 w4 w2 w3",
+      "nothing matches this document"
+  };
+
+  void checkMatches(Query q, String field, int[][] expected) throws IOException {
+    Weight w = searcher.createWeight(searcher.rewrite(q), false, 1);
+    for (int i = 0; i < expected.length; i++) {
+      LeafReaderContext ctx = searcher.leafContexts.get(ReaderUtil.subIndex(expected[i][0], searcher.leafContexts));
+      int doc = expected[i][0] - ctx.docBase;
+      Matches matches = w.matches(ctx, doc);
+      if (matches == null) {
+        assertEquals(expected[i].length, 1);
+        continue;
+      }
+      MatchesIterator it = matches.getMatches(field);
+      if (expected[i].length == 1) {
+        assertNull(it);
+        return;
+      }
+      checkFieldMatches(it, expected[i]);
+      checkFieldMatches(matches.getMatches(field), expected[i]);  // test multiple calls
+    }
+  }
+
+  void checkFieldMatches(MatchesIterator it, int[] expected) throws IOException {
+    int pos = 1;
+    while (it.next()) {
+      //System.out.println(expected[i][pos] + "->" + expected[i][pos + 1] + "[" + expected[i][pos + 2] + "->" + expected[i][pos + 3] + "]");
+      assertEquals(expected[pos], it.startPosition());
+      assertEquals(expected[pos + 1], it.endPosition());
+      assertEquals(expected[pos + 2], it.startOffset());
+      assertEquals(expected[pos + 3], it.endOffset());
+      pos += 4;
+    }
+    assertEquals(expected.length, pos);
+  }
+
+  void checkNoPositionsMatches(Query q, String field, boolean[] expected) throws IOException {
+    Weight w = searcher.createWeight(searcher.rewrite(q), false, 1);
+    for (int i = 0; i < expected.length; i++) {
+      LeafReaderContext ctx = searcher.leafContexts.get(ReaderUtil.subIndex(i, searcher.leafContexts));
+      int doc = i - ctx.docBase;
+      Matches matches = w.matches(ctx, doc);
+      if (expected[i]) {
+        MatchesIterator mi = matches.getMatches(field);
+        assertNull(mi);
+      }
+      else {
+        assertNull(matches);
+      }
+    }
+  }
+
+  void checkTerms(Query q, String field, String[][] expected) throws IOException {
+    Weight w = searcher.createWeight(searcher.rewrite(q), false, 1);
+    for (int i = 0; i < expected.length; i++) {
+      LeafReaderContext ctx = searcher.leafContexts.get(ReaderUtil.subIndex(i, searcher.leafContexts));
+      int doc = i - ctx.docBase;
+      Matches matches = w.matches(ctx, doc);
+      if (matches == null) {
+        assertEquals(expected[i].length, 0);
+        continue;
+      }
+      MatchesIterator it = matches.getMatches(field);
+      if (it == null) {
+        assertEquals(expected[i].length, 0);
+        continue;
+      }
+      int pos = 0;
+      while (it.next()) {
+        assertEquals(expected[i][pos], it.term().utf8ToString());
+        pos += 1;
+      }
+      assertEquals(expected[i].length, pos);
+    }
+  }
+
+  public void testTermQuery() throws IOException {
+    Query q = new TermQuery(new Term(FIELD_WITH_OFFSETS, "w1"));
+    checkMatches(q, FIELD_WITH_OFFSETS, new int[][]{
+        { 0, 0, 0, 0, 2 },
+        { 1, 0, 0, 0, 2 },
+        { 2, 0, 0, 0, 2 },
+        { 3, 0, 0, 0, 2, 2, 2, 6, 8 },
+        { 4 }
+    });
+  }
+
+  public void testTermQueryNoStoredOffsets() throws IOException {
+    Query q = new TermQuery(new Term(FIELD_NO_OFFSETS, "w1"));
+    checkMatches(q, FIELD_NO_OFFSETS, new int[][]{
+        { 0, 0, 0, -1, -1 },
+        { 1, 0, 0, -1, -1 },
+        { 2, 0, 0, -1, -1 },
+        { 3, 0, 0, -1, -1, 2, 2, -1, -1 },
+        { 4 }
+    });
+    checkTerms(q, FIELD_NO_OFFSETS, new String[][]{
+        { "w1" },
+        { "w1" },
+        { "w1" },
+        { "w1", "w1" },
+        {}
+    });
+  }
+
+  public void testTermQueryNoPositions() throws IOException {
+    for (String field : new String[]{ FIELD_DOCS_ONLY, FIELD_FREQS }) {
+      Query q = new TermQuery(new Term(field, "w1"));
+      checkNoPositionsMatches(q, field, new boolean[]{ true, true, true, true, false });
+    }
+  }
+
+  public void testDisjunction() throws IOException {
+    Query q = new BooleanQuery.Builder()
+        .add(new TermQuery(new Term(FIELD_WITH_OFFSETS, "w1")), BooleanClause.Occur.SHOULD)
+        .add(new TermQuery(new Term(FIELD_WITH_OFFSETS, "w3")), BooleanClause.Occur.SHOULD)
+        .build();
+    checkMatches(q, FIELD_WITH_OFFSETS, new int[][]{
+        { 0, 0, 0, 0, 2, 2, 2, 6, 8 },
+        { 1, 0, 0, 0, 2, 1, 1, 3, 5, 3, 3, 9, 11 },
+        { 2, 0, 0, 0, 2 },
+        { 3, 0, 0, 0, 2, 2, 2, 6, 8, 5, 5, 15, 17 },
+        { 4 }
+    });
+    checkTerms(q, FIELD_WITH_OFFSETS, new String[][]{
+        { "w1", "w3" },
+        { "w1", "w3", "w3" },
+        { "w1" },
+        { "w1", "w1", "w3" },
+        {}
+    });
+  }
+
+  public void testDisjunctionNoPositions() throws IOException {
+    for (String field : new String[]{ FIELD_DOCS_ONLY, FIELD_FREQS }) {
+      Query q = new BooleanQuery.Builder()
+          .add(new TermQuery(new Term(field, "w1")), BooleanClause.Occur.SHOULD)
+          .add(new TermQuery(new Term(field, "w3")), BooleanClause.Occur.SHOULD)
+          .build();
+      checkNoPositionsMatches(q, field, new boolean[]{ true, true, true, true, false });
+    }
+  }
+
+  public void testReqOpt() throws IOException {
+    Query q = new BooleanQuery.Builder()
+        .add(new TermQuery(new Term(FIELD_WITH_OFFSETS, "w1")), BooleanClause.Occur.SHOULD)
+        .add(new TermQuery(new Term(FIELD_WITH_OFFSETS, "w3")), BooleanClause.Occur.MUST)
+        .build();
+    checkMatches(q, FIELD_WITH_OFFSETS, new int[][]{
+        { 0, 0, 0, 0, 2, 2, 2, 6, 8 },
+        { 1, 0, 0, 0, 2, 1, 1, 3, 5, 3, 3, 9, 11 },
+        { 2 },
+        { 3, 0, 0, 0, 2, 2, 2, 6, 8, 5, 5, 15, 17 },
+        { 4 }
+    });
+  }
+
+  public void testReqOptNoPositions() throws IOException {
+    for (String field : new String[]{ FIELD_DOCS_ONLY, FIELD_FREQS }) {
+      Query q = new BooleanQuery.Builder()
+          .add(new TermQuery(new Term(field, "w1")), BooleanClause.Occur.SHOULD)
+          .add(new TermQuery(new Term(field, "w3")), BooleanClause.Occur.MUST)
+          .build();
+      checkNoPositionsMatches(q, field, new boolean[]{ true, true, false, true, false });
+    }
+  }
+
+  public void testMinShouldMatch() throws IOException {
+    Query q = new BooleanQuery.Builder()
+        .add(new TermQuery(new Term(FIELD_WITH_OFFSETS, "w3")), BooleanClause.Occur.SHOULD)
+        .add(new BooleanQuery.Builder()
+            .add(new TermQuery(new Term(FIELD_WITH_OFFSETS, "w1")), BooleanClause.Occur.SHOULD)
+            .add(new TermQuery(new Term(FIELD_WITH_OFFSETS, "w4")), BooleanClause.Occur.SHOULD)
+            .add(new TermQuery(new Term(FIELD_WITH_OFFSETS, "xx")), BooleanClause.Occur.SHOULD)
+            .setMinimumNumberShouldMatch(2)
+            .build(), BooleanClause.Occur.SHOULD)
+        .build();
+    checkMatches(q, FIELD_WITH_OFFSETS, new int[][]{
+        { 0, 0, 0, 0, 2, 2, 2, 6, 8, 3, 3, 9, 11 },
+        { 1, 1, 1, 3, 5, 3, 3, 9, 11 },
+        { 2, 0, 0, 0, 2, 1, 1, 3, 5, 4, 4, 12, 14 },
+        { 3, 0, 0, 0, 2, 2, 2, 6, 8, 3, 3, 9, 11, 5, 5, 15, 17 },
+        { 4 }
+    });
+    checkTerms(q, FIELD_WITH_OFFSETS, new String[][]{
+        { "w1", "w3", "w4" },
+        { "w3", "w3" },
+        { "w1", "xx", "w4" },
+        { "w1", "w1", "w4", "w3" },
+        {}
+    });
+  }
+
+  public void testMinShouldMatchNoPositions() throws IOException {
+    for (String field : new String[]{ FIELD_FREQS, FIELD_DOCS_ONLY }) {
+      Query q = new BooleanQuery.Builder()
+          .add(new TermQuery(new Term(field, "w3")), BooleanClause.Occur.SHOULD)
+          .add(new BooleanQuery.Builder()
+              .add(new TermQuery(new Term(field, "w1")), BooleanClause.Occur.SHOULD)
+              .add(new TermQuery(new Term(field, "w4")), BooleanClause.Occur.SHOULD)
+              .add(new TermQuery(new Term(field, "xx")), BooleanClause.Occur.SHOULD)
+              .setMinimumNumberShouldMatch(2)
+              .build(), BooleanClause.Occur.SHOULD)
+          .build();
+      checkNoPositionsMatches(q, field, new boolean[]{ true, true, true, true, false });
+    }
+  }
+
+  public void testExclusion() throws IOException {
+    Query q = new BooleanQuery.Builder()
+        .add(new TermQuery(new Term(FIELD_WITH_OFFSETS, "w3")), BooleanClause.Occur.SHOULD)
+        .add(new TermQuery(new Term(FIELD_WITH_OFFSETS, "zz")), BooleanClause.Occur.MUST_NOT)
+        .build();
+    checkMatches(q, FIELD_WITH_OFFSETS, new int[][]{
+        { 0, 2, 2, 6, 8 },
+        { 1 },
+        { 2 },
+        { 3, 5, 5, 15, 17 },
+        { 4 }
+    });
+  }
+
+  public void testExclusionNoPositions() throws IOException {
+    for (String field : new String[]{ FIELD_FREQS, FIELD_DOCS_ONLY }) {
+      Query q = new BooleanQuery.Builder()
+          .add(new TermQuery(new Term(field, "w3")), BooleanClause.Occur.SHOULD)
+          .add(new TermQuery(new Term(field, "zz")), BooleanClause.Occur.MUST_NOT)
+          .build();
+      checkNoPositionsMatches(q, field, new boolean[]{ true, false, false, true, false });
+    }
+  }
+
+  public void testConjunction() throws IOException {
+    Query q = new BooleanQuery.Builder()
+        .add(new TermQuery(new Term(FIELD_WITH_OFFSETS, "w3")), BooleanClause.Occur.MUST)
+        .add(new TermQuery(new Term(FIELD_WITH_OFFSETS, "w4")), BooleanClause.Occur.MUST)
+        .build();
+    checkMatches(q, FIELD_WITH_OFFSETS, new int[][]{
+        { 0, 2, 2, 6, 8, 3, 3, 9, 11 },
+        { 1 },
+        { 2 },
+        { 3, 3, 3, 9, 11, 5, 5, 15, 17 },
+        { 4 }
+    });
+  }
+
+  public void testConjunctionNoPositions() throws IOException {
+    for (String field : new String[]{ FIELD_FREQS, FIELD_DOCS_ONLY }) {
+      Query q = new BooleanQuery.Builder()
+          .add(new TermQuery(new Term(field, "w3")), BooleanClause.Occur.MUST)
+          .add(new TermQuery(new Term(field, "w4")), BooleanClause.Occur.MUST)
+          .build();
+      checkNoPositionsMatches(q, field, new boolean[]{ true, false, false, true, false });
+    }
+  }
+
+  public void testWildcards() throws IOException {
+    Query q = new PrefixQuery(new Term(FIELD_WITH_OFFSETS, "x"));
+    checkMatches(q, FIELD_WITH_OFFSETS, new int[][]{
+        { 0 },
+        { 1 },
+        { 2, 1, 1, 3, 5 },
+        { 3 },
+        { 4 }
+    });
+    checkTerms(q, FIELD_WITH_OFFSETS, new String[][]{
+        {}, {}, { "xx" }, {}
+    });
+
+    Query rq = new RegexpQuery(new Term(FIELD_WITH_OFFSETS, "w[1-2]"));
+    checkMatches(rq, FIELD_WITH_OFFSETS, new int[][]{
+        { 0, 0, 0, 0, 2, 1, 1, 3, 5 },
+        { 1, 0, 0, 0, 2, 2, 2, 6, 8 },
+        { 2, 0, 0, 0, 2, 2, 2, 6, 8 },
+        { 3, 0, 0, 0, 2, 1, 1, 3, 5, 2, 2, 6, 8, 4, 4, 12, 14 },
+        { 4 }
+    });
+
+  }
+
+  public void testNoMatchWildcards() throws IOException {
+    Query nomatch = new PrefixQuery(new Term(FIELD_WITH_OFFSETS, "wibble"));
+    Matches matches = searcher.createWeight(searcher.rewrite(nomatch), false, 1)
+        .matches(searcher.leafContexts.get(0), 0);
+    assertNull(matches);
+  }
+
+  public void testWildcardsNoPositions() throws IOException {
+    for (String field : new String[]{ FIELD_FREQS, FIELD_DOCS_ONLY }) {
+      Query q = new PrefixQuery(new Term(field, "x"));
+      checkNoPositionsMatches(q, field, new boolean[]{ false, false, true, false, false });
+    }
+  }
+
+  public void testSynonymQuery() throws IOException {
+    Query q = new SynonymQuery(new Term(FIELD_WITH_OFFSETS, "w1"), new Term(FIELD_WITH_OFFSETS, "w2"));
+    checkMatches(q, FIELD_WITH_OFFSETS, new int[][]{
+        { 0, 0, 0, 0, 2, 1, 1, 3, 5 },
+        { 1, 0, 0, 0, 2, 2, 2, 6, 8 },
+        { 2, 0, 0, 0, 2, 2, 2, 6, 8 },
+        { 3, 0, 0, 0, 2, 1, 1, 3, 5, 2, 2, 6, 8, 4, 4, 12, 14 },
+        { 4 }
+    });
+  }
+
+  public void testSynonymQueryNoPositions() throws IOException {
+    for (String field : new String[]{ FIELD_FREQS, FIELD_DOCS_ONLY }) {
+      Query q = new SynonymQuery(new Term(field, "w1"), new Term(field, "w2"));
+      checkNoPositionsMatches(q, field, new boolean[]{ true, true, true, true, false });
+    }
+  }
+
+  public void testMultipleFields() throws IOException {
+    Query q = new BooleanQuery.Builder()
+        .add(new TermQuery(new Term("id", "1")), BooleanClause.Occur.SHOULD)
+        .add(new TermQuery(new Term(FIELD_WITH_OFFSETS, "w3")), BooleanClause.Occur.MUST)
+        .build();
+    Weight w = searcher.createWeight(searcher.rewrite(q), false, 1);
+
+    LeafReaderContext ctx = searcher.leafContexts.get(ReaderUtil.subIndex(1, searcher.leafContexts));
+    Matches m = w.matches(ctx, 1 - ctx.docBase);
+    assertNotNull(m);
+    checkFieldMatches(m.getMatches("id"), new int[]{ -1, 0, 0, -1, -1 });
+    checkFieldMatches(m.getMatches(FIELD_WITH_OFFSETS), new int[]{ -1, 1, 1, 3, 5, 3, 3, 9, 11 });
+    assertNull(m.getMatches("bogus"));
+
+    Set<String> fields = new HashSet<>();
+    for (String field : m) {
+      fields.add(field);
+    }
+    assertEquals(2, fields.size());
+    assertTrue(fields.contains(FIELD_WITH_OFFSETS));
+    assertTrue(fields.contains("id"));
+  }
+
+  protected String[] doc1Fields = {
+      "w1 w2 w3 w4 w5",
+      "w1 w3 w2 w3 zz",
+      "w1 xx w2 yy w4",
+      "w1 w2 w1 w4 w2 w3"
+  };
+
+}

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/502fd4bf/lucene/join/src/java/org/apache/lucene/search/join/ToParentBlockJoinQuery.java
----------------------------------------------------------------------
diff --git a/lucene/join/src/java/org/apache/lucene/search/join/ToParentBlockJoinQuery.java b/lucene/join/src/java/org/apache/lucene/search/join/ToParentBlockJoinQuery.java
index c271ef8..1341f58 100644
--- a/lucene/join/src/java/org/apache/lucene/search/join/ToParentBlockJoinQuery.java
+++ b/lucene/join/src/java/org/apache/lucene/search/join/ToParentBlockJoinQuery.java
@@ -28,6 +28,7 @@ import org.apache.lucene.search.DocIdSetIterator;
 import org.apache.lucene.search.Explanation;
 import org.apache.lucene.search.FilterWeight;
 import org.apache.lucene.search.IndexSearcher;
+import org.apache.lucene.search.Matches;
 import org.apache.lucene.search.Query;
 import org.apache.lucene.search.Scorer;
 import org.apache.lucene.search.ScorerSupplier;
@@ -151,6 +152,28 @@ public class ToParentBlockJoinQuery extends Query {
       }
       return Explanation.noMatch("Not a match");
     }
+
+    @Override
+    public Matches matches(LeafReaderContext context, int doc) throws IOException {
+      // The default implementation would delegate to the joinQuery's Weight, which
+      // matches on children.  We need to match on the parent instead
+      Scorer scorer = scorer(context);
+      if (scorer == null) {
+        return null;
+      }
+      final TwoPhaseIterator twoPhase = scorer.twoPhaseIterator();
+      if (twoPhase == null) {
+        if (scorer.iterator().advance(doc) != doc) {
+          return null;
+        }
+      }
+      else {
+        if (twoPhase.approximation().advance(doc) != doc || twoPhase.matches() == false) {
+          return null;
+        }
+      }
+      return Matches.MATCH_WITH_NO_TERMS;
+    }
   }
 
   private static class ParentApproximation extends DocIdSetIterator {

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/502fd4bf/lucene/queries/src/java/org/apache/lucene/queries/function/FunctionScoreQuery.java
----------------------------------------------------------------------
diff --git a/lucene/queries/src/java/org/apache/lucene/queries/function/FunctionScoreQuery.java b/lucene/queries/src/java/org/apache/lucene/queries/function/FunctionScoreQuery.java
index 174fded..e6c01c0 100644
--- a/lucene/queries/src/java/org/apache/lucene/queries/function/FunctionScoreQuery.java
+++ b/lucene/queries/src/java/org/apache/lucene/queries/function/FunctionScoreQuery.java
@@ -29,6 +29,7 @@ import org.apache.lucene.search.DoubleValuesSource;
 import org.apache.lucene.search.Explanation;
 import org.apache.lucene.search.FilterScorer;
 import org.apache.lucene.search.IndexSearcher;
+import org.apache.lucene.search.Matches;
 import org.apache.lucene.search.Query;
 import org.apache.lucene.search.Scorer;
 import org.apache.lucene.search.Weight;
@@ -148,6 +149,11 @@ public final class FunctionScoreQuery extends Query {
     }
 
     @Override
+    public Matches matches(LeafReaderContext context, int doc) throws IOException {
+      return inner.matches(context, doc);
+    }
+
+    @Override
     public Explanation explain(LeafReaderContext context, int doc) throws IOException {
       Scorer scorer = inner.scorer(context);
       if (scorer.iterator().advance(doc) != doc)

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/502fd4bf/lucene/sandbox/src/java/org/apache/lucene/search/CoveringQuery.java
----------------------------------------------------------------------
diff --git a/lucene/sandbox/src/java/org/apache/lucene/search/CoveringQuery.java b/lucene/sandbox/src/java/org/apache/lucene/search/CoveringQuery.java
index 807e1c9..ad76b1c 100644
--- a/lucene/sandbox/src/java/org/apache/lucene/search/CoveringQuery.java
+++ b/lucene/sandbox/src/java/org/apache/lucene/search/CoveringQuery.java
@@ -137,6 +137,28 @@ public final class CoveringQuery extends Query {
     }
 
     @Override
+    public Matches matches(LeafReaderContext context, int doc) throws IOException {
+      LongValues minMatchValues = minimumNumberMatch.getValues(context, null);
+      if (minMatchValues.advanceExact(doc) == false) {
+        return null;
+      }
+      final long minimumNumberMatch = Math.max(1, minMatchValues.longValue());
+      long matchCount = 0;
+      List<Matches> subMatches = new ArrayList<>();
+      for (Weight weight : weights) {
+        Matches matches = weight.matches(context, doc);
+        if (matches != null) {
+          matchCount++;
+          subMatches.add(matches);
+        }
+      }
+      if (matchCount < minimumNumberMatch) {
+        return null;
+      }
+      return Matches.fromSubMatches(subMatches);
+    }
+
+    @Override
     public Explanation explain(LeafReaderContext context, int doc) throws IOException {
       LongValues minMatchValues = minimumNumberMatch.getValues(context, null);
       if (minMatchValues.advanceExact(doc) == false) {

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/502fd4bf/lucene/test-framework/src/java/org/apache/lucene/search/AssertingMatches.java
----------------------------------------------------------------------
diff --git a/lucene/test-framework/src/java/org/apache/lucene/search/AssertingMatches.java b/lucene/test-framework/src/java/org/apache/lucene/search/AssertingMatches.java
new file mode 100644
index 0000000..c5c6e98
--- /dev/null
+++ b/lucene/test-framework/src/java/org/apache/lucene/search/AssertingMatches.java
@@ -0,0 +1,43 @@
+/*
+ * 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.Iterator;
+
+class AssertingMatches implements Matches {
+
+  private final Matches in;
+
+  AssertingMatches(Matches matches) {
+    this.in = matches;
+  }
+
+  @Override
+  public MatchesIterator getMatches(String field) throws IOException {
+    MatchesIterator mi = in.getMatches(field);
+    if (mi == null)
+      return null;
+    return new AssertingMatchesIterator(mi);
+  }
+
+  @Override
+  public Iterator<String> iterator() {
+    return in.iterator();
+  }
+}

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/502fd4bf/lucene/test-framework/src/java/org/apache/lucene/search/AssertingMatchesIterator.java
----------------------------------------------------------------------
diff --git a/lucene/test-framework/src/java/org/apache/lucene/search/AssertingMatchesIterator.java b/lucene/test-framework/src/java/org/apache/lucene/search/AssertingMatchesIterator.java
new file mode 100644
index 0000000..52fb184
--- /dev/null
+++ b/lucene/test-framework/src/java/org/apache/lucene/search/AssertingMatchesIterator.java
@@ -0,0 +1,77 @@
+/*
+ * 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.util.BytesRef;
+
+class AssertingMatchesIterator implements MatchesIterator {
+
+  private final MatchesIterator in;
+  private State state = State.UNPOSITIONED;
+
+  private enum State { UNPOSITIONED, ITERATING, EXHAUSTED }
+
+  AssertingMatchesIterator(MatchesIterator in) {
+    this.in = in;
+  }
+
+  @Override
+  public boolean next() throws IOException {
+    assert state != State.EXHAUSTED : state;
+    boolean more = in.next();
+    if (more == false) {
+      state = State.EXHAUSTED;
+    }
+    else {
+      state = State.ITERATING;
+    }
+    return more;
+  }
+
+  @Override
+  public int startPosition() {
+    assert state == State.ITERATING : state;
+    return in.startPosition();
+  }
+
+  @Override
+  public int endPosition() {
+    assert state == State.ITERATING : state;
+    return in.endPosition();
+  }
+
+  @Override
+  public int startOffset() throws IOException {
+    assert state == State.ITERATING : state;
+    return in.startOffset();
+  }
+
+  @Override
+  public int endOffset() throws IOException {
+    assert state == State.ITERATING : state;
+    return in.endOffset();
+  }
+
+  @Override
+  public BytesRef term() {
+    assert state == State.ITERATING : state;
+    return in.term();
+  }
+}

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/502fd4bf/lucene/test-framework/src/java/org/apache/lucene/search/AssertingWeight.java
----------------------------------------------------------------------
diff --git a/lucene/test-framework/src/java/org/apache/lucene/search/AssertingWeight.java b/lucene/test-framework/src/java/org/apache/lucene/search/AssertingWeight.java
index b98e6a1..c4ef409 100644
--- a/lucene/test-framework/src/java/org/apache/lucene/search/AssertingWeight.java
+++ b/lucene/test-framework/src/java/org/apache/lucene/search/AssertingWeight.java
@@ -32,6 +32,14 @@ class AssertingWeight extends FilterWeight {
   }
 
   @Override
+  public Matches matches(LeafReaderContext context, int doc) throws IOException {
+    Matches matches = in.matches(context, doc);
+    if (matches == null)
+      return null;
+    return new AssertingMatches(matches);
+  }
+
+  @Override
   public Scorer scorer(LeafReaderContext context) throws IOException {
     if (random.nextBoolean()) {
       final Scorer inScorer = in.scorer(context);