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/03/28 04:58:20 UTC

svn commit: r1086107 [1/2] - in /lucene/dev/branches/flexscoring: lucene/contrib/misc/src/test/org/apache/lucene/misc/ lucene/contrib/queries/src/java/org/apache/lucene/search/ lucene/contrib/queries/src/java/org/apache/lucene/search/similar/ lucene/sr...

Author: rmuir
Date: Mon Mar 28 02:58:19 2011
New Revision: 1086107

URL: http://svn.apache.org/viewvc?rev=1086107&view=rev
Log:
LUCENE-2392: commit my current state

Added:
    lucene/dev/branches/flexscoring/lucene/src/java/org/apache/lucene/search/TFIDFSimilarity.java   (with props)
    lucene/dev/branches/flexscoring/lucene/src/test/org/apache/lucene/search/MockLMSimilarity.java   (with props)
    lucene/dev/branches/flexscoring/lucene/src/test/org/apache/lucene/search/MockLMSimilarityProvider.java   (with props)
Modified:
    lucene/dev/branches/flexscoring/lucene/contrib/misc/src/test/org/apache/lucene/misc/SweetSpotSimilarityTest.java
    lucene/dev/branches/flexscoring/lucene/contrib/queries/src/java/org/apache/lucene/search/FuzzyLikeThisQuery.java
    lucene/dev/branches/flexscoring/lucene/contrib/queries/src/java/org/apache/lucene/search/similar/MoreLikeThis.java
    lucene/dev/branches/flexscoring/lucene/src/java/org/apache/lucene/search/ConstantScoreAutoRewrite.java
    lucene/dev/branches/flexscoring/lucene/src/java/org/apache/lucene/search/DefaultSimilarity.java
    lucene/dev/branches/flexscoring/lucene/src/java/org/apache/lucene/search/ExactPhraseScorer.java
    lucene/dev/branches/flexscoring/lucene/src/java/org/apache/lucene/search/MultiPhraseQuery.java
    lucene/dev/branches/flexscoring/lucene/src/java/org/apache/lucene/search/PhraseQuery.java
    lucene/dev/branches/flexscoring/lucene/src/java/org/apache/lucene/search/PhraseScorer.java
    lucene/dev/branches/flexscoring/lucene/src/java/org/apache/lucene/search/ScoringRewrite.java
    lucene/dev/branches/flexscoring/lucene/src/java/org/apache/lucene/search/Similarity.java
    lucene/dev/branches/flexscoring/lucene/src/java/org/apache/lucene/search/SloppyPhraseScorer.java
    lucene/dev/branches/flexscoring/lucene/src/java/org/apache/lucene/search/TermQuery.java
    lucene/dev/branches/flexscoring/lucene/src/java/org/apache/lucene/search/TermScorer.java
    lucene/dev/branches/flexscoring/lucene/src/java/org/apache/lucene/search/TopTermsRewrite.java
    lucene/dev/branches/flexscoring/lucene/src/java/org/apache/lucene/search/payloads/PayloadNearQuery.java
    lucene/dev/branches/flexscoring/lucene/src/java/org/apache/lucene/search/payloads/PayloadTermQuery.java
    lucene/dev/branches/flexscoring/lucene/src/java/org/apache/lucene/search/spans/SpanScorer.java
    lucene/dev/branches/flexscoring/lucene/src/java/org/apache/lucene/search/spans/SpanWeight.java
    lucene/dev/branches/flexscoring/lucene/src/java/org/apache/lucene/util/PerReaderTermState.java
    lucene/dev/branches/flexscoring/lucene/src/test/org/apache/lucene/index/TestOmitTf.java
    lucene/dev/branches/flexscoring/lucene/src/test/org/apache/lucene/search/JustCompileSearch.java
    lucene/dev/branches/flexscoring/lucene/src/test/org/apache/lucene/search/TestMultiPhraseQuery.java
    lucene/dev/branches/flexscoring/lucene/src/test/org/apache/lucene/search/TestSimilarity.java
    lucene/dev/branches/flexscoring/lucene/src/test/org/apache/lucene/search/TestSimilarityProvider.java
    lucene/dev/branches/flexscoring/lucene/src/test/org/apache/lucene/search/payloads/TestPayloadNearQuery.java
    lucene/dev/branches/flexscoring/lucene/src/test/org/apache/lucene/search/spans/JustCompileSearchSpans.java
    lucene/dev/branches/flexscoring/solr/src/java/org/apache/solr/search/function/IDFValueSource.java
    lucene/dev/branches/flexscoring/solr/src/java/org/apache/solr/search/function/TFValueSource.java
    lucene/dev/branches/flexscoring/solr/src/test/org/apache/solr/search/function/TestFunctionQuery.java

Modified: lucene/dev/branches/flexscoring/lucene/contrib/misc/src/test/org/apache/lucene/misc/SweetSpotSimilarityTest.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/flexscoring/lucene/contrib/misc/src/test/org/apache/lucene/misc/SweetSpotSimilarityTest.java?rev=1086107&r1=1086106&r2=1086107&view=diff
==============================================================================
--- lucene/dev/branches/flexscoring/lucene/contrib/misc/src/test/org/apache/lucene/misc/SweetSpotSimilarityTest.java (original)
+++ lucene/dev/branches/flexscoring/lucene/contrib/misc/src/test/org/apache/lucene/misc/SweetSpotSimilarityTest.java Mon Mar 28 02:58:19 2011
@@ -21,6 +21,7 @@ package org.apache.lucene.misc;
 import org.apache.lucene.search.DefaultSimilarity;
 import org.apache.lucene.search.DefaultSimilarityProvider;
 import org.apache.lucene.search.Similarity;
+import org.apache.lucene.search.TFIDFSimilarity;
 import org.apache.lucene.search.SimilarityProvider;
 import org.apache.lucene.util.LuceneTestCase;
 import org.apache.lucene.index.FieldInvertState;
@@ -170,8 +171,8 @@ public class SweetSpotSimilarityTest ext
   
     SweetSpotSimilarity ss = new SweetSpotSimilarity();
 
-    Similarity d = new DefaultSimilarity();
-    Similarity s = ss;
+    TFIDFSimilarity d = new DefaultSimilarity();
+    TFIDFSimilarity s = ss;
     
     // tf equal
 
@@ -222,7 +223,7 @@ public class SweetSpotSimilarityTest ext
       };
     ss.setHyperbolicTfFactors(3.3f, 7.7f, Math.E, 5.0f);
     
-    Similarity s = ss;
+    TFIDFSimilarity s = ss;
 
     for (int i = 1; i <=1000; i++) {
       assertTrue("MIN tf: i="+i+" : s="+s.tf(i),

Modified: lucene/dev/branches/flexscoring/lucene/contrib/queries/src/java/org/apache/lucene/search/FuzzyLikeThisQuery.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/flexscoring/lucene/contrib/queries/src/java/org/apache/lucene/search/FuzzyLikeThisQuery.java?rev=1086107&r1=1086106&r2=1086107&view=diff
==============================================================================
--- lucene/dev/branches/flexscoring/lucene/contrib/queries/src/java/org/apache/lucene/search/FuzzyLikeThisQuery.java (original)
+++ lucene/dev/branches/flexscoring/lucene/contrib/queries/src/java/org/apache/lucene/search/FuzzyLikeThisQuery.java Mon Mar 28 02:58:19 2011
@@ -51,7 +51,8 @@ import org.apache.lucene.util.PriorityQu
  */
 public class FuzzyLikeThisQuery extends Query
 {
-    static Similarity sim=new DefaultSimilarity();
+    //nocommit? this query is pretty much hardcoded at TF/IDF
+    static TFIDFSimilarity sim=new DefaultSimilarity();
     Query rewrittenQuery=null;
     ArrayList<FieldVals> fieldVals=new ArrayList<FieldVals>();
     Analyzer analyzer;

Modified: lucene/dev/branches/flexscoring/lucene/contrib/queries/src/java/org/apache/lucene/search/similar/MoreLikeThis.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/flexscoring/lucene/contrib/queries/src/java/org/apache/lucene/search/similar/MoreLikeThis.java?rev=1086107&r1=1086106&r2=1086107&view=diff
==============================================================================
--- lucene/dev/branches/flexscoring/lucene/contrib/queries/src/java/org/apache/lucene/search/similar/MoreLikeThis.java (original)
+++ lucene/dev/branches/flexscoring/lucene/contrib/queries/src/java/org/apache/lucene/search/similar/MoreLikeThis.java Mon Mar 28 02:58:19 2011
@@ -44,6 +44,7 @@ import org.apache.lucene.search.IndexSea
 import org.apache.lucene.search.Query;
 import org.apache.lucene.search.ScoreDoc;
 import org.apache.lucene.search.Similarity;
+import org.apache.lucene.search.TFIDFSimilarity;
 import org.apache.lucene.search.TermQuery;
 import org.apache.lucene.search.TopDocs;
 import org.apache.lucene.store.FSDirectory;
@@ -284,7 +285,8 @@ public final class MoreLikeThis {
     /**
      * For idf() calculations.
      */
-    private Similarity similarity;// = new DefaultSimilarity();
+    // nocommit? this is pretty much wired to tf-idf things...
+    private TFIDFSimilarity similarity;// = new DefaultSimilarity();
 
     /**
      * IndexReader to use
@@ -319,17 +321,17 @@ public final class MoreLikeThis {
         this(ir, new DefaultSimilarity());
     }
 
-    public MoreLikeThis(IndexReader ir, Similarity sim){
+    public MoreLikeThis(IndexReader ir, TFIDFSimilarity sim){
       this.ir = ir;
       this.similarity = sim;
     }
 
 
-  public Similarity getSimilarity() {
+  public TFIDFSimilarity getSimilarity() {
     return similarity;
   }
 
-  public void setSimilarity(Similarity similarity) {
+  public void setSimilarity(TFIDFSimilarity similarity) {
     this.similarity = similarity;
   }
 

Modified: lucene/dev/branches/flexscoring/lucene/src/java/org/apache/lucene/search/ConstantScoreAutoRewrite.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/flexscoring/lucene/src/java/org/apache/lucene/search/ConstantScoreAutoRewrite.java?rev=1086107&r1=1086106&r2=1086107&view=diff
==============================================================================
--- lucene/dev/branches/flexscoring/lucene/src/java/org/apache/lucene/search/ConstantScoreAutoRewrite.java (original)
+++ lucene/dev/branches/flexscoring/lucene/src/java/org/apache/lucene/search/ConstantScoreAutoRewrite.java Mon Mar 28 02:58:19 2011
@@ -141,9 +141,9 @@ class ConstantScoreAutoRewrite extends T
       assert termState != null;
       if (pos < 0) {
         pos = (-pos)-1;
-        array.termState[pos].register(termState, readerContext.ord, termsEnum.docFreq());
+        array.termState[pos].register(termState, readerContext.ord, termsEnum.docFreq(), termsEnum.totalTermFreq());
       } else {
-        array.termState[pos] = new PerReaderTermState(topReaderContext, termState, readerContext.ord, termsEnum.docFreq());
+        array.termState[pos] = new PerReaderTermState(topReaderContext, termState, readerContext.ord, termsEnum.docFreq(), termsEnum.totalTermFreq());
       }
       return true;
     }

Modified: lucene/dev/branches/flexscoring/lucene/src/java/org/apache/lucene/search/DefaultSimilarity.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/flexscoring/lucene/src/java/org/apache/lucene/search/DefaultSimilarity.java?rev=1086107&r1=1086106&r2=1086107&view=diff
==============================================================================
--- lucene/dev/branches/flexscoring/lucene/src/java/org/apache/lucene/search/DefaultSimilarity.java (original)
+++ lucene/dev/branches/flexscoring/lucene/src/java/org/apache/lucene/search/DefaultSimilarity.java Mon Mar 28 02:58:19 2011
@@ -20,7 +20,7 @@ import org.apache.lucene.index.FieldInve
  */
 
 /** Expert: Default scoring implementation. */
-public class DefaultSimilarity extends Similarity {
+public class DefaultSimilarity extends TFIDFSimilarity {
 
   /** Implemented as
    *  <code>state.getBoost()*lengthNorm(numTerms)</code>, where
@@ -39,7 +39,7 @@ public class DefaultSimilarity extends S
       numTerms = state.getLength();
     return state.getBoost() * ((float) (1.0 / Math.sqrt(numTerms)));
   }
-  
+
   /** Implemented as <code>sqrt(freq)</code>. */
   @Override
   public float tf(float freq) {

Modified: lucene/dev/branches/flexscoring/lucene/src/java/org/apache/lucene/search/ExactPhraseScorer.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/flexscoring/lucene/src/java/org/apache/lucene/search/ExactPhraseScorer.java?rev=1086107&r1=1086106&r2=1086107&view=diff
==============================================================================
--- lucene/dev/branches/flexscoring/lucene/src/java/org/apache/lucene/search/ExactPhraseScorer.java (original)
+++ lucene/dev/branches/flexscoring/lucene/src/java/org/apache/lucene/search/ExactPhraseScorer.java Mon Mar 28 02:58:19 2011
@@ -21,14 +21,9 @@ import java.io.IOException;
 import java.util.Arrays;
 
 import org.apache.lucene.index.*;
+import org.apache.lucene.index.IndexReader.AtomicReaderContext;
 
 final class ExactPhraseScorer extends Scorer {
-  private final byte[] norms;
-  private final float value;
-
-  private static final int SCORE_CACHE_SIZE = 32;
-  private final float[] scoreCache = new float[SCORE_CACHE_SIZE];
-
   private final int endMinus1;
 
   private final static int CHUNK = 4096;
@@ -60,14 +55,12 @@ final class ExactPhraseScorer extends Sc
   private int docID = -1;
   private int freq;
 
-  private final Similarity similarity;
+  private final Similarity.ExactDocScorer docScorer;
   
   ExactPhraseScorer(Weight weight, PhraseQuery.PostingsAndFreq[] postings,
-                    Similarity similarity, byte[] norms) throws IOException {
+                    Similarity similarity, String field, AtomicReaderContext context) throws IOException {
     super(weight);
-    this.similarity = similarity;
-    this.norms = norms;
-    this.value = weight.getValue();
+    this.docScorer = similarity.exactDocScorer(weight, field, context);
 
     chunkStates = new ChunkState[postings.length];
 
@@ -88,10 +81,6 @@ final class ExactPhraseScorer extends Sc
         return;
       }
     }
-
-    for (int i = 0; i < SCORE_CACHE_SIZE; i++) {
-      scoreCache[i] = similarity.tf((float) i) * value;
-    }
   }
 
   @Override
@@ -206,13 +195,7 @@ final class ExactPhraseScorer extends Sc
 
   @Override
   public float score() throws IOException {
-    final float raw; // raw score
-    if (freq < SCORE_CACHE_SIZE) {
-      raw = scoreCache[freq];
-    } else {
-      raw = similarity.tf((float) freq) * value;
-    }
-    return norms == null ? raw : raw * similarity.decodeNormValue(norms[docID]); // normalize
+    return docScorer.score(docID, freq);
   }
 
   private int phraseFreq() throws IOException {

Modified: lucene/dev/branches/flexscoring/lucene/src/java/org/apache/lucene/search/MultiPhraseQuery.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/flexscoring/lucene/src/java/org/apache/lucene/search/MultiPhraseQuery.java?rev=1086107&r1=1086106&r2=1086107&view=diff
==============================================================================
--- lucene/dev/branches/flexscoring/lucene/src/java/org/apache/lucene/search/MultiPhraseQuery.java (original)
+++ lucene/dev/branches/flexscoring/lucene/src/java/org/apache/lucene/search/MultiPhraseQuery.java Mon Mar 28 02:58:19 2011
@@ -22,12 +22,14 @@ import java.util.*;
 
 import org.apache.lucene.index.IndexReader;
 import org.apache.lucene.index.IndexReader.AtomicReaderContext;
+import org.apache.lucene.index.IndexReader.ReaderContext;
 import org.apache.lucene.index.Term;
 import org.apache.lucene.index.DocsEnum;
 import org.apache.lucene.index.DocsAndPositionsEnum;
 import org.apache.lucene.search.Explanation.IDFExplanation;
 import org.apache.lucene.util.ArrayUtil;
 import org.apache.lucene.util.BytesRef;
+import org.apache.lucene.util.PerReaderTermState;
 import org.apache.lucene.util.ToStringUtils;
 import org.apache.lucene.util.PriorityQueue;
 import org.apache.lucene.util.Bits;
@@ -140,15 +142,16 @@ public class MultiPhraseQuery extends Qu
     public MultiPhraseWeight(IndexSearcher searcher)
       throws IOException {
       this.similarity = searcher.getSimilarityProvider().get(field);
-
+      final ReaderContext context = searcher.getTopReaderContext();
+      
       // compute idf
-      ArrayList<Term> allTerms = new ArrayList<Term>();
+      ArrayList<PerReaderTermState> allTerms = new ArrayList<PerReaderTermState>();
       for(final Term[] terms: termArrays) {
         for (Term term: terms) {
-          allTerms.add(term);
+          allTerms.add(PerReaderTermState.build(context, term, true));
         }
       }
-      idfExp = similarity.idfExplain(allTerms, searcher);
+      idfExp = similarity.computeWeight(searcher, field, allTerms.toArray(new PerReaderTermState[allTerms.size()]));
       idf = idfExp.getIdf();
     }
 
@@ -223,8 +226,7 @@ public class MultiPhraseQuery extends Qu
       }
 
       if (slop == 0) {
-        ExactPhraseScorer s = new ExactPhraseScorer(this, postingsFreqs, similarity,
-            reader.norms(field));
+        ExactPhraseScorer s = new ExactPhraseScorer(this, postingsFreqs, similarity, field, context);
         if (s.noDocs) {
           return null;
         } else {
@@ -232,13 +234,18 @@ public class MultiPhraseQuery extends Qu
         }
       } else {
         return new SloppyPhraseScorer(this, postingsFreqs, similarity,
-                                      slop, reader.norms(field));
+                                      slop, field, context);
       }
     }
 
     @Override
     public Explanation explain(AtomicReaderContext context, int doc)
       throws IOException {
+      //nocommit: fix explains
+      if (!(similarity instanceof TFIDFSimilarity))
+        return new ComplexExplanation();
+      final TFIDFSimilarity similarity = (TFIDFSimilarity) this.similarity;
+      
       ComplexExplanation result = new ComplexExplanation();
       result.setDescription("weight("+getQuery()+" in "+doc+"), product of:");
 

Modified: lucene/dev/branches/flexscoring/lucene/src/java/org/apache/lucene/search/PhraseQuery.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/flexscoring/lucene/src/java/org/apache/lucene/search/PhraseQuery.java?rev=1086107&r1=1086106&r2=1086107&view=diff
==============================================================================
--- lucene/dev/branches/flexscoring/lucene/src/java/org/apache/lucene/search/PhraseQuery.java (original)
+++ lucene/dev/branches/flexscoring/lucene/src/java/org/apache/lucene/search/PhraseQuery.java Mon Mar 28 02:58:19 2011
@@ -22,10 +22,14 @@ import java.util.Set;
 import java.util.ArrayList;
 
 import org.apache.lucene.index.IndexReader.AtomicReaderContext;
+import org.apache.lucene.index.IndexReader.ReaderContext;
 import org.apache.lucene.index.Term;
 import org.apache.lucene.index.DocsAndPositionsEnum;
 import org.apache.lucene.index.IndexReader;
+import org.apache.lucene.index.TermState;
+import org.apache.lucene.index.TermsEnum;
 import org.apache.lucene.search.Explanation.IDFExplanation;
+import org.apache.lucene.util.PerReaderTermState;
 import org.apache.lucene.util.ToStringUtils;
 import org.apache.lucene.util.ArrayUtil;
 import org.apache.lucene.util.Bits;
@@ -143,12 +147,16 @@ public class PhraseQuery extends Query {
     private float queryNorm;
     private float queryWeight;
     private IDFExplanation idfExp;
+    private transient PerReaderTermState states[];
 
     public PhraseWeight(IndexSearcher searcher)
       throws IOException {
       this.similarity = searcher.getSimilarityProvider().get(field);
-
-      idfExp = similarity.idfExplain(terms, searcher);
+      final ReaderContext context = searcher.getTopReaderContext();
+      states = new PerReaderTermState[terms.size()];
+      for (int i = 0; i < terms.size(); i++)
+        states[i] = PerReaderTermState.build(context, terms.get(i), true);
+      idfExp = similarity.computeWeight(searcher, field, states);
       idf = idfExp.getIdf();
     }
 
@@ -183,21 +191,29 @@ public class PhraseQuery extends Query {
       final Bits delDocs = reader.getDeletedDocs();
       for (int i = 0; i < terms.size(); i++) {
         final Term t = terms.get(i);
+        final TermState state = states[i].get(context.ord);
+        if (state == null) /* term doesnt exist in this segment */
+          return null;
         DocsAndPositionsEnum postingsEnum = reader.termPositionsEnum(delDocs,
                                                                      t.field(),
-                                                                     t.bytes());
+                                                                     t.bytes(),
+                                                                     state);
         // PhraseQuery on a field that did not index
         // positions.
         if (postingsEnum == null) {
-          if (reader.termDocsEnum(delDocs, t.field(), t.bytes()) != null) {
+          if (reader.termDocsEnum(delDocs, t.field(), t.bytes(), state) != null) {
             // term does exist, but has no positions
             throw new IllegalStateException("field \"" + t.field() + "\" was indexed with Field.omitTermFreqAndPositions=true; cannot run PhraseQuery (term=" + t.text() + ")");
           } else {
             // term does not exist
+            // nocommit: should be impossible, state should be null?
             return null;
           }
         }
-        postingsFreqs[i] = new PostingsAndFreq(postingsEnum, reader.docFreq(t.field(), t.bytes()), positions.get(i).intValue());
+        // get the docFreq without seeking
+        TermsEnum te = reader.fields().terms(field).getThreadTermsEnum();
+        te.seek(t.bytes(), state);
+        postingsFreqs[i] = new PostingsAndFreq(postingsEnum, te.docFreq(), positions.get(i).intValue());
       }
 
       // sort by increasing docFreq order
@@ -206,8 +222,7 @@ public class PhraseQuery extends Query {
       }
 
       if (slop == 0) {				  // optimize exact case
-        ExactPhraseScorer s = new ExactPhraseScorer(this, postingsFreqs, similarity,
-            reader.norms(field));
+        ExactPhraseScorer s = new ExactPhraseScorer(this, postingsFreqs, similarity, field, context);
         if (s.noDocs) {
           return null;
         } else {
@@ -215,15 +230,18 @@ public class PhraseQuery extends Query {
         }
       } else {
         return
-          new SloppyPhraseScorer(this, postingsFreqs, similarity, slop,
-              reader.norms(field));
+          new SloppyPhraseScorer(this, postingsFreqs, similarity, slop, field, context);
       }
     }
 
     @Override
     public Explanation explain(AtomicReaderContext context, int doc)
       throws IOException {
-
+      //nocommit: fix explains
+      if (!(similarity instanceof TFIDFSimilarity))
+        return new ComplexExplanation();
+      final TFIDFSimilarity similarity = (TFIDFSimilarity) this.similarity;
+      
       ComplexExplanation result = new ComplexExplanation();
       result.setDescription("weight("+getQuery()+" in "+doc+"), product of:");
 

Modified: lucene/dev/branches/flexscoring/lucene/src/java/org/apache/lucene/search/PhraseScorer.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/flexscoring/lucene/src/java/org/apache/lucene/search/PhraseScorer.java?rev=1086107&r1=1086106&r2=1086107&view=diff
==============================================================================
--- lucene/dev/branches/flexscoring/lucene/src/java/org/apache/lucene/search/PhraseScorer.java (original)
+++ lucene/dev/branches/flexscoring/lucene/src/java/org/apache/lucene/search/PhraseScorer.java Mon Mar 28 02:58:19 2011
@@ -19,6 +19,8 @@ package org.apache.lucene.search;
 
 import java.io.IOException;
 
+import org.apache.lucene.index.IndexReader.AtomicReaderContext;
+
 /** Expert: Scoring functionality for phrase queries.
  * <br>A document is considered matching if it contains the phrase-query terms  
  * at "valid" positions. What "valid positions" are
@@ -40,14 +42,12 @@ abstract class PhraseScorer extends Scor
 
   private float freq; //phrase frequency in current doc as computed by phraseFreq().
 
-  protected final Similarity similarity;
+  protected final Similarity.SloppyDocScorer docScorer;
 
   PhraseScorer(Weight weight, PhraseQuery.PostingsAndFreq[] postings,
-      Similarity similarity, byte[] norms) {
+      Similarity similarity, String field, AtomicReaderContext context) throws IOException {
     super(weight);
-    this.similarity = similarity;
-    this.norms = norms;
-    this.value = weight.getValue();
+    docScorer = similarity.sloppyDocScorer(weight, field, context);
 
     // convert tps to a list of phrase positions.
     // note: phrase-position differs from term-position in that its position
@@ -107,9 +107,7 @@ abstract class PhraseScorer extends Scor
 
   @Override
   public float score() throws IOException {
-    //System.out.println("scoring " + first.doc);
-    float raw = similarity.tf(freq) * value; // raw score
-    return norms == null ? raw : raw * similarity.decodeNormValue(norms[first.doc]); // normalize
+    return docScorer.score(first.doc, freq);
   }
 
   @Override

Modified: lucene/dev/branches/flexscoring/lucene/src/java/org/apache/lucene/search/ScoringRewrite.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/flexscoring/lucene/src/java/org/apache/lucene/search/ScoringRewrite.java?rev=1086107&r1=1086106&r2=1086107&view=diff
==============================================================================
--- lucene/dev/branches/flexscoring/lucene/src/java/org/apache/lucene/search/ScoringRewrite.java (original)
+++ lucene/dev/branches/flexscoring/lucene/src/java/org/apache/lucene/search/ScoringRewrite.java Mon Mar 28 02:58:19 2011
@@ -151,12 +151,12 @@ public abstract class ScoringRewrite<Q e
       if (e < 0 ) {
         // duplicate term: update docFreq
         final int pos = (-e)-1;
-        array.termState[pos].register(state, readerContext.ord, termsEnum.docFreq());
+        array.termState[pos].register(state, readerContext.ord, termsEnum.docFreq(), termsEnum.totalTermFreq());
         assert array.boost[pos] == boostAtt.getBoost() : "boost should be equal in all segment TermsEnums";
       } else {
         // new entry: we populate the entry initially
         array.boost[e] = boostAtt.getBoost();
-        array.termState[e] = new PerReaderTermState(topReaderContext, state, readerContext.ord, termsEnum.docFreq());
+        array.termState[e] = new PerReaderTermState(topReaderContext, state, readerContext.ord, termsEnum.docFreq(), termsEnum.totalTermFreq());
         ScoringRewrite.this.checkMaxClauseCount(terms.size());
       }
       return true;

Modified: lucene/dev/branches/flexscoring/lucene/src/java/org/apache/lucene/search/Similarity.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/flexscoring/lucene/src/java/org/apache/lucene/search/Similarity.java?rev=1086107&r1=1086106&r2=1086107&view=diff
==============================================================================
--- lucene/dev/branches/flexscoring/lucene/src/java/org/apache/lucene/search/Similarity.java (original)
+++ lucene/dev/branches/flexscoring/lucene/src/java/org/apache/lucene/search/Similarity.java Mon Mar 28 02:58:19 2011
@@ -19,12 +19,11 @@ package org.apache.lucene.search;
 
 
 import java.io.IOException;
-import java.util.Collection;
 
 import org.apache.lucene.index.FieldInvertState;
-import org.apache.lucene.index.Term;
+import org.apache.lucene.index.IndexReader.AtomicReaderContext;
 import org.apache.lucene.search.Explanation.IDFExplanation;
-import org.apache.lucene.util.SmallFloat;
+import org.apache.lucene.util.PerReaderTermState;
 
 
 /** 
@@ -34,493 +33,6 @@ import org.apache.lucene.util.SmallFloat
  * Overriding computation of these components is a convenient
  * way to alter Lucene scoring.
  *
- * <p>Suggested reading:
- * <a href="http://nlp.stanford.edu/IR-book/html/htmledition/queries-as-vectors-1.html">
- * Introduction To Information Retrieval, Chapter 6</a>.
- *
- * <p>The following describes how Lucene scoring evolves from
- * underlying information retrieval models to (efficient) implementation.
- * We first brief on <i>VSM Score</i>, 
- * then derive from it <i>Lucene's Conceptual Scoring Formula</i>,
- * from which, finally, evolves <i>Lucene's Practical Scoring Function</i> 
- * (the latter is connected directly with Lucene classes and methods).    
- *
- * <p>Lucene combines
- * <a href="http://en.wikipedia.org/wiki/Standard_Boolean_model">
- * Boolean model (BM) of Information Retrieval</a>
- * with
- * <a href="http://en.wikipedia.org/wiki/Vector_Space_Model">
- * Vector Space Model (VSM) of Information Retrieval</a> -
- * documents "approved" by BM are scored by VSM.
- *
- * <p>In VSM, documents and queries are represented as
- * weighted vectors in a multi-dimensional space,
- * where each distinct index term is a dimension,
- * and weights are
- * <a href="http://en.wikipedia.org/wiki/Tfidf">Tf-idf</a> values.
- *
- * <p>VSM does not require weights to be <i>Tf-idf</i> values,
- * but <i>Tf-idf</i> values are believed to produce search results of high quality,
- * and so Lucene is using <i>Tf-idf</i>.
- * <i>Tf</i> and <i>Idf</i> are described in more detail below,
- * but for now, for completion, let's just say that
- * for given term <i>t</i> and document (or query) <i>x</i>,
- * <i>Tf(t,x)</i> varies with the number of occurrences of term <i>t</i> in <i>x</i>
- * (when one increases so does the other) and
- * <i>idf(t)</i> similarly varies with the inverse of the
- * number of index documents containing term <i>t</i>.
- *
- * <p><i>VSM score</i> of document <i>d</i> for query <i>q</i> is the
- * <a href="http://en.wikipedia.org/wiki/Cosine_similarity">
- * Cosine Similarity</a>
- * of the weighted query vectors <i>V(q)</i> and <i>V(d)</i>:
- *
- *  <br>&nbsp;<br>
- *  <table cellpadding="2" cellspacing="2" border="0" align="center">
- *    <tr><td>
- *    <table cellpadding="1" cellspacing="0" border="1" align="center">
- *      <tr><td>
- *      <table cellpadding="2" cellspacing="2" border="0" align="center">
- *        <tr>
- *          <td valign="middle" align="right" rowspan="1">
- *            cosine-similarity(q,d) &nbsp; = &nbsp;
- *          </td>
- *          <td valign="middle" align="center">
- *            <table>
- *               <tr><td align="center"><small>V(q)&nbsp;&middot;&nbsp;V(d)</small></td></tr>
- *               <tr><td align="center">&ndash;&ndash;&ndash;&ndash;&ndash;&ndash;&ndash;&ndash;&ndash;</td></tr>
- *               <tr><td align="center"><small>|V(q)|&nbsp;|V(d)|</small></td></tr>
- *            </table>
- *          </td>
- *        </tr>
- *      </table>
- *      </td></tr>
- *    </table>
- *    </td></tr>
- *    <tr><td>
- *    <center><font=-1><u>VSM Score</u></font></center>
- *    </td></tr>
- *  </table>
- *  <br>&nbsp;<br>
- *   
- *
- * Where <i>V(q)</i> &middot; <i>V(d)</i> is the
- * <a href="http://en.wikipedia.org/wiki/Dot_product">dot product</a>
- * of the weighted vectors,
- * and <i>|V(q)|</i> and <i>|V(d)|</i> are their
- * <a href="http://en.wikipedia.org/wiki/Euclidean_norm#Euclidean_norm">Euclidean norms</a>.
- *
- * <p>Note: the above equation can be viewed as the dot product of
- * the normalized weighted vectors, in the sense that dividing
- * <i>V(q)</i> by its euclidean norm is normalizing it to a unit vector.
- *
- * <p>Lucene refines <i>VSM score</i> for both search quality and usability:
- * <ul>
- *  <li>Normalizing <i>V(d)</i> to the unit vector is known to be problematic in that 
- *  it removes all document length information. 
- *  For some documents removing this info is probably ok, 
- *  e.g. a document made by duplicating a certain paragraph <i>10</i> times,
- *  especially if that paragraph is made of distinct terms. 
- *  But for a document which contains no duplicated paragraphs, 
- *  this might be wrong. 
- *  To avoid this problem, a different document length normalization 
- *  factor is used, which normalizes to a vector equal to or larger 
- *  than the unit vector: <i>doc-len-norm(d)</i>.
- *  </li>
- *
- *  <li>At indexing, users can specify that certain documents are more
- *  important than others, by assigning a document boost.
- *  For this, the score of each document is also multiplied by its boost value
- *  <i>doc-boost(d)</i>.
- *  </li>
- *
- *  <li>Lucene is field based, hence each query term applies to a single
- *  field, document length normalization is by the length of the certain field,
- *  and in addition to document boost there are also document fields boosts.
- *  </li>
- *
- *  <li>The same field can be added to a document during indexing several times,
- *  and so the boost of that field is the multiplication of the boosts of
- *  the separate additions (or parts) of that field within the document.
- *  </li>
- *
- *  <li>At search time users can specify boosts to each query, sub-query, and
- *  each query term, hence the contribution of a query term to the score of
- *  a document is multiplied by the boost of that query term <i>query-boost(q)</i>.
- *  </li>
- *
- *  <li>A document may match a multi term query without containing all
- *  the terms of that query (this is correct for some of the queries),
- *  and users can further reward documents matching more query terms
- *  through a coordination factor, which is usually larger when
- *  more terms are matched: <i>coord-factor(q,d)</i>.
- *  </li>
- * </ul>
- *
- * <p>Under the simplifying assumption of a single field in the index,
- * we get <i>Lucene's Conceptual scoring formula</i>:
- *
- *  <br>&nbsp;<br>
- *  <table cellpadding="2" cellspacing="2" border="0" align="center">
- *    <tr><td>
- *    <table cellpadding="1" cellspacing="0" border="1" align="center">
- *      <tr><td>
- *      <table cellpadding="2" cellspacing="2" border="0" align="center">
- *        <tr>
- *          <td valign="middle" align="right" rowspan="1">
- *            score(q,d) &nbsp; = &nbsp;
- *            <font color="#FF9933">coord-factor(q,d)</font> &middot; &nbsp;
- *            <font color="#CCCC00">query-boost(q)</font> &middot; &nbsp;
- *          </td>
- *          <td valign="middle" align="center">
- *            <table>
- *               <tr><td align="center"><small><font color="#993399">V(q)&nbsp;&middot;&nbsp;V(d)</font></small></td></tr>
- *               <tr><td align="center">&ndash;&ndash;&ndash;&ndash;&ndash;&ndash;&ndash;&ndash;&ndash;</td></tr>
- *               <tr><td align="center"><small><font color="#FF33CC">|V(q)|</font></small></td></tr>
- *            </table>
- *          </td>
- *          <td valign="middle" align="right" rowspan="1">
- *            &nbsp; &middot; &nbsp; <font color="#3399FF">doc-len-norm(d)</font>
- *            &nbsp; &middot; &nbsp; <font color="#3399FF">doc-boost(d)</font>
- *          </td>
- *        </tr>
- *      </table>
- *      </td></tr>
- *    </table>
- *    </td></tr>
- *    <tr><td>
- *    <center><font=-1><u>Lucene Conceptual Scoring Formula</u></font></center>
- *    </td></tr>
- *  </table>
- *  <br>&nbsp;<br>
- *
- * <p>The conceptual formula is a simplification in the sense that (1) terms and documents
- * are fielded and (2) boosts are usually per query term rather than per query.
- *
- * <p>We now describe how Lucene implements this conceptual scoring formula, and
- * derive from it <i>Lucene's Practical Scoring Function</i>.
- *  
- * <p>For efficient score computation some scoring components
- * are computed and aggregated in advance:
- *
- * <ul>
- *  <li><i>Query-boost</i> for the query (actually for each query term)
- *  is known when search starts.
- *  </li>
- *
- *  <li>Query Euclidean norm <i>|V(q)|</i> can be computed when search starts,
- *  as it is independent of the document being scored.
- *  From search optimization perspective, it is a valid question
- *  why bother to normalize the query at all, because all
- *  scored documents will be multiplied by the same <i>|V(q)|</i>,
- *  and hence documents ranks (their order by score) will not
- *  be affected by this normalization.
- *  There are two good reasons to keep this normalization:
- *  <ul>
- *   <li>Recall that
- *   <a href="http://en.wikipedia.org/wiki/Cosine_similarity">
- *   Cosine Similarity</a> can be used find how similar
- *   two documents are. One can use Lucene for e.g.
- *   clustering, and use a document as a query to compute
- *   its similarity to other documents.
- *   In this use case it is important that the score of document <i>d3</i>
- *   for query <i>d1</i> is comparable to the score of document <i>d3</i>
- *   for query <i>d2</i>. In other words, scores of a document for two
- *   distinct queries should be comparable.
- *   There are other applications that may require this.
- *   And this is exactly what normalizing the query vector <i>V(q)</i>
- *   provides: comparability (to a certain extent) of two or more queries.
- *   </li>
- *
- *   <li>Applying query normalization on the scores helps to keep the
- *   scores around the unit vector, hence preventing loss of score data
- *   because of floating point precision limitations.
- *   </li>
- *  </ul>
- *  </li>
- *
- *  <li>Document length norm <i>doc-len-norm(d)</i> and document
- *  boost <i>doc-boost(d)</i> are known at indexing time.
- *  They are computed in advance and their multiplication
- *  is saved as a single value in the index: <i>norm(d)</i>.
- *  (In the equations below, <i>norm(t in d)</i> means <i>norm(field(t) in doc d)</i>
- *  where <i>field(t)</i> is the field associated with term <i>t</i>.)
- *  </li>
- * </ul>
- *
- * <p><i>Lucene's Practical Scoring Function</i> is derived from the above.
- * The color codes demonstrate how it relates
- * to those of the <i>conceptual</i> formula:
- *
- * <P>
- * <table cellpadding="2" cellspacing="2" border="0" align="center">
- *  <tr><td>
- *  <table cellpadding="" cellspacing="2" border="2" align="center">
- *  <tr><td>
- *   <table cellpadding="2" cellspacing="2" border="0" align="center">
- *   <tr>
- *     <td valign="middle" align="right" rowspan="1">
- *       score(q,d) &nbsp; = &nbsp;
- *       <A HREF="#formula_coord"><font color="#FF9933">coord(q,d)</font></A> &nbsp;&middot;&nbsp;
- *       <A HREF="#formula_queryNorm"><font color="#FF33CC">queryNorm(q)</font></A> &nbsp;&middot;&nbsp;
- *     </td>
- *     <td valign="bottom" align="center" rowspan="1">
- *       <big><big><big>&sum;</big></big></big>
- *     </td>
- *     <td valign="middle" align="right" rowspan="1">
- *       <big><big>(</big></big>
- *       <A HREF="#formula_tf"><font color="#993399">tf(t in d)</font></A> &nbsp;&middot;&nbsp;
- *       <A HREF="#formula_idf"><font color="#993399">idf(t)</font></A><sup>2</sup> &nbsp;&middot;&nbsp;
- *       <A HREF="#formula_termBoost"><font color="#CCCC00">t.getBoost()</font></A>&nbsp;&middot;&nbsp;
- *       <A HREF="#formula_norm"><font color="#3399FF">norm(t,d)</font></A>
- *       <big><big>)</big></big>
- *     </td>
- *   </tr>
- *   <tr valigh="top">
- *    <td></td>
- *    <td align="center"><small>t in q</small></td>
- *    <td></td>
- *   </tr>
- *   </table>
- *  </td></tr>
- *  </table>
- * </td></tr>
- * <tr><td>
- *  <center><font=-1><u>Lucene Practical Scoring Function</u></font></center>
- * </td></tr>
- * </table>
- *
- * <p> where
- * <ol>
- *    <li>
- *      <A NAME="formula_tf"></A>
- *      <b><i>tf(t in d)</i></b>
- *      correlates to the term's <i>frequency</i>,
- *      defined as the number of times term <i>t</i> appears in the currently scored document <i>d</i>.
- *      Documents that have more occurrences of a given term receive a higher score.
- *      Note that <i>tf(t in q)</i> is assumed to be <i>1</i> and therefore it does not appear in this equation,
- *      However if a query contains twice the same term, there will be
- *      two term-queries with that same term and hence the computation would still be correct (although
- *      not very efficient).
- *      The default computation for <i>tf(t in d)</i> in
- *      {@link org.apache.lucene.search.DefaultSimilarity#tf(float) DefaultSimilarity} is:
- *
- *      <br>&nbsp;<br>
- *      <table cellpadding="2" cellspacing="2" border="0" align="center">
- *        <tr>
- *          <td valign="middle" align="right" rowspan="1">
- *            {@link org.apache.lucene.search.DefaultSimilarity#tf(float) tf(t in d)} &nbsp; = &nbsp;
- *          </td>
- *          <td valign="top" align="center" rowspan="1">
- *               frequency<sup><big>&frac12;</big></sup>
- *          </td>
- *        </tr>
- *      </table>
- *      <br>&nbsp;<br>
- *    </li>
- *
- *    <li>
- *      <A NAME="formula_idf"></A>
- *      <b><i>idf(t)</i></b> stands for Inverse Document Frequency. This value
- *      correlates to the inverse of <i>docFreq</i>
- *      (the number of documents in which the term <i>t</i> appears).
- *      This means rarer terms give higher contribution to the total score.
- *      <i>idf(t)</i> appears for <i>t</i> in both the query and the document,
- *      hence it is squared in the equation.
- *      The default computation for <i>idf(t)</i> in
- *      {@link org.apache.lucene.search.DefaultSimilarity#idf(int, int) DefaultSimilarity} is:
- *
- *      <br>&nbsp;<br>
- *      <table cellpadding="2" cellspacing="2" border="0" align="center">
- *        <tr>
- *          <td valign="middle" align="right">
- *            {@link org.apache.lucene.search.DefaultSimilarity#idf(int, int) idf(t)}&nbsp; = &nbsp;
- *          </td>
- *          <td valign="middle" align="center">
- *            1 + log <big>(</big>
- *          </td>
- *          <td valign="middle" align="center">
- *            <table>
- *               <tr><td align="center"><small>numDocs</small></td></tr>
- *               <tr><td align="center">&ndash;&ndash;&ndash;&ndash;&ndash;&ndash;&ndash;&ndash;&ndash;</td></tr>
- *               <tr><td align="center"><small>docFreq+1</small></td></tr>
- *            </table>
- *          </td>
- *          <td valign="middle" align="center">
- *            <big>)</big>
- *          </td>
- *        </tr>
- *      </table>
- *      <br>&nbsp;<br>
- *    </li>
- *
- *    <li>
- *      <A NAME="formula_coord"></A>
- *      <b><i>coord(q,d)</i></b>
- *      is a score factor based on how many of the query terms are found in the specified document.
- *      Typically, a document that contains more of the query's terms will receive a higher score
- *      than another document with fewer query terms.
- *      This is a search time factor computed in
- *      {@link SimilarityProvider#coord(int, int) coord(q,d)}
- *      by the SimilarityProvider in effect at search time.
- *      <br>&nbsp;<br>
- *    </li>
- *
- *    <li><b>
- *      <A NAME="formula_queryNorm"></A>
- *      <i>queryNorm(q)</i>
- *      </b>
- *      is a normalizing factor used to make scores between queries comparable.
- *      This factor does not affect document ranking (since all ranked documents are multiplied by the same factor),
- *      but rather just attempts to make scores from different queries (or even different indexes) comparable.
- *      This is a search time factor computed by the SimilarityProvider in effect at search time.
- *
- *      The default computation in
- *      {@link org.apache.lucene.search.DefaultSimilarityProvider#queryNorm(float) DefaultSimilarityProvider}
- *      produces a <a href="http://en.wikipedia.org/wiki/Euclidean_norm#Euclidean_norm">Euclidean norm</a>:
- *      <br>&nbsp;<br>
- *      <table cellpadding="1" cellspacing="0" border="0" align="center">
- *        <tr>
- *          <td valign="middle" align="right" rowspan="1">
- *            queryNorm(q)  &nbsp; = &nbsp;
- *            {@link org.apache.lucene.search.DefaultSimilarityProvider#queryNorm(float) queryNorm(sumOfSquaredWeights)}
- *            &nbsp; = &nbsp;
- *          </td>
- *          <td valign="middle" align="center" rowspan="1">
- *            <table>
- *               <tr><td align="center"><big>1</big></td></tr>
- *               <tr><td align="center"><big>
- *                  &ndash;&ndash;&ndash;&ndash;&ndash;&ndash;&ndash;&ndash;&ndash;&ndash;&ndash;&ndash;&ndash;&ndash;
- *               </big></td></tr>
- *               <tr><td align="center">sumOfSquaredWeights<sup><big>&frac12;</big></sup></td></tr>
- *            </table>
- *          </td>
- *        </tr>
- *      </table>
- *      <br>&nbsp;<br>
- *
- *      The sum of squared weights (of the query terms) is
- *      computed by the query {@link org.apache.lucene.search.Weight} object.
- *      For example, a {@link org.apache.lucene.search.BooleanQuery}
- *      computes this value as:
- *
- *      <br>&nbsp;<br>
- *      <table cellpadding="1" cellspacing="0" border="0"n align="center">
- *        <tr>
- *          <td valign="middle" align="right" rowspan="1">
- *            {@link org.apache.lucene.search.Weight#sumOfSquaredWeights() sumOfSquaredWeights} &nbsp; = &nbsp;
- *            {@link org.apache.lucene.search.Query#getBoost() q.getBoost()} <sup><big>2</big></sup>
- *            &nbsp;&middot;&nbsp;
- *          </td>
- *          <td valign="bottom" align="center" rowspan="1">
- *            <big><big><big>&sum;</big></big></big>
- *          </td>
- *          <td valign="middle" align="right" rowspan="1">
- *            <big><big>(</big></big>
- *            <A HREF="#formula_idf">idf(t)</A> &nbsp;&middot;&nbsp;
- *            <A HREF="#formula_termBoost">t.getBoost()</A>
- *            <big><big>) <sup>2</sup> </big></big>
- *          </td>
- *        </tr>
- *        <tr valigh="top">
- *          <td></td>
- *          <td align="center"><small>t in q</small></td>
- *          <td></td>
- *        </tr>
- *      </table>
- *      <br>&nbsp;<br>
- *
- *    </li>
- *
- *    <li>
- *      <A NAME="formula_termBoost"></A>
- *      <b><i>t.getBoost()</i></b>
- *      is a search time boost of term <i>t</i> in the query <i>q</i> as
- *      specified in the query text
- *      (see <A HREF="../../../../../../queryparsersyntax.html#Boosting a Term">query syntax</A>),
- *      or as set by application calls to
- *      {@link org.apache.lucene.search.Query#setBoost(float) setBoost()}.
- *      Notice that there is really no direct API for accessing a boost of one term in a multi term query,
- *      but rather multi terms are represented in a query as multi
- *      {@link org.apache.lucene.search.TermQuery TermQuery} objects,
- *      and so the boost of a term in the query is accessible by calling the sub-query
- *      {@link org.apache.lucene.search.Query#getBoost() getBoost()}.
- *      <br>&nbsp;<br>
- *    </li>
- *
- *    <li>
- *      <A NAME="formula_norm"></A>
- *      <b><i>norm(t,d)</i></b> encapsulates a few (indexing time) boost and length factors:
- *
- *      <ul>
- *        <li><b>Document boost</b> - set by calling
- *        {@link org.apache.lucene.document.Document#setBoost(float) doc.setBoost()}
- *        before adding the document to the index.
- *        </li>
- *        <li><b>Field boost</b> - set by calling
- *        {@link org.apache.lucene.document.Fieldable#setBoost(float) field.setBoost()}
- *        before adding the field to a document.
- *        </li>
- *        <li><b>lengthNorm</b> - computed
- *        when the document is added to the index in accordance with the number of tokens
- *        of this field in the document, so that shorter fields contribute more to the score.
- *        LengthNorm is computed by the Similarity class in effect at indexing.
- *        </li>
- *      </ul>
- *      The {@link #computeNorm} method is responsible for
- *      combining all of these factors into a single float.
- *
- *      <p>
- *      When a document is added to the index, all the above factors are multiplied.
- *      If the document has multiple fields with the same name, all their boosts are multiplied together:
- *
- *      <br>&nbsp;<br>
- *      <table cellpadding="1" cellspacing="0" border="0"n align="center">
- *        <tr>
- *          <td valign="middle" align="right" rowspan="1">
- *            norm(t,d) &nbsp; = &nbsp;
- *            {@link org.apache.lucene.document.Document#getBoost() doc.getBoost()}
- *            &nbsp;&middot;&nbsp;
- *            lengthNorm
- *            &nbsp;&middot;&nbsp;
- *          </td>
- *          <td valign="bottom" align="center" rowspan="1">
- *            <big><big><big>&prod;</big></big></big>
- *          </td>
- *          <td valign="middle" align="right" rowspan="1">
- *            {@link org.apache.lucene.document.Fieldable#getBoost() f.getBoost}()
- *          </td>
- *        </tr>
- *        <tr valigh="top">
- *          <td></td>
- *          <td align="center"><small>field <i><b>f</b></i> in <i>d</i> named as <i><b>t</b></i></small></td>
- *          <td></td>
- *        </tr>
- *      </table>
- *      <br>&nbsp;<br>
- *      However the resulted <i>norm</i> value is {@link #encodeNormValue(float) encoded} as a single byte
- *      before being stored.
- *      At search time, the norm byte value is read from the index
- *      {@link org.apache.lucene.store.Directory directory} and
- *      {@link #decodeNormValue(byte) decoded} back to a float <i>norm</i> value.
- *      This encoding/decoding, while reducing index size, comes with the price of
- *      precision loss - it is not guaranteed that <i>decode(encode(x)) = x</i>.
- *      For instance, <i>decode(encode(0.89)) = 0.75</i>.
- *      <br>&nbsp;<br>
- *      Compression of norm values to a single byte saves memory at search time, 
- *      because once a field is referenced at search time, its norms - for 
- *      all documents - are maintained in memory.
- *      <br>&nbsp;<br>
- *      The rationale supporting such lossy compression of norm values is that
- *      given the difficulty (and inaccuracy) of users to express their true information
- *      need by a query, only big differences matter.
- *      <br>&nbsp;<br>
- *      Last, note that search time is too late to modify this <i>norm</i> part of scoring, e.g. by
- *      using a different {@link Similarity} for search.
- *      <br>&nbsp;<br>
- *    </li>
- * </ol>
- *
  * @see org.apache.lucene.index.IndexWriterConfig#setSimilarityProvider(SimilarityProvider)
  * @see IndexSearcher#setSimilarityProvider(SimilarityProvider)
  */
@@ -528,20 +40,10 @@ public abstract class Similarity {
   
   public static final int NO_DOC_ID_PROVIDED = -1;
 
-  /** Cache of decoded bytes. */
-  private static final float[] NORM_TABLE = new float[256];
-
-  static {
-    for (int i = 0; i < 256; i++)
-      NORM_TABLE[i] = SmallFloat.byte315ToFloat((byte)i);
-  }
-
   /** Decodes a normalization factor stored in an index.
    * @see #encodeNormValue(float)
    */
-  public float decodeNormValue(byte b) {
-    return NORM_TABLE[b & 0xFF];  // & 0xFF maps negative bytes to positive above 127
-  }
+  public abstract float decodeNormValue(byte b);
 
   /**
    * Computes the normalization value for a field, given the accumulated
@@ -569,40 +71,11 @@ public abstract class Similarity {
   public abstract float computeNorm(FieldInvertState state);
   
   /** Encodes a normalization factor for storage in an index.
-   *
-   * <p>The encoding uses a three-bit mantissa, a five-bit exponent, and
-   * the zero-exponent point at 15, thus
-   * representing values from around 7x10^9 to 2x10^-9 with about one
-   * significant decimal digit of accuracy.  Zero is also represented.
-   * Negative numbers are rounded up to zero.  Values too large to represent
-   * are rounded down to the largest representable value.  Positive values too
-   * small to represent are rounded up to the smallest positive representable
-   * value.
+   * 
    * @see org.apache.lucene.document.Field#setBoost(float)
    * @see org.apache.lucene.util.SmallFloat
    */
-  public byte encodeNormValue(float f) {
-    return SmallFloat.floatToByte315(f);
-  }
-  
-  /** Computes a score factor based on a term or phrase's frequency in a
-   * document.  This value is multiplied by the {@link #idf(int, int)}
-   * factor for each term in the query and these products are then summed to
-   * form the initial score for a document.
-   *
-   * <p>Terms and phrases repeated in a document indicate the topic of the
-   * document, so implementations of this method usually return larger values
-   * when <code>freq</code> is large, and smaller values when <code>freq</code>
-   * is small.
-   *
-   * <p>The default implementation calls {@link #tf(float)}.
-   *
-   * @param freq the frequency of a term within a document
-   * @return a score factor based on a term's within-document frequency
-   */
-  public float tf(int freq) {
-    return tf((float)freq);
-  }
+  public abstract byte encodeNormValue(float f);
 
   /** Computes the amount of a sloppy phrase match, based on an edit distance.
    * This value is summed for each sloppy phrase match in a document to form
@@ -619,124 +92,6 @@ public abstract class Similarity {
    */
   public abstract float sloppyFreq(int distance);
 
-  /** Computes a score factor based on a term or phrase's frequency in a
-   * document.  This value is multiplied by the {@link #idf(int, int)}
-   * factor for each term in the query and these products are then summed to
-   * form the initial score for a document.
-   *
-   * <p>Terms and phrases repeated in a document indicate the topic of the
-   * document, so implementations of this method usually return larger values
-   * when <code>freq</code> is large, and smaller values when <code>freq</code>
-   * is small.
-   *
-   * @param freq the frequency of a term within a document
-   * @return a score factor based on a term's within-document frequency
-   */
-  public abstract float tf(float freq);
-
-  /**
-   * Computes a score factor for a simple term and returns an explanation
-   * for that score factor.
-   * 
-   * <p>
-   * The default implementation uses:
-   * 
-   * <pre>
-   * idf(docFreq, searcher.maxDoc());
-   * </pre>
-   * 
-   * Note that {@link IndexSearcher#maxDoc()} is used instead of
-   * {@link org.apache.lucene.index.IndexReader#numDocs() IndexReader#numDocs()} because also 
-   * {@link IndexSearcher#docFreq(Term)} is used, and when the latter 
-   * is inaccurate, so is {@link IndexSearcher#maxDoc()}, and in the same direction.
-   * In addition, {@link IndexSearcher#maxDoc()} is more efficient to compute
-   *   
-   * @param term the term in question
-   * @param searcher the document collection being searched
-   * @param docFreq externally computed docFreq for this term
-   * @return an IDFExplain object that includes both an idf score factor 
-             and an explanation for the term.
-   * @throws IOException
-   */
-  public IDFExplanation idfExplain(final Term term, final IndexSearcher searcher, int docFreq) throws IOException {
-    final int df = docFreq;
-    final int max = searcher.maxDoc();
-    final float idf = idf(df, max);
-    return new IDFExplanation() {
-        @Override
-        public String explain() {
-          return "idf(docFreq=" + df +
-          ", maxDocs=" + max + ")";
-        }
-        @Override
-        public float getIdf() {
-          return idf;
-        }};
-  }
-
-  /**
-   * This method forwards to {@link
-   * #idfExplain(Term,IndexSearcher,int)} by passing
-   * <code>searcher.docFreq(term)</code> as the docFreq.
-   */
-  public IDFExplanation idfExplain(final Term term, final IndexSearcher searcher) throws IOException {
-    return idfExplain(term, searcher, searcher.docFreq(term));
-  }
-
-  /**
-   * Computes a score factor for a phrase.
-   * 
-   * <p>
-   * The default implementation sums the idf factor for
-   * each term in the phrase.
-   * 
-   * @param terms the terms in the phrase
-   * @param searcher the document collection being searched
-   * @return an IDFExplain object that includes both an idf 
-   *         score factor for the phrase and an explanation 
-   *         for each term.
-   * @throws IOException
-   */
-  public IDFExplanation idfExplain(Collection<Term> terms, IndexSearcher searcher) throws IOException {
-    final int max = searcher.maxDoc();
-    float idf = 0.0f;
-    final StringBuilder exp = new StringBuilder();
-    for (final Term term : terms ) {
-      final int df = searcher.docFreq(term);
-      idf += idf(df, max);
-      exp.append(" ");
-      exp.append(term.text());
-      exp.append("=");
-      exp.append(df);
-    }
-    final float fIdf = idf;
-    return new IDFExplanation() {
-      @Override
-      public float getIdf() {
-        return fIdf;
-      }
-      @Override
-      public String explain() {
-        return exp.toString();
-      }
-    };
-  }
-
-  /** Computes a score factor based on a term's document frequency (the number
-   * of documents which contain the term).  This value is multiplied by the
-   * {@link #tf(int)} factor for each term in the query and these products are
-   * then summed to form the initial score for a document.
-   *
-   * <p>Terms that occur in fewer documents are better indicators of topic, so
-   * implementations of this method usually return larger values for rare terms,
-   * and smaller values for common terms.
-   *
-   * @param docFreq the number of documents which contain the term
-   * @param numDocs the total number of documents in the collection
-   * @return a score factor based on the term's document frequency
-   */
-  public abstract float idf(int docFreq, int numDocs);
-
   /**
    * Calculate a scoring factor based on the data in the payload.  Overriding implementations
    * are responsible for interpreting what is in the payload.  Lucene makes no assumptions about
@@ -758,5 +113,17 @@ public abstract class Similarity {
   {
     return 1;
   }
-
+  
+  public abstract IDFExplanation computeWeight(IndexSearcher searcher, String fieldName, PerReaderTermState... termStats) throws IOException;
+  
+  public abstract ExactDocScorer exactDocScorer(Weight weight, String fieldName, AtomicReaderContext context) throws IOException;
+  public abstract SloppyDocScorer sloppyDocScorer(Weight weight, String fieldName, AtomicReaderContext context) throws IOException;
+  
+  public abstract class ExactDocScorer {
+    public abstract float score(int doc, int freq);
+  }
+  
+  public abstract class SloppyDocScorer {
+    public abstract float score(int doc, float freq);
+  }
 }

Modified: lucene/dev/branches/flexscoring/lucene/src/java/org/apache/lucene/search/SloppyPhraseScorer.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/flexscoring/lucene/src/java/org/apache/lucene/search/SloppyPhraseScorer.java?rev=1086107&r1=1086106&r2=1086107&view=diff
==============================================================================
--- lucene/dev/branches/flexscoring/lucene/src/java/org/apache/lucene/search/SloppyPhraseScorer.java (original)
+++ lucene/dev/branches/flexscoring/lucene/src/java/org/apache/lucene/search/SloppyPhraseScorer.java Mon Mar 28 02:58:19 2011
@@ -20,16 +20,20 @@ package org.apache.lucene.search;
 import java.io.IOException;
 import java.util.HashMap;
 
+import org.apache.lucene.index.IndexReader.AtomicReaderContext;
+
 final class SloppyPhraseScorer extends PhraseScorer {
     private int slop;
     private PhrasePositions repeats[];
     private PhrasePositions tmpPos[]; // for flipping repeating pps.
     private boolean checkedRepeats;
-
+    private final Similarity similarity;
+    
     SloppyPhraseScorer(Weight weight, PhraseQuery.PostingsAndFreq[] postings, Similarity similarity,
-                       int slop, byte[] norms) {
-        super(weight, postings, similarity, norms);
+                       int slop, String field, AtomicReaderContext context) throws IOException {
+        super(weight, postings, similarity, field, context);
         this.slop = slop;
+        this.similarity = similarity;
     }
 
     /**