You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@lucene.apache.org by mi...@apache.org on 2015/02/09 00:53:25 UTC
svn commit: r1658277 [6/38] - in /lucene/dev/branches/lucene6005: ./
dev-tools/ dev-tools/idea/solr/contrib/dataimporthandler/
dev-tools/idea/solr/contrib/velocity/ dev-tools/maven/lucene/replicator/
dev-tools/maven/solr/ dev-tools/maven/solr/contrib/v...
Modified: lucene/dev/branches/lucene6005/lucene/core/src/java/org/apache/lucene/search/FieldValueHitQueue.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene6005/lucene/core/src/java/org/apache/lucene/search/FieldValueHitQueue.java?rev=1658277&r1=1658276&r2=1658277&view=diff
==============================================================================
--- lucene/dev/branches/lucene6005/lucene/core/src/java/org/apache/lucene/search/FieldValueHitQueue.java (original)
+++ lucene/dev/branches/lucene6005/lucene/core/src/java/org/apache/lucene/search/FieldValueHitQueue.java Sun Feb 8 23:53:14 2015
@@ -19,6 +19,7 @@ package org.apache.lucene.search;
import java.io.IOException;
+import org.apache.lucene.index.LeafReaderContext;
import org.apache.lucene.util.PriorityQueue;
/**
@@ -53,17 +54,16 @@ public abstract class FieldValueHitQueue
* there is just one comparator.
*/
private static final class OneComparatorFieldValueHitQueue<T extends FieldValueHitQueue.Entry> extends FieldValueHitQueue<T> {
+
private final int oneReverseMul;
+ private final FieldComparator<?> oneComparator;
public OneComparatorFieldValueHitQueue(SortField[] fields, int size)
throws IOException {
super(fields, size);
-
- SortField field = fields[0];
- setComparator(0, field.getComparator(size, 0));
- oneReverseMul = field.reverse ? -1 : 1;
-
- reverseMul[0] = oneReverseMul;
+ assert fields.length == 1;
+ oneComparator = comparators[0];
+ oneReverseMul = reverseMul[0];
}
/**
@@ -78,7 +78,7 @@ public abstract class FieldValueHitQueue
assert hitA != hitB;
assert hitA.slot != hitB.slot;
- final int c = oneReverseMul * firstComparator.compare(hitA.slot, hitB.slot);
+ final int c = oneReverseMul * oneComparator.compare(hitA.slot, hitB.slot);
if (c != 0) {
return c > 0;
}
@@ -98,14 +98,6 @@ public abstract class FieldValueHitQueue
public MultiComparatorsFieldValueHitQueue(SortField[] fields, int size)
throws IOException {
super(fields, size);
-
- int numComparators = comparators.length;
- for (int i = 0; i < numComparators; ++i) {
- SortField field = fields[i];
-
- reverseMul[i] = field.reverse ? -1 : 1;
- setComparator(i, field.getComparator(size, i));
- }
}
@Override
@@ -130,8 +122,7 @@ public abstract class FieldValueHitQueue
}
// prevent instantiation and extension.
- @SuppressWarnings({"rawtypes","unchecked"})
- private FieldValueHitQueue(SortField[] fields, int size) {
+ private FieldValueHitQueue(SortField[] fields, int size) throws IOException {
super(size);
// When we get here, fields.length is guaranteed to be > 0, therefore no
// need to check it again.
@@ -141,8 +132,14 @@ public abstract class FieldValueHitQueue
// anyway.
this.fields = fields;
int numComparators = fields.length;
- comparators = new FieldComparator[numComparators];
+ comparators = new FieldComparator<?>[numComparators];
reverseMul = new int[numComparators];
+ for (int i = 0; i < numComparators; ++i) {
+ SortField field = fields[i];
+
+ reverseMul[i] = field.reverse ? -1 : 1;
+ comparators[i] = field.getComparator(size, i);
+ }
}
/**
@@ -179,15 +176,17 @@ public abstract class FieldValueHitQueue
return reverseMul;
}
- public void setComparator(int pos, FieldComparator<?> comparator) {
- if (pos==0) firstComparator = comparator;
- comparators[pos] = comparator;
+ public LeafFieldComparator[] getComparators(LeafReaderContext context) throws IOException {
+ LeafFieldComparator[] comparators = new LeafFieldComparator[this.comparators.length];
+ for (int i = 0; i < comparators.length; ++i) {
+ comparators[i] = this.comparators[i].getLeafComparator(context);
+ }
+ return comparators;
}
/** Stores the sort criteria being used. */
protected final SortField[] fields;
- protected final FieldComparator<?>[] comparators; // use setComparator to change this array
- protected FieldComparator<?> firstComparator; // this must always be equal to comparators[0]
+ protected final FieldComparator<?>[] comparators;
protected final int[] reverseMul;
@Override
Modified: lucene/dev/branches/lucene6005/lucene/core/src/java/org/apache/lucene/search/FilterCache.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene6005/lucene/core/src/java/org/apache/lucene/search/FilterCache.java?rev=1658277&r1=1658276&r2=1658277&view=diff
==============================================================================
--- lucene/dev/branches/lucene6005/lucene/core/src/java/org/apache/lucene/search/FilterCache.java (original)
+++ lucene/dev/branches/lucene6005/lucene/core/src/java/org/apache/lucene/search/FilterCache.java Sun Feb 8 23:53:14 2015
@@ -20,6 +20,7 @@ package org.apache.lucene.search;
/**
* A cache for filters.
*
+ * @see LRUFilterCache
* @lucene.experimental
*/
public interface FilterCache {
Modified: lucene/dev/branches/lucene6005/lucene/core/src/java/org/apache/lucene/search/FilterCachingPolicy.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene6005/lucene/core/src/java/org/apache/lucene/search/FilterCachingPolicy.java?rev=1658277&r1=1658276&r2=1658277&view=diff
==============================================================================
--- lucene/dev/branches/lucene6005/lucene/core/src/java/org/apache/lucene/search/FilterCachingPolicy.java (original)
+++ lucene/dev/branches/lucene6005/lucene/core/src/java/org/apache/lucene/search/FilterCachingPolicy.java Sun Feb 8 23:53:14 2015
@@ -29,6 +29,8 @@ import org.apache.lucene.index.TieredMer
*
* Implementations of this class must be thread-safe.
*
+ * @see UsageTrackingFilterCachingPolicy
+ * @see LRUFilterCache
* @lucene.experimental
*/
// TODO: add APIs for integration with IndexWriter.IndexReaderWarmer
Modified: lucene/dev/branches/lucene6005/lucene/core/src/java/org/apache/lucene/search/FilterLeafCollector.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene6005/lucene/core/src/java/org/apache/lucene/search/FilterLeafCollector.java?rev=1658277&r1=1658276&r2=1658277&view=diff
==============================================================================
--- lucene/dev/branches/lucene6005/lucene/core/src/java/org/apache/lucene/search/FilterLeafCollector.java (original)
+++ lucene/dev/branches/lucene6005/lucene/core/src/java/org/apache/lucene/search/FilterLeafCollector.java Sun Feb 8 23:53:14 2015
@@ -44,11 +44,6 @@ public class FilterLeafCollector impleme
}
@Override
- public boolean acceptsDocsOutOfOrder() {
- return in.acceptsDocsOutOfOrder();
- }
-
- @Override
public String toString() {
return getClass().getSimpleName() + "(" + in + ")";
}
Modified: lucene/dev/branches/lucene6005/lucene/core/src/java/org/apache/lucene/search/FilteredQuery.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene6005/lucene/core/src/java/org/apache/lucene/search/FilteredQuery.java?rev=1658277&r1=1658276&r2=1658277&view=diff
==============================================================================
--- lucene/dev/branches/lucene6005/lucene/core/src/java/org/apache/lucene/search/FilteredQuery.java (original)
+++ lucene/dev/branches/lucene6005/lucene/core/src/java/org/apache/lucene/search/FilteredQuery.java Sun Feb 8 23:53:14 2015
@@ -81,11 +81,6 @@ public class FilteredQuery extends Query
public Weight createWeight(final IndexSearcher searcher) throws IOException {
final Weight weight = query.createWeight (searcher);
return new Weight() {
-
- @Override
- public boolean scoresDocsOutOfOrder() {
- return true;
- }
@Override
public float getValueForNormalization() throws IOException {
@@ -138,7 +133,7 @@ public class FilteredQuery extends Query
// return a filtering top scorer
@Override
- public BulkScorer bulkScorer(LeafReaderContext context, boolean scoreDocsInOrder, Bits acceptDocs) throws IOException {
+ public BulkScorer bulkScorer(LeafReaderContext context, Bits acceptDocs) throws IOException {
assert filter != null;
DocIdSet filterDocIdSet = filter.getDocIdSet(context, acceptDocs);
@@ -147,7 +142,7 @@ public class FilteredQuery extends Query
return null;
}
- return strategy.filteredBulkScorer(context, weight, scoreDocsInOrder, filterDocIdSet);
+ return strategy.filteredBulkScorer(context, weight, filterDocIdSet);
}
};
}
@@ -225,12 +220,12 @@ public class FilteredQuery extends Query
}
@Override
- public boolean score(LeafCollector collector, int maxDoc) throws IOException {
+ public int score(LeafCollector collector, int min, int maxDoc) throws IOException {
// the normalization trick already applies the boost of this query,
// so we can use the wrapped scorer directly:
collector.setScorer(scorer);
- if (scorer.docID() == -1) {
- scorer.nextDoc();
+ if (scorer.docID() < min) {
+ scorer.advance(min);
}
while (true) {
final int scorerDoc = scorer.docID();
@@ -244,7 +239,7 @@ public class FilteredQuery extends Query
}
}
- return scorer.docID() != Scorer.NO_MORE_DOCS;
+ return scorer.docID();
}
}
@@ -480,7 +475,7 @@ public class FilteredQuery extends Query
* @return a filtered top scorer
*/
public BulkScorer filteredBulkScorer(LeafReaderContext context,
- Weight weight, boolean scoreDocsInOrder, DocIdSet docIdSet) throws IOException {
+ Weight weight, DocIdSet docIdSet) throws IOException {
Scorer scorer = filteredScorer(context, weight, docIdSet);
if (scorer == null) {
return null;
@@ -603,13 +598,12 @@ public class FilteredQuery extends Query
@Override
public BulkScorer filteredBulkScorer(final LeafReaderContext context,
Weight weight,
- boolean scoreDocsInOrder, // ignored (we always top-score in order)
DocIdSet docIdSet) throws IOException {
Bits filterAcceptDocs = docIdSet.bits();
if (filterAcceptDocs == null) {
// Filter does not provide random-access Bits; we
// must fallback to leapfrog:
- return LEAP_FROG_QUERY_FIRST_STRATEGY.filteredBulkScorer(context, weight, scoreDocsInOrder, docIdSet);
+ return LEAP_FROG_QUERY_FIRST_STRATEGY.filteredBulkScorer(context, weight, docIdSet);
}
final Scorer scorer = weight.scorer(context, null);
return scorer == null ? null : new QueryFirstBulkScorer(scorer, filterAcceptDocs);
Modified: lucene/dev/branches/lucene6005/lucene/core/src/java/org/apache/lucene/search/IndexSearcher.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene6005/lucene/core/src/java/org/apache/lucene/search/IndexSearcher.java?rev=1658277&r1=1658276&r2=1658277&view=diff
==============================================================================
--- lucene/dev/branches/lucene6005/lucene/core/src/java/org/apache/lucene/search/IndexSearcher.java (original)
+++ lucene/dev/branches/lucene6005/lucene/core/src/java/org/apache/lucene/search/IndexSearcher.java Sun Feb 8 23:53:14 2015
@@ -18,19 +18,14 @@ package org.apache.lucene.search;
*/
import java.io.IOException;
+import java.util.ArrayList;
import java.util.Arrays;
-import java.util.Iterator;
import java.util.List;
-import java.util.NoSuchElementException;
import java.util.Set;
import java.util.concurrent.Callable;
-import java.util.concurrent.CompletionService;
import java.util.concurrent.ExecutionException;
-import java.util.concurrent.Executor;
-import java.util.concurrent.ExecutorCompletionService;
import java.util.concurrent.ExecutorService;
-import java.util.concurrent.locks.Lock;
-import java.util.concurrent.locks.ReentrantLock;
+import java.util.concurrent.Future;
import org.apache.lucene.document.Document;
import org.apache.lucene.document.FieldTypes;
@@ -444,28 +439,21 @@ public class IndexSearcher {
if (executor == null) {
return search(leafContexts, weight, after, nDocs);
} else {
- final HitQueue hq = new HitQueue(nDocs, false);
- final Lock lock = new ReentrantLock();
- final ExecutionHelper<TopDocs> runner = new ExecutionHelper<>(executor);
-
- for (int i = 0; i < leafSlices.length; i++) { // search each sub
- runner.submit(new SearcherCallableNoSort(lock, this, leafSlices[i], weight, after, nDocs, hq));
+ final List<Future<TopDocs>> topDocsFutures = new ArrayList<>(leafSlices.length);
+ for (int i = 0; i < leafSlices.length; i++) { // search each leaf slice
+ topDocsFutures.add(executor.submit(new SearcherCallableNoSort(this, leafSlices[i], weight, after, nDocs)));
}
-
- int totalHits = 0;
- float maxScore = Float.NEGATIVE_INFINITY;
- for (final TopDocs topDocs : runner) {
- if(topDocs.totalHits != 0) {
- totalHits += topDocs.totalHits;
- maxScore = Math.max(maxScore, topDocs.getMaxScore());
+ final TopDocs[] topDocs = new TopDocs[leafSlices.length];
+ for (int i = 0; i < leafSlices.length; i++) {
+ try {
+ topDocs[i] = topDocsFutures.get(i).get();
+ } catch (InterruptedException e) {
+ throw new ThreadInterruptedException(e);
+ } catch (ExecutionException e) {
+ throw new RuntimeException(e);
}
}
-
- final ScoreDoc[] scoreDocs = new ScoreDoc[hq.size()];
- for (int i = hq.size() - 1; i >= 0; i--) // put docs in array
- scoreDocs[i] = hq.pop();
-
- return new TopDocs(totalHits, scoreDocs, maxScore);
+ return TopDocs.merge(null, nDocs, topDocs);
}
}
@@ -484,7 +472,7 @@ public class IndexSearcher {
limit = 1;
}
nDocs = Math.min(nDocs, limit);
- TopScoreDocCollector collector = TopScoreDocCollector.create(nDocs, after, !weight.scoresDocsOutOfOrder());
+ TopScoreDocCollector collector = TopScoreDocCollector.create(nDocs, after);
search(leaves, weight, collector);
return collector.topDocs();
}
@@ -529,31 +517,21 @@ public class IndexSearcher {
// use all leaves here!
return search(leafContexts, weight, after, nDocs, sort, fillFields, doDocScores, doMaxScore);
} else {
- final TopFieldCollector topCollector = TopFieldCollector.create(sort, nDocs,
- after,
- fillFields,
- doDocScores,
- doMaxScore,
- false);
-
- final Lock lock = new ReentrantLock();
- final ExecutionHelper<TopFieldDocs> runner = new ExecutionHelper<>(executor);
+ final List<Future<TopFieldDocs>> topDocsFutures = new ArrayList<>(leafSlices.length);
for (int i = 0; i < leafSlices.length; i++) { // search each leaf slice
- runner.submit(
- new SearcherCallableWithSort(lock, this, leafSlices[i], weight, after, nDocs, topCollector, sort, doDocScores, doMaxScore));
+ topDocsFutures.add(executor.submit(new SearcherCallableWithSort(this, leafSlices[i], weight, after, nDocs, sort, doDocScores, doMaxScore)));
}
- int totalHits = 0;
- float maxScore = Float.NEGATIVE_INFINITY;
- for (final TopFieldDocs topFieldDocs : runner) {
- if (topFieldDocs.totalHits != 0) {
- totalHits += topFieldDocs.totalHits;
- maxScore = Math.max(maxScore, topFieldDocs.getMaxScore());
+ final TopFieldDocs[] topDocs = new TopFieldDocs[leafSlices.length];
+ for (int i = 0; i < leafSlices.length; i++) {
+ try {
+ topDocs[i] = topDocsFutures.get(i).get();
+ } catch (InterruptedException e) {
+ throw new ThreadInterruptedException(e);
+ } catch (ExecutionException e) {
+ throw new RuntimeException(e);
}
}
-
- final TopFieldDocs topDocs = (TopFieldDocs) topCollector.topDocs();
-
- return new TopFieldDocs(totalHits, topDocs.scoreDocs, topDocs.fields, topDocs.getMaxScore());
+ return (TopFieldDocs) TopDocs.merge(sort, nDocs, topDocs);
}
}
@@ -574,7 +552,7 @@ public class IndexSearcher {
TopFieldCollector collector = TopFieldCollector.create(sort, nDocs, after,
fillFields, doDocScores,
- doMaxScore, !weight.scoresDocsOutOfOrder());
+ doMaxScore);
search(leaves, weight, collector);
return (TopFieldDocs) collector.topDocs();
}
@@ -613,7 +591,7 @@ public class IndexSearcher {
// continue with the following leaf
continue;
}
- BulkScorer scorer = weight.bulkScorer(ctx, !leafCollector.acceptsDocsOutOfOrder(), ctx.reader().getLiveDocs());
+ BulkScorer scorer = weight.bulkScorer(ctx, ctx.reader().getLiveDocs());
if (scorer != null) {
try {
scorer.score(leafCollector);
@@ -703,42 +681,24 @@ public class IndexSearcher {
*/
private static final class SearcherCallableNoSort implements Callable<TopDocs> {
- private final Lock lock;
private final IndexSearcher searcher;
private final Weight weight;
private final ScoreDoc after;
private final int nDocs;
- private final HitQueue hq;
private final LeafSlice slice;
- public SearcherCallableNoSort(Lock lock, IndexSearcher searcher, LeafSlice slice, Weight weight,
- ScoreDoc after, int nDocs, HitQueue hq) {
- this.lock = lock;
+ public SearcherCallableNoSort(IndexSearcher searcher, LeafSlice slice, Weight weight,
+ ScoreDoc after, int nDocs) {
this.searcher = searcher;
this.weight = weight;
this.after = after;
this.nDocs = nDocs;
- this.hq = hq;
this.slice = slice;
}
@Override
public TopDocs call() throws IOException {
- final TopDocs docs = searcher.search(Arrays.asList(slice.leaves), weight, after, nDocs);
- final ScoreDoc[] scoreDocs = docs.scoreDocs;
- //it would be so nice if we had a thread-safe insert
- lock.lock();
- try {
- for (int j = 0; j < scoreDocs.length; j++) { // merge scoreDocs into hq
- final ScoreDoc scoreDoc = scoreDocs[j];
- if (scoreDoc == hq.insertWithOverflow(scoreDoc)) {
- break;
- }
- }
- } finally {
- lock.unlock();
- }
- return docs;
+ return searcher.search(Arrays.asList(slice.leaves), weight, after, nDocs);
}
}
@@ -748,25 +708,21 @@ public class IndexSearcher {
*/
private static final class SearcherCallableWithSort implements Callable<TopFieldDocs> {
- private final Lock lock;
private final IndexSearcher searcher;
private final Weight weight;
private final int nDocs;
- private final TopFieldCollector hq;
private final Sort sort;
private final LeafSlice slice;
private final FieldDoc after;
private final boolean doDocScores;
private final boolean doMaxScore;
- public SearcherCallableWithSort(Lock lock, IndexSearcher searcher, LeafSlice slice, Weight weight,
- FieldDoc after, int nDocs, TopFieldCollector hq, Sort sort,
+ public SearcherCallableWithSort(IndexSearcher searcher, LeafSlice slice, Weight weight,
+ FieldDoc after, int nDocs, Sort sort,
boolean doDocScores, boolean doMaxScore) {
- this.lock = lock;
this.searcher = searcher;
this.weight = weight;
this.nDocs = nDocs;
- this.hq = hq;
this.sort = sort;
this.slice = slice;
this.after = after;
@@ -774,85 +730,11 @@ public class IndexSearcher {
this.doMaxScore = doMaxScore;
}
- private final FakeScorer fakeScorer = new FakeScorer();
-
@Override
public TopFieldDocs call() throws IOException {
assert slice.leaves.length == 1;
- final TopFieldDocs docs = searcher.search(Arrays.asList(slice.leaves),
- weight, after, nDocs, sort, true, doDocScores || sort.needsScores(), doMaxScore);
- lock.lock();
- try {
- final LeafReaderContext ctx = slice.leaves[0];
- final int base = ctx.docBase;
- final LeafCollector collector = hq.getLeafCollector(ctx);
- collector.setScorer(fakeScorer);
- for(ScoreDoc scoreDoc : docs.scoreDocs) {
- fakeScorer.doc = scoreDoc.doc - base;
- fakeScorer.score = scoreDoc.score;
- collector.collect(scoreDoc.doc-base);
- }
-
- // Carry over maxScore from sub:
- if (doMaxScore && docs.getMaxScore() > hq.maxScore) {
- hq.maxScore = docs.getMaxScore();
- }
- } finally {
- lock.unlock();
- }
- return docs;
- }
- }
-
- /**
- * A helper class that wraps a {@link CompletionService} and provides an
- * iterable interface to the completed {@link Callable} instances.
- *
- * @param <T>
- * the type of the {@link Callable} return value
- */
- private static final class ExecutionHelper<T> implements Iterator<T>, Iterable<T> {
- private final CompletionService<T> service;
- private int numTasks;
-
- ExecutionHelper(final Executor executor) {
- this.service = new ExecutorCompletionService<>(executor);
- }
-
- @Override
- public boolean hasNext() {
- return numTasks > 0;
- }
-
- public void submit(Callable<T> task) {
- this.service.submit(task);
- ++numTasks;
- }
-
- @Override
- public T next() {
- if(!this.hasNext())
- throw new NoSuchElementException("next() is called but hasNext() returned false");
- try {
- return service.take().get();
- } catch (InterruptedException e) {
- throw new ThreadInterruptedException(e);
- } catch (ExecutionException e) {
- throw new RuntimeException(e);
- } finally {
- --numTasks;
- }
- }
-
- @Override
- public void remove() {
- throw new UnsupportedOperationException();
- }
-
- @Override
- public Iterator<T> iterator() {
- // use the shortcut here - this is only used in a private context
- return this;
+ return searcher.search(Arrays.asList(slice.leaves),
+ weight, after, nDocs, sort, true, doDocScores, doMaxScore);
}
}
Modified: lucene/dev/branches/lucene6005/lucene/core/src/java/org/apache/lucene/search/LRUFilterCache.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene6005/lucene/core/src/java/org/apache/lucene/search/LRUFilterCache.java?rev=1658277&r1=1658276&r2=1658277&view=diff
==============================================================================
--- lucene/dev/branches/lucene6005/lucene/core/src/java/org/apache/lucene/search/LRUFilterCache.java (original)
+++ lucene/dev/branches/lucene6005/lucene/core/src/java/org/apache/lucene/search/LRUFilterCache.java Sun Feb 8 23:53:14 2015
@@ -49,6 +49,38 @@ import org.apache.lucene.util.RoaringDoc
* {@link FilterCachingPolicy 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 maxNumberOfCachedFilters = 256;
+ * final long maxRamBytesUsed = 50 * 1024L * 1024L; // 50MB
+ * // these cache and policy instances can be shared across several filters and readers
+ * // it is fine to eg. store them into static variables
+ * final FilterCache filterCache = new LRUFilterCache(maxNumberOfCachedFilters, maxRamBytesUsed);
+ * final FilterCachingPolicy defaultCachingPolicy = new UsageTrackingFilterCachingPolicy();
+ *
+ * // ...
+ *
+ * // Then at search time
+ * Filter myFilter = ...;
+ * Filter myCacheFilter = filterCache.doCache(myFilter, defaultCachingPolicy);
+ * // myCacheFilter is now a wrapper around the original filter that will interact with the cache
+ * IndexSearcher searcher = ...;
+ * TopDocs topDocs = searcher.search(new ConstantScoreQuery(myCacheFilter), 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-filter-class statistics, it is possible to override various callbacks:
+ * {@link #onHit}, {@link #onMiss},
+ * {@link #onFilterCache}, {@link #onFilterEviction},
+ * {@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 FilterCachingPolicy
* @lucene.experimental
*/
public class LRUFilterCache implements FilterCache, Accountable {
@@ -96,6 +128,80 @@ public class LRUFilterCache implements F
ramBytesUsed = 0;
}
+ /**
+ * Expert: callback when there is a cache hit on a given filter.
+ * Implementing this method is typically useful in order to compute more
+ * fine-grained statistics about the filter cache.
+ * @see #onMiss
+ * @lucene.experimental
+ */
+ protected void onHit(Object readerCoreKey, Filter filter) {
+ hitCount += 1;
+ }
+
+ /**
+ * Expert: callback when there is a cache miss on a given filter.
+ * @see #onHit
+ * @lucene.experimental
+ */
+ protected void onMiss(Object readerCoreKey, Filter filter) {
+ assert filter != null;
+ missCount += 1;
+ }
+
+ /**
+ * Expert: callback when a filter is added to this cache.
+ * Implementing this method is typically useful in order to compute more
+ * fine-grained statistics about the filter cache.
+ * @see #onFilterEviction
+ * @lucene.experimental
+ */
+ protected void onFilterCache(Filter filter, long ramBytesUsed) {
+ this.ramBytesUsed += ramBytesUsed;
+ }
+
+ /**
+ * Expert: callback when a filter is evicted from this cache.
+ * @see #onFilterCache
+ * @lucene.experimental
+ */
+ protected void onFilterEviction(Filter filter, 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 filter 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 = mostRecentlyUsedFilters.size();
@@ -107,22 +213,23 @@ public class LRUFilterCache implements F
}
synchronized DocIdSet get(Filter filter, LeafReaderContext context) {
- final LeafCache leafCache = cache.get(context.reader().getCoreCacheKey());
+ final Object readerKey = context.reader().getCoreCacheKey();
+ final LeafCache leafCache = cache.get(readerKey);
if (leafCache == null) {
- missCount += 1;
+ onMiss(readerKey, filter);
return null;
}
// this get call moves the filter to the most-recently-used position
final Filter singleton = uniqueFilters.get(filter);
if (singleton == null) {
- missCount += 1;
+ onMiss(readerKey, filter);
return null;
}
final DocIdSet cached = leafCache.get(singleton);
if (cached == null) {
- missCount += 1;
+ onMiss(readerKey, singleton);
} else {
- hitCount += 1;
+ onHit(readerKey, singleton);
}
return cached;
}
@@ -132,13 +239,14 @@ public class LRUFilterCache implements F
assert set.isCacheable();
Filter singleton = uniqueFilters.putIfAbsent(filter, filter);
if (singleton == null) {
- ramBytesUsed += LINKED_HASHTABLE_RAM_BYTES_PER_ENTRY + ramBytesUsed(filter);
+ onFilterCache(singleton, LINKED_HASHTABLE_RAM_BYTES_PER_ENTRY + ramBytesUsed(filter));
} else {
filter = singleton;
}
- LeafCache leafCache = cache.get(context.reader().getCoreCacheKey());
+ final Object key = context.reader().getCoreCacheKey();
+ LeafCache leafCache = cache.get(key);
if (leafCache == null) {
- leafCache = new LeafCache();
+ leafCache = new LeafCache(key);
final LeafCache previous = cache.put(context.reader().getCoreCacheKey(), leafCache);
ramBytesUsed += HASHTABLE_RAM_BYTES_PER_ENTRY;
assert previous == null;
@@ -172,8 +280,8 @@ public class LRUFilterCache implements F
public synchronized void clearCoreCacheKey(Object coreKey) {
final LeafCache leafCache = cache.remove(coreKey);
if (leafCache != null) {
- ramBytesUsed -= leafCache.ramBytesUsed + HASHTABLE_RAM_BYTES_PER_ENTRY;
- cacheSize -= leafCache.cache.size();
+ ramBytesUsed -= HASHTABLE_RAM_BYTES_PER_ENTRY;
+ onDocIdSetEviction(coreKey, leafCache.cache.size(), leafCache.ramBytesUsed);
}
}
@@ -188,7 +296,7 @@ public class LRUFilterCache implements F
}
private void onEviction(Filter singleton) {
- ramBytesUsed -= LINKED_HASHTABLE_RAM_BYTES_PER_ENTRY + ramBytesUsed(singleton);
+ onFilterEviction(singleton, LINKED_HASHTABLE_RAM_BYTES_PER_ENTRY + ramBytesUsed(singleton));
for (LeafCache leafCache : cache.values()) {
leafCache.remove(singleton);
}
@@ -200,8 +308,7 @@ public class LRUFilterCache implements F
public synchronized void clear() {
cache.clear();
mostRecentlyUsedFilters.clear();
- ramBytesUsed = 0;
- cacheSize = 0;
+ onClear();
}
// pkg-private for testing
@@ -388,17 +495,24 @@ public class LRUFilterCache implements F
// 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<Filter, DocIdSet> cache;
private volatile long ramBytesUsed;
- LeafCache() {
+ LeafCache(Object key) {
+ this.key = key;
cache = new IdentityHashMap<>();
ramBytesUsed = 0;
}
- private void incrementRamBytesUsed(long inc) {
- ramBytesUsed += inc;
- LRUFilterCache.this.ramBytesUsed += inc;
+ private void onDocIdSetCache(long ramBytesUsed) {
+ this.ramBytesUsed += ramBytesUsed;
+ LRUFilterCache.this.onDocIdSetCache(key, ramBytesUsed);
+ }
+
+ private void onDocIdSetEviction(long ramBytesUsed) {
+ this.ramBytesUsed -= ramBytesUsed;
+ LRUFilterCache.this.onDocIdSetEviction(key, 1, ramBytesUsed);
}
DocIdSet get(Filter filter) {
@@ -408,17 +522,14 @@ public class LRUFilterCache implements F
void putIfAbsent(Filter filter, DocIdSet set) {
if (cache.putIfAbsent(filter, set) == null) {
// the set was actually put
- cacheCount += 1;
- cacheSize += 1;
- incrementRamBytesUsed(HASHTABLE_RAM_BYTES_PER_ENTRY + set.ramBytesUsed());
+ onDocIdSetCache(HASHTABLE_RAM_BYTES_PER_ENTRY + set.ramBytesUsed());
}
}
void remove(Filter filter) {
DocIdSet removed = cache.remove(filter);
if (removed != null) {
- cacheSize -= 1;
- incrementRamBytesUsed(-(HASHTABLE_RAM_BYTES_PER_ENTRY + removed.ramBytesUsed()));
+ onDocIdSetEviction(HASHTABLE_RAM_BYTES_PER_ENTRY + removed.ramBytesUsed());
}
}
Modified: lucene/dev/branches/lucene6005/lucene/core/src/java/org/apache/lucene/search/LeafCollector.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene6005/lucene/core/src/java/org/apache/lucene/search/LeafCollector.java?rev=1658277&r1=1658276&r2=1658277&view=diff
==============================================================================
--- lucene/dev/branches/lucene6005/lucene/core/src/java/org/apache/lucene/search/LeafCollector.java (original)
+++ lucene/dev/branches/lucene6005/lucene/core/src/java/org/apache/lucene/search/LeafCollector.java Sun Feb 8 23:53:14 2015
@@ -56,14 +56,9 @@ import java.io.IOException;
* bits.set(docBase + doc);
* }
*
- * // accept docs out of order (for a BitSet it doesn't matter)
- * public boolean acceptsDocsOutOfOrder() {
- * return true;
- * }
- *
* };
* }
- *
+ *
* });
* </pre>
*
@@ -98,22 +93,4 @@ public interface LeafCollector {
*/
void collect(int doc) throws IOException;
- /**
- * Return <code>true</code> if this collector does not
- * require the matching docIDs to be delivered in int sort
- * order (smallest to largest) to {@link #collect}.
- *
- * <p> Most Lucene Query implementations will visit
- * matching docIDs in order. However, some queries
- * (currently limited to certain cases of {@link
- * BooleanQuery}) can achieve faster searching if the
- * <code>Collector</code> allows them to deliver the
- * docIDs out of order.</p>
- *
- * <p> Many collectors don't mind getting docIDs out of
- * order, so it's important to return <code>true</code>
- * here.
- */
- boolean acceptsDocsOutOfOrder();
-
}
Modified: lucene/dev/branches/lucene6005/lucene/core/src/java/org/apache/lucene/search/MatchAllDocsQuery.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene6005/lucene/core/src/java/org/apache/lucene/search/MatchAllDocsQuery.java?rev=1658277&r1=1658276&r2=1658277&view=diff
==============================================================================
--- lucene/dev/branches/lucene6005/lucene/core/src/java/org/apache/lucene/search/MatchAllDocsQuery.java (original)
+++ lucene/dev/branches/lucene6005/lucene/core/src/java/org/apache/lucene/search/MatchAllDocsQuery.java Sun Feb 8 23:53:14 2015
@@ -56,7 +56,7 @@ public class MatchAllDocsQuery extends Q
while(liveDocs != null && doc < maxDoc && !liveDocs.get(doc)) {
doc++;
}
- if (doc == maxDoc) {
+ if (doc >= maxDoc) { // can be > maxDoc when called from advance()
doc = NO_MORE_DOCS;
}
return doc;
Modified: lucene/dev/branches/lucene6005/lucene/core/src/java/org/apache/lucene/search/MultiCollector.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene6005/lucene/core/src/java/org/apache/lucene/search/MultiCollector.java?rev=1658277&r1=1658276&r2=1658277&view=diff
==============================================================================
--- lucene/dev/branches/lucene6005/lucene/core/src/java/org/apache/lucene/search/MultiCollector.java (original)
+++ lucene/dev/branches/lucene6005/lucene/core/src/java/org/apache/lucene/search/MultiCollector.java Sun Feb 8 23:53:14 2015
@@ -124,16 +124,6 @@ public class MultiCollector implements C
}
}
- @Override
- public boolean acceptsDocsOutOfOrder() {
- for (LeafCollector c : collectors) {
- if (!c.acceptsDocsOutOfOrder()) {
- return false;
- }
- }
- return true;
- }
-
}
}
Modified: lucene/dev/branches/lucene6005/lucene/core/src/java/org/apache/lucene/search/MultiPhraseQuery.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene6005/lucene/core/src/java/org/apache/lucene/search/MultiPhraseQuery.java?rev=1658277&r1=1658276&r2=1658277&view=diff
==============================================================================
--- lucene/dev/branches/lucene6005/lucene/core/src/java/org/apache/lucene/search/MultiPhraseQuery.java (original)
+++ lucene/dev/branches/lucene6005/lucene/core/src/java/org/apache/lucene/search/MultiPhraseQuery.java Sun Feb 8 23:53:14 2015
@@ -499,7 +499,7 @@ class UnionDocsAndPositionsEnum extends
@Override
public final int nextDoc() throws IOException {
if (_queue.size() == 0) {
- return NO_MORE_DOCS;
+ return _doc = NO_MORE_DOCS;
}
// TODO: move this init into positions(): if the search
Modified: lucene/dev/branches/lucene6005/lucene/core/src/java/org/apache/lucene/search/NumericComparator.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene6005/lucene/core/src/java/org/apache/lucene/search/NumericComparator.java?rev=1658277&r1=1658276&r2=1658277&view=diff
==============================================================================
--- lucene/dev/branches/lucene6005/lucene/core/src/java/org/apache/lucene/search/NumericComparator.java (original)
+++ lucene/dev/branches/lucene6005/lucene/core/src/java/org/apache/lucene/search/NumericComparator.java Sun Feb 8 23:53:14 2015
@@ -27,7 +27,7 @@ import org.apache.lucene.util.Bits;
/**
* Base FieldComparator class for numeric types
*/
-public abstract class NumericComparator<T extends Number> extends FieldComparator<T> {
+public abstract class NumericComparator<T extends Number> extends SimpleFieldComparator<T> {
private final long[] values;
private final long missingValue;
private long bottom;
@@ -78,14 +78,13 @@ public abstract class NumericComparator<
}
@Override
- public FieldComparator<T> setNextReader(LeafReaderContext context) throws IOException {
+ public void doSetNextReader(LeafReaderContext context) throws IOException {
currentReaderValues = getNumericDocValues(context, field);
docsWithField = DocValues.getDocsWithField(context.reader(), field);
// optimization to remove unneeded checks on the bit interface:
if (docsWithField instanceof Bits.MatchAllBits) {
docsWithField = null;
}
- return this;
}
@Override
Modified: lucene/dev/branches/lucene6005/lucene/core/src/java/org/apache/lucene/search/SimpleCollector.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene6005/lucene/core/src/java/org/apache/lucene/search/SimpleCollector.java?rev=1658277&r1=1658276&r2=1658277&view=diff
==============================================================================
--- lucene/dev/branches/lucene6005/lucene/core/src/java/org/apache/lucene/search/SimpleCollector.java (original)
+++ lucene/dev/branches/lucene6005/lucene/core/src/java/org/apache/lucene/search/SimpleCollector.java Sun Feb 8 23:53:14 2015
@@ -45,9 +45,6 @@ public abstract class SimpleCollector im
// redeclare methods so that javadocs are inherited on sub-classes
@Override
- public abstract boolean acceptsDocsOutOfOrder();
-
- @Override
public abstract void collect(int doc) throws IOException;
}
Modified: lucene/dev/branches/lucene6005/lucene/core/src/java/org/apache/lucene/search/SortRescorer.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene6005/lucene/core/src/java/org/apache/lucene/search/SortRescorer.java?rev=1658277&r1=1658276&r2=1658277&view=diff
==============================================================================
--- lucene/dev/branches/lucene6005/lucene/core/src/java/org/apache/lucene/search/SortRescorer.java (original)
+++ lucene/dev/branches/lucene6005/lucene/core/src/java/org/apache/lucene/search/SortRescorer.java Sun Feb 8 23:53:14 2015
@@ -53,7 +53,7 @@ public class SortRescorer extends Rescor
List<LeafReaderContext> leaves = searcher.getIndexReader().leaves();
- TopFieldCollector collector = TopFieldCollector.create(sort, topN, true, true, true, false);
+ TopFieldCollector collector = TopFieldCollector.create(sort, topN, true, true, true);
// Now merge sort docIDs from hits, with reader's leaves:
int hitUpto = 0;
@@ -61,6 +61,7 @@ public class SortRescorer extends Rescor
int endDoc = 0;
int docBase = 0;
+ LeafCollector leafCollector = null;
FakeScorer fakeScorer = new FakeScorer();
while (hitUpto < hits.length) {
@@ -75,15 +76,15 @@ public class SortRescorer extends Rescor
if (readerContext != null) {
// We advanced to another segment:
- collector.getLeafCollector(readerContext);
- collector.setScorer(fakeScorer);
+ leafCollector = collector.getLeafCollector(readerContext);
+ leafCollector.setScorer(fakeScorer);
docBase = readerContext.docBase;
}
fakeScorer.score = hit.score;
fakeScorer.doc = docID - docBase;
- collector.collect(fakeScorer.doc);
+ leafCollector.collect(fakeScorer.doc);
hitUpto++;
}
Modified: lucene/dev/branches/lucene6005/lucene/core/src/java/org/apache/lucene/search/TopDocsCollector.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene6005/lucene/core/src/java/org/apache/lucene/search/TopDocsCollector.java?rev=1658277&r1=1658276&r2=1658277&view=diff
==============================================================================
--- lucene/dev/branches/lucene6005/lucene/core/src/java/org/apache/lucene/search/TopDocsCollector.java (original)
+++ lucene/dev/branches/lucene6005/lucene/core/src/java/org/apache/lucene/search/TopDocsCollector.java Sun Feb 8 23:53:14 2015
@@ -31,7 +31,7 @@ import org.apache.lucene.util.PriorityQu
* however, you might want to consider overriding all methods, in order to avoid
* a NullPointerException.
*/
-public abstract class TopDocsCollector<T extends ScoreDoc> extends SimpleCollector {
+public abstract class TopDocsCollector<T extends ScoreDoc> implements Collector {
/** This is used in case topDocs() is called with illegal parameters, or there
* simply aren't (enough) results. */