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 2016/07/07 08:04:16 UTC

[3/3] lucene-solr:master: LUCENE-7369: Similarity.coord and BooleanQuery.disableCoord are removed.

LUCENE-7369: Similarity.coord and BooleanQuery.disableCoord are removed.


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

Branch: refs/heads/master
Commit: f1528bf338f42ef0e10f6cf1333286eb05139acd
Parents: 24d6b78
Author: Adrien Grand <jp...@gmail.com>
Authored: Thu Jun 30 19:04:11 2016 +0200
Committer: Adrien Grand <jp...@gmail.com>
Committed: Thu Jul 7 10:02:29 2016 +0200

----------------------------------------------------------------------
 lucene/CHANGES.txt                              |   3 +
 lucene/MIGRATE.txt                              |  11 +
 .../apache/lucene/search/BlendedTermQuery.java  |   6 +-
 .../org/apache/lucene/search/BooleanQuery.java  |  36 +-
 .../org/apache/lucene/search/BooleanScorer.java |  36 +-
 .../lucene/search/BooleanTopLevelScorers.java   | 182 ----
 .../org/apache/lucene/search/BooleanWeight.java | 153 +---
 .../apache/lucene/search/ConjunctionScorer.java |  10 +-
 .../lucene/search/DisjunctionSumScorer.java     |  11 +-
 .../lucene/search/MinShouldMatchSumScorer.java  |   6 +-
 .../apache/lucene/search/MultiPhraseQuery.java  |   1 -
 .../apache/lucene/search/MultiTermQuery.java    |   8 +-
 .../apache/lucene/search/ScoringRewrite.java    |   4 +-
 .../org/apache/lucene/search/package-info.java  |   3 +-
 .../search/similarities/ClassicSimilarity.java  |   6 -
 .../lucene/search/similarities/Similarity.java  |  14 -
 .../search/similarities/TFIDFSimilarity.java    |  34 +-
 .../org/apache/lucene/util/QueryBuilder.java    |   1 -
 .../apache/lucene/index/TestCustomNorms.java    |   5 -
 .../lucene/index/TestMaxTermFrequency.java      |   1 -
 .../test/org/apache/lucene/index/TestNorms.java |   6 -
 .../org/apache/lucene/index/TestOmitTf.java     |   2 -
 .../org/apache/lucene/search/TestBoolean2.java  |  38 -
 .../apache/lucene/search/TestBooleanCoord.java  | 860 -------------------
 .../search/TestBooleanMinShouldMatch.java       |  58 +-
 .../org/apache/lucene/search/TestBooleanOr.java |   2 +-
 .../apache/lucene/search/TestBooleanQuery.java  |  37 +-
 .../search/TestBooleanQueryVisitSubscorers.java |   5 +-
 .../lucene/search/TestBooleanRewrites.java      |  14 -
 .../apache/lucene/search/TestBooleanScorer.java |  15 +-
 .../lucene/search/TestDocValuesScoring.java     |   5 -
 .../lucene/search/TestMinShouldMatch2.java      |   2 +-
 .../search/TestMultiTermConstantScore.java      |   2 -
 .../apache/lucene/search/TestSimilarity.java    |   2 -
 .../lucene/search/TestSimilarityProvider.java   |  10 -
 .../lucene/search/TestSimpleExplanations.java   |  14 -
 .../search/similarities/TestSimilarity2.java    |   5 -
 .../org/apache/lucene/facet/DrillDownQuery.java |   1 -
 .../apache/lucene/facet/TestDrillSideways.java  |   1 -
 .../apache/lucene/queries/CommonTermsQuery.java |  47 +-
 .../lucene/queries/mlt/MoreLikeThisQuery.java   |   1 -
 .../lucene/queries/CommonTermsQueryTest.java    |  10 +-
 .../lucene/queries/TestCustomScoreQuery.java    |   1 -
 .../function/TestLongNormValueSource.java       |   6 -
 .../queries/payloads/TestPayloadScoreQuery.java |   5 -
 .../queries/payloads/TestPayloadTermQuery.java  |   5 -
 .../classic/MultiFieldQueryParser.java          |   1 -
 .../StandardBooleanQueryNodeBuilder.java        | 109 ---
 .../builders/StandardQueryTreeBuilder.java      |   6 +-
 .../builders/SynonymQueryNodeBuilder.java       |  48 ++
 .../nodes/StandardBooleanQueryNode.java         |  49 --
 .../standard/nodes/SynonymQueryNode.java        |  30 +
 .../processors/AnalyzerQueryNodeProcessor.java  |   8 +-
 .../queryparser/simple/SimpleQueryParser.java   |   5 -
 .../xml/builders/BooleanQueryBuilder.java       |   1 -
 .../xml/builders/TermsQueryBuilder.java         |   1 -
 .../classic/TestMultiFieldQueryParser.java      |   1 -
 .../flexible/standard/TestStandardQP.java       |   1 -
 .../simple/TestSimpleQueryParser.java           |   3 -
 .../org/apache/lucene/document/LatLonPoint.java |   1 -
 .../sandbox/queries/FuzzyLikeThisQuery.java     |   1 -
 .../lucene/search/BaseExplanationTestCase.java  |   6 +-
 .../search/similarities/RandomSimilarity.java   |  23 +-
 .../solr/handler/MoreLikeThisHandler.java       |   1 -
 .../component/QueryElevationComponent.java      |   1 -
 .../java/org/apache/solr/schema/LatLonType.java |   2 -
 .../java/org/apache/solr/schema/PointType.java  |   2 -
 .../org/apache/solr/search/DisMaxQParser.java   |   1 -
 .../solr/search/ExtendedDismaxQParser.java      |  16 +-
 .../org/apache/solr/search/QueryParsing.java    |   5 +-
 .../java/org/apache/solr/search/QueryUtils.java |   2 -
 .../apache/solr/search/SimpleQParserPlugin.java |   2 -
 .../apache/solr/search/TermsQParserPlugin.java  |   1 -
 .../apache/solr/search/mlt/CloudMLTQParser.java |   2 -
 .../solr/search/mlt/SimpleMLTQParser.java       |   2 -
 .../similarities/SchemaSimilarityFactory.java   |   2 +-
 .../org/apache/solr/util/SolrPluginUtils.java   |   1 -
 .../solr/search/mlt/CloudMLTQParserTest.java    |   4 +-
 78 files changed, 190 insertions(+), 1833 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/f1528bf3/lucene/CHANGES.txt
----------------------------------------------------------------------
diff --git a/lucene/CHANGES.txt b/lucene/CHANGES.txt
index 24d9f65..f609e83 100644
--- a/lucene/CHANGES.txt
+++ b/lucene/CHANGES.txt
@@ -10,6 +10,9 @@ API Changes
 * LUCENE-2605: Classic QueryParser no longer splits on whitespace by default.
   Use setSplitOnWhitespace(true) to get the old behavior.  (Steve Rowe)
 
+* LUCENE-7369: Similarity.coord and BooleanQuery.disableCoord are removed.
+  (Adrien Grand)
+
 Bug Fixes
 
 Improvements

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/f1528bf3/lucene/MIGRATE.txt
----------------------------------------------------------------------
diff --git a/lucene/MIGRATE.txt b/lucene/MIGRATE.txt
index 2c9bc9c..9b24ecb 100644
--- a/lucene/MIGRATE.txt
+++ b/lucene/MIGRATE.txt
@@ -18,3 +18,14 @@ classes from the java.util.zip package.
 
 Clients wishing to render Explanations as HTML should implement their own
 utilities for this.
+
+## Similarity.coord and BooleanQuery.disableCoord removed (LUCENE-7369)
+
+Coordination factors were a workaround for the fact that the ClassicSimilarity
+does not have strong enough term frequency saturation. This causes disjunctions
+to get better scores on documents that have many occurrences of a few query
+terms than on documents that match most clauses, which is most of time
+undesirable. The new BM25Similarity does not suffer from this problem since it
+has better saturation for the contribution of the term frequency so the coord
+factors have been removed from scores. Things now work as if coords were always
+disabled when constructing boolean queries.

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/f1528bf3/lucene/core/src/java/org/apache/lucene/search/BlendedTermQuery.java
----------------------------------------------------------------------
diff --git a/lucene/core/src/java/org/apache/lucene/search/BlendedTermQuery.java b/lucene/core/src/java/org/apache/lucene/search/BlendedTermQuery.java
index 82aea6d..85b8b0a 100644
--- a/lucene/core/src/java/org/apache/lucene/search/BlendedTermQuery.java
+++ b/lucene/core/src/java/org/apache/lucene/search/BlendedTermQuery.java
@@ -125,16 +125,14 @@ public final class BlendedTermQuery extends Query {
   }
 
   /**
-   * A {@link RewriteMethod} that adds all sub queries to a {@link BooleanQuery}
-   * which has {@link BooleanQuery#isCoordDisabled() coords disabled}. This
-   * {@link RewriteMethod} is useful when matching on several fields is
+   * A {@link RewriteMethod} that adds all sub queries to a {@link BooleanQuery}.
+   * This {@link RewriteMethod} is useful when matching on several fields is
    * considered better than having a good match on a single field.
    */
   public static final RewriteMethod BOOLEAN_REWRITE = new RewriteMethod() {
     @Override
     public Query rewrite(Query[] subQueries) {
       BooleanQuery.Builder merged = new BooleanQuery.Builder();
-      merged.setDisableCoord(true);
       for (Query query : subQueries) {
         merged.add(query, Occur.SHOULD);
       }

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/f1528bf3/lucene/core/src/java/org/apache/lucene/search/BooleanQuery.java
----------------------------------------------------------------------
diff --git a/lucene/core/src/java/org/apache/lucene/search/BooleanQuery.java b/lucene/core/src/java/org/apache/lucene/search/BooleanQuery.java
index 3ac11c8..b48f801 100644
--- a/lucene/core/src/java/org/apache/lucene/search/BooleanQuery.java
+++ b/lucene/core/src/java/org/apache/lucene/search/BooleanQuery.java
@@ -32,7 +32,6 @@ import java.util.Set;
 
 import org.apache.lucene.index.IndexReader;
 import org.apache.lucene.search.BooleanClause.Occur;
-import org.apache.lucene.search.similarities.Similarity;
 
 /** A Query that matches documents matching boolean combinations of other
   * queries, e.g. {@link TermQuery}s, {@link PhraseQuery}s or other
@@ -74,7 +73,6 @@ public class BooleanQuery extends Query implements Iterable<BooleanClause> {
   /** A builder for boolean queries. */
   public static class Builder {
 
-    private boolean disableCoord;
     private int minimumNumberShouldMatch;
     private final List<BooleanClause> clauses = new ArrayList<>();
 
@@ -82,17 +80,6 @@ public class BooleanQuery extends Query implements Iterable<BooleanClause> {
     public Builder() {}
 
     /**
-     * {@link Similarity#coord(int,int)} may be disabled in scoring, as
-     * appropriate. For example, this score factor does not make sense for most
-     * automatically generated queries, like {@link WildcardQuery} and {@link
-     * FuzzyQuery}.
-     */
-    public Builder setDisableCoord(boolean disableCoord) {
-      this.disableCoord = disableCoord;
-      return this;
-    }
-
-    /**
      * Specifies a minimum number of the optional BooleanClauses
      * which must be satisfied.
      *
@@ -142,19 +129,17 @@ public class BooleanQuery extends Query implements Iterable<BooleanClause> {
     /** Create a new {@link BooleanQuery} based on the parameters that have
      *  been set on this builder. */
     public BooleanQuery build() {
-      return new BooleanQuery(disableCoord, minimumNumberShouldMatch, clauses.toArray(new BooleanClause[0]));
+      return new BooleanQuery(minimumNumberShouldMatch, clauses.toArray(new BooleanClause[0]));
     }
 
   }
 
-  private final boolean disableCoord;
   private final int minimumNumberShouldMatch;
   private final List<BooleanClause> clauses;              // used for toString() and getClauses()
   private final Map<Occur, Collection<Query>> clauseSets; // used for equals/hashcode
 
-  private BooleanQuery(boolean disableCoord, int minimumNumberShouldMatch,
+  private BooleanQuery(int minimumNumberShouldMatch,
       BooleanClause[] clauses) {
-    this.disableCoord = disableCoord;
     this.minimumNumberShouldMatch = minimumNumberShouldMatch;
     this.clauses = Collections.unmodifiableList(Arrays.asList(clauses));
     clauseSets = new EnumMap<>(Occur.class);
@@ -170,13 +155,6 @@ public class BooleanQuery extends Query implements Iterable<BooleanClause> {
   }
 
   /**
-   * Return whether the coord factor is disabled.
-   */
-  public boolean isCoordDisabled() {
-    return disableCoord;
-  }
-
-  /**
    * Gets the minimum number of the optional BooleanClauses
    * which must be satisfied.
    */
@@ -223,7 +201,7 @@ public class BooleanQuery extends Query implements Iterable<BooleanClause> {
     if (needsScores == false) {
       query = rewriteNoScoring();
     }
-    return new BooleanWeight(query, searcher, needsScores, disableCoord);
+    return new BooleanWeight(query, searcher, needsScores);
   }
 
   @Override
@@ -258,7 +236,6 @@ public class BooleanQuery extends Query implements Iterable<BooleanClause> {
     // recursively rewrite
     {
       BooleanQuery.Builder builder = new BooleanQuery.Builder();
-      builder.setDisableCoord(isCoordDisabled());
       builder.setMinimumNumberShouldMatch(getMinimumNumberShouldMatch());
       boolean actuallyRewritten = false;
       for (BooleanClause clause : this) {
@@ -284,7 +261,6 @@ public class BooleanQuery extends Query implements Iterable<BooleanClause> {
         // since clauseSets implicitly deduplicates FILTER and MUST_NOT
         // clauses, this means there were duplicates
         BooleanQuery.Builder rewritten = new BooleanQuery.Builder();
-        rewritten.setDisableCoord(disableCoord);
         rewritten.setMinimumNumberShouldMatch(minimumNumberShouldMatch);
         for (Map.Entry<Occur, Collection<Query>> entry : clauseSets.entrySet()) {
           final Occur occur = entry.getKey();
@@ -304,7 +280,6 @@ public class BooleanQuery extends Query implements Iterable<BooleanClause> {
       modified |= filters.removeAll(clauseSets.get(Occur.MUST));
       if (modified) {
         BooleanQuery.Builder builder = new BooleanQuery.Builder();
-        builder.setDisableCoord(isCoordDisabled());
         builder.setMinimumNumberShouldMatch(getMinimumNumberShouldMatch());
         for (BooleanClause clause : clauses) {
           if (clause.getOccur() != Occur.FILTER) {
@@ -355,7 +330,6 @@ public class BooleanQuery extends Query implements Iterable<BooleanClause> {
 
           // now add back the SHOULD clauses
           builder = new BooleanQuery.Builder()
-            .setDisableCoord(isCoordDisabled())
             .setMinimumNumberShouldMatch(getMinimumNumberShouldMatch())
             .add(rewritten, Occur.MUST);
           for (Query query : clauseSets.get(Occur.SHOULD)) {
@@ -414,7 +388,6 @@ public class BooleanQuery extends Query implements Iterable<BooleanClause> {
    * Compares the specified object with this boolean query for equality.
    * Returns true if and only if the provided object<ul>
    * <li>is also a {@link BooleanQuery},</li>
-   * <li>has the same value of {@link #isCoordDisabled()}</li>
    * <li>has the same value of {@link #getMinimumNumberShouldMatch()}</li>
    * <li>has the same {@link Occur#SHOULD} clauses, regardless of the order</li>
    * <li>has the same {@link Occur#MUST} clauses, regardless of the order</li>
@@ -431,12 +404,11 @@ public class BooleanQuery extends Query implements Iterable<BooleanClause> {
 
   private boolean equalsTo(BooleanQuery other) {
     return getMinimumNumberShouldMatch() == other.getMinimumNumberShouldMatch() && 
-           disableCoord == other.disableCoord &&
            clauseSets.equals(other.clauseSets);
   }
 
   private int computeHashCode() {
-    int hashCode = Objects.hash(disableCoord, minimumNumberShouldMatch, clauseSets);
+    int hashCode = Objects.hash(minimumNumberShouldMatch, clauseSets);
     if (hashCode == 0) {
       hashCode = 1;
     }

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/f1528bf3/lucene/core/src/java/org/apache/lucene/search/BooleanScorer.java
----------------------------------------------------------------------
diff --git a/lucene/core/src/java/org/apache/lucene/search/BooleanScorer.java b/lucene/core/src/java/org/apache/lucene/search/BooleanScorer.java
index 73880a8..a264dca 100644
--- a/lucene/core/src/java/org/apache/lucene/search/BooleanScorer.java
+++ b/lucene/core/src/java/org/apache/lucene/search/BooleanScorer.java
@@ -117,7 +117,6 @@ final class BooleanScorer extends BulkScorer {
   // This is basically an inlined FixedBitSet... seems to help with bound checks
   final long[] matching = new long[SET_SIZE];
 
-  final float[] coordFactors;
   final BulkScorerAndDoc[] leads;
   final HeadPriorityQueue head;
   final TailPriorityQueue tail;
@@ -146,7 +145,7 @@ final class BooleanScorer extends BulkScorer {
 
   final OrCollector orCollector = new OrCollector();
 
-  BooleanScorer(BooleanWeight weight, boolean disableCoord, int maxCoord, Collection<BulkScorer> scorers, int minShouldMatch, boolean needsScores) {
+  BooleanScorer(BooleanWeight weight, Collection<BulkScorer> scorers, int minShouldMatch, boolean needsScores) {
     if (minShouldMatch < 1 || minShouldMatch > scorers.size()) {
       throw new IllegalArgumentException("minShouldMatch should be within 1..num_scorers. Got " + minShouldMatch);
     }
@@ -172,11 +171,6 @@ final class BooleanScorer extends BulkScorer {
       }
     }
     this.cost = cost(scorers, minShouldMatch);
-
-    coordFactors = new float[scorers.size() + 1];
-    for (int i = 0; i < coordFactors.length; i++) {
-      coordFactors[i] = disableCoord ? 1.0f : weight.coord(i, maxCoord);
-    }
   }
 
   @Override
@@ -189,7 +183,7 @@ final class BooleanScorer extends BulkScorer {
     final Bucket bucket = buckets[i];
     if (bucket.freq >= minShouldMatch) {
       fakeScorer.freq = bucket.freq;
-      fakeScorer.score = (float) bucket.score * coordFactors[bucket.freq];
+      fakeScorer.score = (float) bucket.score;
       final int doc = base | i;
       fakeScorer.doc = doc;
       collector.collect(doc);
@@ -275,20 +269,20 @@ final class BooleanScorer extends BulkScorer {
     }
   }
 
-  private void scoreWindowSingleScorer(BulkScorerAndDoc bulkScorer, LeafCollector collector, LeafCollector singleClauseCollector,
+  private void scoreWindowSingleScorer(BulkScorerAndDoc bulkScorer, LeafCollector collector,
       Bits acceptDocs, int windowMin, int windowMax, int max) throws IOException {
     assert tail.size() == 0;
     final int nextWindowBase = head.top().next & ~MASK;
     final int end = Math.max(windowMax, Math.min(max, nextWindowBase));
-    
-    bulkScorer.score(singleClauseCollector, acceptDocs, windowMin, end);
-    
+
+    bulkScorer.score(collector, acceptDocs, windowMin, end);
+
     // reset the scorer that should be used for the general case
     collector.setScorer(fakeScorer);
   }
 
   private BulkScorerAndDoc scoreWindow(BulkScorerAndDoc top, LeafCollector collector,
-      LeafCollector singleClauseCollector, Bits acceptDocs, int min, int max) throws IOException {
+      Bits acceptDocs, int min, int max) throws IOException {
     final int windowBase = top.next & ~MASK; // find the window that the next match belongs to
     final int windowMin = Math.max(min, windowBase);
     final int windowMax = Math.min(max, windowBase + SIZE);
@@ -304,7 +298,7 @@ final class BooleanScorer extends BulkScorer {
       // special case: only one scorer can match in the current window,
       // we can collect directly
       final BulkScorerAndDoc bulkScorer = leads[0];
-      scoreWindowSingleScorer(bulkScorer, collector, singleClauseCollector, acceptDocs, windowMin, windowMax, max);
+      scoreWindowSingleScorer(bulkScorer, collector, acceptDocs, windowMin, windowMax, max);
       return head.add(bulkScorer);
     } else {
       // general case, collect through a bit set first and then replay
@@ -318,21 +312,9 @@ final class BooleanScorer extends BulkScorer {
     fakeScorer.doc = -1;
     collector.setScorer(fakeScorer);
 
-    final LeafCollector singleClauseCollector;
-    if (coordFactors[1] == 1f) {
-      singleClauseCollector = collector;
-    } else {
-      singleClauseCollector = new FilterLeafCollector(collector) {
-        @Override
-        public void setScorer(Scorer scorer) throws IOException {
-          super.setScorer(new BooleanTopLevelScorers.BoostedScorer(scorer, coordFactors[1]));
-        }
-      };
-    }
-
     BulkScorerAndDoc top = advance(min);
     while (top.next < max) {
-      top = scoreWindow(top, collector, singleClauseCollector, acceptDocs, min, max);
+      top = scoreWindow(top, collector, acceptDocs, min, max);
     }
 
     return top.next;

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/f1528bf3/lucene/core/src/java/org/apache/lucene/search/BooleanTopLevelScorers.java
----------------------------------------------------------------------
diff --git a/lucene/core/src/java/org/apache/lucene/search/BooleanTopLevelScorers.java b/lucene/core/src/java/org/apache/lucene/search/BooleanTopLevelScorers.java
deleted file mode 100644
index d34e708..0000000
--- a/lucene/core/src/java/org/apache/lucene/search/BooleanTopLevelScorers.java
+++ /dev/null
@@ -1,182 +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 java.util.Arrays;
-import java.util.Collection;
-import java.util.Collections;
-
-import org.apache.lucene.util.Bits;
-
-/** Internal document-at-a-time scorers used to deal with stupid coord() computation */
-class BooleanTopLevelScorers {
-  
-  /** 
-   * Used when there is more than one scorer in a query, but a segment
-   * only had one non-null scorer. This just wraps that scorer directly
-   * to factor in coord().
-   */
-  static class BoostedScorer extends FilterScorer {
-    final float boost;
-    
-    BoostedScorer(Scorer in, float boost) {
-      super(in);
-      this.boost = boost;
-    }
-
-    @Override
-    public float score() throws IOException {
-      return in.score() * boost;
-    }
-
-    @Override
-    public Collection<ChildScorer> getChildren() {
-      return Collections.singleton(new ChildScorer(in, "BOOSTED"));
-    }
-  }
-
-  /**
-   * Used when there is more than one scorer in a query, but a segment
-   * only had one non-null scorer.
-   */
-  static class BoostedBulkScorer extends BulkScorer {
-
-    final BulkScorer in;
-    final float boost;
-
-    BoostedBulkScorer(BulkScorer scorer, float boost) {
-      this.in = scorer;
-      this.boost = boost;
-    }
-
-    @Override
-    public int score(LeafCollector collector, Bits acceptDocs, int min, int max) throws IOException {
-      final LeafCollector wrapped = new FilterLeafCollector(collector) {
-        @Override
-        public void setScorer(Scorer scorer) throws IOException {
-          super.setScorer(new BoostedScorer(scorer, boost));
-        }
-      };
-      return in.score(wrapped, acceptDocs, min, max);
-    }
-
-    @Override
-    public long cost() {
-      return in.cost();
-    }
-
-  }
-
-  /** 
-   * Used when there are both mandatory and optional clauses, but minShouldMatch
-   * dictates that some of the optional clauses must match. The query is a conjunction,
-   * but must compute coord based on how many optional subscorers matched (freq).
-   */
-  static class CoordinatingConjunctionScorer extends ConjunctionScorer {
-    private final float coords[];
-    private final int reqCount;
-    private final Scorer req;
-    private final Scorer opt;
-    
-    CoordinatingConjunctionScorer(Weight weight, float coords[], Scorer req, int reqCount, Scorer opt) {
-      super(weight, Arrays.asList(req, opt), Arrays.asList(req, opt));
-      this.coords = coords;
-      this.req = req;
-      this.reqCount = reqCount;
-      this.opt = opt;
-    }
-    
-    @Override
-    public float score() throws IOException {
-      return (req.score() + opt.score()) * coords[reqCount + opt.freq()];
-    }
-  }
-  
-  /** 
-   * Used when there are mandatory clauses with one optional clause: we compute
-   * coord based on whether the optional clause matched or not.
-   */
-  static class ReqSingleOptScorer extends ReqOptSumScorer {
-    // coord factor if just the required part matches
-    private final float coordReq;
-    // coord factor if both required and optional part matches 
-    private final float coordBoth;
-    
-    public ReqSingleOptScorer(Scorer reqScorer, Scorer optScorer, float coordReq, float coordBoth) {
-      super(reqScorer, optScorer);
-      this.coordReq = coordReq;
-      this.coordBoth = coordBoth;
-    }
-
-    @Override
-    public float score() throws IOException {
-      // TODO: sum into a double and cast to float if we ever send required clauses to BS1
-      int curDoc = reqScorer.docID();
-      float score = reqScorer.score();
-
-      int optScorerDoc = optIterator.docID();
-      if (optScorerDoc < curDoc) {
-        optScorerDoc = optIterator.advance(curDoc);
-      }
-      
-      if (optScorerDoc == curDoc) {
-        score = (score + optScorer.score()) * coordBoth;
-      } else {
-        score = score * coordReq;
-      }
-      
-      return score;
-    }
-  }
-
-  /** 
-   * Used when there are mandatory clauses with optional clauses: we compute
-   * coord based on how many optional subscorers matched (freq).
-   */
-  static class ReqMultiOptScorer extends ReqOptSumScorer {
-    private final int requiredCount;
-    private final float coords[];
-    
-    public ReqMultiOptScorer(Scorer reqScorer, Scorer optScorer, int requiredCount, float coords[]) {
-      super(reqScorer, optScorer);
-      this.requiredCount = requiredCount;
-      this.coords = coords;
-    }
-
-    @Override
-    public float score() throws IOException {
-      // TODO: sum into a double and cast to float if we ever send required clauses to BS1
-      int curDoc = reqScorer.docID();
-      float score = reqScorer.score();
-
-      int optScorerDoc = optIterator.docID();
-      if (optScorerDoc < curDoc) {
-        optScorerDoc = optIterator.advance(curDoc);
-      }
-      
-      if (optScorerDoc == curDoc) {
-        score = (score + optScorer.score()) * coords[requiredCount + optScorer.freq()];
-      } else {
-        score = score * coords[requiredCount];
-      }
-      
-      return score;
-    }
-  }
-}

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/f1528bf3/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 24cbac8..f55a6b7 100644
--- a/lucene/core/src/java/org/apache/lucene/search/BooleanWeight.java
+++ b/lucene/core/src/java/org/apache/lucene/search/BooleanWeight.java
@@ -40,44 +40,17 @@ final class BooleanWeight extends Weight {
   final BooleanQuery query;
   
   final ArrayList<Weight> weights;
-  final int maxCoord;  // num optional + num required
-  final boolean disableCoord;
   final boolean needsScores;
-  final float coords[];
 
-  BooleanWeight(BooleanQuery query, IndexSearcher searcher, boolean needsScores, boolean disableCoord) throws IOException {
+  BooleanWeight(BooleanQuery query, IndexSearcher searcher, boolean needsScores) throws IOException {
     super(query);
     this.query = query;
     this.needsScores = needsScores;
     this.similarity = searcher.getSimilarity(needsScores);
     weights = new ArrayList<>();
-    int i = 0;
-    int maxCoord = 0;
     for (BooleanClause c : query) {
       Weight w = searcher.createWeight(c.getQuery(), needsScores && c.isScoring());
       weights.add(w);
-      if (c.isScoring()) {
-        maxCoord++;
-      }
-      i += 1;
-    }
-    this.maxCoord = maxCoord;
-    
-    // precompute coords (0..N, N).
-    // set disableCoord when its explicit, scores are not needed, no scoring clauses, or the sim doesn't use it.
-    coords = new float[maxCoord+1];
-    Arrays.fill(coords, 1F);
-    coords[0] = 0f;
-    if (maxCoord > 0 && needsScores && disableCoord == false) {
-      // compute coords from the similarity, look for any actual ones.
-      boolean seenActualCoord = false;
-      for (i = 1; i < coords.length; i++) {
-        coords[i] = coord(i, maxCoord);
-        seenActualCoord |= (coords[i] != 1F);
-      }
-      this.disableCoord = seenActualCoord == false;
-    } else {
-      this.disableCoord = true;
     }
   }
 
@@ -109,22 +82,6 @@ final class BooleanWeight extends Weight {
     return sum ;
   }
 
-  public float coord(int overlap, int maxOverlap) {
-    if (overlap == 0) {
-      // special case that there are only non-scoring clauses
-      return 0F;
-    } else if (maxOverlap == 1) {
-      // LUCENE-4300: in most cases of maxOverlap=1, BQ rewrites itself away,
-      // so coord() is not applied. But when BQ cannot optimize itself away
-      // for a single clause (minNrShouldMatch, prohibited clauses, etc), it's
-      // important not to apply coord(1,1) for consistency, it might not be 1.0F
-      return 1F;
-    } else {
-      // common case: use the similarity to compute the coord
-      return similarity.coord(overlap, maxOverlap);
-    }
-  }
-
   @Override
   public void normalize(float norm, float boost) {
     for (Weight w : weights) {
@@ -137,7 +94,6 @@ final class BooleanWeight extends Weight {
   public Explanation explain(LeafReaderContext context, int doc) throws IOException {
     final int minShouldMatch = query.getMinimumNumberShouldMatch();
     List<Explanation> subs = new ArrayList<>();
-    int coord = 0;
     float sum = 0.0f;
     boolean fail = false;
     int matchCount = 0;
@@ -151,7 +107,6 @@ final class BooleanWeight extends Weight {
         if (c.isScoring()) {
           subs.add(e);
           sum += e.getValue();
-          coord++;
         } else if (c.isRequired()) {
           subs.add(Explanation.match(0f, "match on required clause, product of:",
               Explanation.match(0f, Occur.FILTER + " clause"), e));
@@ -178,13 +133,7 @@ final class BooleanWeight extends Weight {
       return Explanation.noMatch("Failure to match minimum number of optional clauses: " + minShouldMatch, subs);
     } else {
       // we have a match
-      Explanation result = Explanation.match(sum, "sum of:", subs);
-      final float coordFactor = disableCoord ? 1.0f : coord(coord, maxCoord);
-      if (coordFactor != 1f) {
-        result = Explanation.match(sum * coordFactor, "product of:",
-            result, Explanation.match(coordFactor, "coord("+coord+"/"+maxCoord+")"));
-      }
-      return result;
+      return Explanation.match(sum, "sum of:", subs);
     }
   }
 
@@ -244,15 +193,10 @@ final class BooleanWeight extends Weight {
     }
 
     if (optional.size() == 1) {
-      BulkScorer opt = optional.get(0);
-      if (!disableCoord && maxCoord > 1) {
-        return new BooleanTopLevelScorers.BoostedBulkScorer(opt, coord(1, maxCoord));
-      } else {
-        return opt;
-      }
+      return optional.get(0);
     }
 
-    return new BooleanScorer(this, disableCoord, maxCoord, optional, Math.max(1, query.getMinimumNumberShouldMatch()), needsScores);
+    return new BooleanScorer(this, optional, Math.max(1, query.getMinimumNumberShouldMatch()), needsScores);
   }
 
   // Return a BulkScorer for the required clauses only,
@@ -275,12 +219,8 @@ final class BooleanWeight extends Weight {
         // no matches
         return null;
       }
-      if (c.isScoring() == false) {
-        if (needsScores) {
-          scorer = disableScoring(scorer);
-        }
-      } else {
-        assert maxCoord == 1;
+      if (c.isScoring() == false && needsScores) {
+        scorer = disableScoring(scorer);
       }
     }
     return scorer;
@@ -350,7 +290,7 @@ final class BooleanWeight extends Weight {
     if (prohibited.isEmpty()) {
       return positiveScorer;
     } else {
-      Scorer prohibitedScorer = opt(prohibited, 1, true);
+      Scorer prohibitedScorer = opt(prohibited, 1);
       if (prohibitedScorer.twoPhaseIterator() != null) {
         // ReqExclBulkScorer can't deal efficiently with two-phased prohibited clauses
         return null;
@@ -432,51 +372,33 @@ final class BooleanWeight extends Weight {
     
     // pure conjunction
     if (optional.isEmpty()) {
-      return excl(req(required, requiredScoring, disableCoord), prohibited);
+      return excl(req(required, requiredScoring), prohibited);
     }
     
     // pure disjunction
     if (required.isEmpty()) {
-      return excl(opt(optional, minShouldMatch, disableCoord), prohibited);
+      return excl(opt(optional, minShouldMatch), prohibited);
     }
     
     // conjunction-disjunction mix:
-    // we create the required and optional pieces with coord disabled, and then
+    // we create the required and optional pieces, and then
     // combine the two: if minNrShouldMatch > 0, then it's a conjunction: because the
-    // optional side must match. otherwise it's required + optional, factoring the
-    // number of optional terms into the coord calculation
+    // optional side must match. otherwise it's required + optional
     
-    Scorer req = excl(req(required, requiredScoring, true), prohibited);
-    Scorer opt = opt(optional, minShouldMatch, true);
+    Scorer req = excl(req(required, requiredScoring), prohibited);
+    Scorer opt = opt(optional, minShouldMatch);
 
-    // TODO: clean this up: it's horrible
-    if (disableCoord) {
-      if (minShouldMatch > 0) {
-        return new ConjunctionScorer(this, Arrays.asList(req, opt), Arrays.asList(req, opt), 1F);
-      } else {
-        return new ReqOptSumScorer(req, opt);          
-      }
-    } else if (optional.size() == 1) {
-      if (minShouldMatch > 0) {
-        return new ConjunctionScorer(this, Arrays.asList(req, opt), Arrays.asList(req, opt), coord(requiredScoring.size()+1, maxCoord));
-      } else {
-        float coordReq = coord(requiredScoring.size(), maxCoord);
-        float coordBoth = coord(requiredScoring.size() + 1, maxCoord);
-        return new BooleanTopLevelScorers.ReqSingleOptScorer(req, opt, coordReq, coordBoth);
-      }
+    if (minShouldMatch > 0) {
+      return new ConjunctionScorer(this, Arrays.asList(req, opt), Arrays.asList(req, opt));
     } else {
-      if (minShouldMatch > 0) {
-        return new BooleanTopLevelScorers.CoordinatingConjunctionScorer(this, coords, req, requiredScoring.size(), opt);
-      } else {
-        return new BooleanTopLevelScorers.ReqMultiOptScorer(req, opt, requiredScoring.size(), coords); 
-      }
+      return new ReqOptSumScorer(req, opt);          
     }
   }
 
   /** Create a new scorer for the given required clauses. Note that
    *  {@code requiredScoring} is a subset of {@code required} containing
    *  required clauses that should participate in scoring. */
-  private Scorer req(List<Scorer> required, List<Scorer> requiredScoring, boolean disableCoord) {
+  private Scorer req(List<Scorer> required, List<Scorer> requiredScoring) {
     if (required.size() == 1) {
       Scorer req = required.get(0);
 
@@ -500,17 +422,9 @@ final class BooleanWeight extends Weight {
         };
       }
       
-      float boost = 1f;
-      if (disableCoord == false) {
-        boost = coord(1, maxCoord);
-      }
-      if (boost == 1f) {
-        return req;
-      }
-      return new BooleanTopLevelScorers.BoostedScorer(req, boost);
+      return req;
     } else {
-      return new ConjunctionScorer(this, required, requiredScoring,
-                                   disableCoord ? 1.0F : coord(requiredScoring.size(), maxCoord));
+      return new ConjunctionScorer(this, required, requiredScoring);
     }
   }
   
@@ -520,34 +434,17 @@ final class BooleanWeight extends Weight {
     } else if (prohibited.size() == 1) {
       return new ReqExclScorer(main, prohibited.get(0));
     } else {
-      float coords[] = new float[prohibited.size()+1];
-      Arrays.fill(coords, 1F);
-      return new ReqExclScorer(main, new DisjunctionSumScorer(this, prohibited, coords, false));
+      return new ReqExclScorer(main, new DisjunctionSumScorer(this, prohibited, false));
     }
   }
   
-  private Scorer opt(List<Scorer> optional, int minShouldMatch, boolean disableCoord) throws IOException {
+  private Scorer opt(List<Scorer> optional, int minShouldMatch) throws IOException {
     if (optional.size() == 1) {
-      Scorer opt = optional.get(0);
-      if (!disableCoord && maxCoord > 1) {
-        return new BooleanTopLevelScorers.BoostedScorer(opt, coord(1, maxCoord));
-      } else {
-        return opt;
-      }
+      return optional.get(0);
+    } else if (minShouldMatch > 1) {
+      return new MinShouldMatchSumScorer(this, optional, minShouldMatch);
     } else {
-      float coords[];
-      if (disableCoord) {
-        // sneaky: when we do a mixed conjunction/disjunction, we need a fake for the disjunction part.
-        coords = new float[optional.size()+1];
-        Arrays.fill(coords, 1F);
-      } else {
-        coords = this.coords;
-      }
-      if (minShouldMatch > 1) {
-        return new MinShouldMatchSumScorer(this, optional, minShouldMatch, coords);
-      } else {
-        return new DisjunctionSumScorer(this, optional, coords, needsScores);
-      }
+      return new DisjunctionSumScorer(this, optional, needsScores);
     }
   }
 }

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/f1528bf3/lucene/core/src/java/org/apache/lucene/search/ConjunctionScorer.java
----------------------------------------------------------------------
diff --git a/lucene/core/src/java/org/apache/lucene/search/ConjunctionScorer.java b/lucene/core/src/java/org/apache/lucene/search/ConjunctionScorer.java
index ab1de92..0066952 100644
--- a/lucene/core/src/java/org/apache/lucene/search/ConjunctionScorer.java
+++ b/lucene/core/src/java/org/apache/lucene/search/ConjunctionScorer.java
@@ -27,17 +27,11 @@ class ConjunctionScorer extends Scorer {
 
   final DocIdSetIterator disi;
   final Scorer[] scorers;
-  final float coord;
-
-  ConjunctionScorer(Weight weight, List<Scorer> required, List<Scorer> scorers) {
-    this(weight, required, scorers, 1f);
-  }
 
   /** Create a new {@link ConjunctionScorer}, note that {@code scorers} must be a subset of {@code required}. */
-  ConjunctionScorer(Weight weight, List<Scorer> required, List<Scorer> scorers, float coord) {
+  ConjunctionScorer(Weight weight, List<Scorer> required, List<Scorer> scorers) {
     super(weight);
     assert required.containsAll(scorers);
-    this.coord = coord;
     this.disi = ConjunctionDISI.intersectScorers(required);
     this.scorers = scorers.toArray(new Scorer[scorers.size()]);
   }
@@ -63,7 +57,7 @@ class ConjunctionScorer extends Scorer {
     for (Scorer scorer : scorers) {
       sum += scorer.score();
     }
-    return coord * (float)sum;
+    return (float) sum;
   }
 
   @Override

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/f1528bf3/lucene/core/src/java/org/apache/lucene/search/DisjunctionSumScorer.java
----------------------------------------------------------------------
diff --git a/lucene/core/src/java/org/apache/lucene/search/DisjunctionSumScorer.java b/lucene/core/src/java/org/apache/lucene/search/DisjunctionSumScorer.java
index 5bf33a2..69858e3 100644
--- a/lucene/core/src/java/org/apache/lucene/search/DisjunctionSumScorer.java
+++ b/lucene/core/src/java/org/apache/lucene/search/DisjunctionSumScorer.java
@@ -22,27 +22,22 @@ import java.util.List;
 
 /** A Scorer for OR like queries, counterpart of <code>ConjunctionScorer</code>.
  */
-final class DisjunctionSumScorer extends DisjunctionScorer { 
-  private final float[] coord;
+final class DisjunctionSumScorer extends DisjunctionScorer {
   
   /** Construct a <code>DisjunctionScorer</code>.
    * @param weight The weight to be used.
    * @param subScorers Array of at least two subscorers.
-   * @param coord Table of coordination factors
    */
-  DisjunctionSumScorer(Weight weight, List<Scorer> subScorers, float[] coord, boolean needsScores) {
+  DisjunctionSumScorer(Weight weight, List<Scorer> subScorers, boolean needsScores) {
     super(weight, subScorers, needsScores);
-    this.coord = coord;
   }
 
   @Override
   protected float score(DisiWrapper topList) throws IOException {
     double score = 0;
-    int freq = 0;
     for (DisiWrapper w = topList; w != null; w = w.next) {
       score += w.scorer.score();
-      freq += 1;
     }
-    return (float)score * coord[freq];
+    return (float)score;
   }
 }

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/f1528bf3/lucene/core/src/java/org/apache/lucene/search/MinShouldMatchSumScorer.java
----------------------------------------------------------------------
diff --git a/lucene/core/src/java/org/apache/lucene/search/MinShouldMatchSumScorer.java b/lucene/core/src/java/org/apache/lucene/search/MinShouldMatchSumScorer.java
index 9653d95..032b5fe 100644
--- a/lucene/core/src/java/org/apache/lucene/search/MinShouldMatchSumScorer.java
+++ b/lucene/core/src/java/org/apache/lucene/search/MinShouldMatchSumScorer.java
@@ -78,7 +78,6 @@ final class MinShouldMatchSumScorer extends Scorer {
   }
 
   final int minShouldMatch;
-  final float[] coord;
 
   // list of scorers which 'lead' the iteration and are currently
   // positioned on 'doc'
@@ -98,7 +97,7 @@ final class MinShouldMatchSumScorer extends Scorer {
   final Collection<ChildScorer> childScorers;
   final long cost;
 
-  MinShouldMatchSumScorer(Weight weight, Collection<Scorer> scorers, int minShouldMatch, float[] coord) {
+  MinShouldMatchSumScorer(Weight weight, Collection<Scorer> scorers, int minShouldMatch) {
     super(weight);
 
     if (minShouldMatch > scorers.size()) {
@@ -109,7 +108,6 @@ final class MinShouldMatchSumScorer extends Scorer {
     }
 
     this.minShouldMatch = minShouldMatch;
-    this.coord = coord;
     this.doc = -1;
 
     head = new DisiPriorityQueue(scorers.size() - minShouldMatch + 1);
@@ -290,7 +288,7 @@ final class MinShouldMatchSumScorer extends Scorer {
     for (DisiWrapper s = lead; s != null; s = s.next) {
       score += s.scorer.score();
     }
-    return coord[freq] * (float) score;
+    return (float) score;
   }
 
   @Override

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/f1528bf3/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 cf4f39c..2d5c389 100644
--- a/lucene/core/src/java/org/apache/lucene/search/MultiPhraseQuery.java
+++ b/lucene/core/src/java/org/apache/lucene/search/MultiPhraseQuery.java
@@ -321,7 +321,6 @@ public class MultiPhraseQuery extends Query {
     } else if (termArrays.length == 1) {                 // optimize one-term case
       Term[] terms = termArrays[0];
       BooleanQuery.Builder builder = new BooleanQuery.Builder();
-      builder.setDisableCoord(true);
       for (Term term : terms) {
         builder.add(new TermQuery(term), BooleanClause.Occur.SHOULD);
       }

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/f1528bf3/lucene/core/src/java/org/apache/lucene/search/MultiTermQuery.java
----------------------------------------------------------------------
diff --git a/lucene/core/src/java/org/apache/lucene/search/MultiTermQuery.java b/lucene/core/src/java/org/apache/lucene/search/MultiTermQuery.java
index ef2b014..d0869d6 100644
--- a/lucene/core/src/java/org/apache/lucene/search/MultiTermQuery.java
+++ b/lucene/core/src/java/org/apache/lucene/search/MultiTermQuery.java
@@ -157,9 +157,7 @@ public abstract class MultiTermQuery extends Query {
     
     @Override
     protected BooleanQuery.Builder getTopLevelBuilder() {
-      BooleanQuery.Builder builder = new BooleanQuery.Builder();
-      builder.setDisableCoord(true);
-      return builder;
+      return new BooleanQuery.Builder();
     }
     
     @Override
@@ -255,9 +253,7 @@ public abstract class MultiTermQuery extends Query {
     
     @Override
     protected BooleanQuery.Builder getTopLevelBuilder() {
-      BooleanQuery.Builder builder = new BooleanQuery.Builder();
-      builder.setDisableCoord(true);
-      return builder;
+      return new BooleanQuery.Builder();
     }
     
     @Override

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/f1528bf3/lucene/core/src/java/org/apache/lucene/search/ScoringRewrite.java
----------------------------------------------------------------------
diff --git a/lucene/core/src/java/org/apache/lucene/search/ScoringRewrite.java b/lucene/core/src/java/org/apache/lucene/search/ScoringRewrite.java
index 38345b3..6f54a86 100644
--- a/lucene/core/src/java/org/apache/lucene/search/ScoringRewrite.java
+++ b/lucene/core/src/java/org/apache/lucene/search/ScoringRewrite.java
@@ -54,9 +54,7 @@ public abstract class ScoringRewrite<B> extends TermCollectingRewrite<B> {
   public final static ScoringRewrite<BooleanQuery.Builder> SCORING_BOOLEAN_REWRITE = new ScoringRewrite<BooleanQuery.Builder>() {
     @Override
     protected BooleanQuery.Builder getTopLevelBuilder() {
-      BooleanQuery.Builder builder = new BooleanQuery.Builder();
-      builder.setDisableCoord(true);
-      return builder;
+      return new BooleanQuery.Builder();
     }
 
     @Override

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/f1528bf3/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 27f7d55..5f9b213 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
@@ -522,8 +522,7 @@
  *    {@link org.apache.lucene.search.Scorer Scorer} is going to be a <code>BooleanScorer2</code> created
  *    from {@link org.apache.lucene.search.BooleanWeight BooleanWeight} (see the section on
  *    <a href="#customQueriesExpert">custom queries</a> for info on changing this).
- * <p>Assuming a BooleanScorer2, we first initialize the Coordinator, which is used to apply the coord() 
- *   factor. We then get a internal Scorer based on the required, optional and prohibited parts of the query.
+ * <p>Assuming a BooleanScorer2, we get a internal Scorer based on the required, optional and prohibited parts of the query.
  *   Using this internal Scorer, the BooleanScorer2 then proceeds into a while loop based on the 
  *   {@link org.apache.lucene.search.DocIdSetIterator#nextDoc DocIdSetIterator.nextDoc()} method. The nextDoc() method advances 
  *   to the next document matching the query. This is an abstract method in the Scorer class and is thus 

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/f1528bf3/lucene/core/src/java/org/apache/lucene/search/similarities/ClassicSimilarity.java
----------------------------------------------------------------------
diff --git a/lucene/core/src/java/org/apache/lucene/search/similarities/ClassicSimilarity.java b/lucene/core/src/java/org/apache/lucene/search/similarities/ClassicSimilarity.java
index 3b6cbdd..93b8261 100644
--- a/lucene/core/src/java/org/apache/lucene/search/similarities/ClassicSimilarity.java
+++ b/lucene/core/src/java/org/apache/lucene/search/similarities/ClassicSimilarity.java
@@ -55,12 +55,6 @@ public class ClassicSimilarity extends TFIDFSimilarity {
 
   /** Sole constructor: parameter-free */
   public ClassicSimilarity() {}
-  
-  /** Implemented as <code>overlap / maxOverlap</code>. */
-  @Override
-  public float coord(int overlap, int maxOverlap) {
-    return overlap / (float)maxOverlap;
-  }
 
   /** Implemented as <code>1/sqrt(sumOfSquaredWeights)</code>. */
   @Override

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/f1528bf3/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 9605c83..e66fa5f 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
@@ -109,20 +109,6 @@ public abstract class Similarity {
    */
   public Similarity() {}
   
-  /** Hook to integrate coordinate-level matching.
-   * <p>
-   * By default this is disabled (returns <code>1</code>), as with
-   * most modern models this will only skew performance, but some
-   * implementations such as {@link TFIDFSimilarity} override this.
-   *
-   * @param overlap the number of query terms matched in the document
-   * @param maxOverlap the total number of terms in the query
-   * @return a score factor based on term overlap with the query
-   */
-  public float coord(int overlap, int maxOverlap) {
-    return 1f;
-  }
-  
   /** Computes the normalization value for a query given the sum of the
    * normalized weights {@link SimWeight#getValueForNormalization()} of 
    * each of the query terms.  This value is passed back to the 

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/f1528bf3/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 a6b1ee5..3e92375 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
@@ -156,10 +156,7 @@ import org.apache.lucene.util.BytesRef;
  *  </li>
  *
  *  <li>A document may match a multi term query without containing all
- *  the terms of that query (this is correct for some of the queries),
- *  and users can further reward documents matching more query terms
- *  through a coordination factor, which is usually larger when
- *  more terms are matched: <i>coord-factor(q,d)</i>.
+ *  the terms of that query (this is correct for some of the queries).
  *  </li>
  * </ul>
  *
@@ -175,7 +172,6 @@ import org.apache.lucene.util.BytesRef;
  *        <tr>
  *          <td valign="middle" align="right" rowspan="1">
  *            score(q,d) &nbsp; = &nbsp;
- *            <span style="color: #FF9933">coord-factor(q,d)</span> &middot; &nbsp;
  *            <span style="color: #CCCC00">query-boost(q)</span> &middot; &nbsp;
  *          </td>
  *          <td valign="middle" align="center">
@@ -266,7 +262,6 @@ import org.apache.lucene.util.BytesRef;
  *   <tr>
  *     <td valign="middle" align="right" rowspan="1">
  *       score(q,d) &nbsp; = &nbsp;
- *       <A HREF="#formula_coord"><span style="color: #FF9933">coord(q,d)</span></A> &nbsp;&middot;&nbsp;
  *       <A HREF="#formula_queryNorm"><span style="color: #FF33CC">queryNorm(q)</span></A> &nbsp;&middot;&nbsp;
  *     </td>
  *     <td valign="bottom" align="center" rowspan="1" style="text-align: center">
@@ -359,18 +354,6 @@ import org.apache.lucene.util.BytesRef;
  *      <br>&nbsp;<br>
  *    </li>
  *
- *    <li>
- *      <A NAME="formula_coord"></A>
- *      <b><i>coord(q,d)</i></b>
- *      is a score factor based on how many of the query terms are found in the specified document.
- *      Typically, a document that contains more of the query's terms will receive a higher score
- *      than another document with fewer query terms.
- *      This is a search time factor computed in
- *      {@link #coord(int, int) coord(q,d)}
- *      by the Similarity in effect at search time.
- *      <br>&nbsp;<br>
- *    </li>
- *
  *    <li><b>
  *      <A NAME="formula_queryNorm"></A>
  *      <i>queryNorm(q)</i>
@@ -512,21 +495,6 @@ public abstract class TFIDFSimilarity extends Similarity {
    */
   public TFIDFSimilarity() {}
   
-  /** Computes a score factor based on the fraction of all query terms that a
-   * document contains.  This value is multiplied into scores.
-   *
-   * <p>The presence of a large portion of the query terms indicates a better
-   * match with the query, so implementations of this method usually return
-   * larger values when the ratio between these parameters is large and smaller
-   * values when the ratio between them is small.
-   *
-   * @param overlap the number of query terms matched in the document
-   * @param maxOverlap the total number of terms in the query
-   * @return a score factor based on term overlap with the query
-   */
-  @Override
-  public abstract float coord(int overlap, int maxOverlap);
-  
   /** Computes the normalization value for a query given the sum of the squared
    * weights of each of the query terms.  This value is multiplied into the
    * weight of each query term. While the classic query normalization factor is

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/f1528bf3/lucene/core/src/java/org/apache/lucene/util/QueryBuilder.java
----------------------------------------------------------------------
diff --git a/lucene/core/src/java/org/apache/lucene/util/QueryBuilder.java b/lucene/core/src/java/org/apache/lucene/util/QueryBuilder.java
index 259335b..309150e 100644
--- a/lucene/core/src/java/org/apache/lucene/util/QueryBuilder.java
+++ b/lucene/core/src/java/org/apache/lucene/util/QueryBuilder.java
@@ -137,7 +137,6 @@ public class QueryBuilder {
     if (query instanceof BooleanQuery) {
       BooleanQuery bq = (BooleanQuery) query;
       BooleanQuery.Builder builder = new BooleanQuery.Builder();
-      builder.setDisableCoord(bq.isCoordDisabled());
       builder.setMinimumNumberShouldMatch((int) (fraction * bq.clauses().size()));
       for (BooleanClause clause : bq) {
         builder.add(clause);

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/f1528bf3/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 c513093..a8d457e 100644
--- a/lucene/core/src/test/org/apache/lucene/index/TestCustomNorms.java
+++ b/lucene/core/src/test/org/apache/lucene/index/TestCustomNorms.java
@@ -95,11 +95,6 @@ public class TestCustomNorms extends LuceneTestCase {
         return delegate;
       }
     }
-
-    @Override
-    public float coord(int overlap, int maxOverlap) {
-      return delegate.coord(overlap, maxOverlap);
-    }
   }
 
   public static class FloatEncodingBoostSimilarity extends Similarity {

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/f1528bf3/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 69a7f7f..871ae30 100644
--- a/lucene/core/src/test/org/apache/lucene/index/TestMaxTermFrequency.java
+++ b/lucene/core/src/test/org/apache/lucene/index/TestMaxTermFrequency.java
@@ -115,7 +115,6 @@ public class TestMaxTermFrequency extends LuceneTestCase {
       return norm;
     }
 
-    @Override public float coord(int overlap, int maxOverlap) { return 0; }
     @Override public float queryNorm(float sumOfSquaredWeights) { return 0; }
     @Override public float tf(float freq) { return 0; }
     @Override public float idf(long docFreq, long docCount) { return 0; }

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/f1528bf3/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 45db69a..642540e 100644
--- a/lucene/core/src/test/org/apache/lucene/index/TestNorms.java
+++ b/lucene/core/src/test/org/apache/lucene/index/TestNorms.java
@@ -64,7 +64,6 @@ public class TestNorms extends LuceneTestCase {
       return state.getLength();
     }
 
-    @Override public float coord(int overlap, int maxOverlap) { return 0; }
     @Override public float queryNorm(float sumOfSquaredWeights) { return 0; }
     @Override public float tf(float freq) { return 0; }
     @Override public float idf(long docFreq, long docCount) { return 0; }
@@ -170,11 +169,6 @@ public class TestNorms extends LuceneTestCase {
         return delegate;
       }
     }
-
-    @Override
-    public float coord(int overlap, int maxOverlap) {
-      return delegate.coord(overlap, maxOverlap);
-    }
   }
 
   

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/f1528bf3/lucene/core/src/test/org/apache/lucene/index/TestOmitTf.java
----------------------------------------------------------------------
diff --git a/lucene/core/src/test/org/apache/lucene/index/TestOmitTf.java b/lucene/core/src/test/org/apache/lucene/index/TestOmitTf.java
index 195f548..3d3384e 100644
--- a/lucene/core/src/test/org/apache/lucene/index/TestOmitTf.java
+++ b/lucene/core/src/test/org/apache/lucene/index/TestOmitTf.java
@@ -48,8 +48,6 @@ public class TestOmitTf extends LuceneTestCase {
     @Override public long encodeNormValue(float f) { return (long) f; }
     @Override
     public float queryNorm(float sumOfSquaredWeights) { return 1.0f; }
-    @Override
-    public float coord(int overlap, int maxOverlap) { return 1.0f; }
     @Override public float lengthNorm(FieldInvertState state) { return state.getBoost(); }
     @Override public float tf(float freq) { return freq; }
     @Override public float sloppyFreq(int distance) { return 2.0f; }

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/f1528bf3/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 0a217e3..ae46e34 100644
--- a/lucene/core/src/test/org/apache/lucene/search/TestBoolean2.java
+++ b/lucene/core/src/test/org/apache/lucene/search/TestBoolean2.java
@@ -32,7 +32,6 @@ import org.apache.lucene.index.IndexWriterConfig;
 import org.apache.lucene.index.RandomIndexWriter;
 import org.apache.lucene.index.Term;
 import org.apache.lucene.search.similarities.ClassicSimilarity;
-import org.apache.lucene.search.similarities.Similarity;
 import org.apache.lucene.store.Directory;
 import org.apache.lucene.store.IOContext;
 import org.apache.lucene.store.MockDirectoryWrapper;
@@ -240,7 +239,6 @@ public class TestBoolean2 extends LuceneTestCase {
   @Test
   public void testQueries01() throws Exception {
     BooleanQuery.Builder query = new BooleanQuery.Builder();
-    query.setDisableCoord(random().nextBoolean());
     query.add(new TermQuery(new Term(field, "w3")), BooleanClause.Occur.MUST);
     query.add(new TermQuery(new Term(field, "xx")), BooleanClause.Occur.MUST);
     int[] expDocNrs = {2,3};
@@ -250,7 +248,6 @@ public class TestBoolean2 extends LuceneTestCase {
   @Test
   public void testQueries02() throws Exception {
     BooleanQuery.Builder query = new BooleanQuery.Builder();
-    query.setDisableCoord(random().nextBoolean());
     query.add(new TermQuery(new Term(field, "w3")), BooleanClause.Occur.MUST);
     query.add(new TermQuery(new Term(field, "xx")), BooleanClause.Occur.SHOULD);
     int[] expDocNrs = {2,3,1,0};
@@ -260,7 +257,6 @@ public class TestBoolean2 extends LuceneTestCase {
   @Test
   public void testQueries03() throws Exception {
     BooleanQuery.Builder query = new BooleanQuery.Builder();
-    query.setDisableCoord(random().nextBoolean());
     query.add(new TermQuery(new Term(field, "w3")), BooleanClause.Occur.SHOULD);
     query.add(new TermQuery(new Term(field, "xx")), BooleanClause.Occur.SHOULD);
     int[] expDocNrs = {2,3,1,0};
@@ -270,7 +266,6 @@ public class TestBoolean2 extends LuceneTestCase {
   @Test
   public void testQueries04() throws Exception {
     BooleanQuery.Builder query = new BooleanQuery.Builder();
-    query.setDisableCoord(random().nextBoolean());
     query.add(new TermQuery(new Term(field, "w3")), BooleanClause.Occur.SHOULD);
     query.add(new TermQuery(new Term(field, "xx")), BooleanClause.Occur.MUST_NOT);
     int[] expDocNrs = {1,0};
@@ -280,7 +275,6 @@ public class TestBoolean2 extends LuceneTestCase {
   @Test
   public void testQueries05() throws Exception {
     BooleanQuery.Builder query = new BooleanQuery.Builder();
-    query.setDisableCoord(random().nextBoolean());
     query.add(new TermQuery(new Term(field, "w3")), BooleanClause.Occur.MUST);
     query.add(new TermQuery(new Term(field, "xx")), BooleanClause.Occur.MUST_NOT);
     int[] expDocNrs = {1,0};
@@ -290,7 +284,6 @@ public class TestBoolean2 extends LuceneTestCase {
   @Test
   public void testQueries06() throws Exception {
     BooleanQuery.Builder query = new BooleanQuery.Builder();
-    query.setDisableCoord(random().nextBoolean());
     query.add(new TermQuery(new Term(field, "w3")), BooleanClause.Occur.MUST);
     query.add(new TermQuery(new Term(field, "xx")), BooleanClause.Occur.MUST_NOT);
     query.add(new TermQuery(new Term(field, "w5")), BooleanClause.Occur.MUST_NOT);
@@ -301,7 +294,6 @@ public class TestBoolean2 extends LuceneTestCase {
   @Test
   public void testQueries07() throws Exception {
     BooleanQuery.Builder query = new BooleanQuery.Builder();
-    query.setDisableCoord(random().nextBoolean());
     query.add(new TermQuery(new Term(field, "w3")), BooleanClause.Occur.MUST_NOT);
     query.add(new TermQuery(new Term(field, "xx")), BooleanClause.Occur.MUST_NOT);
     query.add(new TermQuery(new Term(field, "w5")), BooleanClause.Occur.MUST_NOT);
@@ -312,7 +304,6 @@ public class TestBoolean2 extends LuceneTestCase {
   @Test
   public void testQueries08() throws Exception {
     BooleanQuery.Builder query = new BooleanQuery.Builder();
-    query.setDisableCoord(random().nextBoolean());
     query.add(new TermQuery(new Term(field, "w3")), BooleanClause.Occur.MUST);
     query.add(new TermQuery(new Term(field, "xx")), BooleanClause.Occur.SHOULD);
     query.add(new TermQuery(new Term(field, "w5")), BooleanClause.Occur.MUST_NOT);
@@ -323,7 +314,6 @@ public class TestBoolean2 extends LuceneTestCase {
   @Test
   public void testQueries09() throws Exception {
     BooleanQuery.Builder query = new BooleanQuery.Builder();
-    query.setDisableCoord(random().nextBoolean());
     query.add(new TermQuery(new Term(field, "w3")), BooleanClause.Occur.MUST);
     query.add(new TermQuery(new Term(field, "xx")), BooleanClause.Occur.MUST);
     query.add(new TermQuery(new Term(field, "w2")), BooleanClause.Occur.MUST);
@@ -333,33 +323,6 @@ public class TestBoolean2 extends LuceneTestCase {
   }
 
   @Test
-  public void testQueries10() throws Exception {
-    BooleanQuery.Builder query = new BooleanQuery.Builder();
-    query.setDisableCoord(random().nextBoolean());
-    query.add(new TermQuery(new Term(field, "w3")), BooleanClause.Occur.MUST);
-    query.add(new TermQuery(new Term(field, "xx")), BooleanClause.Occur.MUST);
-    query.add(new TermQuery(new Term(field, "w2")), BooleanClause.Occur.MUST);
-    query.add(new TermQuery(new Term(field, "zz")), BooleanClause.Occur.SHOULD);
-
-    int[] expDocNrs = {2, 3};
-    Similarity oldSimilarity = searcher.getSimilarity(true);
-    Similarity newSimilarity = new ClassicSimilarity() {
-        @Override
-        public float coord(int overlap, int maxOverlap) {
-          return overlap / ((float)maxOverlap - 1);
-        }
-      };
-    try {
-      searcher.setSimilarity(newSimilarity);
-      singleSegmentSearcher.setSimilarity(newSimilarity);
-      queriesTest(query.build(), expDocNrs);
-    } finally {
-      searcher.setSimilarity(oldSimilarity);
-      singleSegmentSearcher.setSimilarity(oldSimilarity);
-    }
-  }
-
-  @Test
   public void testRandomQueries() throws Exception {
     String[] vals = {"w1","w2","w3","w4","w5","xx","yy","zzz"};
 
@@ -434,7 +397,6 @@ public class TestBoolean2 extends LuceneTestCase {
   // more than once.
   public static BooleanQuery.Builder randBoolQuery(Random rnd, boolean allowMust, int level, String field, String[] vals, Callback cb) {
     BooleanQuery.Builder current = new BooleanQuery.Builder();
-    current.setDisableCoord(rnd.nextBoolean());
     for (int i=0; i<rnd.nextInt(vals.length)+1; i++) {
       int qType=0; // term query
       if (level>0) {