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:12 UTC

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

Repository: lucene-solr
Updated Branches:
  refs/heads/master 8836fda95 -> 8fd7ead94


http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/8fd7ead9/lucene/test-framework/src/java/org/apache/lucene/search/similarities/BaseSimilarityTestCase.java
----------------------------------------------------------------------
diff --git a/lucene/test-framework/src/java/org/apache/lucene/search/similarities/BaseSimilarityTestCase.java b/lucene/test-framework/src/java/org/apache/lucene/search/similarities/BaseSimilarityTestCase.java
index 91e64c0..348584e 100644
--- a/lucene/test-framework/src/java/org/apache/lucene/search/similarities/BaseSimilarityTestCase.java
+++ b/lucene/test-framework/src/java/org/apache/lucene/search/similarities/BaseSimilarityTestCase.java
@@ -17,24 +17,18 @@
 package org.apache.lucene.search.similarities;
 
 import java.io.IOException;
-import java.util.ArrayList;
-import java.util.List;
 import java.util.Random;
 
 import org.apache.lucene.document.Document;
-import org.apache.lucene.document.Field;
 import org.apache.lucene.document.FieldType;
 import org.apache.lucene.document.TextField;
-import org.apache.lucene.index.FilterLeafReader;
 import org.apache.lucene.index.LeafReader;
-import org.apache.lucene.index.NumericDocValues;
 import org.apache.lucene.index.RandomIndexWriter;
 import org.apache.lucene.search.CheckHits;
 import org.apache.lucene.search.CollectionStatistics;
 import org.apache.lucene.search.Explanation;
 import org.apache.lucene.search.TermStatistics;
 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.IOUtils;
@@ -54,119 +48,28 @@ import org.junit.BeforeClass;
  * test fails to catch then this test needs to be improved! */
 public abstract class BaseSimilarityTestCase extends LuceneTestCase {
 
-  static LeafReader WITHOUT_NORM;
-  static Directory WITHOUT_NORM_DIR;
-
-  static LeafReader WITH_NORM_BASE;
-  static Directory WITH_NORM_DIR;
-  static List<LeafReader> NORM_VALUES;
+  static LeafReader READER;
+  static Directory DIR;
   
   @BeforeClass
   public static void beforeClass() throws Exception {
-    // without norms
-    WITHOUT_NORM_DIR = newDirectory();
-    RandomIndexWriter writer = new RandomIndexWriter(random(), WITHOUT_NORM_DIR);
-    Document doc = new Document();
-    doc.add(newTextField("field", "value", Field.Store.NO));
-    writer.addDocument(doc);
-    WITHOUT_NORM = getOnlyLeafReader(writer.getReader());
-    writer.close();
-
     // with norms
-    WITH_NORM_DIR = newDirectory();
-    writer = new RandomIndexWriter(random(), WITH_NORM_DIR);
-    doc = new Document();
+    DIR = newDirectory();
+    RandomIndexWriter writer = new RandomIndexWriter(random(), DIR);
+    Document doc = new Document();
     FieldType fieldType = new FieldType(TextField.TYPE_NOT_STORED);
     fieldType.setOmitNorms(true);
     doc.add(newField("field", "value", fieldType));
     writer.addDocument(doc);
-    WITH_NORM_BASE = getOnlyLeafReader(writer.getReader());
+    READER = getOnlyLeafReader(writer.getReader());
     writer.close();
-    
-    // all possible norm values for the doc
-    NORM_VALUES = new ArrayList<>();
-    NORM_VALUES.add(WITHOUT_NORM);
-    for (int i = 1; i < 256; i++) {
-      final long value = i;
-      NORM_VALUES.add(new FilterLeafReader(WITH_NORM_BASE) {
-        @Override
-        public CacheHelper getCoreCacheHelper() {
-          return null;
-        }
-
-        @Override
-        public CacheHelper getReaderCacheHelper() {
-          return null;
-        }
-
-        @Override
-        public NumericDocValues getNormValues(String field) throws IOException {
-          if (field.equals("field")) {
-            return new CannedNorm(value);
-          } else {
-            return super.getNormValues(field);
-          }
-        }
-      });
-    }
   }
   
   @AfterClass
   public static void afterClass() throws Exception {
-    IOUtils.close(WITH_NORM_BASE, WITH_NORM_DIR, WITHOUT_NORM, WITHOUT_NORM_DIR);
-    WITH_NORM_BASE = WITHOUT_NORM = null;
-    WITH_NORM_DIR = WITHOUT_NORM_DIR = null;
-    NORM_VALUES = null;
-  }
-  
-  /** 1-document norms impl of the given value */
-  static class CannedNorm extends NumericDocValues {
-    int docID = -1;
-    final long value;
-    
-    CannedNorm(long value) {
-      this.value = value;
-    }
-
-    @Override
-    public long longValue() throws IOException {
-      return value;
-    }
-
-    @Override
-    public boolean advanceExact(int target) throws IOException {
-      assert target == 0;
-      docID = target;
-      return true;
-    }
-
-    @Override
-    public int docID() {
-      return docID;
-    }
-
-    @Override
-    public int nextDoc() throws IOException {
-      if (docID == -1) {
-        return docID = 0;
-      } else {
-        return docID = NO_MORE_DOCS;
-      }
-    }
-
-    @Override
-    public int advance(int target) throws IOException {
-      if (target == 0) {
-        return docID = 0;
-      } else {
-        return docID = NO_MORE_DOCS;
-      }
-    }
-
-    @Override
-    public long cost() {
-      return 0;
-    }
+    IOUtils.close(READER, DIR);
+    READER = null;
+    DIR = null;
   }
 
   /**
@@ -354,7 +257,7 @@ public abstract class BaseSimilarityTestCase extends LuceneTestCase {
       Similarity similarity = getSimilarity(random);
       for (int j = 0; j < 10; j++) {
         // for each norm value...
-        for (int k = 0; k < NORM_VALUES.size(); k++) {
+        for (int k = 1; k < 256; k++) {
           CollectionStatistics corpus = newCorpus(random, k);
           for (int l = 0; l < 10; l++) {
             TermStatistics term = newTerm(random, corpus);
@@ -441,17 +344,16 @@ public abstract class BaseSimilarityTestCase extends LuceneTestCase {
   /** runs for a single test case, so that if you hit a test failure you can write a reproducer just for that scenario */
   private static void doTestScoring(Similarity similarity, CollectionStatistics corpus, TermStatistics term, float boost, float freq, int norm) throws IOException {
     boolean success = false;
-    SimWeight weight = similarity.computeWeight(boost, corpus, term);
-    SimScorer scorer = similarity.simScorer(weight, NORM_VALUES.get(norm).getContext());
+    SimScorer scorer = similarity.scorer(boost, corpus, term);
     try {
-      float score = scorer.score(0, freq);
+      float score = scorer.score(freq, norm);
       // check that score isn't infinite or negative
       assertTrue("infinite/NaN score: " + score, Float.isFinite(score));
       assertTrue("negative score: " + score, score >= 0);
       float maxScore = scorer.maxScore(freq);
       assertTrue("score > maxScore: " + score + " > " + maxScore, score <= maxScore);
       // check explanation matches
-      Explanation explanation = scorer.explain(0, Explanation.match(freq, "freq, occurrences of term within document"));
+      Explanation explanation = scorer.explain(Explanation.match(freq, "freq, occurrences of term within document"), norm);
       if (score != explanation.getValue().doubleValue()) {
         fail("expected: " + score + ", got: " + explanation);
       }
@@ -467,12 +369,12 @@ public abstract class BaseSimilarityTestCase extends LuceneTestCase {
         prevFreq = Math.nextDown(freq);
       }
       
-      float prevScore = scorer.score(0, prevFreq);
+      float prevScore = scorer.score(prevFreq, norm);
       // check that score isn't infinite or negative
       assertTrue(Float.isFinite(prevScore));
       assertTrue(prevScore >= 0);
       // check explanation matches
-      Explanation prevExplanation = scorer.explain(0, Explanation.match(prevFreq, "freq, occurrences of term within document"));
+      Explanation prevExplanation = scorer.explain(Explanation.match(prevFreq, "freq, occurrences of term within document"), norm);
       if (prevScore != prevExplanation.getValue().doubleValue()) {
         fail("expected: " + prevScore + ", got: " + prevExplanation);
       }
@@ -486,13 +388,12 @@ public abstract class BaseSimilarityTestCase extends LuceneTestCase {
       
       // check score(norm-1), given the same freq it should be >= score(norm) [scores non-decreasing as docs get shorter]
       if (norm > 1) {
-        SimScorer prevNormScorer = similarity.simScorer(weight, NORM_VALUES.get(norm - 1).getContext());
-        float prevNormScore = prevNormScorer.score(0, freq);
+        float prevNormScore = scorer.score(freq, norm - 1);
         // check that score isn't infinite or negative
         assertTrue(Float.isFinite(prevNormScore));
         assertTrue(prevNormScore >= 0);
         // check explanation matches
-        Explanation prevNormExplanation = prevNormScorer.explain(0, Explanation.match(freq, "freq, occurrences of term within document"));
+        Explanation prevNormExplanation = scorer.explain(Explanation.match(freq, "freq, occurrences of term within document"), norm - 1);
         if (prevNormScore != prevNormExplanation.getValue().doubleValue()) {
           fail("expected: " + prevNormScore + ", got: " + prevNormExplanation);
         }
@@ -508,14 +409,13 @@ public abstract class BaseSimilarityTestCase extends LuceneTestCase {
       // check score(term-1), given the same freq/norm it should be >= score(term) [scores non-decreasing as terms get rarer]
       if (term.docFreq() > 1 && freq < term.totalTermFreq()) {
         TermStatistics prevTerm = new TermStatistics(term.term(), term.docFreq() - 1, term.totalTermFreq() - 1);
-        SimWeight prevWeight = similarity.computeWeight(boost, corpus, term);
-        SimScorer prevTermScorer = similarity.simScorer(prevWeight, NORM_VALUES.get(norm).getContext());
-        float prevTermScore = prevTermScorer.score(0, freq);
+        SimScorer prevTermScorer = similarity.scorer(boost, corpus, term);
+        float prevTermScore = prevTermScorer.score(freq, norm);
         // check that score isn't infinite or negative
         assertTrue(Float.isFinite(prevTermScore));
         assertTrue(prevTermScore >= 0);
         // check explanation matches
-        Explanation prevTermExplanation = prevTermScorer.explain(0, Explanation.match(freq, "freq, occurrences of term within document"));
+        Explanation prevTermExplanation = prevTermScorer.explain(Explanation.match(freq, "freq, occurrences of term within document"), norm);
         if (prevTermScore != prevTermExplanation.getValue().doubleValue()) {
           fail("expected: " + prevTermScore + ", got: " + prevTermExplanation);
         }

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/8fd7ead9/lucene/test-framework/src/java/org/apache/lucene/search/spans/AssertingSpanWeight.java
----------------------------------------------------------------------
diff --git a/lucene/test-framework/src/java/org/apache/lucene/search/spans/AssertingSpanWeight.java b/lucene/test-framework/src/java/org/apache/lucene/search/spans/AssertingSpanWeight.java
index a9956b3..65f9599 100644
--- a/lucene/test-framework/src/java/org/apache/lucene/search/spans/AssertingSpanWeight.java
+++ b/lucene/test-framework/src/java/org/apache/lucene/search/spans/AssertingSpanWeight.java
@@ -25,7 +25,7 @@ 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.similarities.Similarity;
+import org.apache.lucene.search.LeafSimScorer;
 
 /**
  * Wraps a SpanWeight with additional asserts
@@ -58,7 +58,7 @@ public class AssertingSpanWeight extends SpanWeight {
   }
 
   @Override
-  public Similarity.SimScorer getSimScorer(LeafReaderContext context) throws IOException {
+  public LeafSimScorer getSimScorer(LeafReaderContext context) throws IOException {
     return in.getSimScorer(context);
   }
 

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/8fd7ead9/solr/core/src/test/org/apache/solr/search/similarities/BaseSimilarityTestCase.java
----------------------------------------------------------------------
diff --git a/solr/core/src/test/org/apache/solr/search/similarities/BaseSimilarityTestCase.java b/solr/core/src/test/org/apache/solr/search/similarities/BaseSimilarityTestCase.java
index d782c2f..6ce4331 100644
--- a/solr/core/src/test/org/apache/solr/search/similarities/BaseSimilarityTestCase.java
+++ b/solr/core/src/test/org/apache/solr/search/similarities/BaseSimilarityTestCase.java
@@ -29,7 +29,7 @@ public abstract class BaseSimilarityTestCase extends SolrTestCaseJ4 {
   protected Similarity getSimilarity(String field) {
     SolrCore core = h.getCore();
     RefCounted<SolrIndexSearcher> searcher = core.getSearcher();
-    Similarity sim = searcher.get().getSimilarity(true);
+    Similarity sim = searcher.get().getSimilarity();
     searcher.decref();
     while (sim instanceof PerFieldSimilarityWrapper) {
       sim = ((PerFieldSimilarityWrapper)sim).get(field);


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

Posted by jp...@apache.org.
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


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

Posted by jp...@apache.org.
LUCENE-8116: SimScorer now only takes a frequency and a norm as per-document scoring factors.


Project: http://git-wip-us.apache.org/repos/asf/lucene-solr/repo
Commit: http://git-wip-us.apache.org/repos/asf/lucene-solr/commit/8fd7ead9
Tree: http://git-wip-us.apache.org/repos/asf/lucene-solr/tree/8fd7ead9
Diff: http://git-wip-us.apache.org/repos/asf/lucene-solr/diff/8fd7ead9

Branch: refs/heads/master
Commit: 8fd7ead940f69a892dfc951a1aa042e8cae806c1
Parents: 8836fda
Author: Adrien Grand <jp...@gmail.com>
Authored: Thu Jan 4 15:13:36 2018 +0100
Committer: Adrien Grand <jp...@gmail.com>
Committed: Thu Jan 4 15:13:36 2018 +0100

----------------------------------------------------------------------
 lucene/CHANGES.txt                              |   6 +
 .../classification/KNearestFuzzyClassifier.java |   2 +-
 .../KNearestNeighborClassifier.java             |   2 +-
 .../org/apache/lucene/search/BooleanWeight.java |   2 +-
 .../apache/lucene/search/ExactPhraseScorer.java |   7 +-
 .../org/apache/lucene/search/IndexSearcher.java |  45 +----
 .../org/apache/lucene/search/LeafSimScorer.java |  74 +++++++
 .../apache/lucene/search/MultiPhraseQuery.java  |  29 +--
 .../org/apache/lucene/search/PhraseQuery.java   |  13 +-
 .../lucene/search/SloppyPhraseScorer.java       |   7 +-
 .../org/apache/lucene/search/SynonymQuery.java  |  35 ++--
 .../org/apache/lucene/search/TermQuery.java     |  15 +-
 .../org/apache/lucene/search/TermScorer.java    |  11 +-
 .../org/apache/lucene/search/package-info.java  |   4 +-
 .../lucene/search/similarities/Axiomatic.java   |  10 +-
 .../search/similarities/BM25Similarity.java     | 142 +++++---------
 .../lucene/search/similarities/BasicStats.java  |   2 +-
 .../search/similarities/BooleanSimilarity.java  |  51 ++---
 .../search/similarities/DFISimilarity.java      |   6 +-
 .../search/similarities/DFRSimilarity.java      |   8 +-
 .../search/similarities/IBSimilarity.java       |   8 +-
 .../similarities/LMDirichletSimilarity.java     |  10 +-
 .../similarities/LMJelinekMercerSimilarity.java |  10 +-
 .../search/similarities/LMSimilarity.java       |   2 +-
 .../search/similarities/MultiSimilarity.java    |  37 +---
 .../similarities/PerFieldSimilarityWrapper.java |  22 +--
 .../lucene/search/similarities/Similarity.java  |  99 ++++------
 .../search/similarities/SimilarityBase.java     |  79 +++-----
 .../search/similarities/TFIDFSimilarity.java    | 133 +++++--------
 .../apache/lucene/search/spans/SpanScorer.java  |   6 +-
 .../apache/lucene/search/spans/SpanWeight.java  |  22 +--
 .../apache/lucene/search/spans/TermSpans.java   |   4 +-
 .../apache/lucene/index/TestCustomNorms.java    |   7 +-
 .../apache/lucene/index/TestCustomTermFreq.java |   9 +-
 .../lucene/index/TestFieldInvertState.java      |   8 +-
 .../apache/lucene/index/TestIndexSorting.java   |   9 +-
 .../lucene/index/TestMaxTermFrequency.java      |  12 +-
 .../test/org/apache/lucene/index/TestNorms.java |   7 +-
 .../lucene/index/TestUniqueTermCount.java       |   8 +-
 .../apache/lucene/search/JustCompileSearch.java |   7 +-
 .../org/apache/lucene/search/TestBoolean2.java  |   4 +-
 .../search/TestBooleanQueryVisitSubscorers.java |  12 +-
 .../lucene/search/TestBooleanRewrites.java      |   2 +-
 .../apache/lucene/search/TestConjunctions.java  |  12 +-
 .../lucene/search/TestDocValuesScoring.java     | 192 -------------------
 .../lucene/search/TestMinShouldMatch2.java      |   9 +-
 .../lucene/search/TestSimilarityProvider.java   |  23 +--
 .../lucene/search/TestSubScorerFreqs.java       |  12 +-
 .../similarities/TestClassicSimilarity.java     |   3 +-
 .../search/similarities/TestSimilarityBase.java |  11 +-
 .../search/spans/TestFieldMaskingSpanQuery.java |   4 +-
 .../lucene/index/memory/TestMemoryIndex.java    |   8 +-
 .../search/TestDiversifiedTopDocsCollector.java | 132 ++++++++-----
 .../function/valuesource/IDFValueSource.java    |   2 +-
 .../function/valuesource/NormValueSource.java   |  10 +-
 .../function/valuesource/TFValueSource.java     |   2 +-
 .../queries/payloads/PayloadScoreQuery.java     |   7 +-
 .../queries/payloads/SpanPayloadCheckQuery.java |   4 +-
 .../function/TestLongNormValueSource.java       |   2 +-
 .../queries/function/TestValueSources.java      |   8 +-
 .../lucene/search/TermAutomatonQuery.java       |   8 +-
 .../lucene/search/TermAutomatonScorer.java      |   7 +-
 .../lucene/index/BaseNormsFormatTestCase.java   |   7 +-
 .../org/apache/lucene/search/QueryUtils.java    |  10 +-
 .../similarities/AssertingSimilarity.java       |  95 ++++-----
 .../similarities/BaseSimilarityTestCase.java    | 140 ++------------
 .../search/spans/AssertingSpanWeight.java       |   4 +-
 .../similarities/BaseSimilarityTestCase.java    |   2 +-
 68 files changed, 606 insertions(+), 1096 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/8fd7ead9/lucene/CHANGES.txt
----------------------------------------------------------------------
diff --git a/lucene/CHANGES.txt b/lucene/CHANGES.txt
index ff94809..16050d1 100644
--- a/lucene/CHANGES.txt
+++ b/lucene/CHANGES.txt
@@ -32,6 +32,9 @@ API Changes
 * LUCENE-8012: Explanation now takes Number rather than float (Alan Woodward, 
   Robert Muir)
 
+* LUCENE-8116: SimScorer now only takes a frequency and a norm as per-document
+  scoring factors. (Adrien Grand)
+
 Changes in Runtime Behavior
 
 * LUCENE-7837: Indices that were created before the previous major version
@@ -46,6 +49,9 @@ Changes in Runtime Behavior
 * LUCENE-7996: FunctionQuery and FunctionScoreQuery now return a score of 0
   when the function produces a negative value. (Adrien Grand)
 
+* LUCENE-8116: Similarities now score fields that omit norms as if the norm was
+  1. This might change score values on fields that omit norms. (Adrien Grand)
+
 Improvements
 
 * LUCENE-7997: Add BaseSimilarityTestCase to sanity check similarities.

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/8fd7ead9/lucene/classification/src/java/org/apache/lucene/classification/KNearestFuzzyClassifier.java
----------------------------------------------------------------------
diff --git a/lucene/classification/src/java/org/apache/lucene/classification/KNearestFuzzyClassifier.java b/lucene/classification/src/java/org/apache/lucene/classification/KNearestFuzzyClassifier.java
index cbd241b..14f9a27 100644
--- a/lucene/classification/src/java/org/apache/lucene/classification/KNearestFuzzyClassifier.java
+++ b/lucene/classification/src/java/org/apache/lucene/classification/KNearestFuzzyClassifier.java
@@ -213,7 +213,7 @@ public class KNearestFuzzyClassifier implements Classifier<BytesRef> {
         ", classFieldName='" + classFieldName + '\'' +
         ", k=" + k +
         ", query=" + query +
-        ", similarity=" + indexSearcher.getSimilarity(true) +
+        ", similarity=" + indexSearcher.getSimilarity() +
         '}';
   }
 }

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/8fd7ead9/lucene/classification/src/java/org/apache/lucene/classification/KNearestNeighborClassifier.java
----------------------------------------------------------------------
diff --git a/lucene/classification/src/java/org/apache/lucene/classification/KNearestNeighborClassifier.java b/lucene/classification/src/java/org/apache/lucene/classification/KNearestNeighborClassifier.java
index f0391f4..e6ad4a3 100644
--- a/lucene/classification/src/java/org/apache/lucene/classification/KNearestNeighborClassifier.java
+++ b/lucene/classification/src/java/org/apache/lucene/classification/KNearestNeighborClassifier.java
@@ -251,7 +251,7 @@ public class KNearestNeighborClassifier implements Classifier<BytesRef> {
         ", classFieldName='" + classFieldName + '\'' +
         ", k=" + k +
         ", query=" + query +
-        ", similarity=" + indexSearcher.getSimilarity(true) +
+        ", similarity=" + indexSearcher.getSimilarity() +
         '}';
   }
 }

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/8fd7ead9/lucene/core/src/java/org/apache/lucene/search/BooleanWeight.java
----------------------------------------------------------------------
diff --git a/lucene/core/src/java/org/apache/lucene/search/BooleanWeight.java b/lucene/core/src/java/org/apache/lucene/search/BooleanWeight.java
index 900a77f..fffdd09 100644
--- a/lucene/core/src/java/org/apache/lucene/search/BooleanWeight.java
+++ b/lucene/core/src/java/org/apache/lucene/search/BooleanWeight.java
@@ -48,7 +48,7 @@ final class BooleanWeight extends Weight {
     super(query);
     this.query = query;
     this.scoreMode = scoreMode;
-    this.similarity = searcher.getSimilarity(scoreMode.needsScores());
+    this.similarity = searcher.getSimilarity();
     weights = new ArrayList<>();
     for (BooleanClause c : query) {
       Weight w = searcher.createWeight(c.getQuery(), c.isScoring() ? scoreMode : ScoreMode.COMPLETE_NO_SCORES, boost);

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/8fd7ead9/lucene/core/src/java/org/apache/lucene/search/ExactPhraseScorer.java
----------------------------------------------------------------------
diff --git a/lucene/core/src/java/org/apache/lucene/search/ExactPhraseScorer.java b/lucene/core/src/java/org/apache/lucene/search/ExactPhraseScorer.java
index f4a7ca7..e2d6d80 100644
--- a/lucene/core/src/java/org/apache/lucene/search/ExactPhraseScorer.java
+++ b/lucene/core/src/java/org/apache/lucene/search/ExactPhraseScorer.java
@@ -22,7 +22,6 @@ import java.util.ArrayList;
 import java.util.List;
 
 import org.apache.lucene.index.PostingsEnum;
-import org.apache.lucene.search.similarities.Similarity;
 
 final class ExactPhraseScorer extends Scorer {
 
@@ -42,13 +41,13 @@ final class ExactPhraseScorer extends Scorer {
 
   private int freq;
 
-  private final Similarity.SimScorer docScorer;
+  private final LeafSimScorer docScorer;
   private final boolean needsScores, needsTotalHitCount;
   private float matchCost;
   private float minCompetitiveScore;
 
   ExactPhraseScorer(Weight weight, PhraseQuery.PostingsAndFreq[] postings,
-                    Similarity.SimScorer docScorer, ScoreMode scoreMode,
+                    LeafSimScorer docScorer, ScoreMode scoreMode,
                     float matchCost) throws IOException {
     super(weight);
     this.docScorer = docScorer;
@@ -123,7 +122,7 @@ final class ExactPhraseScorer extends Scorer {
 
   @Override
   public float maxScore() {
-    return docScorer.maxScore(Integer.MAX_VALUE);
+    return docScorer.maxScore();
   }
 
   /** Advance the given pos enum to the first doc on or after {@code target}.

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/8fd7ead9/lucene/core/src/java/org/apache/lucene/search/IndexSearcher.java
----------------------------------------------------------------------
diff --git a/lucene/core/src/java/org/apache/lucene/search/IndexSearcher.java b/lucene/core/src/java/org/apache/lucene/search/IndexSearcher.java
index 5ee815c..fc87563 100644
--- a/lucene/core/src/java/org/apache/lucene/search/IndexSearcher.java
+++ b/lucene/core/src/java/org/apache/lucene/search/IndexSearcher.java
@@ -32,7 +32,6 @@ import java.util.concurrent.Future;
 
 import org.apache.lucene.document.Document;
 import org.apache.lucene.index.DirectoryReader;
-import org.apache.lucene.index.FieldInvertState;
 import org.apache.lucene.index.IndexReader;
 import org.apache.lucene.index.IndexReaderContext;
 import org.apache.lucene.index.IndexWriter;
@@ -75,36 +74,6 @@ import org.apache.lucene.util.ThreadInterruptedException;
  */
 public class IndexSearcher {
 
-  /** A search-time {@link Similarity} that does not make use of scoring factors
-   *  and may be used when scores are not needed. */
-  private static final Similarity NON_SCORING_SIMILARITY = new Similarity() {
-
-    @Override
-    public long computeNorm(FieldInvertState state) {
-      throw new UnsupportedOperationException("This Similarity may only be used for searching, not indexing");
-    }
-
-    @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() {
-        @Override
-        public float score(int doc, float freq) {
-          return 0f;
-        }
-        @Override
-        public float maxScore(float maxFreq) {
-          return 0f;
-        }
-      };
-    }
-
-  };
-
   private static QueryCache DEFAULT_QUERY_CACHE;
   private static QueryCachingPolicy DEFAULT_CACHING_POLICY = new UsageTrackingQueryCachingPolicy();
   static {
@@ -136,7 +105,7 @@ public class IndexSearcher {
    * Expert: returns a default Similarity instance.
    * In general, this method is only called to initialize searchers and writers.
    * User code and query implementations should respect
-   * {@link IndexSearcher#getSimilarity(boolean)}.
+   * {@link IndexSearcher#getSimilarity()}.
    * @lucene.internal
    */
   public static Similarity getDefaultSimilarity() {
@@ -329,15 +298,11 @@ public class IndexSearcher {
     this.similarity = similarity;
   }
 
-  /** Expert: Get the {@link Similarity} to use to compute scores. When
-   *  {@code needsScores} is {@code false}, this method will return a simple
-   *  {@link Similarity} that does not leverage scoring factors such as norms.
-   *  When {@code needsScores} is {@code true}, this returns the
+  /** Expert: Get the {@link Similarity} to use to compute scores. This returns the
    *  {@link Similarity} that has been set through {@link #setSimilarity(Similarity)}
-   *  or the {@link #getDefaultSimilarity()} default {@link Similarity} if none
-   *  has been set explicitly. */
-  public Similarity getSimilarity(boolean needsScores) {
-    return needsScores ? similarity : NON_SCORING_SIMILARITY;
+   *  or the default {@link Similarity} if none has been set explicitly. */
+  public Similarity getSimilarity() {
+    return similarity;
   }
 
   /**

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/8fd7ead9/lucene/core/src/java/org/apache/lucene/search/LeafSimScorer.java
----------------------------------------------------------------------
diff --git a/lucene/core/src/java/org/apache/lucene/search/LeafSimScorer.java b/lucene/core/src/java/org/apache/lucene/search/LeafSimScorer.java
new file mode 100644
index 0000000..52b7d92
--- /dev/null
+++ b/lucene/core/src/java/org/apache/lucene/search/LeafSimScorer.java
@@ -0,0 +1,74 @@
+/*
+ * 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.index.LeafReader;
+import org.apache.lucene.index.NumericDocValues;
+import org.apache.lucene.search.similarities.Similarity.SimScorer;
+
+/**
+ * {@link SimScorer} on a specific {@link LeafReader}.
+ */
+public final class LeafSimScorer {
+
+  private final SimScorer scorer;
+  private final NumericDocValues norms;
+  private final float maxScore;
+
+  /**
+   * Sole constructor: Score documents of {@code reader} with {@code scorer}.
+   */
+  public LeafSimScorer(SimScorer scorer, LeafReader reader, boolean needsScores, float maxFreq) throws IOException {
+    this.scorer = scorer;
+    norms = needsScores ? reader.getNormValues(scorer.getField()) : null;
+    maxScore = scorer.maxScore(maxFreq);
+  }
+
+  private long getNormValue(int doc) throws IOException {
+    if (norms != null) {
+      boolean found = norms.advanceExact(doc);
+      assert found;
+      return norms.longValue();
+    } else {
+      return 1L; // default norm
+    }
+  }
+
+  /** Score the provided document assuming the given term document frequency.
+   *  This method must be called on non-decreasing sequences of doc ids.
+   *  @see SimScorer#score(float, long) */
+  public float score(int doc, float freq) throws IOException {
+    return scorer.score(freq, getNormValue(doc));
+  }
+
+  /** Explain the score for the provided document assuming the given term document frequency.
+   *  This method must be called on non-decreasing sequences of doc ids.
+   *  @see SimScorer#explain(Explanation, long) */
+  public Explanation explain(int doc, Explanation freqExpl) throws IOException {
+    return scorer.explain(freqExpl, getNormValue(doc));
+  }
+
+  /**
+   * Return an upper bound of the score.
+   * @see SimScorer#maxScore(float)
+   */
+  public float maxScore() {
+    return maxScore;
+  }
+}

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/8fd7ead9/lucene/core/src/java/org/apache/lucene/search/MultiPhraseQuery.java
----------------------------------------------------------------------
diff --git a/lucene/core/src/java/org/apache/lucene/search/MultiPhraseQuery.java b/lucene/core/src/java/org/apache/lucene/search/MultiPhraseQuery.java
index 34361a7..941416e 100644
--- a/lucene/core/src/java/org/apache/lucene/search/MultiPhraseQuery.java
+++ b/lucene/core/src/java/org/apache/lucene/search/MultiPhraseQuery.java
@@ -18,19 +18,26 @@ package org.apache.lucene.search;
 
 
 import java.io.IOException;
-import java.util.*;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.Objects;
+import java.util.Set;
 
-import org.apache.lucene.index.LeafReaderContext;
-import org.apache.lucene.index.LeafReader;
-import org.apache.lucene.index.PostingsEnum;
 import org.apache.lucene.index.IndexReader;
 import org.apache.lucene.index.IndexReaderContext;
+import org.apache.lucene.index.LeafReader;
+import org.apache.lucene.index.LeafReaderContext;
+import org.apache.lucene.index.PostingsEnum;
 import org.apache.lucene.index.Term;
 import org.apache.lucene.index.TermContext;
 import org.apache.lucene.index.TermState;
 import org.apache.lucene.index.Terms;
 import org.apache.lucene.index.TermsEnum;
-import org.apache.lucene.search.similarities.Similarity.SimScorer;
 import org.apache.lucene.search.similarities.Similarity;
 import org.apache.lucene.util.ArrayUtil;
 import org.apache.lucene.util.BytesRef;
@@ -183,7 +190,7 @@ public class MultiPhraseQuery extends Query {
 
   private class MultiPhraseWeight extends Weight {
     private final Similarity similarity;
-    private final Similarity.SimWeight stats;
+    private final Similarity.SimScorer stats;
     private final Map<Term,TermContext> termContexts = new HashMap<>();
     private final ScoreMode scoreMode;
 
@@ -191,7 +198,7 @@ public class MultiPhraseQuery extends Query {
       throws IOException {
       super(MultiPhraseQuery.this);
       this.scoreMode = scoreMode;
-      this.similarity = searcher.getSimilarity(scoreMode.needsScores());
+      this.similarity = searcher.getSimilarity();
       final IndexReaderContext context = searcher.getTopReaderContext();
 
       // compute idf
@@ -212,7 +219,7 @@ public class MultiPhraseQuery extends Query {
       if (allTermStats.isEmpty()) {
         stats = null; // none of the terms were found, we won't use sim at all
       } else {
-        stats = similarity.computeWeight(
+        stats = similarity.scorer(
           boost,
           searcher.collectionStatistics(field),
           allTermStats.toArray(new TermStatistics[allTermStats.size()]));
@@ -282,11 +289,11 @@ public class MultiPhraseQuery extends Query {
 
       if (slop == 0) {
         return new ExactPhraseScorer(this, postingsFreqs,
-                                      similarity.simScorer(stats, context),
+                                      new LeafSimScorer(stats, context.reader(), scoreMode.needsScores(), Integer.MAX_VALUE),
                                       scoreMode, totalMatchCost);
       } else {
         return new SloppyPhraseScorer(this, postingsFreqs, slop,
-                                        similarity.simScorer(stats, context),
+                                        new LeafSimScorer(stats, context.reader(), scoreMode.needsScores(), Float.POSITIVE_INFINITY),
                                         scoreMode.needsScores(), totalMatchCost);
       }
     }
@@ -303,7 +310,7 @@ public class MultiPhraseQuery extends Query {
         int newDoc = scorer.iterator().advance(doc);
         if (newDoc == doc) {
           float freq = slop == 0 ? ((ExactPhraseScorer)scorer).freq() : ((SloppyPhraseScorer)scorer).sloppyFreq();
-          SimScorer docScorer = similarity.simScorer(stats, context);
+          LeafSimScorer docScorer = new LeafSimScorer(stats, context.reader(), scoreMode.needsScores(), Float.POSITIVE_INFINITY);
           Explanation freqExplanation = Explanation.match(freq, "phraseFreq=" + freq);
           Explanation scoreExplanation = docScorer.explain(doc, freqExplanation);
           return Explanation.match(

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/8fd7ead9/lucene/core/src/java/org/apache/lucene/search/PhraseQuery.java
----------------------------------------------------------------------
diff --git a/lucene/core/src/java/org/apache/lucene/search/PhraseQuery.java b/lucene/core/src/java/org/apache/lucene/search/PhraseQuery.java
index 3d359b4..295cc90 100644
--- a/lucene/core/src/java/org/apache/lucene/search/PhraseQuery.java
+++ b/lucene/core/src/java/org/apache/lucene/search/PhraseQuery.java
@@ -37,7 +37,6 @@ import org.apache.lucene.index.TermState;
 import org.apache.lucene.index.Terms;
 import org.apache.lucene.index.TermsEnum;
 import org.apache.lucene.search.similarities.Similarity;
-import org.apache.lucene.search.similarities.Similarity.SimScorer;
 import org.apache.lucene.util.ArrayUtil;
 import org.apache.lucene.util.BytesRef;
 
@@ -352,7 +351,7 @@ public class PhraseQuery extends Query {
 
   private class PhraseWeight extends Weight {
     private final Similarity similarity;
-    private final Similarity.SimWeight stats;
+    private final Similarity.SimScorer stats;
     private final ScoreMode scoreMode;
     private transient TermContext states[];
 
@@ -366,7 +365,7 @@ public class PhraseQuery extends Query {
         throw new IllegalStateException("PhraseWeight requires that the first position is 0, call rewrite first");
       }
       this.scoreMode = scoreMode;
-      this.similarity = searcher.getSimilarity(scoreMode.needsScores());
+      this.similarity = searcher.getSimilarity();
       final IndexReaderContext context = searcher.getTopReaderContext();
       states = new TermContext[terms.length];
       TermStatistics termStats[] = new TermStatistics[terms.length];
@@ -380,7 +379,7 @@ public class PhraseQuery extends Query {
         }
       }
       if (termUpTo > 0) {
-        stats = similarity.computeWeight(boost, searcher.collectionStatistics(field), Arrays.copyOf(termStats, termUpTo));
+        stats = similarity.scorer(boost, searcher.collectionStatistics(field), Arrays.copyOf(termStats, termUpTo));
       } else {
         stats = null; // no terms at all, we won't use similarity
       }
@@ -433,11 +432,11 @@ public class PhraseQuery extends Query {
 
       if (slop == 0) {  // optimize exact case
         return new ExactPhraseScorer(this, postingsFreqs,
-                                      similarity.simScorer(stats, context),
+                                      new LeafSimScorer(stats, context.reader(), scoreMode.needsScores(), Integer.MAX_VALUE),
                                       scoreMode, totalMatchCost);
       } else {
         return new SloppyPhraseScorer(this, postingsFreqs, slop,
-                                        similarity.simScorer(stats, context),
+                                        new LeafSimScorer(stats, context.reader(), scoreMode.needsScores(), Float.POSITIVE_INFINITY),
                                         scoreMode.needsScores(), totalMatchCost);
       }
     }
@@ -459,7 +458,7 @@ public class PhraseQuery extends Query {
         int newDoc = scorer.iterator().advance(doc);
         if (newDoc == doc) {
           float freq = slop == 0 ? ((ExactPhraseScorer)scorer).freq() : ((SloppyPhraseScorer)scorer).sloppyFreq();
-          SimScorer docScorer = similarity.simScorer(stats, context);
+          LeafSimScorer docScorer = new LeafSimScorer(stats, context.reader(), scoreMode.needsScores(), Float.POSITIVE_INFINITY);
           Explanation freqExplanation = Explanation.match(freq, "phraseFreq=" + freq);
           Explanation scoreExplanation = docScorer.explain(doc, freqExplanation);
           return Explanation.match(

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/8fd7ead9/lucene/core/src/java/org/apache/lucene/search/SloppyPhraseScorer.java
----------------------------------------------------------------------
diff --git a/lucene/core/src/java/org/apache/lucene/search/SloppyPhraseScorer.java b/lucene/core/src/java/org/apache/lucene/search/SloppyPhraseScorer.java
index dc5490a..60b77c5 100644
--- a/lucene/core/src/java/org/apache/lucene/search/SloppyPhraseScorer.java
+++ b/lucene/core/src/java/org/apache/lucene/search/SloppyPhraseScorer.java
@@ -26,7 +26,6 @@ import java.util.HashSet;
 import java.util.LinkedHashMap;
 
 import org.apache.lucene.index.Term;
-import org.apache.lucene.search.similarities.Similarity;
 import org.apache.lucene.util.FixedBitSet;
 
 final class SloppyPhraseScorer extends Scorer {
@@ -36,7 +35,7 @@ final class SloppyPhraseScorer extends Scorer {
 
   private float sloppyFreq; //phrase frequency in current doc as computed by phraseFreq().
 
-  private final Similarity.SimScorer docScorer;
+  private final LeafSimScorer docScorer;
   
   private final int slop;
   private final int numPostings;
@@ -55,7 +54,7 @@ final class SloppyPhraseScorer extends Scorer {
   private final float matchCost;
   
   SloppyPhraseScorer(Weight weight, PhraseQuery.PostingsAndFreq[] postings,
-      int slop, Similarity.SimScorer docScorer, boolean needsScores,
+      int slop, LeafSimScorer docScorer, boolean needsScores,
       float matchCost) {
     super(weight);
     this.docScorer = docScorer;
@@ -558,7 +557,7 @@ final class SloppyPhraseScorer extends Scorer {
 
   @Override
   public float maxScore() {
-    return docScorer.maxScore(Float.POSITIVE_INFINITY);
+    return docScorer.maxScore();
   }
 
   @Override

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/8fd7ead9/lucene/core/src/java/org/apache/lucene/search/SynonymQuery.java
----------------------------------------------------------------------
diff --git a/lucene/core/src/java/org/apache/lucene/search/SynonymQuery.java b/lucene/core/src/java/org/apache/lucene/search/SynonymQuery.java
index ce9d6e0..3f4c06d 100644
--- a/lucene/core/src/java/org/apache/lucene/search/SynonymQuery.java
+++ b/lucene/core/src/java/org/apache/lucene/search/SynonymQuery.java
@@ -35,7 +35,6 @@ import org.apache.lucene.index.TermContext;
 import org.apache.lucene.index.TermState;
 import org.apache.lucene.index.TermsEnum;
 import org.apache.lucene.search.similarities.Similarity;
-import org.apache.lucene.search.similarities.Similarity.SimScorer;
 import org.apache.lucene.util.BytesRef;
 
 /**
@@ -129,7 +128,7 @@ public final class SynonymQuery extends Query {
   class SynonymWeight extends Weight {
     private final TermContext termContexts[];
     private final Similarity similarity;
-    private final Similarity.SimWeight simWeight;
+    private final Similarity.SimScorer simWeight;
 
     SynonymWeight(Query query, IndexSearcher searcher, float boost) throws IOException {
       super(query);
@@ -145,10 +144,10 @@ public final class SynonymQuery extends Query {
           totalTermFreq += termStats.totalTermFreq();
         }
       }
-      this.similarity = searcher.getSimilarity(true);
+      this.similarity = searcher.getSimilarity();
       if (docFreq > 0) {
         TermStatistics pseudoStats = new TermStatistics(new BytesRef("synonym pseudo-term"), docFreq, totalTermFreq);
-        this.simWeight = similarity.computeWeight(boost, collectionStats, pseudoStats);
+        this.simWeight = similarity.scorer(boost, collectionStats, pseudoStats);
       } else {
         this.simWeight = null; // no terms exist at all, we won't use similarity
       }
@@ -175,7 +174,7 @@ public final class SynonymQuery extends Query {
             assert scorer instanceof TermScorer;
             freq = ((TermScorer)scorer).freq();
           }
-          SimScorer docScorer = similarity.simScorer(simWeight, context);
+          LeafSimScorer docScorer = new LeafSimScorer(simWeight, context.reader(), true, Float.POSITIVE_INFINITY);
           Explanation freqExplanation = Explanation.match(freq, "termFreq=" + freq);
           Explanation scoreExplanation = docScorer.explain(doc, freqExplanation);
           return Explanation.match(
@@ -190,7 +189,6 @@ public final class SynonymQuery extends Query {
 
     @Override
     public Scorer scorer(LeafReaderContext context) throws IOException {
-      Similarity.SimScorer simScorer = null;
       IndexOptions indexOptions = IndexOptions.NONE;
       if (terms.length > 0) {
         FieldInfo info = context.reader()
@@ -202,21 +200,17 @@ public final class SynonymQuery extends Query {
       }
       // we use termscorers + disjunction as an impl detail
       List<Scorer> subScorers = new ArrayList<>();
-      long maxFreq = 0;
+      long totalMaxFreq = 0;
       for (int i = 0; i < terms.length; i++) {
         TermState state = termContexts[i].get(context.ord);
         if (state != null) {
           TermsEnum termsEnum = context.reader().terms(terms[i].field()).iterator();
           termsEnum.seekExact(terms[i].bytes(), state);
-
-          maxFreq += getMaxFreq(indexOptions, termsEnum.totalTermFreq(), termsEnum.docFreq());
-
+          long termMaxFreq = getMaxFreq(indexOptions, termsEnum.totalTermFreq(), termsEnum.docFreq());
+          totalMaxFreq += termMaxFreq;
           PostingsEnum postings = termsEnum.postings(null, PostingsEnum.FREQS);
-          // lazy init sim, in case no terms exist
-          if (simScorer == null) {
-            simScorer = similarity.simScorer(simWeight, context);
-          }
-          subScorers.add(new TermScorer(this, postings, simScorer, Float.POSITIVE_INFINITY));
+          LeafSimScorer simScorer = new LeafSimScorer(simWeight, context.reader(), true, termMaxFreq);
+          subScorers.add(new TermScorer(this, postings, simScorer));
         }
       }
       if (subScorers.isEmpty()) {
@@ -225,7 +219,8 @@ public final class SynonymQuery extends Query {
         // we must optimize this case (term not in segment), disjunctionscorer requires >= 2 subs
         return subScorers.get(0);
       } else {
-        return new SynonymScorer(simScorer, this, subScorers, maxFreq);
+        LeafSimScorer simScorer = new LeafSimScorer(simWeight, context.reader(), true, totalMaxFreq);
+        return new SynonymScorer(simScorer, this, subScorers);
       }
     }
 
@@ -248,13 +243,11 @@ public final class SynonymQuery extends Query {
   }
 
   static class SynonymScorer extends DisjunctionScorer {
-    private final Similarity.SimScorer similarity;
-    private final float maxFreq;
+    private final LeafSimScorer similarity;
     
-    SynonymScorer(Similarity.SimScorer similarity, Weight weight, List<Scorer> subScorers, float maxFreq) {
+    SynonymScorer(LeafSimScorer similarity, Weight weight, List<Scorer> subScorers) {
       super(weight, subScorers, true);
       this.similarity = similarity;
-      this.maxFreq = maxFreq;
     }
 
     @Override
@@ -264,7 +257,7 @@ public final class SynonymQuery extends Query {
 
     @Override
     public float maxScore() {
-      return similarity.maxScore(maxFreq);
+      return similarity.maxScore();
     }
 
     /** combines TF of all subs. */

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/8fd7ead9/lucene/core/src/java/org/apache/lucene/search/TermQuery.java
----------------------------------------------------------------------
diff --git a/lucene/core/src/java/org/apache/lucene/search/TermQuery.java b/lucene/core/src/java/org/apache/lucene/search/TermQuery.java
index 925fe93..3fa465d 100644
--- a/lucene/core/src/java/org/apache/lucene/search/TermQuery.java
+++ b/lucene/core/src/java/org/apache/lucene/search/TermQuery.java
@@ -33,7 +33,6 @@ import org.apache.lucene.index.TermState;
 import org.apache.lucene.index.Terms;
 import org.apache.lucene.index.TermsEnum;
 import org.apache.lucene.search.similarities.Similarity;
-import org.apache.lucene.search.similarities.Similarity.SimScorer;
 
 /**
  * A Query that matches documents containing a term. This may be combined with
@@ -46,7 +45,7 @@ public class TermQuery extends Query {
 
   final class TermWeight extends Weight {
     private final Similarity similarity;
-    private final Similarity.SimWeight stats;
+    private final Similarity.SimScorer simScorer;
     private final TermContext termStates;
     private final boolean needsScores;
 
@@ -58,7 +57,7 @@ public class TermQuery extends Query {
       }
       this.needsScores = needsScores;
       this.termStates = termStates;
-      this.similarity = searcher.getSimilarity(needsScores);
+      this.similarity = searcher.getSimilarity();
 
       final CollectionStatistics collectionStats;
       final TermStatistics termStats;
@@ -72,9 +71,9 @@ public class TermQuery extends Query {
       }
      
       if (termStats == null) {
-        this.stats = null; // term doesn't exist in any segment, we won't use similarity at all
+        this.simScorer = null; // term doesn't exist in any segment, we won't use similarity at all
       } else {
-        this.stats = similarity.computeWeight(boost, collectionStats, termStats);
+        this.simScorer = similarity.scorer(boost, collectionStats, termStats);
       }
     }
 
@@ -101,8 +100,8 @@ public class TermQuery extends Query {
           .getIndexOptions();
       PostingsEnum docs = termsEnum.postings(null, needsScores ? PostingsEnum.FREQS : PostingsEnum.NONE);
       assert docs != null;
-      return new TermScorer(this, docs, similarity.simScorer(stats, context),
-          getMaxFreq(indexOptions, termsEnum.totalTermFreq(), termsEnum.docFreq()));
+      float maxFreq = getMaxFreq(indexOptions, termsEnum.totalTermFreq(), termsEnum.docFreq());
+      return new TermScorer(this, docs, new LeafSimScorer(simScorer, context.reader(), needsScores, maxFreq));
     }
 
     private long getMaxFreq(IndexOptions indexOptions, long ttf, long df) {
@@ -166,7 +165,7 @@ public class TermQuery extends Query {
         int newDoc = scorer.iterator().advance(doc);
         if (newDoc == doc) {
           float freq = scorer.freq();
-          SimScorer docScorer = similarity.simScorer(stats, context);
+          LeafSimScorer docScorer = new LeafSimScorer(simScorer, context.reader(), true, Integer.MAX_VALUE);
           Explanation freqExplanation = Explanation.match(freq, "freq, occurrences of term within document");
           Explanation scoreExplanation = docScorer.explain(doc, freqExplanation);
           return Explanation.match(

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/8fd7ead9/lucene/core/src/java/org/apache/lucene/search/TermScorer.java
----------------------------------------------------------------------
diff --git a/lucene/core/src/java/org/apache/lucene/search/TermScorer.java b/lucene/core/src/java/org/apache/lucene/search/TermScorer.java
index a4aeb04..653a60e 100644
--- a/lucene/core/src/java/org/apache/lucene/search/TermScorer.java
+++ b/lucene/core/src/java/org/apache/lucene/search/TermScorer.java
@@ -20,14 +20,12 @@ package org.apache.lucene.search;
 import java.io.IOException;
 
 import org.apache.lucene.index.PostingsEnum;
-import org.apache.lucene.search.similarities.Similarity;
 
 /** Expert: A <code>Scorer</code> for documents matching a <code>Term</code>.
  */
 final class TermScorer extends Scorer {
   private final PostingsEnum postingsEnum;
-  private final Similarity.SimScorer docScorer;
-  private final float maxFreq;
+  private final LeafSimScorer docScorer;
 
   /**
    * Construct a <code>TermScorer</code>.
@@ -39,14 +37,11 @@ final class TermScorer extends Scorer {
    * @param docScorer
    *          The <code>Similarity.SimScorer</code> implementation
    *          to be used for score computations.
-   * @param maxFreq
-   *          An upper bound of the term frequency of the searched term in any document.
    */
-  TermScorer(Weight weight, PostingsEnum td, Similarity.SimScorer docScorer, float maxFreq) {
+  TermScorer(Weight weight, PostingsEnum td, LeafSimScorer docScorer) {
     super(weight);
     this.docScorer = docScorer;
     this.postingsEnum = td;
-    this.maxFreq = maxFreq;
   }
 
   @Override
@@ -71,7 +66,7 @@ final class TermScorer extends Scorer {
 
   @Override
   public float maxScore() {
-    return docScorer.maxScore(maxFreq);
+    return docScorer.maxScore();
   }
 
   /** Returns a string representation of this <code>TermScorer</code>. */

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/8fd7ead9/lucene/core/src/java/org/apache/lucene/search/package-info.java
----------------------------------------------------------------------
diff --git a/lucene/core/src/java/org/apache/lucene/search/package-info.java b/lucene/core/src/java/org/apache/lucene/search/package-info.java
index 69c5c2a..7e53da4 100644
--- a/lucene/core/src/java/org/apache/lucene/search/package-info.java
+++ b/lucene/core/src/java/org/apache/lucene/search/package-info.java
@@ -378,7 +378,7 @@
  *                 scored the way it was.
  *                 Typically a weight such as TermWeight
  *                 that scores via a {@link org.apache.lucene.search.similarities.Similarity Similarity} will make use of the Similarity's implementation:
- *                 {@link org.apache.lucene.search.similarities.Similarity.SimScorer#explain(int, Explanation) SimScorer#explain(int doc, Explanation freq)}.
+ *                 {@link org.apache.lucene.search.similarities.Similarity.SimScorer#explain(Explanation, long) SimScorer#explain(Explanation freq, long norm)}.
  *             </li>
  *         </ol>
  * <a name="scorerClass"></a>
@@ -402,7 +402,7 @@
  *                 {@link org.apache.lucene.search.Scorer#score score()} &mdash; Return the score of the
  *                 current document. This value can be determined in any appropriate way for an application. For instance, the
  *                 {@link org.apache.lucene.search.TermScorer TermScorer} simply defers to the configured Similarity:
- *                 {@link org.apache.lucene.search.similarities.Similarity.SimScorer#score(int, float) SimScorer.score(int doc, float freq)}.
+ *                 {@link org.apache.lucene.search.similarities.Similarity.SimScorer#score(float, long) SimScorer.score(float freq, long norm)}.
  *             </li>
  *             <li>
  *                 {@link org.apache.lucene.search.Scorer#getChildren getChildren()} &mdash; Returns any child subscorers

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/8fd7ead9/lucene/core/src/java/org/apache/lucene/search/similarities/Axiomatic.java
----------------------------------------------------------------------
diff --git a/lucene/core/src/java/org/apache/lucene/search/similarities/Axiomatic.java b/lucene/core/src/java/org/apache/lucene/search/similarities/Axiomatic.java
index 1522e5d..3865933 100644
--- a/lucene/core/src/java/org/apache/lucene/search/similarities/Axiomatic.java
+++ b/lucene/core/src/java/org/apache/lucene/search/similarities/Axiomatic.java
@@ -120,10 +120,10 @@ public abstract class Axiomatic extends SimilarityBase {
 
   @Override
   protected Explanation explain(
-      BasicStats stats, int doc, Explanation freq, double docLen) {    
+      BasicStats stats, Explanation freq, double docLen) {    
     List<Explanation> subs = new ArrayList<>();
     double f = freq.getValue().doubleValue();
-    explain(subs, stats, doc, f, docLen);
+    explain(subs, stats, f, docLen);
     
     double score = tf(stats, f, docLen)
         * ln(stats, f, docLen)
@@ -132,7 +132,7 @@ public abstract class Axiomatic extends SimilarityBase {
         - gamma(stats, f, docLen);
 
     Explanation explanation = Explanation.match((float) score,
-        "score(" + getClass().getSimpleName() + ", doc=" + doc + ", freq=" + freq.getValue() +"), computed from:",
+        "score(" + getClass().getSimpleName() + ", freq=" + freq.getValue() +"), computed from:",
         subs);
     if (stats.boost != 1f) {
       explanation = Explanation.match((float) (score * stats.boost), "Boosted score, computed as (score * boost) from:",
@@ -148,7 +148,7 @@ public abstract class Axiomatic extends SimilarityBase {
   }
 
   @Override
-  protected void explain(List<Explanation> subs, BasicStats stats, int doc,
+  protected void explain(List<Explanation> subs, BasicStats stats,
                          double freq, double docLen) {
     if (stats.getBoost() != 1.0d) {
       subs.add(Explanation.match((float) stats.getBoost(),
@@ -165,7 +165,7 @@ public abstract class Axiomatic extends SimilarityBase {
     subs.add(tflnExplain(stats, freq, docLen));
     subs.add(idfExplain(stats, freq, docLen));
     subs.add(Explanation.match((float) gamma(stats, freq, docLen), "gamma"));
-    super.explain(subs, stats, doc, freq, docLen);
+    super.explain(subs, stats, freq, docLen);
   }
 
   /**

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/8fd7ead9/lucene/core/src/java/org/apache/lucene/search/similarities/BM25Similarity.java
----------------------------------------------------------------------
diff --git a/lucene/core/src/java/org/apache/lucene/search/similarities/BM25Similarity.java b/lucene/core/src/java/org/apache/lucene/search/similarities/BM25Similarity.java
index dce156b..09bef40 100644
--- a/lucene/core/src/java/org/apache/lucene/search/similarities/BM25Similarity.java
+++ b/lucene/core/src/java/org/apache/lucene/search/similarities/BM25Similarity.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.TermStatistics;
@@ -176,7 +174,7 @@ public class BM25Similarity extends Similarity {
   }
 
   @Override
-  public final SimWeight computeWeight(float boost, CollectionStatistics collectionStats, TermStatistics... termStats) {
+  public final SimScorer scorer(float boost, CollectionStatistics collectionStats, TermStatistics... termStats) {
     Explanation idf = termStats.length == 1 ? idfExplain(collectionStats, termStats[0]) : idfExplain(collectionStats, termStats);
     float avgdl = avgFieldLength(collectionStats);
 
@@ -184,100 +182,17 @@ public class BM25Similarity extends Similarity {
     for (int i = 0; i < cache.length; i++) {
       cache[i] = k1 * ((1 - b) + b * LENGTH_TABLE[i] / avgdl);
     }
-    return new BM25Stats(collectionStats.field(), boost, k1, idf, avgdl, cache);
-  }
-
-  @Override
-  public final SimScorer simScorer(SimWeight stats, LeafReaderContext context) throws IOException {
-    BM25Stats bm25stats = (BM25Stats) stats;
-    return new BM25DocScorer(bm25stats, context.reader().getNormValues(bm25stats.field));
-  }
-  
-  private class BM25DocScorer extends SimScorer {
-    private final BM25Stats stats;
-    private final float weightValue; // boost * idf * (k1 + 1)
-    private final NumericDocValues norms;
-    /** precomputed cache for all length values */
-    private final float[] lengthCache;
-    /** precomputed norm[256] with k1 * ((1 - b) + b * dl / avgdl) */
-    private final float[] cache;
-    
-    BM25DocScorer(BM25Stats stats, NumericDocValues norms) throws IOException {
-      this.stats = stats;
-      this.weightValue = stats.weight;
-      this.norms = norms;
-      lengthCache = LENGTH_TABLE;
-      cache = stats.cache;
-    }
-    
-    @Override
-    public float score(int doc, float freq) throws IOException {
-      // if there are no norms, we act as if b=0
-      double norm;
-      if (norms == null) {
-        norm = k1;
-      } else {
-        boolean found = norms.advanceExact(doc);
-        assert found;
-        norm = cache[((byte) norms.longValue()) & 0xFF];
-      }
-      return weightValue * (float) (freq / (freq + norm));
-    }
-
-    @Override
-    public float maxScore(float maxFreq) {
-      // TODO: leverage maxFreq and the min norm from the cache
-      return weightValue;
-    }
-
-    @Override
-    public Explanation explain(int doc, Explanation freq) throws IOException {
-      List<Explanation> subs = new ArrayList<>();
-      subs.addAll(stats.explain());
-      Explanation tfExpl = explainTF(doc, freq);
-      subs.add(tfExpl);
-      return Explanation.match(stats.weight * tfExpl.getValue().floatValue(),
-          "score(doc="+doc+",freq="+freq.getValue()+"), product of:", subs);
-    }
-    
-    private Explanation explainTF(int doc, Explanation freq) throws IOException {
-      List<Explanation> subs = new ArrayList<>();
-      subs.add(freq);
-      subs.add(Explanation.match(k1, "k1, term saturation parameter"));
-      if (norms == null) {
-        subs.add(Explanation.match(0, "b, field omits length norms"));
-        return Explanation.match(
-            (float) (freq.getValue().floatValue() / (freq.getValue().floatValue() + (double) k1)),
-            "tf, computed as freq / (freq + k1) from:", subs);
-      } else {
-        boolean found = norms.advanceExact(doc);
-        assert found;
-        byte norm = (byte) norms.longValue();
-        float doclen = lengthCache[norm & 0xff];
-        subs.add(Explanation.match(b, "b, length normalization parameter"));
-        if ((norm & 0xFF) > 39) {
-          subs.add(Explanation.match(doclen, "dl, length of field (approximate)"));
-        } else {
-          subs.add(Explanation.match(doclen, "dl, length of field"));
-        }
-        subs.add(Explanation.match(stats.avgdl, "avgdl, average length of field"));
-        float normValue = k1 * ((1 - b) + b * doclen / stats.avgdl);
-        return Explanation.match(
-            (float) (freq.getValue().floatValue() / (freq.getValue().floatValue() + (double) normValue)),
-            "tf, computed as freq / (freq + k1 * (1 - b + b * dl / avgdl)) from:", subs);
-      }
-    }
-
+    return new BM25Scorer(collectionStats.field(), boost, k1, b, idf, avgdl, cache);
   }
   
   /** Collection statistics for the BM25 model. */
-  private static class BM25Stats extends SimWeight {
-    /** field name, for pulling norms */
-    private final String field;
+  private static class BM25Scorer extends SimScorer {
     /** query boost */
     private final float boost;
     /** k1 value for scale factor */
     private final float k1;
+    /** b value for length normalization impact */
+    private final float b;
     /** BM25's idf */
     private final Explanation idf;
     /** The average document length. */
@@ -287,17 +202,57 @@ public class BM25Similarity extends Similarity {
     /** weight (idf * boost) */
     private final float weight;
 
-    BM25Stats(String field, float boost, float k1, Explanation idf, float avgdl, float[] cache) {
-      this.field = field;
+    BM25Scorer(String field, float boost, float k1, float b, Explanation idf, float avgdl, float[] cache) {
+      super(field);
       this.boost = boost;
       this.idf = idf;
       this.avgdl = avgdl;
       this.k1 = k1;
+      this.b = b;
       this.cache = cache;
       this.weight = (k1 + 1) * boost * idf.getValue().floatValue();
     }
 
-    private List<Explanation> explain() {
+    @Override
+    public float score(float freq, long encodedNorm) throws IOException {
+      double norm = cache[((byte) encodedNorm) & 0xFF];
+      return weight * (float) (freq / (freq + norm));
+    }
+
+    @Override
+    public float maxScore(float maxFreq) {
+      // TODO: leverage maxFreq and the min norm from the cache
+      return weight;
+    }
+
+    @Override
+    public Explanation explain(Explanation freq, long encodedNorm) throws IOException {
+      List<Explanation> subs = new ArrayList<>(explainConstantFactors());
+      Explanation tfExpl = explainTF(freq, encodedNorm);
+      subs.add(tfExpl);
+      return Explanation.match(weight * tfExpl.getValue().floatValue(),
+          "score(freq="+freq.getValue()+"), product of:", subs);
+    }
+    
+    private Explanation explainTF(Explanation freq, long norm) throws IOException {
+      List<Explanation> subs = new ArrayList<>();
+      subs.add(freq);
+      subs.add(Explanation.match(k1, "k1, term saturation parameter"));
+      float doclen = LENGTH_TABLE[((byte) norm) & 0xff];
+      subs.add(Explanation.match(b, "b, length normalization parameter"));
+      if ((norm & 0xFF) > 39) {
+        subs.add(Explanation.match(doclen, "dl, length of field (approximate)"));
+      } else {
+        subs.add(Explanation.match(doclen, "dl, length of field"));
+      }
+      subs.add(Explanation.match(avgdl, "avgdl, average length of field"));
+      float normValue = k1 * ((1 - b) + b * doclen / avgdl);
+      return Explanation.match(
+          (float) (freq.getValue().floatValue() / (freq.getValue().floatValue() + (double) normValue)),
+          "tf, computed as freq / (freq + k1 * (1 - b + b * dl / avgdl)) from:", subs);
+    }
+
+    private List<Explanation> explainConstantFactors() {
       List<Explanation> subs = new ArrayList<>();
       // scale factor
       subs.add(Explanation.match(k1 + 1, "scaling factor, k1 + 1"));
@@ -311,7 +266,6 @@ public class BM25Similarity extends Similarity {
     }
   }
 
-
   @Override
   public String toString() {
     return "BM25(k1=" + k1 + ",b=" + b + ")";

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/8fd7ead9/lucene/core/src/java/org/apache/lucene/search/similarities/BasicStats.java
----------------------------------------------------------------------
diff --git a/lucene/core/src/java/org/apache/lucene/search/similarities/BasicStats.java b/lucene/core/src/java/org/apache/lucene/search/similarities/BasicStats.java
index cc3cab4..dc9356f 100644
--- a/lucene/core/src/java/org/apache/lucene/search/similarities/BasicStats.java
+++ b/lucene/core/src/java/org/apache/lucene/search/similarities/BasicStats.java
@@ -23,7 +23,7 @@ import org.apache.lucene.index.Terms;
  * Stores all statistics commonly used ranking methods.
  * @lucene.experimental
  */
-public class BasicStats extends Similarity.SimWeight {
+public class BasicStats {
   final String field;
   /** The number of documents. */
   protected long numberOfDocuments;

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/8fd7ead9/lucene/core/src/java/org/apache/lucene/search/similarities/BooleanSimilarity.java
----------------------------------------------------------------------
diff --git a/lucene/core/src/java/org/apache/lucene/search/similarities/BooleanSimilarity.java b/lucene/core/src/java/org/apache/lucene/search/similarities/BooleanSimilarity.java
index 7134172..2690365 100644
--- a/lucene/core/src/java/org/apache/lucene/search/similarities/BooleanSimilarity.java
+++ b/lucene/core/src/java/org/apache/lucene/search/similarities/BooleanSimilarity.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;
@@ -47,44 +46,36 @@ public class BooleanSimilarity extends Similarity {
   }
 
   @Override
-  public SimWeight computeWeight(float boost, CollectionStatistics collectionStats, TermStatistics... termStats) {
-    return new BooleanWeight(boost);
+  public SimScorer scorer(float boost, CollectionStatistics collectionStats, TermStatistics... termStats) {
+    return new BooleanWeight(collectionStats.field(), boost);
   }
 
-  private static class BooleanWeight extends SimWeight {
+  private static class BooleanWeight extends SimScorer {
     final float boost;
 
-    BooleanWeight(float boost) {
+    BooleanWeight(String field, float boost) {
+      super(field);
       this.boost = boost;
     }
-  }
-
-  @Override
-  public SimScorer simScorer(SimWeight weight, LeafReaderContext context) throws IOException {
-    final float boost = ((BooleanWeight) weight).boost;
-
-    return new SimScorer() {
 
-      @Override
-      public float score(int doc, float freq) throws IOException {
-        return boost;
-      }
-
-      @Override
-      public float maxScore(float maxFreq) {
-        return boost;
-      }
+    @Override
+    public float score(float freq, long norm) throws IOException {
+      return boost;
+    }
 
-      @Override
-      public Explanation explain(int doc, Explanation freq) throws IOException {
-        Explanation queryBoostExpl = Explanation.match(boost, "boost, query boost");
-        return Explanation.match(
-            queryBoostExpl.getValue(),
-            "score(" + getClass().getSimpleName() + ", doc=" + doc + "), computed from:",
-            queryBoostExpl);
-      }
+    @Override
+    public float maxScore(float maxFreq) {
+      return boost;
+    }
 
-    };
+    @Override
+    public Explanation explain(Explanation freq, long norm) throws IOException {
+      Explanation queryBoostExpl = Explanation.match(boost, "boost, query boost");
+      return Explanation.match(
+          queryBoostExpl.getValue(),
+          "score(" + getClass().getSimpleName() + "), computed from:",
+          queryBoostExpl);
+    }
   }
 
 }

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/8fd7ead9/lucene/core/src/java/org/apache/lucene/search/similarities/DFISimilarity.java
----------------------------------------------------------------------
diff --git a/lucene/core/src/java/org/apache/lucene/search/similarities/DFISimilarity.java b/lucene/core/src/java/org/apache/lucene/search/similarities/DFISimilarity.java
index 66f22be..44da93c 100644
--- a/lucene/core/src/java/org/apache/lucene/search/similarities/DFISimilarity.java
+++ b/lucene/core/src/java/org/apache/lucene/search/similarities/DFISimilarity.java
@@ -79,12 +79,12 @@ public class DFISimilarity extends SimilarityBase {
 
   @Override
   protected Explanation explain(
-      BasicStats stats, int doc, Explanation freq, double docLen) {
+      BasicStats stats, Explanation freq, double docLen) {
     final double expected = (stats.getTotalTermFreq() + 1) * docLen /
         (stats.getNumberOfFieldTokens() + 1);
     if (freq.getValue().doubleValue() <= expected){
       return Explanation.match((float) 0, "score(" +
-          getClass().getSimpleName() + ", doc=" + doc + ", freq=" +
+          getClass().getSimpleName() + ", freq=" +
           freq.getValue() +"), equals to 0");
     }
     Explanation explExpected = Explanation.match((float) expected,
@@ -103,7 +103,7 @@ public class DFISimilarity extends SimilarityBase {
 
     return Explanation.match(
         (float) score(stats, freq.getValue().doubleValue(), docLen),
-        "score(" + getClass().getSimpleName() + ", doc=" + doc + ", freq=" +
+        "score(" + getClass().getSimpleName() + ", freq=" +
             freq.getValue() +"), computed as boost * log2(measure + 1) from:",
         Explanation.match( (float)stats.getBoost(), "boost, query boost"),
         explMeasure);

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/8fd7ead9/lucene/core/src/java/org/apache/lucene/search/similarities/DFRSimilarity.java
----------------------------------------------------------------------
diff --git a/lucene/core/src/java/org/apache/lucene/search/similarities/DFRSimilarity.java b/lucene/core/src/java/org/apache/lucene/search/similarities/DFRSimilarity.java
index a41e35c..1677168 100644
--- a/lucene/core/src/java/org/apache/lucene/search/similarities/DFRSimilarity.java
+++ b/lucene/core/src/java/org/apache/lucene/search/similarities/DFRSimilarity.java
@@ -121,7 +121,7 @@ public class DFRSimilarity extends SimilarityBase {
 
   @Override
   protected void explain(List<Explanation> subs,
-      BasicStats stats, int doc, double freq, double docLen) {
+      BasicStats stats, double freq, double docLen) {
     if (stats.getBoost() != 1.0d) {
       subs.add(Explanation.match( (float)stats.getBoost(), "boost, query boost"));
     }
@@ -136,13 +136,13 @@ public class DFRSimilarity extends SimilarityBase {
 
   @Override
   protected Explanation explain(
-      BasicStats stats, int doc, Explanation freq, double docLen) {
+      BasicStats stats, Explanation freq, double docLen) {
     List<Explanation> subs = new ArrayList<>();
-    explain(subs, stats, doc, freq.getValue().doubleValue(), docLen);
+    explain(subs, stats, freq.getValue().doubleValue(), docLen);
 
     return Explanation.match(
         (float) score(stats, freq.getValue().doubleValue(), docLen),
-        "score(" + getClass().getSimpleName() + ", doc=" + doc + ", freq=" +
+        "score(" + getClass().getSimpleName() + ", freq=" +
             freq.getValue() +"), computed as boost * " +
             "basicModel.score(stats, tfn) * afterEffect.score(stats, tfn) from:",
         subs);

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/8fd7ead9/lucene/core/src/java/org/apache/lucene/search/similarities/IBSimilarity.java
----------------------------------------------------------------------
diff --git a/lucene/core/src/java/org/apache/lucene/search/similarities/IBSimilarity.java b/lucene/core/src/java/org/apache/lucene/search/similarities/IBSimilarity.java
index 9a57608..231d554 100644
--- a/lucene/core/src/java/org/apache/lucene/search/similarities/IBSimilarity.java
+++ b/lucene/core/src/java/org/apache/lucene/search/similarities/IBSimilarity.java
@@ -112,7 +112,7 @@ public class IBSimilarity extends SimilarityBase {
 
   @Override
   protected void explain(
-      List<Explanation> subs, BasicStats stats, int doc, double freq, double docLen) {
+      List<Explanation> subs, BasicStats stats, double freq, double docLen) {
     if (stats.getBoost() != 1.0d) {
       subs.add(Explanation.match((float)stats.getBoost(), "boost, query boost"));
     }
@@ -125,13 +125,13 @@ public class IBSimilarity extends SimilarityBase {
 
   @Override
   protected Explanation explain(
-      BasicStats stats, int doc, Explanation freq, double docLen) {
+      BasicStats stats, Explanation freq, double docLen) {
     List<Explanation> subs = new ArrayList<>();
-    explain(subs, stats, doc, freq.getValue().doubleValue(), docLen);
+    explain(subs, stats, freq.getValue().doubleValue(), docLen);
 
     return Explanation.match(
         (float) score(stats, freq.getValue().doubleValue(), docLen),
-        "score(" + getClass().getSimpleName() + ", doc=" + doc + ", freq=" +
+        "score(" + getClass().getSimpleName() + ", freq=" +
             freq.getValue() +"), computed as boost * " +
             "distribution.score(stats, normalization.tfn(stats, freq," +
             " docLen), lambda.lambda(stats)) from:",

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/8fd7ead9/lucene/core/src/java/org/apache/lucene/search/similarities/LMDirichletSimilarity.java
----------------------------------------------------------------------
diff --git a/lucene/core/src/java/org/apache/lucene/search/similarities/LMDirichletSimilarity.java b/lucene/core/src/java/org/apache/lucene/search/similarities/LMDirichletSimilarity.java
index c12cba4..7522c17 100644
--- a/lucene/core/src/java/org/apache/lucene/search/similarities/LMDirichletSimilarity.java
+++ b/lucene/core/src/java/org/apache/lucene/search/similarities/LMDirichletSimilarity.java
@@ -84,7 +84,7 @@ public class LMDirichletSimilarity extends LMSimilarity {
   }
 
   @Override
-  protected void explain(List<Explanation> subs, BasicStats stats, int doc,
+  protected void explain(List<Explanation> subs, BasicStats stats,
       double freq, double docLen) {
     if (stats.getBoost() != 1.0d) {
       subs.add(Explanation.match((float) stats.getBoost(), "query boost"));
@@ -107,18 +107,18 @@ public class LMDirichletSimilarity extends LMSimilarity {
         (float)Math.log(mu / (docLen + mu)),
         "document norm, computed as log(mu / (dl + mu))"));
     subs.add(Explanation.match((float) docLen,"dl, length of field"));
-    super.explain(subs, stats, doc, freq, docLen);
+    super.explain(subs, stats, freq, docLen);
   }
 
   @Override
   protected Explanation explain(
-      BasicStats stats, int doc, Explanation freq, double docLen) {
+      BasicStats stats, Explanation freq, double docLen) {
     List<Explanation> subs = new ArrayList<>();
-    explain(subs, stats, doc, freq.getValue().doubleValue(), docLen);
+    explain(subs, stats, freq.getValue().doubleValue(), docLen);
 
     return Explanation.match(
         (float) score(stats, freq.getValue().doubleValue(), docLen),
-        "score(" + getClass().getSimpleName() + ", doc=" + doc + ", freq=" +
+        "score(" + getClass().getSimpleName() + ", freq=" +
             freq.getValue() +"), computed as boost * " +
             "(term weight + document norm) from:",
         subs);

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/8fd7ead9/lucene/core/src/java/org/apache/lucene/search/similarities/LMJelinekMercerSimilarity.java
----------------------------------------------------------------------
diff --git a/lucene/core/src/java/org/apache/lucene/search/similarities/LMJelinekMercerSimilarity.java b/lucene/core/src/java/org/apache/lucene/search/similarities/LMJelinekMercerSimilarity.java
index 42e5a7b..dde0650 100644
--- a/lucene/core/src/java/org/apache/lucene/search/similarities/LMJelinekMercerSimilarity.java
+++ b/lucene/core/src/java/org/apache/lucene/search/similarities/LMJelinekMercerSimilarity.java
@@ -74,7 +74,7 @@ public class LMJelinekMercerSimilarity extends LMSimilarity {
   }
 
   @Override
-  protected void explain(List<Explanation> subs, BasicStats stats, int doc,
+  protected void explain(List<Explanation> subs, BasicStats stats,
       double freq, double docLen) {
     if (stats.getBoost() != 1.0d) {
       subs.add(Explanation.match((float) stats.getBoost(), "boost"));
@@ -88,18 +88,18 @@ public class LMJelinekMercerSimilarity extends LMSimilarity {
         "freq, number of occurrences of term in the document");
     subs.add(explFreq);
     subs.add(Explanation.match((float) docLen,"dl, length of field"));
-    super.explain(subs, stats, doc, freq, docLen);
+    super.explain(subs, stats, freq, docLen);
   }
 
   @Override
   protected Explanation explain(
-      BasicStats stats, int doc, Explanation freq, double docLen) {
+      BasicStats stats, Explanation freq, double docLen) {
     List<Explanation> subs = new ArrayList<>();
-    explain(subs, stats, doc, freq.getValue().doubleValue(), docLen);
+    explain(subs, stats, freq.getValue().doubleValue(), docLen);
 
     return Explanation.match(
         (float) score(stats, freq.getValue().doubleValue(), docLen),
-        "score(" + getClass().getSimpleName() + ", doc=" + doc + ", freq=" +
+        "score(" + getClass().getSimpleName() + ", freq=" +
             freq.getValue() +"), computed as boost * " +
             "log(1 + ((1 - lambda) * freq / dl) /(lambda * P)) from:",
         subs);

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/8fd7ead9/lucene/core/src/java/org/apache/lucene/search/similarities/LMSimilarity.java
----------------------------------------------------------------------
diff --git a/lucene/core/src/java/org/apache/lucene/search/similarities/LMSimilarity.java b/lucene/core/src/java/org/apache/lucene/search/similarities/LMSimilarity.java
index 8154806..73a1276 100644
--- a/lucene/core/src/java/org/apache/lucene/search/similarities/LMSimilarity.java
+++ b/lucene/core/src/java/org/apache/lucene/search/similarities/LMSimilarity.java
@@ -70,7 +70,7 @@ public abstract class LMSimilarity extends SimilarityBase {
   }
 
   @Override
-  protected void explain(List<Explanation> subExpls, BasicStats stats, int doc,
+  protected void explain(List<Explanation> subExpls, BasicStats stats,
       double freq, double docLen) {
     subExpls.add(Explanation.match((float) collectionModel.computeProbability(stats),
                                    "collection probability"));

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/8fd7ead9/lucene/core/src/java/org/apache/lucene/search/similarities/MultiSimilarity.java
----------------------------------------------------------------------
diff --git a/lucene/core/src/java/org/apache/lucene/search/similarities/MultiSimilarity.java b/lucene/core/src/java/org/apache/lucene/search/similarities/MultiSimilarity.java
index 2f48cc6..3526db4 100644
--- a/lucene/core/src/java/org/apache/lucene/search/similarities/MultiSimilarity.java
+++ b/lucene/core/src/java/org/apache/lucene/search/similarities/MultiSimilarity.java
@@ -22,7 +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.search.CollectionStatistics;
 import org.apache.lucene.search.Explanation;
 import org.apache.lucene.search.TermStatistics;
@@ -49,35 +48,27 @@ public class MultiSimilarity extends Similarity {
   }
 
   @Override
-  public SimWeight computeWeight(float boost, CollectionStatistics collectionStats, TermStatistics... termStats) {
-    SimWeight subStats[] = new SimWeight[sims.length];
-    for (int i = 0; i < subStats.length; i++) {
-      subStats[i] = sims[i].computeWeight(boost, collectionStats, termStats);
-    }
-    return new MultiStats(subStats);
-  }
-
-  @Override
-  public SimScorer simScorer(SimWeight stats, LeafReaderContext context) throws IOException {
+  public SimScorer scorer(float boost, CollectionStatistics collectionStats, TermStatistics... termStats) {
     SimScorer subScorers[] = new SimScorer[sims.length];
     for (int i = 0; i < subScorers.length; i++) {
-      subScorers[i] = sims[i].simScorer(((MultiStats)stats).subStats[i], context);
+      subScorers[i] = sims[i].scorer(boost, collectionStats, termStats);
     }
-    return new MultiSimScorer(subScorers);
+    return new MultiSimScorer(collectionStats.field(), subScorers);
   }
   
   static class MultiSimScorer extends SimScorer {
     private final SimScorer subScorers[];
     
-    MultiSimScorer(SimScorer subScorers[]) {
+    MultiSimScorer(String field, SimScorer subScorers[]) {
+      super(field);
       this.subScorers = subScorers;
     }
     
     @Override
-    public float score(int doc, float freq) throws IOException {
+    public float score(float freq, long norm) throws IOException {
       float sum = 0.0f;
       for (SimScorer subScorer : subScorers) {
-        sum += subScorer.score(doc, freq);
+        sum += subScorer.score(freq, norm);
       }
       return sum;
     }
@@ -92,21 +83,13 @@ public class MultiSimilarity extends Similarity {
     }
 
     @Override
-    public Explanation explain(int doc, Explanation freq) throws IOException {
+    public Explanation explain(Explanation freq, long norm) throws IOException {
       List<Explanation> subs = new ArrayList<>();
       for (SimScorer subScorer : subScorers) {
-        subs.add(subScorer.explain(doc, freq));
+        subs.add(subScorer.explain(freq, norm));
       }
-      return Explanation.match(score(doc, freq.getValue().floatValue()), "sum of:", subs);
+      return Explanation.match(score(freq.getValue().floatValue(), norm), "sum of:", subs);
     }
 
   }
-
-  static class MultiStats extends SimWeight {
-    final SimWeight subStats[];
-    
-    MultiStats(SimWeight subStats[]) {
-      this.subStats = subStats;
-    }
-  }
 }

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/8fd7ead9/lucene/core/src/java/org/apache/lucene/search/similarities/PerFieldSimilarityWrapper.java
----------------------------------------------------------------------
diff --git a/lucene/core/src/java/org/apache/lucene/search/similarities/PerFieldSimilarityWrapper.java b/lucene/core/src/java/org/apache/lucene/search/similarities/PerFieldSimilarityWrapper.java
index 6c05616..ee2381f 100644
--- a/lucene/core/src/java/org/apache/lucene/search/similarities/PerFieldSimilarityWrapper.java
+++ b/lucene/core/src/java/org/apache/lucene/search/similarities/PerFieldSimilarityWrapper.java
@@ -17,9 +17,6 @@
 package org.apache.lucene.search.similarities;
 
 
-import java.io.IOException;
-
-import org.apache.lucene.index.LeafReaderContext;
 import org.apache.lucene.index.FieldInvertState;
 import org.apache.lucene.search.CollectionStatistics;
 import org.apache.lucene.search.TermStatistics;
@@ -46,26 +43,13 @@ public abstract class PerFieldSimilarityWrapper extends Similarity {
   }
 
   @Override
-  public final SimWeight computeWeight(float boost, CollectionStatistics collectionStats, TermStatistics... termStats) {
-    PerFieldSimWeight weight = new PerFieldSimWeight();
-    weight.delegate = get(collectionStats.field());
-    weight.delegateWeight = weight.delegate.computeWeight(boost, collectionStats, termStats);
-    return weight;
-  }
-
-  @Override
-  public final SimScorer simScorer(SimWeight weight, LeafReaderContext context) throws IOException {
-    PerFieldSimWeight perFieldWeight = (PerFieldSimWeight) weight;
-    return perFieldWeight.delegate.simScorer(perFieldWeight.delegateWeight, context);
+  public final SimScorer scorer(float boost, CollectionStatistics collectionStats, TermStatistics... termStats) {
+    return get(collectionStats.field()).scorer(boost, collectionStats, termStats);
   }
   
   /** 
    * Returns a {@link Similarity} for scoring a field.
    */
   public abstract Similarity get(String name);
-  
-  static class PerFieldSimWeight extends SimWeight {
-    Similarity delegate;
-    SimWeight delegateWeight;
-  }
+
 }

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/8fd7ead9/lucene/core/src/java/org/apache/lucene/search/similarities/Similarity.java
----------------------------------------------------------------------
diff --git a/lucene/core/src/java/org/apache/lucene/search/similarities/Similarity.java b/lucene/core/src/java/org/apache/lucene/search/similarities/Similarity.java
index 5f0bcd0..a2ebe4a 100644
--- a/lucene/core/src/java/org/apache/lucene/search/similarities/Similarity.java
+++ b/lucene/core/src/java/org/apache/lucene/search/similarities/Similarity.java
@@ -19,16 +19,14 @@ package org.apache.lucene.search.similarities;
 
 import java.io.IOException;
 import java.util.Collections;
+import java.util.Objects;
 
+import org.apache.lucene.document.NumericDocValuesField;
 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.IndexSearcher;
-import org.apache.lucene.search.PhraseQuery;
-import org.apache.lucene.search.TermQuery;
 import org.apache.lucene.search.TermStatistics;
-import org.apache.lucene.search.spans.SpanQuery;
 import org.apache.lucene.util.SmallFloat;
 
 /** 
@@ -38,9 +36,9 @@ import org.apache.lucene.util.SmallFloat;
  * <p>
  * This is a low-level API, you should only extend this API if you want to implement 
  * an information retrieval <i>model</i>.  If you are instead looking for a convenient way 
- * to alter Lucene's scoring, consider extending a higher-level implementation
- * such as {@link TFIDFSimilarity}, which implements the vector space model with this API, or 
- * just tweaking the default implementation: {@link BM25Similarity}.
+ * to alter Lucene's scoring, consider just tweaking the default implementation:
+ * {@link BM25Similarity} or extend {@link SimilarityBase}, which makes it easy to compute
+ * a score from index statistics.
  * <p>
  * Similarity determines how Lucene weights terms, and Lucene interacts with
  * this class at both <a href="#indextime">index-time</a> and 
@@ -49,23 +47,22 @@ import org.apache.lucene.util.SmallFloat;
  * <a name="indextime">Indexing Time</a>
  * At indexing time, the indexer calls {@link #computeNorm(FieldInvertState)}, allowing
  * the Similarity implementation to set a per-document value for the field that will 
- * be later accessible via {@link org.apache.lucene.index.LeafReader#getNormValues(String)}.  Lucene makes no assumption
- * about what is in this norm, but it is most useful for encoding length normalization 
- * information.
+ * be later accessible via {@link org.apache.lucene.index.LeafReader#getNormValues(String)}.
+ * Lucene makes no assumption about what is in this norm, but it is most useful for
+ * encoding length normalization information.
  * <p>
  * Implementations should carefully consider how the normalization is encoded: while
- * Lucene's {@link BM25Similarity} encodes a combination of index-time boost
- * and length normalization information with {@link SmallFloat} into a single byte, this 
- * might not be suitable for all purposes.
+ * Lucene's {@link BM25Similarity} encodes length normalization information with
+ * {@link SmallFloat} into a single byte, this might not be suitable for all purposes.
  * <p>
  * Many formulas require the use of average document length, which can be computed via a 
  * combination of {@link CollectionStatistics#sumTotalTermFreq()} and 
- * {@link CollectionStatistics#maxDoc()} or {@link CollectionStatistics#docCount()}, 
- * depending upon whether the average should reflect field sparsity.
+ * {@link CollectionStatistics#docCount()}.
  * <p>
- * Additional scoring factors can be stored in named
- * <code>NumericDocValuesField</code>s and accessed
- * at query-time with {@link org.apache.lucene.index.LeafReader#getNumericDocValues(String)}.
+ * Additional scoring factors can be stored in named {@link NumericDocValuesField}s and
+ * accessed at query-time with {@link org.apache.lucene.index.LeafReader#getNumericDocValues(String)}.
+ * However this should not be done in the {@link Similarity} but externally, for instance
+ * by using <tt>FunctionScoreQuery</tt>.
  * <p>
  * Finally, using index-time boosts (either via folding into the normalization byte or
  * via DocValues), is an inefficient way to boost the scores of different fields if the
@@ -76,14 +73,13 @@ import org.apache.lucene.util.SmallFloat;
  * <a name="querytime">Query time</a>
  * At query-time, Queries interact with the Similarity via these steps:
  * <ol>
- *   <li>The {@link #computeWeight(float, CollectionStatistics, TermStatistics...)} method is called a single time,
+ *   <li>The {@link #scorer(float, CollectionStatistics, TermStatistics...)} method is called a single time,
  *       allowing the implementation to compute any statistics (such as IDF, average document length, etc)
  *       across <i>the entire collection</i>. The {@link TermStatistics} and {@link CollectionStatistics} passed in 
  *       already contain all of the raw statistics involved, so a Similarity can freely use any combination
  *       of statistics without causing any additional I/O. Lucene makes no assumption about what is 
- *       stored in the returned {@link Similarity.SimWeight} object.
- *   <li>For each segment in the index, the Query creates a {@link #simScorer(SimWeight, org.apache.lucene.index.LeafReaderContext)}
- *       The score() method is called for each matching document.
+ *       stored in the returned {@link Similarity.SimScorer} object.
+ *   <li>Then {@link SimScorer#score(float, long)} is called for every matching document to compute its score.
  * </ol>
  * <p>
  * <a name="explaintime">Explanations</a>
@@ -126,37 +122,38 @@ public abstract class Similarity {
    * @param termStats term-level statistics, such as the document frequency of a term across the collection.
    * @return SimWeight object with the information this Similarity needs to score a query.
    */
-  public abstract SimWeight computeWeight(float boost,
+  public abstract SimScorer scorer(float boost,
       CollectionStatistics collectionStats, TermStatistics... termStats);
-
-  /**
-   * Creates a new {@link Similarity.SimScorer} to score matching documents from a segment of the inverted index.
-   * @param weight collection information from {@link #computeWeight(float, CollectionStatistics, TermStatistics...)}
-   * @param context segment of the inverted index to be scored.
-   * @return SloppySimScorer for scoring documents across <code>context</code>
-   * @throws IOException if there is a low-level I/O error
-   */
-  public abstract SimScorer simScorer(SimWeight weight, LeafReaderContext context) throws IOException;
   
-  /**
-   * API for scoring "sloppy" queries such as {@link TermQuery},
-   * {@link SpanQuery}, and {@link PhraseQuery}.
+  /** Stores the weight for a query across the indexed collection. This abstract
+   * implementation is empty; descendants of {@code Similarity} should
+   * subclass {@code SimWeight} and define the statistics they require in the
+   * subclass. Examples include idf, average field length, etc.
    */
   public static abstract class SimScorer {
-    
+
+    private final String field;
+
     /**
      * Sole constructor. (For invocation by subclass 
-     * constructors, typically implicit.)
+     * constructors.)
      */
-    public SimScorer() {}
+    public SimScorer(String field) {
+      this.field = Objects.requireNonNull(field);
+    }
+
+    /** Return the field that this {@link SimScorer} operates on. */
+    public final String getField() {
+      return field;
+    }
 
     /**
      * Score a single document
-     * @param doc document id within the inverted index segment
      * @param freq sloppy term frequency
+     * @param norm encoded normalization factor, as returned by {@link Similarity#computeNorm}, or {@code 1} if norms are disabled
      * @return document's score
      */
-    public abstract float score(int doc, float freq) throws IOException;
+    public abstract float score(float freq, long norm) throws IOException;
 
     /**
      * Return the maximum score that this scorer may produce for freqs in {@code ]0, maxFreq]}.
@@ -167,30 +164,16 @@ public abstract class Similarity {
 
     /**
      * Explain the score for a single document
-     * @param doc document id within the inverted index segment
      * @param freq Explanation of how the sloppy term frequency was computed
+     * @param norm encoded normalization factor, as returned by {@link Similarity#computeNorm}, or {@code 1} if norms are disabled
      * @return document's score
      */
-    public Explanation explain(int doc, Explanation freq) throws IOException {
+    public Explanation explain(Explanation freq, long norm) throws IOException {
       return Explanation.match(
-          score(doc, freq.getValue().floatValue()),
-          "score(doc=" + doc + ",freq=" + freq.getValue() +"), with freq of:",
+          score(freq.getValue().floatValue(), norm),
+          "score(freq=" + freq.getValue() +"), with freq of:",
           Collections.singleton(freq));
     }
-  }
-  
-  /** Stores the weight for a query across the indexed collection. This abstract
-   * implementation is empty; descendants of {@code Similarity} should
-   * subclass {@code SimWeight} and define the statistics they require in the
-   * subclass. Examples include idf, average field length, etc.
-   */
-  public static abstract class SimWeight {
-    
-    /**
-     * Sole constructor. (For invocation by subclass 
-     * constructors, typically implicit.)
-     */
-    public SimWeight() {}
 
   }
 }

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/8fd7ead9/lucene/core/src/java/org/apache/lucene/search/similarities/SimilarityBase.java
----------------------------------------------------------------------
diff --git a/lucene/core/src/java/org/apache/lucene/search/similarities/SimilarityBase.java b/lucene/core/src/java/org/apache/lucene/search/similarities/SimilarityBase.java
index f227f38..f750b12 100644
--- a/lucene/core/src/java/org/apache/lucene/search/similarities/SimilarityBase.java
+++ b/lucene/core/src/java/org/apache/lucene/search/similarities/SimilarityBase.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.TermStatistics;
@@ -33,7 +31,7 @@ import org.apache.lucene.util.SmallFloat;
  * A subclass of {@code Similarity} that provides a simplified API for its
  * descendants. Subclasses are only required to implement the {@link #score}
  * and {@link #toString()} methods. Implementing
- * {@link #explain(List, BasicStats, int, double, double)} is optional,
+ * {@link #explain(List, BasicStats, double, double)} is optional,
  * inasmuch as SimilarityBase already provides a basic explanation of the score
  * and the term frequency. However, implementers of a subclass are encouraged to
  * include as much detail about the scoring method as possible.
@@ -82,13 +80,18 @@ public abstract class SimilarityBase extends Similarity {
   }
   
   @Override
-  public final SimWeight computeWeight(float boost, CollectionStatistics collectionStats, TermStatistics... termStats) {
-    BasicStats stats[] = new BasicStats[termStats.length];
+  public final SimScorer scorer(float boost, CollectionStatistics collectionStats, TermStatistics... termStats) {
+    SimScorer weights[] = new SimScorer[termStats.length];
     for (int i = 0; i < termStats.length; i++) {
-      stats[i] = newStats(collectionStats.field(), boost);
-      fillBasicStats(stats[i], collectionStats, termStats[i]);
+      BasicStats stats = newStats(collectionStats.field(), boost);
+      fillBasicStats(stats, collectionStats, termStats[i]);
+      weights[i] = new BasicSimScorer(stats);
+    }
+    if (weights.length == 1) {
+      return weights[0];
+    } else {
+      return new MultiSimilarity.MultiSimScorer(collectionStats.field(), weights);
     }
-    return stats.length == 1 ? stats[0] : new MultiSimilarity.MultiStats(stats);
   }
   
   /** Factory method to return a custom stats object */
@@ -137,12 +140,11 @@ public abstract class SimilarityBase extends Similarity {
    * 
    * @param subExpls the list of details of the explanation to extend
    * @param stats the corpus level statistics.
-   * @param doc the document id.
    * @param freq the term frequency.
    * @param docLen the document length.
    */
   protected void explain(
-      List<Explanation> subExpls, BasicStats stats, int doc, double freq, double docLen) {}
+      List<Explanation> subExpls, BasicStats stats, double freq, double docLen) {}
   
   /**
    * Explains the score. The implementation here provides a basic explanation
@@ -151,43 +153,24 @@ public abstract class SimilarityBase extends Similarity {
    * attaches the score (computed via the {@link #score(BasicStats, double, double)}
    * method) and the explanation for the term frequency. Subclasses content with
    * this format may add additional details in
-   * {@link #explain(List, BasicStats, int, double, double)}.
+   * {@link #explain(List, BasicStats, double, double)}.
    *  
    * @param stats the corpus level statistics.
-   * @param doc the document id.
    * @param freq the term frequency and its explanation.
    * @param docLen the document length.
    * @return the explanation.
    */
   protected Explanation explain(
-      BasicStats stats, int doc, Explanation freq, double docLen) {
+      BasicStats stats, Explanation freq, double docLen) {
     List<Explanation> subs = new ArrayList<>();
-    explain(subs, stats, doc, freq.getValue().floatValue(), docLen);
+    explain(subs, stats, freq.getValue().floatValue(), docLen);
     
     return Explanation.match(
         (float) score(stats, freq.getValue().floatValue(), docLen),
-        "score(" + getClass().getSimpleName() + ", doc=" + doc + ", freq=" + freq.getValue() +"), computed from:",
+        "score(" + getClass().getSimpleName() + ", freq=" + freq.getValue() +"), computed from:",
         subs);
   }
   
-  @Override
-  public final SimScorer simScorer(SimWeight stats, LeafReaderContext context) throws IOException {
-    if (stats instanceof MultiSimilarity.MultiStats) {
-      // a multi term query (e.g. phrase). return the summation, 
-      // scoring almost as if it were boolean query
-      SimWeight subStats[] = ((MultiSimilarity.MultiStats) stats).subStats;
-      SimScorer subScorers[] = new SimScorer[subStats.length];
-      for (int i = 0; i < subScorers.length; i++) {
-        BasicStats basicstats = (BasicStats) subStats[i];
-        subScorers[i] = new BasicSimScorer(basicstats, context.reader().getNormValues(basicstats.field));
-      }
-      return new MultiSimilarity.MultiSimScorer(subScorers);
-    } else {
-      BasicStats basicstats = (BasicStats) stats;
-      return new BasicSimScorer(basicstats, context.reader().getNormValues(basicstats.field));
-    }
-  }
-  
   /**
    * Subclasses must override this method to return the name of the Similarity
    * and preferably the values of parameters (if any) as well.
@@ -227,33 +210,27 @@ public abstract class SimilarityBase extends Similarity {
   
   // --------------------------------- Classes ---------------------------------
   
-  /** Delegates the {@link #score(int, float)} and
-   * {@link #explain(int, Explanation)} methods to
+  /** Delegates the {@link #score(float, long)} and
+   * {@link #explain(Explanation, long)} methods to
    * {@link SimilarityBase#score(BasicStats, double, double)} and
-   * {@link SimilarityBase#explain(BasicStats, int, Explanation, double)},
+   * {@link SimilarityBase#explain(BasicStats, Explanation, double)},
    * respectively.
    */
   final class BasicSimScorer extends SimScorer {
-    private final BasicStats stats;
-    private final NumericDocValues norms;
+    final BasicStats stats;
     
-    BasicSimScorer(BasicStats stats, NumericDocValues norms) throws IOException {
+    BasicSimScorer(BasicStats stats) {
+      super(stats.field);
       this.stats = stats;
-      this.norms = norms;
     }
 
-    double getLengthValue(int doc) throws IOException {
-      if (norms == null) {
-        return 1D;
-      }
-      boolean found = norms.advanceExact(doc);
-      assert found;
-      return LENGTH_TABLE[Byte.toUnsignedInt((byte) norms.longValue())];
+    double getLengthValue(long norm) throws IOException {
+      return LENGTH_TABLE[Byte.toUnsignedInt((byte) norm)];
     }
     
     @Override
-    public float score(int doc, float freq) throws IOException {
-      return (float) SimilarityBase.this.score(stats, freq, getLengthValue(doc));
+    public float score(float freq, long norm) throws IOException {
+      return (float) SimilarityBase.this.score(stats, freq, getLengthValue(norm));
     }
 
     @Override
@@ -262,8 +239,8 @@ public abstract class SimilarityBase extends Similarity {
     }
 
     @Override
-    public Explanation explain(int doc, Explanation freq) throws IOException {
-      return SimilarityBase.this.explain(stats, doc, freq, getLengthValue(doc));
+    public Explanation explain(Explanation freq, long norm) throws IOException {
+      return SimilarityBase.this.explain(stats, freq, getLengthValue(norm));
     }
 
   }