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

[2/3] lucene-solr:master: LUCENE-8116: SimScorer now only takes a frequency and a norm as per-document scoring factors.

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/8fd7ead9/lucene/core/src/java/org/apache/lucene/search/similarities/TFIDFSimilarity.java
----------------------------------------------------------------------
diff --git a/lucene/core/src/java/org/apache/lucene/search/similarities/TFIDFSimilarity.java b/lucene/core/src/java/org/apache/lucene/search/similarities/TFIDFSimilarity.java
index 0452371..de317d0 100644
--- a/lucene/core/src/java/org/apache/lucene/search/similarities/TFIDFSimilarity.java
+++ b/lucene/core/src/java/org/apache/lucene/search/similarities/TFIDFSimilarity.java
@@ -22,8 +22,6 @@ import java.util.ArrayList;
 import java.util.List;
 
 import org.apache.lucene.index.FieldInvertState;
-import org.apache.lucene.index.LeafReaderContext;
-import org.apache.lucene.index.NumericDocValues;
 import org.apache.lucene.search.CollectionStatistics;
 import org.apache.lucene.search.Explanation;
 import org.apache.lucene.search.IndexSearcher;
@@ -511,7 +509,7 @@ public abstract class TFIDFSimilarity extends Similarity {
   }
 
   @Override
-  public final SimWeight computeWeight(float boost, CollectionStatistics collectionStats, TermStatistics... termStats) {
+  public final SimScorer scorer(float boost, CollectionStatistics collectionStats, TermStatistics... termStats) {
     final Explanation idf = termStats.length == 1
     ? idfExplain(collectionStats, termStats[0])
     : idfExplain(collectionStats, termStats);
@@ -522,110 +520,69 @@ public abstract class TFIDFSimilarity extends Similarity {
       normTable[i] = norm;
     }
     normTable[0] = 1f / normTable[255];
-    return new IDFStats(collectionStats.field(), boost, idf, normTable);
+    return new TFIDFScorer(collectionStats.field(), boost, idf, normTable);
   }
 
-  @Override
-  public final SimScorer simScorer(SimWeight stats, LeafReaderContext context) throws IOException {
-    IDFStats idfstats = (IDFStats) stats;
-    // the norms only encode the length, we need a translation table that depends on how lengthNorm is implemented
-    final float[] normTable = idfstats.normTable;
-    return new TFIDFSimScorer(idfstats, context.reader().getNormValues(idfstats.field), normTable);
-  }
-  
-  private final class TFIDFSimScorer extends SimScorer {
-    private final IDFStats stats;
-    private final float weightValue;
-    private final NumericDocValues norms;
-    private final float[] normTable;
-    
-    TFIDFSimScorer(IDFStats stats, NumericDocValues norms, float[] normTable) throws IOException {
-      this.stats = stats;
-      this.weightValue = stats.queryWeight;
-      this.norms = norms;
-      this.normTable = normTable;
-    }
-    
-    @Override
-    public float score(int doc, float freq) throws IOException {
-      final float raw = tf(freq) * weightValue; // compute tf(f)*weight
-
-      if (norms == null) {
-        return raw;
-      } else {
-        boolean found = norms.advanceExact(doc);
-        assert found;
-        float normValue = normTable[(int) (norms.longValue() & 0xFF)];
-        return raw * normValue;  // normalize for field
-      }
-    }
-
-    @Override
-    public float maxScore(float maxFreq) {
-      final float raw = tf(maxFreq) * weightValue;
-      if (norms == null) {
-        return raw;
-      } else {
-        float maxNormValue = Float.NEGATIVE_INFINITY;
-        for (float norm : normTable) {
-          maxNormValue = Math.max(maxNormValue, norm);
-        }
-        return raw * maxNormValue;
-      }
-    }
-
-    @Override
-    public Explanation explain(int doc, Explanation freq) throws IOException {
-      return explainScore(doc, freq, stats, norms, normTable);
-    }
-  }
   
   /** Collection statistics for the TF-IDF model. The only statistic of interest
    * to this model is idf. */
-  static class IDFStats extends SimWeight {
-    private final String field;
+  class TFIDFScorer extends SimScorer {
     /** The idf and its explanation */
     private final Explanation idf;
     private final float boost;
     private final float queryWeight;
     final float[] normTable;
     
-    public IDFStats(String field, float boost, Explanation idf, float[] normTable) {
+    public TFIDFScorer(String field, float boost, Explanation idf, float[] normTable) {
+      super(field);
       // TODO: Validate?
-      this.field = field;
       this.idf = idf;
       this.boost = boost;
       this.queryWeight = boost * idf.getValue().floatValue();
       this.normTable = normTable;
     }
-  }  
 
-  private Explanation explainScore(int doc, Explanation freq, IDFStats stats, NumericDocValues norms, float[] normTable) throws IOException {
-    List<Explanation> subs = new ArrayList<Explanation>();
-    if (stats.boost != 1F) {
-      subs.add(Explanation.match(stats.boost, "boost"));
+    @Override
+    public float score(float freq, long norm) throws IOException {
+      final float raw = tf(freq) * queryWeight; // compute tf(f)*weight
+      float normValue = normTable[(int) (norm & 0xFF)];
+      return raw * normValue;  // normalize for field
     }
-    subs.add(stats.idf);
-    Explanation tf = Explanation.match(tf(freq.getValue().floatValue()), "tf(freq="+freq.getValue()+"), with freq of:", freq);
-    subs.add(tf);
 
-    float norm;
-    if (norms == null) {
-      norm = 1f;
-    } else {
-      boolean found = norms.advanceExact(doc);
-      assert found;
-      norm = normTable[(int) (norms.longValue() & 0xFF)];
+    @Override
+    public float maxScore(float maxFreq) {
+      final float raw = tf(maxFreq) * queryWeight;
+      float maxNormValue = Float.NEGATIVE_INFINITY;
+      for (float norm : normTable) {
+        maxNormValue = Math.max(maxNormValue, norm);
+      }
+      return raw * maxNormValue;
     }
-    
-    Explanation fieldNorm = Explanation.match(
-        norm,
-        "fieldNorm(doc=" + doc + ")");
-    subs.add(fieldNorm);
-    
-    return Explanation.match(
-        stats.queryWeight * tf.getValue().floatValue() * norm,
-        "score(doc="+doc+",freq="+freq.getValue()+"), product of:",
-        subs);
-  }
+
+    @Override
+    public Explanation explain(Explanation freq, long norm) throws IOException {
+      return explainScore(freq, norm, normTable);
+    }
+
+    private Explanation explainScore(Explanation freq, long encodedNorm, float[] normTable) throws IOException {
+      List<Explanation> subs = new ArrayList<Explanation>();
+      if (boost != 1F) {
+        subs.add(Explanation.match(boost, "boost"));
+      }
+      subs.add(idf);
+      Explanation tf = Explanation.match(tf(freq.getValue().floatValue()), "tf(freq="+freq.getValue()+"), with freq of:", freq);
+      subs.add(tf);
+
+      float norm = normTable[(int) (encodedNorm & 0xFF)];
+      
+      Explanation fieldNorm = Explanation.match(norm, "fieldNorm");
+      subs.add(fieldNorm);
+      
+      return Explanation.match(
+          queryWeight * tf.getValue().floatValue() * norm,
+          "score(freq="+freq.getValue()+"), product of:",
+          subs);
+    }
+  }  
+
 }

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/8fd7ead9/lucene/core/src/java/org/apache/lucene/search/spans/SpanScorer.java
----------------------------------------------------------------------
diff --git a/lucene/core/src/java/org/apache/lucene/search/spans/SpanScorer.java b/lucene/core/src/java/org/apache/lucene/search/spans/SpanScorer.java
index 57a68e4..044ac7a 100644
--- a/lucene/core/src/java/org/apache/lucene/search/spans/SpanScorer.java
+++ b/lucene/core/src/java/org/apache/lucene/search/spans/SpanScorer.java
@@ -21,9 +21,9 @@ import java.io.IOException;
 import java.util.Objects;
 
 import org.apache.lucene.search.DocIdSetIterator;
+import org.apache.lucene.search.LeafSimScorer;
 import org.apache.lucene.search.Scorer;
 import org.apache.lucene.search.TwoPhaseIterator;
-import org.apache.lucene.search.similarities.Similarity;
 
 /**
  * A basic {@link Scorer} over {@link Spans}.
@@ -32,7 +32,7 @@ import org.apache.lucene.search.similarities.Similarity;
 public class SpanScorer extends Scorer {
 
   protected final Spans spans;
-  protected final Similarity.SimScorer docScorer;
+  protected final LeafSimScorer docScorer;
 
   /** accumulated sloppy freq (computed in setFreqCurrentDoc) */
   private float freq;
@@ -41,7 +41,7 @@ public class SpanScorer extends Scorer {
   private int lastScoredDoc = -1; // last doc we called setFreqCurrentDoc() for
 
   /** Sole constructor. */
-  public SpanScorer(SpanWeight weight, Spans spans, Similarity.SimScorer docScorer) {
+  public SpanScorer(SpanWeight weight, Spans spans, LeafSimScorer docScorer) {
     super(weight);
     this.spans = Objects.requireNonNull(spans);
     this.docScorer = docScorer;

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/8fd7ead9/lucene/core/src/java/org/apache/lucene/search/spans/SpanWeight.java
----------------------------------------------------------------------
diff --git a/lucene/core/src/java/org/apache/lucene/search/spans/SpanWeight.java b/lucene/core/src/java/org/apache/lucene/search/spans/SpanWeight.java
index 0dad614..f1d8894 100644
--- a/lucene/core/src/java/org/apache/lucene/search/spans/SpanWeight.java
+++ b/lucene/core/src/java/org/apache/lucene/search/spans/SpanWeight.java
@@ -28,10 +28,10 @@ import org.apache.lucene.index.TermContext;
 import org.apache.lucene.search.CollectionStatistics;
 import org.apache.lucene.search.Explanation;
 import org.apache.lucene.search.IndexSearcher;
+import org.apache.lucene.search.LeafSimScorer;
 import org.apache.lucene.search.TermStatistics;
 import org.apache.lucene.search.Weight;
 import org.apache.lucene.search.similarities.Similarity;
-import org.apache.lucene.search.similarities.Similarity.SimScorer;
 
 /**
  * Expert-only.  Public for use by other weight implementations
@@ -72,7 +72,7 @@ public abstract class SpanWeight extends Weight {
   }
 
   protected final Similarity similarity;
-  protected final Similarity.SimWeight simWeight;
+  protected final Similarity.SimScorer simScorer;
   protected final String field;
 
   /**
@@ -86,11 +86,11 @@ public abstract class SpanWeight extends Weight {
   public SpanWeight(SpanQuery query, IndexSearcher searcher, Map<Term, TermContext> termContexts, float boost) throws IOException {
     super(query);
     this.field = query.getField();
-    this.similarity = searcher.getSimilarity(termContexts != null);
-    this.simWeight = buildSimWeight(query, searcher, termContexts, boost);
+    this.similarity = searcher.getSimilarity();
+    this.simScorer = buildSimWeight(query, searcher, termContexts, boost);
   }
 
-  private Similarity.SimWeight buildSimWeight(SpanQuery query, IndexSearcher searcher, Map<Term, TermContext> termContexts, float boost) throws IOException {
+  private Similarity.SimScorer buildSimWeight(SpanQuery query, IndexSearcher searcher, Map<Term, TermContext> termContexts, float boost) throws IOException {
     if (termContexts == null || termContexts.size() == 0 || query.getField() == null)
       return null;
     TermStatistics[] termStats = new TermStatistics[termContexts.size()];
@@ -103,7 +103,7 @@ public abstract class SpanWeight extends Weight {
     }
     CollectionStatistics collectionStats = searcher.collectionStatistics(query.getField());
     if (termUpTo > 0) {
-      return similarity.computeWeight(boost, collectionStats, Arrays.copyOf(termStats, termUpTo));
+      return similarity.scorer(boost, collectionStats, Arrays.copyOf(termStats, termUpTo));
     } else {
       return null; // no terms at all exist, we won't use similarity
     }
@@ -129,18 +129,18 @@ public abstract class SpanWeight extends Weight {
     if (spans == null) {
       return null;
     }
-    final Similarity.SimScorer docScorer = getSimScorer(context);
+    final LeafSimScorer docScorer = getSimScorer(context);
     return new SpanScorer(this, spans, docScorer);
   }
 
   /**
-   * Return a SimScorer for this context
+   * Return a LeafSimScorer for this context
    * @param context the LeafReaderContext
    * @return a SimWeight
    * @throws IOException on error
    */
-  public Similarity.SimScorer getSimScorer(LeafReaderContext context) throws IOException {
-    return simWeight == null ? null : similarity.simScorer(simWeight, context);
+  public LeafSimScorer getSimScorer(LeafReaderContext context) throws IOException {
+    return simScorer == null ? null : new LeafSimScorer(simScorer, context.reader(), true, Float.POSITIVE_INFINITY);
   }
 
   @Override
@@ -150,7 +150,7 @@ public abstract class SpanWeight extends Weight {
       int newDoc = scorer.iterator().advance(doc);
       if (newDoc == doc) {
         float freq = scorer.sloppyFreq();
-        SimScorer docScorer = similarity.simScorer(simWeight, context);
+        LeafSimScorer docScorer = new LeafSimScorer(simScorer, context.reader(), true, Float.POSITIVE_INFINITY);
         Explanation freqExplanation = Explanation.match(freq, "phraseFreq=" + freq);
         Explanation scoreExplanation = docScorer.explain(doc, freqExplanation);
         return Explanation.match(scoreExplanation.getValue(),

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/8fd7ead9/lucene/core/src/java/org/apache/lucene/search/spans/TermSpans.java
----------------------------------------------------------------------
diff --git a/lucene/core/src/java/org/apache/lucene/search/spans/TermSpans.java b/lucene/core/src/java/org/apache/lucene/search/spans/TermSpans.java
index f1e1aed..625bb0e 100644
--- a/lucene/core/src/java/org/apache/lucene/search/spans/TermSpans.java
+++ b/lucene/core/src/java/org/apache/lucene/search/spans/TermSpans.java
@@ -22,7 +22,7 @@ import java.util.Objects;
 import org.apache.lucene.index.PostingsEnum;
 import org.apache.lucene.index.Term;
 import org.apache.lucene.search.DocIdSetIterator;
-import org.apache.lucene.search.similarities.Similarity;
+import org.apache.lucene.search.LeafSimScorer;
 
 /**
  * Expert:
@@ -39,7 +39,7 @@ public class TermSpans extends Spans {
   protected boolean readPayload;
   private final float positionsCost;
 
-  public TermSpans(Similarity.SimScorer scorer,
+  public TermSpans(LeafSimScorer scorer,
                     PostingsEnum postings, Term term, float positionsCost) {
     this.postings = Objects.requireNonNull(postings);
     this.term = Objects.requireNonNull(term);

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/8fd7ead9/lucene/core/src/test/org/apache/lucene/index/TestCustomNorms.java
----------------------------------------------------------------------
diff --git a/lucene/core/src/test/org/apache/lucene/index/TestCustomNorms.java b/lucene/core/src/test/org/apache/lucene/index/TestCustomNorms.java
index a811192..7737de1 100644
--- a/lucene/core/src/test/org/apache/lucene/index/TestCustomNorms.java
+++ b/lucene/core/src/test/org/apache/lucene/index/TestCustomNorms.java
@@ -103,12 +103,7 @@ public class TestCustomNorms extends LuceneTestCase {
     }
     
     @Override
-    public SimWeight computeWeight(float boost, CollectionStatistics collectionStats, TermStatistics... termStats) {
-      throw new UnsupportedOperationException();
-    }
-
-    @Override
-    public SimScorer simScorer(SimWeight weight, LeafReaderContext context) throws IOException {
+    public SimScorer scorer(float boost, CollectionStatistics collectionStats, TermStatistics... termStats) {
       throw new UnsupportedOperationException();
     }
   }

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/8fd7ead9/lucene/core/src/test/org/apache/lucene/index/TestCustomTermFreq.java
----------------------------------------------------------------------
diff --git a/lucene/core/src/test/org/apache/lucene/index/TestCustomTermFreq.java b/lucene/core/src/test/org/apache/lucene/index/TestCustomTermFreq.java
index d2eff25..8bb81d2 100644
--- a/lucene/core/src/test/org/apache/lucene/index/TestCustomTermFreq.java
+++ b/lucene/core/src/test/org/apache/lucene/index/TestCustomTermFreq.java
@@ -17,8 +17,6 @@
 
 package org.apache.lucene.index;
 
-import java.io.IOException;
-
 import org.apache.lucene.analysis.MockAnalyzer;
 import org.apache.lucene.analysis.TokenStream;
 import org.apache.lucene.analysis.tokenattributes.CharTermAttribute;
@@ -432,12 +430,7 @@ public class TestCustomTermFreq extends LuceneTestCase {
     }
     
     @Override
-    public SimWeight computeWeight(float boost, CollectionStatistics collectionStats, TermStatistics... termStats) {
-      throw new UnsupportedOperationException();
-    }
-
-    @Override
-    public SimScorer simScorer(SimWeight weight, LeafReaderContext context) throws IOException {
+    public SimScorer scorer(float boost, CollectionStatistics collectionStats, TermStatistics... termStats) {
       throw new UnsupportedOperationException();
     }
   }

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/8fd7ead9/lucene/core/src/test/org/apache/lucene/index/TestFieldInvertState.java
----------------------------------------------------------------------
diff --git a/lucene/core/src/test/org/apache/lucene/index/TestFieldInvertState.java b/lucene/core/src/test/org/apache/lucene/index/TestFieldInvertState.java
index f78b7fa..08635fc 100644
--- a/lucene/core/src/test/org/apache/lucene/index/TestFieldInvertState.java
+++ b/lucene/core/src/test/org/apache/lucene/index/TestFieldInvertState.java
@@ -17,7 +17,6 @@
 
 package org.apache.lucene.index;
 
-import java.io.IOException;
 import java.util.HashMap;
 import java.util.Map;
 
@@ -54,12 +53,7 @@ public class TestFieldInvertState extends LuceneTestCase {
     }
     
     @Override
-    public SimWeight computeWeight(float boost, CollectionStatistics collectionStats, TermStatistics... termStats) {
-      throw new UnsupportedOperationException();
-    }
-
-    @Override
-    public SimScorer simScorer(SimWeight weight, LeafReaderContext context) throws IOException {
+    public SimScorer scorer(float boost, CollectionStatistics collectionStats, TermStatistics... termStats) {
       throw new UnsupportedOperationException();
     }
   }

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/8fd7ead9/lucene/core/src/test/org/apache/lucene/index/TestIndexSorting.java
----------------------------------------------------------------------
diff --git a/lucene/core/src/test/org/apache/lucene/index/TestIndexSorting.java b/lucene/core/src/test/org/apache/lucene/index/TestIndexSorting.java
index 0a8799d..6b43c16 100644
--- a/lucene/core/src/test/org/apache/lucene/index/TestIndexSorting.java
+++ b/lucene/core/src/test/org/apache/lucene/index/TestIndexSorting.java
@@ -1947,13 +1947,8 @@ public class TestIndexSorting extends LuceneTestCase {
     }
 
     @Override
-    public SimWeight computeWeight(float boost, CollectionStatistics collectionStats, TermStatistics... termStats) {
-      return in.computeWeight(boost, collectionStats, termStats);
-    }
-
-    @Override
-    public SimScorer simScorer(SimWeight weight, LeafReaderContext context) throws IOException {
-      return in.simScorer(weight, context);
+    public SimScorer scorer(float boost, CollectionStatistics collectionStats, TermStatistics... termStats) {
+      return in.scorer(boost, collectionStats, termStats);
     }
 
   }

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/8fd7ead9/lucene/core/src/test/org/apache/lucene/index/TestMaxTermFrequency.java
----------------------------------------------------------------------
diff --git a/lucene/core/src/test/org/apache/lucene/index/TestMaxTermFrequency.java b/lucene/core/src/test/org/apache/lucene/index/TestMaxTermFrequency.java
index f391c5a..cf29392 100644
--- a/lucene/core/src/test/org/apache/lucene/index/TestMaxTermFrequency.java
+++ b/lucene/core/src/test/org/apache/lucene/index/TestMaxTermFrequency.java
@@ -109,17 +109,11 @@ public class TestMaxTermFrequency extends LuceneTestCase {
     }
 
     @Override
-    public SimWeight computeWeight(float boost, CollectionStatistics collectionStats, TermStatistics... termStats) {
-      return new SimWeight() {};
-    }
-
-    @Override
-    public SimScorer simScorer(SimWeight weight, LeafReaderContext context) throws IOException {
-
-      return new SimScorer() {
+    public SimScorer scorer(float boost, CollectionStatistics collectionStats, TermStatistics... termStats) {
+      return new SimScorer(collectionStats.field()) {
 
         @Override
-        public float score(int doc, float freq) throws IOException {
+        public float score(float freq, long norm) throws IOException {
           return 0;
         }
 

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/8fd7ead9/lucene/core/src/test/org/apache/lucene/index/TestNorms.java
----------------------------------------------------------------------
diff --git a/lucene/core/src/test/org/apache/lucene/index/TestNorms.java b/lucene/core/src/test/org/apache/lucene/index/TestNorms.java
index 70c7a32..805c7e5 100644
--- a/lucene/core/src/test/org/apache/lucene/index/TestNorms.java
+++ b/lucene/core/src/test/org/apache/lucene/index/TestNorms.java
@@ -116,12 +116,7 @@ public class TestNorms extends LuceneTestCase {
     }
 
     @Override
-    public SimWeight computeWeight(float boost, CollectionStatistics collectionStats, TermStatistics... termStats) {
-      throw new UnsupportedOperationException();
-    }
-
-    @Override
-    public SimScorer simScorer(SimWeight weight, LeafReaderContext context) throws IOException {
+    public SimScorer scorer(float boost, CollectionStatistics collectionStats, TermStatistics... termStats) {
       throw new UnsupportedOperationException();
     }
   } 

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/8fd7ead9/lucene/core/src/test/org/apache/lucene/index/TestUniqueTermCount.java
----------------------------------------------------------------------
diff --git a/lucene/core/src/test/org/apache/lucene/index/TestUniqueTermCount.java b/lucene/core/src/test/org/apache/lucene/index/TestUniqueTermCount.java
index a0fca4c..2de0234 100644
--- a/lucene/core/src/test/org/apache/lucene/index/TestUniqueTermCount.java
+++ b/lucene/core/src/test/org/apache/lucene/index/TestUniqueTermCount.java
@@ -17,7 +17,6 @@
 package org.apache.lucene.index;
 
 
-import java.io.IOException;
 import java.util.ArrayList;
 import java.util.HashSet;
 
@@ -106,12 +105,7 @@ public class TestUniqueTermCount extends LuceneTestCase {
     }
 
     @Override
-    public SimWeight computeWeight(float boost, CollectionStatistics collectionStats, TermStatistics... termStats) {
-      throw new UnsupportedOperationException();
-    }
-
-    @Override
-    public SimScorer simScorer(SimWeight weight, LeafReaderContext context) throws IOException {
+    public SimScorer scorer(float boost, CollectionStatistics collectionStats, TermStatistics... termStats) {
       throw new UnsupportedOperationException();
     }
   }

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/8fd7ead9/lucene/core/src/test/org/apache/lucene/search/JustCompileSearch.java
----------------------------------------------------------------------
diff --git a/lucene/core/src/test/org/apache/lucene/search/JustCompileSearch.java b/lucene/core/src/test/org/apache/lucene/search/JustCompileSearch.java
index 0523e2c..c85732e 100644
--- a/lucene/core/src/test/org/apache/lucene/search/JustCompileSearch.java
+++ b/lucene/core/src/test/org/apache/lucene/search/JustCompileSearch.java
@@ -194,12 +194,7 @@ final class JustCompileSearch {
   static final class JustCompileSimilarity extends Similarity {
 
     @Override
-    public SimWeight computeWeight(float boost, CollectionStatistics collectionStats, TermStatistics... termStats) {
-      throw new UnsupportedOperationException(UNSUPPORTED_MSG);
-    }
-
-    @Override
-    public SimScorer simScorer(SimWeight stats, LeafReaderContext context) {
+    public SimScorer scorer(float boost, CollectionStatistics collectionStats, TermStatistics... termStats) {
       throw new UnsupportedOperationException(UNSUPPORTED_MSG);
     }
 

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/8fd7ead9/lucene/core/src/test/org/apache/lucene/search/TestBoolean2.java
----------------------------------------------------------------------
diff --git a/lucene/core/src/test/org/apache/lucene/search/TestBoolean2.java b/lucene/core/src/test/org/apache/lucene/search/TestBoolean2.java
index 9556281..a9e2891 100644
--- a/lucene/core/src/test/org/apache/lucene/search/TestBoolean2.java
+++ b/lucene/core/src/test/org/apache/lucene/search/TestBoolean2.java
@@ -148,7 +148,7 @@ public class TestBoolean2 extends LuceneTestCase {
     }
     singleSegmentReader = DirectoryReader.open(singleSegmentDirectory);
     singleSegmentSearcher = newSearcher(singleSegmentReader);
-    singleSegmentSearcher.setSimilarity(searcher.getSimilarity(true));
+    singleSegmentSearcher.setSimilarity(searcher.getSimilarity());
     
     // Make big index
     dir2 = copyOf(directory);
@@ -379,7 +379,7 @@ public class TestBoolean2 extends LuceneTestCase {
         QueryUtils.check(random(), q1,searcher); // baseline sim
         try {
           // a little hackish, QueryUtils.check is too costly to do on bigSearcher in this loop.
-          searcher.setSimilarity(bigSearcher.getSimilarity(true)); // random sim
+          searcher.setSimilarity(bigSearcher.getSimilarity()); // random sim
           QueryUtils.check(random(), q1, searcher);
         } finally {
           searcher.setSimilarity(new ClassicSimilarity()); // restore

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/8fd7ead9/lucene/core/src/test/org/apache/lucene/search/TestBooleanQueryVisitSubscorers.java
----------------------------------------------------------------------
diff --git a/lucene/core/src/test/org/apache/lucene/search/TestBooleanQueryVisitSubscorers.java b/lucene/core/src/test/org/apache/lucene/search/TestBooleanQueryVisitSubscorers.java
index 19f45f8..37a1b57 100644
--- a/lucene/core/src/test/org/apache/lucene/search/TestBooleanQueryVisitSubscorers.java
+++ b/lucene/core/src/test/org/apache/lucene/search/TestBooleanQueryVisitSubscorers.java
@@ -38,7 +38,6 @@ import org.apache.lucene.index.RandomIndexWriter;
 import org.apache.lucene.index.Term;
 import org.apache.lucene.search.BooleanClause.Occur;
 import org.apache.lucene.search.Scorer.ChildScorer;
-import org.apache.lucene.search.similarities.BasicStats;
 import org.apache.lucene.search.similarities.ClassicSimilarity;
 import org.apache.lucene.search.similarities.Similarity;
 import org.apache.lucene.store.Directory;
@@ -329,15 +328,10 @@ public class TestBooleanQueryVisitSubscorers extends LuceneTestCase {
     }
 
     @Override
-    public SimWeight computeWeight(float boost, CollectionStatistics collectionStats, TermStatistics... termStats) {
-      return new BasicStats("", boost);
-    }
-
-    @Override
-    public SimScorer simScorer(SimWeight weight, LeafReaderContext context) throws IOException {
-      return new SimScorer() {
+    public SimScorer scorer(float boost, CollectionStatistics collectionStats, TermStatistics... termStats) {
+      return new SimScorer(collectionStats.field()) {
         @Override
-        public float score(int doc, float freq) throws IOException {
+        public float score(float freq, long norm) throws IOException {
           return freq;
         }
         @Override

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/8fd7ead9/lucene/core/src/test/org/apache/lucene/search/TestBooleanRewrites.java
----------------------------------------------------------------------
diff --git a/lucene/core/src/test/org/apache/lucene/search/TestBooleanRewrites.java b/lucene/core/src/test/org/apache/lucene/search/TestBooleanRewrites.java
index d21f373..292dfa9 100644
--- a/lucene/core/src/test/org/apache/lucene/search/TestBooleanRewrites.java
+++ b/lucene/core/src/test/org/apache/lucene/search/TestBooleanRewrites.java
@@ -357,7 +357,7 @@ public class TestBooleanRewrites extends LuceneTestCase {
         return original;
       }
     };
-    searcher2.setSimilarity(searcher1.getSimilarity(true));
+    searcher2.setSimilarity(searcher1.getSimilarity());
 
     final int iters = atLeast(1000);
     for (int i = 0; i < iters; ++i) {

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/8fd7ead9/lucene/core/src/test/org/apache/lucene/search/TestConjunctions.java
----------------------------------------------------------------------
diff --git a/lucene/core/src/test/org/apache/lucene/search/TestConjunctions.java b/lucene/core/src/test/org/apache/lucene/search/TestConjunctions.java
index a4e9596..9b79675 100644
--- a/lucene/core/src/test/org/apache/lucene/search/TestConjunctions.java
+++ b/lucene/core/src/test/org/apache/lucene/search/TestConjunctions.java
@@ -34,7 +34,6 @@ import org.apache.lucene.index.FieldInvertState;
 import org.apache.lucene.index.IndexReader;
 import org.apache.lucene.index.IndexWriter;
 import org.apache.lucene.index.IndexWriterConfig;
-import org.apache.lucene.index.LeafReaderContext;
 import org.apache.lucene.index.RandomIndexWriter;
 import org.apache.lucene.index.Term;
 import org.apache.lucene.search.similarities.Similarity;
@@ -100,16 +99,11 @@ public class TestConjunctions extends LuceneTestCase {
     }
 
     @Override
-    public SimWeight computeWeight(float boost,
+    public SimScorer scorer(float boost,
         CollectionStatistics collectionStats, TermStatistics... termStats) {
-      return new SimWeight() {};
-    }
-
-    @Override
-    public SimScorer simScorer(SimWeight weight, LeafReaderContext context) throws IOException {
-      return new SimScorer() {
+      return new SimScorer(collectionStats.field()) {
         @Override
-        public float score(int doc, float freq) {
+        public float score(float freq, long norm) {
           return freq;
         }
 

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/8fd7ead9/lucene/core/src/test/org/apache/lucene/search/TestDocValuesScoring.java
----------------------------------------------------------------------
diff --git a/lucene/core/src/test/org/apache/lucene/search/TestDocValuesScoring.java b/lucene/core/src/test/org/apache/lucene/search/TestDocValuesScoring.java
deleted file mode 100644
index 8856431..0000000
--- a/lucene/core/src/test/org/apache/lucene/search/TestDocValuesScoring.java
+++ /dev/null
@@ -1,192 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.lucene.search;
-
-
-import java.io.IOException;
-
-import org.apache.lucene.document.Document;
-import org.apache.lucene.document.Field;
-import org.apache.lucene.document.FloatDocValuesField;
-import org.apache.lucene.index.DocValues;
-import org.apache.lucene.index.FieldInvertState;
-import org.apache.lucene.index.IndexReader;
-import org.apache.lucene.index.LeafReaderContext;
-import org.apache.lucene.index.NumericDocValues;
-import org.apache.lucene.index.RandomIndexWriter;
-import org.apache.lucene.index.Term;
-import org.apache.lucene.search.similarities.PerFieldSimilarityWrapper;
-import org.apache.lucene.search.similarities.Similarity;
-import org.apache.lucene.store.Directory;
-import org.apache.lucene.util.LuceneTestCase;
-
-/**
- * Tests the use of indexdocvalues in scoring.
- * 
- * In the example, a docvalues field is used as a per-document boost (separate from the norm)
- * @lucene.experimental
- */
-public class TestDocValuesScoring extends LuceneTestCase {
-  private static final float SCORE_EPSILON = 0.001f; /* for comparing floats */
-
-  public void testSimple() throws Exception {    
-    Directory dir = newDirectory();
-    RandomIndexWriter iw = new RandomIndexWriter(random(), dir);
-    Document doc = new Document();
-    Field field = newTextField("foo", "", Field.Store.NO);
-    doc.add(field);
-    Field dvField = new FloatDocValuesField("foo_boost", 0.0F);
-    doc.add(dvField);
-    Field field2 = newTextField("bar", "", Field.Store.NO);
-    doc.add(field2);
-    
-    field.setStringValue("quick brown fox");
-    field2.setStringValue("quick brown fox");
-    dvField.setFloatValue(2f); // boost x2
-    iw.addDocument(doc);
-    field.setStringValue("jumps over lazy brown dog");
-    field2.setStringValue("jumps over lazy brown dog");
-    dvField.setFloatValue(4f); // boost x4
-    iw.addDocument(doc);
-    IndexReader ir = iw.getReader();
-    iw.close();
-    
-    // no boosting
-    IndexSearcher searcher1 = newSearcher(ir, false);
-    final Similarity base = searcher1.getSimilarity(true);
-    // boosting
-    IndexSearcher searcher2 = newSearcher(ir, false);
-    searcher2.setSimilarity(new PerFieldSimilarityWrapper() {
-      final Similarity fooSim = new BoostingSimilarity(base, "foo_boost");
-
-      @Override
-      public Similarity get(String field) {
-        return "foo".equals(field) ? fooSim : base;
-      }
-    });
-    
-    // in this case, we searched on field "foo". first document should have 2x the score.
-    TermQuery tq = new TermQuery(new Term("foo", "quick"));
-    QueryUtils.check(random(), tq, searcher1);
-    QueryUtils.check(random(), tq, searcher2);
-    
-    TopDocs noboost = searcher1.search(tq, 10);
-    TopDocs boost = searcher2.search(tq, 10);
-    assertEquals(1, noboost.totalHits);
-    assertEquals(1, boost.totalHits);
-    
-    //System.out.println(searcher2.explain(tq, boost.scoreDocs[0].doc));
-    assertEquals(boost.scoreDocs[0].score, noboost.scoreDocs[0].score*2f, SCORE_EPSILON);
-    
-    // this query matches only the second document, which should have 4x the score.
-    tq = new TermQuery(new Term("foo", "jumps"));
-    QueryUtils.check(random(), tq, searcher1);
-    QueryUtils.check(random(), tq, searcher2);
-    
-    noboost = searcher1.search(tq, 10);
-    boost = searcher2.search(tq, 10);
-    assertEquals(1, noboost.totalHits);
-    assertEquals(1, boost.totalHits);
-    
-    assertEquals(boost.scoreDocs[0].score, noboost.scoreDocs[0].score*4f, SCORE_EPSILON);
-    
-    // search on on field bar just for kicks, nothing should happen, since we setup
-    // our sim provider to only use foo_boost for field foo.
-    tq = new TermQuery(new Term("bar", "quick"));
-    QueryUtils.check(random(), tq, searcher1);
-    QueryUtils.check(random(), tq, searcher2);
-    
-    noboost = searcher1.search(tq, 10);
-    boost = searcher2.search(tq, 10);
-    assertEquals(1, noboost.totalHits);
-    assertEquals(1, boost.totalHits);
-    
-    assertEquals(boost.scoreDocs[0].score, noboost.scoreDocs[0].score, SCORE_EPSILON);
-
-    ir.close();
-    dir.close();
-  }
-  
-  /**
-   * Similarity that wraps another similarity and boosts the final score
-   * according to whats in a docvalues field.
-   * 
-   * @lucene.experimental
-   */
-  static class BoostingSimilarity extends Similarity {
-    private final Similarity sim;
-    private final String boostField;
-    
-    public BoostingSimilarity(Similarity sim, String boostField) {
-      this.sim = sim;
-      this.boostField = boostField;
-    }
-    
-    @Override
-    public long computeNorm(FieldInvertState state) {
-      return sim.computeNorm(state);
-    }
-
-    @Override
-    public SimWeight computeWeight(float boost, CollectionStatistics collectionStats, TermStatistics... termStats) {
-      return sim.computeWeight(boost, collectionStats, termStats);
-    }
-
-    @Override
-    public SimScorer simScorer(SimWeight stats, LeafReaderContext context) throws IOException {
-      final SimScorer sub = sim.simScorer(stats, context);
-      final NumericDocValues values = DocValues.getNumeric(context.reader(), boostField);
-      
-      return new SimScorer() {
-
-        private float getValueForDoc(int doc) throws IOException {
-          int curDocID = values.docID();
-          if (doc < curDocID) {
-            throw new IllegalArgumentException("doc=" + doc + " is before curDocID=" + curDocID);
-          }
-          if (doc > curDocID) {
-            curDocID = values.advance(doc);
-          }
-          if (curDocID == doc) {
-            return Float.intBitsToFloat((int)values.longValue());
-          } else {
-            return 0f;
-          }
-        }
-        
-        @Override
-        public float score(int doc, float freq) throws IOException {
-          return getValueForDoc(doc) * sub.score(doc, freq);
-        }
-
-        @Override
-        public float maxScore(float maxFreq) {
-          return Float.POSITIVE_INFINITY;
-        }
-
-        @Override
-        public Explanation explain(int doc, Explanation freq) throws IOException {
-          Explanation boostExplanation = Explanation.match(getValueForDoc(doc), "indexDocValue(" + boostField + ")");
-          Explanation simExplanation = sub.explain(doc, freq);
-          return Explanation.match(
-              boostExplanation.getValue().doubleValue() * simExplanation.getValue().doubleValue(),
-              "product of:", boostExplanation, simExplanation);
-        }
-      };
-    }
-  }
-}

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/8fd7ead9/lucene/core/src/test/org/apache/lucene/search/TestMinShouldMatch2.java
----------------------------------------------------------------------
diff --git a/lucene/core/src/test/org/apache/lucene/search/TestMinShouldMatch2.java b/lucene/core/src/test/org/apache/lucene/search/TestMinShouldMatch2.java
index 924a1af..268e8fc 100644
--- a/lucene/core/src/test/org/apache/lucene/search/TestMinShouldMatch2.java
+++ b/lucene/core/src/test/org/apache/lucene/search/TestMinShouldMatch2.java
@@ -37,7 +37,6 @@ import org.apache.lucene.index.Term;
 import org.apache.lucene.index.TermContext;
 import org.apache.lucene.search.similarities.ClassicSimilarity;
 import org.apache.lucene.search.similarities.Similarity.SimScorer;
-import org.apache.lucene.search.similarities.Similarity.SimWeight;
 import org.apache.lucene.store.Directory;
 import org.apache.lucene.util.BytesRef;
 import org.apache.lucene.util.LuceneTestCase;
@@ -310,7 +309,7 @@ public class TestMinShouldMatch2 extends LuceneTestCase {
     final int maxDoc;
 
     final Set<Long> ords = new HashSet<>();
-    final SimScorer[] sims;
+    final LeafSimScorer[] sims;
     final int minNrShouldMatch;
     
     double score = Float.NaN;
@@ -321,7 +320,7 @@ public class TestMinShouldMatch2 extends LuceneTestCase {
       this.maxDoc = reader.maxDoc();
       BooleanQuery bq = (BooleanQuery) weight.getQuery();
       this.minNrShouldMatch = bq.getMinimumNumberShouldMatch();
-      this.sims = new SimScorer[(int)dv.getValueCount()];
+      this.sims = new LeafSimScorer[(int)dv.getValueCount()];
       for (BooleanClause clause : bq.clauses()) {
         assert !clause.isProhibited();
         assert !clause.isRequired();
@@ -331,10 +330,10 @@ public class TestMinShouldMatch2 extends LuceneTestCase {
           boolean success = ords.add(ord);
           assert success; // no dups
           TermContext context = TermContext.build(reader.getContext(), term);
-          SimWeight w = weight.similarity.computeWeight(1f,
+          SimScorer w = weight.similarity.scorer(1f,
                         searcher.collectionStatistics("field"),
                         searcher.termStatistics(term, context));
-          sims[(int)ord] = weight.similarity.simScorer(w, reader.getContext());
+          sims[(int)ord] = new LeafSimScorer(w, reader, true, 1);
         }
       }
     }

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/8fd7ead9/lucene/core/src/test/org/apache/lucene/search/TestSimilarityProvider.java
----------------------------------------------------------------------
diff --git a/lucene/core/src/test/org/apache/lucene/search/TestSimilarityProvider.java b/lucene/core/src/test/org/apache/lucene/search/TestSimilarityProvider.java
index a6970f9..f0e3b22 100644
--- a/lucene/core/src/test/org/apache/lucene/search/TestSimilarityProvider.java
+++ b/lucene/core/src/test/org/apache/lucene/search/TestSimilarityProvider.java
@@ -25,7 +25,6 @@ import org.apache.lucene.document.Field;
 import org.apache.lucene.index.DirectoryReader;
 import org.apache.lucene.index.FieldInvertState;
 import org.apache.lucene.index.IndexWriterConfig;
-import org.apache.lucene.index.LeafReaderContext;
 import org.apache.lucene.index.MultiDocValues;
 import org.apache.lucene.index.NumericDocValues;
 import org.apache.lucene.index.RandomIndexWriter;
@@ -113,16 +112,11 @@ public class TestSimilarityProvider extends LuceneTestCase {
     }
 
     @Override
-    public SimWeight computeWeight(float boost, CollectionStatistics collectionStats, TermStatistics... termStats) {
-      return new SimWeight() {};
-    }
-
-    @Override
-    public SimScorer simScorer(SimWeight weight, LeafReaderContext context) throws IOException {
-      return new SimScorer() {
+    public SimScorer scorer(float boost, CollectionStatistics collectionStats, TermStatistics... termStats) {
+      return new SimScorer(collectionStats.field()) {
 
         @Override
-        public float score(int doc, float freq) throws IOException {
+        public float score(float freq, long norm) throws IOException {
           return 1;
         }
 
@@ -143,16 +137,11 @@ public class TestSimilarityProvider extends LuceneTestCase {
     }
 
     @Override
-    public SimWeight computeWeight(float boost, CollectionStatistics collectionStats, TermStatistics... termStats) {
-      return new SimWeight() {};
-    }
-
-    @Override
-    public SimScorer simScorer(SimWeight weight, LeafReaderContext context) throws IOException {
-      return new SimScorer() {
+    public SimScorer scorer(float boost, CollectionStatistics collectionStats, TermStatistics... termStats) {
+      return new SimScorer(collectionStats.field()) {
 
         @Override
-        public float score(int doc, float freq) throws IOException {
+        public float score(float freq, long norm) throws IOException {
           return 10;
         }
 

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/8fd7ead9/lucene/core/src/test/org/apache/lucene/search/TestSubScorerFreqs.java
----------------------------------------------------------------------
diff --git a/lucene/core/src/test/org/apache/lucene/search/TestSubScorerFreqs.java b/lucene/core/src/test/org/apache/lucene/search/TestSubScorerFreqs.java
index 7278a3b..1cc4d59 100644
--- a/lucene/core/src/test/org/apache/lucene/search/TestSubScorerFreqs.java
+++ b/lucene/core/src/test/org/apache/lucene/search/TestSubScorerFreqs.java
@@ -34,7 +34,6 @@ import org.apache.lucene.index.RandomIndexWriter;
 import org.apache.lucene.index.Term;
 import org.apache.lucene.search.BooleanClause.Occur;
 import org.apache.lucene.search.Scorer.ChildScorer;
-import org.apache.lucene.search.similarities.BasicStats;
 import org.apache.lucene.search.similarities.Similarity;
 import org.apache.lucene.store.Directory;
 import org.apache.lucene.store.RAMDirectory;
@@ -231,15 +230,10 @@ public class TestSubScorerFreqs extends LuceneTestCase {
     }
 
     @Override
-    public SimWeight computeWeight(float boost, CollectionStatistics collectionStats, TermStatistics... termStats) {
-      return new BasicStats("", boost);
-    }
-
-    @Override
-    public SimScorer simScorer(SimWeight weight, LeafReaderContext context) throws IOException {
-      return new SimScorer() {
+    public SimScorer scorer(float boost, CollectionStatistics collectionStats, TermStatistics... termStats) {
+      return new SimScorer(collectionStats.field()) {
         @Override
-        public float score(int doc, float freq) throws IOException {
+        public float score(float freq, long norm) throws IOException {
           return freq;
         }
 

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/8fd7ead9/lucene/core/src/test/org/apache/lucene/search/similarities/TestClassicSimilarity.java
----------------------------------------------------------------------
diff --git a/lucene/core/src/test/org/apache/lucene/search/similarities/TestClassicSimilarity.java b/lucene/core/src/test/org/apache/lucene/search/similarities/TestClassicSimilarity.java
index a0fa0f3..eb7a590 100644
--- a/lucene/core/src/test/org/apache/lucene/search/similarities/TestClassicSimilarity.java
+++ b/lucene/core/src/test/org/apache/lucene/search/similarities/TestClassicSimilarity.java
@@ -36,7 +36,6 @@ import org.apache.lucene.search.IndexSearcher;
 import org.apache.lucene.search.Query;
 import org.apache.lucene.search.TermQuery;
 import org.apache.lucene.search.TopDocs;
-import org.apache.lucene.search.similarities.TFIDFSimilarity.IDFStats;
 import org.apache.lucene.store.Directory;
 import org.apache.lucene.util.IOUtils;
 import org.apache.lucene.util.TestUtil;
@@ -157,7 +156,7 @@ public class TestClassicSimilarity extends BaseSimilarityTestCase {
   
   public void testSaneNormValues() throws IOException {
     ClassicSimilarity sim = new ClassicSimilarity();
-    TFIDFSimilarity.IDFStats stats = (IDFStats) sim.computeWeight(1f, indexSearcher.collectionStatistics("test"));
+    TFIDFSimilarity.TFIDFScorer stats = (TFIDFSimilarity.TFIDFScorer) sim.scorer(1f, indexSearcher.collectionStatistics("test"));
     for (int i = 0; i < 256; i++) {
       float boost = stats.normTable[i];
       assertFalse("negative boost: " + boost + ", byte=" + i, boost < 0.0f);

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/8fd7ead9/lucene/core/src/test/org/apache/lucene/search/similarities/TestSimilarityBase.java
----------------------------------------------------------------------
diff --git a/lucene/core/src/test/org/apache/lucene/search/similarities/TestSimilarityBase.java b/lucene/core/src/test/org/apache/lucene/search/similarities/TestSimilarityBase.java
index 279e30c..b263582 100644
--- a/lucene/core/src/test/org/apache/lucene/search/similarities/TestSimilarityBase.java
+++ b/lucene/core/src/test/org/apache/lucene/search/similarities/TestSimilarityBase.java
@@ -37,6 +37,7 @@ 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.search.similarities.SimilarityBase.BasicSimScorer;
 import org.apache.lucene.store.Directory;
 import org.apache.lucene.util.BytesRef;
 import org.apache.lucene.util.LuceneTestCase;
@@ -207,13 +208,13 @@ public class TestSimilarityBase extends LuceneTestCase {
    */
   private void unitTestCore(BasicStats stats, float freq, int docLen) {
     for (SimilarityBase sim : sims) {
-      BasicStats realStats = (BasicStats) sim.computeWeight(
+      BasicStats realStats = ((BasicSimScorer) sim.scorer(
           (float)stats.getBoost(),
           toCollectionStats(stats), 
-          toTermStats(stats));
+          toTermStats(stats))).stats;
       float score = (float)sim.score(realStats, freq, docLen);
       float explScore = sim.explain(
-          realStats, 1, Explanation.match(freq, "freq"), docLen).getValue().floatValue();
+          realStats, Explanation.match(freq, "freq"), docLen).getValue().floatValue();
       assertFalse("Score infinite: " + sim.toString(), Float.isInfinite(score));
       assertFalse("Score NaN: " + sim.toString(), Float.isNaN(score));
       assertTrue("Score negative: " + sim.toString(), score >= 0);
@@ -489,10 +490,10 @@ public class TestSimilarityBase extends LuceneTestCase {
    */
   private void correctnessTestCore(SimilarityBase sim, float gold) {
     BasicStats stats = createStats();
-    BasicStats realStats = (BasicStats) sim.computeWeight(
+    BasicStats realStats = ((BasicSimScorer) sim.scorer(
         (float)stats.getBoost(),
         toCollectionStats(stats), 
-        toTermStats(stats));
+        toTermStats(stats))).stats;
     float score = (float) sim.score(realStats, FREQ, DOC_LEN);
     assertEquals(
         sim.toString() + " score not correct.", gold, score, FLOAT_EPSILON);

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/8fd7ead9/lucene/core/src/test/org/apache/lucene/search/spans/TestFieldMaskingSpanQuery.java
----------------------------------------------------------------------
diff --git a/lucene/core/src/test/org/apache/lucene/search/spans/TestFieldMaskingSpanQuery.java b/lucene/core/src/test/org/apache/lucene/search/spans/TestFieldMaskingSpanQuery.java
index 8ed0462..f72ea66 100644
--- a/lucene/core/src/test/org/apache/lucene/search/spans/TestFieldMaskingSpanQuery.java
+++ b/lucene/core/src/test/org/apache/lucene/search/spans/TestFieldMaskingSpanQuery.java
@@ -235,7 +235,7 @@ public class TestFieldMaskingSpanQuery extends LuceneTestCase {
   
   public void testSimple2() throws Exception {
     assumeTrue("Broken scoring: LUCENE-3723", 
-        searcher.getSimilarity(true) instanceof TFIDFSimilarity);
+        searcher.getSimilarity() instanceof TFIDFSimilarity);
     SpanQuery q1 = new SpanTermQuery(new Term("gender", "female"));
     SpanQuery q2 = new SpanTermQuery(new Term("last", "smith"));
     SpanQuery q = new SpanNearQuery(new SpanQuery[]
@@ -291,7 +291,7 @@ public class TestFieldMaskingSpanQuery extends LuceneTestCase {
   
   public void testSpans2() throws Exception {
     assumeTrue("Broken scoring: LUCENE-3723",
-        searcher.getSimilarity(true) instanceof TFIDFSimilarity);
+        searcher.getSimilarity() instanceof TFIDFSimilarity);
     SpanQuery qA1 = new SpanTermQuery(new Term("gender", "female"));
     SpanQuery qA2 = new SpanTermQuery(new Term("first",  "james"));
     SpanQuery qA  = new SpanOrQuery(qA1, new FieldMaskingSpanQuery(qA2, "gender"));

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/8fd7ead9/lucene/memory/src/test/org/apache/lucene/index/memory/TestMemoryIndex.java
----------------------------------------------------------------------
diff --git a/lucene/memory/src/test/org/apache/lucene/index/memory/TestMemoryIndex.java b/lucene/memory/src/test/org/apache/lucene/index/memory/TestMemoryIndex.java
index a4b9d71..94cf974 100644
--- a/lucene/memory/src/test/org/apache/lucene/index/memory/TestMemoryIndex.java
+++ b/lucene/memory/src/test/org/apache/lucene/index/memory/TestMemoryIndex.java
@@ -50,7 +50,6 @@ import org.apache.lucene.index.IndexOptions;
 import org.apache.lucene.index.IndexReader;
 import org.apache.lucene.index.IndexableField;
 import org.apache.lucene.index.LeafReader;
-import org.apache.lucene.index.LeafReaderContext;
 import org.apache.lucene.index.NumericDocValues;
 import org.apache.lucene.index.PostingsEnum;
 import org.apache.lucene.index.SortedDocValues;
@@ -200,12 +199,7 @@ public class TestMemoryIndex extends LuceneTestCase {
       }
 
       @Override
-      public SimWeight computeWeight(float boost, CollectionStatistics collectionStats, TermStatistics... termStats) {
-        throw new UnsupportedOperationException();
-      }
-
-      @Override
-      public SimScorer simScorer(SimWeight weight, LeafReaderContext context) throws IOException {
+      public SimScorer scorer(float boost, CollectionStatistics collectionStats, TermStatistics... termStats) {
         throw new UnsupportedOperationException();
       }
 

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/8fd7ead9/lucene/misc/src/test/org/apache/lucene/search/TestDiversifiedTopDocsCollector.java
----------------------------------------------------------------------
diff --git a/lucene/misc/src/test/org/apache/lucene/search/TestDiversifiedTopDocsCollector.java b/lucene/misc/src/test/org/apache/lucene/search/TestDiversifiedTopDocsCollector.java
index 2e8491d..5e98349f 100644
--- a/lucene/misc/src/test/org/apache/lucene/search/TestDiversifiedTopDocsCollector.java
+++ b/lucene/misc/src/test/org/apache/lucene/search/TestDiversifiedTopDocsCollector.java
@@ -19,6 +19,7 @@ package org.apache.lucene.search;
 import java.io.IOException;
 import java.util.HashMap;
 import java.util.Map;
+import java.util.Set;
 
 import org.apache.lucene.document.Document;
 import org.apache.lucene.document.Field;
@@ -27,7 +28,6 @@ import org.apache.lucene.document.SortedDocValuesField;
 import org.apache.lucene.document.StoredField;
 import org.apache.lucene.index.BinaryDocValues;
 import org.apache.lucene.index.DocValues;
-import org.apache.lucene.index.FieldInvertState;
 import org.apache.lucene.index.IndexReader;
 import org.apache.lucene.index.LeafReaderContext;
 import org.apache.lucene.index.MultiDocValues;
@@ -36,7 +36,6 @@ import org.apache.lucene.index.RandomIndexWriter;
 import org.apache.lucene.index.SortedDocValues;
 import org.apache.lucene.index.Term;
 import org.apache.lucene.search.BooleanClause.Occur;
-import org.apache.lucene.search.similarities.Similarity;
 import org.apache.lucene.store.Directory;
 import org.apache.lucene.util.BytesRef;
 import org.apache.lucene.util.LuceneTestCase;
@@ -358,7 +357,7 @@ public class TestDiversifiedTopDocsCollector extends LuceneTestCase {
         Occur.SHOULD));
     testQuery.add(new BooleanClause(new TermQuery(new Term("year", "1969")),
         Occur.SHOULD));
-    return testQuery.build();
+    return new DocValueScoreQuery(testQuery.build(), "weeksAtNumberOne");
   }
 
   @Override
@@ -411,10 +410,6 @@ public class TestDiversifiedTopDocsCollector extends LuceneTestCase {
     writer.close();
     searcher = newSearcher(reader);
     artistDocValues = MultiDocValues.getSortedValues(reader, "artist");
-
-    // All searches sort by song popularity 
-    final Similarity base = searcher.getSimilarity(true);
-    searcher.setSimilarity(new DocValueSimilarity(base, "weeksAtNumberOne"));
   }
 
   @Override
@@ -442,61 +437,108 @@ public class TestDiversifiedTopDocsCollector extends LuceneTestCase {
     return result;
   }
 
-  /**
-   * Similarity that wraps another similarity and replaces the final score
-   * according to whats in a docvalues field.
-   * 
-   * @lucene.experimental
-   */
-  static class DocValueSimilarity extends Similarity {
-    private final Similarity sim;
-    private final String scoreValueField;
-
-    public DocValueSimilarity(Similarity sim, String scoreValueField) {
-      this.sim = sim;
-      this.scoreValueField = scoreValueField;
-    }
+  private static final class DocValueScoreQuery extends Query {
 
+    private final Query query;
+    private final String scoreField;
+    
+    DocValueScoreQuery(Query query, String scoreField) {
+      this.query = query;
+      this.scoreField = scoreField;
+    }
+    
     @Override
-    public long computeNorm(FieldInvertState state) {
-      return sim.computeNorm(state);
+    public String toString(String field) {
+      return "DocValueScore(" + query.toString(field) + ")";
     }
 
     @Override
-    public SimWeight computeWeight(float boost,
-        CollectionStatistics collectionStats, TermStatistics... termStats) {
-      return sim.computeWeight(boost, collectionStats, termStats);
+    public boolean equals(Object obj) {
+      if (obj instanceof DocValueScoreQuery == false) {
+        return false;
+      }
+      return query.equals(((DocValueScoreQuery) obj).query);
     }
 
     @Override
-    public SimScorer simScorer(SimWeight stats, LeafReaderContext context)
-        throws IOException {
+    public int hashCode() {
+      int h = getClass().hashCode();
+      h = 31 * h + query.hashCode();
+      h = 31 * h + scoreField.hashCode();
+      return h;
+    }
 
-      final NumericDocValues values = DocValues.getNumeric(context.reader(), scoreValueField);
+    @Override
+    public Query rewrite(IndexReader reader) throws IOException {
+      Query rewritten = query.rewrite(reader);
+      if (rewritten != query) {
+        return new DocValueScoreQuery(rewritten, scoreField);
+      }
+      return super.rewrite(reader);
+    }
 
-      return new SimScorer() {
+    @Override
+    public Weight createWeight(IndexSearcher searcher, ScoreMode scoreMode, float boost) throws IOException {
+      if (scoreMode.needsScores() == false) {
+        return query.createWeight(searcher, scoreMode, boost);
+      }
+      Weight inner = query.createWeight(searcher, ScoreMode.COMPLETE_NO_SCORES, boost);
+      return new Weight(this) {
+        
         @Override
-        public float score(int doc, float freq) throws IOException {
-          if (doc != values.docID()) {
-            values.advance(doc);
-          }
-          if (doc == values.docID()) {
-            return Float.intBitsToFloat((int) values.longValue());
-          } else {
-            return 0f;
-          }
+        public boolean isCacheable(LeafReaderContext ctx) {
+          return true;
         }
-
+        
         @Override
-        public float maxScore(float maxFreq) {
-          return Float.MAX_VALUE;
+        public Scorer scorer(LeafReaderContext context) throws IOException {
+          Scorer innerScorer = inner.scorer(context);
+          NumericDocValues scoreFactors = DocValues.getNumeric(context.reader(), scoreField);
+          return new Scorer(this) {
+            
+            @Override
+            public float score() throws IOException {
+              if (scoreFactors.advanceExact(docID())) {
+                return Float.intBitsToFloat((int) scoreFactors.longValue());
+              }
+              return 0;
+            }
+            
+            @Override
+            public float maxScore() {
+              return Float.POSITIVE_INFINITY;
+            }
+            
+            @Override
+            public DocIdSetIterator iterator() {
+              return innerScorer.iterator();
+            }
+            
+            @Override
+            public int docID() {
+              return innerScorer.docID();
+            }
+          };
         }
-
+        
         @Override
-        public Explanation explain(int doc, Explanation freq) throws IOException {
-          return Explanation.match(score(doc, 0f), "indexDocValue(" + scoreValueField + ")");
+        public void extractTerms(Set<Term> terms) {
+          inner.extractTerms(terms);
+        }
+        
+        @Override
+        public Explanation explain(LeafReaderContext context, int doc) throws IOException {
+          Scorer s = scorer(context);
+          if (s != null) {
+            int advanced = s.iterator().advance(doc);
+            if (doc != advanced) {
+              return Explanation.match(s.score(), "match");
+            }
+          }
+          return Explanation.noMatch("no match");
         }
       };
     }
   }
+
 }

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/8fd7ead9/lucene/queries/src/java/org/apache/lucene/queries/function/valuesource/IDFValueSource.java
----------------------------------------------------------------------
diff --git a/lucene/queries/src/java/org/apache/lucene/queries/function/valuesource/IDFValueSource.java b/lucene/queries/src/java/org/apache/lucene/queries/function/valuesource/IDFValueSource.java
index 34e5697..4192f2d 100644
--- a/lucene/queries/src/java/org/apache/lucene/queries/function/valuesource/IDFValueSource.java
+++ b/lucene/queries/src/java/org/apache/lucene/queries/function/valuesource/IDFValueSource.java
@@ -47,7 +47,7 @@ public class IDFValueSource extends DocFreqValueSource {
   @Override
   public FunctionValues getValues(Map context, LeafReaderContext readerContext) throws IOException {
     IndexSearcher searcher = (IndexSearcher)context.get("searcher");
-    TFIDFSimilarity sim = asTFIDF(searcher.getSimilarity(true), field);
+    TFIDFSimilarity sim = asTFIDF(searcher.getSimilarity(), field);
     if (sim == null) {
       throw new UnsupportedOperationException("requires a TFIDFSimilarity (such as ClassicSimilarity)");
     }

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/8fd7ead9/lucene/queries/src/java/org/apache/lucene/queries/function/valuesource/NormValueSource.java
----------------------------------------------------------------------
diff --git a/lucene/queries/src/java/org/apache/lucene/queries/function/valuesource/NormValueSource.java b/lucene/queries/src/java/org/apache/lucene/queries/function/valuesource/NormValueSource.java
index ea63de9..682c0c0 100644
--- a/lucene/queries/src/java/org/apache/lucene/queries/function/valuesource/NormValueSource.java
+++ b/lucene/queries/src/java/org/apache/lucene/queries/function/valuesource/NormValueSource.java
@@ -25,11 +25,11 @@ import org.apache.lucene.queries.function.ValueSource;
 import org.apache.lucene.queries.function.docvalues.FloatDocValues;
 import org.apache.lucene.search.CollectionStatistics;
 import org.apache.lucene.search.IndexSearcher;
+import org.apache.lucene.search.LeafSimScorer;
 import org.apache.lucene.search.TermStatistics;
 import org.apache.lucene.search.similarities.TFIDFSimilarity;
 import org.apache.lucene.util.BytesRef;
 import org.apache.lucene.search.similarities.Similarity.SimScorer;
-import org.apache.lucene.search.similarities.Similarity.SimWeight;
 
 /** 
  * Function that returns the decoded norm for every document.
@@ -62,16 +62,16 @@ public class NormValueSource extends ValueSource {
   @Override
   public FunctionValues getValues(Map context, LeafReaderContext readerContext) throws IOException {
     IndexSearcher searcher = (IndexSearcher)context.get("searcher");
-    final TFIDFSimilarity similarity = IDFValueSource.asTFIDF(searcher.getSimilarity(true), field);
+    final TFIDFSimilarity similarity = IDFValueSource.asTFIDF(searcher.getSimilarity(), field);
     if (similarity == null) {
       throw new UnsupportedOperationException("requires a TFIDFSimilarity (such as ClassicSimilarity)");
     }
     // Only works if the contribution of the tf is 1 when the freq is 1 and contribution of the idf
     // is 1 when docCount == docFreq == 1
-    final SimWeight simWeight = similarity.computeWeight(1f,
+    final SimScorer simScorer = similarity.scorer(1f,
         new CollectionStatistics(field, 1, 1, 1, 1),
         new TermStatistics(new BytesRef("bogus"), 1, 1));
-    final SimScorer simScorer = similarity.simScorer(simWeight, readerContext);
+    final LeafSimScorer leafSimScorer = new LeafSimScorer(simScorer, readerContext.reader(), true, Float.POSITIVE_INFINITY);
     
     return new FloatDocValues(this) {
       int lastDocID = -1;
@@ -81,7 +81,7 @@ public class NormValueSource extends ValueSource {
           throw new AssertionError("docs out of order: lastDocID=" + lastDocID + " docID=" + docID);
         }
         lastDocID = docID;
-        return simScorer.score(docID, 1f);
+        return leafSimScorer.score(docID, 1f);
       }
     };
   }

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/8fd7ead9/lucene/queries/src/java/org/apache/lucene/queries/function/valuesource/TFValueSource.java
----------------------------------------------------------------------
diff --git a/lucene/queries/src/java/org/apache/lucene/queries/function/valuesource/TFValueSource.java b/lucene/queries/src/java/org/apache/lucene/queries/function/valuesource/TFValueSource.java
index baed0ff..731ab1f 100644
--- a/lucene/queries/src/java/org/apache/lucene/queries/function/valuesource/TFValueSource.java
+++ b/lucene/queries/src/java/org/apache/lucene/queries/function/valuesource/TFValueSource.java
@@ -51,7 +51,7 @@ public class TFValueSource extends TermFreqValueSource {
   public FunctionValues getValues(Map context, LeafReaderContext readerContext) throws IOException {
     final Terms terms = readerContext.reader().terms(indexedField);
     IndexSearcher searcher = (IndexSearcher)context.get("searcher");
-    final TFIDFSimilarity similarity = IDFValueSource.asTFIDF(searcher.getSimilarity(true), indexedField);
+    final TFIDFSimilarity similarity = IDFValueSource.asTFIDF(searcher.getSimilarity(), indexedField);
     if (similarity == null) {
       throw new UnsupportedOperationException("requires a TFIDFSimilarity (such as ClassicSimilarity)");
     }

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/8fd7ead9/lucene/queries/src/java/org/apache/lucene/queries/payloads/PayloadScoreQuery.java
----------------------------------------------------------------------
diff --git a/lucene/queries/src/java/org/apache/lucene/queries/payloads/PayloadScoreQuery.java b/lucene/queries/src/java/org/apache/lucene/queries/payloads/PayloadScoreQuery.java
index 6704ba1..c618886 100644
--- a/lucene/queries/src/java/org/apache/lucene/queries/payloads/PayloadScoreQuery.java
+++ b/lucene/queries/src/java/org/apache/lucene/queries/payloads/PayloadScoreQuery.java
@@ -28,10 +28,9 @@ import org.apache.lucene.index.Term;
 import org.apache.lucene.index.TermContext;
 import org.apache.lucene.search.Explanation;
 import org.apache.lucene.search.IndexSearcher;
+import org.apache.lucene.search.LeafSimScorer;
 import org.apache.lucene.search.Query;
 import org.apache.lucene.search.ScoreMode;
-import org.apache.lucene.search.similarities.Similarity;
-import org.apache.lucene.search.similarities.Similarity.SimScorer;
 import org.apache.lucene.search.spans.FilterSpans;
 import org.apache.lucene.search.spans.SpanCollector;
 import org.apache.lucene.search.spans.SpanQuery;
@@ -150,7 +149,7 @@ public class PayloadScoreQuery extends SpanQuery {
       Spans spans = getSpans(context, Postings.PAYLOADS);
       if (spans == null)
         return null;
-      SimScorer docScorer = innerWeight.getSimScorer(context);
+      LeafSimScorer docScorer = innerWeight.getSimScorer(context);
       PayloadSpans payloadSpans = new PayloadSpans(spans, decoder);
       return new PayloadSpanScorer(this, payloadSpans, docScorer);
     }
@@ -228,7 +227,7 @@ public class PayloadScoreQuery extends SpanQuery {
 
     private final PayloadSpans spans;
 
-    private PayloadSpanScorer(SpanWeight weight, PayloadSpans spans, Similarity.SimScorer docScorer) throws IOException {
+    private PayloadSpanScorer(SpanWeight weight, PayloadSpans spans, LeafSimScorer docScorer) throws IOException {
       super(weight, spans, docScorer);
       this.spans = spans;
     }

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/8fd7ead9/lucene/queries/src/java/org/apache/lucene/queries/payloads/SpanPayloadCheckQuery.java
----------------------------------------------------------------------
diff --git a/lucene/queries/src/java/org/apache/lucene/queries/payloads/SpanPayloadCheckQuery.java b/lucene/queries/src/java/org/apache/lucene/queries/payloads/SpanPayloadCheckQuery.java
index 8b23122..05d6682 100644
--- a/lucene/queries/src/java/org/apache/lucene/queries/payloads/SpanPayloadCheckQuery.java
+++ b/lucene/queries/src/java/org/apache/lucene/queries/payloads/SpanPayloadCheckQuery.java
@@ -28,9 +28,9 @@ import org.apache.lucene.index.Term;
 import org.apache.lucene.index.TermContext;
 import org.apache.lucene.index.Terms;
 import org.apache.lucene.search.IndexSearcher;
+import org.apache.lucene.search.LeafSimScorer;
 import org.apache.lucene.search.Query;
 import org.apache.lucene.search.ScoreMode;
-import org.apache.lucene.search.similarities.Similarity;
 import org.apache.lucene.search.spans.FilterSpans;
 import org.apache.lucene.search.spans.FilterSpans.AcceptStatus;
 import org.apache.lucene.search.spans.SpanCollector;
@@ -127,7 +127,7 @@ public class SpanPayloadCheckQuery extends SpanQuery {
       if (spans == null) {
         return null;
       }
-      final Similarity.SimScorer docScorer = getSimScorer(context);
+      final LeafSimScorer docScorer = getSimScorer(context);
       return new SpanScorer(this, spans, docScorer);
     }
 

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/8fd7ead9/lucene/queries/src/test/org/apache/lucene/queries/function/TestLongNormValueSource.java
----------------------------------------------------------------------
diff --git a/lucene/queries/src/test/org/apache/lucene/queries/function/TestLongNormValueSource.java b/lucene/queries/src/test/org/apache/lucene/queries/function/TestLongNormValueSource.java
index 842c117..ef0f476 100644
--- a/lucene/queries/src/test/org/apache/lucene/queries/function/TestLongNormValueSource.java
+++ b/lucene/queries/src/test/org/apache/lucene/queries/function/TestLongNormValueSource.java
@@ -82,7 +82,7 @@ public class TestLongNormValueSource extends LuceneTestCase {
   }
 
   public void testNorm() throws Exception {
-    Similarity saved = searcher.getSimilarity(true);
+    Similarity saved = searcher.getSimilarity();
     try {
       // no norm field (so agnostic to indexed similarity)
       searcher.setSimilarity(sim);

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/8fd7ead9/lucene/queries/src/test/org/apache/lucene/queries/function/TestValueSources.java
----------------------------------------------------------------------
diff --git a/lucene/queries/src/test/org/apache/lucene/queries/function/TestValueSources.java b/lucene/queries/src/test/org/apache/lucene/queries/function/TestValueSources.java
index 876fec8..b12f026 100644
--- a/lucene/queries/src/test/org/apache/lucene/queries/function/TestValueSources.java
+++ b/lucene/queries/src/test/org/apache/lucene/queries/function/TestValueSources.java
@@ -225,7 +225,7 @@ public class TestValueSources extends LuceneTestCase {
   }
   
   public void testIDF() throws Exception {
-    Similarity saved = searcher.getSimilarity(true);
+    Similarity saved = searcher.getSimilarity();
     try {
       searcher.setSimilarity(new ClassicSimilarity());
       ValueSource vs = new IDFValueSource("bogus", "bogus", "text", new BytesRef("test"));
@@ -362,7 +362,7 @@ public class TestValueSources extends LuceneTestCase {
   }
   
   public void testNorm() throws Exception {
-    Similarity saved = searcher.getSimilarity(true);
+    Similarity saved = searcher.getSimilarity();
     try {
       // no norm field (so agnostic to indexed similarity)
       searcher.setSimilarity(new ClassicSimilarity());
@@ -414,7 +414,7 @@ public class TestValueSources extends LuceneTestCase {
   }
 
   public void testQuery() throws Exception {
-    Similarity saved = searcher.getSimilarity(true);
+    Similarity saved = searcher.getSimilarity();
 
     try {
       searcher.setSimilarity(new ClassicSimilarity());
@@ -521,7 +521,7 @@ public class TestValueSources extends LuceneTestCase {
   }
   
   public void testTF() throws Exception {
-    Similarity saved = searcher.getSimilarity(true);
+    Similarity saved = searcher.getSimilarity();
     try {
       // no norm field (so agnostic to indexed similarity)
       searcher.setSimilarity(new ClassicSimilarity());

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/8fd7ead9/lucene/sandbox/src/java/org/apache/lucene/search/TermAutomatonQuery.java
----------------------------------------------------------------------
diff --git a/lucene/sandbox/src/java/org/apache/lucene/search/TermAutomatonQuery.java b/lucene/sandbox/src/java/org/apache/lucene/search/TermAutomatonQuery.java
index d5607da..92c8d59 100644
--- a/lucene/sandbox/src/java/org/apache/lucene/search/TermAutomatonQuery.java
+++ b/lucene/sandbox/src/java/org/apache/lucene/search/TermAutomatonQuery.java
@@ -335,14 +335,14 @@ public class TermAutomatonQuery extends Query {
   final class TermAutomatonWeight extends Weight {
     final Automaton automaton;
     private final Map<Integer,TermContext> termStates;
-    private final Similarity.SimWeight stats;
+    private final Similarity.SimScorer stats;
     private final Similarity similarity;
 
     public TermAutomatonWeight(Automaton automaton, IndexSearcher searcher, Map<Integer,TermContext> termStates, float boost) throws IOException {
       super(TermAutomatonQuery.this);
       this.automaton = automaton;
       this.termStates = termStates;
-      this.similarity = searcher.getSimilarity(true);
+      this.similarity = searcher.getSimilarity();
       List<TermStatistics> allTermStats = new ArrayList<>();
       for(Map.Entry<Integer,BytesRef> ent : idToTerm.entrySet()) {
         Integer termID = ent.getKey();
@@ -357,7 +357,7 @@ public class TermAutomatonQuery extends Query {
       if (allTermStats.isEmpty()) {
         stats = null; // no terms matched at all, will not use sim
       } else {
-        stats = similarity.computeWeight(boost, searcher.collectionStatistics(field),
+        stats = similarity.scorer(boost, searcher.collectionStatistics(field),
                                          allTermStats.toArray(new TermStatistics[allTermStats.size()]));
       }
     }
@@ -397,7 +397,7 @@ public class TermAutomatonQuery extends Query {
       }
 
       if (any) {
-        return new TermAutomatonScorer(this, enums, anyTermID, idToTerm, similarity.simScorer(stats, context));
+        return new TermAutomatonScorer(this, enums, anyTermID, idToTerm, new LeafSimScorer(stats, context.reader(), true, Float.POSITIVE_INFINITY));
       } else {
         return null;
       }

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/8fd7ead9/lucene/sandbox/src/java/org/apache/lucene/search/TermAutomatonScorer.java
----------------------------------------------------------------------
diff --git a/lucene/sandbox/src/java/org/apache/lucene/search/TermAutomatonScorer.java b/lucene/sandbox/src/java/org/apache/lucene/search/TermAutomatonScorer.java
index 27270e7..6094c01 100644
--- a/lucene/sandbox/src/java/org/apache/lucene/search/TermAutomatonScorer.java
+++ b/lucene/sandbox/src/java/org/apache/lucene/search/TermAutomatonScorer.java
@@ -21,7 +21,6 @@ import java.util.Map;
 
 import org.apache.lucene.search.TermAutomatonQuery.EnumAndScorer;
 import org.apache.lucene.search.TermAutomatonQuery.TermAutomatonWeight;
-import org.apache.lucene.search.similarities.Similarity;
 import org.apache.lucene.util.ArrayUtil;
 import org.apache.lucene.util.BytesRef;
 import org.apache.lucene.util.PriorityQueue;
@@ -47,7 +46,7 @@ class TermAutomatonScorer extends Scorer {
   // This is -1 if wildcard (null) terms were not used, else it's the id
   // of the wildcard term:
   private final int anyTermID;
-  private final Similarity.SimScorer docScorer;
+  private final LeafSimScorer docScorer;
 
   private int numSubsOnDoc;
 
@@ -56,7 +55,7 @@ class TermAutomatonScorer extends Scorer {
   private int docID = -1;
   private int freq;
 
-  public TermAutomatonScorer(TermAutomatonWeight weight, EnumAndScorer[] subs, int anyTermID, Map<Integer,BytesRef> idToTerm, Similarity.SimScorer docScorer) throws IOException {
+  public TermAutomatonScorer(TermAutomatonWeight weight, EnumAndScorer[] subs, int anyTermID, Map<Integer,BytesRef> idToTerm, LeafSimScorer docScorer) throws IOException {
     super(weight);
     //System.out.println("  automaton:\n" + weight.automaton.toDot());
     this.runAutomaton = new TermRunAutomaton(weight.automaton, subs.length);
@@ -362,7 +361,7 @@ class TermAutomatonScorer extends Scorer {
 
   @Override
   public float maxScore() {
-    return docScorer.maxScore(Float.POSITIVE_INFINITY);
+    return docScorer.maxScore();
   }
 
   static class TermRunAutomaton extends RunAutomaton {

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/8fd7ead9/lucene/test-framework/src/java/org/apache/lucene/index/BaseNormsFormatTestCase.java
----------------------------------------------------------------------
diff --git a/lucene/test-framework/src/java/org/apache/lucene/index/BaseNormsFormatTestCase.java b/lucene/test-framework/src/java/org/apache/lucene/index/BaseNormsFormatTestCase.java
index 370d009..617a721 100644
--- a/lucene/test-framework/src/java/org/apache/lucene/index/BaseNormsFormatTestCase.java
+++ b/lucene/test-framework/src/java/org/apache/lucene/index/BaseNormsFormatTestCase.java
@@ -534,12 +534,7 @@ public abstract class BaseNormsFormatTestCase extends BaseIndexFileFormatTestCas
     }
 
     @Override
-    public SimWeight computeWeight(float boost, CollectionStatistics collectionStats, TermStatistics... termStats) {
-      throw new UnsupportedOperationException();
-    }
-
-    @Override
-    public SimScorer simScorer(SimWeight weight, LeafReaderContext context) throws IOException {
+    public SimScorer scorer(float boost, CollectionStatistics collectionStats, TermStatistics... termStats) {
       throw new UnsupportedOperationException();
     }
   }

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/8fd7ead9/lucene/test-framework/src/java/org/apache/lucene/search/QueryUtils.java
----------------------------------------------------------------------
diff --git a/lucene/test-framework/src/java/org/apache/lucene/search/QueryUtils.java b/lucene/test-framework/src/java/org/apache/lucene/search/QueryUtils.java
index 1773977..fa11311 100644
--- a/lucene/test-framework/src/java/org/apache/lucene/search/QueryUtils.java
+++ b/lucene/test-framework/src/java/org/apache/lucene/search/QueryUtils.java
@@ -161,7 +161,7 @@ public class QueryUtils {
     };
 
     IndexSearcher out = LuceneTestCase.newSearcher(new MultiReader(readers));
-    out.setSimilarity(s.getSimilarity(true));
+    out.setSimilarity(s.getSimilarity());
     return out;
   }
 
@@ -373,7 +373,7 @@ public class QueryUtils {
             if (lastReader[0] != null) {
               final LeafReader previousReader = lastReader[0];
               IndexSearcher indexSearcher = LuceneTestCase.newSearcher(previousReader, false);
-              indexSearcher.setSimilarity(s.getSimilarity(true));
+              indexSearcher.setSimilarity(s.getSimilarity());
               Weight w = indexSearcher.createNormalizedWeight(q, ScoreMode.COMPLETE);
               LeafReaderContext ctx = (LeafReaderContext)indexSearcher.getTopReaderContext();
               Scorer scorer = w.scorer(ctx);
@@ -403,7 +403,7 @@ public class QueryUtils {
           // previous reader, hits NO_MORE_DOCS
           final LeafReader previousReader = lastReader[0];
           IndexSearcher indexSearcher = LuceneTestCase.newSearcher(previousReader, false);
-          indexSearcher.setSimilarity(s.getSimilarity(true));
+          indexSearcher.setSimilarity(s.getSimilarity());
           Weight w = indexSearcher.createNormalizedWeight(q, ScoreMode.COMPLETE);
           LeafReaderContext ctx = previousReader.getContext();
           Scorer scorer = w.scorer(ctx);
@@ -475,7 +475,7 @@ public class QueryUtils {
         if (lastReader[0] != null) {
           final LeafReader previousReader = lastReader[0];
           IndexSearcher indexSearcher = LuceneTestCase.newSearcher(previousReader, false);
-          indexSearcher.setSimilarity(s.getSimilarity(true));
+          indexSearcher.setSimilarity(s.getSimilarity());
           Weight w = indexSearcher.createNormalizedWeight(q, ScoreMode.COMPLETE);
           Scorer scorer = w.scorer((LeafReaderContext)indexSearcher.getTopReaderContext());
           if (scorer != null) {
@@ -503,7 +503,7 @@ public class QueryUtils {
       // previous reader, hits NO_MORE_DOCS
       final LeafReader previousReader = lastReader[0];
       IndexSearcher indexSearcher = LuceneTestCase.newSearcher(previousReader, false);
-      indexSearcher.setSimilarity(s.getSimilarity(true));
+      indexSearcher.setSimilarity(s.getSimilarity());
       Weight w = indexSearcher.createNormalizedWeight(q, ScoreMode.COMPLETE);
       Scorer scorer = w.scorer((LeafReaderContext)indexSearcher.getTopReaderContext());
       if (scorer != null) {

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/8fd7ead9/lucene/test-framework/src/java/org/apache/lucene/search/similarities/AssertingSimilarity.java
----------------------------------------------------------------------
diff --git a/lucene/test-framework/src/java/org/apache/lucene/search/similarities/AssertingSimilarity.java b/lucene/test-framework/src/java/org/apache/lucene/search/similarities/AssertingSimilarity.java
index ac41ea91..afc33ba 100644
--- a/lucene/test-framework/src/java/org/apache/lucene/search/similarities/AssertingSimilarity.java
+++ b/lucene/test-framework/src/java/org/apache/lucene/search/similarities/AssertingSimilarity.java
@@ -19,7 +19,6 @@ package org.apache.lucene.search.similarities;
 import java.io.IOException;
 
 import org.apache.lucene.index.FieldInvertState;
-import org.apache.lucene.index.LeafReaderContext;
 import org.apache.lucene.search.CollectionStatistics;
 import org.apache.lucene.search.Explanation;
 import org.apache.lucene.search.TermStatistics;
@@ -48,7 +47,7 @@ public class AssertingSimilarity extends Similarity {
   }
 
   @Override
-  public SimWeight computeWeight(float boost, CollectionStatistics collectionStats, TermStatistics... termStats) {
+  public SimScorer scorer(float boost, CollectionStatistics collectionStats, TermStatistics... termStats) {
     assert boost >= 0;
     assert collectionStats != null;
     assert termStats.length > 0;
@@ -56,70 +55,54 @@ public class AssertingSimilarity extends Similarity {
       assert term != null;
     }
     // TODO: check that TermStats is in bounds with respect to collection? e.g. docFreq <= maxDoc
-    SimWeight weight = delegate.computeWeight(boost, collectionStats, termStats);
-    assert weight != null;
-    return new AssertingWeight(weight, boost);
+    SimScorer scorer = delegate.scorer(boost, collectionStats, termStats);
+    assert scorer != null;
+    return new AssertingSimScorer(scorer, boost);
   }
   
-  static class AssertingWeight extends SimWeight {
-    final SimWeight delegate;
+  static class AssertingSimScorer extends SimScorer {
+    final SimScorer delegate;
     final float boost;
     
-    AssertingWeight(SimWeight delegate, float boost) {
+    AssertingSimScorer(SimScorer delegate, float boost) {
+      super(delegate.getField());
       this.delegate = delegate;
       this.boost = boost;
     }
-  }
-
-  @Override
-  public SimScorer simScorer(SimWeight weight, LeafReaderContext context) throws IOException {
-    assert weight != null;
-    assert context != null;
-    AssertingWeight assertingWeight = (AssertingWeight)weight;
-    SimScorer delegateScorer = delegate.simScorer(assertingWeight.delegate, context);
-    assert delegateScorer != null;
 
-    return new SimScorer() {
-      @Override
-      public float score(int doc, float freq) throws IOException {
-        // doc in bounds
-        assert doc >= 0;
-        assert doc < context.reader().maxDoc();
-        // freq in bounds
-        assert Float.isFinite(freq);
-        assert freq > 0;
-        // result in bounds
-        float score = delegateScorer.score(doc, freq);
-        assert Float.isFinite(score);
-        assert score <= maxScore(freq);
-        assert score >= 0;
-        return score;
-      }
+    @Override
+    public float score(float freq, long norm) throws IOException {
+      // freq in bounds
+      assert Float.isFinite(freq);
+      assert freq > 0;
+      // result in bounds
+      float score = delegate.score(freq, norm);
+      assert Float.isFinite(score);
+      assert score <= maxScore(freq);
+      assert score >= 0;
+      return score;
+    }
 
-      @Override
-      public float maxScore(float maxFreq) {
-        float maxScore = delegateScorer.maxScore(maxFreq);
-        assert Float.isNaN(maxScore) == false;
-        return maxScore;
-      }
+    @Override
+    public float maxScore(float maxFreq) {
+      float maxScore = delegate.maxScore(maxFreq);
+      assert Float.isNaN(maxScore) == false;
+      return maxScore;
+    }
 
-      @Override
-      public Explanation explain(int doc, Explanation freq) throws IOException {
-        // doc in bounds
-        assert doc >= 0;
-        assert doc < context.reader().maxDoc();
-        // freq in bounds 
-        assert freq != null;
-        assert Float.isFinite(freq.getValue().floatValue());
-        // result in bounds
-        Explanation explanation = delegateScorer.explain(doc, freq);
-        assert explanation != null;
-        assert Float.isFinite(explanation.getValue().floatValue());
-        // result matches score exactly
-        assert explanation.getValue().floatValue() == delegateScorer.score(doc, freq.getValue().floatValue());
-        return explanation;
-      }
-    };
+    @Override
+    public Explanation explain(Explanation freq, long norm) throws IOException {
+      // freq in bounds 
+      assert freq != null;
+      assert Float.isFinite(freq.getValue().floatValue());
+      // result in bounds
+      Explanation explanation = delegate.explain(freq, norm);
+      assert explanation != null;
+      assert Float.isFinite(explanation.getValue().floatValue());
+      // result matches score exactly
+      assert explanation.getValue().floatValue() == delegate.score(freq.getValue().floatValue(), norm);
+      return explanation;
+    }
   }
 
   @Override