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 2017/02/16 14:30:11 UTC

[1/4] lucene-solr:branch_6x: LUCENE-7692: PatternReplaceCharFilterFactory should implement MultiTermAware.

Repository: lucene-solr
Updated Branches:
  refs/heads/branch_6x 2e67777ff -> 1bfa057d5


LUCENE-7692: PatternReplaceCharFilterFactory should implement MultiTermAware.


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

Branch: refs/heads/branch_6x
Commit: 4a636c41ca7c5ee902da561f31c5d386cb54687f
Parents: 2e67777f
Author: Adrien Grand <jp...@gmail.com>
Authored: Thu Feb 16 09:34:41 2017 +0100
Committer: Adrien Grand <jp...@gmail.com>
Committed: Thu Feb 16 14:53:46 2017 +0100

----------------------------------------------------------------------
 lucene/CHANGES.txt                                          | 3 +++
 .../analysis/pattern/PatternReplaceCharFilterFactory.java   | 9 ++++++++-
 2 files changed, 11 insertions(+), 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/4a636c41/lucene/CHANGES.txt
----------------------------------------------------------------------
diff --git a/lucene/CHANGES.txt b/lucene/CHANGES.txt
index 6fd5fb1..29a5390 100644
--- a/lucene/CHANGES.txt
+++ b/lucene/CHANGES.txt
@@ -65,6 +65,9 @@ Bug Fixes
 * LUCENE-7679: MemoryIndex was ignoring omitNorms settings on passed-in
   IndexableFields. (Alan Woodward)
 
+* LUCENE-7692: PatternReplaceCharFilterFactory now implements MultiTermAware.
+  (Adrien Grand)
+
 Improvements
 
 * LUCENE-7055: Added Weight#scorerSupplier, which allows to estimate the cost

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/4a636c41/lucene/analysis/common/src/java/org/apache/lucene/analysis/pattern/PatternReplaceCharFilterFactory.java
----------------------------------------------------------------------
diff --git a/lucene/analysis/common/src/java/org/apache/lucene/analysis/pattern/PatternReplaceCharFilterFactory.java b/lucene/analysis/common/src/java/org/apache/lucene/analysis/pattern/PatternReplaceCharFilterFactory.java
index 70e9728..5319cbc 100644
--- a/lucene/analysis/common/src/java/org/apache/lucene/analysis/pattern/PatternReplaceCharFilterFactory.java
+++ b/lucene/analysis/common/src/java/org/apache/lucene/analysis/pattern/PatternReplaceCharFilterFactory.java
@@ -22,7 +22,9 @@ import java.util.Map;
 import java.util.regex.Pattern;
 
 import org.apache.lucene.analysis.CharFilter;
+import org.apache.lucene.analysis.util.AbstractAnalysisFactory;
 import org.apache.lucene.analysis.util.CharFilterFactory;
+import org.apache.lucene.analysis.util.MultiTermAwareComponent;
 
 /**
  * Factory for {@link PatternReplaceCharFilter}. 
@@ -37,7 +39,7 @@ import org.apache.lucene.analysis.util.CharFilterFactory;
  * 
  * @since Solr 3.1
  */
-public class PatternReplaceCharFilterFactory extends CharFilterFactory {
+public class PatternReplaceCharFilterFactory extends CharFilterFactory implements MultiTermAwareComponent {
   private final Pattern pattern;
   private final String replacement;
 
@@ -55,4 +57,9 @@ public class PatternReplaceCharFilterFactory extends CharFilterFactory {
   public CharFilter create(Reader input) {
     return new PatternReplaceCharFilter(pattern, replacement, input);
   }
+
+  @Override
+  public AbstractAnalysisFactory getMultiTermComponent() {
+    return this;
+  }
 }


[2/4] lucene-solr:branch_6x: LUCENE-7685: Remove equals/rewrite hacks from block join queries.

Posted by jp...@apache.org.
LUCENE-7685: Remove equals/rewrite hacks from block join queries.


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

Branch: refs/heads/branch_6x
Commit: e092d4f344432cf56b0059027a3365d30227a925
Parents: 4a636c4
Author: Adrien Grand <jp...@gmail.com>
Authored: Thu Feb 16 09:37:59 2017 +0100
Committer: Adrien Grand <jp...@gmail.com>
Committed: Thu Feb 16 14:53:55 2017 +0100

----------------------------------------------------------------------
 lucene/CHANGES.txt                              |  4 ++++
 .../search/join/ToChildBlockJoinQuery.java      | 23 +++-----------------
 .../search/join/ToParentBlockJoinQuery.java     | 23 +++-----------------
 3 files changed, 10 insertions(+), 40 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/e092d4f3/lucene/CHANGES.txt
----------------------------------------------------------------------
diff --git a/lucene/CHANGES.txt b/lucene/CHANGES.txt
index 29a5390..4d6cd54 100644
--- a/lucene/CHANGES.txt
+++ b/lucene/CHANGES.txt
@@ -68,6 +68,10 @@ Bug Fixes
 * LUCENE-7692: PatternReplaceCharFilterFactory now implements MultiTermAware.
   (Adrien Grand)
 
+* LUCENE-7685: ToParentBlockJoinQuery and ToChildBlockJoinQuery now use the
+  rewritten child query in their equals and hashCode implementations.
+  (Adrien Grand)
+
 Improvements
 
 * LUCENE-7055: Added Weight#scorerSupplier, which allows to estimate the cost

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/e092d4f3/lucene/join/src/java/org/apache/lucene/search/join/ToChildBlockJoinQuery.java
----------------------------------------------------------------------
diff --git a/lucene/join/src/java/org/apache/lucene/search/join/ToChildBlockJoinQuery.java b/lucene/join/src/java/org/apache/lucene/search/join/ToChildBlockJoinQuery.java
index 830e29b..71771fd 100644
--- a/lucene/join/src/java/org/apache/lucene/search/join/ToChildBlockJoinQuery.java
+++ b/lucene/join/src/java/org/apache/lucene/search/join/ToChildBlockJoinQuery.java
@@ -52,13 +52,6 @@ public class ToChildBlockJoinQuery extends Query {
   private final BitSetProducer parentsFilter;
   private final Query parentQuery;
 
-  // If we are rewritten, this is the original parentQuery we
-  // were passed; we use this for .equals() and
-  // .hashCode().  This makes rewritten query equal the
-  // original, so that user does not have to .rewrite() their
-  // query before searching:
-  private final Query origParentQuery;
-
   /**
    * Create a ToChildBlockJoinQuery.
    * 
@@ -67,14 +60,6 @@ public class ToChildBlockJoinQuery extends Query {
    */
   public ToChildBlockJoinQuery(Query parentQuery, BitSetProducer parentsFilter) {
     super();
-    this.origParentQuery = parentQuery;
-    this.parentQuery = parentQuery;
-    this.parentsFilter = parentsFilter;
-  }
-
-  private ToChildBlockJoinQuery(Query origParentQuery, Query parentQuery, BitSetProducer parentsFilter) {
-    super();
-    this.origParentQuery = origParentQuery;
     this.parentQuery = parentQuery;
     this.parentsFilter = parentsFilter;
   }
@@ -312,9 +297,7 @@ public class ToChildBlockJoinQuery extends Query {
   public Query rewrite(IndexReader reader) throws IOException {
     final Query parentRewrite = parentQuery.rewrite(reader);
     if (parentRewrite != parentQuery) {
-      return new ToChildBlockJoinQuery(parentQuery,
-                                parentRewrite,
-                                parentsFilter);
+      return new ToChildBlockJoinQuery(parentRewrite, parentsFilter);
     } else {
       return super.rewrite(reader);
     }
@@ -332,7 +315,7 @@ public class ToChildBlockJoinQuery extends Query {
   }
 
   private boolean equalsTo(ToChildBlockJoinQuery other) {
-    return origParentQuery.equals(other.origParentQuery) &&
+    return parentQuery.equals(other.parentQuery) &&
            parentsFilter.equals(other.parentsFilter);
   }
 
@@ -340,7 +323,7 @@ public class ToChildBlockJoinQuery extends Query {
   public int hashCode() {
     final int prime = 31;
     int hash = classHash();
-    hash = prime * hash + origParentQuery.hashCode();
+    hash = prime * hash + parentQuery.hashCode();
     hash = prime * hash + parentsFilter.hashCode();
     return hash;
   }

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/e092d4f3/lucene/join/src/java/org/apache/lucene/search/join/ToParentBlockJoinQuery.java
----------------------------------------------------------------------
diff --git a/lucene/join/src/java/org/apache/lucene/search/join/ToParentBlockJoinQuery.java b/lucene/join/src/java/org/apache/lucene/search/join/ToParentBlockJoinQuery.java
index 0e15ac2..5d0c27b 100644
--- a/lucene/join/src/java/org/apache/lucene/search/join/ToParentBlockJoinQuery.java
+++ b/lucene/join/src/java/org/apache/lucene/search/join/ToParentBlockJoinQuery.java
@@ -67,13 +67,6 @@ public class ToParentBlockJoinQuery extends Query {
 
   private final BitSetProducer parentsFilter;
   private final Query childQuery;
-
-  // If we are rewritten, this is the original childQuery we
-  // were passed; we use this for .equals() and
-  // .hashCode().  This makes rewritten query equal the
-  // original, so that user does not have to .rewrite() their
-  // query before searching:
-  private final Query origChildQuery;
   private final ScoreMode scoreMode;
 
   /** Create a ToParentBlockJoinQuery.
@@ -85,15 +78,6 @@ public class ToParentBlockJoinQuery extends Query {
    **/
   public ToParentBlockJoinQuery(Query childQuery, BitSetProducer parentsFilter, ScoreMode scoreMode) {
     super();
-    this.origChildQuery = childQuery;
-    this.childQuery = childQuery;
-    this.parentsFilter = parentsFilter;
-    this.scoreMode = scoreMode;
-  }
-
-  private ToParentBlockJoinQuery(Query origChildQuery, Query childQuery, BitSetProducer parentsFilter, ScoreMode scoreMode) {
-    super();
-    this.origChildQuery = origChildQuery;
     this.childQuery = childQuery;
     this.parentsFilter = parentsFilter;
     this.scoreMode = scoreMode;
@@ -377,8 +361,7 @@ public class ToParentBlockJoinQuery extends Query {
   public Query rewrite(IndexReader reader) throws IOException {
     final Query childRewrite = childQuery.rewrite(reader);
     if (childRewrite != childQuery) {
-      return new ToParentBlockJoinQuery(origChildQuery,
-                                childRewrite,
+      return new ToParentBlockJoinQuery(childRewrite,
                                 parentsFilter,
                                 scoreMode);
     } else {
@@ -398,7 +381,7 @@ public class ToParentBlockJoinQuery extends Query {
   }
 
   private boolean equalsTo(ToParentBlockJoinQuery other) {
-    return origChildQuery.equals(other.origChildQuery) &&
+    return childQuery.equals(other.childQuery) &&
            parentsFilter.equals(other.parentsFilter) &&
            scoreMode == other.scoreMode;
   }
@@ -407,7 +390,7 @@ public class ToParentBlockJoinQuery extends Query {
   public int hashCode() {
     final int prime = 31;
     int hash = classHash();
-    hash = prime * hash + origChildQuery.hashCode();
+    hash = prime * hash + childQuery.hashCode();
     hash = prime * hash + scoreMode.hashCode();
     hash = prime * hash + parentsFilter.hashCode();
     return hash;


[4/4] lucene-solr:branch_6x: LUCENE-7677: Cache compound filters earlier than regular filters.

Posted by jp...@apache.org.
LUCENE-7677: Cache compound filters earlier than regular filters.


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

Branch: refs/heads/branch_6x
Commit: 1bfa057d5c9d89b116031baa7493ee422b4cbabb
Parents: 5682c33
Author: Adrien Grand <jp...@gmail.com>
Authored: Thu Feb 16 09:42:50 2017 +0100
Committer: Adrien Grand <jp...@gmail.com>
Committed: Thu Feb 16 15:02:27 2017 +0100

----------------------------------------------------------------------
 lucene/CHANGES.txt                              |  4 +
 .../search/UsageTrackingQueryCachingPolicy.java | 12 ++-
 .../TestUsageTrackingFilterCachingPolicy.java   | 81 ++++++++++++++++++++
 3 files changed, 96 insertions(+), 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/1bfa057d/lucene/CHANGES.txt
----------------------------------------------------------------------
diff --git a/lucene/CHANGES.txt b/lucene/CHANGES.txt
index f63fb22..0e41e2f 100644
--- a/lucene/CHANGES.txt
+++ b/lucene/CHANGES.txt
@@ -91,6 +91,10 @@ Improvements
   since they are plenty fast. This also has the side-effect of leaving more
   space in the history for costly filters. (Adrien Grand)
 
+* LUCENE-7677: UsageTrackingQueryCachingPolicy now caches compound queries a bit
+  earlier than regular queries in order to improve cache efficiency.
+  (Adrien Grand)
+
 Optimizations
 
 * LUCENE-7641: Optimized point range queries to compute documents that do not

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/1bfa057d/lucene/core/src/java/org/apache/lucene/search/UsageTrackingQueryCachingPolicy.java
----------------------------------------------------------------------
diff --git a/lucene/core/src/java/org/apache/lucene/search/UsageTrackingQueryCachingPolicy.java b/lucene/core/src/java/org/apache/lucene/search/UsageTrackingQueryCachingPolicy.java
index 568931d..1bbed5c 100644
--- a/lucene/core/src/java/org/apache/lucene/search/UsageTrackingQueryCachingPolicy.java
+++ b/lucene/core/src/java/org/apache/lucene/search/UsageTrackingQueryCachingPolicy.java
@@ -122,7 +122,17 @@ public class UsageTrackingQueryCachingPolicy implements QueryCachingPolicy {
       return 2;
     } else {
       // default: cache after the filter has been seen 5 times
-      return 5;
+      int minFrequency = 5;
+      if (query instanceof BooleanQuery
+          || query instanceof DisjunctionMaxQuery) {
+        // Say you keep reusing a boolean query that looks like "A OR B" and
+        // never use the A and B queries out of that context. 5 times after it
+        // has been used, we would cache both A, B and A OR B, which is
+        // wasteful. So instead we cache compound queries a bit earlier so that
+        // we would only cache "A OR B" in that case.
+        minFrequency--;
+      }
+      return minFrequency;
     }
   }
 

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/1bfa057d/lucene/core/src/test/org/apache/lucene/search/TestUsageTrackingFilterCachingPolicy.java
----------------------------------------------------------------------
diff --git a/lucene/core/src/test/org/apache/lucene/search/TestUsageTrackingFilterCachingPolicy.java b/lucene/core/src/test/org/apache/lucene/search/TestUsageTrackingFilterCachingPolicy.java
index eed3cb7..2d20904 100644
--- a/lucene/core/src/test/org/apache/lucene/search/TestUsageTrackingFilterCachingPolicy.java
+++ b/lucene/core/src/test/org/apache/lucene/search/TestUsageTrackingFilterCachingPolicy.java
@@ -18,8 +18,14 @@ package org.apache.lucene.search;
 
 import java.io.IOException;
 
+import org.apache.lucene.document.Document;
 import org.apache.lucene.document.IntPoint;
+import org.apache.lucene.index.IndexReader;
+import org.apache.lucene.index.LeafReaderContext;
+import org.apache.lucene.index.RandomIndexWriter;
 import org.apache.lucene.index.Term;
+import org.apache.lucene.search.BooleanClause.Occur;
+import org.apache.lucene.store.Directory;
 import org.apache.lucene.util.LuceneTestCase;
 
 public class TestUsageTrackingFilterCachingPolicy extends LuceneTestCase {
@@ -48,4 +54,79 @@ public class TestUsageTrackingFilterCachingPolicy extends LuceneTestCase {
     assertFalse(policy.shouldCache(q));
   }
 
+  public void testBooleanQueries() throws IOException {
+    Directory dir = newDirectory();
+    RandomIndexWriter w = new RandomIndexWriter(random(), dir);
+    w.addDocument(new Document());
+    IndexReader reader = w.getReader();
+    w.close();
+    
+    IndexSearcher searcher = new IndexSearcher(reader);
+    UsageTrackingQueryCachingPolicy policy = new UsageTrackingQueryCachingPolicy();
+    LRUQueryCache cache = new LRUQueryCache(10, Long.MAX_VALUE, new LRUQueryCache.MinSegmentSizePredicate(1, 0f));
+    searcher.setQueryCache(cache);
+    searcher.setQueryCachingPolicy(policy);
+
+    DummyQuery q1 = new DummyQuery(1);
+    DummyQuery q2 = new DummyQuery(2);
+    BooleanQuery bq = new BooleanQuery.Builder()
+        .add(q1, Occur.SHOULD)
+        .add(q2, Occur.SHOULD)
+        .build();
+
+    for (int i = 0; i < 3; ++i) {
+      searcher.count(bq);
+    }
+    assertEquals(0, cache.getCacheSize()); // nothing cached yet, too early
+
+    searcher.count(bq);
+    assertEquals(1, cache.getCacheSize()); // the bq got cached, but not q1 and q2
+
+    for (int i = 0; i < 10; ++i) {
+      searcher.count(bq);
+    }
+    assertEquals(1, cache.getCacheSize()); // q1 and q2 still not cached since we do not pull scorers on them
+
+    searcher.count(q1);
+    assertEquals(2, cache.getCacheSize()); // q1 used on its own -> cached
+
+    reader.close();
+    dir.close();
+  }
+
+  private static class DummyQuery extends Query {
+
+    private final int id;
+
+    DummyQuery(int id) {
+      this.id = id;
+    }
+
+    @Override
+    public String toString(String field) {
+      return "dummy";
+    }
+
+    @Override
+    public boolean equals(Object obj) {
+      return sameClassAs(obj) && ((DummyQuery) obj).id == id;
+    }
+
+    @Override
+    public int hashCode() {
+      return id;
+    }
+
+    @Override
+    public Weight createWeight(IndexSearcher searcher, boolean needsScores) throws IOException {
+      return new ConstantScoreWeight(DummyQuery.this) {
+        @Override
+        public Scorer scorer(LeafReaderContext context) throws IOException {
+          return new ConstantScoreScorer(this, score(), DocIdSetIterator.all(1));
+        }
+      };
+    }
+
+  }
+
 }


[3/4] lucene-solr:branch_6x: LUCENE-7680: Never cache term filters.

Posted by jp...@apache.org.
LUCENE-7680: Never cache term filters.


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

Branch: refs/heads/branch_6x
Commit: 5682c33b5c3444905e2638d735f935d198c4cea6
Parents: e092d4f
Author: Adrien Grand <jp...@gmail.com>
Authored: Thu Feb 16 09:40:31 2017 +0100
Committer: Adrien Grand <jp...@gmail.com>
Committed: Thu Feb 16 14:54:02 2017 +0100

----------------------------------------------------------------------
 lucene/CHANGES.txt                              |  4 +
 .../search/UsageTrackingQueryCachingPolicy.java | 78 +++++++++++++-------
 .../TestUsageTrackingFilterCachingPolicy.java   | 11 +++
 3 files changed, 66 insertions(+), 27 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/5682c33b/lucene/CHANGES.txt
----------------------------------------------------------------------
diff --git a/lucene/CHANGES.txt b/lucene/CHANGES.txt
index 4d6cd54..f63fb22 100644
--- a/lucene/CHANGES.txt
+++ b/lucene/CHANGES.txt
@@ -87,6 +87,10 @@ Improvements
   of the less descriptive FileNotFound or NoSuchFileException (Mike Drob via 
   Mike McCandless, Erick Erickson)
 
+* LUCENE-7680: UsageTrackingQueryCachingPolicy never caches term filters anymore
+  since they are plenty fast. This also has the side-effect of leaving more
+  space in the history for costly filters. (Adrien Grand)
+
 Optimizations
 
 * LUCENE-7641: Optimized point range queries to compute documents that do not

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/5682c33b/lucene/core/src/java/org/apache/lucene/search/UsageTrackingQueryCachingPolicy.java
----------------------------------------------------------------------
diff --git a/lucene/core/src/java/org/apache/lucene/search/UsageTrackingQueryCachingPolicy.java b/lucene/core/src/java/org/apache/lucene/search/UsageTrackingQueryCachingPolicy.java
index 035947f..568931d 100644
--- a/lucene/core/src/java/org/apache/lucene/search/UsageTrackingQueryCachingPolicy.java
+++ b/lucene/core/src/java/org/apache/lucene/search/UsageTrackingQueryCachingPolicy.java
@@ -27,7 +27,7 @@ import org.apache.lucene.util.FrequencyTrackingRingBuffer;
  *
  * @lucene.experimental
  */
-public final class UsageTrackingQueryCachingPolicy implements QueryCachingPolicy {
+public class UsageTrackingQueryCachingPolicy implements QueryCachingPolicy {
 
   // the hash code that we use as a sentinel in the ring buffer.
   private static final int SENTINEL = Integer.MIN_VALUE;
@@ -54,16 +54,49 @@ public final class UsageTrackingQueryCachingPolicy implements QueryCachingPolicy
         isPointQuery(query);
   }
 
-  static boolean isCheap(Query query) {
-    // same for cheap queries
-    // these queries are so cheap that they usually do not need caching
-    return query instanceof TermQuery;
+  private static boolean shouldNeverCache(Query query) {
+    if (query instanceof TermQuery) {
+      // We do not bother caching term queries since they are already plenty fast.
+      return true;
+    }
+
+    if (query instanceof MatchAllDocsQuery) {
+      // MatchAllDocsQuery has an iterator that is faster than what a bit set could do.
+      return true;
+    }
+
+    // For the below queries, it's cheap to notice they cannot match any docs so
+    // we do not bother caching them.
+    if (query instanceof MatchNoDocsQuery) {
+      return false;
+    }
+
+    if (query instanceof BooleanQuery) {
+      BooleanQuery bq = (BooleanQuery) query;
+      if (bq.clauses().isEmpty()) {
+        return true;
+      }
+    }
+
+    if (query instanceof DisjunctionMaxQuery) {
+      DisjunctionMaxQuery dmq = (DisjunctionMaxQuery) query;
+      if (dmq.getDisjuncts().isEmpty()) {
+        return true;
+      }
+    }
+
+    return false;
   }
 
   private final FrequencyTrackingRingBuffer recentlyUsedFilters;
 
   /**
-   * Create a new instance.
+   * Expert: Create a new instance with a configurable history size. Beware of
+   * passing too large values for the size of the history, either
+   * {@link #minFrequencyToCache} returns low values and this means some filters
+   * that are rarely used will be cached, which would hurt performance. Or
+   * {@link #minFrequencyToCache} returns high values that are function of the
+   * size of the history but then filters will be slow to make it to the cache.
    *
    * @param historySize               the number of recently used filters to track
    */
@@ -71,20 +104,22 @@ public final class UsageTrackingQueryCachingPolicy implements QueryCachingPolicy
     this.recentlyUsedFilters = new FrequencyTrackingRingBuffer(historySize, SENTINEL);
   }
 
-  /** Create a new instance with an history size of 256. */
+  /** Create a new instance with an history size of 256. This should be a good
+   *  default for most cases. */
   public UsageTrackingQueryCachingPolicy() {
     this(256);
   }
 
   /**
-   * For a given query, return how many times it should appear in the history
-   * before being cached.
+   * For a given filter, return how many times it should appear in the history
+   * before being cached. The default implementation returns 2 for filters that
+   * need to evaluate against the entire index to build a {@link DocIdSetIterator},
+   * like {@link MultiTermQuery}, point-based queries or {@link TermInSetQuery},
+   * and 5 for other filters.
    */
   protected int minFrequencyToCache(Query query) {
     if (isCostly(query)) {
       return 2;
-    } else if (isCheap(query)) {
-      return 20;
     } else {
       // default: cache after the filter has been seen 5 times
       return 5;
@@ -96,6 +131,10 @@ public final class UsageTrackingQueryCachingPolicy implements QueryCachingPolicy
     assert query instanceof BoostQuery == false;
     assert query instanceof ConstantScoreQuery == false;
 
+    if (shouldNeverCache(query)) {
+      return;
+    }
+
     // call hashCode outside of sync block
     // in case it's somewhat expensive:
     int hashCode = query.hashCode();
@@ -123,24 +162,9 @@ public final class UsageTrackingQueryCachingPolicy implements QueryCachingPolicy
 
   @Override
   public boolean shouldCache(Query query) throws IOException {
-    if (query instanceof MatchAllDocsQuery
-        // MatchNoDocsQuery currently rewrites to a BooleanQuery,
-        // but who knows, it might get its own Weight one day
-        || query instanceof MatchNoDocsQuery) {
+    if (shouldNeverCache(query)) {
       return false;
     }
-    if (query instanceof BooleanQuery) {
-      BooleanQuery bq = (BooleanQuery) query;
-      if (bq.clauses().isEmpty()) {
-        return false;
-      }
-    }
-    if (query instanceof DisjunctionMaxQuery) {
-      DisjunctionMaxQuery dmq = (DisjunctionMaxQuery) query;
-      if (dmq.getDisjuncts().isEmpty()) {
-        return false;
-      }
-    }
     final int frequency = frequency(query);
     final int minFrequency = minFrequencyToCache(query);
     return frequency >= minFrequency;

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/5682c33b/lucene/core/src/test/org/apache/lucene/search/TestUsageTrackingFilterCachingPolicy.java
----------------------------------------------------------------------
diff --git a/lucene/core/src/test/org/apache/lucene/search/TestUsageTrackingFilterCachingPolicy.java b/lucene/core/src/test/org/apache/lucene/search/TestUsageTrackingFilterCachingPolicy.java
index 29ed22f..eed3cb7 100644
--- a/lucene/core/src/test/org/apache/lucene/search/TestUsageTrackingFilterCachingPolicy.java
+++ b/lucene/core/src/test/org/apache/lucene/search/TestUsageTrackingFilterCachingPolicy.java
@@ -16,6 +16,8 @@
  */
 package org.apache.lucene.search;
 
+import java.io.IOException;
+
 import org.apache.lucene.document.IntPoint;
 import org.apache.lucene.index.Term;
 import org.apache.lucene.util.LuceneTestCase;
@@ -37,4 +39,13 @@ public class TestUsageTrackingFilterCachingPolicy extends LuceneTestCase {
     assertFalse(policy.shouldCache(q));
   }
 
+  public void testNeverCacheTermFilter() throws IOException {
+    Query q = new TermQuery(new Term("foo", "bar"));
+    UsageTrackingQueryCachingPolicy policy = new UsageTrackingQueryCachingPolicy();
+    for (int i = 0; i < 1000; ++i) {
+      policy.onUse(q);
+    }
+    assertFalse(policy.shouldCache(q));
+  }
+
 }