You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@lucene.apache.org by ro...@apache.org on 2017/01/14 10:52:19 UTC

[1/4] lucene-solr:master: LUCENE-7627: Add #intersect(CompiledAutomaton) to Sorted*DocValues

Repository: lucene-solr
Updated Branches:
  refs/heads/branch_6x e37b777e1 -> 966316f79
  refs/heads/master 53d5af17d -> 9403372fb


LUCENE-7627: Add #intersect(CompiledAutomaton) to Sorted*DocValues


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

Branch: refs/heads/master
Commit: 8fa0a8dd1e5eb3a5e2553c346372d203d00e575b
Parents: 53d5af1
Author: Alan Woodward <ro...@apache.org>
Authored: Wed Jan 11 12:07:11 2017 +0000
Committer: Alan Woodward <ro...@apache.org>
Committed: Sat Jan 14 09:40:19 2017 +0000

----------------------------------------------------------------------
 lucene/CHANGES.txt                              |  4 +++
 .../apache/lucene/index/SortedDocValues.java    | 22 ++++++++++++++
 .../apache/lucene/index/SortedSetDocValues.java | 22 ++++++++++++++
 .../index/BaseDocValuesFormatTestCase.java      | 32 ++++++++++++++++++++
 4 files changed, 80 insertions(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/8fa0a8dd/lucene/CHANGES.txt
----------------------------------------------------------------------
diff --git a/lucene/CHANGES.txt b/lucene/CHANGES.txt
index 30943d2..58201d6 100644
--- a/lucene/CHANGES.txt
+++ b/lucene/CHANGES.txt
@@ -126,6 +126,10 @@ New features
   concurrently across all segments in the index (Emmanuel Keller via
   Mike McCandless)
 
+* LUCENE-7627: Added .intersect methods to SortedDocValues and 
+  SortedSetDocValues to allow filtering their TermsEnums with a
+  CompiledAutomaton (Alan Woodward, Mike McCandless)
+
 Bug Fixes
 
 * LUCENE-7547: JapaneseTokenizerFactory was failing to close the

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/8fa0a8dd/lucene/core/src/java/org/apache/lucene/index/SortedDocValues.java
----------------------------------------------------------------------
diff --git a/lucene/core/src/java/org/apache/lucene/index/SortedDocValues.java b/lucene/core/src/java/org/apache/lucene/index/SortedDocValues.java
index e2d7dfd..087e487 100644
--- a/lucene/core/src/java/org/apache/lucene/index/SortedDocValues.java
+++ b/lucene/core/src/java/org/apache/lucene/index/SortedDocValues.java
@@ -20,6 +20,7 @@ package org.apache.lucene.index;
 import java.io.IOException;
 
 import org.apache.lucene.util.BytesRef;
+import org.apache.lucene.util.automaton.CompiledAutomaton;
 
 /**
  * A per-document byte[] with presorted values.  This is fundamentally an
@@ -110,4 +111,25 @@ public abstract class SortedDocValues extends BinaryDocValues {
     return new SortedDocValuesTermsEnum(this);
   }
 
+  /**
+   * Returns a {@link TermsEnum} over the values, filtered by a {@link CompiledAutomaton}
+   * The enum supports {@link TermsEnum#ord()}.
+   */
+  public TermsEnum intersect(CompiledAutomaton automaton) throws IOException {
+    TermsEnum in = termsEnum();
+    switch (automaton.type) {
+      case NONE:
+        return TermsEnum.EMPTY;
+      case ALL:
+        return in;
+      case SINGLE:
+        return new SingleTermsEnum(in, automaton.term);
+      case NORMAL:
+        return new AutomatonTermsEnum(in, automaton);
+      default:
+        // unreachable
+        throw new RuntimeException("unhandled case");
+    }
+  }
+
 }

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/8fa0a8dd/lucene/core/src/java/org/apache/lucene/index/SortedSetDocValues.java
----------------------------------------------------------------------
diff --git a/lucene/core/src/java/org/apache/lucene/index/SortedSetDocValues.java b/lucene/core/src/java/org/apache/lucene/index/SortedSetDocValues.java
index 6d02c25..9e1c6a3 100644
--- a/lucene/core/src/java/org/apache/lucene/index/SortedSetDocValues.java
+++ b/lucene/core/src/java/org/apache/lucene/index/SortedSetDocValues.java
@@ -20,6 +20,7 @@ package org.apache.lucene.index;
 import java.io.IOException;
 
 import org.apache.lucene.util.BytesRef;
+import org.apache.lucene.util.automaton.CompiledAutomaton;
 
 /**
  * A multi-valued version of {@link SortedDocValues}.
@@ -102,4 +103,25 @@ public abstract class SortedSetDocValues extends DocValuesIterator {
   public TermsEnum termsEnum() throws IOException {
     return new SortedSetDocValuesTermsEnum(this);
   }
+
+  /**
+   * Returns a {@link TermsEnum} over the values, filtered by a {@link CompiledAutomaton}
+   * The enum supports {@link TermsEnum#ord()}.
+   */
+  public TermsEnum intersect(CompiledAutomaton automaton) throws IOException {
+    TermsEnum in = termsEnum();
+    switch (automaton.type) {
+      case NONE:
+        return TermsEnum.EMPTY;
+      case ALL:
+        return in;
+      case SINGLE:
+        return new SingleTermsEnum(in, automaton.term);
+      case NORMAL:
+        return new AutomatonTermsEnum(in, automaton);
+      default:
+        // unreachable
+        throw new RuntimeException("unhandled case");
+    }
+  }
 }

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/8fa0a8dd/lucene/test-framework/src/java/org/apache/lucene/index/BaseDocValuesFormatTestCase.java
----------------------------------------------------------------------
diff --git a/lucene/test-framework/src/java/org/apache/lucene/index/BaseDocValuesFormatTestCase.java b/lucene/test-framework/src/java/org/apache/lucene/index/BaseDocValuesFormatTestCase.java
index d55f212..8cb6665 100644
--- a/lucene/test-framework/src/java/org/apache/lucene/index/BaseDocValuesFormatTestCase.java
+++ b/lucene/test-framework/src/java/org/apache/lucene/index/BaseDocValuesFormatTestCase.java
@@ -67,6 +67,8 @@ import org.apache.lucene.util.IOUtils;
 import org.apache.lucene.util.TestUtil;
 
 import com.carrotsearch.randomizedtesting.generators.RandomPicks;
+import org.apache.lucene.util.automaton.CompiledAutomaton;
+import org.apache.lucene.util.automaton.RegExp;
 
 import static org.apache.lucene.index.SortedSetDocValues.NO_MORE_ORDS;
 import static org.apache.lucene.search.DocIdSetIterator.NO_MORE_DOCS;
@@ -906,6 +908,21 @@ public abstract class BaseDocValuesFormatTestCase extends BaseIndexFileFormatTes
     termsEnum.seekExact(2);
     assertEquals("world", termsEnum.term().utf8ToString());
     assertEquals(2, termsEnum.ord());
+
+    // NORMAL automaton
+    termsEnum = dv.intersect(new CompiledAutomaton(new RegExp(".*l.*").toAutomaton()));
+    assertEquals("hello", termsEnum.next().utf8ToString());
+    assertEquals(1, termsEnum.ord());
+    assertEquals("world", termsEnum.next().utf8ToString());
+    assertEquals(2, termsEnum.ord());
+    assertNull(termsEnum.next());
+
+    // SINGLE automaton
+    termsEnum = dv.intersect(new CompiledAutomaton(new RegExp("hello").toAutomaton()));
+    assertEquals("hello", termsEnum.next().utf8ToString());
+    assertEquals(1, termsEnum.ord());
+    assertNull(termsEnum.next());
+
     ireader.close();
     directory.close();
   }
@@ -2057,6 +2074,21 @@ public abstract class BaseDocValuesFormatTestCase extends BaseIndexFileFormatTes
     termsEnum.seekExact(2);
     assertEquals("world", termsEnum.term().utf8ToString());
     assertEquals(2, termsEnum.ord());
+
+    // NORMAL automaton
+    termsEnum = dv.intersect(new CompiledAutomaton(new RegExp(".*l.*").toAutomaton()));
+    assertEquals("hello", termsEnum.next().utf8ToString());
+    assertEquals(1, termsEnum.ord());
+    assertEquals("world", termsEnum.next().utf8ToString());
+    assertEquals(2, termsEnum.ord());
+    assertNull(termsEnum.next());
+
+    // SINGLE automaton
+    termsEnum = dv.intersect(new CompiledAutomaton(new RegExp("hello").toAutomaton()));
+    assertEquals("hello", termsEnum.next().utf8ToString());
+    assertEquals(1, termsEnum.ord());
+    assertNull(termsEnum.next());
+
     ireader.close();
     directory.close();
   }


[2/4] lucene-solr:master: Add getMatchingChildren() method to Scorer

Posted by ro...@apache.org.
Add getMatchingChildren() method to Scorer


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

Branch: refs/heads/master
Commit: 9403372fbc36aced848bf8aa498bc71b7b94567b
Parents: 8fa0a8d
Author: Alan Woodward <ro...@apache.org>
Authored: Sat Jan 14 09:08:02 2017 +0000
Committer: Alan Woodward <ro...@apache.org>
Committed: Sat Jan 14 10:28:51 2017 +0000

----------------------------------------------------------------------
 lucene/CHANGES.txt                              |  4 +++
 .../apache/lucene/search/DisjunctionScorer.java |  8 +++++
 .../lucene/search/MinShouldMatchSumScorer.java  |  9 ++++++
 .../java/org/apache/lucene/search/Scorer.java   |  8 +++++
 .../search/TestBooleanQueryVisitSubscorers.java | 33 ++++++++++++++++++++
 5 files changed, 62 insertions(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/9403372f/lucene/CHANGES.txt
----------------------------------------------------------------------
diff --git a/lucene/CHANGES.txt b/lucene/CHANGES.txt
index 58201d6..540188e 100644
--- a/lucene/CHANGES.txt
+++ b/lucene/CHANGES.txt
@@ -130,6 +130,10 @@ New features
   SortedSetDocValues to allow filtering their TermsEnums with a
   CompiledAutomaton (Alan Woodward, Mike McCandless)
 
+* LUCENE-7628: Scorer now has a getMatchingChildren() method that will 
+  return all child scorers positioned on the current document.  (Alan
+  Woodward)
+
 Bug Fixes
 
 * LUCENE-7547: JapaneseTokenizerFactory was failing to close the

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/9403372f/lucene/core/src/java/org/apache/lucene/search/DisjunctionScorer.java
----------------------------------------------------------------------
diff --git a/lucene/core/src/java/org/apache/lucene/search/DisjunctionScorer.java b/lucene/core/src/java/org/apache/lucene/search/DisjunctionScorer.java
index c53942a..a76999e 100644
--- a/lucene/core/src/java/org/apache/lucene/search/DisjunctionScorer.java
+++ b/lucene/core/src/java/org/apache/lucene/search/DisjunctionScorer.java
@@ -202,4 +202,12 @@ abstract class DisjunctionScorer extends Scorer {
     return children;
   }
 
+  @Override
+  public Collection<ChildScorer> getMatchingChildren() throws IOException {
+    List<ChildScorer> children = new ArrayList<>();
+    for (DisiWrapper w = getSubMatches(); w != null; w = w.next) {
+      children.add(new ChildScorer(w.scorer, "SHOULD"));
+    }
+    return children;
+  }
 }

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/9403372f/lucene/core/src/java/org/apache/lucene/search/MinShouldMatchSumScorer.java
----------------------------------------------------------------------
diff --git a/lucene/core/src/java/org/apache/lucene/search/MinShouldMatchSumScorer.java b/lucene/core/src/java/org/apache/lucene/search/MinShouldMatchSumScorer.java
index 032b5fe..b977400 100644
--- a/lucene/core/src/java/org/apache/lucene/search/MinShouldMatchSumScorer.java
+++ b/lucene/core/src/java/org/apache/lucene/search/MinShouldMatchSumScorer.java
@@ -133,6 +133,15 @@ final class MinShouldMatchSumScorer extends Scorer {
   }
 
   @Override
+  public Collection<ChildScorer> getMatchingChildren() throws IOException {
+    List<ChildScorer> children = new ArrayList<>();
+    for (DisiWrapper s = lead; s != null; s = s.next) {
+      children.add(new ChildScorer(s.scorer, "SHOULD"));
+    }
+    return children;
+  }
+
+  @Override
   public DocIdSetIterator iterator() {
     return new DocIdSetIterator() {
 

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/9403372f/lucene/core/src/java/org/apache/lucene/search/Scorer.java
----------------------------------------------------------------------
diff --git a/lucene/core/src/java/org/apache/lucene/search/Scorer.java b/lucene/core/src/java/org/apache/lucene/search/Scorer.java
index f434327..2e35e91 100644
--- a/lucene/core/src/java/org/apache/lucene/search/Scorer.java
+++ b/lucene/core/src/java/org/apache/lucene/search/Scorer.java
@@ -82,6 +82,14 @@ public abstract class Scorer {
   public Collection<ChildScorer> getChildren() {
     return Collections.emptyList();
   }
+
+  /**
+   * Returns child sub-scorers that match the current document
+   * @lucene.experimental
+   */
+  public Collection<ChildScorer> getMatchingChildren() throws IOException {
+    return getChildren();
+  }
   
   /** A child Scorer and its relationship to its parent.
    * the meaning of the relationship depends upon the parent query. 

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/9403372f/lucene/core/src/test/org/apache/lucene/search/TestBooleanQueryVisitSubscorers.java
----------------------------------------------------------------------
diff --git a/lucene/core/src/test/org/apache/lucene/search/TestBooleanQueryVisitSubscorers.java b/lucene/core/src/test/org/apache/lucene/search/TestBooleanQueryVisitSubscorers.java
index 60ba528..0909d5d 100644
--- a/lucene/core/src/test/org/apache/lucene/search/TestBooleanQueryVisitSubscorers.java
+++ b/lucene/core/src/test/org/apache/lucene/search/TestBooleanQueryVisitSubscorers.java
@@ -108,6 +108,39 @@ public class TestBooleanQueryVisitSubscorers extends LuceneTestCase {
     assertEquals(2, tfs.get(1).intValue()); // f2:search + f2:lucene
     assertEquals(2, tfs.get(2).intValue()); // f2:search + f2:lucene
   }
+
+  public void testDisjunctionMatches() throws IOException {
+    BooleanQuery.Builder bq1 = new BooleanQuery.Builder();
+    bq1.add(new TermQuery(new Term(F1, "lucene")), Occur.SHOULD);
+    bq1.add(new PhraseQuery(F2, "search", "engine"), Occur.SHOULD);
+
+    Weight w1 = scorerSearcher.createNormalizedWeight(bq1.build(), true);
+    Scorer s1 = w1.scorer(reader.leaves().get(0));
+    assertEquals(0, s1.iterator().nextDoc());
+    assertEquals(2, s1.getMatchingChildren().size());
+
+    BooleanQuery.Builder bq2 = new BooleanQuery.Builder();
+    bq2.add(new TermQuery(new Term(F1, "lucene")), Occur.SHOULD);
+    bq2.add(new PhraseQuery(F2, "search", "library"), Occur.SHOULD);
+
+    Weight w2 = scorerSearcher.createNormalizedWeight(bq2.build(), true);
+    Scorer s2 = w2.scorer(reader.leaves().get(0));
+    assertEquals(0, s2.iterator().nextDoc());
+    assertEquals(1, s2.getMatchingChildren().size());
+  }
+
+  public void testMinShouldMatchMatches() throws IOException {
+    BooleanQuery.Builder bq = new BooleanQuery.Builder();
+    bq.add(new TermQuery(new Term(F1, "lucene")), Occur.SHOULD);
+    bq.add(new TermQuery(new Term(F2, "lucene")), Occur.SHOULD);
+    bq.add(new PhraseQuery(F2, "search", "library"), Occur.SHOULD);
+    bq.setMinimumNumberShouldMatch(2);
+
+    Weight w = scorerSearcher.createNormalizedWeight(bq.build(), true);
+    Scorer s = w.scorer(reader.leaves().get(0));
+    assertEquals(0, s.iterator().nextDoc());
+    assertEquals(2, s.getMatchingChildren().size());
+  }
   
   public void testConjunctions() throws IOException {
     BooleanQuery.Builder bq = new BooleanQuery.Builder();


[3/4] lucene-solr:branch_6x: LUCENE-7627: Add #intersect(CompiledAutomaton) to Sorted*DocValues

Posted by ro...@apache.org.
LUCENE-7627: Add #intersect(CompiledAutomaton) to Sorted*DocValues


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

Branch: refs/heads/branch_6x
Commit: c2c758bb71e621b1d8c5d8b228b8dfe4ec50acfe
Parents: e37b777
Author: Alan Woodward <ro...@apache.org>
Authored: Wed Jan 11 12:07:11 2017 +0000
Committer: Alan Woodward <ro...@apache.org>
Committed: Sat Jan 14 10:32:14 2017 +0000

----------------------------------------------------------------------
 lucene/CHANGES.txt                              |  4 +++
 .../apache/lucene/index/SortedDocValues.java    | 25 +++++++++++++++
 .../apache/lucene/index/SortedSetDocValues.java | 24 +++++++++++++++
 .../index/BaseDocValuesFormatTestCase.java      | 32 ++++++++++++++++++++
 4 files changed, 85 insertions(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/c2c758bb/lucene/CHANGES.txt
----------------------------------------------------------------------
diff --git a/lucene/CHANGES.txt b/lucene/CHANGES.txt
index ff5d138..7dc7239 100644
--- a/lucene/CHANGES.txt
+++ b/lucene/CHANGES.txt
@@ -67,6 +67,10 @@ New features
   concurrently across all segments in the index (Emmanuel Keller via
   Mike McCandless)
 
+* LUCENE-7627: Added .intersect methods to SortedDocValues and 
+  SortedSetDocValues to allow filtering their TermsEnums with a
+  CompiledAutomaton (Alan Woodward, Mike McCandless)
+
 Bug Fixes
 
 * LUCENE-7547: JapaneseTokenizerFactory was failing to close the

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/c2c758bb/lucene/core/src/java/org/apache/lucene/index/SortedDocValues.java
----------------------------------------------------------------------
diff --git a/lucene/core/src/java/org/apache/lucene/index/SortedDocValues.java b/lucene/core/src/java/org/apache/lucene/index/SortedDocValues.java
index e9a55a3..87f8b7c 100644
--- a/lucene/core/src/java/org/apache/lucene/index/SortedDocValues.java
+++ b/lucene/core/src/java/org/apache/lucene/index/SortedDocValues.java
@@ -17,7 +17,10 @@
 package org.apache.lucene.index;
 
 
+import java.io.IOException;
+
 import org.apache.lucene.util.BytesRef;
+import org.apache.lucene.util.automaton.CompiledAutomaton;
 
 /**
  * A per-document byte[] with presorted values.
@@ -104,4 +107,26 @@ public abstract class SortedDocValues extends BinaryDocValues {
   public TermsEnum termsEnum() {
     return new SortedDocValuesTermsEnum(this);
   }
+
+  /**
+   * Returns a {@link TermsEnum} over the values, filtered by a {@link CompiledAutomaton}
+   * The enum supports {@link TermsEnum#ord()}.
+   */
+  public TermsEnum intersect(CompiledAutomaton automaton) throws IOException {
+    TermsEnum in = termsEnum();
+    switch (automaton.type) {
+      case NONE:
+        return TermsEnum.EMPTY;
+      case ALL:
+        return in;
+      case SINGLE:
+        return new SingleTermsEnum(in, automaton.term);
+      case NORMAL:
+        return new AutomatonTermsEnum(in, automaton);
+      default:
+        // unreachable
+        throw new RuntimeException("unhandled case");
+    }
+  }
+  
 }

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/c2c758bb/lucene/core/src/java/org/apache/lucene/index/SortedSetDocValues.java
----------------------------------------------------------------------
diff --git a/lucene/core/src/java/org/apache/lucene/index/SortedSetDocValues.java b/lucene/core/src/java/org/apache/lucene/index/SortedSetDocValues.java
index f68efcc..64abd64 100644
--- a/lucene/core/src/java/org/apache/lucene/index/SortedSetDocValues.java
+++ b/lucene/core/src/java/org/apache/lucene/index/SortedSetDocValues.java
@@ -17,7 +17,10 @@
 package org.apache.lucene.index;
 
 
+import java.io.IOException;
+
 import org.apache.lucene.util.BytesRef;
+import org.apache.lucene.util.automaton.CompiledAutomaton;
 
 /**
  * A per-document set of presorted byte[] values.
@@ -103,4 +106,25 @@ public abstract class SortedSetDocValues {
   public TermsEnum termsEnum() {
     return new SortedSetDocValuesTermsEnum(this);
   }
+
+  /**
+   * Returns a {@link TermsEnum} over the values, filtered by a {@link CompiledAutomaton}
+   * The enum supports {@link TermsEnum#ord()}.
+   */
+  public TermsEnum intersect(CompiledAutomaton automaton) throws IOException {
+    TermsEnum in = termsEnum();
+    switch (automaton.type) {
+      case NONE:
+        return TermsEnum.EMPTY;
+      case ALL:
+        return in;
+      case SINGLE:
+        return new SingleTermsEnum(in, automaton.term);
+      case NORMAL:
+        return new AutomatonTermsEnum(in, automaton);
+      default:
+        // unreachable
+        throw new RuntimeException("unhandled case");
+    }
+  }
 }

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/c2c758bb/lucene/test-framework/src/java/org/apache/lucene/index/BaseDocValuesFormatTestCase.java
----------------------------------------------------------------------
diff --git a/lucene/test-framework/src/java/org/apache/lucene/index/BaseDocValuesFormatTestCase.java b/lucene/test-framework/src/java/org/apache/lucene/index/BaseDocValuesFormatTestCase.java
index 85ac12f..b573c20 100644
--- a/lucene/test-framework/src/java/org/apache/lucene/index/BaseDocValuesFormatTestCase.java
+++ b/lucene/test-framework/src/java/org/apache/lucene/index/BaseDocValuesFormatTestCase.java
@@ -63,6 +63,8 @@ import org.apache.lucene.util.IOUtils;
 import org.apache.lucene.util.TestUtil;
 
 import com.carrotsearch.randomizedtesting.generators.RandomPicks;
+import org.apache.lucene.util.automaton.CompiledAutomaton;
+import org.apache.lucene.util.automaton.RegExp;
 
 import static org.apache.lucene.index.SortedSetDocValues.NO_MORE_ORDS;
 
@@ -879,6 +881,21 @@ public abstract class BaseDocValuesFormatTestCase extends BaseIndexFileFormatTes
     termsEnum.seekExact(2);
     assertEquals("world", termsEnum.term().utf8ToString());
     assertEquals(2, termsEnum.ord());
+
+    // NORMAL automaton
+    termsEnum = dv.intersect(new CompiledAutomaton(new RegExp(".*l.*").toAutomaton()));
+    assertEquals("hello", termsEnum.next().utf8ToString());
+    assertEquals(1, termsEnum.ord());
+    assertEquals("world", termsEnum.next().utf8ToString());
+    assertEquals(2, termsEnum.ord());
+    assertNull(termsEnum.next());
+
+    // SINGLE automaton
+    termsEnum = dv.intersect(new CompiledAutomaton(new RegExp("hello").toAutomaton()));
+    assertEquals("hello", termsEnum.next().utf8ToString());
+    assertEquals(1, termsEnum.ord());
+    assertNull(termsEnum.next());
+
     ireader.close();
     directory.close();
   }
@@ -1937,6 +1954,21 @@ public abstract class BaseDocValuesFormatTestCase extends BaseIndexFileFormatTes
     termsEnum.seekExact(2);
     assertEquals("world", termsEnum.term().utf8ToString());
     assertEquals(2, termsEnum.ord());
+
+    // NORMAL automaton
+    termsEnum = dv.intersect(new CompiledAutomaton(new RegExp(".*l.*").toAutomaton()));
+    assertEquals("hello", termsEnum.next().utf8ToString());
+    assertEquals(1, termsEnum.ord());
+    assertEquals("world", termsEnum.next().utf8ToString());
+    assertEquals(2, termsEnum.ord());
+    assertNull(termsEnum.next());
+
+    // SINGLE automaton
+    termsEnum = dv.intersect(new CompiledAutomaton(new RegExp("hello").toAutomaton()));
+    assertEquals("hello", termsEnum.next().utf8ToString());
+    assertEquals(1, termsEnum.ord());
+    assertNull(termsEnum.next());
+
     ireader.close();
     directory.close();
   }


[4/4] lucene-solr:branch_6x: Add getMatchingChildren() method to Scorer

Posted by ro...@apache.org.
Add getMatchingChildren() method to Scorer


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

Branch: refs/heads/branch_6x
Commit: 966316f79cbc41ebbd78c03035333a8398b87912
Parents: c2c758b
Author: Alan Woodward <ro...@apache.org>
Authored: Sat Jan 14 09:08:02 2017 +0000
Committer: Alan Woodward <ro...@apache.org>
Committed: Sat Jan 14 10:32:14 2017 +0000

----------------------------------------------------------------------
 lucene/CHANGES.txt                              |  4 +++
 .../apache/lucene/search/DisjunctionScorer.java |  8 +++++
 .../lucene/search/MinShouldMatchSumScorer.java  |  9 ++++++
 .../java/org/apache/lucene/search/Scorer.java   |  8 +++++
 .../search/TestBooleanQueryVisitSubscorers.java | 33 ++++++++++++++++++++
 5 files changed, 62 insertions(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/966316f7/lucene/CHANGES.txt
----------------------------------------------------------------------
diff --git a/lucene/CHANGES.txt b/lucene/CHANGES.txt
index 7dc7239..747eea3 100644
--- a/lucene/CHANGES.txt
+++ b/lucene/CHANGES.txt
@@ -71,6 +71,10 @@ New features
   SortedSetDocValues to allow filtering their TermsEnums with a
   CompiledAutomaton (Alan Woodward, Mike McCandless)
 
+* LUCENE-7628: Scorer now has a getMatchingChildren() method that will 
+  return all child scorers positioned on the current document.  (Alan
+  Woodward)
+
 Bug Fixes
 
 * LUCENE-7547: JapaneseTokenizerFactory was failing to close the

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/966316f7/lucene/core/src/java/org/apache/lucene/search/DisjunctionScorer.java
----------------------------------------------------------------------
diff --git a/lucene/core/src/java/org/apache/lucene/search/DisjunctionScorer.java b/lucene/core/src/java/org/apache/lucene/search/DisjunctionScorer.java
index c53942a..a76999e 100644
--- a/lucene/core/src/java/org/apache/lucene/search/DisjunctionScorer.java
+++ b/lucene/core/src/java/org/apache/lucene/search/DisjunctionScorer.java
@@ -202,4 +202,12 @@ abstract class DisjunctionScorer extends Scorer {
     return children;
   }
 
+  @Override
+  public Collection<ChildScorer> getMatchingChildren() throws IOException {
+    List<ChildScorer> children = new ArrayList<>();
+    for (DisiWrapper w = getSubMatches(); w != null; w = w.next) {
+      children.add(new ChildScorer(w.scorer, "SHOULD"));
+    }
+    return children;
+  }
 }

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/966316f7/lucene/core/src/java/org/apache/lucene/search/MinShouldMatchSumScorer.java
----------------------------------------------------------------------
diff --git a/lucene/core/src/java/org/apache/lucene/search/MinShouldMatchSumScorer.java b/lucene/core/src/java/org/apache/lucene/search/MinShouldMatchSumScorer.java
index 9653d95..1835669 100644
--- a/lucene/core/src/java/org/apache/lucene/search/MinShouldMatchSumScorer.java
+++ b/lucene/core/src/java/org/apache/lucene/search/MinShouldMatchSumScorer.java
@@ -135,6 +135,15 @@ final class MinShouldMatchSumScorer extends Scorer {
   }
 
   @Override
+  public Collection<ChildScorer> getMatchingChildren() throws IOException {
+    List<ChildScorer> children = new ArrayList<>();
+    for (DisiWrapper s = lead; s != null; s = s.next) {
+      children.add(new ChildScorer(s.scorer, "SHOULD"));
+    }
+    return children;
+  }
+
+  @Override
   public DocIdSetIterator iterator() {
     return new DocIdSetIterator() {
 

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/966316f7/lucene/core/src/java/org/apache/lucene/search/Scorer.java
----------------------------------------------------------------------
diff --git a/lucene/core/src/java/org/apache/lucene/search/Scorer.java b/lucene/core/src/java/org/apache/lucene/search/Scorer.java
index f434327..2e35e91 100644
--- a/lucene/core/src/java/org/apache/lucene/search/Scorer.java
+++ b/lucene/core/src/java/org/apache/lucene/search/Scorer.java
@@ -82,6 +82,14 @@ public abstract class Scorer {
   public Collection<ChildScorer> getChildren() {
     return Collections.emptyList();
   }
+
+  /**
+   * Returns child sub-scorers that match the current document
+   * @lucene.experimental
+   */
+  public Collection<ChildScorer> getMatchingChildren() throws IOException {
+    return getChildren();
+  }
   
   /** A child Scorer and its relationship to its parent.
    * the meaning of the relationship depends upon the parent query. 

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/966316f7/lucene/core/src/test/org/apache/lucene/search/TestBooleanQueryVisitSubscorers.java
----------------------------------------------------------------------
diff --git a/lucene/core/src/test/org/apache/lucene/search/TestBooleanQueryVisitSubscorers.java b/lucene/core/src/test/org/apache/lucene/search/TestBooleanQueryVisitSubscorers.java
index 443246b..ca27a60 100644
--- a/lucene/core/src/test/org/apache/lucene/search/TestBooleanQueryVisitSubscorers.java
+++ b/lucene/core/src/test/org/apache/lucene/search/TestBooleanQueryVisitSubscorers.java
@@ -108,6 +108,39 @@ public class TestBooleanQueryVisitSubscorers extends LuceneTestCase {
     assertEquals(2, tfs.get(1).intValue()); // f2:search + f2:lucene
     assertEquals(2, tfs.get(2).intValue()); // f2:search + f2:lucene
   }
+
+  public void testDisjunctionMatches() throws IOException {
+    BooleanQuery.Builder bq1 = new BooleanQuery.Builder();
+    bq1.add(new TermQuery(new Term(F1, "lucene")), Occur.SHOULD);
+    bq1.add(new PhraseQuery(F2, "search", "engine"), Occur.SHOULD);
+
+    Weight w1 = scorerSearcher.createNormalizedWeight(bq1.build(), true);
+    Scorer s1 = w1.scorer(reader.leaves().get(0));
+    assertEquals(0, s1.iterator().nextDoc());
+    assertEquals(2, s1.getMatchingChildren().size());
+
+    BooleanQuery.Builder bq2 = new BooleanQuery.Builder();
+    bq2.add(new TermQuery(new Term(F1, "lucene")), Occur.SHOULD);
+    bq2.add(new PhraseQuery(F2, "search", "library"), Occur.SHOULD);
+
+    Weight w2 = scorerSearcher.createNormalizedWeight(bq2.build(), true);
+    Scorer s2 = w2.scorer(reader.leaves().get(0));
+    assertEquals(0, s2.iterator().nextDoc());
+    assertEquals(1, s2.getMatchingChildren().size());
+  }
+
+  public void testMinShouldMatchMatches() throws IOException {
+    BooleanQuery.Builder bq = new BooleanQuery.Builder();
+    bq.add(new TermQuery(new Term(F1, "lucene")), Occur.SHOULD);
+    bq.add(new TermQuery(new Term(F2, "lucene")), Occur.SHOULD);
+    bq.add(new PhraseQuery(F2, "search", "library"), Occur.SHOULD);
+    bq.setMinimumNumberShouldMatch(2);
+
+    Weight w = scorerSearcher.createNormalizedWeight(bq.build(), true);
+    Scorer s = w.scorer(reader.leaves().get(0));
+    assertEquals(0, s.iterator().nextDoc());
+    assertEquals(2, s.getMatchingChildren().size());
+  }
   
   public void testConjunctions() throws IOException {
     BooleanQuery.Builder bq = new BooleanQuery.Builder();