You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@lucene.apache.org by rm...@apache.org on 2011/11/04 10:23:57 UTC

svn commit: r1197455 - in /lucene/dev/trunk/lucene/src: java/org/apache/lucene/search/ java/org/apache/lucene/search/similarities/ java/org/apache/lucene/search/spans/ test/org/apache/lucene/index/ test/org/apache/lucene/search/ test/org/apache/lucene/...

Author: rmuir
Date: Fri Nov  4 09:23:56 2011
New Revision: 1197455

URL: http://svn.apache.org/viewvc?rev=1197455&view=rev
Log:
LUCENE-3555: add support for distributed stats

Added:
    lucene/dev/trunk/lucene/src/java/org/apache/lucene/search/CollectionStatistics.java   (with props)
    lucene/dev/trunk/lucene/src/java/org/apache/lucene/search/TermStatistics.java   (with props)
Removed:
    lucene/dev/trunk/lucene/src/test/org/apache/lucene/search/similarities/SpoofIndexSearcher.java
Modified:
    lucene/dev/trunk/lucene/src/java/org/apache/lucene/search/IndexSearcher.java
    lucene/dev/trunk/lucene/src/java/org/apache/lucene/search/MultiPhraseQuery.java
    lucene/dev/trunk/lucene/src/java/org/apache/lucene/search/PhraseQuery.java
    lucene/dev/trunk/lucene/src/java/org/apache/lucene/search/TermQuery.java
    lucene/dev/trunk/lucene/src/java/org/apache/lucene/search/similarities/BM25Similarity.java
    lucene/dev/trunk/lucene/src/java/org/apache/lucene/search/similarities/LMSimilarity.java
    lucene/dev/trunk/lucene/src/java/org/apache/lucene/search/similarities/MultiSimilarity.java
    lucene/dev/trunk/lucene/src/java/org/apache/lucene/search/similarities/Similarity.java
    lucene/dev/trunk/lucene/src/java/org/apache/lucene/search/similarities/SimilarityBase.java
    lucene/dev/trunk/lucene/src/java/org/apache/lucene/search/similarities/TFIDFSimilarity.java
    lucene/dev/trunk/lucene/src/java/org/apache/lucene/search/spans/SpanWeight.java
    lucene/dev/trunk/lucene/src/test/org/apache/lucene/index/TestOmitTf.java
    lucene/dev/trunk/lucene/src/test/org/apache/lucene/search/JustCompileSearch.java
    lucene/dev/trunk/lucene/src/test/org/apache/lucene/search/TestDocValuesScoring.java
    lucene/dev/trunk/lucene/src/test/org/apache/lucene/search/TestMultiPhraseQuery.java
    lucene/dev/trunk/lucene/src/test/org/apache/lucene/search/TestSimilarity.java
    lucene/dev/trunk/lucene/src/test/org/apache/lucene/search/payloads/TestPayloadNearQuery.java
    lucene/dev/trunk/lucene/src/test/org/apache/lucene/search/similarities/TestSimilarityBase.java

Added: lucene/dev/trunk/lucene/src/java/org/apache/lucene/search/CollectionStatistics.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/src/java/org/apache/lucene/search/CollectionStatistics.java?rev=1197455&view=auto
==============================================================================
--- lucene/dev/trunk/lucene/src/java/org/apache/lucene/search/CollectionStatistics.java (added)
+++ lucene/dev/trunk/lucene/src/java/org/apache/lucene/search/CollectionStatistics.java Fri Nov  4 09:23:56 2011
@@ -0,0 +1,72 @@
+package org.apache.lucene.search;
+
+import org.apache.lucene.index.IndexReader; // javadocs
+import org.apache.lucene.index.Terms;       // javadocs
+
+/**
+ * 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.
+ */
+
+/**
+ * Contains statistics for a collection (field)
+ * @lucene.experimental
+ */
+public class CollectionStatistics {
+  private final String field;
+  private final int maxDoc;
+  private final int docCount;
+  private final long sumTotalTermFreq;
+  private final long sumDocFreq;
+  
+  public CollectionStatistics(String field, int maxDoc, int docCount, long sumTotalTermFreq, long sumDocFreq) {
+    this.field = field;
+    this.maxDoc = maxDoc;
+    this.docCount = docCount;
+    this.sumTotalTermFreq = sumTotalTermFreq;
+    this.sumDocFreq = sumDocFreq;
+  }
+  
+  /** returns the field name */
+  public String field() {
+    return field;
+  }
+  
+  /** returns the total number of documents, regardless of 
+   * whether they all contain values for this field. 
+   * @see IndexReader#maxDoc() */
+  public int maxDoc() {
+    return maxDoc;
+  }
+  
+  /** returns the total number of documents that
+   * have at least one term for this field. 
+   * @see Terms#getDocCount() */
+  public int docCount() {
+    return docCount;
+  }
+  
+  /** returns the total number of tokens for this field
+   * @see Terms#getSumTotalTermFreq() */
+  public long sumTotalTermFreq() {
+    return sumTotalTermFreq;
+  }
+  
+  /** returns the total number of postings for this field 
+   * @see Terms#getSumDocFreq() */
+  public long sumDocFreq() {
+    return sumDocFreq;
+  }
+}

Modified: lucene/dev/trunk/lucene/src/java/org/apache/lucene/search/IndexSearcher.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/src/java/org/apache/lucene/search/IndexSearcher.java?rev=1197455&r1=1197454&r2=1197455&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/src/java/org/apache/lucene/search/IndexSearcher.java (original)
+++ lucene/dev/trunk/lucene/src/java/org/apache/lucene/search/IndexSearcher.java Fri Nov  4 09:23:56 2011
@@ -35,14 +35,18 @@ import org.apache.lucene.index.CorruptIn
 import org.apache.lucene.index.IndexReader.AtomicReaderContext;
 import org.apache.lucene.index.IndexReader.ReaderContext;
 import org.apache.lucene.index.IndexReader;
+import org.apache.lucene.index.MultiFields;
 import org.apache.lucene.index.StoredFieldVisitor;
 import org.apache.lucene.index.Term;
+import org.apache.lucene.index.Terms;
 import org.apache.lucene.search.similarities.DefaultSimilarityProvider;
 import org.apache.lucene.search.similarities.SimilarityProvider;
 import org.apache.lucene.store.Directory;
 import org.apache.lucene.store.NIOFSDirectory;    // javadoc
 import org.apache.lucene.util.Bits;
+import org.apache.lucene.util.BytesRef;
 import org.apache.lucene.util.ReaderUtil;
+import org.apache.lucene.util.TermContext;
 import org.apache.lucene.util.ThreadInterruptedException;
 
 /** Implements search over a single IndexReader.
@@ -860,4 +864,34 @@ public class IndexSearcher implements Cl
   public String toString() {
     return "IndexSearcher(" + reader + "; executor=" + executor + ")";
   }
+  
+  /**
+   * Returns {@link TermStatistics} for a term
+   * @lucene.experimental
+   */
+  public TermStatistics termStatistics(Term term, TermContext context) throws IOException {
+    return new TermStatistics(term.bytes(), context.docFreq(), context.totalTermFreq());
+  };
+  
+  /**
+   * Returns {@link CollectionStatistics} for a field
+   * @lucene.experimental
+   */
+  public CollectionStatistics collectionStatistics(String field) throws IOException {
+    final int docCount;
+    final long sumTotalTermFreq;
+    final long sumDocFreq;
+    
+    Terms terms = MultiFields.getTerms(reader, field);
+    if (terms == null) {
+      docCount = 0;
+      sumTotalTermFreq = 0;
+      sumDocFreq = 0;
+    } else {
+      docCount = terms.getDocCount();
+      sumTotalTermFreq = terms.getSumTotalTermFreq();
+      sumDocFreq = terms.getSumDocFreq();
+    }
+    return new CollectionStatistics(field, reader.maxDoc(), docCount, sumTotalTermFreq, sumDocFreq);
+  }
 }

Modified: lucene/dev/trunk/lucene/src/java/org/apache/lucene/search/MultiPhraseQuery.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/src/java/org/apache/lucene/search/MultiPhraseQuery.java?rev=1197455&r1=1197454&r2=1197455&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/src/java/org/apache/lucene/search/MultiPhraseQuery.java (original)
+++ lucene/dev/trunk/lucene/src/java/org/apache/lucene/search/MultiPhraseQuery.java Fri Nov  4 09:23:56 2011
@@ -141,13 +141,15 @@ public class MultiPhraseQuery extends Qu
       final ReaderContext context = searcher.getTopReaderContext();
       
       // compute idf
-      ArrayList<TermContext> allTerms = new ArrayList<TermContext>();
+      ArrayList<TermStatistics> allTermStats = new ArrayList<TermStatistics>();
       for(final Term[] terms: termArrays) {
         for (Term term: terms) {
-          allTerms.add(TermContext.build(context, term, true));
+          TermContext termContext = TermContext.build(context, term, true);
+          allTermStats.add(searcher.termStatistics(term, termContext));
         }
       }
-      stats = similarity.computeStats(searcher, field, getBoost(), allTerms.toArray(new TermContext[allTerms.size()]));
+      stats = similarity.computeStats(searcher.collectionStatistics(field), 
+          getBoost(), allTermStats.toArray(new TermStatistics[allTermStats.size()]));
     }
 
     @Override

Modified: lucene/dev/trunk/lucene/src/java/org/apache/lucene/search/PhraseQuery.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/src/java/org/apache/lucene/search/PhraseQuery.java?rev=1197455&r1=1197454&r2=1197455&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/src/java/org/apache/lucene/search/PhraseQuery.java (original)
+++ lucene/dev/trunk/lucene/src/java/org/apache/lucene/search/PhraseQuery.java Fri Nov  4 09:23:56 2011
@@ -190,9 +190,13 @@ public class PhraseQuery extends Query {
       this.similarity = searcher.getSimilarityProvider().get(field);
       final ReaderContext context = searcher.getTopReaderContext();
       states = new TermContext[terms.size()];
-      for (int i = 0; i < terms.size(); i++)
-        states[i] = TermContext.build(context, terms.get(i), true);
-      stats = similarity.computeStats(searcher, field, getBoost(), states);
+      TermStatistics termStats[] = new TermStatistics[terms.size()];
+      for (int i = 0; i < terms.size(); i++) {
+        final Term term = terms.get(i);
+        states[i] = TermContext.build(context, term, true);
+        termStats[i] = searcher.termStatistics(term, states[i]);
+      }
+      stats = similarity.computeStats(searcher.collectionStatistics(field), getBoost(), termStats);
     }
 
     @Override

Modified: lucene/dev/trunk/lucene/src/java/org/apache/lucene/search/TermQuery.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/src/java/org/apache/lucene/search/TermQuery.java?rev=1197455&r1=1197454&r2=1197455&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/src/java/org/apache/lucene/search/TermQuery.java (original)
+++ lucene/dev/trunk/lucene/src/java/org/apache/lucene/search/TermQuery.java Fri Nov  4 09:23:56 2011
@@ -54,7 +54,10 @@ public class TermQuery extends Query {
       assert termStates != null : "TermContext must not be null";
       this.termStates = termStates;
       this.similarity = searcher.getSimilarityProvider().get(term.field());
-      this.stats = similarity.computeStats(searcher, term.field(), getBoost(), termStates);
+      this.stats = similarity.computeStats(
+          searcher.collectionStatistics(term.field()), 
+          getBoost(), 
+          searcher.termStatistics(term, termStates));
     }
 
     @Override

Added: lucene/dev/trunk/lucene/src/java/org/apache/lucene/search/TermStatistics.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/src/java/org/apache/lucene/search/TermStatistics.java?rev=1197455&view=auto
==============================================================================
--- lucene/dev/trunk/lucene/src/java/org/apache/lucene/search/TermStatistics.java (added)
+++ lucene/dev/trunk/lucene/src/java/org/apache/lucene/search/TermStatistics.java Fri Nov  4 09:23:56 2011
@@ -0,0 +1,53 @@
+package org.apache.lucene.search;
+
+/**
+ * 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.
+ */
+
+import org.apache.lucene.index.IndexReader; // javadocs
+import org.apache.lucene.util.BytesRef;
+/**
+ * Contains statistics for a specific term
+ * @lucene.experimental
+ */
+public class TermStatistics {
+  private final BytesRef term;
+  private final int docFreq;
+  private final long totalTermFreq;
+  
+  public TermStatistics(BytesRef term, int docFreq, long totalTermFreq) {
+    this.term = term;
+    this.docFreq = docFreq;
+    this.totalTermFreq = totalTermFreq;
+  }
+  
+  /** returns the term text */
+  public BytesRef term() {
+    return term;
+  }
+  
+  /** returns the number of documents this term occurs in 
+   * @see IndexReader#docFreq(String, BytesRef) */
+  public int docFreq() {
+    return docFreq;
+  }
+  
+  /** returns the total number of occurrences of this term
+   * @see IndexReader#totalTermFreq(String, BytesRef) */
+  public long totalTermFreq() {
+    return totalTermFreq;
+  }
+}

Modified: lucene/dev/trunk/lucene/src/java/org/apache/lucene/search/similarities/BM25Similarity.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/src/java/org/apache/lucene/search/similarities/BM25Similarity.java?rev=1197455&r1=1197454&r2=1197455&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/src/java/org/apache/lucene/search/similarities/BM25Similarity.java (original)
+++ lucene/dev/trunk/lucene/src/java/org/apache/lucene/search/similarities/BM25Similarity.java Fri Nov  4 09:23:56 2011
@@ -20,14 +20,12 @@ package org.apache.lucene.search.similar
 import java.io.IOException;
 
 import org.apache.lucene.index.FieldInvertState;
-import org.apache.lucene.index.MultiFields;
 import org.apache.lucene.index.IndexReader.AtomicReaderContext;
-import org.apache.lucene.index.Terms;
+import org.apache.lucene.search.CollectionStatistics;
 import org.apache.lucene.search.Explanation;
-import org.apache.lucene.search.IndexSearcher;
+import org.apache.lucene.search.TermStatistics;
 import org.apache.lucene.util.BytesRef;
 import org.apache.lucene.util.SmallFloat;
-import org.apache.lucene.util.TermContext;
 
 /**
  * BM25 Similarity. Introduced in Stephen E. Robertson, Steve Walker,
@@ -75,15 +73,13 @@ public class BM25Similarity extends Simi
   /** The default implementation computes the average as <code>sumTotalTermFreq / maxDoc</code>,
    * or returns <code>1</code> if the index does not store sumTotalTermFreq (Lucene 3.x indexes
    * or any field that omits frequency information). */
-  protected float avgFieldLength(IndexSearcher searcher, String field) throws IOException {
-    Terms terms = MultiFields.getTerms(searcher.getIndexReader(), field);
-    if (terms == null) {
-      // field does not exist;
-      return 1f;
-    }
-    long sumTotalTermFreq = terms.getSumTotalTermFreq();
-    long maxdoc = searcher.maxDoc();
-    return sumTotalTermFreq == -1 ? 1f : (float) (sumTotalTermFreq / (double) maxdoc);
+  protected float avgFieldLength(CollectionStatistics collectionStats) {
+    final long sumTotalTermFreq = collectionStats.sumTotalTermFreq();
+    if (sumTotalTermFreq <= 0) {
+      return 1f;       // field does not exist, or stat is unsupported
+    } else {
+      return (float) (sumTotalTermFreq / (double) collectionStats.maxDoc());
+    }
   }
   
   /** The default implementation encodes <code>boost / sqrt(length)</code>
@@ -131,19 +127,19 @@ public class BM25Similarity extends Simi
     return encodeNormValue(state.getBoost(), numTerms);
   }
 
-  public Explanation idfExplain(TermContext stats, final IndexSearcher searcher) throws IOException {
-    final int df = stats.docFreq();
-    final int max = searcher.maxDoc();
+  public Explanation idfExplain(CollectionStatistics collectionStats, TermStatistics termStats) {
+    final int df = termStats.docFreq();
+    final int max = collectionStats.maxDoc();
     final float idf = idf(df, max);
     return new Explanation(idf, "idf(docFreq=" + df + ", maxDocs=" + max + ")");
   }
 
-  public Explanation idfExplain(final TermContext stats[], IndexSearcher searcher) throws IOException {
-    final int max = searcher.maxDoc();
+  public Explanation idfExplain(CollectionStatistics collectionStats, TermStatistics termStats[]) {
+    final int max = collectionStats.maxDoc();
     float idf = 0.0f;
     final Explanation exp = new Explanation();
     exp.setDescription("idf(), sum of:");
-    for (final TermContext stat : stats ) {
+    for (final TermStatistics stat : termStats ) {
       final int df = stat.docFreq();
       final float termIdf = idf(df, max);
       exp.addDetail(new Explanation(termIdf, "idf(docFreq=" + df + ", maxDocs=" + max + ")"));
@@ -154,10 +150,10 @@ public class BM25Similarity extends Simi
   }
 
   @Override
-  public final Stats computeStats(IndexSearcher searcher, String fieldName, float queryBoost, TermContext... termStats) throws IOException {
-    Explanation idf = termStats.length == 1 ? idfExplain(termStats[0], searcher) : idfExplain(termStats, searcher);
+  public final Stats computeStats(CollectionStatistics collectionStats, float queryBoost, TermStatistics... termStats) {
+    Explanation idf = termStats.length == 1 ? idfExplain(collectionStats, termStats[0]) : idfExplain(collectionStats, termStats);
 
-    float avgdl = avgFieldLength(searcher, fieldName);
+    float avgdl = avgFieldLength(collectionStats);
 
     // compute freq-independent part of bm25 equation across all norm values
     float cache[] = new float[256];

Modified: lucene/dev/trunk/lucene/src/java/org/apache/lucene/search/similarities/LMSimilarity.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/src/java/org/apache/lucene/search/similarities/LMSimilarity.java?rev=1197455&r1=1197454&r2=1197455&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/src/java/org/apache/lucene/search/similarities/LMSimilarity.java (original)
+++ lucene/dev/trunk/lucene/src/java/org/apache/lucene/search/similarities/LMSimilarity.java Fri Nov  4 09:23:56 2011
@@ -17,11 +17,9 @@ package org.apache.lucene.search.similar
  * limitations under the License.
  */
 
-import java.io.IOException;
-
+import org.apache.lucene.search.CollectionStatistics;
 import org.apache.lucene.search.Explanation;
-import org.apache.lucene.search.IndexSearcher;
-import org.apache.lucene.util.TermContext;
+import org.apache.lucene.search.TermStatistics;
 
 /**
  * Abstract superclass for language modeling Similarities. The following inner
@@ -62,8 +60,8 @@ public abstract class LMSimilarity exten
    * usual statistics.
    */
   @Override
-  protected void fillBasicStats(BasicStats stats, IndexSearcher searcher, String fieldName, TermContext termContext) throws IOException {
-    super.fillBasicStats(stats, searcher, fieldName, termContext);
+  protected void fillBasicStats(BasicStats stats, CollectionStatistics collectionStats, TermStatistics termStats) {
+    super.fillBasicStats(stats, collectionStats, termStats);
     LMStats lmStats = (LMStats) stats;
     lmStats.setCollectionProbability(collectionModel.computeProbability(stats));
   }

Modified: lucene/dev/trunk/lucene/src/java/org/apache/lucene/search/similarities/MultiSimilarity.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/src/java/org/apache/lucene/search/similarities/MultiSimilarity.java?rev=1197455&r1=1197454&r2=1197455&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/src/java/org/apache/lucene/search/similarities/MultiSimilarity.java (original)
+++ lucene/dev/trunk/lucene/src/java/org/apache/lucene/search/similarities/MultiSimilarity.java Fri Nov  4 09:23:56 2011
@@ -21,10 +21,10 @@ import java.io.IOException;
 
 import org.apache.lucene.index.FieldInvertState;
 import org.apache.lucene.index.IndexReader.AtomicReaderContext;
+import org.apache.lucene.search.CollectionStatistics;
 import org.apache.lucene.search.Explanation;
-import org.apache.lucene.search.IndexSearcher;
+import org.apache.lucene.search.TermStatistics;
 import org.apache.lucene.util.BytesRef;
-import org.apache.lucene.util.TermContext;
 
 /**
  * Implements the CombSUM method for combining evidence from multiple
@@ -45,10 +45,10 @@ public class MultiSimilarity extends Sim
   }
 
   @Override
-  public Stats computeStats(IndexSearcher searcher, String fieldName, float queryBoost, TermContext... termContexts) throws IOException {
+  public Stats computeStats(CollectionStatistics collectionStats, float queryBoost, TermStatistics... termStats) {
     Stats subStats[] = new Stats[sims.length];
     for (int i = 0; i < subStats.length; i++) {
-      subStats[i] = sims[i].computeStats(searcher, fieldName, queryBoost, termContexts);
+      subStats[i] = sims[i].computeStats(collectionStats, queryBoost, termStats);
     }
     return new MultiStats(subStats);
   }

Modified: lucene/dev/trunk/lucene/src/java/org/apache/lucene/search/similarities/Similarity.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/src/java/org/apache/lucene/search/similarities/Similarity.java?rev=1197455&r1=1197454&r2=1197455&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/src/java/org/apache/lucene/search/similarities/Similarity.java (original)
+++ lucene/dev/trunk/lucene/src/java/org/apache/lucene/search/similarities/Similarity.java Fri Nov  4 09:23:56 2011
@@ -26,11 +26,13 @@ import org.apache.lucene.index.IndexRead
 import org.apache.lucene.index.IndexReader.AtomicReaderContext;
 import org.apache.lucene.index.Terms; // javadoc
 import org.apache.lucene.search.BooleanQuery;
+import org.apache.lucene.search.CollectionStatistics;
 import org.apache.lucene.search.Explanation;
 import org.apache.lucene.search.IndexSearcher;
 import org.apache.lucene.search.PhraseQuery;
 import org.apache.lucene.search.Query;
 import org.apache.lucene.search.TermQuery;
+import org.apache.lucene.search.TermStatistics;
 import org.apache.lucene.search.spans.SpanQuery; // javadoc
 import org.apache.lucene.util.BytesRef;
 import org.apache.lucene.util.SmallFloat; // javadoc
@@ -81,10 +83,10 @@ import org.apache.lucene.util.TermContex
  * <a name="querytime"/>
  * At query-time, Queries interact with the Similarity via these steps:
  * <ol>
- *   <li>The {@link #computeStats(IndexSearcher, String, float, TermContext...)} method is called a single time,
+ *   <li>The {@link #computeStats(CollectionStatistics, float, TermStatistics...)} method is called a single time,
  *       allowing the implementation to compute any statistics (such as IDF, average document length, etc)
- *       across <i>the entire collection</i>. The {@link TermContext}s passed in are already positioned
- *       to the terms involved with the raw statistics involved, so a Similarity can freely use any combination
+ *       across <i>the entire collection</i>. The {@link TermStatistics} passed in already contain
+ *       the raw statistics involved, so a Similarity can freely use any combination
  *       of term statistics without causing any additional I/O. Lucene makes no assumption about what is 
  *       stored in the returned {@link Similarity.Stats} object.
  *   <li>The query normalization process occurs a single time: {@link Similarity.Stats#getValueForNormalization()}
@@ -128,7 +130,7 @@ public abstract class Similarity {
   /**
    * Compute any collection-level stats (e.g. IDF, average document length, etc) needed for scoring a query.
    */
-  public abstract Stats computeStats(IndexSearcher searcher, String fieldName, float queryBoost, TermContext... termContexts) throws IOException;
+  public abstract Stats computeStats(CollectionStatistics collectionStats, float queryBoost, TermStatistics... termStats);
   
   /**
    * returns a new {@link Similarity.ExactDocScorer}.

Modified: lucene/dev/trunk/lucene/src/java/org/apache/lucene/search/similarities/SimilarityBase.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/src/java/org/apache/lucene/search/similarities/SimilarityBase.java?rev=1197455&r1=1197454&r2=1197455&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/src/java/org/apache/lucene/search/similarities/SimilarityBase.java (original)
+++ lucene/dev/trunk/lucene/src/java/org/apache/lucene/search/similarities/SimilarityBase.java Fri Nov  4 09:23:56 2011
@@ -20,15 +20,12 @@ package org.apache.lucene.search.similar
 import java.io.IOException;
 
 import org.apache.lucene.index.FieldInvertState;
-import org.apache.lucene.index.IndexReader;
 import org.apache.lucene.index.IndexReader.AtomicReaderContext;
-import org.apache.lucene.index.MultiFields;
-import org.apache.lucene.index.Terms;
+import org.apache.lucene.search.CollectionStatistics;
 import org.apache.lucene.search.Explanation;
-import org.apache.lucene.search.IndexSearcher;
+import org.apache.lucene.search.TermStatistics;
 import org.apache.lucene.util.BytesRef;
 import org.apache.lucene.util.SmallFloat;
-import org.apache.lucene.util.TermContext;
 
 /**
  * A subclass of {@code Similarity} that provides a simplified API for its
@@ -71,12 +68,11 @@ public abstract class SimilarityBase ext
   }
   
   @Override
-  public final Stats computeStats(IndexSearcher searcher, String fieldName,
-      float queryBoost, TermContext... termContexts) throws IOException {
-    BasicStats stats[] = new BasicStats[termContexts.length];
-    for (int i = 0; i < termContexts.length; i++) {
+  public final Stats computeStats(CollectionStatistics collectionStats, float queryBoost, TermStatistics... termStats) {
+    BasicStats stats[] = new BasicStats[termStats.length];
+    for (int i = 0; i < termStats.length; i++) {
       stats[i] = newStats(queryBoost);
-      fillBasicStats(stats[i], searcher, fieldName, termContexts[i]);
+      fillBasicStats(stats[i], collectionStats, termStats[i]);
     }
     return stats.length == 1 ? stats[0] : new MultiSimilarity.MultiStats(stats);
   }
@@ -88,13 +84,11 @@ public abstract class SimilarityBase ext
   
   /** Fills all member fields defined in {@code BasicStats} in {@code stats}. 
    *  Subclasses can override this method to fill additional stats. */
-  protected void fillBasicStats(BasicStats stats, IndexSearcher searcher,
-      String fieldName, TermContext termContext) throws IOException {
-    IndexReader reader = searcher.getIndexReader();
-    int numberOfDocuments = reader.maxDoc();
+  protected void fillBasicStats(BasicStats stats, CollectionStatistics collectionStats, TermStatistics termStats) {
+    int numberOfDocuments = collectionStats.maxDoc();
     
-    int docFreq = termContext.docFreq();
-    long totalTermFreq = termContext.totalTermFreq();
+    int docFreq = termStats.docFreq();
+    long totalTermFreq = termStats.totalTermFreq();
 
     // codec does not supply totalTermFreq: substitute docFreq
     if (totalTermFreq == -1) {
@@ -103,25 +97,19 @@ public abstract class SimilarityBase ext
 
     final long numberOfFieldTokens;
     final float avgFieldLength;
-    
-    Terms terms = MultiFields.getTerms(searcher.getIndexReader(), fieldName);
-    if (terms == null) {
-      // field does not exist;
-      numberOfFieldTokens = 0;
-      avgFieldLength = 1;
-    } else {
-      long sumTotalTermFreq = terms.getSumTotalTermFreq();
 
+    long sumTotalTermFreq = collectionStats.sumTotalTermFreq();
+
+    if (sumTotalTermFreq <= 0) {
+      // field does not exist;
       // We have to provide something if codec doesnt supply these measures,
       // or if someone omitted frequencies for the field... negative values cause
       // NaN/Inf for some scorers.
-      if (sumTotalTermFreq == -1) {
-        numberOfFieldTokens = docFreq;
-        avgFieldLength = 1;
-      } else {
-        numberOfFieldTokens = sumTotalTermFreq;
-        avgFieldLength = (float)numberOfFieldTokens / numberOfDocuments;
-      }
+      numberOfFieldTokens = docFreq;
+      avgFieldLength = 1;
+    } else {
+      numberOfFieldTokens = sumTotalTermFreq;
+      avgFieldLength = (float)numberOfFieldTokens / numberOfDocuments;
     }
  
     // TODO: add sumDocFreq for field (numberOfFieldPostings)

Modified: lucene/dev/trunk/lucene/src/java/org/apache/lucene/search/similarities/TFIDFSimilarity.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/src/java/org/apache/lucene/search/similarities/TFIDFSimilarity.java?rev=1197455&r1=1197454&r2=1197455&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/src/java/org/apache/lucene/search/similarities/TFIDFSimilarity.java (original)
+++ lucene/dev/trunk/lucene/src/java/org/apache/lucene/search/similarities/TFIDFSimilarity.java Fri Nov  4 09:23:56 2011
@@ -22,9 +22,11 @@ import java.io.IOException;
 
 import org.apache.lucene.index.IndexReader.AtomicReaderContext;
 import org.apache.lucene.index.Term;
+import org.apache.lucene.search.CollectionStatistics;
 import org.apache.lucene.search.Explanation;
 import org.apache.lucene.search.IndexSearcher;
 import org.apache.lucene.search.PhraseQuery;
+import org.apache.lucene.search.TermStatistics;
 import org.apache.lucene.util.BytesRef;
 import org.apache.lucene.util.TermContext;
 import org.apache.lucene.util.SmallFloat;
@@ -575,15 +577,15 @@ public abstract class TFIDFSimilarity ex
    * is inaccurate, so is {@link IndexSearcher#maxDoc()}, and in the same direction.
    * In addition, {@link IndexSearcher#maxDoc()} is more efficient to compute
    *   
-   * @param stats statistics of the term in question
-   * @param searcher the document collection being searched
+   * @param collectionStats collection-level statistics
+   * @param termStats term-level statistics for the term
    * @return an Explain object that includes both an idf score factor 
              and an explanation for the term.
    * @throws IOException
    */
-  public Explanation idfExplain(TermContext stats, final IndexSearcher searcher) throws IOException {
-    final int df = stats.docFreq();
-    final int max = searcher.maxDoc();
+  public Explanation idfExplain(CollectionStatistics collectionStats, TermStatistics termStats) {
+    final int df = termStats.docFreq();
+    final int max = collectionStats.maxDoc();
     final float idf = idf(df, max);
     return new Explanation(idf, "idf(docFreq=" + df + ", maxDocs=" + max + ")");
   }
@@ -595,19 +597,19 @@ public abstract class TFIDFSimilarity ex
    * The default implementation sums the idf factor for
    * each term in the phrase.
    * 
-   * @param stats statistics of the terms in the phrase
-   * @param searcher the document collection being searched
+   * @param collectionStats collection-level statistics
+   * @param termStats term-level statistics for the terms in the phrase
    * @return an Explain object that includes both an idf 
    *         score factor for the phrase and an explanation 
    *         for each term.
    * @throws IOException
    */
-  public Explanation idfExplain(final TermContext stats[], IndexSearcher searcher) throws IOException {
-    final int max = searcher.maxDoc();
+  public Explanation idfExplain(CollectionStatistics collectionStats, TermStatistics termStats[]) {
+    final int max = collectionStats.maxDoc();
     float idf = 0.0f;
     final Explanation exp = new Explanation();
     exp.setDescription("idf(), sum of:");
-    for (final TermContext stat : stats ) {
+    for (final TermStatistics stat : termStats ) {
       final int df = stat.docFreq();
       final float termIdf = idf(df, max);
       exp.addDetail(new Explanation(termIdf, "idf(docFreq=" + df + ", maxDocs=" + max + ")"));
@@ -693,11 +695,10 @@ public abstract class TFIDFSimilarity ex
   public abstract float scorePayload(int doc, int start, int end, BytesRef payload);
 
   @Override
-  public final Stats computeStats(IndexSearcher searcher, String fieldName, float queryBoost,
-      TermContext... termContexts) throws IOException {
-    final Explanation idf = termContexts.length == 1
-    ? idfExplain(termContexts[0], searcher)
-    : idfExplain(termContexts, searcher);
+  public final Stats computeStats(CollectionStatistics collectionStats, float queryBoost, TermStatistics... termStats) {
+    final Explanation idf = termStats.length == 1
+    ? idfExplain(collectionStats, termStats[0])
+    : idfExplain(collectionStats, termStats);
     return new IDFStats(idf, queryBoost);
   }
 

Modified: lucene/dev/trunk/lucene/src/java/org/apache/lucene/search/spans/SpanWeight.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/src/java/org/apache/lucene/search/spans/SpanWeight.java?rev=1197455&r1=1197454&r2=1197455&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/src/java/org/apache/lucene/search/spans/SpanWeight.java (original)
+++ lucene/dev/trunk/lucene/src/java/org/apache/lucene/search/spans/SpanWeight.java Fri Nov  4 09:23:56 2011
@@ -48,10 +48,17 @@ public class SpanWeight extends Weight {
     query.extractTerms(terms);
     final ReaderContext context = searcher.getTopReaderContext();
     final TermContext states[] = new TermContext[terms.size()];
+    final TermStatistics termStats[] = new TermStatistics[terms.size()];
     int i = 0;
-    for (Term term : terms)
-      states[i++] = TermContext.build(context, term, true);
-    stats = similarity.computeStats(searcher, query.getField(), query.getBoost(), states);
+    for (Term term : terms) {
+      states[i] = TermContext.build(context, term, true);
+      termStats[i] = searcher.termStatistics(term, states[i]);
+      i++;
+    }
+    stats = similarity.computeStats(
+        searcher.collectionStatistics(query.getField()), 
+        query.getBoost(), 
+        termStats);
   }
 
   @Override

Modified: lucene/dev/trunk/lucene/src/test/org/apache/lucene/index/TestOmitTf.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/src/test/org/apache/lucene/index/TestOmitTf.java?rev=1197455&r1=1197454&r2=1197455&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/src/test/org/apache/lucene/index/TestOmitTf.java (original)
+++ lucene/dev/trunk/lucene/src/test/org/apache/lucene/index/TestOmitTf.java Fri Nov  4 09:23:56 2011
@@ -21,7 +21,6 @@ import java.io.IOException;
 
 import org.apache.lucene.util.BytesRef;
 import org.apache.lucene.util.LuceneTestCase;
-import org.apache.lucene.util.TermContext;
 import org.apache.lucene.analysis.Analyzer;
 import org.apache.lucene.analysis.MockAnalyzer;
 import org.apache.lucene.document.Document;
@@ -50,7 +49,7 @@ public class TestOmitTf extends LuceneTe
         @Override public float tf(float freq) { return freq; }
         @Override public float sloppyFreq(int distance) { return 2.0f; }
         @Override public float idf(int docFreq, int numDocs) { return 1.0f; }
-        @Override public Explanation idfExplain(TermContext[] terms, IndexSearcher searcher) throws IOException {
+        @Override public Explanation idfExplain(CollectionStatistics collectionStats, TermStatistics[] termStats) {
           return new Explanation(1.0f, "Inexplicable");
         }
         @Override public float scorePayload(int doc, int start, int end, BytesRef payload) { return 1.0f; }

Modified: lucene/dev/trunk/lucene/src/test/org/apache/lucene/search/JustCompileSearch.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/src/test/org/apache/lucene/search/JustCompileSearch.java?rev=1197455&r1=1197454&r2=1197455&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/src/test/org/apache/lucene/search/JustCompileSearch.java (original)
+++ lucene/dev/trunk/lucene/src/test/org/apache/lucene/search/JustCompileSearch.java Fri Nov  4 09:23:56 2011
@@ -22,12 +22,8 @@ import java.io.IOException;
 import org.apache.lucene.index.IndexReader.AtomicReaderContext;
 import org.apache.lucene.search.similarities.Similarity;
 import org.apache.lucene.search.similarities.SimilarityProvider;
-import org.apache.lucene.search.similarities.Similarity.ExactDocScorer;
-import org.apache.lucene.search.similarities.Similarity.SloppyDocScorer;
-import org.apache.lucene.search.similarities.Similarity.Stats;
 import org.apache.lucene.util.Bits;
 import org.apache.lucene.util.BytesRef;
-import org.apache.lucene.util.TermContext;
 import org.apache.lucene.index.FieldInvertState;
 import org.apache.lucene.util.PriorityQueue;
 
@@ -250,7 +246,7 @@ final class JustCompileSearch {
   static final class JustCompileSimilarity extends Similarity {
 
     @Override
-    public Stats computeStats(IndexSearcher searcher, String fieldName, float queryBoost, TermContext... termContexts) throws IOException {
+    public Stats computeStats(CollectionStatistics collectionStats, float queryBoost, TermStatistics... termStats) {
       throw new UnsupportedOperationException(UNSUPPORTED_MSG);
     }
 

Modified: lucene/dev/trunk/lucene/src/test/org/apache/lucene/search/TestDocValuesScoring.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/src/test/org/apache/lucene/search/TestDocValuesScoring.java?rev=1197455&r1=1197454&r2=1197455&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/src/test/org/apache/lucene/search/TestDocValuesScoring.java (original)
+++ lucene/dev/trunk/lucene/src/test/org/apache/lucene/search/TestDocValuesScoring.java Fri Nov  4 09:23:56 2011
@@ -161,8 +161,8 @@ public class TestDocValuesScoring extend
     }
 
     @Override
-    public Stats computeStats(IndexSearcher searcher, String fieldName, float queryBoost, TermContext... termContexts) throws IOException {
-      return sim.computeStats(searcher, fieldName, queryBoost, termContexts);
+    public Stats computeStats(CollectionStatistics collectionStats, float queryBoost, TermStatistics... termStats) {
+      return sim.computeStats(collectionStats, queryBoost, termStats);
     }
 
     @Override

Modified: lucene/dev/trunk/lucene/src/test/org/apache/lucene/search/TestMultiPhraseQuery.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/src/test/org/apache/lucene/search/TestMultiPhraseQuery.java?rev=1197455&r1=1197454&r2=1197455&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/src/test/org/apache/lucene/search/TestMultiPhraseQuery.java (original)
+++ lucene/dev/trunk/lucene/src/test/org/apache/lucene/search/TestMultiPhraseQuery.java Fri Nov  4 09:23:56 2011
@@ -316,8 +316,7 @@ public class TestMultiPhraseQuery extend
         return new DefaultSimilarity() {
           
           @Override
-          public Explanation idfExplain(TermContext stats[],
-              IndexSearcher searcher) throws IOException {
+          public Explanation idfExplain(CollectionStatistics collectionStats, TermStatistics termStats[]) {
             return new Explanation(10f, "just a test");
           } 
         };

Modified: lucene/dev/trunk/lucene/src/test/org/apache/lucene/search/TestSimilarity.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/src/test/org/apache/lucene/search/TestSimilarity.java?rev=1197455&r1=1197454&r2=1197455&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/src/test/org/apache/lucene/search/TestSimilarity.java (original)
+++ lucene/dev/trunk/lucene/src/test/org/apache/lucene/search/TestSimilarity.java Fri Nov  4 09:23:56 2011
@@ -18,7 +18,6 @@ package org.apache.lucene.search;
  */
 
 import org.apache.lucene.util.LuceneTestCase;
-import org.apache.lucene.util.TermContext;
 
 import java.io.IOException;
 
@@ -50,7 +49,7 @@ public class TestSimilarity extends Luce
         @Override public float tf(float freq) { return freq; }
         @Override public float sloppyFreq(int distance) { return 2.0f; }
         @Override public float idf(int docFreq, int numDocs) { return 1.0f; }
-        @Override public Explanation idfExplain(TermContext[] stats, IndexSearcher searcher) throws IOException {
+        @Override public Explanation idfExplain(CollectionStatistics collectionStats, TermStatistics[] stats) {
           return new Explanation(1.0f, "Inexplicable"); 
         }
       };

Modified: lucene/dev/trunk/lucene/src/test/org/apache/lucene/search/payloads/TestPayloadNearQuery.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/src/test/org/apache/lucene/search/payloads/TestPayloadNearQuery.java?rev=1197455&r1=1197454&r2=1197455&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/src/test/org/apache/lucene/search/payloads/TestPayloadNearQuery.java (original)
+++ lucene/dev/trunk/lucene/src/test/org/apache/lucene/search/payloads/TestPayloadNearQuery.java Fri Nov  4 09:23:56 2011
@@ -27,10 +27,12 @@ import org.apache.lucene.index.IndexRead
 import org.apache.lucene.index.Payload;
 import org.apache.lucene.index.RandomIndexWriter;
 import org.apache.lucene.index.Term;
+import org.apache.lucene.search.CollectionStatistics;
 import org.apache.lucene.search.Explanation;
 import org.apache.lucene.search.QueryUtils;
 import org.apache.lucene.search.ScoreDoc;
 import org.apache.lucene.search.IndexSearcher;
+import org.apache.lucene.search.TermStatistics;
 import org.apache.lucene.search.TopDocs;
 import org.apache.lucene.search.similarities.DefaultSimilarity;
 import org.apache.lucene.search.similarities.Similarity;
@@ -42,7 +44,6 @@ import org.apache.lucene.store.Directory
 import org.apache.lucene.util.BytesRef;
 import org.apache.lucene.util.English;
 import org.apache.lucene.util.LuceneTestCase;
-import org.apache.lucene.util.TermContext;
 import org.junit.AfterClass;
 import org.junit.BeforeClass;
 
@@ -346,7 +347,7 @@ public class TestPayloadNearQuery extend
     
         // idf used for phrase queries
         @Override 
-        public Explanation idfExplain(TermContext states[], IndexSearcher searcher) throws IOException {
+        public Explanation idfExplain(CollectionStatistics collectionStats, TermStatistics[] termStats) {
           return new Explanation(1.0f, "Inexplicable");
         }
       };

Modified: lucene/dev/trunk/lucene/src/test/org/apache/lucene/search/similarities/TestSimilarityBase.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/src/test/org/apache/lucene/search/similarities/TestSimilarityBase.java?rev=1197455&r1=1197454&r2=1197455&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/src/test/org/apache/lucene/search/similarities/TestSimilarityBase.java (original)
+++ lucene/dev/trunk/lucene/src/test/org/apache/lucene/search/similarities/TestSimilarityBase.java Fri Nov  4 09:23:56 2011
@@ -30,12 +30,15 @@ import org.apache.lucene.index.OrdTermSt
 import org.apache.lucene.index.RandomIndexWriter;
 import org.apache.lucene.index.Term;
 import org.apache.lucene.index.codecs.CodecProvider;
+import org.apache.lucene.search.CollectionStatistics;
 import org.apache.lucene.search.Explanation;
 import org.apache.lucene.search.IndexSearcher;
 import org.apache.lucene.search.Query;
 import org.apache.lucene.search.TermQuery;
+import org.apache.lucene.search.TermStatistics;
 import org.apache.lucene.search.TopDocs;
 import org.apache.lucene.store.Directory;
+import org.apache.lucene.util.BytesRef;
 import org.apache.lucene.util.LuceneTestCase;
 import org.apache.lucene.util.TermContext;
 import org.junit.Ignore;
@@ -172,7 +175,14 @@ public class TestSimilarityBase extends 
     stats.setTotalTermFreq(TOTAL_TERM_FREQ);
     return stats;
   }
-
+  
+  private CollectionStatistics toCollectionStats(BasicStats stats) {
+    return new CollectionStatistics("spoof", stats.getNumberOfDocuments(), -1, stats.getNumberOfFieldTokens(), -1);
+  }
+  
+  private TermStatistics toTermStats(BasicStats stats) {
+    return new TermStatistics(new BytesRef("spoofyText"), stats.getDocFreq(), stats.getTotalTermFreq());
+  }
   /**
    * The generic test core called by all unit test methods. It calls the
    * {@link SimilarityBase#score(BasicStats, float, int)} method of all
@@ -180,17 +190,11 @@ public class TestSimilarityBase extends 
    * is a finite positive real number.
    */
   private void unitTestCore(BasicStats stats, float freq, int docLen)
-      throws IOException {
-    // We have to fake everything, because computeStats() can be overridden and
-    // there is no way to inject false data after fillBasicStats().
-    SpoofIndexSearcher searcher = new SpoofIndexSearcher(stats);
-    TermContext tc = new TermContext(
-        searcher.getIndexReader().getTopReaderContext(),
-        new OrdTermState(), 0, stats.getDocFreq(), stats.getTotalTermFreq());
-    
+      throws IOException { 
     for (SimilarityBase sim : sims) {
-      BasicStats realStats = (BasicStats) sim.computeStats(new SpoofIndexSearcher(stats),
-          "spoof", stats.getTotalBoost(), tc);
+      BasicStats realStats = (BasicStats) sim.computeStats(toCollectionStats(stats), 
+          stats.getTotalBoost(),
+          toTermStats(stats));
       float score = sim.score(realStats, freq, docLen);
       float explScore = sim.explain(
           realStats, 1, new Explanation(freq, "freq"), docLen).getValue();
@@ -520,16 +524,10 @@ public class TestSimilarityBase extends 
    */
   private void correctnessTestCore(SimilarityBase sim, float gold)
       throws IOException {
-    // We have to fake everything, because computeStats() can be overridden and
-    // there is no way to inject false data after fillBasicStats().
     BasicStats stats = createStats();
-    SpoofIndexSearcher searcher = new SpoofIndexSearcher(stats);
-    TermContext tc = new TermContext(
-        searcher.getIndexReader().getTopReaderContext(),
-        new OrdTermState(), 0, stats.getDocFreq(), stats.getTotalTermFreq());
-    
-    BasicStats realStats = (BasicStats) sim.computeStats(
-        searcher, "spoof", stats.getTotalBoost(), tc);
+    BasicStats realStats = (BasicStats) sim.computeStats(toCollectionStats(stats), 
+        stats.getTotalBoost(),
+        toTermStats(stats));
     float score = sim.score(realStats, FREQ, DOC_LEN);
     assertEquals(
         sim.toString() + " score not correct.", gold, score, FLOAT_EPSILON);