You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@lucene.apache.org by jp...@apache.org on 2017/01/17 13:32:38 UTC

[2/3] lucene-solr:branch_6x: LUCENE-7055: Add ScorerProvider to get an estimation of the cost of scorers before building them.

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/aba65396/lucene/sandbox/src/java/org/apache/lucene/search/IndexOrDocValuesQuery.java
----------------------------------------------------------------------
diff --git a/lucene/sandbox/src/java/org/apache/lucene/search/IndexOrDocValuesQuery.java b/lucene/sandbox/src/java/org/apache/lucene/search/IndexOrDocValuesQuery.java
new file mode 100644
index 0000000..6ac980a
--- /dev/null
+++ b/lucene/sandbox/src/java/org/apache/lucene/search/IndexOrDocValuesQuery.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 org.apache.lucene.index.IndexReader;
+import org.apache.lucene.index.LeafReaderContext;
+
+/**
+ * A query that uses either an index (points or terms) or doc values in order
+ * to run a range query, depending which one is more efficient.
+ */
+public final class IndexOrDocValuesQuery extends Query {
+
+  private final Query indexQuery, dvQuery;
+
+  /**
+   * Constructor that takes both a query that executes on an index structure
+   * like the inverted index or the points tree, and another query that
+   * executes on doc values. Both queries must match the same documents and
+   * attribute constant scores.
+   */
+  public IndexOrDocValuesQuery(Query indexQuery, Query dvQuery) {
+    this.indexQuery = indexQuery;
+    this.dvQuery = dvQuery;
+  }
+
+  @Override
+  public String toString(String field) {
+    return indexQuery.toString(field);
+  }
+
+  @Override
+  public boolean equals(Object obj) {
+    if (sameClassAs(obj) == false) {
+      return false;
+    }
+    IndexOrDocValuesQuery that = (IndexOrDocValuesQuery) obj;
+    return indexQuery.equals(that.indexQuery) && dvQuery.equals(that.dvQuery);
+  }
+
+  @Override
+  public int hashCode() {
+    int h = classHash();
+    h = 31 * h + indexQuery.hashCode();
+    h = 31 * h + dvQuery.hashCode();
+    return h;
+  }
+
+  @Override
+  public Query rewrite(IndexReader reader) throws IOException {
+    Query indexRewrite = indexQuery.rewrite(reader);
+    Query dvRewrite = dvQuery.rewrite(reader);
+    if (indexQuery != indexRewrite || dvQuery != dvRewrite) {
+      return new IndexOrDocValuesQuery(indexRewrite, dvRewrite);
+    }
+    return this;
+  }
+
+  @Override
+  public Weight createWeight(IndexSearcher searcher, boolean needsScores) throws IOException {
+    final Weight indexWeight = indexQuery.createWeight(searcher, needsScores);
+    final Weight dvWeight = dvQuery.createWeight(searcher, needsScores);
+    return new ConstantScoreWeight(this) {
+      @Override
+      public BulkScorer bulkScorer(LeafReaderContext context) throws IOException {
+        return indexWeight.bulkScorer(context);
+      }
+
+      @Override
+      public ScorerSupplier scorerSupplier(LeafReaderContext context) throws IOException {
+        final ScorerSupplier indexScorerSupplier = indexWeight.scorerSupplier(context);
+        final ScorerSupplier dvScorerSupplier = dvWeight.scorerSupplier(context); 
+        if (indexScorerSupplier == null || dvScorerSupplier == null) {
+          return null;
+        }
+        return new ScorerSupplier() {
+          @Override
+          public Scorer get(boolean randomAccess) throws IOException {
+            return (randomAccess ? dvScorerSupplier : indexScorerSupplier).get(randomAccess);
+          }
+
+          @Override
+          public long cost() {
+            return Math.min(indexScorerSupplier.cost(), dvScorerSupplier.cost());
+          }
+        };
+      }
+
+      @Override
+      public Scorer scorer(LeafReaderContext context) throws IOException {
+        ScorerSupplier scorerSupplier = scorerSupplier(context);
+        if (scorerSupplier == null) {
+          return null;
+        }
+        return scorerSupplier.get(false);
+      }
+    };
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/aba65396/lucene/sandbox/src/test/org/apache/lucene/search/TestIndexOrDocValuesQuery.java
----------------------------------------------------------------------
diff --git a/lucene/sandbox/src/test/org/apache/lucene/search/TestIndexOrDocValuesQuery.java b/lucene/sandbox/src/test/org/apache/lucene/search/TestIndexOrDocValuesQuery.java
new file mode 100644
index 0000000..2a16e5d
--- /dev/null
+++ b/lucene/sandbox/src/test/org/apache/lucene/search/TestIndexOrDocValuesQuery.java
@@ -0,0 +1,89 @@
+/*
+ * 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.document.Document;
+import org.apache.lucene.document.Field.Store;
+import org.apache.lucene.document.LongPoint;
+import org.apache.lucene.document.NumericDocValuesField;
+import org.apache.lucene.document.StringField;
+import org.apache.lucene.index.DirectoryReader;
+import org.apache.lucene.index.IndexReader;
+import org.apache.lucene.index.IndexWriter;
+import org.apache.lucene.index.Term;
+import org.apache.lucene.search.BooleanClause.Occur;
+import org.apache.lucene.store.Directory;
+import org.apache.lucene.util.LuceneTestCase;
+import org.apache.lucene.util.TestUtil;
+
+public class TestIndexOrDocValuesQuery extends LuceneTestCase {
+
+  public void testUseIndexForSelectiveQueries() throws IOException {
+    Directory dir = newDirectory();
+    IndexWriter w = new IndexWriter(dir, newIndexWriterConfig()
+        // relies on costs and PointValues.estimateCost so we need the default codec
+        .setCodec(TestUtil.getDefaultCodec()));
+    for (int i = 0; i < 2000; ++i) {
+      Document doc = new Document();
+      if (i == 42) {
+        doc.add(new StringField("f1", "bar", Store.NO));
+        doc.add(new LongPoint("f2", 42L));
+        doc.add(new NumericDocValuesField("f2", 42L));
+      } else if (i == 100) {
+        doc.add(new StringField("f1", "foo", Store.NO));
+        doc.add(new LongPoint("f2", 2L));
+        doc.add(new NumericDocValuesField("f2", 2L));
+      } else {
+        doc.add(new StringField("f1", "bar", Store.NO));
+        doc.add(new LongPoint("f2", 2L));
+        doc.add(new NumericDocValuesField("f2", 2L));
+      }
+      w.addDocument(doc);
+    }
+    w.forceMerge(1);
+    IndexReader reader = DirectoryReader.open(w);
+    IndexSearcher searcher = newSearcher(reader);
+    searcher.setQueryCache(null);
+
+    // The term query is more selective, so the IndexOrDocValuesQuery should use doc values
+    final Query q1 = new BooleanQuery.Builder()
+        .add(new TermQuery(new Term("f1", "foo")), Occur.MUST)
+        .add(new IndexOrDocValuesQuery(LongPoint.newExactQuery("f2", 2), new DocValuesNumbersQuery("f2", 2L)), Occur.MUST)
+        .build();
+
+    final Weight w1 = searcher.createNormalizedWeight(q1, random().nextBoolean());
+    final Scorer s1 = w1.scorer(reader.leaves().get(0));
+    assertNotNull(s1.twoPhaseIterator()); // means we use doc values
+
+    // The term query is less selective, so the IndexOrDocValuesQuery should use points
+    final Query q2 = new BooleanQuery.Builder()
+        .add(new TermQuery(new Term("f1", "bar")), Occur.MUST)
+        .add(new IndexOrDocValuesQuery(LongPoint.newExactQuery("f2", 42), new DocValuesNumbersQuery("f2", 42L)), Occur.MUST)
+        .build();
+
+    final Weight w2 = searcher.createNormalizedWeight(q2, random().nextBoolean());
+    final Scorer s2 = w2.scorer(reader.leaves().get(0));
+    assertNull(s2.twoPhaseIterator()); // means we use points
+
+    reader.close();
+    w.close();
+    dir.close();
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/aba65396/lucene/test-framework/src/java/org/apache/lucene/codecs/asserting/AssertingPointsFormat.java
----------------------------------------------------------------------
diff --git a/lucene/test-framework/src/java/org/apache/lucene/codecs/asserting/AssertingPointsFormat.java b/lucene/test-framework/src/java/org/apache/lucene/codecs/asserting/AssertingPointsFormat.java
index c6f5485..c397129 100644
--- a/lucene/test-framework/src/java/org/apache/lucene/codecs/asserting/AssertingPointsFormat.java
+++ b/lucene/test-framework/src/java/org/apache/lucene/codecs/asserting/AssertingPointsFormat.java
@@ -175,6 +175,13 @@ public final class AssertingPointsFormat extends PointsFormat {
     }
 
     @Override
+    public long estimatePointCount(String fieldName, IntersectVisitor visitor) {
+      final long value = in.estimatePointCount(fieldName, visitor);
+      assert value >= 0;
+      return value;
+    }
+
+    @Override
     public long ramBytesUsed() {
       long v = in.ramBytesUsed();
       assert v >= 0;

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/aba65396/lucene/test-framework/src/java/org/apache/lucene/codecs/cranky/CrankyPointsFormat.java
----------------------------------------------------------------------
diff --git a/lucene/test-framework/src/java/org/apache/lucene/codecs/cranky/CrankyPointsFormat.java b/lucene/test-framework/src/java/org/apache/lucene/codecs/cranky/CrankyPointsFormat.java
index fd2260b..6491b40 100644
--- a/lucene/test-framework/src/java/org/apache/lucene/codecs/cranky/CrankyPointsFormat.java
+++ b/lucene/test-framework/src/java/org/apache/lucene/codecs/cranky/CrankyPointsFormat.java
@@ -125,6 +125,11 @@ class CrankyPointsFormat extends PointsFormat {
     }
 
     @Override
+    public long estimatePointCount(String fieldName, IntersectVisitor visitor) {
+      return delegate.estimatePointCount(fieldName, visitor);
+    }
+
+    @Override
     public byte[] getMinPackedValue(String fieldName) throws IOException {
       if (random.nextInt(100) == 0) {
         throw new IOException("Fake IOException");

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/aba65396/lucene/test-framework/src/java/org/apache/lucene/search/AssertingWeight.java
----------------------------------------------------------------------
diff --git a/lucene/test-framework/src/java/org/apache/lucene/search/AssertingWeight.java b/lucene/test-framework/src/java/org/apache/lucene/search/AssertingWeight.java
index 75529df..7b6727d 100644
--- a/lucene/test-framework/src/java/org/apache/lucene/search/AssertingWeight.java
+++ b/lucene/test-framework/src/java/org/apache/lucene/search/AssertingWeight.java
@@ -33,9 +33,45 @@ class AssertingWeight extends FilterWeight {
 
   @Override
   public Scorer scorer(LeafReaderContext context) throws IOException {
-    final Scorer inScorer = in.scorer(context);
-    assert inScorer == null || inScorer.docID() == -1;
-    return AssertingScorer.wrap(new Random(random.nextLong()), inScorer, needsScores);
+    if (random.nextBoolean()) {
+      final Scorer inScorer = in.scorer(context);
+      assert inScorer == null || inScorer.docID() == -1;
+      return AssertingScorer.wrap(new Random(random.nextLong()), inScorer, needsScores);
+    } else {
+      final ScorerSupplier scorerSupplier = scorerSupplier(context);
+      if (scorerSupplier == null) {
+        return null;
+      }
+      if (random.nextBoolean()) {
+        // Evil: make sure computing the cost has no side effects
+        scorerSupplier.cost();
+      }
+      return scorerSupplier.get(false);
+    }
+  }
+
+  @Override
+  public ScorerSupplier scorerSupplier(LeafReaderContext context) throws IOException {
+    final ScorerSupplier inScorerSupplier = in.scorerSupplier(context);
+    if (inScorerSupplier == null) {
+      return null;
+    }
+    return new ScorerSupplier() {
+      private boolean getCalled = false;
+      @Override
+      public Scorer get(boolean randomAccess) throws IOException {
+        assert getCalled == false;
+        getCalled = true;
+        return AssertingScorer.wrap(new Random(random.nextLong()), inScorerSupplier.get(randomAccess), needsScores);
+      }
+
+      @Override
+      public long cost() {
+        final long cost = inScorerSupplier.cost();
+        assert cost >= 0;
+        return cost;
+      }
+    };
   }
 
   @Override