You are viewing a plain text version of this content. The canonical link for it is here.
Posted to issues@lucene.apache.org by GitBox <gi...@apache.org> on 2021/11/22 19:35:24 UTC

[GitHub] [lucene] cammiemw opened a new pull request #463: LUCENE-10157 - Added IndriOr, IndriWeightedSum, and IndriQueryParser

cammiemw opened a new pull request #463:
URL: https://github.com/apache/lucene/pull/463


   <!--
   _(If you are a project committer then you may remove some/all of the following template.)_
   
   Before creating a pull request, please file an issue in the ASF Jira system for Lucene:
   
   * https://issues.apache.org/jira/projects/LUCENE
   
   You will need to create an account in Jira in order to create an issue.
   
   The title of the PR should reference the Jira issue number in the form:
   
   * LUCENE-####: <short description of problem or changes>
   
   LUCENE must be fully capitalized. A short description helps people scanning pull requests for items they can work on.
   
   Properly referencing the issue in the title ensures that Jira is correctly updated with code review comments and commits. -->
   
   
   # Description
   
   This pull request adds additional functionality from the Indri search engine (https://www.lemurproject.org/indri/) to lucene.  The Indri AND operator was added in lucene release 8.7, and these changes add 2 more operators (the Indri OR and the Indri Weighted Sum) and an Indri Query Parser.
   
   # Solution
   
   New classes were added to support the Indri OR, Indri Weighted Sum, and Indri Query Parser.
   
   # Tests
   
   TestIndriOrQuery, TestIndriWeightedSumQuery, and TestIndriQueryParser
   
   # Checklist
   
   Please review the following and check all that apply:
   
   - [x] I have reviewed the guidelines for [How to Contribute](https://wiki.apache.org/lucene/HowToContribute) and my code conforms to the standards described there to the best of my ability.
   - [x] I have created a Jira issue and added the issue ID to my pull request title.
   - [ ] I have given Lucene maintainers [access](https://help.github.com/en/articles/allowing-changes-to-a-pull-request-branch-created-from-a-fork) to contribute to my PR branch. (optional but recommended)
   - [x] I have developed this patch against the `main` branch.
   - [x] I have run `./gradlew check`.
   - [x] I have added tests for my changes.
   


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@lucene.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



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


[GitHub] [lucene] cpoerschke commented on a change in pull request #463: LUCENE-10157 - Added IndriOr, IndriWeightedSum, and IndriQueryParser

Posted by GitBox <gi...@apache.org>.
cpoerschke commented on a change in pull request #463:
URL: https://github.com/apache/lucene/pull/463#discussion_r780405554



##########
File path: lucene/core/src/java/org/apache/lucene/search/IndriAndWeight.java
##########
@@ -38,8 +38,21 @@ public IndriAndWeight(
     this.boost = boost;
     this.scoreMode = scoreMode;
     weights = new ArrayList<>();
+    // Calculate total boost score so that boost can be normalized
+    float boostSum = 0;
     for (BooleanClause c : query) {
-      Weight w = searcher.createWeight(c.getQuery(), scoreMode, 1.0f);
+      if (c.getQuery() instanceof BoostQuery) {
+        boostSum += ((BoostQuery) c.getQuery()).getBoost();

Review comment:
       subjective: perhaps introduce a local variable to avoid duplicate `c.getQuery()` calls.
   
   ```suggestion
         Query q = c.getQuery();
         if (q instanceof BoostQuery) {
           boostSum += ((BoostQuery) q).getBoost();
   ```

##########
File path: lucene/core/src/java/org/apache/lucene/search/IndriOrWeight.java
##########
@@ -0,0 +1,135 @@
+/*
+ * 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 IndriOrWeight extends Weight {
+
+  private final IndriQuery query;
+  private final float boost;
+  private final ArrayList<Weight> weights;
+  private final ScoreMode scoreMode;
+
+  public IndriOrWeight(IndriOrQuery query, IndexSearcher searcher, ScoreMode scoreMode, float boost)
+      throws IOException {
+    super(query);
+    this.query = query;
+    this.boost = boost;
+    this.scoreMode = scoreMode;
+    weights = new ArrayList<>();

Review comment:
       subjective/style: `this.weights` here and at line 55 even though (unlike for the other members) it's not necessary
   ```suggestion
       this.weights = new ArrayList<>();
   ```

##########
File path: lucene/core/src/java/org/apache/lucene/search/IndriOrWeight.java
##########
@@ -0,0 +1,135 @@
+/*
+ * 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. */

Review comment:
       ```suggestion
   /** The Weight for IndriOrQuery, used to normalize, score and explain these queries. */
   ```

##########
File path: lucene/core/src/java/org/apache/lucene/search/IndriOrScorer.java
##########
@@ -0,0 +1,61 @@
+/*
+ * 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 IndriOrScorer extends IndriDisjunctionScorer {
+
+  protected IndriOrScorer(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 {
+    float score = 1;
+    for (Scorer scorer : subScorers) {
+      int scorerDocId = scorer.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 = (1 - Math.exp(scorer.score()));
+      } else {
+        tempScore = (1 - Math.exp(scorer.smoothingScore(docId)));
+      }
+      score *= tempScore;

Review comment:
       Curious if this is the bot here trying to say that `score` and `tempScore` should be `float` instead? Noting that elsewhere above in the PR some locals are `float` rather than `double`.

##########
File path: lucene/core/src/java/org/apache/lucene/search/IndriWeightedSumWeight.java
##########
@@ -0,0 +1,135 @@
+/*
+ * 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 IndriWeightedSumWeight extends Weight {
+  private final IndriWeightedSumQuery query;
+
+  private final ArrayList<Weight> weights;
+  private final ScoreMode scoreMode;
+  private final float boost;
+
+  public IndriWeightedSumWeight(
+      IndriWeightedSumQuery query, IndexSearcher searcher, ScoreMode scoreMode, float boost)
+      throws IOException {
+    super(query);
+    this.query = query;
+    this.boost = boost;
+    this.scoreMode = scoreMode;
+    weights = new ArrayList<>();
+    // Calculate total boost score so that boost can be normalized
+    float boostSum = 0;
+    for (BooleanClause c : query) {
+      if (c.getQuery() instanceof BoostQuery) {
+        boostSum += ((BoostQuery) c.getQuery()).getBoost();
+      } else {
+        boostSum++;
+      }
+    }
+    for (BooleanClause c : query) {
+      float subBoost = 1.0f;
+      if (c.getQuery() instanceof BoostQuery) {
+        subBoost = ((BoostQuery) c.getQuery()).getBoost() / boostSum;
+      }
+      Weight w = searcher.createWeight(c.getQuery(), scoreMode, subBoost);
+      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 IndriWeightedSumScorer(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 {

Review comment:
       It seems the three classes for and/or/sum share some common code e.g. in `explain` here. Have you considered having a common `IndriWeight` class which they can extend to reduce common code repetition?




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@lucene.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



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


[GitHub] [lucene] sonatype-lift[bot] commented on a change in pull request #463: LUCENE-10157 - Added IndriOr, IndriWeightedSum, and IndriQueryParser

Posted by GitBox <gi...@apache.org>.
sonatype-lift[bot] commented on a change in pull request #463:
URL: https://github.com/apache/lucene/pull/463#discussion_r768196992



##########
File path: lucene/queryparser/src/java/org/apache/lucene/queryparser/indri/IndriQueryParser.java
##########
@@ -0,0 +1,357 @@
+/*
+ * 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.queryparser.indri;
+
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.List;
+import java.util.Locale;
+import org.apache.lucene.analysis.Analyzer;
+import org.apache.lucene.index.Term;
+import org.apache.lucene.search.BooleanClause;
+import org.apache.lucene.search.BooleanClause.Occur;
+import org.apache.lucene.search.BoostQuery;
+import org.apache.lucene.search.IndriAndQuery;
+import org.apache.lucene.search.IndriOrQuery;
+import org.apache.lucene.search.IndriWeightedSumQuery;
+import org.apache.lucene.search.Query;
+import org.apache.lucene.search.TermQuery;
+import org.apache.lucene.util.BytesRef;
+
+/**
+ * IndriQueryParser is used to parse human readable query syntax and create Indri queries.
+ *
+ * <p><b>Query Operators</b>
+ *
+ * <ul>
+ *   <li>'{@code #and(t1 t2)}' specifies {@code IndriAnd} operation: <code>#and(term1 term2)</code>
+ *   <li>'{@code #or(t1 t2)}' specifies {@code IndriOr} operation: <code>#or(term1 term2)</code>
+ *   <li>'{@code #wsum(boost1 token1 boost2 token2)}' specifies {@code IndriOr} operation: <code>
+ *       #wsum(1.0 term1 2.0 term2)</code>
+ * </ul>
+ *
+ * <p>The default operator is {@code IndriAnd} if no other operator is specified. For example, the
+ * following will {@code IndriAnd} {@code token1} and {@code token2} together: <code>token1 token2
+ * </code>
+ */
+public class IndriQueryParser {
+
+  private static final String AND = "and";
+  private static final String OR = "or";
+  private static final String WAND = "wand";
+  private static final String WEIGHT = "weight";
+  private static final String WSUM = "wsum";
+
+  private final Analyzer analyzer;
+  private String field;
+
+  public IndriQueryParser(Analyzer analyzer, String field) throws IOException {
+    this.analyzer = analyzer;
+    this.field = field;
+  }
+
+  /**
+   * Count the number of occurrences of character c in string s.
+   *
+   * @param c A character.
+   * @param s A string.
+   */
+  private static int countChars(String s, char c) {
+    int count = 0;
+
+    for (int i = 0; i < s.length(); i++) {
+      if (s.charAt(i) == c) {
+        count++;
+      }
+    }
+    return count;
+  }
+
+  /**
+   * Get the index of the right parenenthesis that balances the left-most parenthesis. Return -1 if
+   * it doesn't exist.
+   *
+   * @param s A string containing a query.
+   */
+  private static int indexOfBalencingParen(String s) {
+    int depth = 0;
+
+    for (int i = 0; i < s.length(); i++) {
+      if (s.charAt(i) == '(') {
+        depth++;
+      } else if (s.charAt(i) == ')') {
+        depth--;
+
+        if (depth == 0) {
+          return i;
+        }
+      }
+    }
+    return -1;
+  }
+
+  private QueryParserOperatorQuery createOperator(String operatorName, Occur occur) {
+    QueryParserOperatorQuery operatorQuery = new QueryParserOperatorQuery();
+
+    int operatorDistance = 0;
+    String operatorNameLowerCase = (new String(operatorName)).toLowerCase(Locale.ROOT);
+    operatorNameLowerCase = operatorNameLowerCase.replace("#", "");
+    operatorNameLowerCase = operatorNameLowerCase.replace("~", "");
+
+    operatorQuery.setOperator(operatorNameLowerCase);
+    operatorQuery.setField(field);
+    operatorQuery.setDistance(operatorDistance);
+    operatorQuery.setOccur(occur);
+
+    return operatorQuery;
+  }
+
+  private class PopWeight {

Review comment:
       *ClassCanBeStatic:*  Inner class is non-static but does not reference enclosing class [(details)](https://errorprone.info/bugpattern/ClassCanBeStatic)
   (at-me [in a reply](https://help.sonatype.com/lift/talking-to-lift) with `help` or `ignore`)

##########
File path: lucene/core/src/java/org/apache/lucene/search/IndriAndScorer.java
##########
@@ -42,30 +42,20 @@ public float smoothingScore(List<Scorer> subScorers, int docId) throws IOExcepti
   }
 
   private float scoreDoc(List<Scorer> subScorers, int docId) throws IOException {
-    double score = 0;
-    double boostSum = 0.0;
+    float score = 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();
+      int scorerDocId = scorer.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 = scorer.score();
+      } else {
+        tempScore = scorer.smoothingScore(docId);
       }
+      score += tempScore;

Review comment:
       *NarrowingCompoundAssignment:*  Compound assignments from double to float hide lossy casts [(details)](https://errorprone.info/bugpattern/NarrowingCompoundAssignment)
   (at-me [in a reply](https://help.sonatype.com/lift/talking-to-lift) with `help` or `ignore`)

##########
File path: lucene/core/src/java/org/apache/lucene/search/IndriOrScorer.java
##########
@@ -0,0 +1,61 @@
+/*
+ * 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 IndriOrScorer extends IndriDisjunctionScorer {
+
+  protected IndriOrScorer(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 {
+    float score = 1;
+    for (Scorer scorer : subScorers) {
+      int scorerDocId = scorer.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 = (1 - Math.exp(scorer.score()));
+      } else {
+        tempScore = (1 - Math.exp(scorer.smoothingScore(docId)));
+      }
+      score *= tempScore;

Review comment:
       *NarrowingCompoundAssignment:*  Compound assignments from double to float hide lossy casts [(details)](https://errorprone.info/bugpattern/NarrowingCompoundAssignment)
   (at-me [in a reply](https://help.sonatype.com/lift/talking-to-lift) with `help` or `ignore`)

##########
File path: lucene/core/src/java/org/apache/lucene/search/IndriWeightedSumScorer.java
##########
@@ -0,0 +1,58 @@
+/*
+ * 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 IndriWeightedSumScorer extends IndriDisjunctionScorer {
+
+  protected IndriWeightedSumScorer(
+      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 {
+    float score = 0;
+    for (Scorer scorer : subScorers) {
+      int scorerDocId = scorer.docID();
+      double subScore = 0;
+      if (docId == scorerDocId) {
+        subScore += Math.exp(scorer.score());
+      } else {
+        subScore += Math.exp(scorer.smoothingScore(docId));
+      }
+      score += subScore;

Review comment:
       *NarrowingCompoundAssignment:*  Compound assignments from double to float hide lossy casts [(details)](https://errorprone.info/bugpattern/NarrowingCompoundAssignment)
   (at-me [in a reply](https://help.sonatype.com/lift/talking-to-lift) with `help` or `ignore`)

##########
File path: lucene/core/src/java/org/apache/lucene/search/IndriOrScorer.java
##########
@@ -0,0 +1,61 @@
+/*
+ * 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 IndriOrScorer extends IndriDisjunctionScorer {
+
+  protected IndriOrScorer(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 {
+    float score = 1;
+    for (Scorer scorer : subScorers) {
+      int scorerDocId = scorer.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 = (1 - Math.exp(scorer.score()));
+      } else {
+        tempScore = (1 - Math.exp(scorer.smoothingScore(docId)));
+      }
+      score *= tempScore;
+    }
+    return (float) (Math.log(1.0 - score));

Review comment:
       *UnnecessaryParentheses:*  Unnecessary use of grouping parentheses [(details)](https://errorprone.info/bugpattern/UnnecessaryParentheses)
   (at-me [in a reply](https://help.sonatype.com/lift/talking-to-lift) with `help` or `ignore`)

##########
File path: lucene/core/src/java/org/apache/lucene/search/IndriWeightedSumScorer.java
##########
@@ -0,0 +1,58 @@
+/*
+ * 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 IndriWeightedSumScorer extends IndriDisjunctionScorer {
+
+  protected IndriWeightedSumScorer(
+      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 {
+    float score = 0;
+    for (Scorer scorer : subScorers) {
+      int scorerDocId = scorer.docID();
+      double subScore = 0;
+      if (docId == scorerDocId) {
+        subScore += Math.exp(scorer.score());
+      } else {
+        subScore += Math.exp(scorer.smoothingScore(docId));
+      }
+      score += subScore;
+    }
+    return (float) (Math.log(score));

Review comment:
       *UnnecessaryParentheses:*  Unnecessary use of grouping parentheses [(details)](https://errorprone.info/bugpattern/UnnecessaryParentheses)
   (at-me [in a reply](https://help.sonatype.com/lift/talking-to-lift) with `help` or `ignore`)

##########
File path: lucene/queryparser/src/java/org/apache/lucene/queryparser/indri/IndriQueryParser.java
##########
@@ -0,0 +1,357 @@
+/*
+ * 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.queryparser.indri;
+
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.List;
+import java.util.Locale;
+import org.apache.lucene.analysis.Analyzer;
+import org.apache.lucene.index.Term;
+import org.apache.lucene.search.BooleanClause;
+import org.apache.lucene.search.BooleanClause.Occur;
+import org.apache.lucene.search.BoostQuery;
+import org.apache.lucene.search.IndriAndQuery;
+import org.apache.lucene.search.IndriOrQuery;
+import org.apache.lucene.search.IndriWeightedSumQuery;
+import org.apache.lucene.search.Query;
+import org.apache.lucene.search.TermQuery;
+import org.apache.lucene.util.BytesRef;
+
+/**
+ * IndriQueryParser is used to parse human readable query syntax and create Indri queries.
+ *
+ * <p><b>Query Operators</b>
+ *
+ * <ul>
+ *   <li>'{@code #and(t1 t2)}' specifies {@code IndriAnd} operation: <code>#and(term1 term2)</code>
+ *   <li>'{@code #or(t1 t2)}' specifies {@code IndriOr} operation: <code>#or(term1 term2)</code>
+ *   <li>'{@code #wsum(boost1 token1 boost2 token2)}' specifies {@code IndriOr} operation: <code>
+ *       #wsum(1.0 term1 2.0 term2)</code>
+ * </ul>
+ *
+ * <p>The default operator is {@code IndriAnd} if no other operator is specified. For example, the
+ * following will {@code IndriAnd} {@code token1} and {@code token2} together: <code>token1 token2
+ * </code>
+ */
+public class IndriQueryParser {
+
+  private static final String AND = "and";
+  private static final String OR = "or";
+  private static final String WAND = "wand";
+  private static final String WEIGHT = "weight";
+  private static final String WSUM = "wsum";
+
+  private final Analyzer analyzer;
+  private String field;
+
+  public IndriQueryParser(Analyzer analyzer, String field) throws IOException {
+    this.analyzer = analyzer;
+    this.field = field;
+  }
+
+  /**
+   * Count the number of occurrences of character c in string s.
+   *
+   * @param c A character.
+   * @param s A string.
+   */
+  private static int countChars(String s, char c) {
+    int count = 0;
+
+    for (int i = 0; i < s.length(); i++) {
+      if (s.charAt(i) == c) {
+        count++;
+      }
+    }
+    return count;
+  }
+
+  /**
+   * Get the index of the right parenenthesis that balances the left-most parenthesis. Return -1 if
+   * it doesn't exist.
+   *
+   * @param s A string containing a query.
+   */
+  private static int indexOfBalencingParen(String s) {
+    int depth = 0;
+
+    for (int i = 0; i < s.length(); i++) {
+      if (s.charAt(i) == '(') {
+        depth++;
+      } else if (s.charAt(i) == ')') {
+        depth--;
+
+        if (depth == 0) {
+          return i;
+        }
+      }
+    }
+    return -1;
+  }
+
+  private QueryParserOperatorQuery createOperator(String operatorName, Occur occur) {
+    QueryParserOperatorQuery operatorQuery = new QueryParserOperatorQuery();
+
+    int operatorDistance = 0;
+    String operatorNameLowerCase = (new String(operatorName)).toLowerCase(Locale.ROOT);
+    operatorNameLowerCase = operatorNameLowerCase.replace("#", "");
+    operatorNameLowerCase = operatorNameLowerCase.replace("~", "");
+
+    operatorQuery.setOperator(operatorNameLowerCase);
+    operatorQuery.setField(field);
+    operatorQuery.setDistance(operatorDistance);
+    operatorQuery.setOccur(occur);
+
+    return operatorQuery;
+  }
+
+  private class PopWeight {
+    private Float weight;
+    private String queryString;
+
+    public Float getWeight() {
+      return weight;
+    }
+
+    public void setWeight(Float weight) {
+      this.weight = weight;
+    }
+
+    public String getQueryString() {
+      return queryString;
+    }
+
+    public void setQueryString(String queryString) {
+      this.queryString = queryString;
+    }
+  }
+
+  /**
+   * Remove a weight from an argument string. Return the weight and the modified argument string.
+   */
+  private PopWeight popWeight(String argString, Float weight) {
+
+    String[] substrings = argString.split("[ \t]+", 2);
+
+    if (substrings.length < 2) {
+      syntaxError("Missing weight or query argument");
+    }
+
+    PopWeight popWeight = new PopWeight();
+    popWeight.setWeight(Float.valueOf(substrings[0]));
+    popWeight.setQueryString(substrings[1]);
+
+    return popWeight;
+  }
+
+  /**
+   * Remove a subQuery from an argument string. Return the subquery and the modified argument
+   * string.
+   */
+  private String popSubquery(
+      String argString, QueryParserOperatorQuery queryTree, Float weight, Occur occur) {
+
+    int i = indexOfBalencingParen(argString);
+
+    if (i < 0) { // Query syntax error. The parser
+      i = argString.length(); // handles it. Here, just don't fail.
+    }
+
+    String subquery = argString.substring(0, i + 1);
+    queryTree.addSubquery(parseQueryString(subquery, occur), weight);
+
+    argString = argString.substring(i + 1);
+
+    return argString;
+  }
+
+  /** Remove a term from an argument string. Return the term and the modified argument string. */
+  private String popTerm(
+      String argString, QueryParserOperatorQuery queryTree, Float weight, Occur occur) {
+    String[] substrings = argString.split("[ \t\n\r]+", 2);
+    String token = substrings[0];
+
+    // Split the token into a term and a field.
+    int delimiter = token.indexOf('.');
+    String term = null;
+
+    if (delimiter < 0) {
+      term = token;
+    } else { // Remove the field from the token
+      field = token.substring(delimiter + 1).toLowerCase(Locale.ROOT);
+      term = token.substring(0, delimiter);
+    }
+
+    final BytesRef normalizedTerm = analyzer.normalize(field, term);
+    QueryParserTermQuery termQuery = new QueryParserTermQuery();
+    termQuery.setTerm(normalizedTerm.utf8ToString());
+    termQuery.setField(field);
+    termQuery.setOccur(occur);
+    queryTree.addSubquery(termQuery, weight);
+
+    if (substrings.length < 2) { // Is this the last argument?
+      argString = "";
+    } else {
+      argString = substrings[1];
+    }
+
+    return argString;
+  }
+
+  private QueryParserQuery parseQueryString(String queryString, Occur occur) {
+    // Create the query tree
+    // This simple parser is sensitive to parenthensis placement, so
+    // check for basic errors first.
+    queryString = queryString.trim(); // The last character should be ')'
+
+    if ((countChars(queryString, '(') == 0)
+        || (countChars(queryString, '(') != countChars(queryString, ')'))
+        || (indexOfBalencingParen(queryString) != (queryString.length() - 1))) {
+      // throw IllegalArgumentException("Missing, unbalanced, or misplaced
+      // parentheses");
+    }
+
+    // The query language is prefix-oriented, so the query string can
+    // be processed left to right. At each step, a substring is
+    // popped from the head (left) of the string, and is converted to
+    // a Qry object that is added to the query tree. Subqueries are
+    // handled via recursion.
+
+    // Find the left-most query operator and start the query tree.
+    String[] substrings = queryString.split("[(]", 2);
+    String queryOperator = AND;
+    if (substrings.length > 1) {
+      queryOperator = substrings[0].trim();
+    }
+    QueryParserOperatorQuery queryTree = createOperator(queryOperator, occur);
+
+    // Start consuming queryString by removing the query operator and
+    // its terminating ')'. queryString is always the part of the
+    // query that hasn't been processed yet.
+
+    if (substrings.length > 1) {
+      queryString = substrings[1];
+      queryString = queryString.substring(0, queryString.lastIndexOf(")")).trim();
+    }
+
+    // Each pass below handles one argument to the query operator.
+    // Note: An argument can be a token that produces multiple terms
+    // (e.g., "near-death") or a subquery (e.g., "#and (a b c)").
+    // Recurse on subqueries.
+
+    while (queryString.length() > 0) {
+
+      // If the operator uses weighted query arguments, each pass of
+      // this loop must handle "weight arg". Handle the weight first.
+
+      Float weight = null;
+      if ((queryTree.getOperator().equals(WEIGHT))
+          || (queryTree.getOperator().equals(WAND))

Review comment:
       *UnnecessaryParentheses:*  Unnecessary use of grouping parentheses [(details)](https://errorprone.info/bugpattern/UnnecessaryParentheses)
   (at-me [in a reply](https://help.sonatype.com/lift/talking-to-lift) with `help` or `ignore`)

##########
File path: lucene/queryparser/src/java/org/apache/lucene/queryparser/indri/IndriQueryParser.java
##########
@@ -0,0 +1,357 @@
+/*
+ * 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.queryparser.indri;
+
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.List;
+import java.util.Locale;
+import org.apache.lucene.analysis.Analyzer;
+import org.apache.lucene.index.Term;
+import org.apache.lucene.search.BooleanClause;
+import org.apache.lucene.search.BooleanClause.Occur;
+import org.apache.lucene.search.BoostQuery;
+import org.apache.lucene.search.IndriAndQuery;
+import org.apache.lucene.search.IndriOrQuery;
+import org.apache.lucene.search.IndriWeightedSumQuery;
+import org.apache.lucene.search.Query;
+import org.apache.lucene.search.TermQuery;
+import org.apache.lucene.util.BytesRef;
+
+/**
+ * IndriQueryParser is used to parse human readable query syntax and create Indri queries.
+ *
+ * <p><b>Query Operators</b>
+ *
+ * <ul>
+ *   <li>'{@code #and(t1 t2)}' specifies {@code IndriAnd} operation: <code>#and(term1 term2)</code>
+ *   <li>'{@code #or(t1 t2)}' specifies {@code IndriOr} operation: <code>#or(term1 term2)</code>
+ *   <li>'{@code #wsum(boost1 token1 boost2 token2)}' specifies {@code IndriOr} operation: <code>
+ *       #wsum(1.0 term1 2.0 term2)</code>
+ * </ul>
+ *
+ * <p>The default operator is {@code IndriAnd} if no other operator is specified. For example, the
+ * following will {@code IndriAnd} {@code token1} and {@code token2} together: <code>token1 token2
+ * </code>
+ */
+public class IndriQueryParser {
+
+  private static final String AND = "and";
+  private static final String OR = "or";
+  private static final String WAND = "wand";
+  private static final String WEIGHT = "weight";
+  private static final String WSUM = "wsum";
+
+  private final Analyzer analyzer;
+  private String field;
+
+  public IndriQueryParser(Analyzer analyzer, String field) throws IOException {
+    this.analyzer = analyzer;
+    this.field = field;
+  }
+
+  /**
+   * Count the number of occurrences of character c in string s.
+   *
+   * @param c A character.
+   * @param s A string.
+   */
+  private static int countChars(String s, char c) {
+    int count = 0;
+
+    for (int i = 0; i < s.length(); i++) {
+      if (s.charAt(i) == c) {
+        count++;
+      }
+    }
+    return count;
+  }
+
+  /**
+   * Get the index of the right parenenthesis that balances the left-most parenthesis. Return -1 if
+   * it doesn't exist.
+   *
+   * @param s A string containing a query.
+   */
+  private static int indexOfBalencingParen(String s) {
+    int depth = 0;
+
+    for (int i = 0; i < s.length(); i++) {
+      if (s.charAt(i) == '(') {
+        depth++;
+      } else if (s.charAt(i) == ')') {
+        depth--;
+
+        if (depth == 0) {
+          return i;
+        }
+      }
+    }
+    return -1;
+  }
+
+  private QueryParserOperatorQuery createOperator(String operatorName, Occur occur) {
+    QueryParserOperatorQuery operatorQuery = new QueryParserOperatorQuery();
+
+    int operatorDistance = 0;
+    String operatorNameLowerCase = (new String(operatorName)).toLowerCase(Locale.ROOT);
+    operatorNameLowerCase = operatorNameLowerCase.replace("#", "");
+    operatorNameLowerCase = operatorNameLowerCase.replace("~", "");
+
+    operatorQuery.setOperator(operatorNameLowerCase);
+    operatorQuery.setField(field);
+    operatorQuery.setDistance(operatorDistance);
+    operatorQuery.setOccur(occur);
+
+    return operatorQuery;
+  }
+
+  private class PopWeight {
+    private Float weight;
+    private String queryString;
+
+    public Float getWeight() {
+      return weight;
+    }
+
+    public void setWeight(Float weight) {
+      this.weight = weight;
+    }
+
+    public String getQueryString() {
+      return queryString;
+    }
+
+    public void setQueryString(String queryString) {
+      this.queryString = queryString;
+    }
+  }
+
+  /**
+   * Remove a weight from an argument string. Return the weight and the modified argument string.
+   */
+  private PopWeight popWeight(String argString, Float weight) {
+
+    String[] substrings = argString.split("[ \t]+", 2);
+
+    if (substrings.length < 2) {
+      syntaxError("Missing weight or query argument");
+    }
+
+    PopWeight popWeight = new PopWeight();
+    popWeight.setWeight(Float.valueOf(substrings[0]));
+    popWeight.setQueryString(substrings[1]);
+
+    return popWeight;
+  }
+
+  /**
+   * Remove a subQuery from an argument string. Return the subquery and the modified argument
+   * string.
+   */
+  private String popSubquery(
+      String argString, QueryParserOperatorQuery queryTree, Float weight, Occur occur) {
+
+    int i = indexOfBalencingParen(argString);
+
+    if (i < 0) { // Query syntax error. The parser
+      i = argString.length(); // handles it. Here, just don't fail.
+    }
+
+    String subquery = argString.substring(0, i + 1);
+    queryTree.addSubquery(parseQueryString(subquery, occur), weight);
+
+    argString = argString.substring(i + 1);
+
+    return argString;
+  }
+
+  /** Remove a term from an argument string. Return the term and the modified argument string. */
+  private String popTerm(
+      String argString, QueryParserOperatorQuery queryTree, Float weight, Occur occur) {
+    String[] substrings = argString.split("[ \t\n\r]+", 2);
+    String token = substrings[0];
+
+    // Split the token into a term and a field.
+    int delimiter = token.indexOf('.');
+    String term = null;
+
+    if (delimiter < 0) {
+      term = token;
+    } else { // Remove the field from the token
+      field = token.substring(delimiter + 1).toLowerCase(Locale.ROOT);
+      term = token.substring(0, delimiter);
+    }
+
+    final BytesRef normalizedTerm = analyzer.normalize(field, term);
+    QueryParserTermQuery termQuery = new QueryParserTermQuery();
+    termQuery.setTerm(normalizedTerm.utf8ToString());
+    termQuery.setField(field);
+    termQuery.setOccur(occur);
+    queryTree.addSubquery(termQuery, weight);
+
+    if (substrings.length < 2) { // Is this the last argument?
+      argString = "";
+    } else {
+      argString = substrings[1];
+    }
+
+    return argString;
+  }
+
+  private QueryParserQuery parseQueryString(String queryString, Occur occur) {
+    // Create the query tree
+    // This simple parser is sensitive to parenthensis placement, so
+    // check for basic errors first.
+    queryString = queryString.trim(); // The last character should be ')'
+
+    if ((countChars(queryString, '(') == 0)
+        || (countChars(queryString, '(') != countChars(queryString, ')'))
+        || (indexOfBalencingParen(queryString) != (queryString.length() - 1))) {
+      // throw IllegalArgumentException("Missing, unbalanced, or misplaced
+      // parentheses");
+    }
+
+    // The query language is prefix-oriented, so the query string can
+    // be processed left to right. At each step, a substring is
+    // popped from the head (left) of the string, and is converted to
+    // a Qry object that is added to the query tree. Subqueries are
+    // handled via recursion.
+
+    // Find the left-most query operator and start the query tree.
+    String[] substrings = queryString.split("[(]", 2);
+    String queryOperator = AND;
+    if (substrings.length > 1) {
+      queryOperator = substrings[0].trim();
+    }
+    QueryParserOperatorQuery queryTree = createOperator(queryOperator, occur);
+
+    // Start consuming queryString by removing the query operator and
+    // its terminating ')'. queryString is always the part of the
+    // query that hasn't been processed yet.
+
+    if (substrings.length > 1) {
+      queryString = substrings[1];
+      queryString = queryString.substring(0, queryString.lastIndexOf(")")).trim();
+    }
+
+    // Each pass below handles one argument to the query operator.
+    // Note: An argument can be a token that produces multiple terms
+    // (e.g., "near-death") or a subquery (e.g., "#and (a b c)").
+    // Recurse on subqueries.
+
+    while (queryString.length() > 0) {
+
+      // If the operator uses weighted query arguments, each pass of
+      // this loop must handle "weight arg". Handle the weight first.
+
+      Float weight = null;
+      if ((queryTree.getOperator().equals(WEIGHT))

Review comment:
       *UnnecessaryParentheses:*  Unnecessary use of grouping parentheses [(details)](https://errorprone.info/bugpattern/UnnecessaryParentheses)
   (at-me [in a reply](https://help.sonatype.com/lift/talking-to-lift) with `help` or `ignore`)

##########
File path: lucene/queryparser/src/java/org/apache/lucene/queryparser/indri/IndriQueryParser.java
##########
@@ -0,0 +1,357 @@
+/*
+ * 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.queryparser.indri;
+
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.List;
+import java.util.Locale;
+import org.apache.lucene.analysis.Analyzer;
+import org.apache.lucene.index.Term;
+import org.apache.lucene.search.BooleanClause;
+import org.apache.lucene.search.BooleanClause.Occur;
+import org.apache.lucene.search.BoostQuery;
+import org.apache.lucene.search.IndriAndQuery;
+import org.apache.lucene.search.IndriOrQuery;
+import org.apache.lucene.search.IndriWeightedSumQuery;
+import org.apache.lucene.search.Query;
+import org.apache.lucene.search.TermQuery;
+import org.apache.lucene.util.BytesRef;
+
+/**
+ * IndriQueryParser is used to parse human readable query syntax and create Indri queries.
+ *
+ * <p><b>Query Operators</b>
+ *
+ * <ul>
+ *   <li>'{@code #and(t1 t2)}' specifies {@code IndriAnd} operation: <code>#and(term1 term2)</code>
+ *   <li>'{@code #or(t1 t2)}' specifies {@code IndriOr} operation: <code>#or(term1 term2)</code>
+ *   <li>'{@code #wsum(boost1 token1 boost2 token2)}' specifies {@code IndriOr} operation: <code>
+ *       #wsum(1.0 term1 2.0 term2)</code>
+ * </ul>
+ *
+ * <p>The default operator is {@code IndriAnd} if no other operator is specified. For example, the
+ * following will {@code IndriAnd} {@code token1} and {@code token2} together: <code>token1 token2
+ * </code>
+ */
+public class IndriQueryParser {
+
+  private static final String AND = "and";
+  private static final String OR = "or";
+  private static final String WAND = "wand";
+  private static final String WEIGHT = "weight";
+  private static final String WSUM = "wsum";
+
+  private final Analyzer analyzer;
+  private String field;
+
+  public IndriQueryParser(Analyzer analyzer, String field) throws IOException {
+    this.analyzer = analyzer;
+    this.field = field;
+  }
+
+  /**
+   * Count the number of occurrences of character c in string s.
+   *
+   * @param c A character.
+   * @param s A string.
+   */
+  private static int countChars(String s, char c) {
+    int count = 0;
+
+    for (int i = 0; i < s.length(); i++) {
+      if (s.charAt(i) == c) {
+        count++;
+      }
+    }
+    return count;
+  }
+
+  /**
+   * Get the index of the right parenenthesis that balances the left-most parenthesis. Return -1 if
+   * it doesn't exist.
+   *
+   * @param s A string containing a query.
+   */
+  private static int indexOfBalencingParen(String s) {
+    int depth = 0;
+
+    for (int i = 0; i < s.length(); i++) {
+      if (s.charAt(i) == '(') {
+        depth++;
+      } else if (s.charAt(i) == ')') {
+        depth--;
+
+        if (depth == 0) {
+          return i;
+        }
+      }
+    }
+    return -1;
+  }
+
+  private QueryParserOperatorQuery createOperator(String operatorName, Occur occur) {
+    QueryParserOperatorQuery operatorQuery = new QueryParserOperatorQuery();
+
+    int operatorDistance = 0;
+    String operatorNameLowerCase = (new String(operatorName)).toLowerCase(Locale.ROOT);

Review comment:
       *UnnecessaryParentheses:*  Unnecessary use of grouping parentheses [(details)](https://errorprone.info/bugpattern/UnnecessaryParentheses)
   (at-me [in a reply](https://help.sonatype.com/lift/talking-to-lift) with `help` or `ignore`)

##########
File path: lucene/queryparser/src/java/org/apache/lucene/queryparser/indri/IndriQueryParser.java
##########
@@ -0,0 +1,357 @@
+/*
+ * 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.queryparser.indri;
+
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.List;
+import java.util.Locale;
+import org.apache.lucene.analysis.Analyzer;
+import org.apache.lucene.index.Term;
+import org.apache.lucene.search.BooleanClause;
+import org.apache.lucene.search.BooleanClause.Occur;
+import org.apache.lucene.search.BoostQuery;
+import org.apache.lucene.search.IndriAndQuery;
+import org.apache.lucene.search.IndriOrQuery;
+import org.apache.lucene.search.IndriWeightedSumQuery;
+import org.apache.lucene.search.Query;
+import org.apache.lucene.search.TermQuery;
+import org.apache.lucene.util.BytesRef;
+
+/**
+ * IndriQueryParser is used to parse human readable query syntax and create Indri queries.
+ *
+ * <p><b>Query Operators</b>
+ *
+ * <ul>
+ *   <li>'{@code #and(t1 t2)}' specifies {@code IndriAnd} operation: <code>#and(term1 term2)</code>
+ *   <li>'{@code #or(t1 t2)}' specifies {@code IndriOr} operation: <code>#or(term1 term2)</code>
+ *   <li>'{@code #wsum(boost1 token1 boost2 token2)}' specifies {@code IndriOr} operation: <code>
+ *       #wsum(1.0 term1 2.0 term2)</code>
+ * </ul>
+ *
+ * <p>The default operator is {@code IndriAnd} if no other operator is specified. For example, the
+ * following will {@code IndriAnd} {@code token1} and {@code token2} together: <code>token1 token2
+ * </code>
+ */
+public class IndriQueryParser {
+
+  private static final String AND = "and";
+  private static final String OR = "or";
+  private static final String WAND = "wand";
+  private static final String WEIGHT = "weight";
+  private static final String WSUM = "wsum";
+
+  private final Analyzer analyzer;
+  private String field;
+
+  public IndriQueryParser(Analyzer analyzer, String field) throws IOException {
+    this.analyzer = analyzer;
+    this.field = field;
+  }
+
+  /**
+   * Count the number of occurrences of character c in string s.
+   *
+   * @param c A character.
+   * @param s A string.
+   */
+  private static int countChars(String s, char c) {
+    int count = 0;
+
+    for (int i = 0; i < s.length(); i++) {
+      if (s.charAt(i) == c) {
+        count++;
+      }
+    }
+    return count;
+  }
+
+  /**
+   * Get the index of the right parenenthesis that balances the left-most parenthesis. Return -1 if
+   * it doesn't exist.
+   *
+   * @param s A string containing a query.
+   */
+  private static int indexOfBalencingParen(String s) {
+    int depth = 0;
+
+    for (int i = 0; i < s.length(); i++) {
+      if (s.charAt(i) == '(') {
+        depth++;
+      } else if (s.charAt(i) == ')') {
+        depth--;
+
+        if (depth == 0) {
+          return i;
+        }
+      }
+    }
+    return -1;
+  }
+
+  private QueryParserOperatorQuery createOperator(String operatorName, Occur occur) {
+    QueryParserOperatorQuery operatorQuery = new QueryParserOperatorQuery();
+
+    int operatorDistance = 0;
+    String operatorNameLowerCase = (new String(operatorName)).toLowerCase(Locale.ROOT);
+    operatorNameLowerCase = operatorNameLowerCase.replace("#", "");
+    operatorNameLowerCase = operatorNameLowerCase.replace("~", "");
+
+    operatorQuery.setOperator(operatorNameLowerCase);
+    operatorQuery.setField(field);
+    operatorQuery.setDistance(operatorDistance);
+    operatorQuery.setOccur(occur);
+
+    return operatorQuery;
+  }
+
+  private class PopWeight {
+    private Float weight;
+    private String queryString;
+
+    public Float getWeight() {
+      return weight;
+    }
+
+    public void setWeight(Float weight) {
+      this.weight = weight;
+    }
+
+    public String getQueryString() {
+      return queryString;
+    }
+
+    public void setQueryString(String queryString) {
+      this.queryString = queryString;
+    }
+  }
+
+  /**
+   * Remove a weight from an argument string. Return the weight and the modified argument string.
+   */
+  private PopWeight popWeight(String argString, Float weight) {

Review comment:
       *UnusedVariable:*  The parameter 'weight' is never read. [(details)](https://errorprone.info/bugpattern/UnusedVariable)
   (at-me [in a reply](https://help.sonatype.com/lift/talking-to-lift) with `help` or `ignore`)




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@lucene.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



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


[GitHub] [lucene] TomMD commented on a change in pull request #463: LUCENE-10157 - Added IndriOr, IndriWeightedSum, and IndriQueryParser

Posted by GitBox <gi...@apache.org>.
TomMD commented on a change in pull request #463:
URL: https://github.com/apache/lucene/pull/463#discussion_r780437567



##########
File path: lucene/core/src/java/org/apache/lucene/search/IndriOrScorer.java
##########
@@ -0,0 +1,61 @@
+/*
+ * 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 IndriOrScorer extends IndriDisjunctionScorer {
+
+  protected IndriOrScorer(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 {
+    float score = 1;
+    for (Scorer scorer : subScorers) {
+      int scorerDocId = scorer.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 = (1 - Math.exp(scorer.score()));
+      } else {
+        tempScore = (1 - Math.exp(scorer.smoothingScore(docId)));
+      }
+      score *= tempScore;

Review comment:
       It's saying that something big (double, 64 bit) is being combined into something smaller (float, 32 bit). For example, this is the same concept as warnings you get with C when assigning a variable of type `int` to a variable of type `size_t`. Making `score` a `double` would resolve the warning, though functionality still depends on the magnitude and the floating point multiplication not causing bounds or accuracy issues.




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@lucene.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



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


[GitHub] [lucene] TomMD commented on a change in pull request #463: LUCENE-10157 - Added IndriOr, IndriWeightedSum, and IndriQueryParser

Posted by GitBox <gi...@apache.org>.
TomMD commented on a change in pull request #463:
URL: https://github.com/apache/lucene/pull/463#discussion_r780437567



##########
File path: lucene/core/src/java/org/apache/lucene/search/IndriOrScorer.java
##########
@@ -0,0 +1,61 @@
+/*
+ * 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 IndriOrScorer extends IndriDisjunctionScorer {
+
+  protected IndriOrScorer(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 {
+    float score = 1;
+    for (Scorer scorer : subScorers) {
+      int scorerDocId = scorer.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 = (1 - Math.exp(scorer.score()));
+      } else {
+        tempScore = (1 - Math.exp(scorer.smoothingScore(docId)));
+      }
+      score *= tempScore;

Review comment:
       It's saying that something big (double, 64 bit) is being combined into something smaller (float, 32 bit). For example, this is the same concept as warnings you get with C when assigning a variable of type `int` to a variable of type `size_t`. Making `score` a `double` would resolve the warning, though functionality still depends on the magnitude and the floating point multiplication not causing bounds or accuracy issues.




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@lucene.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



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


[GitHub] [lucene] cpoerschke commented on a change in pull request #463: LUCENE-10157 - Added IndriOr, IndriWeightedSum, and IndriQueryParser

Posted by GitBox <gi...@apache.org>.
cpoerschke commented on a change in pull request #463:
URL: https://github.com/apache/lucene/pull/463#discussion_r780405554



##########
File path: lucene/core/src/java/org/apache/lucene/search/IndriAndWeight.java
##########
@@ -38,8 +38,21 @@ public IndriAndWeight(
     this.boost = boost;
     this.scoreMode = scoreMode;
     weights = new ArrayList<>();
+    // Calculate total boost score so that boost can be normalized
+    float boostSum = 0;
     for (BooleanClause c : query) {
-      Weight w = searcher.createWeight(c.getQuery(), scoreMode, 1.0f);
+      if (c.getQuery() instanceof BoostQuery) {
+        boostSum += ((BoostQuery) c.getQuery()).getBoost();

Review comment:
       subjective: perhaps introduce a local variable to avoid duplicate `c.getQuery()` calls.
   
   ```suggestion
         Query q = c.getQuery();
         if (q instanceof BoostQuery) {
           boostSum += ((BoostQuery) q).getBoost();
   ```

##########
File path: lucene/core/src/java/org/apache/lucene/search/IndriOrWeight.java
##########
@@ -0,0 +1,135 @@
+/*
+ * 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 IndriOrWeight extends Weight {
+
+  private final IndriQuery query;
+  private final float boost;
+  private final ArrayList<Weight> weights;
+  private final ScoreMode scoreMode;
+
+  public IndriOrWeight(IndriOrQuery query, IndexSearcher searcher, ScoreMode scoreMode, float boost)
+      throws IOException {
+    super(query);
+    this.query = query;
+    this.boost = boost;
+    this.scoreMode = scoreMode;
+    weights = new ArrayList<>();

Review comment:
       subjective/style: `this.weights` here and at line 55 even though (unlike for the other members) it's not necessary
   ```suggestion
       this.weights = new ArrayList<>();
   ```

##########
File path: lucene/core/src/java/org/apache/lucene/search/IndriOrWeight.java
##########
@@ -0,0 +1,135 @@
+/*
+ * 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. */

Review comment:
       ```suggestion
   /** The Weight for IndriOrQuery, used to normalize, score and explain these queries. */
   ```

##########
File path: lucene/core/src/java/org/apache/lucene/search/IndriOrScorer.java
##########
@@ -0,0 +1,61 @@
+/*
+ * 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 IndriOrScorer extends IndriDisjunctionScorer {
+
+  protected IndriOrScorer(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 {
+    float score = 1;
+    for (Scorer scorer : subScorers) {
+      int scorerDocId = scorer.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 = (1 - Math.exp(scorer.score()));
+      } else {
+        tempScore = (1 - Math.exp(scorer.smoothingScore(docId)));
+      }
+      score *= tempScore;

Review comment:
       Curious if this is the bot here trying to say that `score` and `tempScore` should be `float` instead? Noting that elsewhere above in the PR some locals are `float` rather than `double`.

##########
File path: lucene/core/src/java/org/apache/lucene/search/IndriWeightedSumWeight.java
##########
@@ -0,0 +1,135 @@
+/*
+ * 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 IndriWeightedSumWeight extends Weight {
+  private final IndriWeightedSumQuery query;
+
+  private final ArrayList<Weight> weights;
+  private final ScoreMode scoreMode;
+  private final float boost;
+
+  public IndriWeightedSumWeight(
+      IndriWeightedSumQuery query, IndexSearcher searcher, ScoreMode scoreMode, float boost)
+      throws IOException {
+    super(query);
+    this.query = query;
+    this.boost = boost;
+    this.scoreMode = scoreMode;
+    weights = new ArrayList<>();
+    // Calculate total boost score so that boost can be normalized
+    float boostSum = 0;
+    for (BooleanClause c : query) {
+      if (c.getQuery() instanceof BoostQuery) {
+        boostSum += ((BoostQuery) c.getQuery()).getBoost();
+      } else {
+        boostSum++;
+      }
+    }
+    for (BooleanClause c : query) {
+      float subBoost = 1.0f;
+      if (c.getQuery() instanceof BoostQuery) {
+        subBoost = ((BoostQuery) c.getQuery()).getBoost() / boostSum;
+      }
+      Weight w = searcher.createWeight(c.getQuery(), scoreMode, subBoost);
+      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 IndriWeightedSumScorer(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 {

Review comment:
       It seems the three classes for and/or/sum share some common code e.g. in `explain` here. Have you considered having a common `IndriWeight` class which they can extend to reduce common code repetition?




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@lucene.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



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


[GitHub] [lucene] cammiemw commented on pull request #463: LUCENE-10157 - Added IndriOr, IndriWeightedSum, and IndriQueryParser

Posted by GitBox <gi...@apache.org>.
cammiemw commented on pull request #463:
URL: https://github.com/apache/lucene/pull/463#issuecomment-1009289345


   Hi @cpoerschke!  Thank you for your comments.  I have made the changes to accept most of your suggestions.  For the weight explain methods, I went through and tweaked them so that they are not the same anymore and do a better job of explaining the scores.  The only issue I have is that I cannot fix the bot error for the double to float conversion because Math.log returns a double and the lucene API expects a float.  Let me know if there is anything else that you think I should do.
   
   I have also added additional unit tests.  I am going to comment on the jira issue as well to see what Adrien Grand thinks of the functionality.  


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@lucene.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



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