You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@lucene.apache.org by mi...@apache.org on 2021/01/29 19:47:43 UTC

[lucene-solr] branch master updated: LUCENE-9537: Add initial Indri search engine functionality to Lucene

This is an automated email from the ASF dual-hosted git repository.

mikemccand pushed a commit to branch master
in repository https://gitbox.apache.org/repos/asf/lucene-solr.git


The following commit(s) were added to refs/heads/master by this push:
     new 9cc5c9b  LUCENE-9537: Add initial Indri search engine functionality to Lucene
9cc5c9b is described below

commit 9cc5c9b7983cf181280e18819f8516dc16ccf1ef
Author: cammiemw <ca...@gmail.com>
AuthorDate: Fri Jan 29 14:47:24 2021 -0500

    LUCENE-9537: Add initial Indri search engine functionality to Lucene
---
 lucene/CHANGES.txt                                 |  13 +-
 .../org/apache/lucene/search/IndexSearcher.java    |  10 +-
 .../org/apache/lucene/search/IndriAndQuery.java    |  35 +++++
 .../org/apache/lucene/search/IndriAndScorer.java   |  71 +++++++++
 .../org/apache/lucene/search/IndriAndWeight.java   | 123 +++++++++++++++
 .../lucene/search/IndriDisjunctionScorer.java      |  77 ++++++++++
 .../java/org/apache/lucene/search/IndriQuery.java  |  97 ++++++++++++
 .../java/org/apache/lucene/search/IndriScorer.java |  51 +++++++
 .../java/org/apache/lucene/search/Scorable.java    |  15 ++
 .../java/org/apache/lucene/search/TermScorer.java  |   5 +
 .../similarities/IndriDirichletSimilarity.java     | 112 ++++++++++++++
 .../apache/lucene/search/TestIndriAndQuery.java    | 165 +++++++++++++++++++++
 .../similarities/TestIndriDirichletSimilarity.java |  49 ++++++
 .../similarities/BaseSimilarityTestCase.java       |  16 +-
 .../org/apache/solr/ltr/feature/TestFeature.java   |   3 +
 15 files changed, 832 insertions(+), 10 deletions(-)

diff --git a/lucene/CHANGES.txt b/lucene/CHANGES.txt
index 3a36066..44aec5a 100644
--- a/lucene/CHANGES.txt
+++ b/lucene/CHANGES.txt
@@ -1,4 +1,4 @@
-Lucene Change Log
+Lucene Change Log
 
 For more information on past and future Lucene versions, please see:
 http://s.apache.org/luceneversions
@@ -84,6 +84,17 @@ API Changes
 
 * LUCENE-9646: Set BM25Similarity discountOverlaps via the constructor (Patrick Marty via Bruno Roustant)
 
+* LUCENE-9537:  Added smoothingScore method and default implementation to 
+  Scorable abstract class.  The smoothing score allows scorers to calculate a 
+  score for a document where the search term or subquery is not present.  The 
+  smoothing score acts like an idf so that documents that do not have terms or 
+  subqueries that are more frequent in the index are not penalized as much as 
+  documents that do not have less frequent terms or subqueries and prevents 
+  scores which are the product or terms or subqueries from going to zero. Added 
+  the implementation of the Indri AND and the IndriDirichletSimilarity from the 
+  academic Indri search engine: http://www.lemurproject.org/indri.php. 
+  (Cameron VandenBerg)   
+
 Improvements
 
 * LUCENE-9687: Hunspell support improvements: add SpellChecker API, support default encoding and
diff --git a/lucene/core/src/java/org/apache/lucene/search/IndexSearcher.java b/lucene/core/src/java/org/apache/lucene/search/IndexSearcher.java
index e44debb..61fe790 100644
--- a/lucene/core/src/java/org/apache/lucene/search/IndexSearcher.java
+++ b/lucene/core/src/java/org/apache/lucene/search/IndexSearcher.java
@@ -184,8 +184,8 @@ public class IndexSearcher {
   /**
    * Runs searches for each segment separately, using the provided Executor. NOTE: if you are using
    * {@link NIOFSDirectory}, do not use the shutdownNow method of ExecutorService as this uses
-   * Thread.interrupt under-the-hood which can silently close file descriptors (see <a
-   * href="https://issues.apache.org/jira/browse/LUCENE-2239">LUCENE-2239</a>).
+   * Thread.interrupt under-the-hood which can silently close file descriptors (see <a href=
+   * "https://issues.apache.org/jira/browse/LUCENE-2239">LUCENE-2239</a>).
    *
    * @lucene.experimental
    */
@@ -199,8 +199,8 @@ public class IndexSearcher {
    * <p>Given a non-<code>null</code> {@link Executor} this method runs searches for each segment
    * separately, using the provided Executor. NOTE: if you are using {@link NIOFSDirectory}, do not
    * use the shutdownNow method of ExecutorService as this uses Thread.interrupt under-the-hood
-   * which can silently close file descriptors (see <a
-   * href="https://issues.apache.org/jira/browse/LUCENE-2239">LUCENE-2239</a>).
+   * which can silently close file descriptors (see <a href=
+   * "https://issues.apache.org/jira/browse/LUCENE-2239">LUCENE-2239</a>).
    *
    * @see IndexReaderContext
    * @see IndexReader#getContext()
@@ -729,7 +729,7 @@ public class IndexSearcher {
       throws IOException {
 
     // TODO: should we make this
-    // threaded...?  the Collector could be sync'd?
+    // threaded...? the Collector could be sync'd?
     // always use single thread:
     for (LeafReaderContext ctx : leaves) { // search each subreader
       final LeafCollector leafCollector;
diff --git a/lucene/core/src/java/org/apache/lucene/search/IndriAndQuery.java b/lucene/core/src/java/org/apache/lucene/search/IndriAndQuery.java
new file mode 100644
index 0000000..08ca1f3
--- /dev/null
+++ b/lucene/core/src/java/org/apache/lucene/search/IndriAndQuery.java
@@ -0,0 +1,35 @@
+/*
+ * 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.List;
+
+/** A Query that matches documents matching combinations of subqueries. */
+public class IndriAndQuery extends IndriQuery {
+
+  public IndriAndQuery(List<BooleanClause> clauses) {
+    super(clauses);
+  }
+
+  @Override
+  public Weight createWeight(IndexSearcher searcher, ScoreMode scoreMode, float boost)
+      throws IOException {
+    IndriAndQuery query = this;
+    return new IndriAndWeight(query, searcher, ScoreMode.TOP_SCORES, boost);
+  }
+}
diff --git a/lucene/core/src/java/org/apache/lucene/search/IndriAndScorer.java b/lucene/core/src/java/org/apache/lucene/search/IndriAndScorer.java
new file mode 100644
index 0000000..28c33db
--- /dev/null
+++ b/lucene/core/src/java/org/apache/lucene/search/IndriAndScorer.java
@@ -0,0 +1,71 @@
+/*
+ * 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.List;
+
+/**
+ * Combines scores of subscorers. If a subscorer does not contain the docId, a smoothing score is
+ * calculated for that document/subscorer combination.
+ */
+public class IndriAndScorer extends IndriDisjunctionScorer {
+
+  protected IndriAndScorer(Weight weight, List<Scorer> subScorers, ScoreMode scoreMode, float boost)
+      throws IOException {
+    super(weight, subScorers, scoreMode, boost);
+  }
+
+  @Override
+  public float score(List<Scorer> subScorers) throws IOException {
+    int docId = this.docID();
+    return scoreDoc(subScorers, docId);
+  }
+
+  @Override
+  public float smoothingScore(List<Scorer> subScorers, int docId) throws IOException {
+    return scoreDoc(subScorers, docId);
+  }
+
+  private float scoreDoc(List<Scorer> subScorers, int docId) throws IOException {
+    double score = 0;
+    double boostSum = 0.0;
+    for (Scorer scorer : subScorers) {
+      if (scorer instanceof IndriScorer) {
+        IndriScorer indriScorer = (IndriScorer) scorer;
+        int scorerDocId = indriScorer.docID();
+        // If the query exists in the document, score the document
+        // Otherwise, compute a smoothing score, which acts like an idf
+        // for subqueries/terms
+        double tempScore = 0;
+        if (docId == scorerDocId) {
+          tempScore = indriScorer.score();
+        } else {
+          tempScore = indriScorer.smoothingScore(docId);
+        }
+        tempScore *= indriScorer.getBoost();
+        score += tempScore;
+        boostSum += indriScorer.getBoost();
+      }
+    }
+    if (boostSum == 0) {
+      return 0;
+    } else {
+      return (float) (score / boostSum);
+    }
+  }
+}
diff --git a/lucene/core/src/java/org/apache/lucene/search/IndriAndWeight.java b/lucene/core/src/java/org/apache/lucene/search/IndriAndWeight.java
new file mode 100644
index 0000000..ee75a9c
--- /dev/null
+++ b/lucene/core/src/java/org/apache/lucene/search/IndriAndWeight.java
@@ -0,0 +1,123 @@
+/*
+ * 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.Iterator;
+import java.util.List;
+import org.apache.lucene.index.LeafReaderContext;
+
+/** The Weight for IndriAndQuery, used to normalize, score and explain these queries. */
+public class IndriAndWeight extends Weight {
+
+  private final IndriAndQuery query;
+  private final ArrayList<Weight> weights;
+  private final ScoreMode scoreMode;
+  private final float boost;
+
+  public IndriAndWeight(
+      IndriAndQuery query, IndexSearcher searcher, ScoreMode scoreMode, float boost)
+      throws IOException {
+    super(query);
+    this.query = query;
+    this.boost = boost;
+    this.scoreMode = scoreMode;
+    weights = new ArrayList<>();
+    for (BooleanClause c : query) {
+      Weight w = searcher.createWeight(c.getQuery(), scoreMode, 1.0f);
+      weights.add(w);
+    }
+  }
+
+  private Scorer getScorer(LeafReaderContext context) throws IOException {
+    List<Scorer> subScorers = new ArrayList<>();
+
+    for (Weight w : weights) {
+      Scorer scorer = w.scorer(context);
+      if (scorer != null) {
+        subScorers.add(scorer);
+      }
+    }
+
+    if (subScorers.isEmpty()) {
+      return null;
+    }
+    Scorer scorer = subScorers.get(0);
+    if (subScorers.size() > 1) {
+      scorer = new IndriAndScorer(this, subScorers, scoreMode, boost);
+    }
+    return scorer;
+  }
+
+  @Override
+  public Scorer scorer(LeafReaderContext context) throws IOException {
+    return getScorer(context);
+  }
+
+  @Override
+  public BulkScorer bulkScorer(LeafReaderContext context) throws IOException {
+    Scorer scorer = getScorer(context);
+    if (scorer != null) {
+      BulkScorer bulkScorer = new DefaultBulkScorer(scorer);
+      return bulkScorer;
+    }
+    return null;
+  }
+
+  @Override
+  public boolean isCacheable(LeafReaderContext ctx) {
+    for (Weight w : weights) {
+      if (w.isCacheable(ctx) == false) return false;
+    }
+    return true;
+  }
+
+  @Override
+  public Explanation explain(LeafReaderContext context, int doc) throws IOException {
+    List<Explanation> subs = new ArrayList<>();
+    boolean fail = false;
+    Iterator<BooleanClause> cIter = query.iterator();
+    for (Iterator<Weight> wIter = weights.iterator(); wIter.hasNext(); ) {
+      Weight w = wIter.next();
+      BooleanClause c = cIter.next();
+      Explanation e = w.explain(context, doc);
+      if (e.isMatch()) {
+        subs.add(e);
+      } else if (c.isRequired()) {
+        subs.add(
+            Explanation.noMatch(
+                "no match on required clause (" + c.getQuery().toString() + ")", e));
+        fail = true;
+      }
+    }
+    if (fail) {
+      return Explanation.noMatch(
+          "Failure to meet condition(s) of required/prohibited clause(s)", subs);
+    } else {
+      Scorer scorer = scorer(context);
+      if (scorer != null) {
+        int advanced = scorer.iterator().advance(doc);
+        assert advanced == doc;
+        return Explanation.match(scorer.score(), "sum of:", subs);
+      } else {
+        return Explanation.noMatch(
+            "Failure to meet condition(s) of required/prohibited clause(s)", subs);
+      }
+    }
+  }
+}
diff --git a/lucene/core/src/java/org/apache/lucene/search/IndriDisjunctionScorer.java b/lucene/core/src/java/org/apache/lucene/search/IndriDisjunctionScorer.java
new file mode 100644
index 0000000..1cfab60
--- /dev/null
+++ b/lucene/core/src/java/org/apache/lucene/search/IndriDisjunctionScorer.java
@@ -0,0 +1,77 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.lucene.search;
+
+import java.io.IOException;
+import java.util.List;
+
+/**
+ * The Indri implemenation of a disjunction scorer which stores the subscorers for the child
+ * queries. The score and smoothingScore methods use the list of all subscorers and not just the
+ * matches so that a smoothingScore can be calculated if there is not an exact match.
+ */
+public abstract class IndriDisjunctionScorer extends IndriScorer {
+
+  private final List<Scorer> subScorersList;
+  private final DisiPriorityQueue subScorers;
+  private final DocIdSetIterator approximation;
+
+  protected IndriDisjunctionScorer(
+      Weight weight, List<Scorer> subScorersList, ScoreMode scoreMode, float boost) {
+    super(weight, boost);
+    this.subScorersList = subScorersList;
+    this.subScorers = new DisiPriorityQueue(subScorersList.size());
+    for (Scorer scorer : subScorersList) {
+      final DisiWrapper w = new DisiWrapper(scorer);
+      this.subScorers.add(w);
+    }
+    this.approximation = new DisjunctionDISIApproximation(this.subScorers);
+  }
+
+  @Override
+  public DocIdSetIterator iterator() {
+    return approximation;
+  }
+
+  @Override
+  public float getMaxScore(int upTo) throws IOException {
+    return 0;
+  }
+
+  public List<Scorer> getSubMatches() throws IOException {
+    return subScorersList;
+  }
+
+  abstract float score(List<Scorer> subScorers) throws IOException;
+
+  public abstract float smoothingScore(List<Scorer> subScorers, int docId) throws IOException;
+
+  @Override
+  public float score() throws IOException {
+    return score(getSubMatches());
+  }
+
+  @Override
+  public float smoothingScore(int docId) throws IOException {
+    return smoothingScore(getSubMatches(), docId);
+  }
+
+  @Override
+  public int docID() {
+    return subScorers.top().doc;
+  }
+}
diff --git a/lucene/core/src/java/org/apache/lucene/search/IndriQuery.java b/lucene/core/src/java/org/apache/lucene/search/IndriQuery.java
new file mode 100644
index 0000000..7fed297
--- /dev/null
+++ b/lucene/core/src/java/org/apache/lucene/search/IndriQuery.java
@@ -0,0 +1,97 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.lucene.search;
+
+import java.io.IOException;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Objects;
+
+/**
+ * A Basic abstract query that all IndriQueries can extend to implement toString, equals,
+ * getClauses, and iterator.
+ */
+public abstract class IndriQuery extends Query implements Iterable<BooleanClause> {
+
+  private List<BooleanClause> clauses;
+
+  public IndriQuery(List<BooleanClause> clauses) {
+    this.clauses = clauses;
+  }
+
+  @Override
+  public abstract Weight createWeight(IndexSearcher searcher, ScoreMode scoreMode, float boost)
+      throws IOException;
+
+  @Override
+  public String toString(String field) {
+    StringBuilder buffer = new StringBuilder();
+
+    int i = 0;
+    for (BooleanClause c : this) {
+      buffer.append(c.getOccur().toString());
+
+      Query subQuery = c.getQuery();
+      if (subQuery instanceof BooleanQuery) { // wrap sub-bools in parens
+        buffer.append("(");
+        buffer.append(subQuery.toString(field));
+        buffer.append(")");
+      } else {
+        buffer.append(subQuery.toString(field));
+      }
+
+      if (i != clauses.size() - 1) {
+        buffer.append(" ");
+      }
+      i += 1;
+    }
+
+    return buffer.toString();
+  }
+
+  @Override
+  public boolean equals(Object o) {
+    return sameClassAs(o) && equalsTo(getClass().cast(o));
+  }
+
+  @Override
+  public void visit(QueryVisitor visitor) {
+    visitor.visitLeaf(this);
+  }
+
+  private boolean equalsTo(IndriQuery other) {
+    return clauses.equals(other.clauses);
+  }
+
+  @Override
+  public int hashCode() {
+    int hashCode = Objects.hash(clauses);
+    if (hashCode == 0) {
+      hashCode = 1;
+    }
+    return hashCode;
+  }
+
+  @Override
+  public Iterator<BooleanClause> iterator() {
+    return clauses.iterator();
+  }
+
+  public List<BooleanClause> getClauses() {
+    return this.clauses;
+  }
+}
diff --git a/lucene/core/src/java/org/apache/lucene/search/IndriScorer.java b/lucene/core/src/java/org/apache/lucene/search/IndriScorer.java
new file mode 100644
index 0000000..d152530
--- /dev/null
+++ b/lucene/core/src/java/org/apache/lucene/search/IndriScorer.java
@@ -0,0 +1,51 @@
+/*
+ * 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;
+
+/**
+ * The Indri parent scorer that stores the boost so that IndriScorers can use the boost outside of
+ * the term.
+ */
+public abstract class IndriScorer extends Scorer {
+
+  private float boost;
+
+  protected IndriScorer(Weight weight, float boost) {
+    super(weight);
+    this.boost = boost;
+  }
+
+  @Override
+  public abstract DocIdSetIterator iterator();
+
+  @Override
+  public abstract float getMaxScore(int upTo) throws IOException;
+
+  @Override
+  public abstract float score() throws IOException;
+
+  public abstract float smoothingScore(int docId) throws IOException;
+
+  @Override
+  public abstract int docID();
+
+  public float getBoost() {
+    return this.boost;
+  }
+}
diff --git a/lucene/core/src/java/org/apache/lucene/search/Scorable.java b/lucene/core/src/java/org/apache/lucene/search/Scorable.java
index 4a509d4..801dcc8 100644
--- a/lucene/core/src/java/org/apache/lucene/search/Scorable.java
+++ b/lucene/core/src/java/org/apache/lucene/search/Scorable.java
@@ -27,6 +27,21 @@ public abstract class Scorable {
   /** Returns the score of the current document matching the query. */
   public abstract float score() throws IOException;
 
+  /**
+   * Returns the smoothing score of the current document matching the query. This score is used when
+   * the query/term does not appear in the document, and behaves like an idf. The smoothing score is
+   * particularly important when the Scorer returns a product of probabilities so that the document
+   * score does not go to zero when one probability is zero. This can return 0 or a smoothing score.
+   *
+   * <p>Smoothing scores are described in many papers, including: Metzler, D. and Croft, W. B. ,
+   * "Combining the Language Model and Inference Network Approaches to Retrieval," Information
+   * Processing and Management Special Issue on Bayesian Networks and Information Retrieval, 40(5),
+   * pp.735-750.
+   */
+  public float smoothingScore(int docId) throws IOException {
+    return 0f;
+  }
+
   /** Returns the doc ID that is currently being scored. */
   public abstract int docID();
 
diff --git a/lucene/core/src/java/org/apache/lucene/search/TermScorer.java b/lucene/core/src/java/org/apache/lucene/search/TermScorer.java
index e64a0f9..51b8068 100644
--- a/lucene/core/src/java/org/apache/lucene/search/TermScorer.java
+++ b/lucene/core/src/java/org/apache/lucene/search/TermScorer.java
@@ -76,6 +76,11 @@ public final class TermScorer extends Scorer {
   }
 
   @Override
+  public float smoothingScore(int docId) throws IOException {
+    return docScorer.score(docId, 0);
+  }
+
+  @Override
   public int advanceShallow(int target) throws IOException {
     return impactsDisi.advanceShallow(target);
   }
diff --git a/lucene/core/src/java/org/apache/lucene/search/similarities/IndriDirichletSimilarity.java b/lucene/core/src/java/org/apache/lucene/search/similarities/IndriDirichletSimilarity.java
new file mode 100644
index 0000000..e494bc9
--- /dev/null
+++ b/lucene/core/src/java/org/apache/lucene/search/similarities/IndriDirichletSimilarity.java
@@ -0,0 +1,112 @@
+/*
+ * 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.similarities;
+
+import java.util.List;
+import java.util.Locale;
+import org.apache.lucene.search.Explanation;
+
+/**
+ * Bayesian smoothing using Dirichlet priors as implemented in the Indri Search engine
+ * (http://www.lemurproject.org/indri.php). Indri Dirichelet Smoothing!
+ *
+ * <pre class="prettyprint">
+ * tf_E + mu*P(t|D) P(t|E)= documentLength + documentMu
+ * mu*P(t|C) + tf_D where P(t|D)= doclen + mu
+ * </pre>
+ *
+ * <p>A larger value for mu, produces more smoothing. Smoothing is most important for short
+ * documents where the probabilities are more granular.
+ */
+public class IndriDirichletSimilarity extends LMSimilarity {
+
+  /** The &mu; parameter. */
+  private final float mu;
+
+  /** Instantiates the similarity with the provided &mu; parameter. */
+  public IndriDirichletSimilarity(CollectionModel collectionModel, float mu) {
+    super(collectionModel);
+    this.mu = mu;
+  }
+
+  /** Instantiates the similarity with the provided &mu; parameter. */
+  public IndriDirichletSimilarity(float mu) {
+    this.mu = mu;
+  }
+
+  /** Instantiates the similarity with the default &mu; value of 2000. */
+  public IndriDirichletSimilarity(CollectionModel collectionModel) {
+    this(collectionModel, 2000);
+  }
+
+  /** Instantiates the similarity with the default &mu; value of 2000. */
+  public IndriDirichletSimilarity() {
+    this(new IndriCollectionModel(), 2000);
+  }
+
+  @Override
+  protected double score(BasicStats stats, double freq, double docLen) {
+    double collectionProbability = ((LMStats) stats).getCollectionProbability();
+    double score = (freq + (mu * collectionProbability)) / (docLen + mu);
+    return (Math.log(score));
+  }
+
+  @Override
+  protected void explain(List<Explanation> subs, BasicStats stats, double freq, double docLen) {
+    if (stats.getBoost() != 1.0f) {
+      subs.add(Explanation.match(stats.getBoost(), "boost"));
+    }
+
+    subs.add(Explanation.match(mu, "mu"));
+    double collectionProbability = ((LMStats) stats).getCollectionProbability();
+    Explanation weightExpl =
+        Explanation.match(
+            (float) Math.log((freq + (mu * collectionProbability)) / (docLen + mu)), "term weight");
+    subs.add(weightExpl);
+    subs.add(Explanation.match((float) Math.log(mu / (docLen + mu)), "document norm"));
+    super.explain(subs, stats, freq, docLen);
+  }
+
+  /** Returns the &mu; parameter. */
+  public float getMu() {
+    return mu;
+  }
+
+  public String getName() {
+    return String.format(Locale.ROOT, "IndriDirichlet(%f)", getMu());
+  }
+
+  /**
+   * Models {@code p(w|C)} as the number of occurrences of the term in the collection, divided by
+   * the total number of tokens {@code + 1}.
+   */
+  public static class IndriCollectionModel implements CollectionModel {
+
+    /** Sole constructor: parameter-free */
+    public IndriCollectionModel() {}
+
+    @Override
+    public double computeProbability(BasicStats stats) {
+      return ((double) stats.getTotalTermFreq()) / ((double) stats.getNumberOfFieldTokens());
+    }
+
+    @Override
+    public String getName() {
+      return null;
+    }
+  }
+}
diff --git a/lucene/core/src/test/org/apache/lucene/search/TestIndriAndQuery.java b/lucene/core/src/test/org/apache/lucene/search/TestIndriAndQuery.java
new file mode 100644
index 0000000..5be257e
--- /dev/null
+++ b/lucene/core/src/test/org/apache/lucene/search/TestIndriAndQuery.java
@@ -0,0 +1,165 @@
+/*
+ * 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.util.Arrays;
+import org.apache.lucene.analysis.MockAnalyzer;
+import org.apache.lucene.analysis.MockTokenFilter;
+import org.apache.lucene.analysis.MockTokenizer;
+import org.apache.lucene.document.Document;
+import org.apache.lucene.document.Field;
+import org.apache.lucene.document.TextField;
+import org.apache.lucene.index.IndexReader;
+import org.apache.lucene.index.RandomIndexWriter;
+import org.apache.lucene.index.Term;
+import org.apache.lucene.search.BooleanClause.Occur;
+import org.apache.lucene.search.similarities.IndriDirichletSimilarity;
+import org.apache.lucene.search.similarities.Similarity;
+import org.apache.lucene.store.Directory;
+import org.apache.lucene.util.LuceneTestCase;
+
+public class TestIndriAndQuery extends LuceneTestCase {
+
+  /** threshold for comparing floats */
+  public static final float SCORE_COMP_THRESH = 0.0000f;
+
+  public Similarity sim = new IndriDirichletSimilarity();
+  public Directory index;
+  public IndexReader r;
+  public IndexSearcher s;
+
+  @Override
+  public void setUp() throws Exception {
+    super.setUp();
+
+    index = newDirectory();
+    RandomIndexWriter writer =
+        new RandomIndexWriter(
+            random(),
+            index,
+            newIndexWriterConfig(
+                    new MockAnalyzer(
+                        random(), MockTokenizer.WHITESPACE, true, MockTokenFilter.ENGLISH_STOPSET))
+                .setSimilarity(sim)
+                .setMergePolicy(newLogMergePolicy()));
+    // Query is "President Washington"
+    {
+      Document d1 = new Document();
+      d1.add(newField("id", "d1", TextField.TYPE_STORED));
+      d1.add(
+          newTextField(
+              "body", "President Washington was the first leader of the US", Field.Store.YES));
+      writer.addDocument(d1);
+    }
+
+    {
+      Document d2 = new Document();
+      d2.add(newField("id", "d2", TextField.TYPE_STORED));
+      d2.add(
+          newTextField(
+              "body",
+              "The president is head of the executive branch of government",
+              Field.Store.YES));
+      writer.addDocument(d2);
+    }
+
+    {
+      Document d3 = new Document();
+      d3.add(newField("id", "d3", TextField.TYPE_STORED));
+      d3.add(
+          newTextField(
+              "body", "George Washington was a general in the Revolutionary War", Field.Store.YES));
+      writer.addDocument(d3);
+    }
+
+    {
+      Document d4 = new Document();
+      d4.add(newField("id", "d4", TextField.TYPE_STORED));
+      d4.add(newTextField("body", "A company or college can have a president", Field.Store.YES));
+      writer.addDocument(d4);
+    }
+
+    writer.forceMerge(1);
+    r = getOnlyLeafReader(writer.getReader());
+    writer.close();
+    s = new IndexSearcher(r);
+    s.setSimilarity(sim);
+  }
+
+  @Override
+  public void tearDown() throws Exception {
+    r.close();
+    index.close();
+    super.tearDown();
+  }
+
+  public void testSimpleQuery1() throws Exception {
+
+    BooleanClause clause1 = new BooleanClause(tq("body", "george"), Occur.SHOULD);
+    BooleanClause clause2 = new BooleanClause(tq("body", "washington"), Occur.SHOULD);
+
+    IndriAndQuery q = new IndriAndQuery(Arrays.asList(clause1, clause2));
+
+    ScoreDoc[] h = s.search(q, 1000).scoreDocs;
+
+    try {
+      assertEquals("2 docs should match " + q.toString(), 2, h.length);
+    } catch (Error e) {
+      printHits("testSimpleEqualScores1", h, s);
+      throw e;
+    }
+  }
+
+  public void testSimpleQuery2() throws Exception {
+
+    BooleanClause clause1 = new BooleanClause(tq("body", "president"), Occur.SHOULD);
+    BooleanClause clause2 = new BooleanClause(tq("body", "washington"), Occur.SHOULD);
+
+    IndriAndQuery q = new IndriAndQuery(Arrays.asList(clause1, clause2));
+
+    ScoreDoc[] h = s.search(q, 1000).scoreDocs;
+
+    try {
+      assertEquals("all docs should match " + q.toString(), 4, h.length);
+    } catch (Error e) {
+      printHits("testSimpleEqualScores1", h, s);
+      throw e;
+    }
+  }
+
+  /** macro */
+  protected Query tq(String f, String t) {
+    return new TermQuery(new Term(f, t));
+  }
+
+  /** macro */
+  protected Query tq(String f, String t, float b) {
+    Query q = tq(f, t);
+    return new BoostQuery(q, b);
+  }
+
+  protected void printHits(String test, ScoreDoc[] h, IndexSearcher searcher) throws Exception {
+
+    System.err.println("------- " + test + " -------");
+
+    for (int i = 0; i < h.length; i++) {
+      Document d = searcher.doc(h[i].doc);
+      float score = h[i].score;
+      System.err.println("#" + i + ": " + score + " - " + d.get("body"));
+    }
+  }
+}
diff --git a/lucene/core/src/test/org/apache/lucene/search/similarities/TestIndriDirichletSimilarity.java b/lucene/core/src/test/org/apache/lucene/search/similarities/TestIndriDirichletSimilarity.java
new file mode 100644
index 0000000..39300d0
--- /dev/null
+++ b/lucene/core/src/test/org/apache/lucene/search/similarities/TestIndriDirichletSimilarity.java
@@ -0,0 +1,49 @@
+/*
+ * 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.similarities;
+
+import java.util.Random;
+
+public class TestIndriDirichletSimilarity extends BaseSimilarityTestCase {
+
+  @Override
+  protected Similarity getSimilarity(Random random) {
+    // smoothing parameter mu, unbounded
+    final float mu;
+    switch (random.nextInt(4)) {
+      case 0:
+        // minimum value
+        mu = 0;
+        break;
+      case 1:
+        // tiny value
+        mu = Float.MIN_VALUE;
+        break;
+      case 2:
+        // maximum value
+        // we just limit the test to "reasonable" mu values but don't enforce
+        // this anywhere.
+        mu = Integer.MAX_VALUE;
+        break;
+      default:
+        // random value
+        mu = Integer.MAX_VALUE * random.nextFloat();
+        break;
+    }
+    return new IndriDirichletSimilarity(mu);
+  }
+}
diff --git a/lucene/test-framework/src/java/org/apache/lucene/search/similarities/BaseSimilarityTestCase.java b/lucene/test-framework/src/java/org/apache/lucene/search/similarities/BaseSimilarityTestCase.java
index 2394306..0354f66 100644
--- a/lucene/test-framework/src/java/org/apache/lucene/search/similarities/BaseSimilarityTestCase.java
+++ b/lucene/test-framework/src/java/org/apache/lucene/search/similarities/BaseSimilarityTestCase.java
@@ -368,7 +368,9 @@ public abstract class BaseSimilarityTestCase extends LuceneTestCase {
       float score = scorer.score(freq, norm);
       // check that score isn't infinite or negative
       assertTrue("infinite/NaN score: " + score, Float.isFinite(score));
-      assertTrue("negative score: " + score, score >= 0);
+      if (!(similarity instanceof IndriDirichletSimilarity)) {
+        assertTrue("negative score: " + score, score >= 0);
+      }
       assertTrue("greater than maxScore: " + score + ">" + maxScore, score <= maxScore);
       // check explanation matches
       Explanation explanation =
@@ -395,7 +397,9 @@ public abstract class BaseSimilarityTestCase extends LuceneTestCase {
       float prevScore = scorer.score(prevFreq, norm);
       // check that score isn't infinite or negative
       assertTrue(Float.isFinite(prevScore));
-      assertTrue(prevScore >= 0);
+      if (!(similarity instanceof IndriDirichletSimilarity)) {
+        assertTrue(prevScore >= 0);
+      }
       // check explanation matches
       Explanation prevExplanation =
           scorer.explain(
@@ -419,7 +423,9 @@ public abstract class BaseSimilarityTestCase extends LuceneTestCase {
         float prevNormScore = scorer.score(freq, norm - 1);
         // check that score isn't infinite or negative
         assertTrue(Float.isFinite(prevNormScore));
-        assertTrue(prevNormScore >= 0);
+        if (!(similarity instanceof IndriDirichletSimilarity)) {
+          assertTrue(prevNormScore >= 0);
+        }
         // check explanation matches
         Explanation prevNormExplanation =
             scorer.explain(
@@ -459,7 +465,9 @@ public abstract class BaseSimilarityTestCase extends LuceneTestCase {
         float prevTermScore = prevTermScorer.score(freq, norm);
         // check that score isn't infinite or negative
         assertTrue(Float.isFinite(prevTermScore));
-        assertTrue(prevTermScore >= 0);
+        if (!(similarity instanceof IndriDirichletSimilarity)) {
+          assertTrue(prevTermScore >= 0);
+        }
         // check explanation matches
         Explanation prevTermExplanation =
             prevTermScorer.explain(
diff --git a/solr/contrib/ltr/src/test/org/apache/solr/ltr/feature/TestFeature.java b/solr/contrib/ltr/src/test/org/apache/solr/ltr/feature/TestFeature.java
index 6c4dc08..e14012b 100644
--- a/solr/contrib/ltr/src/test/org/apache/solr/ltr/feature/TestFeature.java
+++ b/solr/contrib/ltr/src/test/org/apache/solr/ltr/feature/TestFeature.java
@@ -33,6 +33,9 @@ public class TestFeature extends SolrTestCase {
         // the FilterFeatureScorer may simply inherit Scorer's default implementation
         if (scorerClassMethod.getName().equals("twoPhaseIterator")) continue;
 
+        // the FilterFeatureScorer may simply inherit Scorer's default implementation
+        if (scorerClassMethod.getName().equals("smoothingScore")) continue;
+        
         // the FilterFeatureScorer's implementation does not influence its parent Weight
         if (scorerClassMethod.getName().equals("getWeight")) continue;