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

svn commit: r1159912 [1/2] - in /lucene/dev/branches/flexscoring/lucene/src: java/org/apache/lucene/search/similarities/ test/org/apache/lucene/search/ test/org/apache/lucene/search/similarities/

Author: rmuir
Date: Sat Aug 20 20:23:51 2011
New Revision: 1159912

URL: http://svn.apache.org/viewvc?rev=1159912&view=rev
Log:
LUCENE-3386: move similarites to src/java

Added:
    lucene/dev/branches/flexscoring/lucene/src/java/org/apache/lucene/search/similarities/
    lucene/dev/branches/flexscoring/lucene/src/java/org/apache/lucene/search/similarities/AfterEffect.java   (with props)
    lucene/dev/branches/flexscoring/lucene/src/java/org/apache/lucene/search/similarities/AfterEffectB.java   (with props)
    lucene/dev/branches/flexscoring/lucene/src/java/org/apache/lucene/search/similarities/AfterEffectL.java   (with props)
    lucene/dev/branches/flexscoring/lucene/src/java/org/apache/lucene/search/similarities/BM25Similarity.java   (with props)
    lucene/dev/branches/flexscoring/lucene/src/java/org/apache/lucene/search/similarities/BM25SimilarityProvider.java   (with props)
    lucene/dev/branches/flexscoring/lucene/src/java/org/apache/lucene/search/similarities/BasicModel.java   (with props)
    lucene/dev/branches/flexscoring/lucene/src/java/org/apache/lucene/search/similarities/BasicModelBE.java   (with props)
    lucene/dev/branches/flexscoring/lucene/src/java/org/apache/lucene/search/similarities/BasicModelD.java   (with props)
    lucene/dev/branches/flexscoring/lucene/src/java/org/apache/lucene/search/similarities/BasicModelG.java   (with props)
    lucene/dev/branches/flexscoring/lucene/src/java/org/apache/lucene/search/similarities/BasicModelIF.java   (with props)
    lucene/dev/branches/flexscoring/lucene/src/java/org/apache/lucene/search/similarities/BasicModelIn.java   (with props)
    lucene/dev/branches/flexscoring/lucene/src/java/org/apache/lucene/search/similarities/BasicModelIne.java   (with props)
    lucene/dev/branches/flexscoring/lucene/src/java/org/apache/lucene/search/similarities/BasicModelP.java   (with props)
    lucene/dev/branches/flexscoring/lucene/src/java/org/apache/lucene/search/similarities/DFRSimilarity.java   (with props)
    lucene/dev/branches/flexscoring/lucene/src/java/org/apache/lucene/search/similarities/Distribution.java   (with props)
    lucene/dev/branches/flexscoring/lucene/src/java/org/apache/lucene/search/similarities/DistributionLL.java   (with props)
    lucene/dev/branches/flexscoring/lucene/src/java/org/apache/lucene/search/similarities/DistributionSPL.java   (with props)
    lucene/dev/branches/flexscoring/lucene/src/java/org/apache/lucene/search/similarities/EasySimilarity.java   (with props)
    lucene/dev/branches/flexscoring/lucene/src/java/org/apache/lucene/search/similarities/EasyStats.java   (with props)
    lucene/dev/branches/flexscoring/lucene/src/java/org/apache/lucene/search/similarities/IBSimilarity.java   (with props)
    lucene/dev/branches/flexscoring/lucene/src/java/org/apache/lucene/search/similarities/LMDirichletSimilarity.java   (with props)
    lucene/dev/branches/flexscoring/lucene/src/java/org/apache/lucene/search/similarities/LMJelinekMercerSimilarity.java   (with props)
    lucene/dev/branches/flexscoring/lucene/src/java/org/apache/lucene/search/similarities/LMSimilarity.java   (with props)
    lucene/dev/branches/flexscoring/lucene/src/java/org/apache/lucene/search/similarities/Lambda.java   (with props)
    lucene/dev/branches/flexscoring/lucene/src/java/org/apache/lucene/search/similarities/LambdaDF.java   (with props)
    lucene/dev/branches/flexscoring/lucene/src/java/org/apache/lucene/search/similarities/LambdaTTF.java   (with props)
    lucene/dev/branches/flexscoring/lucene/src/java/org/apache/lucene/search/similarities/Normalization.java   (with props)
    lucene/dev/branches/flexscoring/lucene/src/java/org/apache/lucene/search/similarities/NormalizationH1.java   (with props)
    lucene/dev/branches/flexscoring/lucene/src/java/org/apache/lucene/search/similarities/NormalizationH2.java   (with props)
Removed:
    lucene/dev/branches/flexscoring/lucene/src/test/org/apache/lucene/search/MockBM25Similarity.java
    lucene/dev/branches/flexscoring/lucene/src/test/org/apache/lucene/search/MockBM25SimilarityProvider.java
    lucene/dev/branches/flexscoring/lucene/src/test/org/apache/lucene/search/MockLMSimilarity.java
    lucene/dev/branches/flexscoring/lucene/src/test/org/apache/lucene/search/MockLMSimilarityProvider.java
    lucene/dev/branches/flexscoring/lucene/src/test/org/apache/lucene/search/similarities/AfterEffect.java
    lucene/dev/branches/flexscoring/lucene/src/test/org/apache/lucene/search/similarities/AfterEffectB.java
    lucene/dev/branches/flexscoring/lucene/src/test/org/apache/lucene/search/similarities/AfterEffectL.java
    lucene/dev/branches/flexscoring/lucene/src/test/org/apache/lucene/search/similarities/BasicModel.java
    lucene/dev/branches/flexscoring/lucene/src/test/org/apache/lucene/search/similarities/BasicModelBE.java
    lucene/dev/branches/flexscoring/lucene/src/test/org/apache/lucene/search/similarities/BasicModelD.java
    lucene/dev/branches/flexscoring/lucene/src/test/org/apache/lucene/search/similarities/BasicModelG.java
    lucene/dev/branches/flexscoring/lucene/src/test/org/apache/lucene/search/similarities/BasicModelIF.java
    lucene/dev/branches/flexscoring/lucene/src/test/org/apache/lucene/search/similarities/BasicModelIn.java
    lucene/dev/branches/flexscoring/lucene/src/test/org/apache/lucene/search/similarities/BasicModelIne.java
    lucene/dev/branches/flexscoring/lucene/src/test/org/apache/lucene/search/similarities/BasicModelP.java
    lucene/dev/branches/flexscoring/lucene/src/test/org/apache/lucene/search/similarities/DFRSimilarity.java
    lucene/dev/branches/flexscoring/lucene/src/test/org/apache/lucene/search/similarities/Distribution.java
    lucene/dev/branches/flexscoring/lucene/src/test/org/apache/lucene/search/similarities/DistributionLL.java
    lucene/dev/branches/flexscoring/lucene/src/test/org/apache/lucene/search/similarities/DistributionSPL.java
    lucene/dev/branches/flexscoring/lucene/src/test/org/apache/lucene/search/similarities/EasySimilarity.java
    lucene/dev/branches/flexscoring/lucene/src/test/org/apache/lucene/search/similarities/EasyStats.java
    lucene/dev/branches/flexscoring/lucene/src/test/org/apache/lucene/search/similarities/IBSimilarity.java
    lucene/dev/branches/flexscoring/lucene/src/test/org/apache/lucene/search/similarities/LMDirichletSimilarity.java
    lucene/dev/branches/flexscoring/lucene/src/test/org/apache/lucene/search/similarities/LMJelinekMercerSimilarity.java
    lucene/dev/branches/flexscoring/lucene/src/test/org/apache/lucene/search/similarities/LMSimilarity.java
    lucene/dev/branches/flexscoring/lucene/src/test/org/apache/lucene/search/similarities/Lambda.java
    lucene/dev/branches/flexscoring/lucene/src/test/org/apache/lucene/search/similarities/LambdaDF.java
    lucene/dev/branches/flexscoring/lucene/src/test/org/apache/lucene/search/similarities/LambdaTTF.java
    lucene/dev/branches/flexscoring/lucene/src/test/org/apache/lucene/search/similarities/Normalization.java
    lucene/dev/branches/flexscoring/lucene/src/test/org/apache/lucene/search/similarities/NormalizationH1.java
    lucene/dev/branches/flexscoring/lucene/src/test/org/apache/lucene/search/similarities/NormalizationH2.java

Added: lucene/dev/branches/flexscoring/lucene/src/java/org/apache/lucene/search/similarities/AfterEffect.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/flexscoring/lucene/src/java/org/apache/lucene/search/similarities/AfterEffect.java?rev=1159912&view=auto
==============================================================================
--- lucene/dev/branches/flexscoring/lucene/src/java/org/apache/lucene/search/similarities/AfterEffect.java (added)
+++ lucene/dev/branches/flexscoring/lucene/src/java/org/apache/lucene/search/similarities/AfterEffect.java Sat Aug 20 20:23:51 2011
@@ -0,0 +1,63 @@
+package org.apache.lucene.search.similarities;
+
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+import org.apache.lucene.search.Explanation;
+
+/**
+ * This class acts as the base class for the implementations of the <em>first
+ * normalization of the informative content</em> in the DFR framework. This
+ * component is also called the <em>after effect</em> and is defined by the
+ * formula <em>Inf<sub>2</sub> = 1 - Prob<sub>2</sub></em>, where
+ * <em>Prob<sub>2</sub></em> measures the <em>information gain</em>.
+ * 
+ * @see DFRSimilarity
+ * @lucene.experimental
+ */
+public abstract class AfterEffect {
+  /** Returns the aftereffect score. */
+  public abstract float score(EasyStats stats, float tfn);
+  
+  /** Returns an explanation for the score. */
+  public abstract Explanation explain(EasyStats stats, float tfn);
+
+  /** Implementation used when there is no aftereffect. */
+  public static final class NoAfterEffect extends AfterEffect {
+    @Override
+    public final float score(EasyStats stats, float tfn) {
+      return 1f;
+    }
+
+    @Override
+    public final Explanation explain(EasyStats stats, float tfn) {
+      return new Explanation(1, "no aftereffect");
+    }
+    
+    @Override
+    public String toString() {
+      return "";
+    }
+  }
+  
+  /**
+   * Subclasses must override this method to return the code of the
+   * after effect formula. Refer to the original paper for the list. 
+   */
+  @Override
+  public abstract String toString();
+}

Added: lucene/dev/branches/flexscoring/lucene/src/java/org/apache/lucene/search/similarities/AfterEffectB.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/flexscoring/lucene/src/java/org/apache/lucene/search/similarities/AfterEffectB.java?rev=1159912&view=auto
==============================================================================
--- lucene/dev/branches/flexscoring/lucene/src/java/org/apache/lucene/search/similarities/AfterEffectB.java (added)
+++ lucene/dev/branches/flexscoring/lucene/src/java/org/apache/lucene/search/similarities/AfterEffectB.java Sat Aug 20 20:23:51 2011
@@ -0,0 +1,49 @@
+package org.apache.lucene.search.similarities;
+
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+import org.apache.lucene.search.Explanation;
+
+/**
+ * Model of the information gain based on the ration of two Bernoulli processes.
+ * @lucene.experimental
+ */
+public class AfterEffectB extends AfterEffect {
+  @Override
+  public final float score(EasyStats stats, float tfn) {
+    long F = stats.getTotalTermFreq();
+    int n = stats.getDocFreq();
+    return (F + 1) / (n * (tfn + 1));
+  }
+  
+  @Override
+  public final Explanation explain(EasyStats stats, float tfn) {
+    Explanation result = new Explanation();
+    result.setDescription(getClass().getSimpleName() + ", computed from: ");
+    result.setValue(score(stats, tfn));
+    result.addDetail(new Explanation(tfn, "tfn"));
+    result.addDetail(new Explanation(stats.getTotalTermFreq(), "totalTermFreq"));
+    result.addDetail(new Explanation(stats.getDocFreq(), "docFreq"));
+    return result;
+  }
+
+  @Override
+  public String toString() {
+    return "B";
+  }
+}

Added: lucene/dev/branches/flexscoring/lucene/src/java/org/apache/lucene/search/similarities/AfterEffectL.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/flexscoring/lucene/src/java/org/apache/lucene/search/similarities/AfterEffectL.java?rev=1159912&view=auto
==============================================================================
--- lucene/dev/branches/flexscoring/lucene/src/java/org/apache/lucene/search/similarities/AfterEffectL.java (added)
+++ lucene/dev/branches/flexscoring/lucene/src/java/org/apache/lucene/search/similarities/AfterEffectL.java Sat Aug 20 20:23:51 2011
@@ -0,0 +1,45 @@
+package org.apache.lucene.search.similarities;
+
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+import org.apache.lucene.search.Explanation;
+
+/**
+ * Model of the information gain based on Laplace's law of succession.
+ * @lucene.experimental
+ */
+public class AfterEffectL extends AfterEffect {
+  @Override
+  public final float score(EasyStats stats, float tfn) {
+    return 1 / (tfn + 1);
+  }
+  
+  @Override
+  public final Explanation explain(EasyStats stats, float tfn) {
+    Explanation result = new Explanation();
+    result.setDescription(getClass().getSimpleName() + ", computed from: ");
+    result.setValue(score(stats, tfn));
+    result.addDetail(new Explanation(tfn, "tfn"));
+    return result;
+  }
+  
+  @Override
+  public String toString() {
+    return "L";
+  }
+}

Added: lucene/dev/branches/flexscoring/lucene/src/java/org/apache/lucene/search/similarities/BM25Similarity.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/flexscoring/lucene/src/java/org/apache/lucene/search/similarities/BM25Similarity.java?rev=1159912&view=auto
==============================================================================
--- lucene/dev/branches/flexscoring/lucene/src/java/org/apache/lucene/search/similarities/BM25Similarity.java (added)
+++ lucene/dev/branches/flexscoring/lucene/src/java/org/apache/lucene/search/similarities/BM25Similarity.java Sat Aug 20 20:23:51 2011
@@ -0,0 +1,202 @@
+package org.apache.lucene.search.similarities;
+
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+import java.io.IOException;
+
+import org.apache.lucene.index.FieldInvertState;
+import org.apache.lucene.index.MultiFields;
+import org.apache.lucene.index.IndexReader.AtomicReaderContext;
+import org.apache.lucene.search.IndexSearcher;
+import org.apache.lucene.search.Similarity;
+import org.apache.lucene.util.BytesRef;
+import org.apache.lucene.util.SmallFloat;
+import org.apache.lucene.util.TermContext;
+
+/**
+ * BM25 Similarity.
+ */
+public class BM25Similarity extends Similarity {
+  private final float k1;
+  private final float b;
+  
+  public BM25Similarity(float k1, float b) {
+    this.k1 = k1;
+    this.b  = b;
+  }
+  
+  /** Sets the default values for BM25:
+   * <ul>
+   *   <li>{@code k1 = 1.2},
+   *   <li>{@code b = 0.75}.</li>
+   * </ul>
+   */
+  public BM25Similarity() {
+    this.k1 = 1.2f;
+    this.b  = 0.75f;
+  }
+  
+  /** Implemented as <code>log(1 + (numDocs - docFreq + 0.5)/(docFreq + 0.5))</code>. */
+  public float idf(int docFreq, int numDocs) {
+    return (float) Math.log(1 + ((numDocs - docFreq + 0.5D)/(docFreq + 0.5D)));
+  }
+  
+  /** Implemented as <code>1 / (distance + 1)</code>. */
+  public float sloppyFreq(int distance) {
+    return 1.0f / (distance + 1);
+  }
+  
+  /** The default implementation returns <code>1</code> */
+  public float scorePayload(int doc, int start, int end, BytesRef payload) {
+    return 1;
+  }
+  
+  /** return avg doc length across the field (or 1 if the codec does not store sumTotalTermFreq) */
+  public float avgFieldLength(IndexSearcher searcher, String field) throws IOException {
+    long sumTotalTermFreq = MultiFields.getTerms(searcher.getIndexReader(), field).getSumTotalTermFreq();
+    long maxdoc = searcher.getIndexReader().maxDoc();
+    return sumTotalTermFreq == -1 ? 1f : (float) (sumTotalTermFreq / (double) maxdoc);
+  }
+
+  @Override
+  public byte computeNorm(FieldInvertState state) {
+    final int numTerms = state.getLength() - state.getNumOverlap();
+    return encodeNormValue(state.getBoost() / (float) Math.sqrt(numTerms));
+  }
+  
+  public float decodeNormValue(byte b) {
+    return NORM_TABLE[b & 0xFF];
+  }
+
+  public byte encodeNormValue(float f) {
+    return SmallFloat.floatToByte315(f);
+  }
+  
+  /** Cache of decoded bytes. */
+  private static final float[] NORM_TABLE = new float[256];
+
+  static {
+    for (int i = 0; i < 256; i++) {
+      float f = SmallFloat.byte315ToFloat((byte)i);
+      NORM_TABLE[i] = 1.0f / (f*f);
+    }
+  }
+
+  @Override
+  public Stats computeStats(IndexSearcher searcher, String fieldName, float queryBoost, TermContext... termStats) throws IOException {
+    float value = 0.0f;
+    final int max = searcher.maxDoc();
+    
+    for (final TermContext stat : termStats ) {
+      value += idf(stat.docFreq(), max);
+    }
+    
+    return new BM25Stats(value, queryBoost, avgFieldLength(searcher, fieldName));
+  }
+
+  @Override
+  public final ExactDocScorer exactDocScorer(Stats stats, String fieldName, AtomicReaderContext context) throws IOException {
+    return new ExactBM25DocScorer((BM25Stats) stats, context.reader.norms(fieldName));
+  }
+
+  @Override
+  public final SloppyDocScorer sloppyDocScorer(Stats stats, String fieldName, AtomicReaderContext context) throws IOException {
+    return new SloppyBM25DocScorer((BM25Stats) stats, context.reader.norms(fieldName));
+  }
+  
+  private class ExactBM25DocScorer extends ExactDocScorer {
+    private final float weightValue;
+    private final byte[] norms;
+    private final float avgdl;
+    
+    ExactBM25DocScorer(BM25Stats stats, byte norms[]) {
+      this.weightValue = stats.weight;
+      this.avgdl = stats.avgdl;
+      this.norms = norms;
+    }
+    
+    // todo: optimize
+    @Override
+    public float score(int doc, int freq) {
+      // if there are no norms, we act as if b=0
+      float norm = norms == null ? k1 : k1 * ((1 - b) + b * (decodeNormValue(norms[doc])) / (avgdl));
+      return weightValue * (freq * (k1 + 1)) / (freq + norm);
+    }
+  }
+  
+  private class SloppyBM25DocScorer extends SloppyDocScorer {
+    private final float weightValue;
+    private final byte[] norms;
+    private final float avgdl;
+    
+    SloppyBM25DocScorer(BM25Stats stats, byte norms[]) {
+      this.weightValue = stats.weight;
+      this.avgdl = stats.avgdl;
+      this.norms = norms;
+    }
+    
+    // todo: optimize
+    @Override
+    public float score(int doc, float freq) {
+      // if there are no norms, we act as if b=0
+      float norm = norms == null ? k1 : k1 * ((1 - b) + b * (decodeNormValue(norms[doc])) / (avgdl));
+      return weightValue * (freq * (k1 + 1)) / (freq + norm);
+    }
+
+    @Override
+    public float computeSlopFactor(int distance) {
+      return sloppyFreq(distance);
+    }
+
+    @Override
+    public float computePayloadFactor(int doc, int start, int end, BytesRef payload) {
+      return scorePayload(doc, start, end, payload);
+    }
+  }
+  
+  /** Collection statistics for the BM25 model. */
+  private static class BM25Stats extends Stats {
+    /** BM25's idf */
+    private final float idf;
+    /** The average document length. */
+    private final float avgdl;
+    /** query's inner boost */
+    private final float queryBoost;
+    /** weight (idf * boost) */
+    private float weight;
+
+    BM25Stats(float idf, float queryBoost, float avgdl) {
+      this.idf = idf;
+      this.queryBoost = queryBoost;
+      this.avgdl = avgdl;
+    }
+
+    @Override
+    public float getValueForNormalization() {
+      // we return a TF-IDF like normalization to be nice, but we don't actually normalize ourselves.
+      final float queryWeight = idf * queryBoost;
+      return queryWeight * queryWeight;
+    }
+
+    @Override
+    public void normalize(float queryNorm, float topLevelBoost) {
+      // we don't normalize with queryNorm at all, we just capture the top-level boost
+      this.weight = idf * queryBoost * topLevelBoost;
+    } 
+  }
+}

Added: lucene/dev/branches/flexscoring/lucene/src/java/org/apache/lucene/search/similarities/BM25SimilarityProvider.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/flexscoring/lucene/src/java/org/apache/lucene/search/similarities/BM25SimilarityProvider.java?rev=1159912&view=auto
==============================================================================
--- lucene/dev/branches/flexscoring/lucene/src/java/org/apache/lucene/search/similarities/BM25SimilarityProvider.java (added)
+++ lucene/dev/branches/flexscoring/lucene/src/java/org/apache/lucene/search/similarities/BM25SimilarityProvider.java Sat Aug 20 20:23:51 2011
@@ -0,0 +1,34 @@
+package org.apache.lucene.search.similarities;
+
+import org.apache.lucene.search.DefaultSimilarityProvider;
+
+/**
+ * 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.
+ */
+
+public class BM25SimilarityProvider extends DefaultSimilarityProvider {
+
+  private BM25Similarity impl = new BM25Similarity();
+
+  public BM25Similarity get(String field) {
+    return impl;
+  }
+  
+  /** Sets the {@code Similarity} returned by {@link #get}. */
+  public void set(BM25Similarity sim) {
+    impl = sim;
+  }
+}

Added: lucene/dev/branches/flexscoring/lucene/src/java/org/apache/lucene/search/similarities/BasicModel.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/flexscoring/lucene/src/java/org/apache/lucene/search/similarities/BasicModel.java?rev=1159912&view=auto
==============================================================================
--- lucene/dev/branches/flexscoring/lucene/src/java/org/apache/lucene/search/similarities/BasicModel.java (added)
+++ lucene/dev/branches/flexscoring/lucene/src/java/org/apache/lucene/search/similarities/BasicModel.java Sat Aug 20 20:23:51 2011
@@ -0,0 +1,60 @@
+package org.apache.lucene.search.similarities;
+
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+import org.apache.lucene.search.Explanation;
+
+/**
+ * This class acts as the base class for the specific <em>basic model</em>
+ * implementations in the DFR framework. Basic models compute the
+ * <em>informative content Inf<sub>1</sub> = -log<sub>2</sub>Prob<sub>1</sub>
+ * </em>.
+ * 
+ * @see DFRSimilarity
+ * @lucene.experimental
+ */
+public abstract class BasicModel {
+  /** Returns the informative content score. */
+  public abstract float score(EasyStats stats, float tfn);
+  
+  /**
+   * Returns an explanation for the score.
+   * <p>Most basic models use the number of documents and the total term
+   * frequency to compute Inf<sub>1</sub>. This method provides a generic
+   * explanation for such models. Subclasses that use other statistics must
+   * override this method.</p>
+   */
+  public Explanation explain(EasyStats stats, float tfn) {
+    Explanation result = new Explanation();
+    result.setDescription(getClass().getSimpleName() + ", computed from: ");
+    result.setValue(score(stats, tfn));
+    result.addDetail(new Explanation(tfn, "tfn"));
+    result.addDetail(
+        new Explanation(stats.getNumberOfDocuments(), "numberOfDocuments"));
+    result.addDetail(
+        new Explanation(stats.getTotalTermFreq(), "totalTermFreq"));
+    return result;
+  }
+  
+  /**
+   * Subclasses must override this method to return the code of the
+   * basic model formula. Refer to the original paper for the list. 
+   */
+  @Override
+  public abstract String toString();
+}

Added: lucene/dev/branches/flexscoring/lucene/src/java/org/apache/lucene/search/similarities/BasicModelBE.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/flexscoring/lucene/src/java/org/apache/lucene/search/similarities/BasicModelBE.java?rev=1159912&view=auto
==============================================================================
--- lucene/dev/branches/flexscoring/lucene/src/java/org/apache/lucene/search/similarities/BasicModelBE.java (added)
+++ lucene/dev/branches/flexscoring/lucene/src/java/org/apache/lucene/search/similarities/BasicModelBE.java Sat Aug 20 20:23:51 2011
@@ -0,0 +1,48 @@
+package org.apache.lucene.search.similarities;
+
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+import static org.apache.lucene.search.similarities.EasySimilarity.log2;
+
+/**
+ * Limiting form of the Bose-Einstein model. The formula used in Lucene differs
+ * slightly from the one in the original paper: to avoid underflow for small
+ * values of {@code N} and {@code F}, {@code N} is increased by {@code 1} and
+ * {@code F} is ensured to be at least {@code tfn + 1}. 
+ * @lucene.experimental
+ */
+public class BasicModelBE extends BasicModel {
+  @Override
+  public final float score(EasyStats stats, float tfn) {
+    long N = stats.getNumberOfDocuments() + 1;
+//    long F = stats.getTotalTermFreq() + 1;
+    long F = Math.max(stats.getTotalTermFreq(), (long)(tfn + 0.5) + 1);
+    return (float)(-log2((N - 1) * Math.E)
+        + f(N + F - 1, N + F - tfn - 2) - f(F, F - tfn));
+  }
+  
+  /** The <em>f</em> helper function defined for <em>B<sub>E</sub></em>. */
+  private final double f(long n, float m) {
+    return (m + 0.5) * log2((double)n / m) + (n - m) * log2(n);
+  }
+  
+  @Override
+  public String toString() {
+    return "Be";
+  }
+}

Added: lucene/dev/branches/flexscoring/lucene/src/java/org/apache/lucene/search/similarities/BasicModelD.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/flexscoring/lucene/src/java/org/apache/lucene/search/similarities/BasicModelD.java?rev=1159912&view=auto
==============================================================================
--- lucene/dev/branches/flexscoring/lucene/src/java/org/apache/lucene/search/similarities/BasicModelD.java (added)
+++ lucene/dev/branches/flexscoring/lucene/src/java/org/apache/lucene/search/similarities/BasicModelD.java Sat Aug 20 20:23:51 2011
@@ -0,0 +1,47 @@
+package org.apache.lucene.search.similarities;
+
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+import static org.apache.lucene.search.similarities.EasySimilarity.log2;
+
+/**
+ * Implements the approximation of the binomial model with the divergence
+ * for DFR. The formula used in Lucene differs slightly from the one in the
+ * original paper: to avoid underflow for small values of {@code N} and
+ * {@code F}, {@code N} is increased by {@code 1} and
+ * {@code F} is ensured to be at least {@code tfn + 1}.
+ * @lucene.experimental
+ */
+public class BasicModelD extends BasicModel {
+  @Override
+  public final float score(EasyStats stats, float tfn) {
+    long F = Math.max(stats.getTotalTermFreq(), (long)(tfn + 0.5) + 1);
+//    long F = stats.getTotalTermFreq() + 1;
+    double phi = (double)tfn / F;
+    double nphi = 1 - phi;
+    double p = 1.0 / (stats.getNumberOfDocuments() + 1);
+    double D = phi * log2(phi / p) + nphi * log2(nphi / (1 - p));
+    // nocommit return (float)(D * F + 0.5 * log2(2 * Math.PI * tfn * nphi));
+    return (float)(D * F + 0.5 * log2(1 + 2 * Math.PI * tfn * nphi));
+  }
+  
+  @Override
+  public String toString() {
+    return "D";
+  }
+}

Added: lucene/dev/branches/flexscoring/lucene/src/java/org/apache/lucene/search/similarities/BasicModelG.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/flexscoring/lucene/src/java/org/apache/lucene/search/similarities/BasicModelG.java?rev=1159912&view=auto
==============================================================================
--- lucene/dev/branches/flexscoring/lucene/src/java/org/apache/lucene/search/similarities/BasicModelG.java (added)
+++ lucene/dev/branches/flexscoring/lucene/src/java/org/apache/lucene/search/similarities/BasicModelG.java Sat Aug 20 20:23:51 2011
@@ -0,0 +1,38 @@
+package org.apache.lucene.search.similarities;
+
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+import static org.apache.lucene.search.similarities.EasySimilarity.log2;
+
+/**
+ * Geometric as limiting form of the Bose-Einstein model.
+ * @lucene.experimental
+ */
+public class BasicModelG extends BasicModel {
+  @Override
+  public final float score(EasyStats stats, float tfn) {
+    double lambda = stats.getTotalTermFreq() / (double) stats.getNumberOfDocuments();
+    // -log(1 / (lambda + 1)) -> log(lambda + 1)
+    return (float)(log2(lambda + 1) + tfn * log2((1 + lambda) / lambda));
+  }
+
+  @Override
+  public String toString() {
+    return "G";
+  }
+}

Added: lucene/dev/branches/flexscoring/lucene/src/java/org/apache/lucene/search/similarities/BasicModelIF.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/flexscoring/lucene/src/java/org/apache/lucene/search/similarities/BasicModelIF.java?rev=1159912&view=auto
==============================================================================
--- lucene/dev/branches/flexscoring/lucene/src/java/org/apache/lucene/search/similarities/BasicModelIF.java (added)
+++ lucene/dev/branches/flexscoring/lucene/src/java/org/apache/lucene/search/similarities/BasicModelIF.java Sat Aug 20 20:23:51 2011
@@ -0,0 +1,38 @@
+package org.apache.lucene.search.similarities;
+
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+import static org.apache.lucene.search.similarities.EasySimilarity.log2;
+
+/**
+ * An approximation of the <em>I(n<sub>e</sub>)</em> model.
+ * @lucene.experimental
+ */ 
+public class BasicModelIF extends BasicModel {
+  @Override
+  public final float score(EasyStats stats, float tfn) {
+    int N = stats.getNumberOfDocuments();
+    long F = stats.getTotalTermFreq();
+    return tfn * (float)(log2(1 + (N + 1) / (F + 0.5)));
+  }
+
+  @Override
+  public String toString() {
+    return "I(F)";
+  }
+}

Added: lucene/dev/branches/flexscoring/lucene/src/java/org/apache/lucene/search/similarities/BasicModelIn.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/flexscoring/lucene/src/java/org/apache/lucene/search/similarities/BasicModelIn.java?rev=1159912&view=auto
==============================================================================
--- lucene/dev/branches/flexscoring/lucene/src/java/org/apache/lucene/search/similarities/BasicModelIn.java (added)
+++ lucene/dev/branches/flexscoring/lucene/src/java/org/apache/lucene/search/similarities/BasicModelIn.java Sat Aug 20 20:23:51 2011
@@ -0,0 +1,52 @@
+package org.apache.lucene.search.similarities;
+
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+import org.apache.lucene.search.Explanation;
+import static org.apache.lucene.search.similarities.EasySimilarity.log2;
+
+/**
+ * The basic tf-idf model of randomness.
+ * @lucene.experimental
+ */ 
+public class BasicModelIn extends BasicModel {
+  @Override
+  public final float score(EasyStats stats, float tfn) {
+    int N = stats.getNumberOfDocuments();
+    int n = stats.getDocFreq();
+    return tfn * (float)(log2((N + 1) / (n + 0.5)));
+  }
+  
+  @Override
+  public final Explanation explain(EasyStats stats, float tfn) {
+    Explanation result = new Explanation();
+    result.setDescription(getClass().getSimpleName() + ", computed from: ");
+    result.setValue(score(stats, tfn));
+    result.addDetail(new Explanation(tfn, "tfn"));
+    result.addDetail(
+        new Explanation(stats.getNumberOfDocuments(), "numberOfDocuments"));
+    result.addDetail(
+        new Explanation(stats.getDocFreq(), "docFreq"));
+    return result;
+  }
+
+  @Override
+  public String toString() {
+    return "I(n)";
+  }
+}

Added: lucene/dev/branches/flexscoring/lucene/src/java/org/apache/lucene/search/similarities/BasicModelIne.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/flexscoring/lucene/src/java/org/apache/lucene/search/similarities/BasicModelIne.java?rev=1159912&view=auto
==============================================================================
--- lucene/dev/branches/flexscoring/lucene/src/java/org/apache/lucene/search/similarities/BasicModelIne.java (added)
+++ lucene/dev/branches/flexscoring/lucene/src/java/org/apache/lucene/search/similarities/BasicModelIne.java Sat Aug 20 20:23:51 2011
@@ -0,0 +1,40 @@
+package org.apache.lucene.search.similarities;
+
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+import static org.apache.lucene.search.similarities.EasySimilarity.log2;
+
+/**
+ * Tf-idf model of randomness, based on a mixture of Poisson and inverse
+ * document frequency.
+ * @lucene.experimental
+ */ 
+public class BasicModelIne extends BasicModel {
+  @Override
+  public final float score(EasyStats stats, float tfn) {
+    int N = stats.getNumberOfDocuments();
+    long F = stats.getTotalTermFreq();
+    double ne = N * (1 - Math.pow((N - 1) / (double)N, F));
+    return tfn * (float)(log2((N + 1) / (ne + 0.5)));
+  }
+
+  @Override
+  public String toString() {
+    return "I(ne)";
+  }
+}

Added: lucene/dev/branches/flexscoring/lucene/src/java/org/apache/lucene/search/similarities/BasicModelP.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/flexscoring/lucene/src/java/org/apache/lucene/search/similarities/BasicModelP.java?rev=1159912&view=auto
==============================================================================
--- lucene/dev/branches/flexscoring/lucene/src/java/org/apache/lucene/search/similarities/BasicModelP.java (added)
+++ lucene/dev/branches/flexscoring/lucene/src/java/org/apache/lucene/search/similarities/BasicModelP.java Sat Aug 20 20:23:51 2011
@@ -0,0 +1,46 @@
+package org.apache.lucene.search.similarities;
+
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+import static org.apache.lucene.search.similarities.EasySimilarity.log2;
+
+/**
+ * Implements the Poisson approximation for the binomial model for DFR.
+ * @lucene.experimental
+ */
+public class BasicModelP extends BasicModel {
+  /** {@code log2(Math.E)}, precomputed. */
+  protected static double LOG2_E = log2(Math.E);
+  
+  @Override
+  public final float score(EasyStats stats, float tfn) {
+    float lambda = (float)stats.getTotalTermFreq() / stats.getNumberOfDocuments();
+//    System.out.printf("tfn=%f, lambda=%f, log1=%f, log2=%f%n", tfn, lambda,
+//        tfn / lambda, 2 * Math.PI * tfn);
+    // nocommit
+    float score = (float)(tfn * log2(tfn / lambda)
+        + (lambda + 1 / (12 * tfn) - tfn) * LOG2_E
+        + 0.5 * log2(2 * Math.PI * tfn));
+    return score > 0.0f ? score : 0.0f;
+  }
+
+  @Override
+  public String toString() {
+    return "P";
+  }
+}

Added: lucene/dev/branches/flexscoring/lucene/src/java/org/apache/lucene/search/similarities/DFRSimilarity.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/flexscoring/lucene/src/java/org/apache/lucene/search/similarities/DFRSimilarity.java?rev=1159912&view=auto
==============================================================================
--- lucene/dev/branches/flexscoring/lucene/src/java/org/apache/lucene/search/similarities/DFRSimilarity.java (added)
+++ lucene/dev/branches/flexscoring/lucene/src/java/org/apache/lucene/search/similarities/DFRSimilarity.java Sat Aug 20 20:23:51 2011
@@ -0,0 +1,106 @@
+package org.apache.lucene.search.similarities;
+
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+import org.apache.lucene.search.Explanation;
+
+/**
+ * Implements the <em>divergence from randomness (DFR)</em> framework
+ * introduced in Gianni Amati and Cornelis Joost Van Rijsbergen. 2002.
+ * Probabilistic models of information retrieval based on measuring the
+ * divergence from randomness. ACM Trans. Inf. Syst. 20, 4 (October 2002),
+ * 357-389.
+ * <p>The DFR scoring formula is composed of three separate components: the
+ * <em>basic model</em>, the <em>aftereffect</em> and an additional
+ * <em>normalization</em> component, represented by the classes
+ * {@code BasicModel}, {@code AfterEffect} and {@code Normalization},
+ * respectively. The names of these classes were chosen to match the names of
+ * their counterparts in the Terrier IR engine.</p>
+ * <p>Note that <em>qtf</em>, the multiplicity of term-occurrence in the query,
+ * is not handled by this implementation.</p>
+ * 
+ * @see BasicModel
+ * @see AfterEffect
+ * @see Normalization
+ * @lucene.experimental
+ */
+public class DFRSimilarity extends EasySimilarity {
+  /** The basic model for information content. */
+  protected final BasicModel basicModel;
+  /** The first normalization of the information content. */
+  protected final AfterEffect afterEffect;
+  /** The term frequency normalization. */
+  protected final Normalization normalization;
+  
+  public DFRSimilarity(BasicModel basicModel,
+                       AfterEffect afterEffect,
+                       Normalization normalization) {
+    if (basicModel == null || afterEffect == null || normalization == null) {
+      throw new NullPointerException("null parameters not allowed.");
+    }
+    this.basicModel = basicModel;
+    this.afterEffect = afterEffect;
+    this.normalization = normalization;
+  }
+
+  /** Creates a DFR model with no normalization. */
+  public DFRSimilarity(BasicModel basicModel,
+                       AfterEffect afterEffect) {
+    this(basicModel, afterEffect, new Normalization.NoNormalization());
+  }
+  
+  /** Creates a DFR model with no aftereffect. */
+  public DFRSimilarity(BasicModel basicModel,
+                       Normalization normalization) {
+    this(basicModel, new AfterEffect.NoAfterEffect(), normalization);
+  }
+  
+  /** Creates a DFR model with only a basic model. */
+  public DFRSimilarity(BasicModel basicModel) {
+    this(basicModel,
+         new AfterEffect.NoAfterEffect(),
+         new Normalization.NoNormalization());
+  }
+  
+  @Override
+  protected float score(EasyStats stats, float freq, float docLen) {
+    float tfn = normalization.tfn(stats, freq, docLen);
+    return stats.getTotalBoost() *
+        basicModel.score(stats, tfn) * afterEffect.score(stats, tfn);
+  }
+
+  @Override
+  protected void explain(Explanation expl,
+      EasyStats stats, int doc, float freq, float docLen) {
+    if (stats.getTotalBoost() != 1.0f) {
+      expl.addDetail(new Explanation(stats.getTotalBoost(), "boost"));
+    }
+    
+    Explanation normExpl = normalization.explain(stats, freq, docLen);
+    float tfn = normExpl.getValue();
+    expl.addDetail(normExpl);
+    expl.addDetail(basicModel.explain(stats, tfn));
+    expl.addDetail(afterEffect.explain(stats, tfn));
+  }
+
+  @Override
+  public String toString() {
+    return "DFR " + basicModel.toString() + afterEffect.toString()
+                  + normalization.toString();
+  }
+}

Added: lucene/dev/branches/flexscoring/lucene/src/java/org/apache/lucene/search/similarities/Distribution.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/flexscoring/lucene/src/java/org/apache/lucene/search/similarities/Distribution.java?rev=1159912&view=auto
==============================================================================
--- lucene/dev/branches/flexscoring/lucene/src/java/org/apache/lucene/search/similarities/Distribution.java (added)
+++ lucene/dev/branches/flexscoring/lucene/src/java/org/apache/lucene/search/similarities/Distribution.java Sat Aug 20 20:23:51 2011
@@ -0,0 +1,45 @@
+package org.apache.lucene.search.similarities;
+
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+import org.apache.lucene.search.Explanation;
+
+/**
+ * The probabilistic distribution used to model term occurrence
+ * in information-based models.
+ * @see IBSimilarity
+ * @lucene.experimental
+ */
+public abstract class Distribution {
+  /** Computes the score. */
+  public abstract float score(EasyStats stats, float tfn, float lambda);
+  
+  /** Explains the score. Returns the name of the model only, since
+   * both {@code tfn} and {@code lambda} are explained elsewhere. */
+  public Explanation explain(EasyStats stats, float tfn, float lambda) {
+    return new Explanation(
+        score(stats, tfn, lambda), getClass().getSimpleName());
+  }
+  
+  /**
+   * Subclasses must override this method to return the name of the
+   * distribution. 
+   */
+  @Override
+  public abstract String toString();
+}

Added: lucene/dev/branches/flexscoring/lucene/src/java/org/apache/lucene/search/similarities/DistributionLL.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/flexscoring/lucene/src/java/org/apache/lucene/search/similarities/DistributionLL.java?rev=1159912&view=auto
==============================================================================
--- lucene/dev/branches/flexscoring/lucene/src/java/org/apache/lucene/search/similarities/DistributionLL.java (added)
+++ lucene/dev/branches/flexscoring/lucene/src/java/org/apache/lucene/search/similarities/DistributionLL.java Sat Aug 20 20:23:51 2011
@@ -0,0 +1,37 @@
+package org.apache.lucene.search.similarities;
+
+/**
+ * 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.
+ */
+
+/**
+ * Log-logistic distribution.
+ * <p>Unlike for DFR, the natural logarithm is used, as
+ * it is faster to compute and the original paper does not express any
+ * preference to a specific base.</p>
+ * @lucene.experimental
+ */
+public class DistributionLL extends Distribution {
+  @Override
+  public final float score(EasyStats stats, float tfn, float lambda) {
+    return (float)-Math.log(lambda / (tfn + lambda));
+  }
+  
+  @Override
+  public String toString() {
+    return "LL";
+  }
+}

Added: lucene/dev/branches/flexscoring/lucene/src/java/org/apache/lucene/search/similarities/DistributionSPL.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/flexscoring/lucene/src/java/org/apache/lucene/search/similarities/DistributionSPL.java?rev=1159912&view=auto
==============================================================================
--- lucene/dev/branches/flexscoring/lucene/src/java/org/apache/lucene/search/similarities/DistributionSPL.java (added)
+++ lucene/dev/branches/flexscoring/lucene/src/java/org/apache/lucene/search/similarities/DistributionSPL.java Sat Aug 20 20:23:51 2011
@@ -0,0 +1,42 @@
+package org.apache.lucene.search.similarities;
+
+/**
+ * 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.
+ */
+
+/**
+ * The smoothed power-law (SPL) distribution for the information-based framework
+ * that is described in the original paper.
+ * <p>Unlike for DFR, the natural logarithm is used, as
+ * it is faster to compute and the original paper does not express any
+ * preference to a specific base.</p>
+ * @lucene.experimental
+ */
+public class DistributionSPL extends Distribution {
+  @Override
+  public final float score(EasyStats stats, float tfn, float lambda) {
+    if (lambda == 1f) {
+      lambda = 0.99f;
+    }
+    return (float)-Math.log(
+        (Math.pow(lambda, (tfn / (tfn + 1))) - lambda) / (1 - lambda));
+  }
+  
+  @Override
+  public String toString() {
+    return "SPL";
+  }
+}

Added: lucene/dev/branches/flexscoring/lucene/src/java/org/apache/lucene/search/similarities/EasySimilarity.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/flexscoring/lucene/src/java/org/apache/lucene/search/similarities/EasySimilarity.java?rev=1159912&view=auto
==============================================================================
--- lucene/dev/branches/flexscoring/lucene/src/java/org/apache/lucene/search/similarities/EasySimilarity.java (added)
+++ lucene/dev/branches/flexscoring/lucene/src/java/org/apache/lucene/search/similarities/EasySimilarity.java Sat Aug 20 20:23:51 2011
@@ -0,0 +1,298 @@
+package org.apache.lucene.search.similarities;
+
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+import java.io.IOException;
+
+import org.apache.lucene.index.FieldInvertState;
+import org.apache.lucene.index.IndexReader;
+import org.apache.lucene.index.IndexReader.AtomicReaderContext;
+import org.apache.lucene.index.MultiFields;
+import org.apache.lucene.search.Explanation;
+import org.apache.lucene.search.IndexSearcher;
+import org.apache.lucene.search.Similarity;
+import org.apache.lucene.util.BytesRef;
+import org.apache.lucene.util.SmallFloat;
+import org.apache.lucene.util.TermContext;
+
+/**
+ * A subclass of {@code Similarity} that provides a simplified API for its
+ * descendants.
+ * @lucene.experimental
+ */
+public abstract class EasySimilarity extends Similarity {
+  /** For {@link #log2(double)}. Precomputed for efficiency reasons. */
+  private static final double LOG_2 = Math.log(2);
+  
+  /** @see #setDiscountOverlaps */
+  protected boolean discountOverlaps = true;
+  
+  /** Determines whether overlap tokens (Tokens with
+   *  0 position increment) are ignored when computing
+   *  norm.  By default this is true, meaning overlap
+   *  tokens do not count when computing norms.
+   *
+   *  @lucene.experimental
+   *
+   *  @see #computeNorm
+   */
+  public void setDiscountOverlaps(boolean v) {
+    discountOverlaps = v;
+  }
+
+  /** @see #setDiscountOverlaps */
+  public boolean getDiscountOverlaps() {
+    return discountOverlaps;
+  }
+  
+  /**
+   * Calls {@link #fillEasyStats(EasyStats, IndexSearcher, String, TermContext...)}.
+   * Subclasses that override this method may invoke {@code fillStats} with any
+   * subclass of {@code EasyStats}.
+   */
+  @Override
+  public EasyStats computeStats(IndexSearcher searcher, String fieldName,
+      float queryBoost, TermContext... termContexts) throws IOException {
+    EasyStats stats = new EasyStats(queryBoost);
+    fillEasyStats(stats, searcher, fieldName, termContexts);
+    return stats;
+  }
+  
+  /** Fills all member fields defined in {@code EasyStats} in {@code stats}. */
+  protected final void fillEasyStats(EasyStats stats, IndexSearcher searcher,
+      String fieldName, TermContext... termContexts) throws IOException {
+    IndexReader reader = searcher.getIndexReader();
+    int numberOfDocuments = reader.maxDoc();
+    long numberOfFieldTokens = MultiFields.getTerms(searcher.getIndexReader(),
+        fieldName).getSumTotalTermFreq();
+    float avgFieldLength = (float)numberOfFieldTokens / numberOfDocuments;
+    
+    // nocommit Take the minimum of term frequencies for phrases. This is not
+    // correct though, we'll need something like a scorePhrase(MultiStats ...)
+    int docFreq = Integer.MAX_VALUE;
+    long totalTermFreq = Integer.MAX_VALUE;
+    for (final TermContext context : termContexts) {
+      docFreq = Math.min(docFreq, context.docFreq());
+      totalTermFreq = Math.min(totalTermFreq, context.totalTermFreq());
+    }
+    
+    // We have to provide something if codec doesnt supply these measures,
+    // or if someone omitted frequencies for the field... negative values cause
+    // NaN/Inf for some scorers.
+    if (numberOfFieldTokens == -1) {
+      numberOfFieldTokens = docFreq;
+      avgFieldLength = 1;
+    }
+    if (totalTermFreq == -1) {
+      totalTermFreq = docFreq;
+    }
+    
+    stats.setNumberOfDocuments(numberOfDocuments);
+    stats.setNumberOfFieldTokens(numberOfFieldTokens);
+    stats.setAvgFieldLength(avgFieldLength);
+    stats.setDocFreq(docFreq);
+    stats.setTotalTermFreq(totalTermFreq);
+  }
+  
+  /**
+   * Scores the document {@code doc}.
+   * <p>Subclasses must apply their scoring formula in this class.</p>
+   * @param stats the corpus level statistics.
+   * @param freq the term frequency.
+   * @param docLen the document length.
+   * @return the score.
+   */
+  protected abstract float score(EasyStats stats, float freq, float docLen);
+  
+  /**
+   * Subclasses should implement this method to explain the score. {@code expl}
+   * already contains the score, the name of the class and the doc id, as well
+   * as the term frequency and its explanation; subclasses can add additional
+   * clauses to explain details of their scoring formulae.
+   * <p>The default implementation does nothing.</p>
+   * 
+   * @param expl the explanation to extend with details.
+   * @param stats the corpus level statistics.
+   * @param doc the document id.
+   * @param freq the term frequency.
+   * @param docLen the document length.
+   */
+  protected void explain(
+      Explanation expl, EasyStats stats, int doc, float freq, float docLen) {}
+  
+  /**
+   * Explains the score. The implementation here provides a basic explanation
+   * in the format <em>score(name-of-similarity, doc=doc-id,
+   * freq=term-frequency), computed from:</em>, and
+   * attaches the score (computed via the {@link #score(EasyStats, float, byte)}
+   * method) and the explanation for the term frequency. Subclasses content with
+   * this format may add additional details in
+   * {@link #explain(Explanation, EasyStats, int, float, byte)}.
+   *  
+   * @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(
+      EasyStats stats, int doc, Explanation freq, float docLen) {
+    Explanation result = new Explanation(); 
+    result.setValue(score(stats, freq.getValue(), docLen));
+    result.setDescription("score(" + getClass().getSimpleName() +
+        ", doc=" + doc + ", freq=" + freq.getValue() +"), computed from:");
+    result.addDetail(freq);
+    
+    explain(result, stats, doc, freq.getValue(), docLen);
+    
+    return result;
+  }
+  
+  @Override
+  public ExactDocScorer exactDocScorer(Stats stats, String fieldName,
+      AtomicReaderContext context) throws IOException {
+    return new EasyExactDocScorer((EasyStats) stats,
+                                  context.reader.norms(fieldName));
+  }
+  
+  @Override
+  public SloppyDocScorer sloppyDocScorer(Stats stats, String fieldName,
+      AtomicReaderContext context) throws IOException {
+    return new EasySloppyDocScorer((EasyStats) stats,
+                                   context.reader.norms(fieldName));
+  }
+  
+  /**
+   * Subclasses must override this method to return the name of the Similarity
+   * and preferably the values of parameters (if any) as well.
+   */
+  @Override
+  public abstract String toString();  // nocommit: to Similarity?
+
+  // ------------------------------ Norm handling ------------------------------
+  
+  /** Norm -> document length map. */
+  private static final float[] NORM_TABLE = new float[256];
+
+  static {
+    for (int i = 0; i < 256; i++) {
+      float floatNorm = SmallFloat.byte315ToFloat((byte)i);
+      NORM_TABLE[i] = 1.0f / (floatNorm * floatNorm);
+    }
+  }
+
+  /** Encodes the document length in the same way as {@link TFIDFSimilarity}. */
+  @Override
+  public byte computeNorm(FieldInvertState state) {
+    final float numTerms;
+    if (discountOverlaps)
+      numTerms = state.getLength() - state.getNumOverlap();
+    else
+      numTerms = state.getLength() / state.getBoost();
+    return encodeNormValue(numTerms);
+  }
+  
+  /** Decodes a normalization factor (document length) stored in an index.
+   * @see #encodeNormValue(float)
+   */
+  protected float decodeNormValue(byte norm) {
+    return NORM_TABLE[norm & 0xFF];  // & 0xFF maps negative bytes to positive above 127
+  }
+  
+  /** Encodes the length to a byte via SmallFloat. */
+  protected byte encodeNormValue(float length) {
+    return SmallFloat.floatToByte315((float)(1.0 / Math.sqrt(length)));
+  }
+  
+  // ----------------------------- Static methods ------------------------------
+  
+  /** Returns the base two logarithm of {@code x}. */
+  public static double log2(double x) {
+    // Put this to a 'util' class if we need more of these.
+    return Math.log(x) / LOG_2;
+  }
+  
+  // --------------------------------- Classes ---------------------------------
+  
+  /** Delegates the {@link #score(int, int)} and
+   * {@link #explain(int, Explanation)} methods to
+   * {@link EasySimilarity#score(EasyStats, float, int)} and
+   * {@link EasySimilarity#explain(EasyStats, int, Explanation, int)},
+   * respectively.
+   */
+  private class EasyExactDocScorer extends ExactDocScorer {
+    private final EasyStats stats;
+    private final byte[] norms;
+    
+    EasyExactDocScorer(EasyStats stats, byte norms[]) {
+      this.stats = stats;
+      this.norms = norms;
+    }
+    
+    @Override
+    public float score(int doc, int freq) {
+      // We have to supply something in case norms are omitted
+      return EasySimilarity.this.score(stats, freq,
+          norms == null ? freq : decodeNormValue(norms[doc]));
+    }
+    
+    @Override
+    public Explanation explain(int doc, Explanation freq) {
+      return EasySimilarity.this.explain(stats, doc, freq,
+          norms == null ? freq.getValue() : decodeNormValue(norms[doc]));
+    }
+  }
+  
+  /** Delegates the {@link #score(int, int)} and
+   * {@link #explain(int, Explanation)} methods to
+   * {@link EasySimilarity#score(EasyStats, float, int)} and
+   * {@link EasySimilarity#explain(EasyStats, int, Explanation, int)},
+   * respectively.
+   */
+  private class EasySloppyDocScorer extends SloppyDocScorer {
+    private final EasyStats stats;
+    private final byte[] norms;
+    
+    EasySloppyDocScorer(EasyStats stats, byte norms[]) {
+      this.stats = stats;
+      this.norms = norms;
+    }
+    
+    @Override
+    public float score(int doc, float freq) {
+      // We have to supply something in case norms are omitted
+      return EasySimilarity.this.score(stats, freq,
+          norms == null ? freq : decodeNormValue(norms[doc]));
+    }
+    @Override
+    public Explanation explain(int doc, Explanation freq) {
+      return EasySimilarity.this.explain(stats, doc, freq,
+          norms == null ? freq.getValue() : decodeNormValue(norms[doc]));
+    }
+
+    @Override
+    public float computeSlopFactor(int distance) {
+      return 1.0f / (distance + 1);
+    }
+
+    @Override
+    public float computePayloadFactor(int doc, int start, int end, BytesRef payload) {
+      return 1f;
+    }
+  }
+}

Added: lucene/dev/branches/flexscoring/lucene/src/java/org/apache/lucene/search/similarities/EasyStats.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/flexscoring/lucene/src/java/org/apache/lucene/search/similarities/EasyStats.java?rev=1159912&view=auto
==============================================================================
--- lucene/dev/branches/flexscoring/lucene/src/java/org/apache/lucene/search/similarities/EasyStats.java (added)
+++ lucene/dev/branches/flexscoring/lucene/src/java/org/apache/lucene/search/similarities/EasyStats.java Sat Aug 20 20:23:51 2011
@@ -0,0 +1,146 @@
+package org.apache.lucene.search.similarities;
+
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+import org.apache.lucene.index.IndexReader;
+import org.apache.lucene.index.Terms;
+import org.apache.lucene.search.Similarity;
+
+/**
+ * Stores all statistics commonly used ranking methods.
+ * @lucene.experimental
+ */
+public class EasyStats extends Similarity.Stats {
+  /** The number of documents. */
+  protected int numberOfDocuments;
+  /** The total number of tokens in the field. */
+  protected long numberOfFieldTokens;
+  /** The average field length. */
+  protected float avgFieldLength;
+  /** The document frequency. */
+  protected int docFreq;
+  /** The total number of occurrences of this term across all documents. */
+  protected long totalTermFreq;
+  
+  // -------------------------- Boost-related stuff --------------------------
+  
+  /** Query's inner boost. */
+  protected final float queryBoost;
+  /** Any outer query's boost. */
+  protected float topLevelBoost;
+  /** For most Similarities, the immediate and the top level query boosts are
+   * not handled differently. Hence, this field is just the product of the
+   * other two. */
+  protected float totalBoost;
+  
+  /** Constructor. Sets the query boost. */
+  public EasyStats(float queryBoost) {
+    this.queryBoost = queryBoost;
+    this.totalBoost = queryBoost;
+  }
+  
+  // ------------------------- Getter/setter methods -------------------------
+  
+  /** Returns the number of documents. */
+  public int getNumberOfDocuments() {
+    return numberOfDocuments;
+  }
+  
+  /** Sets the number of documents. */
+  public void setNumberOfDocuments(int numberOfDocuments) {
+    this.numberOfDocuments = numberOfDocuments;
+  }
+  
+  /**
+   * Returns the total number of tokens in the field.
+   * @see Terms#getSumTotalTermFreq()
+   */
+  public long getNumberOfFieldTokens() {
+    return numberOfFieldTokens;
+  }
+  
+  /**
+   * Sets the total number of tokens in the field.
+   * @see Terms#getSumTotalTermFreq()
+   */
+  public void setNumberOfFieldTokens(long numberOfFieldTokens) {
+    this.numberOfFieldTokens = numberOfFieldTokens;
+  }
+  
+  /** Returns the average field length. */
+  public float getAvgFieldLength() {
+    return avgFieldLength;
+  }
+  
+  /** Sets the average field length. */
+  public void setAvgFieldLength(float avgFieldLength) {
+    this.avgFieldLength = avgFieldLength;
+  }
+  
+  /** Returns the document frequency. */
+  public int getDocFreq() {
+    return docFreq;
+  }
+  
+  /** Sets the document frequency. */
+  public void setDocFreq(int docFreq) {
+    this.docFreq = docFreq;
+  }
+  
+  /** Returns the total number of occurrences of this term across all documents. */
+  public long getTotalTermFreq() {
+    return totalTermFreq;
+  }
+  
+  /** Sets the total number of occurrences of this term across all documents. */
+  public void setTotalTermFreq(long totalTermFreq) {
+    this.totalTermFreq = totalTermFreq;
+  }
+  
+  // -------------------------- Boost-related stuff --------------------------
+  
+  /** The square of the raw normalization value.
+   * @see #rawNormalizationValue() */
+  @Override
+  public float getValueForNormalization() {
+    float rawValue = rawNormalizationValue();
+    return rawValue * rawValue;
+  }
+  
+  /** Computes the raw normalization value. This basic implementation returns
+   * the query boost. Subclasses may override this method to include other
+   * factors (such as idf), or to save the value for inclusion in
+   * {@link #normalize(float, float)}, etc.
+   */
+  protected float rawNormalizationValue() {
+    return queryBoost;
+  }
+  
+  /** No normalization is done. {@code topLevelBoost} is saved in the object,
+   * however. */
+  @Override
+  public void normalize(float queryNorm, float topLevelBoost) {
+    this.topLevelBoost = topLevelBoost;
+    totalBoost = queryBoost * topLevelBoost;
+  }
+  
+  /** Returns the total boost. */
+  public float getTotalBoost() {
+    return totalBoost;
+  }
+}

Added: lucene/dev/branches/flexscoring/lucene/src/java/org/apache/lucene/search/similarities/IBSimilarity.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/flexscoring/lucene/src/java/org/apache/lucene/search/similarities/IBSimilarity.java?rev=1159912&view=auto
==============================================================================
--- lucene/dev/branches/flexscoring/lucene/src/java/org/apache/lucene/search/similarities/IBSimilarity.java (added)
+++ lucene/dev/branches/flexscoring/lucene/src/java/org/apache/lucene/search/similarities/IBSimilarity.java Sat Aug 20 20:23:51 2011
@@ -0,0 +1,99 @@
+package org.apache.lucene.search.similarities;
+
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+import org.apache.lucene.search.Explanation;
+
+/**
+ * Provides a framework for the family of information-based models, as described
+ * in St&eacute;phane Clinchant and Eric Gaussier. 2010. Information-based
+ * models for ad hoc IR. In Proceeding of the 33rd international ACM SIGIR
+ * conference on Research and development in information retrieval (SIGIR '10).
+ * ACM, New York, NY, USA, 234-241.
+ * <p>The retrieval function is of the form <em>RSV(q, d) = &sum;
+ * -x<sup>q</sup><sub>w</sub> log Prob(X<sub>w</sub> &ge;
+ * t<sup>d</sup><sub>w</sub> | &lambda;<sub>w</sub>)</em>, where
+ * <ul>
+ *   <li><em>x<sup>q</sup><sub>w</sub></em> is the query boost;</li>
+ *   <li><em>X<sub>w</sub></em> is a random variable that counts the occurrences
+ *   of word <em>w</em>;</li>
+ *   <li><em>t<sup>d</sup><sub>w</sub></em> is the normalized term frequency;</li>
+ *   <li><em>&lambda;<sub>w</sub></em> is a parameter.</li>
+ * </ul>
+ * </p>
+ * <p>The framework described in the paper has many similarities to the DFR
+ * framework (see {@link DFRSimilarity}). It is possible that the two
+ * Similarities will be merged at one point.</p>
+ * @lucene.experimental 
+ */
+public class IBSimilarity extends EasySimilarity {
+  /** The probabilistic distribution used to model term occurrence. */
+  protected final Distribution distribution;
+  /** The <em>lambda (&lambda;<sub>w</sub>)</em> parameter. */
+  protected final Lambda lambda;
+  /** The term frequency normalization. */
+  protected final Normalization normalization;
+  
+  public IBSimilarity(Distribution distribution,
+                      Lambda lambda,
+                      Normalization normalization) {
+    this.distribution = distribution;
+    this.lambda = lambda;
+    this.normalization = normalization;
+  }
+  
+  /** Creates an instance with no normalization. */
+  public IBSimilarity(Distribution distribution, Lambda lambda) {
+    this(distribution, lambda, new Normalization.NoNormalization());
+  }
+  
+  @Override
+  protected float score(EasyStats stats, float freq, float docLen) {
+    return stats.getTotalBoost() *
+        distribution.score(
+            stats,
+            normalization.tfn(stats, freq, docLen),
+            lambda.lambda(stats));
+  }
+
+  @Override
+  protected void explain(
+      Explanation expl, EasyStats stats, int doc, float freq, float docLen) {
+    if (stats.getTotalBoost() != 1.0f) {
+      expl.addDetail(new Explanation(stats.getTotalBoost(), "boost"));
+    }
+    Explanation normExpl = normalization.explain(stats, freq, docLen);
+    Explanation lambdaExpl = lambda.explain(stats);
+    expl.addDetail(normExpl);
+    expl.addDetail(lambdaExpl);
+    expl.addDetail(distribution.explain(
+        stats, normExpl.getValue(), lambdaExpl.getValue()));
+  }
+  
+  /**
+   * The name of IB methods follow the pattern
+   * {@code IB <distribution> <lambda><normalization>}. The name of the
+   * distribution is the same as in the original paper; for the names of lambda
+   * parameters, refer to the javadoc of the {@link Lambda} classes.
+   */
+  @Override
+  public String toString() {
+    return "IB " + distribution.toString() + "-" + lambda.toString()
+                 + normalization.toString();
+  }
+}

Added: lucene/dev/branches/flexscoring/lucene/src/java/org/apache/lucene/search/similarities/LMDirichletSimilarity.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/flexscoring/lucene/src/java/org/apache/lucene/search/similarities/LMDirichletSimilarity.java?rev=1159912&view=auto
==============================================================================
--- lucene/dev/branches/flexscoring/lucene/src/java/org/apache/lucene/search/similarities/LMDirichletSimilarity.java (added)
+++ lucene/dev/branches/flexscoring/lucene/src/java/org/apache/lucene/search/similarities/LMDirichletSimilarity.java Sat Aug 20 20:23:51 2011
@@ -0,0 +1,97 @@
+package org.apache.lucene.search.similarities;
+
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+import org.apache.lucene.search.Explanation;
+
+/**
+ * Bayesian smoothing using Dirichlet priors. From Chengxiang Zhai and John
+ * Lafferty. 2001. A study of smoothing methods for language models applied to
+ * Ad Hoc information retrieval. In Proceedings of the 24th annual international
+ * ACM SIGIR conference on Research and development in information retrieval
+ * (SIGIR '01). ACM, New York, NY, USA, 334-342.
+ * <p>
+ * The formula as defined the paper assigns a negative score to documents that
+ * contain the term, but with fewer occurrences than predicted by the collection
+ * language model. The Lucene implementation returns {@code 0} for such
+ * documents.
+ * </p>
+ * 
+ * @lucene.experimental
+ */
+public class LMDirichletSimilarity extends LMSimilarity {
+  /** The &mu; parameter. */
+  private final float mu;
+  
+  /** @param mu the &mu; parameter. */
+  public LMDirichletSimilarity(CollectionModel collectionModel, float mu) {
+    super(collectionModel);
+    this.mu = mu;
+  }
+  
+  /** @param mu the &mu; parameter. */
+  public LMDirichletSimilarity(float mu) {
+    this.mu = mu;
+  }
+
+  /** Instantiates the similarity with the default &mu; value of 2000. */
+  public LMDirichletSimilarity(CollectionModel collectionModel) {
+    this(collectionModel, 2000);
+  }
+  
+  /** Instantiates the similarity with the default &mu; value of 2000. */
+  public LMDirichletSimilarity() {
+    this(2000);
+  }
+  
+  @Override
+  protected float score(EasyStats stats, float freq, float docLen) {
+    float score = stats.getTotalBoost() * (float)(Math.log(1 + freq /
+        (mu * ((LMStats)stats).getCollectionProbability())) +
+        Math.log(mu / (docLen + mu)));
+    return score > 0.0f ? score : 0.0f;
+  }
+  
+  @Override
+  protected void explain(Explanation expl, EasyStats stats, int doc,
+      float freq, float docLen) {
+    if (stats.getTotalBoost() != 1.0f) {
+      expl.addDetail(new Explanation(stats.getTotalBoost(), "boost"));
+    }
+
+    expl.addDetail(new Explanation(mu, "mu"));
+    Explanation weightExpl = new Explanation();
+    weightExpl.setValue((float)Math.log(1 + freq /
+        (mu * ((LMStats)stats).getCollectionProbability())));
+    weightExpl.setDescription("term weight");
+    expl.addDetail(weightExpl);
+    expl.addDetail(new Explanation(
+        (float)Math.log(mu / (docLen + mu)), "document norm"));
+    super.explain(expl, stats, doc, freq, docLen);
+  }
+
+  /** Returns the &mu; parameter. */
+  public float getMu() {
+    return mu;
+  }
+  
+  @Override
+  public String getName() {
+    return String.format("Dirichlet(%f)", getMu());
+  }
+}

Added: lucene/dev/branches/flexscoring/lucene/src/java/org/apache/lucene/search/similarities/LMJelinekMercerSimilarity.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/flexscoring/lucene/src/java/org/apache/lucene/search/similarities/LMJelinekMercerSimilarity.java?rev=1159912&view=auto
==============================================================================
--- lucene/dev/branches/flexscoring/lucene/src/java/org/apache/lucene/search/similarities/LMJelinekMercerSimilarity.java (added)
+++ lucene/dev/branches/flexscoring/lucene/src/java/org/apache/lucene/search/similarities/LMJelinekMercerSimilarity.java Sat Aug 20 20:23:51 2011
@@ -0,0 +1,78 @@
+package org.apache.lucene.search.similarities;
+
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+import org.apache.lucene.search.Explanation;
+import org.apache.lucene.search.similarities.LMSimilarity.LMStats;
+
+/**
+ * Language model based on the Jelinek-Mercer smoothing method. From Chengxiang
+ * Zhai and John Lafferty. 2001. A study of smoothing methods for language
+ * models applied to Ad Hoc information retrieval. In Proceedings of the 24th
+ * annual international ACM SIGIR conference on Research and development in
+ * information retrieval (SIGIR '01). ACM, New York, NY, USA, 334-342.
+ * <p>The model has a single parameter, &lambda;. According to said paper, the
+ * optimal value depends on both the collection and the query. The optimal value
+ * is around {@code 0.1} for title queries and {@code 0.7} for long queries.</p>
+ *
+ * @lucene.experimental
+ */
+public class LMJelinekMercerSimilarity extends LMSimilarity {
+  /** The &lambda; parameter. */
+  private final float lambda;
+  
+  /** @param lambda the &lambda; parameter. */
+  public LMJelinekMercerSimilarity(
+      CollectionModel collectionModel, float lambda) {
+    super(collectionModel);
+    this.lambda = lambda;
+  }
+
+  /** @param lambda the &lambda; parameter. */
+  public LMJelinekMercerSimilarity(float lambda) {
+    this.lambda = lambda;
+  }
+  
+  @Override
+  protected float score(EasyStats stats, float freq, float docLen) {
+    return stats.getTotalBoost() *
+        (float)Math.log(1 +
+            ((1 - lambda) * freq / docLen) /
+            (lambda * ((LMStats)stats).getCollectionProbability()));
+  }
+  
+  @Override
+  protected void explain(Explanation expl, EasyStats stats, int doc,
+      float freq, float docLen) {
+    if (stats.getTotalBoost() != 1.0f) {
+      expl.addDetail(new Explanation(stats.getTotalBoost(), "boost"));
+    }
+    expl.addDetail(new Explanation(lambda, "lambda"));
+    super.explain(expl, stats, doc, freq, docLen);
+  }
+
+  /** Returns the &lambda; parameter. */
+  public float getLambda() {
+    return lambda;
+  }
+
+  @Override
+  public String getName() {
+    return String.format("Jelinek-Mercer(%f)", getLambda());
+  }
+}