You are viewing a plain text version of this content. The canonical link for it is here.
Posted to dev@lucene.apache.org by romseygeek <gi...@git.apache.org> on 2018/03/28 14:46:03 UTC

[GitHub] lucene-solr pull request #345: LUCENE-8229: Add Weight.matches() method

GitHub user romseygeek opened a pull request:

    https://github.com/apache/lucene-solr/pull/345

    LUCENE-8229: Add Weight.matches() method

    This PR adds a method to Weight that allows iterating over the matching positions (and offsets if available) for a particular document and field.

You can merge this pull request into a Git repository by running:

    $ git pull https://github.com/romseygeek/lucene-solr weight-matches

Alternatively you can review and apply these changes as the patch at:

    https://github.com/apache/lucene-solr/pull/345.patch

To close this pull request, make a commit to your master/trunk branch
with (at least) the following in the commit message:

    This closes #345
    
----

----


---

---------------------------------------------------------------------
To unsubscribe, e-mail: dev-unsubscribe@lucene.apache.org
For additional commands, e-mail: dev-help@lucene.apache.org


[GitHub] lucene-solr pull request #345: LUCENE-8229: Add Weight.matches() method

Posted by jpountz <gi...@git.apache.org>.
Github user jpountz commented on a diff in the pull request:

    https://github.com/apache/lucene-solr/pull/345#discussion_r179439321
  
    --- Diff: lucene/core/src/java/org/apache/lucene/search/Matches.java ---
    @@ -0,0 +1,161 @@
    +/*
    + * 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.HashMap;
    +import java.util.HashSet;
    +import java.util.List;
    +import java.util.Map;
    +import java.util.Set;
    +
    +import org.apache.lucene.index.LeafReaderContext;
    +import org.apache.lucene.util.BytesRef;
    +
    +/**
    + * Reports the positions and optionally offsets of all matching terms in a query
    + * for a single document
    + *
    + * To find all fields that have matches, call {@link #getMatchFields()}
    + *
    + * To obtain a {@link MatchesIterator} for a particular field, call {@link #getMatches(String)}
    + */
    +public class Matches {
    +
    +  private final Map<String, MatchesIterator> matches;
    +
    +  /**
    +   * Create a simple {@link Matches} for a single field
    +   */
    +  public static Matches fromField(String field, MatchesIterator it) {
    +    if (it == null) {
    +      return null;
    +    }
    +    return new Matches(field, it);
    +  }
    +
    +  /**
    +   * Create an empty {@link Matches} for a Weight
    +   *
    +   * If the Weight's parent query does not match this document, returns {@code null},
    +   * otherwise returns a {@link Matches} document with an empty iterator on the given
    +   * fields
    +   */
    +  public static Matches emptyMatches(LeafReaderContext context, int doc, Weight weight, String... fields) throws IOException {
    +    Scorer scorer = weight.scorer(context);
    +    if (scorer == null || scorer.iterator().advance(doc) != doc) {
    +      return null;
    +    }
    +    List<Matches> matches = new ArrayList<>();
    +    for (String field : fields) {
    +      matches.add(Matches.fromField(field, EMPTY));
    +    }
    +    return Matches.fromSubMatches(matches);
    +  }
    +
    +  /**
    +   * Amalgamate a collection of {@link Matches} into a single object
    +   */
    +  public static Matches fromSubMatches(List<Matches> subMatches) throws IOException {
    +    if (subMatches == null || subMatches.size() == 0) {
    +      return null;
    +    }
    +    if (subMatches.size() == 1) {
    +      return subMatches.get(0);
    +    }
    +    Map<String, MatchesIterator> matches = new HashMap<>();
    +    Set<String> allFields = new HashSet<>();
    +    for (Matches m : subMatches) {
    +      allFields.addAll(m.getMatchFields());
    +    }
    +    for (String field : allFields) {
    +      List<MatchesIterator> mis = new ArrayList<>();
    +      for (Matches m : subMatches) {
    +        MatchesIterator mi = m.getMatches(field);
    +        if (mi != null) {
    +          mis.add(mi);
    +        }
    +      }
    +      matches.put(field, DisjunctionMatchesIterator.fromSubIterators(mis));
    +    }
    +    return new Matches(matches);
    +  }
    +
    +  /**
    +   * Create a {@link Matches} from a map of fields to iterators
    +   */
    +  protected Matches(Map<String, MatchesIterator> matches) {
    +    this.matches = matches;
    +  }
    +
    +  private Matches(String field, MatchesIterator iterator) {
    +    this.matches = new HashMap<>();
    +    this.matches.put(field, iterator);
    --- End diff --
    
    Collections.singletonMap?


---

---------------------------------------------------------------------
To unsubscribe, e-mail: dev-unsubscribe@lucene.apache.org
For additional commands, e-mail: dev-help@lucene.apache.org


[GitHub] lucene-solr issue #345: LUCENE-8229: Add Weight.matches() method

Posted by romseygeek <gi...@git.apache.org>.
Github user romseygeek commented on the issue:

    https://github.com/apache/lucene-solr/pull/345
  
    I added an assertion to QueryUtils that calls Weight.matches() for each collected document and ensures that the response isn't null (this actually caught a bug in MTQ's handling).  I've also rejigged things so that you can call Matches.getMatches(field) multiple times, although the call isn't thread-safe.


---

---------------------------------------------------------------------
To unsubscribe, e-mail: dev-unsubscribe@lucene.apache.org
For additional commands, e-mail: dev-help@lucene.apache.org


[GitHub] lucene-solr pull request #345: LUCENE-8229: Add Weight.matches() method

Posted by jpountz <gi...@git.apache.org>.
Github user jpountz commented on a diff in the pull request:

    https://github.com/apache/lucene-solr/pull/345#discussion_r179741828
  
    --- Diff: lucene/core/src/java/org/apache/lucene/search/DisjunctionMatchesIterator.java ---
    @@ -0,0 +1,160 @@
    +/*
    + * 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.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.
    + */
    +public final class DisjunctionMatchesIterator implements MatchesIterator {
    --- End diff --
    
    can we reduce visibility?


---

---------------------------------------------------------------------
To unsubscribe, e-mail: dev-unsubscribe@lucene.apache.org
For additional commands, e-mail: dev-help@lucene.apache.org


[GitHub] lucene-solr pull request #345: LUCENE-8229: Add Weight.matches() method

Posted by jpountz <gi...@git.apache.org>.
Github user jpountz commented on a diff in the pull request:

    https://github.com/apache/lucene-solr/pull/345#discussion_r179442642
  
    --- Diff: lucene/core/src/java/org/apache/lucene/search/DisjunctionMatchesIterator.java ---
    @@ -0,0 +1,158 @@
    +/*
    + * 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.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.
    + */
    +public 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
    +   */
    +  public static DisjunctionMatchesIterator fromTerms(LeafReaderContext context, int doc, String field, List<Term> terms) throws IOException {
    +    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
    +   */
    +  public static DisjunctionMatchesIterator fromTermsEnum(LeafReaderContext context, int doc, String field, BytesRefIterator terms) throws IOException {
    +    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;
    +    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());
    +      }
    +    };
    +    for (MatchesIterator mi : matches) {
    +      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;
    +    }
    --- End diff --
    
    do we need to remove dups?


---

---------------------------------------------------------------------
To unsubscribe, e-mail: dev-unsubscribe@lucene.apache.org
For additional commands, e-mail: dev-help@lucene.apache.org


[GitHub] lucene-solr issue #345: LUCENE-8229: Add Weight.matches() method

Posted by romseygeek <gi...@git.apache.org>.
Github user romseygeek commented on the issue:

    https://github.com/apache/lucene-solr/pull/345
  
    I extended MatchesAsserter to keep track of the last matching document, and check that if the document one below the current collection is not a hit, then its Matches is null.


---

---------------------------------------------------------------------
To unsubscribe, e-mail: dev-unsubscribe@lucene.apache.org
For additional commands, e-mail: dev-help@lucene.apache.org


[GitHub] lucene-solr pull request #345: LUCENE-8229: Add Weight.matches() method

Posted by jpountz <gi...@git.apache.org>.
Github user jpountz commented on a diff in the pull request:

    https://github.com/apache/lucene-solr/pull/345#discussion_r179443149
  
    --- Diff: solr/core/src/java/org/apache/solr/update/DeleteByQueryWrapper.java ---
    @@ -77,6 +78,11 @@ public void extractTerms(Set<Term> terms) {
             throw new UnsupportedOperationException();
           }
     
    +      @Override
    +      public Matches matches(LeafReaderContext context, int doc) throws IOException {
    +        return Matches.emptyMatches(context, doc, this, "*");  // TODO is there a way of reporting matches that makes sense here?
    --- End diff --
    
    let's just throw an UOE?


---

---------------------------------------------------------------------
To unsubscribe, e-mail: dev-unsubscribe@lucene.apache.org
For additional commands, e-mail: dev-help@lucene.apache.org


[GitHub] lucene-solr issue #345: LUCENE-8229: Add Weight.matches() method

Posted by jpountz <gi...@git.apache.org>.
Github user jpountz commented on the issue:

    https://github.com/apache/lucene-solr/pull/345
  
    > instead of directly returning the MatchesIterator
    
    I asked for this change so that you always get a fresh new iterator, even if you already iterated positions on the same field. I agree that making the call fully lazy would be ideal but I'm not too worried about it either given that this API is expected to be slow, eg. you can't use it to run queries.


---

---------------------------------------------------------------------
To unsubscribe, e-mail: dev-unsubscribe@lucene.apache.org
For additional commands, e-mail: dev-help@lucene.apache.org


[GitHub] lucene-solr pull request #345: LUCENE-8229: Add Weight.matches() method

Posted by jpountz <gi...@git.apache.org>.
Github user jpountz commented on a diff in the pull request:

    https://github.com/apache/lucene-solr/pull/345#discussion_r179742203
  
    --- Diff: lucene/core/src/java/org/apache/lucene/search/DisjunctionMatchesIterator.java ---
    @@ -0,0 +1,160 @@
    +/*
    + * 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.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.
    + */
    +public 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
    +   */
    +  public static DisjunctionMatchesIterator fromTerms(LeafReaderContext context, int doc, String field, List<Term> terms) throws IOException {
    +    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
    +   */
    +  public static DisjunctionMatchesIterator fromTermsEnum(LeafReaderContext context, int doc, String field, BytesRefIterator terms) throws IOException {
    --- End diff --
    
    maybe return MatchesIterator? the impl shouldn't matter


---

---------------------------------------------------------------------
To unsubscribe, e-mail: dev-unsubscribe@lucene.apache.org
For additional commands, e-mail: dev-help@lucene.apache.org


[GitHub] lucene-solr issue #345: LUCENE-8229: Add Weight.matches() method

Posted by jpountz <gi...@git.apache.org>.
Github user jpountz commented on the issue:

    https://github.com/apache/lucene-solr/pull/345
  
    Can we also check that it returns null on non-matching documents?


---

---------------------------------------------------------------------
To unsubscribe, e-mail: dev-unsubscribe@lucene.apache.org
For additional commands, e-mail: dev-help@lucene.apache.org


[GitHub] lucene-solr issue #345: LUCENE-8229: Add Weight.matches() method

Posted by romseygeek <gi...@git.apache.org>.
Github user romseygeek commented on the issue:

    https://github.com/apache/lucene-solr/pull/345
  
    > In term of testing, I think we should enforce some consistency with extractTerms
    
    The two aren't always consistent - for example, TermInSetQuery and MultiTermQueryConstantScoreWrapper don't extract any Terms, to stop highlighter explosion, but they do report Matches for specific terms, because they can check them against single documents.


---

---------------------------------------------------------------------
To unsubscribe, e-mail: dev-unsubscribe@lucene.apache.org
For additional commands, e-mail: dev-help@lucene.apache.org


[GitHub] lucene-solr pull request #345: LUCENE-8229: Add Weight.matches() method

Posted by jpountz <gi...@git.apache.org>.
Github user jpountz commented on a diff in the pull request:

    https://github.com/apache/lucene-solr/pull/345#discussion_r179443767
  
    --- Diff: lucene/core/src/java/org/apache/lucene/search/MatchAllDocsQuery.java ---
    @@ -35,6 +35,12 @@ public Weight createWeight(IndexSearcher searcher, ScoreMode scoreMode, float bo
           public String toString() {
             return "weight(" + MatchAllDocsQuery.this + ")";
           }
    +
    +      @Override
    +      public Matches matches(LeafReaderContext context, int doc) throws IOException {
    +        return Matches.emptyMatches(context, doc, this, "*");
    --- End diff --
    
    this looks hacky


---

---------------------------------------------------------------------
To unsubscribe, e-mail: dev-unsubscribe@lucene.apache.org
For additional commands, e-mail: dev-help@lucene.apache.org


[GitHub] lucene-solr pull request #345: LUCENE-8229: Add Weight.matches() method

Posted by jpountz <gi...@git.apache.org>.
Github user jpountz commented on a diff in the pull request:

    https://github.com/apache/lucene-solr/pull/345#discussion_r179744351
  
    --- Diff: lucene/core/src/java/org/apache/lucene/search/Weight.java ---
    @@ -69,6 +69,21 @@ protected Weight(Query query) {
        */
       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
    --- End diff --
    
    maybe mention that a match without positions will be reported as an empty instance?


---

---------------------------------------------------------------------
To unsubscribe, e-mail: dev-unsubscribe@lucene.apache.org
For additional commands, e-mail: dev-help@lucene.apache.org


[GitHub] lucene-solr pull request #345: LUCENE-8229: Add Weight.matches() method

Posted by jpountz <gi...@git.apache.org>.
Github user jpountz commented on a diff in the pull request:

    https://github.com/apache/lucene-solr/pull/345#discussion_r180496635
  
    --- Diff: lucene/join/src/java/org/apache/lucene/search/join/ToParentBlockJoinQuery.java ---
    @@ -151,6 +152,17 @@ public Explanation explain(LeafReaderContext context, int doc) throws IOExceptio
           }
           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 || scorer.iterator().advance(doc) != doc) {
    +        return null;
    +      }
    --- End diff --
    
    let's check the two-phase iterator here as well if applicable?


---

---------------------------------------------------------------------
To unsubscribe, e-mail: dev-unsubscribe@lucene.apache.org
For additional commands, e-mail: dev-help@lucene.apache.org


[GitHub] lucene-solr pull request #345: LUCENE-8229: Add Weight.matches() method

Posted by jpountz <gi...@git.apache.org>.
Github user jpountz commented on a diff in the pull request:

    https://github.com/apache/lucene-solr/pull/345#discussion_r180492694
  
    --- Diff: lucene/core/src/java/org/apache/lucene/search/DisjunctionMatchesIterator.java ---
    @@ -0,0 +1,168 @@
    +/*
    + * 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 {
    +    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 {
    +    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());
    --- End diff --
    
    should tie-break by term to return positions in deterministic order?


---

---------------------------------------------------------------------
To unsubscribe, e-mail: dev-unsubscribe@lucene.apache.org
For additional commands, e-mail: dev-help@lucene.apache.org


[GitHub] lucene-solr pull request #345: LUCENE-8229: Add Weight.matches() method

Posted by jpountz <gi...@git.apache.org>.
Github user jpountz commented on a diff in the pull request:

    https://github.com/apache/lucene-solr/pull/345#discussion_r179742139
  
    --- Diff: lucene/core/src/java/org/apache/lucene/search/DisjunctionMatchesIterator.java ---
    @@ -0,0 +1,160 @@
    +/*
    + * 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.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.
    + */
    +public 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
    +   */
    +  public static DisjunctionMatchesIterator fromTerms(LeafReaderContext context, int doc, String field, List<Term> terms) throws IOException {
    +    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
    +   */
    +  public static DisjunctionMatchesIterator fromTermsEnum(LeafReaderContext context, int doc, String field, BytesRefIterator terms) throws IOException {
    +    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;
    +    return new DisjunctionMatchesIterator(mis);
    --- End diff --
    
    should we specialize the size==1 case as well?


---

---------------------------------------------------------------------
To unsubscribe, e-mail: dev-unsubscribe@lucene.apache.org
For additional commands, e-mail: dev-help@lucene.apache.org


[GitHub] lucene-solr pull request #345: LUCENE-8229: Add Weight.matches() method

Posted by jpountz <gi...@git.apache.org>.
Github user jpountz commented on a diff in the pull request:

    https://github.com/apache/lucene-solr/pull/345#discussion_r179439888
  
    --- Diff: lucene/core/src/java/org/apache/lucene/search/Matches.java ---
    @@ -0,0 +1,161 @@
    +/*
    + * 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.HashMap;
    +import java.util.HashSet;
    +import java.util.List;
    +import java.util.Map;
    +import java.util.Set;
    +
    +import org.apache.lucene.index.LeafReaderContext;
    +import org.apache.lucene.util.BytesRef;
    +
    +/**
    + * Reports the positions and optionally offsets of all matching terms in a query
    + * for a single document
    + *
    + * To find all fields that have matches, call {@link #getMatchFields()}
    + *
    + * To obtain a {@link MatchesIterator} for a particular field, call {@link #getMatches(String)}
    + */
    +public class Matches {
    +
    +  private final Map<String, MatchesIterator> matches;
    +
    +  /**
    +   * Create a simple {@link Matches} for a single field
    +   */
    +  public static Matches fromField(String field, MatchesIterator it) {
    +    if (it == null) {
    +      return null;
    +    }
    +    return new Matches(field, it);
    +  }
    +
    +  /**
    +   * Create an empty {@link Matches} for a Weight
    +   *
    +   * If the Weight's parent query does not match this document, returns {@code null},
    +   * otherwise returns a {@link Matches} document with an empty iterator on the given
    +   * fields
    +   */
    +  public static Matches emptyMatches(LeafReaderContext context, int doc, Weight weight, String... fields) throws IOException {
    +    Scorer scorer = weight.scorer(context);
    +    if (scorer == null || scorer.iterator().advance(doc) != doc) {
    +      return null;
    +    }
    +    List<Matches> matches = new ArrayList<>();
    +    for (String field : fields) {
    +      matches.add(Matches.fromField(field, EMPTY));
    +    }
    +    return Matches.fromSubMatches(matches);
    +  }
    +
    +  /**
    +   * Amalgamate a collection of {@link Matches} into a single object
    +   */
    +  public static Matches fromSubMatches(List<Matches> subMatches) throws IOException {
    +    if (subMatches == null || subMatches.size() == 0) {
    +      return null;
    +    }
    +    if (subMatches.size() == 1) {
    +      return subMatches.get(0);
    +    }
    +    Map<String, MatchesIterator> matches = new HashMap<>();
    +    Set<String> allFields = new HashSet<>();
    +    for (Matches m : subMatches) {
    +      allFields.addAll(m.getMatchFields());
    +    }
    +    for (String field : allFields) {
    +      List<MatchesIterator> mis = new ArrayList<>();
    +      for (Matches m : subMatches) {
    +        MatchesIterator mi = m.getMatches(field);
    +        if (mi != null) {
    +          mis.add(mi);
    +        }
    +      }
    +      matches.put(field, DisjunctionMatchesIterator.fromSubIterators(mis));
    +    }
    +    return new Matches(matches);
    +  }
    +
    +  /**
    +   * Create a {@link Matches} from a map of fields to iterators
    +   */
    +  protected Matches(Map<String, MatchesIterator> matches) {
    +    this.matches = matches;
    +  }
    +
    +  private Matches(String field, MatchesIterator iterator) {
    +    this.matches = new HashMap<>();
    +    this.matches.put(field, iterator);
    +  }
    +
    +  /**
    +   * Returns a {@link MatchesIterator} over the matches for a single field,
    +   * or {@code null} if there are no matches in that field
    +   */
    +  public MatchesIterator getMatches(String field) {
    +    return matches.get(field);
    +  }
    +
    +  /**
    +   * Returns the fields with matches for this document
    +   */
    +  public Set<String> getMatchFields() {
    +    return matches.keySet();
    +  }
    +
    +  private static final BytesRef EMPTY_BYTES = new BytesRef();
    --- End diff --
    
    let's make it an instance member of EMPTY? It doesn't look used elsewhere.


---

---------------------------------------------------------------------
To unsubscribe, e-mail: dev-unsubscribe@lucene.apache.org
For additional commands, e-mail: dev-help@lucene.apache.org


[GitHub] lucene-solr pull request #345: LUCENE-8229: Add Weight.matches() method

Posted by jpountz <gi...@git.apache.org>.
Github user jpountz commented on a diff in the pull request:

    https://github.com/apache/lucene-solr/pull/345#discussion_r179737921
  
    --- Diff: lucene/core/src/java/org/apache/lucene/search/DisjunctionMatchesIterator.java ---
    @@ -0,0 +1,160 @@
    +/*
    + * 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.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.
    + */
    +public 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
    +   */
    +  public static DisjunctionMatchesIterator fromTerms(LeafReaderContext context, int doc, String field, List<Term> terms) throws IOException {
    +    return fromTermsEnum(context, doc, field, asBytesRefIterator(terms));
    --- End diff --
    
    let's validate that all terms have `field` as a field, or directly take a list of BytesRefs?


---

---------------------------------------------------------------------
To unsubscribe, e-mail: dev-unsubscribe@lucene.apache.org
For additional commands, e-mail: dev-help@lucene.apache.org


[GitHub] lucene-solr issue #345: LUCENE-8229: Add Weight.matches() method

Posted by romseygeek <gi...@git.apache.org>.
Github user romseygeek commented on the issue:

    https://github.com/apache/lucene-solr/pull/345
  
    > For instance I don't think it should try to report matches whenever there are no terms involved
    
    The reason I did this was to make Boolean matching accurate.  Say for example you have a query that looks like `field:term OR (field:term2 AND date:[2014-2018])`.  If a document contains both `term` and `term2`, but its date field falls outside that range, then we should only report a match on `term`, which means the BooleanQuery matches() method needs to know about matches from the date field.  I think it's also useful when reporting 'which fields did this query hit on'.  The MatchAllDocsQuery is a corner case, I admit, but it's the only one I think?


---

---------------------------------------------------------------------
To unsubscribe, e-mail: dev-unsubscribe@lucene.apache.org
For additional commands, e-mail: dev-help@lucene.apache.org


[GitHub] lucene-solr issue #345: LUCENE-8229: Add Weight.matches() method

Posted by dsmiley <gi...@git.apache.org>.
Github user dsmiley commented on the issue:

    https://github.com/apache/lucene-solr/pull/345
  
    Curious; what led to the addition of the MatchesIteratorSupplier instead of directly returning the MatchesIterator?  Whatever your response may be, does the eager call to `.get` in Matches.forField defeat the purpose?


---

---------------------------------------------------------------------
To unsubscribe, e-mail: dev-unsubscribe@lucene.apache.org
For additional commands, e-mail: dev-help@lucene.apache.org


[GitHub] lucene-solr pull request #345: LUCENE-8229: Add Weight.matches() method

Posted by jpountz <gi...@git.apache.org>.
Github user jpountz commented on a diff in the pull request:

    https://github.com/apache/lucene-solr/pull/345#discussion_r179439647
  
    --- Diff: lucene/core/src/java/org/apache/lucene/search/Matches.java ---
    @@ -0,0 +1,161 @@
    +/*
    + * 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.HashMap;
    +import java.util.HashSet;
    +import java.util.List;
    +import java.util.Map;
    +import java.util.Set;
    +
    +import org.apache.lucene.index.LeafReaderContext;
    +import org.apache.lucene.util.BytesRef;
    +
    +/**
    + * Reports the positions and optionally offsets of all matching terms in a query
    + * for a single document
    + *
    + * To find all fields that have matches, call {@link #getMatchFields()}
    + *
    + * To obtain a {@link MatchesIterator} for a particular field, call {@link #getMatches(String)}
    + */
    +public class Matches {
    +
    +  private final Map<String, MatchesIterator> matches;
    +
    +  /**
    +   * Create a simple {@link Matches} for a single field
    +   */
    +  public static Matches fromField(String field, MatchesIterator it) {
    +    if (it == null) {
    +      return null;
    +    }
    +    return new Matches(field, it);
    +  }
    +
    +  /**
    +   * Create an empty {@link Matches} for a Weight
    +   *
    +   * If the Weight's parent query does not match this document, returns {@code null},
    +   * otherwise returns a {@link Matches} document with an empty iterator on the given
    +   * fields
    +   */
    +  public static Matches emptyMatches(LeafReaderContext context, int doc, Weight weight, String... fields) throws IOException {
    +    Scorer scorer = weight.scorer(context);
    +    if (scorer == null || scorer.iterator().advance(doc) != doc) {
    +      return null;
    +    }
    +    List<Matches> matches = new ArrayList<>();
    +    for (String field : fields) {
    +      matches.add(Matches.fromField(field, EMPTY));
    +    }
    +    return Matches.fromSubMatches(matches);
    +  }
    +
    +  /**
    +   * Amalgamate a collection of {@link Matches} into a single object
    +   */
    +  public static Matches fromSubMatches(List<Matches> subMatches) throws IOException {
    +    if (subMatches == null || subMatches.size() == 0) {
    +      return null;
    +    }
    +    if (subMatches.size() == 1) {
    +      return subMatches.get(0);
    +    }
    +    Map<String, MatchesIterator> matches = new HashMap<>();
    +    Set<String> allFields = new HashSet<>();
    +    for (Matches m : subMatches) {
    +      allFields.addAll(m.getMatchFields());
    +    }
    +    for (String field : allFields) {
    +      List<MatchesIterator> mis = new ArrayList<>();
    +      for (Matches m : subMatches) {
    +        MatchesIterator mi = m.getMatches(field);
    +        if (mi != null) {
    +          mis.add(mi);
    +        }
    +      }
    +      matches.put(field, DisjunctionMatchesIterator.fromSubIterators(mis));
    +    }
    +    return new Matches(matches);
    +  }
    +
    +  /**
    +   * Create a {@link Matches} from a map of fields to iterators
    +   */
    +  protected Matches(Map<String, MatchesIterator> matches) {
    +    this.matches = matches;
    +  }
    +
    +  private Matches(String field, MatchesIterator iterator) {
    +    this.matches = new HashMap<>();
    +    this.matches.put(field, iterator);
    +  }
    +
    +  /**
    +   * Returns a {@link MatchesIterator} over the matches for a single field,
    +   * or {@code null} if there are no matches in that field
    +   */
    +  public MatchesIterator getMatches(String field) {
    +    return matches.get(field);
    +  }
    +
    +  /**
    +   * Returns the fields with matches for this document
    +   */
    +  public Set<String> getMatchFields() {
    +    return matches.keySet();
    +  }
    --- End diff --
    
    should the API be more like oal.index.Fields?


---

---------------------------------------------------------------------
To unsubscribe, e-mail: dev-unsubscribe@lucene.apache.org
For additional commands, e-mail: dev-help@lucene.apache.org


[GitHub] lucene-solr issue #345: LUCENE-8229: Add Weight.matches() method

Posted by romseygeek <gi...@git.apache.org>.
Github user romseygeek commented on the issue:

    https://github.com/apache/lucene-solr/pull/345
  
    bq. I agree that making the call fully lazy would be ideal
    
    This can't be done, unfortunately, as we need to know up-front if there's a match or not.  Hence the slightly odd indirection/eager-call two-step.  I'll add a comment in the code to make it clearer.


---

---------------------------------------------------------------------
To unsubscribe, e-mail: dev-unsubscribe@lucene.apache.org
For additional commands, e-mail: dev-help@lucene.apache.org


[GitHub] lucene-solr pull request #345: LUCENE-8229: Add Weight.matches() method

Posted by romseygeek <gi...@git.apache.org>.
Github user romseygeek commented on a diff in the pull request:

    https://github.com/apache/lucene-solr/pull/345#discussion_r179679658
  
    --- Diff: lucene/core/src/java/org/apache/lucene/search/DisjunctionMatchesIterator.java ---
    @@ -0,0 +1,158 @@
    +/*
    + * 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.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.
    + */
    +public 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
    +   */
    +  public static DisjunctionMatchesIterator fromTerms(LeafReaderContext context, int doc, String field, List<Term> terms) throws IOException {
    +    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
    +   */
    +  public static DisjunctionMatchesIterator fromTermsEnum(LeafReaderContext context, int doc, String field, BytesRefIterator terms) throws IOException {
    +    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;
    +    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);
    --- End diff --
    
    I've moved the notion of a match with no terms up a level to the Matches object, so the check for this is now done in Matches.fromSubMatches().


---

---------------------------------------------------------------------
To unsubscribe, e-mail: dev-unsubscribe@lucene.apache.org
For additional commands, e-mail: dev-help@lucene.apache.org


[GitHub] lucene-solr pull request #345: LUCENE-8229: Add Weight.matches() method

Posted by jpountz <gi...@git.apache.org>.
Github user jpountz commented on a diff in the pull request:

    https://github.com/apache/lucene-solr/pull/345#discussion_r179442210
  
    --- Diff: lucene/core/src/java/org/apache/lucene/search/DisjunctionMatchesIterator.java ---
    @@ -0,0 +1,158 @@
    +/*
    + * 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.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.
    + */
    +public 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
    +   */
    +  public static DisjunctionMatchesIterator fromTerms(LeafReaderContext context, int doc, String field, List<Term> terms) throws IOException {
    +    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
    +   */
    +  public static DisjunctionMatchesIterator fromTermsEnum(LeafReaderContext context, int doc, String field, BytesRefIterator terms) throws IOException {
    +    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;
    +    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);
    --- End diff --
    
    we might also want to remove the `EMPTY` instances?


---

---------------------------------------------------------------------
To unsubscribe, e-mail: dev-unsubscribe@lucene.apache.org
For additional commands, e-mail: dev-help@lucene.apache.org


[GitHub] lucene-solr pull request #345: LUCENE-8229: Add Weight.matches() method

Posted by jpountz <gi...@git.apache.org>.
Github user jpountz commented on a diff in the pull request:

    https://github.com/apache/lucene-solr/pull/345#discussion_r179737753
  
    --- Diff: lucene/core/src/java/org/apache/lucene/search/Matches.java ---
    @@ -0,0 +1,116 @@
    +/*
    + * 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.HashMap;
    +import java.util.HashSet;
    +import java.util.Iterator;
    +import java.util.List;
    +import java.util.Map;
    +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)}
    + */
    +public class Matches implements Iterable<String> {
    +
    +  /**
    +   * Indicates a match with no term positions, for example on a Point or DocValues field
    +   */
    +  public static final Matches MATCH_WITH_NO_TERMS = new Matches(Collections.emptyMap());
    +
    +  private final Map<String, MatchesIterator> matches;
    +
    +  /**
    +   * Create a simple {@link Matches} for a single field
    +   */
    +  public static Matches fromField(String field, MatchesIterator it) {
    +    if (it == null) {
    +      return null;
    +    }
    +    return new Matches(field, it);
    +  }
    +
    +  /**
    +   * Amalgamate a collection of {@link Matches} into a single object
    +   */
    +  public static Matches fromSubMatches(List<Matches> subMatches) throws IOException {
    +    if (subMatches == null || subMatches.size() == 0) {
    +      return null;
    +    }
    +    subMatches = subMatches.stream().filter(m -> m != MATCH_WITH_NO_TERMS).collect(Collectors.toList());
    +    if (subMatches.size() == 0) {
    +      return MATCH_WITH_NO_TERMS;
    +    }
    +    if (subMatches.size() == 1) {
    +      return subMatches.get(0);
    +    }
    +    Map<String, MatchesIterator> matches = new HashMap<>();
    +    Set<String> allFields = new HashSet<>();
    +    for (Matches m : subMatches) {
    +      for (String field : m) {
    +        allFields.add(field);
    +      }
    +    }
    +    for (String field : allFields) {
    +      List<MatchesIterator> mis = new ArrayList<>();
    +      for (Matches m : subMatches) {
    +        MatchesIterator mi = m.getMatches(field);
    +        if (mi != null) {
    +          mis.add(mi);
    +        }
    +      }
    +      matches.put(field, DisjunctionMatchesIterator.fromSubIterators(mis));
    +    }
    +    return new Matches(matches);
    +  }
    +
    +  /**
    +   * Create a {@link Matches} from a map of fields to iterators
    +   */
    +  protected Matches(Map<String, MatchesIterator> matches) {
    +    this.matches = matches;
    +  }
    +
    +  private Matches(String field, MatchesIterator iterator) {
    +    this.matches = Collections.singletonMap(field, iterator);
    +  }
    +
    +  /**
    +   * Returns a {@link MatchesIterator} over the matches for a single field,
    +   * or {@code null} if there are no matches in that field.
    +   *
    +   * This method always returns the same iterator, so clients should only
    +   * call it once per field
    --- End diff --
    
    I find these semantics a bit error-prone. I'd like something like `Fields` where each call creates a new iterator better. 


---

---------------------------------------------------------------------
To unsubscribe, e-mail: dev-unsubscribe@lucene.apache.org
For additional commands, e-mail: dev-help@lucene.apache.org


[GitHub] lucene-solr pull request #345: LUCENE-8229: Add Weight.matches() method

Posted by romseygeek <gi...@git.apache.org>.
Github user romseygeek commented on a diff in the pull request:

    https://github.com/apache/lucene-solr/pull/345#discussion_r179913304
  
    --- Diff: lucene/core/src/java/org/apache/lucene/search/Weight.java ---
    @@ -69,6 +69,21 @@ protected Weight(Query query) {
        */
       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
    --- End diff --
    
    ++


---

---------------------------------------------------------------------
To unsubscribe, e-mail: dev-unsubscribe@lucene.apache.org
For additional commands, e-mail: dev-help@lucene.apache.org


[GitHub] lucene-solr pull request #345: LUCENE-8229: Add Weight.matches() method

Posted by jpountz <gi...@git.apache.org>.
Github user jpountz commented on a diff in the pull request:

    https://github.com/apache/lucene-solr/pull/345#discussion_r179439495
  
    --- Diff: lucene/core/src/java/org/apache/lucene/search/Matches.java ---
    @@ -0,0 +1,161 @@
    +/*
    + * 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.HashMap;
    +import java.util.HashSet;
    +import java.util.List;
    +import java.util.Map;
    +import java.util.Set;
    +
    +import org.apache.lucene.index.LeafReaderContext;
    +import org.apache.lucene.util.BytesRef;
    +
    +/**
    + * Reports the positions and optionally offsets of all matching terms in a query
    + * for a single document
    + *
    + * To find all fields that have matches, call {@link #getMatchFields()}
    + *
    + * To obtain a {@link MatchesIterator} for a particular field, call {@link #getMatches(String)}
    + */
    +public class Matches {
    +
    +  private final Map<String, MatchesIterator> matches;
    +
    +  /**
    +   * Create a simple {@link Matches} for a single field
    +   */
    +  public static Matches fromField(String field, MatchesIterator it) {
    +    if (it == null) {
    +      return null;
    +    }
    +    return new Matches(field, it);
    +  }
    +
    +  /**
    +   * Create an empty {@link Matches} for a Weight
    +   *
    +   * If the Weight's parent query does not match this document, returns {@code null},
    +   * otherwise returns a {@link Matches} document with an empty iterator on the given
    +   * fields
    +   */
    +  public static Matches emptyMatches(LeafReaderContext context, int doc, Weight weight, String... fields) throws IOException {
    +    Scorer scorer = weight.scorer(context);
    +    if (scorer == null || scorer.iterator().advance(doc) != doc) {
    +      return null;
    +    }
    +    List<Matches> matches = new ArrayList<>();
    +    for (String field : fields) {
    +      matches.add(Matches.fromField(field, EMPTY));
    +    }
    +    return Matches.fromSubMatches(matches);
    +  }
    +
    +  /**
    +   * Amalgamate a collection of {@link Matches} into a single object
    +   */
    +  public static Matches fromSubMatches(List<Matches> subMatches) throws IOException {
    +    if (subMatches == null || subMatches.size() == 0) {
    +      return null;
    +    }
    +    if (subMatches.size() == 1) {
    +      return subMatches.get(0);
    +    }
    +    Map<String, MatchesIterator> matches = new HashMap<>();
    +    Set<String> allFields = new HashSet<>();
    +    for (Matches m : subMatches) {
    +      allFields.addAll(m.getMatchFields());
    +    }
    +    for (String field : allFields) {
    +      List<MatchesIterator> mis = new ArrayList<>();
    +      for (Matches m : subMatches) {
    +        MatchesIterator mi = m.getMatches(field);
    +        if (mi != null) {
    +          mis.add(mi);
    +        }
    +      }
    +      matches.put(field, DisjunctionMatchesIterator.fromSubIterators(mis));
    +    }
    +    return new Matches(matches);
    +  }
    +
    +  /**
    +   * Create a {@link Matches} from a map of fields to iterators
    +   */
    +  protected Matches(Map<String, MatchesIterator> matches) {
    +    this.matches = matches;
    +  }
    +
    +  private Matches(String field, MatchesIterator iterator) {
    +    this.matches = new HashMap<>();
    +    this.matches.put(field, iterator);
    +  }
    +
    +  /**
    +   * Returns a {@link MatchesIterator} over the matches for a single field,
    +   * or {@code null} if there are no matches in that field
    +   */
    +  public MatchesIterator getMatches(String field) {
    +    return matches.get(field);
    --- End diff --
    
    I would expect this method to return a new iterator. But here if you call getMatches, then use it, and then call getMatches again, you will get an already consumed instance.


---

---------------------------------------------------------------------
To unsubscribe, e-mail: dev-unsubscribe@lucene.apache.org
For additional commands, e-mail: dev-help@lucene.apache.org


[GitHub] lucene-solr pull request #345: LUCENE-8229: Add Weight.matches() method

Posted by jpountz <gi...@git.apache.org>.
Github user jpountz commented on a diff in the pull request:

    https://github.com/apache/lucene-solr/pull/345#discussion_r179442549
  
    --- Diff: lucene/core/src/java/org/apache/lucene/search/DisjunctionMatchesIterator.java ---
    @@ -0,0 +1,158 @@
    +/*
    + * 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.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.
    + */
    +public class DisjunctionMatchesIterator implements MatchesIterator {
    --- End diff --
    
    make it final?


---

---------------------------------------------------------------------
To unsubscribe, e-mail: dev-unsubscribe@lucene.apache.org
For additional commands, e-mail: dev-help@lucene.apache.org


[GitHub] lucene-solr pull request #345: LUCENE-8229: Add Weight.matches() method

Posted by romseygeek <gi...@git.apache.org>.
Github user romseygeek commented on a diff in the pull request:

    https://github.com/apache/lucene-solr/pull/345#discussion_r179680356
  
    --- Diff: lucene/core/src/java/org/apache/lucene/search/DisjunctionMatchesIterator.java ---
    @@ -0,0 +1,158 @@
    +/*
    + * 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.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.
    + */
    +public 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
    +   */
    +  public static DisjunctionMatchesIterator fromTerms(LeafReaderContext context, int doc, String field, List<Term> terms) throws IOException {
    +    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
    +   */
    +  public static DisjunctionMatchesIterator fromTermsEnum(LeafReaderContext context, int doc, String field, BytesRefIterator terms) throws IOException {
    +    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;
    +    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());
    +      }
    +    };
    +    for (MatchesIterator mi : matches) {
    +      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;
    +    }
    --- End diff --
    
    I think retaining the duplicates could provide useful information - for instance, you might want to weight terms that appear in multiple subqueries higher when scoring passages for highlighting.


---

---------------------------------------------------------------------
To unsubscribe, e-mail: dev-unsubscribe@lucene.apache.org
For additional commands, e-mail: dev-help@lucene.apache.org


[GitHub] lucene-solr pull request #345: LUCENE-8229: Add Weight.matches() method

Posted by jpountz <gi...@git.apache.org>.
Github user jpountz commented on a diff in the pull request:

    https://github.com/apache/lucene-solr/pull/345#discussion_r180494440
  
    --- Diff: lucene/core/src/java/org/apache/lucene/search/Weight.java ---
    @@ -69,6 +69,24 @@ protected Weight(Query query) {
        */
       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 || scorer.iterator().advance(doc) != doc) {
    --- End diff --
    
    we might want to check the two-phase iterator instead if there is one so that `iterator().advance()` doesn't potentially visit millions of candidates before finding one that actually matches. See for instance ConstantScoreWeight.explain.


---

---------------------------------------------------------------------
To unsubscribe, e-mail: dev-unsubscribe@lucene.apache.org
For additional commands, e-mail: dev-help@lucene.apache.org


[GitHub] lucene-solr pull request #345: LUCENE-8229: Add Weight.matches() method

Posted by jpountz <gi...@git.apache.org>.
Github user jpountz commented on a diff in the pull request:

    https://github.com/apache/lucene-solr/pull/345#discussion_r179688171
  
    --- Diff: lucene/core/src/java/org/apache/lucene/search/DisjunctionMatchesIterator.java ---
    @@ -0,0 +1,158 @@
    +/*
    + * 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.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.
    + */
    +public 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
    +   */
    +  public static DisjunctionMatchesIterator fromTerms(LeafReaderContext context, int doc, String field, List<Term> terms) throws IOException {
    +    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
    +   */
    +  public static DisjunctionMatchesIterator fromTermsEnum(LeafReaderContext context, int doc, String field, BytesRefIterator terms) throws IOException {
    +    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;
    +    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());
    +      }
    +    };
    +    for (MatchesIterator mi : matches) {
    +      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;
    +    }
    --- End diff --
    
    OK. Let's mention this in the javadocs?


---

---------------------------------------------------------------------
To unsubscribe, e-mail: dev-unsubscribe@lucene.apache.org
For additional commands, e-mail: dev-help@lucene.apache.org


[GitHub] lucene-solr pull request #345: LUCENE-8229: Add Weight.matches() method

Posted by dsmiley <gi...@git.apache.org>.
Github user dsmiley commented on a diff in the pull request:

    https://github.com/apache/lucene-solr/pull/345#discussion_r180628960
  
    --- Diff: lucene/core/src/java/org/apache/lucene/search/DisjunctionMatchesIterator.java ---
    @@ -0,0 +1,168 @@
    +/*
    + * 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 {
    +    for (Term term : terms) {
    +      if (Objects.equals(field, term.field()) == false) {
    --- End diff --
    
    Why do you use Objects.equals instead of field.equals(term.field())?  Neither side can be null; right?


---

---------------------------------------------------------------------
To unsubscribe, e-mail: dev-unsubscribe@lucene.apache.org
For additional commands, e-mail: dev-help@lucene.apache.org


[GitHub] lucene-solr pull request #345: LUCENE-8229: Add Weight.matches() method

Posted by romseygeek <gi...@git.apache.org>.
Github user romseygeek commented on a diff in the pull request:

    https://github.com/apache/lucene-solr/pull/345#discussion_r179679922
  
    --- Diff: lucene/core/src/java/org/apache/lucene/search/DisjunctionMatchesIterator.java ---
    @@ -0,0 +1,158 @@
    +/*
    + * 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.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.
    + */
    +public 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
    +   */
    +  public static DisjunctionMatchesIterator fromTerms(LeafReaderContext context, int doc, String field, List<Term> terms) throws IOException {
    +    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
    +   */
    +  public static DisjunctionMatchesIterator fromTermsEnum(LeafReaderContext context, int doc, String field, BytesRefIterator terms) throws IOException {
    +    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;
    +    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);
    --- End diff --
    
    I've moved the notion of a match with no terms up a level into the Matches object, so this is dealt with in Matches.fromSubMatches() now.


---

---------------------------------------------------------------------
To unsubscribe, e-mail: dev-unsubscribe@lucene.apache.org
For additional commands, e-mail: dev-help@lucene.apache.org


[GitHub] lucene-solr pull request #345: LUCENE-8229: Add Weight.matches() method

Posted by jpountz <gi...@git.apache.org>.
Github user jpountz commented on a diff in the pull request:

    https://github.com/apache/lucene-solr/pull/345#discussion_r179442909
  
    --- Diff: lucene/core/src/java/org/apache/lucene/search/MatchesIterator.java ---
    @@ -0,0 +1,69 @@
    +/*
    + * 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}.
    --- End diff --
    
    I think it's worth repeating in the javadocs of each of these methods.


---

---------------------------------------------------------------------
To unsubscribe, e-mail: dev-unsubscribe@lucene.apache.org
For additional commands, e-mail: dev-help@lucene.apache.org


[GitHub] lucene-solr pull request #345: LUCENE-8229: Add Weight.matches() method

Posted by romseygeek <gi...@git.apache.org>.
Github user romseygeek commented on a diff in the pull request:

    https://github.com/apache/lucene-solr/pull/345#discussion_r180669841
  
    --- Diff: lucene/core/src/java/org/apache/lucene/search/DisjunctionMatchesIterator.java ---
    @@ -0,0 +1,168 @@
    +/*
    + * 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 {
    +    for (Term term : terms) {
    +      if (Objects.equals(field, term.field()) == false) {
    --- End diff --
    
    Paranoia mainly :) But this is probably better checked in an Objects.requireNonNull - will update.


---

---------------------------------------------------------------------
To unsubscribe, e-mail: dev-unsubscribe@lucene.apache.org
For additional commands, e-mail: dev-help@lucene.apache.org


[GitHub] lucene-solr pull request #345: LUCENE-8229: Add Weight.matches() method

Posted by jpountz <gi...@git.apache.org>.
Github user jpountz commented on a diff in the pull request:

    https://github.com/apache/lucene-solr/pull/345#discussion_r179743866
  
    --- Diff: lucene/core/src/test/org/apache/lucene/search/TestMatchesIterator.java ---
    @@ -0,0 +1,317 @@
    +/*
    + * 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;
    +
    +  public static final String FIELD_WITH_OFFSETS = "field_offsets";
    +  public static final String FIELD_NO_OFFSETS = "field_no_offsets";
    +
    +  public static final FieldType OFFSETS = new FieldType(TextField.TYPE_STORED);
    +  static {
    +    OFFSETS.setIndexOptions(IndexOptions.DOCS_AND_FREQS_AND_POSITIONS_AND_OFFSETS);
    +  }
    +
    +  @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(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.createNormalizedWeight(q, ScoreMode.COMPLETE_NO_SCORES);
    +    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);
    +      checkFieldMatches(it, expected[i]);
    +    }
    +  }
    +
    +  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 checkTerms(Query q, String field, String[][] expected) throws IOException {
    +    Weight w = searcher.createNormalizedWeight(q, ScoreMode.COMPLETE_NO_SCORES);
    +    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);
    +      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 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 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 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 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 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 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 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 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.createNormalizedWeight(q, ScoreMode.COMPLETE);
    +
    +    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"
    +  };
    +
    +}
    --- End diff --
    
    can you test the behaviour with DOCS and DOCS_AND_FREQS as well?


---

---------------------------------------------------------------------
To unsubscribe, e-mail: dev-unsubscribe@lucene.apache.org
For additional commands, e-mail: dev-help@lucene.apache.org


[GitHub] lucene-solr pull request #345: LUCENE-8229: Add Weight.matches() method

Posted by romseygeek <gi...@git.apache.org>.
Github user romseygeek commented on a diff in the pull request:

    https://github.com/apache/lucene-solr/pull/345#discussion_r179913306
  
    --- Diff: lucene/core/src/test/org/apache/lucene/search/TestMatchesIterator.java ---
    @@ -0,0 +1,317 @@
    +/*
    + * 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;
    +
    +  public static final String FIELD_WITH_OFFSETS = "field_offsets";
    +  public static final String FIELD_NO_OFFSETS = "field_no_offsets";
    +
    +  public static final FieldType OFFSETS = new FieldType(TextField.TYPE_STORED);
    +  static {
    +    OFFSETS.setIndexOptions(IndexOptions.DOCS_AND_FREQS_AND_POSITIONS_AND_OFFSETS);
    +  }
    +
    +  @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(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.createNormalizedWeight(q, ScoreMode.COMPLETE_NO_SCORES);
    +    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);
    +      checkFieldMatches(it, expected[i]);
    +    }
    +  }
    +
    +  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 checkTerms(Query q, String field, String[][] expected) throws IOException {
    +    Weight w = searcher.createNormalizedWeight(q, ScoreMode.COMPLETE_NO_SCORES);
    +    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);
    +      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 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 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 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 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 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 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 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 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.createNormalizedWeight(q, ScoreMode.COMPLETE);
    +
    +    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"
    +  };
    +
    +}
    --- End diff --
    
    ++


---

---------------------------------------------------------------------
To unsubscribe, e-mail: dev-unsubscribe@lucene.apache.org
For additional commands, e-mail: dev-help@lucene.apache.org