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 2015/02/27 19:09:02 UTC

svn commit: r1662774 [1/2] - in /lucene/dev/trunk: lucene/ lucene/core/src/java/org/apache/lucene/index/ lucene/core/src/java/org/apache/lucene/search/ lucene/core/src/java/org/apache/lucene/util/ lucene/core/src/java/org/apache/lucene/util/packed/ luc...

Author: jpountz
Date: Fri Feb 27 18:09:01 2015
New Revision: 1662774

URL: http://svn.apache.org/r1662774
Log:
LUCENE-6303: CachingWrapperFilter -> CachingWrapperQuery, FilterCache -> QueryCache and added caching to IndexSearcher.

Added:
    lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/search/CachingWrapperQuery.java   (with props)
    lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/search/LRUQueryCache.java   (with props)
    lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/search/QueryCache.java   (with props)
    lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/search/QueryCachingPolicy.java   (with props)
    lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/search/UsageTrackingQueryCachingPolicy.java   (with props)
    lucene/dev/trunk/lucene/core/src/test/org/apache/lucene/search/TestCachingWrapperQuery.java   (with props)
    lucene/dev/trunk/lucene/core/src/test/org/apache/lucene/search/TestLRUQueryCache.java   (with props)
Removed:
    lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/search/CachingWrapperFilter.java
    lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/search/FilterCache.java
    lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/search/FilterCachingPolicy.java
    lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/search/LRUFilterCache.java
    lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/search/UsageTrackingFilterCachingPolicy.java
    lucene/dev/trunk/lucene/core/src/test/org/apache/lucene/search/TestCachingWrapperFilter.java
    lucene/dev/trunk/lucene/core/src/test/org/apache/lucene/search/TestLRUFilterCache.java
Modified:
    lucene/dev/trunk/lucene/CHANGES.txt
    lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/index/FilterLeafReader.java
    lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/search/BooleanQuery.java
    lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/search/BooleanWeight.java
    lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/search/ConstantScoreQuery.java
    lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/search/DocIdSet.java
    lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/search/FilteredDocIdSet.java
    lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/search/FilteredQuery.java
    lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/search/IndexSearcher.java
    lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/util/BitDocIdSet.java
    lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/util/NotDocIdSet.java
    lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/util/RoaringDocIdSet.java
    lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/util/packed/EliasFanoDocIdSet.java
    lucene/dev/trunk/lucene/core/src/test/org/apache/lucene/index/TestOmitTf.java
    lucene/dev/trunk/lucene/core/src/test/org/apache/lucene/search/TestConstantScoreQuery.java
    lucene/dev/trunk/lucene/core/src/test/org/apache/lucene/search/TestDocValuesRangeQuery.java
    lucene/dev/trunk/lucene/core/src/test/org/apache/lucene/search/TestFilterCachingPolicy.java
    lucene/dev/trunk/lucene/core/src/test/org/apache/lucene/search/TestFilteredQuery.java
    lucene/dev/trunk/lucene/core/src/test/org/apache/lucene/search/TestPrefixRandom.java
    lucene/dev/trunk/lucene/core/src/test/org/apache/lucene/search/TestQueryWrapperFilter.java
    lucene/dev/trunk/lucene/core/src/test/org/apache/lucene/search/TestRegexpRandom2.java
    lucene/dev/trunk/lucene/core/src/test/org/apache/lucene/search/TestUsageTrackingFilterCachingPolicy.java
    lucene/dev/trunk/lucene/core/src/test/org/apache/lucene/util/TestNotDocIdSet.java
    lucene/dev/trunk/lucene/facet/src/java/org/apache/lucene/facet/DrillSideways.java
    lucene/dev/trunk/lucene/facet/src/java/org/apache/lucene/facet/DrillSidewaysQuery.java
    lucene/dev/trunk/lucene/facet/src/test/org/apache/lucene/facet/range/TestRangeFacetCounts.java
    lucene/dev/trunk/lucene/grouping/src/test/org/apache/lucene/search/grouping/GroupingSearchTest.java
    lucene/dev/trunk/lucene/grouping/src/test/org/apache/lucene/search/grouping/TestGrouping.java
    lucene/dev/trunk/lucene/join/src/java/org/apache/lucene/search/join/BitDocIdSetCachingWrapperFilter.java
    lucene/dev/trunk/lucene/join/src/java/org/apache/lucene/search/join/ToParentBlockJoinCollector.java
    lucene/dev/trunk/lucene/join/src/test/org/apache/lucene/search/join/TestBlockJoinValidation.java
    lucene/dev/trunk/lucene/misc/src/test/org/apache/lucene/index/TestBlockJoinSorter.java
    lucene/dev/trunk/lucene/queryparser/src/java/org/apache/lucene/queryparser/xml/builders/CachedFilterBuilder.java
    lucene/dev/trunk/lucene/test-framework/src/java/org/apache/lucene/util/LuceneTestCase.java
    lucene/dev/trunk/solr/core/src/java/org/apache/solr/search/BitDocSet.java
    lucene/dev/trunk/solr/core/src/java/org/apache/solr/search/DocSetBase.java
    lucene/dev/trunk/solr/core/src/java/org/apache/solr/search/LuceneQueryOptimizer.java
    lucene/dev/trunk/solr/core/src/java/org/apache/solr/search/SortedIntDocSet.java

Modified: lucene/dev/trunk/lucene/CHANGES.txt
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/CHANGES.txt?rev=1662774&r1=1662773&r2=1662774&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/CHANGES.txt (original)
+++ lucene/dev/trunk/lucene/CHANGES.txt Fri Feb 27 18:09:01 2015
@@ -51,6 +51,9 @@ New Features
 * LUCENE-6294: Added oal.search.CollectorManager to allow for parallelization
   of the document collection process on IndexSearcher. (Adrien Grand)
 
+* LUCENE-6303: Added filter caching baked into IndexSearcher and enabled by
+  default. (Adrien Grand)
+
 Bug Fixes
 
 * LUCENE-6249: StandardQueryParser doesn't support pure negative clauses. 
@@ -179,6 +182,9 @@ API Changes
   removed. Use PrefixQuery, TermRangeQuery and NumericRangeQuery instead.
   (Adrien Grand)
 
+* LUCENE-6303: Replaced FilterCache with QueryCache and CachingWrapperFilter
+  with CachingWrapperQuery. (Adrien Grand)
+
 Other
 
 * LUCENE-6248: Remove unused odd constants from StandardSyntaxParser.jj

Modified: lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/index/FilterLeafReader.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/index/FilterLeafReader.java?rev=1662774&r1=1662773&r2=1662774&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/index/FilterLeafReader.java (original)
+++ lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/index/FilterLeafReader.java Fri Feb 27 18:09:01 2015
@@ -20,7 +20,7 @@ package org.apache.lucene.index;
 import java.io.IOException;
 import java.util.Iterator;
 
-import org.apache.lucene.search.CachingWrapperFilter;
+import org.apache.lucene.search.CachingWrapperQuery;
 import org.apache.lucene.util.AttributeSource;
 import org.apache.lucene.util.Bits;
 import org.apache.lucene.util.BytesRef;
@@ -38,7 +38,7 @@ import org.apache.lucene.util.BytesRef;
  * <p><b>NOTE</b>: If this {@link FilterLeafReader} does not change the
  * content the contained reader, you could consider overriding
  * {@link #getCoreCacheKey()} so that
- * {@link CachingWrapperFilter} shares the same entries for this atomic reader
+ * {@link CachingWrapperQuery} shares the same entries for this atomic reader
  * and the wrapped one. {@link #getCombinedCoreAndDeletesKey()} could be
  * overridden as well if the {@link #getLiveDocs() live docs} are not changed
  * either.

Modified: lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/search/BooleanQuery.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/search/BooleanQuery.java?rev=1662774&r1=1662773&r2=1662774&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/search/BooleanQuery.java (original)
+++ lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/search/BooleanQuery.java Fri Feb 27 18:09:01 2015
@@ -25,6 +25,7 @@ import java.util.Set;
 
 import org.apache.lucene.index.IndexReader;
 import org.apache.lucene.index.Term;
+import org.apache.lucene.search.BooleanClause.Occur;
 import org.apache.lucene.search.similarities.Similarity;
 import org.apache.lucene.util.ToStringUtils;
 
@@ -158,9 +159,27 @@ public class BooleanQuery extends Query
   @Override
   public final Iterator<BooleanClause> iterator() { return clauses().iterator(); }
 
+  private static BooleanQuery downgradeMustClauseToFilter(BooleanQuery bq) {
+    BooleanQuery clone = bq.clone();
+    clone.clauses.clear();
+    for (BooleanClause clause : bq.clauses()) {
+      if (clause.getOccur() == Occur.MUST) {
+        clone.add(clause.getQuery(), Occur.FILTER);
+      } else {
+        clone.add(clause);
+      }
+    }
+    return clone;
+  }
+
   @Override
   public Weight createWeight(IndexSearcher searcher, boolean needsScores) throws IOException {
-    return new BooleanWeight(this, searcher, needsScores, disableCoord);
+    BooleanQuery query = this;
+    if (needsScores == false) {
+      // we rewrite MUST clauses to FILTER for caching
+      query = downgradeMustClauseToFilter(query);
+    }
+    return new BooleanWeight(query, searcher, needsScores, disableCoord);
   }
 
   @Override

Modified: lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/search/BooleanWeight.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/search/BooleanWeight.java?rev=1662774&r1=1662773&r2=1662774&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/search/BooleanWeight.java (original)
+++ lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/search/BooleanWeight.java Fri Feb 27 18:09:01 2015
@@ -52,7 +52,7 @@ public class BooleanWeight extends Weigh
     weights = new ArrayList<>(query.clauses().size());
     for (int i = 0 ; i < query.clauses().size(); i++) {
       BooleanClause c = query.clauses().get(i);
-      Weight w = c.getQuery().createWeight(searcher, needsScores && c.isScoring());
+      Weight w = searcher.createWeight(c.getQuery(), needsScores && c.isScoring());
       weights.add(w);
       if (c.isScoring()) {
         maxCoord++;

Added: lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/search/CachingWrapperQuery.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/search/CachingWrapperQuery.java?rev=1662774&view=auto
==============================================================================
--- lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/search/CachingWrapperQuery.java (added)
+++ lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/search/CachingWrapperQuery.java Fri Feb 27 18:09:01 2015
@@ -0,0 +1,253 @@
+package org.apache.lucene.search;
+
+/*
+ * 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 java.util.ArrayList;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.List;
+import java.util.Map;
+import java.util.WeakHashMap;
+
+import org.apache.lucene.index.IndexReader;
+import org.apache.lucene.index.LeafReader;
+import org.apache.lucene.index.LeafReaderContext;
+import org.apache.lucene.util.Accountable;
+import org.apache.lucene.util.Accountables;
+import org.apache.lucene.util.Bits;
+import org.apache.lucene.util.RoaringDocIdSet;
+
+/**
+ * Wraps another {@link Query}'s result and caches it when scores are not
+ * needed.  The purpose is to allow queries to simply care about matching and
+ * scoring, and then wrap with this class to add caching.
+ */
+public class CachingWrapperQuery extends Query implements Accountable {
+  private Query query; // not final because of clone
+  private final QueryCachingPolicy policy;
+  private final Map<Object,DocIdSet> cache = Collections.synchronizedMap(new WeakHashMap<Object,DocIdSet>());
+
+  /** Wraps another query's result and caches it according to the provided policy.
+   * @param query Query to cache results of
+   * @param policy policy defining which filters should be cached on which segments
+   */
+  public CachingWrapperQuery(Query query, QueryCachingPolicy policy) {
+    this.query = query;
+    this.policy = policy;
+  }
+
+  /** Same as {@link CachingWrapperQuery#CachingWrapperQuery(Query, QueryCachingPolicy)}
+   *  but enforces the use of the
+   *  {@link QueryCachingPolicy.CacheOnLargeSegments#DEFAULT} policy. */
+  public CachingWrapperQuery(Query query) {
+    this(query, QueryCachingPolicy.CacheOnLargeSegments.DEFAULT);
+  }
+
+  @Override
+  public CachingWrapperQuery clone() {
+    final CachingWrapperQuery clone = (CachingWrapperQuery) super.clone();
+    clone.query = query.clone();
+    return clone;
+  }
+
+  /**
+   * Gets the contained query.
+   * @return the contained query.
+   */
+  public Query getQuery() {
+    return query;
+  }
+  
+  @Override
+  public float getBoost() {
+    return query.getBoost();
+  }
+  
+  @Override
+  public void setBoost(float b) {
+    query.setBoost(b);
+  }
+  
+  /**
+   * Default cache implementation: uses {@link RoaringDocIdSet}.
+   */
+  protected DocIdSet cacheImpl(DocIdSetIterator iterator, LeafReader reader) throws IOException {
+    return new RoaringDocIdSet.Builder(reader.maxDoc()).add(iterator).build();
+  }
+
+  @Override
+  public Query rewrite(IndexReader reader) throws IOException {
+    final Query rewritten = query.rewrite(reader);
+    if (query == rewritten) {
+      return this;
+    } else {
+      CachingWrapperQuery clone = clone();
+      clone.query = rewritten;
+      return clone;
+    }
+  }
+
+  // for testing
+  int hitCount, missCount;
+
+  @Override
+  public Weight createWeight(IndexSearcher searcher, boolean needsScores) throws IOException {
+    final Weight weight = query.createWeight(searcher, needsScores);
+    if (needsScores) {
+      // our cache is not sufficient, we need scores too
+      return weight;
+    }
+    policy.onUse(weight.getQuery());
+    return new ConstantScoreWeight(weight.getQuery()) {
+      @Override
+      Scorer scorer(LeafReaderContext context, Bits acceptDocs, float score) throws IOException {
+        final LeafReader reader = context.reader();
+        final Object key = reader.getCoreCacheKey();
+
+        DocIdSet docIdSet = cache.get(key);
+        if (docIdSet != null) {
+          hitCount++;
+        } else if (policy.shouldCache(query, context)) {
+          missCount++;
+          final Scorer scorer = weight.scorer(context, null);
+          if (scorer == null) {
+            docIdSet = DocIdSet.EMPTY;
+          } else {
+            docIdSet = cacheImpl(scorer, context.reader());
+          }
+          cache.put(key, docIdSet);
+        } else {
+          return weight.scorer(context, acceptDocs);
+        }
+
+        assert docIdSet != null;
+        if (docIdSet == DocIdSet.EMPTY) {
+          return null;
+        }
+        final DocIdSetIterator approximation = docIdSet.iterator();
+        if (approximation == null) {
+          return null;
+        }
+
+        final DocIdSetIterator disi;
+        final TwoPhaseIterator twoPhaseView;
+        if (acceptDocs == null) {
+          twoPhaseView = null;
+          disi = approximation;
+        } else {
+          twoPhaseView = new TwoPhaseIterator() {
+            
+            @Override
+            public boolean matches() throws IOException {
+              final int doc = approximation.docID();
+              return acceptDocs.get(doc);
+            }
+            
+            @Override
+            public DocIdSetIterator approximation() {
+              return approximation;
+            }
+          };
+          disi = TwoPhaseIterator.asDocIdSetIterator(twoPhaseView);
+        }
+        return new Scorer(weight) {
+
+          @Override
+          public TwoPhaseIterator asTwoPhaseIterator() {
+            return twoPhaseView;
+          }
+
+          @Override
+          public float score() throws IOException {
+            return 0f;
+          }
+
+          @Override
+          public int freq() throws IOException {
+            return 1;
+          }
+
+          @Override
+          public int docID() {
+            return disi.docID();
+          }
+
+          @Override
+          public int nextDoc() throws IOException {
+            return disi.nextDoc();
+          }
+
+          @Override
+          public int advance(int target) throws IOException {
+            return disi.advance(target);
+          }
+
+          @Override
+          public long cost() {
+            return disi.cost();
+          }
+          
+        };
+      }
+    };
+  }
+  
+  @Override
+  public String toString(String field) {
+    return getClass().getSimpleName() + "("+query.toString(field)+")";
+  }
+
+  @Override
+  public boolean equals(Object o) {
+    if (o == null || !getClass().equals(o.getClass())) return false;
+    final CachingWrapperQuery other = (CachingWrapperQuery) o;
+    return this.query.equals(other.query);
+  }
+
+  @Override
+  public int hashCode() {
+    return (query.hashCode() ^ getClass().hashCode());
+  }
+
+  @Override
+  public long ramBytesUsed() {
+
+    // Sync only to pull the current set of values:
+    List<DocIdSet> docIdSets;
+    synchronized(cache) {
+      docIdSets = new ArrayList<>(cache.values());
+    }
+
+    long total = 0;
+    for(DocIdSet dis : docIdSets) {
+      total += dis.ramBytesUsed();
+    }
+
+    return total;
+  }
+
+  @Override
+  public Collection<Accountable> getChildResources() {
+    // Sync to pull the current set of values:
+    synchronized (cache) {
+      // no need to clone, Accountable#namedAccountables already copies the data
+      return Accountables.namedAccountables("segment", cache);
+    }
+  }
+}

Modified: lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/search/ConstantScoreQuery.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/search/ConstantScoreQuery.java?rev=1662774&r1=1662773&r2=1662774&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/search/ConstantScoreQuery.java (original)
+++ lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/search/ConstantScoreQuery.java Fri Feb 27 18:09:01 2015
@@ -256,7 +256,7 @@ public class ConstantScoreQuery extends
 
   @Override
   public Weight createWeight(IndexSearcher searcher, boolean needsScores) throws IOException {
-    final Weight innerWeight = query.createWeight(searcher, false);
+    final Weight innerWeight = searcher.createWeight(query, false);
     if (needsScores) {
       return new ConstantScoreQuery.ConstantWeight(innerWeight);
     } else {

Modified: lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/search/DocIdSet.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/search/DocIdSet.java?rev=1662774&r1=1662773&r2=1662774&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/search/DocIdSet.java (original)
+++ lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/search/DocIdSet.java Fri Feb 27 18:09:01 2015
@@ -36,11 +36,6 @@ public abstract class DocIdSet implement
       return DocIdSetIterator.empty();
     }
     
-    @Override
-    public boolean isCacheable() {
-      return true;
-    }
-    
     // we explicitly provide no random access, as this filter is 100% sparse and iterator exits faster
     @Override
     public Bits bits() {
@@ -82,14 +77,4 @@ public abstract class DocIdSet implement
     return null;
   }
 
-  /**
-   * This method is a hint for {@link CachingWrapperFilter}, if this <code>DocIdSet</code>
-   * should be cached without copying it. The default is to return
-   * <code>false</code>. If you have an own <code>DocIdSet</code> implementation
-   * that does its iteration very effective and fast without doing disk I/O,
-   * override this method and return <code>true</code>.
-   */
-  public boolean isCacheable() {
-    return false;
-  }
 }

Modified: lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/search/FilteredDocIdSet.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/search/FilteredDocIdSet.java?rev=1662774&r1=1662773&r2=1662774&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/search/FilteredDocIdSet.java (original)
+++ lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/search/FilteredDocIdSet.java Fri Feb 27 18:09:01 2015
@@ -58,12 +58,6 @@ public abstract class FilteredDocIdSet e
     return _innerSet;
   }
 
-  /** This DocIdSet implementation is cacheable if the inner set is cacheable. */
-  @Override
-  public boolean isCacheable() {
-    return _innerSet.isCacheable();
-  }
-
   @Override
   public long ramBytesUsed() {
     return RamUsageEstimator.NUM_BYTES_OBJECT_REF + _innerSet.ramBytesUsed();

Modified: lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/search/FilteredQuery.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/search/FilteredQuery.java?rev=1662774&r1=1662773&r2=1662774&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/search/FilteredQuery.java (original)
+++ lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/search/FilteredQuery.java Fri Feb 27 18:09:01 2015
@@ -37,7 +37,7 @@ import org.apache.lucene.util.ToStringUt
  * query is used in a search - use a CachingWrapperFilter to avoid
  * regenerating the bits every time.
  * @since   1.4
- * @see     CachingWrapperFilter
+ * @see     CachingWrapperQuery
  */
 public class FilteredQuery extends Query {
 

Modified: lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/search/IndexSearcher.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/search/IndexSearcher.java?rev=1662774&r1=1662773&r2=1662774&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/search/IndexSearcher.java (original)
+++ lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/search/IndexSearcher.java Fri Feb 27 18:09:01 2015
@@ -23,6 +23,7 @@ import java.util.Arrays;
 import java.util.Collection;
 import java.util.Collections;
 import java.util.List;
+import java.util.Objects;
 import java.util.Set;
 import java.util.concurrent.Callable;
 import java.util.concurrent.ExecutionException;
@@ -72,6 +73,11 @@ import org.apache.lucene.util.ThreadInte
  * use your own (non-Lucene) objects instead.</p>
  */
 public class IndexSearcher {
+
+  // 32MB and at most 10,000 queries
+  private static final QueryCache DEFAULT_QUERY_CACHE = new LRUQueryCache(10000, 1 << 25);
+  private static final QueryCachingPolicy DEFAULT_CACHING_POLICY = new UsageTrackingQueryCachingPolicy();
+
   final IndexReader reader; // package private for testing!
   
   // NOTE: these members might change in incompatible ways
@@ -86,7 +92,10 @@ public class IndexSearcher {
 
   // the default Similarity
   private static final Similarity defaultSimilarity = new DefaultSimilarity();
-  
+
+  private QueryCache queryCache = DEFAULT_QUERY_CACHE;
+  private QueryCachingPolicy queryCachingPolicy = DEFAULT_CACHING_POLICY;
+
   /**
    * Expert: returns a default Similarity instance.
    * In general, this method is only called to initialize searchers and writers.
@@ -156,7 +165,28 @@ public class IndexSearcher {
   public IndexSearcher(IndexReaderContext context) {
     this(context, null);
   }
-  
+
+  /**
+   * Set the {@link QueryCache} to use when scores are not needed.
+   * A value of {@code null} indicates that query matches should never be
+   * cached. This method should be called <b>before</b> starting using this
+   * {@link IndexSearcher}.
+   * @see QueryCache
+   */
+  public void setQueryCache(QueryCache queryCache) {
+    this.queryCache = queryCache;
+  }
+
+  /**
+   * Set the {@link QueryCachingPolicy} to use for query caching.
+   * This method should be called <b>before</b> starting using this
+   * {@link IndexSearcher}.
+   * @see QueryCachingPolicy
+   */
+  public void setQueryCachingPolicy(QueryCachingPolicy queryCachingPolicy) {
+    this.queryCachingPolicy = Objects.requireNonNull(queryCachingPolicy);
+  }
+
   /**
    * Expert: Creates an array of leaf slices each holding a subset of the given leaves.
    * Each {@link LeafSlice} is executed in a single thread. By default there
@@ -557,7 +587,7 @@ public class IndexSearcher {
    */
   public Weight createNormalizedWeight(Query query, boolean needsScores) throws IOException {
     query = rewrite(query);
-    Weight weight = query.createWeight(this, needsScores);
+    Weight weight = createWeight(query, needsScores);
     float v = weight.getValueForNormalization();
     float norm = getSimilarity().queryNorm(v);
     if (Float.isInfinite(norm) || Float.isNaN(norm)) {
@@ -566,7 +596,21 @@ public class IndexSearcher {
     weight.normalize(norm, 1.0f);
     return weight;
   }
-  
+
+  /**
+   * Creates a {@link Weight} for the given query, potentially adding caching
+   * if possible and configured.
+   * @lucene.experimental
+   */
+  public Weight createWeight(Query query, boolean needsScores) throws IOException {
+    final QueryCache queryCache = this.queryCache;
+    Weight weight = query.createWeight(this, needsScores);
+    if (needsScores == false && queryCache != null) {
+      weight = queryCache.doCache(weight, queryCachingPolicy);
+    }
+    return weight;
+  }
+
   /**
    * Returns this searchers the top-level {@link IndexReaderContext}.
    * @see IndexReader#getContext()

Added: lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/search/LRUQueryCache.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/search/LRUQueryCache.java?rev=1662774&view=auto
==============================================================================
--- lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/search/LRUQueryCache.java (added)
+++ lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/search/LRUQueryCache.java Fri Feb 27 18:09:01 2015
@@ -0,0 +1,629 @@
+package org.apache.lucene.search;
+
+/*
+ * 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 java.util.ArrayList;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.IdentityHashMap;
+import java.util.Iterator;
+import java.util.LinkedHashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+
+import org.apache.lucene.index.LeafReader;
+import org.apache.lucene.index.LeafReader.CoreClosedListener;
+import org.apache.lucene.index.LeafReaderContext;
+import org.apache.lucene.util.Accountable;
+import org.apache.lucene.util.Accountables;
+import org.apache.lucene.util.Bits;
+import org.apache.lucene.util.RamUsageEstimator;
+import org.apache.lucene.util.RoaringDocIdSet;
+
+/**
+ * A {@link QueryCache} that evicts queries using a LRU (least-recently-used)
+ * eviction policy in order to remain under a given maximum size and number of
+ * bytes used.
+ *
+ * This class is thread-safe.
+ *
+ * Note that query eviction runs in linear time with the total number of
+ * segments that have cache entries so this cache works best with
+ * {@link QueryCachingPolicy caching policies} that only cache on "large"
+ * segments, and it is advised to not share this cache across too many indices.
+ *
+ * Typical usage looks like this:
+ * <pre class="prettyprint">
+ *   final int maxNumberOfCachedQueries = 256;
+ *   final long maxRamBytesUsed = 50 * 1024L * 1024L; // 50MB
+ *   // these cache and policy instances can be shared across several queries and readers
+ *   // it is fine to eg. store them into static variables
+ *   final QueryCache queryCache = new LRUQueryCache(maxNumberOfCachedQueries, maxRamBytesUsed);
+ *   final QueryCachingPolicy defaultCachingPolicy = new UsageTrackingQueryCachingPolicy();
+ *
+ *   // ...
+ *
+ *   // Then at search time
+ *   Query myQuery = ...;
+ *   Query myCacheQuery = queryCache.doCache(myQuery, defaultCachingPolicy);
+ *   // myCacheQuery is now a wrapper around the original query that will interact with the cache
+ *   IndexSearcher searcher = ...;
+ *   TopDocs topDocs = searcher.search(new ConstantScoreQuery(myCacheQuery), 10);
+ * </pre>
+ *
+ * This cache exposes some global statistics ({@link #getHitCount() hit count},
+ * {@link #getMissCount() miss count}, {@link #getCacheSize() number of cache
+ * entries}, {@link #getCacheCount() total number of DocIdSets that have ever
+ * been cached}, {@link #getEvictionCount() number of evicted entries}). In
+ * case you would like to have more fine-grained statistics, such as per-index
+ * or per-query-class statistics, it is possible to override various callbacks:
+ * {@link #onHit}, {@link #onMiss},
+ * {@link #onQueryCache}, {@link #onQueryEviction},
+ * {@link #onDocIdSetCache}, {@link #onDocIdSetEviction} and {@link #onClear}.
+ * It is better to not perform heavy computations in these methods though since
+ * they are called synchronously and under a lock.
+ *
+ * @see QueryCachingPolicy
+ * @lucene.experimental
+ */
+public class LRUQueryCache implements QueryCache, Accountable {
+
+  // memory usage of a simple term query
+  static final long QUERY_DEFAULT_RAM_BYTES_USED = 192;
+
+  static final long HASHTABLE_RAM_BYTES_PER_ENTRY =
+      2 * RamUsageEstimator.NUM_BYTES_OBJECT_REF // key + value
+      * 2; // hash tables need to be oversized to avoid collisions, assume 2x capacity
+
+  static final long LINKED_HASHTABLE_RAM_BYTES_PER_ENTRY =
+      HASHTABLE_RAM_BYTES_PER_ENTRY
+      + 2 * RamUsageEstimator.NUM_BYTES_OBJECT_REF; // previous & next references
+
+  private final int maxSize;
+  private final long maxRamBytesUsed;
+  // maps queries that are contained in the cache to a singleton so that this
+  // cache does not store several copies of the same query
+  private final Map<Query, Query> uniqueQueries;
+  // The contract between this set and the per-leaf caches is that per-leaf caches
+  // are only allowed to store sub-sets of the queries that are contained in
+  // mostRecentlyUsedQueries. This is why write operations are performed under a lock
+  private final Set<Query> mostRecentlyUsedQueries;
+  private final Map<Object, LeafCache> cache;
+
+  // these variables are volatile so that we do not need to sync reads
+  // but increments need to be performed under the lock
+  private volatile long ramBytesUsed;
+  private volatile long hitCount;
+  private volatile long missCount;
+  private volatile long cacheCount;
+  private volatile long cacheSize;
+
+  /**
+   * Create a new instance that will cache at most <code>maxSize</code> queries
+   * with at most <code>maxRamBytesUsed</code> bytes of memory.
+   */
+  public LRUQueryCache(int maxSize, long maxRamBytesUsed) {
+    this.maxSize = maxSize;
+    this.maxRamBytesUsed = maxRamBytesUsed;
+    uniqueQueries = new LinkedHashMap<>(16, 0.75f, true);
+    mostRecentlyUsedQueries = uniqueQueries.keySet();
+    cache = new IdentityHashMap<>();
+    ramBytesUsed = 0;
+  }
+
+  /**
+   * Expert: callback when there is a cache hit on a given query.
+   * Implementing this method is typically useful in order to compute more
+   * fine-grained statistics about the query cache.
+   * @see #onMiss
+   * @lucene.experimental
+   */
+  protected void onHit(Object readerCoreKey, Query query) {
+    hitCount += 1;
+  }
+
+  /**
+   * Expert: callback when there is a cache miss on a given query.
+   * @see #onHit
+   * @lucene.experimental
+   */
+  protected void onMiss(Object readerCoreKey, Query query) {
+    assert query != null;
+    missCount += 1;
+  }
+
+  /**
+   * Expert: callback when a query is added to this cache.
+   * Implementing this method is typically useful in order to compute more
+   * fine-grained statistics about the query cache.
+   * @see #onQueryEviction
+   * @lucene.experimental
+   */
+  protected void onQueryCache(Query query, long ramBytesUsed) {
+    this.ramBytesUsed += ramBytesUsed;
+  }
+
+  /**
+   * Expert: callback when a query is evicted from this cache.
+   * @see #onQueryCache
+   * @lucene.experimental
+   */
+  protected void onQueryEviction(Query query, long ramBytesUsed) {
+    this.ramBytesUsed -= ramBytesUsed;
+  }
+
+  /**
+   * Expert: callback when a {@link DocIdSet} is added to this cache.
+   * Implementing this method is typically useful in order to compute more
+   * fine-grained statistics about the query cache.
+   * @see #onDocIdSetEviction
+   * @lucene.experimental
+   */
+  protected void onDocIdSetCache(Object readerCoreKey, long ramBytesUsed) {
+    cacheSize += 1;
+    cacheCount += 1;
+    this.ramBytesUsed += ramBytesUsed;
+  }
+
+  /**
+   * Expert: callback when one or more {@link DocIdSet}s are removed from this
+   * cache.
+   * @see #onDocIdSetCache
+   * @lucene.experimental
+   */
+  protected void onDocIdSetEviction(Object readerCoreKey, int numEntries, long sumRamBytesUsed) {
+    this.ramBytesUsed -= sumRamBytesUsed;
+    cacheSize -= numEntries;
+  }
+
+  /**
+   * Expert: callback when the cache is completely cleared.
+   * @lucene.experimental
+   */
+  protected void onClear() {
+    ramBytesUsed = 0;
+    cacheSize = 0;
+  }
+
+  /** Whether evictions are required. */
+  boolean requiresEviction() {
+    final int size = mostRecentlyUsedQueries.size();
+    if (size == 0) {
+      return false;
+    } else {
+      return size > maxSize || ramBytesUsed() > maxRamBytesUsed;
+    }
+  }
+
+  synchronized DocIdSet get(Query key, LeafReaderContext context) {
+    key = QueryCache.cacheKey(key);
+    final Object readerKey = context.reader().getCoreCacheKey();
+    final LeafCache leafCache = cache.get(readerKey);
+    if (leafCache == null) {
+      onMiss(readerKey, key);
+      return null;
+    }
+    // this get call moves the query to the most-recently-used position
+    final Query singleton = uniqueQueries.get(key);
+    if (singleton == null) {
+      onMiss(readerKey, key);
+      return null;
+    }
+    final DocIdSet cached = leafCache.get(singleton);
+    if (cached == null) {
+      onMiss(readerKey, singleton);
+    } else {
+      onHit(readerKey, singleton);
+    }
+    return cached;
+  }
+
+  synchronized void putIfAbsent(Query query, LeafReaderContext context, DocIdSet set) {
+    // under a lock to make sure that mostRecentlyUsedQueries and cache remain sync'ed
+    // we don't want to have user-provided queries as keys in our cache since queries are mutable
+    query = query.clone();
+    query.setBoost(1f);
+    assert query == QueryCache.cacheKey(query);
+    Query singleton = uniqueQueries.putIfAbsent(query, query);
+    if (singleton == null) {
+      onQueryCache(singleton, LINKED_HASHTABLE_RAM_BYTES_PER_ENTRY + ramBytesUsed(query));
+    } else {
+      query = singleton;
+    }
+    final Object key = context.reader().getCoreCacheKey();
+    LeafCache leafCache = cache.get(key);
+    if (leafCache == null) {
+      leafCache = new LeafCache(key);
+      final LeafCache previous = cache.put(context.reader().getCoreCacheKey(), leafCache);
+      ramBytesUsed += HASHTABLE_RAM_BYTES_PER_ENTRY;
+      assert previous == null;
+      // we just created a new leaf cache, need to register a close listener
+      context.reader().addCoreClosedListener(new CoreClosedListener() {
+        @Override
+        public void onClose(Object ownerCoreCacheKey) {
+          clearCoreCacheKey(ownerCoreCacheKey);
+        }
+      });
+    }
+    leafCache.putIfAbsent(query, set);
+    evictIfNecessary();
+  }
+
+  synchronized void evictIfNecessary() {
+    // under a lock to make sure that mostRecentlyUsedQueries and cache keep sync'ed
+    if (requiresEviction()) {
+      Iterator<Query> iterator = mostRecentlyUsedQueries.iterator();
+      do {
+        final Query query = iterator.next();
+        iterator.remove();
+        onEviction(query);
+      } while (iterator.hasNext() && requiresEviction());
+    }
+  }
+
+  /**
+   * Remove all cache entries for the given core cache key.
+   */
+  public synchronized void clearCoreCacheKey(Object coreKey) {
+    final LeafCache leafCache = cache.remove(coreKey);
+    if (leafCache != null) {
+      ramBytesUsed -= HASHTABLE_RAM_BYTES_PER_ENTRY;
+      onDocIdSetEviction(coreKey, leafCache.cache.size(), leafCache.ramBytesUsed);
+    }
+  }
+
+  /**
+   * Remove all cache entries for the given query.
+   */
+  public synchronized void clearQuery(Query query) {
+    final Query singleton = uniqueQueries.remove(QueryCache.cacheKey(query));
+    if (singleton != null) {
+      onEviction(singleton);
+    }
+  }
+
+  private void onEviction(Query singleton) {
+    onQueryEviction(singleton, LINKED_HASHTABLE_RAM_BYTES_PER_ENTRY + ramBytesUsed(singleton));
+    for (LeafCache leafCache : cache.values()) {
+      leafCache.remove(singleton);
+    }
+  }
+
+  /**
+   * Clear the content of this cache.
+   */
+  public synchronized void clear() {
+    cache.clear();
+    mostRecentlyUsedQueries.clear();
+    onClear();
+  }
+
+  // pkg-private for testing
+  synchronized void assertConsistent() {
+    if (requiresEviction()) {
+      throw new AssertionError("requires evictions: size=" + mostRecentlyUsedQueries.size()
+          + ", maxSize=" + maxSize + ", ramBytesUsed=" + ramBytesUsed() + ", maxRamBytesUsed=" + maxRamBytesUsed);
+    }
+    for (LeafCache leafCache : cache.values()) {
+      Set<Query> keys = Collections.newSetFromMap(new IdentityHashMap<>());
+      keys.addAll(leafCache.cache.keySet());
+      keys.removeAll(mostRecentlyUsedQueries);
+      if (!keys.isEmpty()) {
+        throw new AssertionError("One leaf cache contains more keys than the top-level cache: " + keys);
+      }
+    }
+    long recomputedRamBytesUsed =
+          HASHTABLE_RAM_BYTES_PER_ENTRY * cache.size()
+        + LINKED_HASHTABLE_RAM_BYTES_PER_ENTRY * uniqueQueries.size();
+    for (Query query : mostRecentlyUsedQueries) {
+      recomputedRamBytesUsed += ramBytesUsed(query);
+    }
+    for (LeafCache leafCache : cache.values()) {
+      recomputedRamBytesUsed += HASHTABLE_RAM_BYTES_PER_ENTRY * leafCache.cache.size();
+      for (DocIdSet set : leafCache.cache.values()) {
+        recomputedRamBytesUsed += set.ramBytesUsed();
+      }
+    }
+    if (recomputedRamBytesUsed != ramBytesUsed) {
+      throw new AssertionError("ramBytesUsed mismatch : " + ramBytesUsed + " != " + recomputedRamBytesUsed);
+    }
+
+    long recomputedCacheSize = 0;
+    for (LeafCache leafCache : cache.values()) {
+      recomputedCacheSize += leafCache.cache.size();
+    }
+    if (recomputedCacheSize != getCacheSize()) {
+      throw new AssertionError("cacheSize mismatch : " + getCacheSize() + " != " + recomputedCacheSize);
+    }
+  }
+
+  // pkg-private for testing
+  // return the list of cached queries in LRU order
+  synchronized List<Query> cachedQueries() {
+    return new ArrayList<>(mostRecentlyUsedQueries);
+  }
+
+  @Override
+  public Weight doCache(Weight weight, QueryCachingPolicy policy) {
+    while (weight instanceof CachingWrapperWeight) {
+      weight = ((CachingWrapperWeight) weight).in;
+    }
+
+    return new CachingWrapperWeight(weight, policy);
+  }
+
+  @Override
+  public long ramBytesUsed() {
+    return ramBytesUsed;
+  }
+
+  @Override
+  public Collection<Accountable> getChildResources() {
+    synchronized (this) {
+      return Accountables.namedAccountables("segment", cache);
+    }
+  }
+
+  /**
+   * Return the number of bytes used by the given query. The default
+   * implementation returns {@link Accountable#ramBytesUsed()} if the query
+   * implements {@link Accountable} and <code>1024</code> otherwise.
+   */
+  protected long ramBytesUsed(Query query) {
+    if (query instanceof Accountable) {
+      return ((Accountable) query).ramBytesUsed();
+    }
+    return QUERY_DEFAULT_RAM_BYTES_USED;
+  }
+
+  /**
+   * Default cache implementation: uses {@link RoaringDocIdSet}.
+   */
+  protected DocIdSet cacheImpl(DocIdSetIterator iterator, LeafReader reader) throws IOException {
+    return new RoaringDocIdSet.Builder(reader.maxDoc()).add(iterator).build();
+  }
+
+  /**
+   * Return the total number of times that a {@link Query} has been looked up
+   * in this {@link QueryCache}. Note that this number is incremented once per
+   * segment so running a cached query only once will increment this counter
+   * by the number of segments that are wrapped by the searcher.
+   * Note that by definition, {@link #getTotalCount()} is the sum of
+   * {@link #getHitCount()} and {@link #getMissCount()}.
+   * @see #getHitCount()
+   * @see #getMissCount()
+   */
+  public final long getTotalCount() {
+    return getHitCount() + getMissCount();
+  }
+
+  /**
+   * Over the {@link #getTotalCount() total} number of times that a query has
+   * been looked up, return how many times a cached {@link DocIdSet} has been
+   * found and returned.
+   * @see #getTotalCount()
+   * @see #getMissCount()
+   */
+  public final long getHitCount() {
+    return hitCount;
+  }
+
+  /**
+   * Over the {@link #getTotalCount() total} number of times that a query has
+   * been looked up, return how many times this query was not contained in the
+   * cache.
+   * @see #getTotalCount()
+   * @see #getHitCount()
+   */
+  public final long getMissCount() {
+    return missCount;
+  }
+
+  /**
+   * Return the total number of {@link DocIdSet}s which are currently stored
+   * in the cache.
+   * @see #getCacheCount()
+   * @see #getEvictionCount()
+   */
+  public final long getCacheSize() {
+    return cacheSize;
+  }
+
+  /**
+   * Return the total number of cache entries that have been generated and put
+   * in the cache. It is highly desirable to have a {@link #getHitCount() hit
+   * count} that is much higher than the {@link #getCacheCount() cache count}
+   * as the opposite would indicate that the query cache makes efforts in order
+   * to cache queries but then they do not get reused.
+   * @see #getCacheSize()
+   * @see #getEvictionCount()
+   */
+  public final long getCacheCount() {
+    return cacheCount;
+  }
+
+  /**
+   * Return the number of cache entries that have been removed from the cache
+   * either in order to stay under the maximum configured size/ram usage, or
+   * because a segment has been closed. High numbers of evictions might mean
+   * that queries are not reused or that the {@link QueryCachingPolicy
+   * caching policy} caches too aggressively on NRT segments which get merged
+   * early.
+   * @see #getCacheCount()
+   * @see #getCacheSize()
+   */
+  public final long getEvictionCount() {
+    return getCacheCount() - getCacheSize();
+  }
+
+  // this class is not thread-safe, everything but ramBytesUsed needs to be called under a lock
+  private class LeafCache implements Accountable {
+
+    private final Object key;
+    private final Map<Query, DocIdSet> cache;
+    private volatile long ramBytesUsed;
+
+    LeafCache(Object key) {
+      this.key = key;
+      cache = new IdentityHashMap<>();
+      ramBytesUsed = 0;
+    }
+
+    private void onDocIdSetCache(long ramBytesUsed) {
+      this.ramBytesUsed += ramBytesUsed;
+      LRUQueryCache.this.onDocIdSetCache(key, ramBytesUsed);
+    }
+
+    private void onDocIdSetEviction(long ramBytesUsed) {
+      this.ramBytesUsed -= ramBytesUsed;
+      LRUQueryCache.this.onDocIdSetEviction(key, 1, ramBytesUsed);
+    }
+
+    DocIdSet get(Query query) {
+      assert query == QueryCache.cacheKey(query);
+      return cache.get(query);
+    }
+
+    void putIfAbsent(Query query, DocIdSet set) {
+      assert query == QueryCache.cacheKey(query);
+      if (cache.putIfAbsent(query, set) == null) {
+        // the set was actually put
+        onDocIdSetCache(HASHTABLE_RAM_BYTES_PER_ENTRY + set.ramBytesUsed());
+      }
+    }
+
+    void remove(Query query) {
+      assert query == QueryCache.cacheKey(query);
+      DocIdSet removed = cache.remove(query);
+      if (removed != null) {
+        onDocIdSetEviction(HASHTABLE_RAM_BYTES_PER_ENTRY + removed.ramBytesUsed());
+      }
+    }
+
+    @Override
+    public long ramBytesUsed() {
+      return ramBytesUsed;
+    }
+
+  }
+
+  private class CachingWrapperWeight extends ConstantScoreWeight {
+
+    private final Weight in;
+    private final QueryCachingPolicy policy;
+
+    CachingWrapperWeight(Weight in, QueryCachingPolicy policy) {
+      super(in.getQuery());
+      this.in = in;
+      this.policy = policy;
+    }
+
+    @Override
+    Scorer scorer(LeafReaderContext context, Bits acceptDocs, float score) throws IOException {
+      if (context.ord == 0) {
+        policy.onUse(getQuery());
+      }
+      DocIdSet docIdSet = get(in.getQuery(), context);
+      if (docIdSet == null) {
+        if (policy.shouldCache(in.getQuery(), context)) {
+          final Scorer scorer = in.scorer(context, null);
+          if (scorer == null) {
+            docIdSet = DocIdSet.EMPTY;
+          } else {
+            docIdSet = cacheImpl(scorer, context.reader());
+          }
+          putIfAbsent(in.getQuery(), context, docIdSet);
+        } else {
+          return in.scorer(context, acceptDocs);
+        }
+      }
+
+      assert docIdSet != null;
+      if (docIdSet == DocIdSet.EMPTY) {
+        return null;
+      }
+      final DocIdSetIterator approximation = docIdSet.iterator();
+      if (approximation == null) {
+        return null;
+      }
+
+      final DocIdSetIterator disi;
+      final TwoPhaseIterator twoPhaseView;
+      if (acceptDocs == null) {
+        twoPhaseView = null;
+        disi = approximation;
+      } else {
+        twoPhaseView = new TwoPhaseIterator() {
+
+          @Override
+          public boolean matches() throws IOException {
+            final int doc = approximation.docID();
+            return acceptDocs.get(doc);
+          }
+
+          @Override
+          public DocIdSetIterator approximation() {
+            return approximation;
+          }
+        };
+        disi = TwoPhaseIterator.asDocIdSetIterator(twoPhaseView);
+      }
+      return new Scorer(this) {
+
+        @Override
+        public TwoPhaseIterator asTwoPhaseIterator() {
+          return twoPhaseView;
+        }
+
+        @Override
+        public float score() throws IOException {
+          return 0f;
+        }
+
+        @Override
+        public int freq() throws IOException {
+          return 1;
+        }
+
+        @Override
+        public int docID() {
+          return disi.docID();
+        }
+
+        @Override
+        public int nextDoc() throws IOException {
+          return disi.nextDoc();
+        }
+
+        @Override
+        public int advance(int target) throws IOException {
+          return disi.advance(target);
+        }
+
+        @Override
+        public long cost() {
+          return disi.cost();
+        }
+
+      };
+    }
+
+  }
+}

Added: lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/search/QueryCache.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/search/QueryCache.java?rev=1662774&view=auto
==============================================================================
--- lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/search/QueryCache.java (added)
+++ lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/search/QueryCache.java Fri Feb 27 18:09:01 2015
@@ -0,0 +1,52 @@
+package org.apache.lucene.search;
+
+/*
+ * 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.
+ */
+
+/**
+ * A cache for queries.
+ *
+ * @see LRUQueryCache
+ * @lucene.experimental
+ */
+public interface QueryCache {
+
+  /**
+   * Return a key for the given query that only takes matching documents into
+   * account. Boosts will be ignored.
+   * @lucene.internal
+   */
+  public static Query cacheKey(Query query) {
+    if (query.getBoost() == 1f) {
+      return query;
+    } else {
+      Query key = query.clone();
+      key.setBoost(1f);
+      assert key == cacheKey(key);
+      return key;
+    }
+  }
+
+  /**
+   * Return a wrapper around the provided <code>weight</code> that will cache
+   * matching docs per-segment accordingly to the given <code>policy</code>.
+   * NOTE: The returned weight will only be equivalent if scores are not needed.
+   * @see Collector#needsScores()
+   */
+  Weight doCache(Weight weight, QueryCachingPolicy policy);
+
+}

Added: lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/search/QueryCachingPolicy.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/search/QueryCachingPolicy.java?rev=1662774&view=auto
==============================================================================
--- lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/search/QueryCachingPolicy.java (added)
+++ lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/search/QueryCachingPolicy.java Fri Feb 27 18:09:01 2015
@@ -0,0 +1,107 @@
+package org.apache.lucene.search;
+
+/*
+ * 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.IndexReaderContext;
+import org.apache.lucene.index.LeafReaderContext;
+import org.apache.lucene.index.ReaderUtil;
+import org.apache.lucene.index.TieredMergePolicy;
+
+/**
+ * A policy defining which filters should be cached.
+ *
+ * Implementations of this class must be thread-safe.
+ *
+ * @see UsageTrackingQueryCachingPolicy
+ * @see LRUQueryCache
+ * @lucene.experimental
+ */
+// TODO: add APIs for integration with IndexWriter.IndexReaderWarmer
+public interface QueryCachingPolicy {
+
+  /** A simple policy that caches all the provided filters on all segments. */
+  public static final QueryCachingPolicy ALWAYS_CACHE = new QueryCachingPolicy() {
+
+    @Override
+    public void onUse(Query query) {}
+
+    @Override
+    public boolean shouldCache(Query query, LeafReaderContext context) throws IOException {
+      return true;
+    }
+
+  };
+
+  /** A simple policy that only caches on the largest segments of an index.
+   *  The reasoning is that these segments likely account for most of the
+   *  execution time of queries and are also more likely to stay around longer
+   *  than small segments, which makes them more interesting for caching.
+   */
+  public static class CacheOnLargeSegments implements QueryCachingPolicy {
+
+    /** {@link CacheOnLargeSegments} instance that only caches on segments that
+     *  account for more than 3% of the total index size. This should guarantee
+     *  that all segments from the upper {@link TieredMergePolicy tier} will be
+     *  cached while ensuring that at most <tt>33</tt> segments can make it to
+     *  the cache (given that some implementations such as {@link LRUQueryCache}
+     *  perform better when the number of cached segments is low). */
+    public static final CacheOnLargeSegments DEFAULT = new CacheOnLargeSegments(0.03f);
+
+    private final float minSizeRatio;
+
+    /**
+     * Create a {@link CacheOnLargeSegments} instance that only caches on a
+     * given segment if its number of documents divided by the total number of
+     * documents in the index is greater than or equal to
+     * <code>minSizeRatio</code>.
+     */
+    public CacheOnLargeSegments(float minSizeRatio) {
+      if (minSizeRatio <= 0 || minSizeRatio >= 1) {
+        throw new IllegalArgumentException("minSizeRatio must be in ]0, 1[, got " + minSizeRatio);
+      }
+      this.minSizeRatio = minSizeRatio;
+    }
+
+    @Override
+    public void onUse(Query query) {}
+
+    @Override
+    public boolean shouldCache(Query query, LeafReaderContext context) throws IOException {
+      final IndexReaderContext topLevelContext = ReaderUtil.getTopLevelContext(context);
+      final float sizeRatio = (float) context.reader().maxDoc() / topLevelContext.reader().maxDoc();
+      return sizeRatio >= minSizeRatio;
+    }
+
+  };
+
+  /** Callback that is called every time that a cached filter is used.
+   *  This is typically useful if the policy wants to track usage statistics
+   *  in order to make decisions. */
+  void onUse(Query query);
+
+  /** Whether the given {@link DocIdSet} should be cached on a given segment.
+   *  This method will be called on each leaf context to know if the filter
+   *  should be cached on this particular leaf. The filter cache will first
+   *  attempt to load a {@link DocIdSet} from the cache. If it is not cached
+   *  yet and this method returns <tt>true</tt> then a cache entry will be
+   *  generated. Otherwise an uncached set will be returned. */
+  boolean shouldCache(Query query, LeafReaderContext context) throws IOException;
+
+}

Added: lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/search/UsageTrackingQueryCachingPolicy.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/search/UsageTrackingQueryCachingPolicy.java?rev=1662774&view=auto
==============================================================================
--- lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/search/UsageTrackingQueryCachingPolicy.java (added)
+++ lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/search/UsageTrackingQueryCachingPolicy.java Fri Feb 27 18:09:01 2015
@@ -0,0 +1,119 @@
+package org.apache.lucene.search;
+
+/*
+ * 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.LeafReaderContext;
+import org.apache.lucene.util.FrequencyTrackingRingBuffer;
+
+
+/**
+ * A {@link QueryCachingPolicy} that tracks usage statistics of recently-used
+ * filters in order to decide on which filters are worth caching.
+ *
+ * It also uses some heuristics on segments, filters and the doc id sets that
+ * they produce in order to cache more aggressively when the execution cost
+ * significantly outweighs the caching overhead.
+ *
+ * @lucene.experimental
+ */
+public final 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;
+
+  static boolean isCostly(Query query) {
+    // This does not measure the cost of iterating over the filter (for this we
+    // already have the DocIdSetIterator#cost API) but the cost to build the
+    // DocIdSet in the first place
+    return query instanceof MultiTermQuery ||
+        query instanceof MultiTermQueryConstantScoreWrapper;
+  }
+
+  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 final QueryCachingPolicy.CacheOnLargeSegments segmentPolicy;
+  private final FrequencyTrackingRingBuffer recentlyUsedFilters;
+
+  /**
+   * Create a new instance.
+   *
+   * @param minSizeRatio              the minimum size ratio for segments to be cached, see {@link QueryCachingPolicy.CacheOnLargeSegments}
+   * @param historySize               the number of recently used filters to track
+   */
+  public UsageTrackingQueryCachingPolicy(
+      float minSizeRatio,
+      int historySize) {
+    this(new QueryCachingPolicy.CacheOnLargeSegments(minSizeRatio), historySize);
+  }
+
+  /** Create a new instance with an history size of 256. */
+  public UsageTrackingQueryCachingPolicy() {
+    this(QueryCachingPolicy.CacheOnLargeSegments.DEFAULT, 256);
+  }
+
+  private UsageTrackingQueryCachingPolicy(
+      QueryCachingPolicy.CacheOnLargeSegments segmentPolicy,
+      int historySize) {
+    this.segmentPolicy = segmentPolicy;
+    this.recentlyUsedFilters = new FrequencyTrackingRingBuffer(historySize, SENTINEL);
+  }
+
+  /**
+   * For a given query, return how many times it should appear in the history
+   * before being cached.
+   */
+  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;
+    }
+  }
+
+  @Override
+  public void onUse(Query query) {
+    // we only track hash codes, which
+    synchronized (this) {
+      recentlyUsedFilters.add(QueryCache.cacheKey(query).hashCode());
+    }
+  }
+
+  synchronized int frequency(Query query) {
+    return recentlyUsedFilters.frequency(QueryCache.cacheKey(query).hashCode());
+  }
+
+  @Override
+  public boolean shouldCache(Query query, LeafReaderContext context) throws IOException {
+    if (segmentPolicy.shouldCache(query, context) == false) {
+      return false;
+    }
+    final int frequency = frequency(query);
+    final int minFrequency = minFrequencyToCache(query);
+    return frequency >= minFrequency;
+  }
+
+}

Modified: lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/util/BitDocIdSet.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/util/BitDocIdSet.java?rev=1662774&r1=1662773&r2=1662774&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/util/BitDocIdSet.java (original)
+++ lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/util/BitDocIdSet.java Fri Feb 27 18:09:01 2015
@@ -61,12 +61,6 @@ public class BitDocIdSet extends DocIdSe
     return set;
   }
 
-  /** This DocIdSet implementation is cacheable. */
-  @Override
-  public boolean isCacheable() {
-    return true;
-  }
-
   @Override
   public long ramBytesUsed() {
     return BASE_RAM_BYTES_USED + set.ramBytesUsed();

Modified: lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/util/NotDocIdSet.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/util/NotDocIdSet.java?rev=1662774&r1=1662773&r2=1662774&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/util/NotDocIdSet.java (original)
+++ lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/util/NotDocIdSet.java Fri Feb 27 18:09:01 2015
@@ -42,11 +42,6 @@ public final class NotDocIdSet extends D
   }
 
   @Override
-  public boolean isCacheable() {
-    return in.isCacheable();
-  }
-
-  @Override
   public Bits bits() throws IOException {
     final Bits inBits = in.bits();
     if (inBits == null) {

Modified: lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/util/RoaringDocIdSet.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/util/RoaringDocIdSet.java?rev=1662774&r1=1662773&r2=1662774&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/util/RoaringDocIdSet.java (original)
+++ lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/util/RoaringDocIdSet.java Fri Feb 27 18:09:01 2015
@@ -243,11 +243,6 @@ public class RoaringDocIdSet extends Doc
   }
 
   @Override
-  public boolean isCacheable() {
-    return true;
-  }
-
-  @Override
   public long ramBytesUsed() {
     return ramBytesUsed;
   }

Modified: lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/util/packed/EliasFanoDocIdSet.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/util/packed/EliasFanoDocIdSet.java?rev=1662774&r1=1662773&r2=1662774&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/util/packed/EliasFanoDocIdSet.java (original)
+++ lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/util/packed/EliasFanoDocIdSet.java Fri Feb 27 18:09:01 2015
@@ -111,14 +111,6 @@ public class EliasFanoDocIdSet extends D
     };
   }
 
-  /** This DocIdSet implementation is cacheable.
-   * @return <code>true</code>
-   */
-  @Override
-  public boolean isCacheable() {
-    return true;
-  }
-
   @Override
   public boolean equals(Object other) {
     return ((other instanceof EliasFanoDocIdSet))

Modified: lucene/dev/trunk/lucene/core/src/test/org/apache/lucene/index/TestOmitTf.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/core/src/test/org/apache/lucene/index/TestOmitTf.java?rev=1662774&r1=1662773&r2=1662774&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/core/src/test/org/apache/lucene/index/TestOmitTf.java (original)
+++ lucene/dev/trunk/lucene/core/src/test/org/apache/lucene/index/TestOmitTf.java Fri Feb 27 18:09:01 2015
@@ -327,6 +327,10 @@ public class TestOmitTf extends LuceneTe
                     new CountingHitCollector() {
                       private Scorer scorer;
                       @Override
+                      public boolean needsScores() {
+                        return true;
+                      }
+                      @Override
                       public final void setScorer(Scorer scorer) {
                         this.scorer = scorer;
                       }
@@ -345,6 +349,10 @@ public class TestOmitTf extends LuceneTe
                     new CountingHitCollector() {
                       private Scorer scorer;
                       @Override
+                      public boolean needsScores() {
+                        return true;
+                      }
+                      @Override
                       public final void setScorer(Scorer scorer) {
                         this.scorer = scorer;
                       }
@@ -366,6 +374,10 @@ public class TestOmitTf extends LuceneTe
                     new CountingHitCollector() {
                       private Scorer scorer;
                       @Override
+                      public boolean needsScores() {
+                        return true;
+                      }
+                      @Override
                       public final void setScorer(Scorer scorer) {
                         this.scorer = scorer;
                       }
@@ -385,6 +397,10 @@ public class TestOmitTf extends LuceneTe
                     new CountingHitCollector() {
                       private Scorer scorer;
                       @Override
+                      public boolean needsScores() {
+                        return true;
+                      }
+                      @Override
                       public final void setScorer(Scorer scorer) {
                         this.scorer = scorer;
                       }

Added: lucene/dev/trunk/lucene/core/src/test/org/apache/lucene/search/TestCachingWrapperQuery.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/core/src/test/org/apache/lucene/search/TestCachingWrapperQuery.java?rev=1662774&view=auto
==============================================================================
--- lucene/dev/trunk/lucene/core/src/test/org/apache/lucene/search/TestCachingWrapperQuery.java (added)
+++ lucene/dev/trunk/lucene/core/src/test/org/apache/lucene/search/TestCachingWrapperQuery.java Fri Feb 27 18:09:01 2015
@@ -0,0 +1,309 @@
+package org.apache.lucene.search;
+
+/*
+ * 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.analysis.MockAnalyzer;
+import org.apache.lucene.document.Document;
+import org.apache.lucene.document.Field;
+import org.apache.lucene.document.StringField;
+import org.apache.lucene.index.DirectoryReader;
+import org.apache.lucene.index.IndexReader;
+import org.apache.lucene.index.LeafReaderContext;
+import org.apache.lucene.index.RandomIndexWriter;
+import org.apache.lucene.index.SerialMergeScheduler;
+import org.apache.lucene.index.SlowCompositeReaderWrapper;
+import org.apache.lucene.index.Term;
+import org.apache.lucene.store.Directory;
+import org.apache.lucene.util.IOUtils;
+import org.apache.lucene.util.LuceneTestCase;
+
+public class TestCachingWrapperQuery extends LuceneTestCase {
+  Directory dir;
+  DirectoryReader ir;
+  IndexSearcher is;
+  RandomIndexWriter iw;
+  
+  @Override
+  public void setUp() throws Exception {
+    super.setUp();
+    dir = newDirectory();
+    iw = new RandomIndexWriter(random(), dir);
+    Document doc = new Document();
+    Field idField = new StringField("id", "", Field.Store.NO);
+    doc.add(idField);
+    // add 500 docs with id 0..499
+    for (int i = 0; i < 500; i++) {
+      idField.setStringValue(Integer.toString(i));
+      iw.addDocument(doc);
+    }
+    // delete 20 of them
+    for (int i = 0; i < 20; i++) {
+      iw.deleteDocuments(new Term("id", Integer.toString(random().nextInt(iw.maxDoc()))));
+    }
+    ir = iw.getReader();
+    is = newSearcher(ir);
+  }
+  
+  @Override
+  public void tearDown() throws Exception {
+    iw.close();
+    IOUtils.close(ir, dir);
+    super.tearDown();
+  }
+
+  private void assertQueryEquals(Query f1, Query f2) throws Exception {
+    // wrap into CSQ so that scores are not needed
+    TopDocs hits1 = is.search(new ConstantScoreQuery(f1), ir.maxDoc());
+    TopDocs hits2 = is.search(new ConstantScoreQuery(f2), ir.maxDoc());
+    assertEquals(hits1.totalHits, hits2.totalHits);
+    CheckHits.checkEqual(f1, hits1.scoreDocs, hits2.scoreDocs);
+    // now do it again to confirm caching works
+    TopDocs hits3 = is.search(new ConstantScoreQuery(f1), ir.maxDoc());
+    TopDocs hits4 = is.search(new ConstantScoreQuery(f2), ir.maxDoc());
+    assertEquals(hits3.totalHits, hits4.totalHits);
+    CheckHits.checkEqual(f1, hits3.scoreDocs, hits4.scoreDocs);
+  }
+  
+  /** test null iterator */
+  public void testEmpty() throws Exception {
+    Query expected = new BooleanQuery();
+    Query cached = new CachingWrapperQuery(expected, MAYBE_CACHE_POLICY);
+    assertQueryEquals(expected, cached);
+  }
+  
+  /** test iterator returns NO_MORE_DOCS */
+  public void testEmpty2() throws Exception {
+    BooleanQuery expected = new BooleanQuery();
+    expected.add(new TermQuery(new Term("id", "0")), BooleanClause.Occur.MUST);
+    expected.add(new TermQuery(new Term("id", "0")), BooleanClause.Occur.MUST_NOT);
+    Query cached = new CachingWrapperQuery(expected, MAYBE_CACHE_POLICY);
+    assertQueryEquals(expected, cached);
+  }
+  
+  /** test iterator returns single document */
+  public void testSingle() throws Exception {
+    for (int i = 0; i < 10; i++) {
+      int id = random().nextInt(ir.maxDoc());
+      Query expected = new TermQuery(new Term("id", Integer.toString(id)));
+      Query cached = new CachingWrapperQuery(expected, MAYBE_CACHE_POLICY);
+      assertQueryEquals(expected, cached);
+    }
+  }
+  
+  /** test sparse filters (match single documents) */
+  public void testSparse() throws Exception {
+    for (int i = 0; i < 10; i++) {
+      int id_start = random().nextInt(ir.maxDoc()-1);
+      int id_end = id_start + 1;
+      Query expected = TermRangeQuery.newStringRange("id",
+          Integer.toString(id_start), Integer.toString(id_end), true, true);
+      Query cached = new CachingWrapperQuery(expected, MAYBE_CACHE_POLICY);
+      assertQueryEquals(expected, cached);
+    }
+  }
+  
+  /** test dense filters (match entire index) */
+  public void testDense() throws Exception {
+    Query query = new MatchAllDocsQuery();
+    Filter expected = new QueryWrapperFilter(query);
+    Query cached = new CachingWrapperQuery(expected, MAYBE_CACHE_POLICY);
+    assertQueryEquals(expected, cached);
+  }
+
+  public void testCachingWorks() throws Exception {
+    Directory dir = newDirectory();
+    RandomIndexWriter writer = new RandomIndexWriter(random(), dir);
+    writer.close();
+
+    IndexReader reader = SlowCompositeReaderWrapper.wrap(DirectoryReader.open(dir));
+    IndexSearcher searcher = newSearcher(reader);
+    LeafReaderContext context = (LeafReaderContext) reader.getContext();
+    MockFilter filter = new MockFilter();
+    CachingWrapperQuery cacher = new CachingWrapperQuery(filter, QueryCachingPolicy.ALWAYS_CACHE);
+
+    // first time, nested filter is called
+    cacher.createWeight(searcher, false).scorer(context, context.reader().getLiveDocs());
+    assertTrue("first time", filter.wasCalled());
+
+    // make sure no exception if cache is holding the wrong docIdSet
+    cacher.createWeight(searcher, false).scorer(context, context.reader().getLiveDocs());
+
+    // second time, nested filter should not be called
+    filter.clear();
+    cacher.createWeight(searcher, false).scorer(context, context.reader().getLiveDocs());
+    assertFalse("second time", filter.wasCalled());
+
+    reader.close();
+    dir.close();
+  }
+
+  public void testEnforceDeletions() throws Exception {
+    Directory dir = newDirectory();
+    RandomIndexWriter writer = new RandomIndexWriter(
+        random(),
+        dir,
+        newIndexWriterConfig(new MockAnalyzer(random())).
+            setMergeScheduler(new SerialMergeScheduler()).
+            // asserts below requires no unexpected merges:
+            setMergePolicy(newLogMergePolicy(10))
+    );
+
+    // NOTE: cannot use writer.getReader because RIW (on
+    // flipping a coin) may give us a newly opened reader,
+    // but we use .reopen on this reader below and expect to
+    // (must) get an NRT reader:
+    DirectoryReader reader = DirectoryReader.open(writer.w, true);
+    // same reason we don't wrap?
+    IndexSearcher searcher = newSearcher(reader, false);
+
+    // add a doc, refresh the reader, and check that it's there
+    Document doc = new Document();
+    doc.add(newStringField("id", "1", Field.Store.YES));
+    writer.addDocument(doc);
+
+    reader = refreshReader(reader);
+    searcher = newSearcher(reader, false);
+
+    TopDocs docs = searcher.search(new MatchAllDocsQuery(), 1);
+    assertEquals("Should find a hit...", 1, docs.totalHits);
+
+    final Query startQuery = new TermQuery(new Term("id", "1"));
+
+    CachingWrapperQuery query = new CachingWrapperQuery(startQuery, QueryCachingPolicy.ALWAYS_CACHE);
+
+    docs = searcher.search(new ConstantScoreQuery(query), 1);
+    assertTrue(query.ramBytesUsed() > 0);
+
+    assertEquals("[query + filter] Should find a hit...", 1, docs.totalHits);
+
+    Query constantScore = new ConstantScoreQuery(query);
+    docs = searcher.search(constantScore, 1);
+    assertEquals("[just filter] Should find a hit...", 1, docs.totalHits);
+
+    // make sure we get a cache hit when we reopen reader
+    // that had no change to deletions
+
+    // fake delete (deletes nothing):
+    writer.deleteDocuments(new Term("foo", "bar"));
+
+    IndexReader oldReader = reader;
+    reader = refreshReader(reader);
+    assertTrue(reader == oldReader);
+    int missCount = query.missCount;
+    docs = searcher.search(constantScore, 1);
+    assertEquals("[just filter] Should find a hit...", 1, docs.totalHits);
+
+    // cache hit:
+    assertEquals(missCount, query.missCount);
+
+    // now delete the doc, refresh the reader, and see that it's not there
+    writer.deleteDocuments(new Term("id", "1"));
+
+    // NOTE: important to hold ref here so GC doesn't clear
+    // the cache entry!  Else the assert below may sometimes
+    // fail:
+    oldReader = reader;
+    reader = refreshReader(reader);
+
+    searcher = newSearcher(reader, false);
+
+    missCount = query.missCount;
+    docs = searcher.search(new ConstantScoreQuery(query), 1);
+    assertEquals("[query + filter] Should *not* find a hit...", 0, docs.totalHits);
+
+    // cache hit
+    assertEquals(missCount, query.missCount);
+    docs = searcher.search(constantScore, 1);
+    assertEquals("[just filter] Should *not* find a hit...", 0, docs.totalHits);
+
+    // apply deletes dynamically:
+    query = new CachingWrapperQuery(startQuery, QueryCachingPolicy.ALWAYS_CACHE);
+    writer.addDocument(doc);
+    reader = refreshReader(reader);
+    searcher = newSearcher(reader, false);
+
+    docs = searcher.search(new ConstantScoreQuery(query), 1);
+    assertEquals("[query + filter] Should find a hit...", 1, docs.totalHits);
+    missCount = query.missCount;
+    assertTrue(missCount > 0);
+    constantScore = new ConstantScoreQuery(query);
+    docs = searcher.search(constantScore, 1);
+    assertEquals("[just filter] Should find a hit...", 1, docs.totalHits);
+    assertEquals(missCount, query.missCount);
+
+    writer.addDocument(doc);
+
+    // NOTE: important to hold ref here so GC doesn't clear
+    // the cache entry!  Else the assert below may sometimes
+    // fail:
+    oldReader = reader;
+
+    reader = refreshReader(reader);
+    searcher = newSearcher(reader, false);
+        
+    docs = searcher.search(new ConstantScoreQuery(query), 1);
+    assertEquals("[query + filter] Should find 2 hits...", 2, docs.totalHits);
+    assertTrue(query.missCount > missCount);
+    missCount = query.missCount;
+
+    constantScore = new ConstantScoreQuery(query);
+    docs = searcher.search(constantScore, 1);
+    assertEquals("[just filter] Should find a hit...", 2, docs.totalHits);
+    assertEquals(missCount, query.missCount);
+
+    // now delete the doc, refresh the reader, and see that it's not there
+    writer.deleteDocuments(new Term("id", "1"));
+
+    reader = refreshReader(reader);
+    searcher = newSearcher(reader, false);
+
+    docs = searcher.search(new ConstantScoreQuery(query), 1);
+    assertEquals("[query + filter] Should *not* find a hit...", 0, docs.totalHits);
+    // CWF reused the same entry (it dynamically applied the deletes):
+    assertEquals(missCount, query.missCount);
+
+    docs = searcher.search(constantScore, 1);
+    assertEquals("[just filter] Should *not* find a hit...", 0, docs.totalHits);
+    // CWF reused the same entry (it dynamically applied the deletes):
+    assertEquals(missCount, query.missCount);
+
+    // NOTE: silliness to make sure JRE does not eliminate
+    // our holding onto oldReader to prevent
+    // CachingWrapperFilter's WeakHashMap from dropping the
+    // entry:
+    assertTrue(oldReader != null);
+
+    reader.close();
+    writer.close();
+    dir.close();
+  }
+
+  private static DirectoryReader refreshReader(DirectoryReader reader) throws IOException {
+    DirectoryReader oldReader = reader;
+    reader = DirectoryReader.openIfChanged(reader);
+    if (reader != null) {
+      oldReader.close();
+      return reader;
+    } else {
+      return oldReader;
+    }
+  }
+
+}

Modified: lucene/dev/trunk/lucene/core/src/test/org/apache/lucene/search/TestConstantScoreQuery.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/core/src/test/org/apache/lucene/search/TestConstantScoreQuery.java?rev=1662774&r1=1662773&r2=1662774&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/core/src/test/org/apache/lucene/search/TestConstantScoreQuery.java (original)
+++ lucene/dev/trunk/lucene/core/src/test/org/apache/lucene/search/TestConstantScoreQuery.java Fri Feb 27 18:09:01 2015
@@ -23,10 +23,12 @@ import org.apache.lucene.document.Docume
 import org.apache.lucene.document.Field;
 import org.apache.lucene.index.DirectoryReader;
 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.similarities.DefaultSimilarity;
 import org.apache.lucene.store.Directory;
+import org.apache.lucene.util.Bits;
 import org.apache.lucene.util.LuceneTestCase;
 
 /** This class only tests some basic functionality in CSQ, the main parts are mostly
@@ -93,6 +95,7 @@ public class TestConstantScoreQuery exte
       writer.close();
       // we don't wrap with AssertingIndexSearcher in order to have the original scorer in setScorer.
       searcher = newSearcher(reader, true, false);
+      searcher.setQueryCache(null); // to assert on scorer impl
       
       // set a similarity that does not normalize our boost away
       searcher.setSimilarity(new DefaultSimilarity() {
@@ -127,6 +130,27 @@ public class TestConstantScoreQuery exte
     }
   }
 
+  // a filter for which other queries don't have special rewrite rules
+  private static class FilterWrapper extends Filter {
+
+    private final Filter in;
+    
+    FilterWrapper(Filter in) {
+      this.in = in;
+    }
+    
+    @Override
+    public DocIdSet getDocIdSet(LeafReaderContext context, Bits acceptDocs) throws IOException {
+      return in.getDocIdSet(context, acceptDocs);
+    }
+
+    @Override
+    public String toString(String field) {
+      return in.toString(field);
+    }
+    
+  }
+
   public void testConstantScoreQueryAndFilter() throws Exception {
     Directory d = newDirectory();
     RandomIndexWriter w = new RandomIndexWriter(random(), d);
@@ -139,13 +163,13 @@ public class TestConstantScoreQuery exte
     IndexReader r = w.getReader();
     w.close();
 
-    Filter filterB = new CachingWrapperFilter(new QueryWrapperFilter(new TermQuery(new Term("field", "b"))));
+    Filter filterB = new FilterWrapper(new QueryWrapperFilter(new TermQuery(new Term("field", "b"))));
     Query query = new ConstantScoreQuery(filterB);
 
     IndexSearcher s = newSearcher(r);
     assertEquals(1, s.search(new FilteredQuery(query, filterB), 1).totalHits); // Query for field:b, Filter field:b
 
-    Filter filterA = new CachingWrapperFilter(new QueryWrapperFilter(new TermQuery(new Term("field", "a"))));
+    Filter filterA = new FilterWrapper(new QueryWrapperFilter(new TermQuery(new Term("field", "a"))));
     query = new ConstantScoreQuery(filterA);
 
     assertEquals(0, s.search(new FilteredQuery(query, filterB), 1).totalHits); // Query field:b, Filter field:a

Modified: lucene/dev/trunk/lucene/core/src/test/org/apache/lucene/search/TestDocValuesRangeQuery.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/core/src/test/org/apache/lucene/search/TestDocValuesRangeQuery.java?rev=1662774&r1=1662773&r2=1662774&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/core/src/test/org/apache/lucene/search/TestDocValuesRangeQuery.java (original)
+++ lucene/dev/trunk/lucene/core/src/test/org/apache/lucene/search/TestDocValuesRangeQuery.java Fri Feb 27 18:09:01 2015
@@ -267,12 +267,12 @@ public class TestDocValuesRangeQuery ext
     iw.close();
 
     Query q1 = DocValuesRangeQuery.newLongRange("dv1", 0L, 100L, random().nextBoolean(), random().nextBoolean());
-    Weight w = searcher.createNormalizedWeight(q1, random().nextBoolean());
+    Weight w = searcher.createNormalizedWeight(q1, true);
     Scorer s = w.scorer(ctx, null);
     assertNotNull(s.asTwoPhaseIterator());
 
     Query q2 = DocValuesRangeQuery.newBytesRefRange("dv2", toSortableBytes(0L), toSortableBytes(100L), random().nextBoolean(), random().nextBoolean());
-    w = searcher.createNormalizedWeight(q2, random().nextBoolean());
+    w = searcher.createNormalizedWeight(q2, true);
     s = w.scorer(ctx, null);
     assertNotNull(s.asTwoPhaseIterator());
 

Modified: lucene/dev/trunk/lucene/core/src/test/org/apache/lucene/search/TestFilterCachingPolicy.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/core/src/test/org/apache/lucene/search/TestFilterCachingPolicy.java?rev=1662774&r1=1662773&r2=1662774&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/core/src/test/org/apache/lucene/search/TestFilterCachingPolicy.java (original)
+++ lucene/dev/trunk/lucene/core/src/test/org/apache/lucene/search/TestFilterCachingPolicy.java Fri Feb 27 18:09:01 2015
@@ -38,11 +38,11 @@ public class TestFilterCachingPolicy ext
     }
     final IndexReader reader = w.getReader();
     for (float minSizeRatio : new float[] {Float.MIN_VALUE, 0.01f, 0.1f, 0.9f}) {
-      final FilterCachingPolicy policy = new FilterCachingPolicy.CacheOnLargeSegments(minSizeRatio);
+      final QueryCachingPolicy policy = new QueryCachingPolicy.CacheOnLargeSegments(minSizeRatio);
       for (LeafReaderContext ctx : reader.leaves()) {
         final Filter filter = new QueryWrapperFilter(new TermQuery(new Term("field", "value")));
         final DocIdSet set = null;
-        final boolean shouldCache = policy.shouldCache(filter, ctx, set);
+        final boolean shouldCache = policy.shouldCache(filter, ctx);
         final float sizeRatio = (float) ctx.reader().maxDoc() / reader.maxDoc();
         assertEquals(sizeRatio >= minSizeRatio, shouldCache);
       }

Modified: lucene/dev/trunk/lucene/core/src/test/org/apache/lucene/search/TestFilteredQuery.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/core/src/test/org/apache/lucene/search/TestFilteredQuery.java?rev=1662774&r1=1662773&r2=1662774&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/core/src/test/org/apache/lucene/search/TestFilteredQuery.java (original)
+++ lucene/dev/trunk/lucene/core/src/test/org/apache/lucene/search/TestFilteredQuery.java Fri Feb 27 18:09:01 2015
@@ -302,17 +302,38 @@ public class TestFilteredQuery extends L
     tChainedFilters(false);
   }
   
+  // a filter for which other queries don't have special rewrite rules
+  private static class FilterWrapper extends Filter {
+
+    private final Filter in;
+    
+    FilterWrapper(Filter in) {
+      this.in = in;
+    }
+    
+    @Override
+    public DocIdSet getDocIdSet(LeafReaderContext context, Bits acceptDocs) throws IOException {
+      return in.getDocIdSet(context, acceptDocs);
+    }
+
+    @Override
+    public String toString(String field) {
+      return in.toString(field);
+    }
+    
+  }
+  
   private void tChainedFilters(final boolean useRandomAccess) throws Exception {
     Query query = new FilteredQuery(new FilteredQuery(
-      new MatchAllDocsQuery(), new CachingWrapperFilter(new QueryWrapperFilter(new TermQuery(new Term("field", "three")))), randomFilterStrategy(random(), useRandomAccess)),
-      new CachingWrapperFilter(new QueryWrapperFilter(new TermQuery(new Term("field", "four")))), randomFilterStrategy(random(), useRandomAccess));
+      new MatchAllDocsQuery(), new FilterWrapper(new QueryWrapperFilter(new TermQuery(new Term("field", "three")))), randomFilterStrategy(random(), useRandomAccess)),
+      new FilterWrapper(new QueryWrapperFilter(new TermQuery(new Term("field", "four")))), randomFilterStrategy(random(), useRandomAccess));
     ScoreDoc[] hits = searcher.search(query, 10).scoreDocs;
     assertEquals(2, hits.length);
     QueryUtils.check(random(), query, searcher);    
 
     // one more:
     query = new FilteredQuery(query,
-      new CachingWrapperFilter(new QueryWrapperFilter(new TermQuery(new Term("field", "five")))), randomFilterStrategy(random(), useRandomAccess));
+      new FilterWrapper(new QueryWrapperFilter(new TermQuery(new Term("field", "five")))), randomFilterStrategy(random(), useRandomAccess));
     hits = searcher.search(query, 10).scoreDocs;
     assertEquals(1, hits.length);
     QueryUtils.check(random(), query, searcher);    
@@ -390,8 +411,8 @@ public class TestFilteredQuery extends L
   }
 
   public void testRewrite() throws Exception {
-    assertRewrite(new FilteredQuery(new TermQuery(new Term("field", "one")), new CachingWrapperFilter(new QueryWrapperFilter(new PrefixQuery(new Term("field", "o")))), randomFilterStrategy()), FilteredQuery.class);
-    assertRewrite(new FilteredQuery(new PrefixQuery(new Term("field", "one")), new CachingWrapperFilter(new QueryWrapperFilter(new PrefixQuery(new Term("field", "o")))), randomFilterStrategy()), FilteredQuery.class);
+    assertRewrite(new FilteredQuery(new TermQuery(new Term("field", "one")), new FilterWrapper(new QueryWrapperFilter(new PrefixQuery(new Term("field", "o")))), randomFilterStrategy()), FilteredQuery.class);
+    assertRewrite(new FilteredQuery(new PrefixQuery(new Term("field", "one")), new FilterWrapper(new QueryWrapperFilter(new PrefixQuery(new Term("field", "o")))), randomFilterStrategy()), FilteredQuery.class);
   }
   
   public void testGetFilterStrategy() {