Merging with trunk.

git-svn-id: https://svn.apache.org/repos/asf/lucene/dev/branches/solr5914@1585024 13f79535-47bb-0310-9956-ffa450edef68
This commit is contained in:
Dawid Weiss 2014-04-05 09:02:57 +00:00
commit c72e8820a1
130 changed files with 1899 additions and 1642 deletions
dev-tools/idea/.idea/libraries
lucene
CHANGES.txtMIGRATE.txt
core/src
facet/src
grouping/src/java/org/apache/lucene/search/grouping
highlighter/src/test/org/apache/lucene/search/highlight
join/src
memory/src/java/org/apache/lucene/index/memory
misc/src
java/org/apache/lucene/index/sorter
test/org/apache/lucene/index/sorter
queryparser/src/test/org/apache/lucene/queryparser/surround/query
spatial/src

View File

@ -2,7 +2,7 @@
<library name="JUnit">
<CLASSES>
<root url="jar://$PROJECT_DIR$/lucene/test-framework/lib/junit-4.10.jar!/" />
<root url="jar://$PROJECT_DIR$/lucene/test-framework/lib/randomizedtesting-runner-2.1.1.jar!/" />
<root url="jar://$PROJECT_DIR$/lucene/test-framework/lib/randomizedtesting-runner-2.1.3.jar!/" />
</CLASSES>
<JAVADOC />
<SOURCES />

View File

@ -50,6 +50,9 @@ API Changes
via setReader.
(Benson Margulies via Robert Muir - pull request #16)
* LUCENE-5527: The Collector API has been refactored to use a dedicated Collector
per leaf. (Shikhar Bhushan, Adrien Grand)
Documentation
* LUCENE-5392: Add/improve analysis package documentation to reflect
@ -187,6 +190,8 @@ API Changes
* LUCENE-5543: Remove/deprecate Directory.fileExists (Mike McCandless)
* LUCENE-5565: Refactor SpatialPrefixTree/Cell to not use Strings. (David Smiley)
Optimizations
* LUCENE-5468: HunspellStemFilter uses 10 to 100x less RAM. It also loads
@ -231,6 +236,16 @@ Bug fixes
* LUCENE-5568: Benchmark module's "default.codec" option didn't work. (David Smiley)
* LUCENE-5574: Closing a near-real-time reader no longer attempts to
delete unreferenced files if the original writer has been closed;
this could cause index corruption in certain cases where index files
were directly changed (deleted, overwritten, etc.) in the index
directory outside of Lucene. (Simon Willnauer, Shai Erera, Robert
Muir, Mike McCandless)
* LUCENE-5570: Don't let FSDirectory.sync() create new zero-byte files, instead throw
exception if a file is missing. (Uwe Schindler, Mike McCandless, Robert Muir)
Test Framework
* LUCENE-5577: Temporary folder and file management (and cleanup facilities)

View File

@ -12,3 +12,10 @@ of the return type is enough to upgrade.
The constructor of Tokenizer no longer takes Reader, as this was a leftover
from before it was reusable. See the org.apache.lucene.analysis package
documentation for more details.
## Refactored Collector API (LUCENE-5299)
The Collector API has been refactored to use a different Collector instance
per segment. It is possible to migrate existing collectors painlessly by
extending SimpleCollector instead of Collector: SimpleCollector is a
specialization of Collector that returns itself as a per-segment Collector.

View File

@ -29,6 +29,7 @@ import java.util.List;
import java.util.Map;
import java.util.regex.Matcher;
import org.apache.lucene.store.AlreadyClosedException;
import org.apache.lucene.store.Directory;
import org.apache.lucene.store.NoSuchDirectoryException;
import org.apache.lucene.util.CollectionUtil;
@ -262,6 +263,14 @@ final class IndexFileDeleter implements Closeable {
deleteCommits();
}
private void ensureOpen() throws AlreadyClosedException {
if (writer == null) {
throw new AlreadyClosedException("this IndexWriter is closed");
} else {
writer.ensureOpen(false);
}
}
public SegmentInfos getLastSegmentInfos() {
return lastSegmentInfos;
}
@ -578,6 +587,7 @@ final class IndexFileDeleter implements Closeable {
void deleteFile(String fileName)
throws IOException {
assert locked();
ensureOpen();
try {
if (infoStream.isEnabled("IFD")) {
infoStream.message("IFD", "delete \"" + fileName + "\"");

View File

@ -4566,8 +4566,7 @@ public class IndexWriter implements Closeable, TwoPhaseCommit{
deleter.revisitPolicy();
}
// Called by DirectoryReader.doClose
synchronized void deletePendingFiles() throws IOException {
private synchronized void deletePendingFiles() throws IOException {
deleter.deletePendingFiles();
}
@ -4665,10 +4664,12 @@ public class IndexWriter implements Closeable, TwoPhaseCommit{
}
synchronized void incRefDeleter(SegmentInfos segmentInfos) throws IOException {
ensureOpen();
deleter.incRef(segmentInfos, false);
}
synchronized void decRefDeleter(SegmentInfos segmentInfos) throws IOException {
ensureOpen();
deleter.decRef(segmentInfos);
}

View File

@ -25,6 +25,7 @@ import java.util.HashMap;
import java.util.List;
import java.util.Map;
import org.apache.lucene.store.AlreadyClosedException;
import org.apache.lucene.store.Directory;
import org.apache.lucene.store.IOContext;
import org.apache.lucene.util.IOUtils;
@ -365,11 +366,15 @@ final class StandardDirectoryReader extends DirectoryReader {
}
if (writer != null) {
writer.decRefDeleter(segmentInfos);
// Since we just closed, writer may now be able to
// delete unused files:
writer.deletePendingFiles();
try {
writer.decRefDeleter(segmentInfos);
} catch (AlreadyClosedException ex) {
// This is OK, it just means our original writer was
// closed before we were, and this may leave some
// un-referenced files in the index, which is
// harmless. The next time IW is opened on the
// index, it will delete them.
}
}
// throw the first exception

View File

@ -61,7 +61,7 @@ import org.apache.lucene.search.BooleanQuery.BooleanWeight;
final class BooleanScorer extends BulkScorer {
private static final class BooleanScorerCollector extends Collector {
private static final class BooleanScorerCollector extends SimpleCollector {
private BucketTable bucketTable;
private int mask;
private Scorer scorer;
@ -92,11 +92,6 @@ final class BooleanScorer extends BulkScorer {
}
}
@Override
public void setNextReader(AtomicReaderContext context) {
// not needed by this implementation
}
@Override
public void setScorer(Scorer scorer) {
this.scorer = scorer;
@ -136,7 +131,7 @@ final class BooleanScorer extends BulkScorer {
}
}
public Collector newCollector(int mask) {
public LeafCollector newCollector(int mask) {
return new BooleanScorerCollector(mask, this);
}
@ -148,12 +143,12 @@ final class BooleanScorer extends BulkScorer {
// TODO: re-enable this if BQ ever sends us required clauses
//public boolean required = false;
public boolean prohibited;
public Collector collector;
public LeafCollector collector;
public SubScorer next;
public boolean more;
public SubScorer(BulkScorer scorer, boolean required, boolean prohibited,
Collector collector, SubScorer next) {
LeafCollector collector, SubScorer next) {
if (required) {
throw new IllegalArgumentException("this scorer cannot handle required=true");
}
@ -200,7 +195,7 @@ final class BooleanScorer extends BulkScorer {
}
@Override
public boolean score(Collector collector, int max) throws IOException {
public boolean score(LeafCollector collector, int max) throws IOException {
boolean more;
Bucket tmp;

View File

@ -31,7 +31,7 @@ public abstract class BulkScorer {
/** Scores and collects all matching documents.
* @param collector The collector to which all matching documents are passed.
*/
public void score(Collector collector) throws IOException {
public void score(LeafCollector collector) throws IOException {
score(collector, Integer.MAX_VALUE);
}
@ -42,5 +42,5 @@ public abstract class BulkScorer {
* @param max Score up to, but not including, this doc
* @return true if more matching documents may remain.
*/
public abstract boolean score(Collector collector, int max) throws IOException;
public abstract boolean score(LeafCollector collector, int max) throws IOException;
}

View File

@ -18,10 +18,12 @@ package org.apache.lucene.search;
*/
import org.apache.lucene.index.AtomicReaderContext;
import org.apache.lucene.util.ArrayUtil;
import org.apache.lucene.util.RamUsageEstimator;
import java.io.IOException;
import java.util.ArrayList;
import java.util.Arrays;
import java.util.List;
/**
@ -38,317 +40,280 @@ import java.util.List;
* scoring is cached) per collected document. If the result
* set is large this can easily be a very substantial amount
* of RAM!
*
* <p><b>NOTE</b>: this class caches at least 128 documents
* before checking RAM limits.
*
*
* <p>See the Lucene <tt>modules/grouping</tt> module for more
* details including a full code example.</p>
*
* @lucene.experimental
*/
public abstract class CachingCollector extends Collector {
// Max out at 512K arrays
private static final int MAX_ARRAY_SIZE = 512 * 1024;
public abstract class CachingCollector extends FilterCollector {
private static final int INITIAL_ARRAY_SIZE = 128;
private final static int[] EMPTY_INT_ARRAY = new int[0];
private static class SegStart {
public final AtomicReaderContext readerContext;
public final int end;
public SegStart(AtomicReaderContext readerContext, int end) {
this.readerContext = readerContext;
this.end = end;
}
}
private static final class CachedScorer extends Scorer {
// NOTE: these members are package-private b/c that way accessing them from
// the outer class does not incur access check by the JVM. The same
// situation would be if they were defined in the outer class as private
// members.
int doc;
float score;
private CachedScorer() { super(null); }
@Override
public final float score() { return score; }
@Override
public final int advance(int target) { throw new UnsupportedOperationException(); }
@Override
public final int docID() { return doc; }
@Override
public final int freq() { throw new UnsupportedOperationException(); }
@Override
public final int nextDoc() { throw new UnsupportedOperationException(); }
@Override
public long cost() { return 1; }
}
private static class NoScoreCachingCollector extends CachingCollector {
List<Boolean> acceptDocsOutOfOrders;
List<AtomicReaderContext> contexts;
List<int[]> docs;
int maxDocsToCache;
NoScoreCachingLeafCollector lastCollector;
NoScoreCachingCollector(Collector in, int maxDocsToCache) {
super(in);
this.maxDocsToCache = maxDocsToCache;
contexts = new ArrayList<>();
acceptDocsOutOfOrders = new ArrayList<>();
docs = new ArrayList<>();
}
// A CachingCollector which caches scores
private static final class ScoreCachingCollector extends CachingCollector {
private final CachedScorer cachedScorer;
private final List<float[]> cachedScores;
private Scorer scorer;
private float[] curScores;
ScoreCachingCollector(Collector other, double maxRAMMB) {
super(other, maxRAMMB, true);
cachedScorer = new CachedScorer();
cachedScores = new ArrayList<>();
curScores = new float[INITIAL_ARRAY_SIZE];
cachedScores.add(curScores);
protected NoScoreCachingLeafCollector wrap(LeafCollector in, int maxDocsToCache) {
return new NoScoreCachingLeafCollector(in, maxDocsToCache);
}
ScoreCachingCollector(Collector other, int maxDocsToCache) {
super(other, maxDocsToCache);
cachedScorer = new CachedScorer();
cachedScores = new ArrayList<>();
curScores = new float[INITIAL_ARRAY_SIZE];
cachedScores.add(curScores);
public LeafCollector getLeafCollector(AtomicReaderContext context) throws IOException {
postCollection();
final LeafCollector in = this.in.getLeafCollector(context);
if (contexts != null) {
contexts.add(context);
acceptDocsOutOfOrders.add(in.acceptsDocsOutOfOrder());
}
if (maxDocsToCache >= 0) {
return lastCollector = wrap(in, maxDocsToCache);
} else {
return in;
}
}
protected void invalidate() {
maxDocsToCache = -1;
contexts = null;
this.docs = null;
}
protected void postCollect(NoScoreCachingLeafCollector collector) {
final int[] docs = collector.cachedDocs();
maxDocsToCache -= docs.length;
this.docs.add(docs);
}
private void postCollection() {
if (lastCollector != null) {
if (!lastCollector.hasCache()) {
invalidate();
} else {
postCollect(lastCollector);
}
lastCollector = null;
}
}
protected void collect(LeafCollector collector, int i) throws IOException {
final int[] docs = this.docs.get(i);
for (int doc : docs) {
collector.collect(doc);
}
}
public void replay(Collector other) throws IOException {
postCollection();
if (!isCached()) {
throw new IllegalStateException("cannot replay: cache was cleared because too much RAM was required");
}
assert docs.size() == contexts.size();
for (int i = 0; i < contexts.size(); ++i) {
final AtomicReaderContext context = contexts.get(i);
final boolean docsInOrder = !acceptDocsOutOfOrders.get(i);
final LeafCollector collector = other.getLeafCollector(context);
if (!collector.acceptsDocsOutOfOrder() && !docsInOrder) {
throw new IllegalArgumentException(
"cannot replay: given collector does not support "
+ "out-of-order collection, while the wrapped collector does. "
+ "Therefore cached documents may be out-of-order.");
}
collect(collector, i);
}
}
}
private static class ScoreCachingCollector extends NoScoreCachingCollector {
List<float[]> scores;
ScoreCachingCollector(Collector in, int maxDocsToCache) {
super(in, maxDocsToCache);
scores = new ArrayList<>();
}
protected NoScoreCachingLeafCollector wrap(LeafCollector in, int maxDocsToCache) {
return new ScoreCachingLeafCollector(in, maxDocsToCache);
}
@Override
protected void postCollect(NoScoreCachingLeafCollector collector) {
final ScoreCachingLeafCollector coll = (ScoreCachingLeafCollector) collector;
super.postCollect(coll);
scores.add(coll.cachedScores());
}
protected void collect(LeafCollector collector, int i) throws IOException {
final int[] docs = this.docs.get(i);
final float[] scores = this.scores.get(i);
assert docs.length == scores.length;
final CachedScorer scorer = new CachedScorer();
collector.setScorer(scorer);
for (int j = 0; j < docs.length; ++j) {
scorer.doc = docs[j];
scorer.score = scores[j];
collector.collect(scorer.doc);
}
}
}
private class NoScoreCachingLeafCollector extends FilterLeafCollector {
final int maxDocsToCache;
int[] docs;
int docCount;
NoScoreCachingLeafCollector(LeafCollector in, int maxDocsToCache) {
super(in);
this.maxDocsToCache = maxDocsToCache;
docs = new int[Math.min(maxDocsToCache, INITIAL_ARRAY_SIZE)];
docCount = 0;
}
protected void grow(int newLen) {
docs = Arrays.copyOf(docs, newLen);
}
protected void invalidate() {
docs = null;
docCount = -1;
cached = false;
}
protected void buffer(int doc) throws IOException {
docs[docCount] = doc;
}
@Override
public void collect(int doc) throws IOException {
if (curDocs == null) {
// Cache was too large
cachedScorer.score = scorer.score();
cachedScorer.doc = doc;
other.collect(doc);
return;
}
// Allocate a bigger array or abort caching
if (upto == curDocs.length) {
base += upto;
// Compute next array length - don't allocate too big arrays
int nextLength = 8*curDocs.length;
if (nextLength > MAX_ARRAY_SIZE) {
nextLength = MAX_ARRAY_SIZE;
}
if (base + nextLength > maxDocsToCache) {
// try to allocate a smaller array
nextLength = maxDocsToCache - base;
if (nextLength <= 0) {
// Too many docs to collect -- clear cache
curDocs = null;
curScores = null;
cachedSegs.clear();
cachedDocs.clear();
cachedScores.clear();
cachedScorer.score = scorer.score();
cachedScorer.doc = doc;
other.collect(doc);
return;
if (docs != null) {
if (docCount >= docs.length) {
if (docCount >= maxDocsToCache) {
invalidate();
} else {
final int newLen = Math.min(ArrayUtil.oversize(docCount + 1, RamUsageEstimator.NUM_BYTES_INT), maxDocsToCache);
grow(newLen);
}
}
curDocs = new int[nextLength];
cachedDocs.add(curDocs);
curScores = new float[nextLength];
cachedScores.add(curScores);
upto = 0;
if (docs != null) {
buffer(doc);
++docCount;
}
}
curDocs[upto] = doc;
cachedScorer.score = curScores[upto] = scorer.score();
upto++;
cachedScorer.doc = doc;
other.collect(doc);
super.collect(doc);
}
@Override
public void replay(Collector other) throws IOException {
replayInit(other);
int curUpto = 0;
int curBase = 0;
int chunkUpto = 0;
curDocs = EMPTY_INT_ARRAY;
for (SegStart seg : cachedSegs) {
other.setNextReader(seg.readerContext);
other.setScorer(cachedScorer);
while (curBase + curUpto < seg.end) {
if (curUpto == curDocs.length) {
curBase += curDocs.length;
curDocs = cachedDocs.get(chunkUpto);
curScores = cachedScores.get(chunkUpto);
chunkUpto++;
curUpto = 0;
}
cachedScorer.score = curScores[curUpto];
cachedScorer.doc = curDocs[curUpto];
other.collect(curDocs[curUpto++]);
}
}
boolean hasCache() {
return docs != null;
}
int[] cachedDocs() {
return docs == null ? null : Arrays.copyOf(docs, docCount);
}
}
private class ScoreCachingLeafCollector extends NoScoreCachingLeafCollector {
Scorer scorer;
float[] scores;
ScoreCachingLeafCollector(LeafCollector in, int maxDocsToCache) {
super(in, maxDocsToCache);
scores = new float[docs.length];
}
@Override
public void setScorer(Scorer scorer) throws IOException {
this.scorer = scorer;
other.setScorer(cachedScorer);
super.setScorer(scorer);
}
@Override
public String toString() {
if (isCached()) {
return "CachingCollector (" + (base+upto) + " docs & scores cached)";
} else {
return "CachingCollector (cache was cleared)";
}
protected void grow(int newLen) {
super.grow(newLen);
scores = Arrays.copyOf(scores, newLen);
}
@Override
protected void invalidate() {
super.invalidate();
scores = null;
}
@Override
protected void buffer(int doc) throws IOException {
super.buffer(doc);
scores[docCount] = scorer.score();
}
float[] cachedScores() {
return docs == null ? null : Arrays.copyOf(scores, docCount);
}
}
// A CachingCollector which does not cache scores
private static final class NoScoreCachingCollector extends CachingCollector {
NoScoreCachingCollector(Collector other, double maxRAMMB) {
super(other, maxRAMMB, false);
}
NoScoreCachingCollector(Collector other, int maxDocsToCache) {
super(other, maxDocsToCache);
}
@Override
public void collect(int doc) throws IOException {
if (curDocs == null) {
// Cache was too large
other.collect(doc);
return;
}
// Allocate a bigger array or abort caching
if (upto == curDocs.length) {
base += upto;
// Compute next array length - don't allocate too big arrays
int nextLength = 8*curDocs.length;
if (nextLength > MAX_ARRAY_SIZE) {
nextLength = MAX_ARRAY_SIZE;
}
if (base + nextLength > maxDocsToCache) {
// try to allocate a smaller array
nextLength = maxDocsToCache - base;
if (nextLength <= 0) {
// Too many docs to collect -- clear cache
curDocs = null;
cachedSegs.clear();
cachedDocs.clear();
other.collect(doc);
return;
}
}
curDocs = new int[nextLength];
cachedDocs.add(curDocs);
upto = 0;
}
curDocs[upto] = doc;
upto++;
other.collect(doc);
}
@Override
public void replay(Collector other) throws IOException {
replayInit(other);
int curUpto = 0;
int curbase = 0;
int chunkUpto = 0;
curDocs = EMPTY_INT_ARRAY;
for (SegStart seg : cachedSegs) {
other.setNextReader(seg.readerContext);
while (curbase + curUpto < seg.end) {
if (curUpto == curDocs.length) {
curbase += curDocs.length;
curDocs = cachedDocs.get(chunkUpto);
chunkUpto++;
curUpto = 0;
}
other.collect(curDocs[curUpto++]);
}
}
}
@Override
public void setScorer(Scorer scorer) throws IOException {
other.setScorer(scorer);
}
@Override
public String toString() {
if (isCached()) {
return "CachingCollector (" + (base+upto) + " docs cached)";
} else {
return "CachingCollector (cache was cleared)";
}
}
}
// TODO: would be nice if a collector defined a
// needsScores() method so we can specialize / do checks
// up front. This is only relevant for the ScoreCaching
// version -- if the wrapped Collector does not need
// scores, it can avoid cachedScorer entirely.
protected final Collector other;
protected final int maxDocsToCache;
protected final List<SegStart> cachedSegs = new ArrayList<>();
protected final List<int[]> cachedDocs;
private AtomicReaderContext lastReaderContext;
protected int[] curDocs;
protected int upto;
protected int base;
protected int lastDocBase;
/**
* Creates a {@link CachingCollector} which does not wrap another collector.
* The cached documents and scores can later be {@link #replay(Collector)
* replayed}.
*
*
* @param acceptDocsOutOfOrder
* whether documents are allowed to be collected out-of-order
*/
public static CachingCollector create(final boolean acceptDocsOutOfOrder, boolean cacheScores, double maxRAMMB) {
Collector other = new Collector() {
Collector other = new SimpleCollector() {
@Override
public boolean acceptsDocsOutOfOrder() {
return acceptDocsOutOfOrder;
}
@Override
public void setScorer(Scorer scorer) {}
@Override
public void collect(int doc) {}
@Override
public void setNextReader(AtomicReaderContext context) {}
};
return create(other, cacheScores, maxRAMMB);
}
@ -356,7 +321,7 @@ public abstract class CachingCollector extends Collector {
/**
* Create a new {@link CachingCollector} that wraps the given collector and
* caches documents and scores up to the specified RAM threshold.
*
*
* @param other
* the Collector to wrap and delegate calls to.
* @param cacheScores
@ -368,7 +333,12 @@ public abstract class CachingCollector extends Collector {
* scores are cached.
*/
public static CachingCollector create(Collector other, boolean cacheScores, double maxRAMMB) {
return cacheScores ? new ScoreCachingCollector(other, maxRAMMB) : new NoScoreCachingCollector(other, maxRAMMB);
int bytesPerDoc = RamUsageEstimator.NUM_BYTES_INT;
if (cacheScores) {
bytesPerDoc += RamUsageEstimator.NUM_BYTES_FLOAT;
}
final int maxDocsToCache = (int) ((maxRAMMB * 1024 * 1024) / bytesPerDoc);
return create(other, cacheScores, maxDocsToCache);
}
/**
@ -388,74 +358,26 @@ public abstract class CachingCollector extends Collector {
public static CachingCollector create(Collector other, boolean cacheScores, int maxDocsToCache) {
return cacheScores ? new ScoreCachingCollector(other, maxDocsToCache) : new NoScoreCachingCollector(other, maxDocsToCache);
}
// Prevent extension from non-internal classes
private CachingCollector(Collector other, double maxRAMMB, boolean cacheScores) {
this.other = other;
cachedDocs = new ArrayList<>();
curDocs = new int[INITIAL_ARRAY_SIZE];
cachedDocs.add(curDocs);
int bytesPerDoc = RamUsageEstimator.NUM_BYTES_INT;
if (cacheScores) {
bytesPerDoc += RamUsageEstimator.NUM_BYTES_FLOAT;
}
maxDocsToCache = (int) ((maxRAMMB * 1024 * 1024) / bytesPerDoc);
private boolean cached;
private CachingCollector(Collector in) {
super(in);
cached = true;
}
private CachingCollector(Collector other, int maxDocsToCache) {
this.other = other;
cachedDocs = new ArrayList<>();
curDocs = new int[INITIAL_ARRAY_SIZE];
cachedDocs.add(curDocs);
this.maxDocsToCache = maxDocsToCache;
}
@Override
public boolean acceptsDocsOutOfOrder() {
return other.acceptsDocsOutOfOrder();
}
public boolean isCached() {
return curDocs != null;
}
@Override
public void setNextReader(AtomicReaderContext context) throws IOException {
other.setNextReader(context);
if (lastReaderContext != null) {
cachedSegs.add(new SegStart(lastReaderContext, base+upto));
}
lastReaderContext = context;
}
/** Reused by the specialized inner classes. */
void replayInit(Collector other) {
if (!isCached()) {
throw new IllegalStateException("cannot replay: cache was cleared because too much RAM was required");
}
if (!other.acceptsDocsOutOfOrder() && this.other.acceptsDocsOutOfOrder()) {
throw new IllegalArgumentException(
"cannot replay: given collector does not support "
+ "out-of-order collection, while the wrapped collector does. "
+ "Therefore cached documents may be out-of-order.");
}
//System.out.println("CC: replay totHits=" + (upto + base));
if (lastReaderContext != null) {
cachedSegs.add(new SegStart(lastReaderContext, base+upto));
lastReaderContext = null;
}
/**
* Return true is this collector is able to replay collection.
*/
public final boolean isCached() {
return cached;
}
/**
* Replays the cached doc IDs (and scores) to the given Collector. If this
* instance does not cache scores, then Scorer is not set on
* {@code other.setScorer} as well as scores are not replayed.
*
*
* @throws IllegalStateException
* if this collector is not cached (i.e., if the RAM limits were too
* low for the number of documents + scores to cache).
@ -464,5 +386,5 @@ public abstract class CachingCollector extends Collector {
* while the collector passed to the ctor does.
*/
public abstract void replay(Collector other) throws IOException;
}

View File

@ -17,7 +17,7 @@ package org.apache.lucene.search;
* limitations under the License.
*/
/** Throw this exception in {@link Collector#collect(int)} to prematurely
/** Throw this exception in {@link LeafCollector#collect(int)} to prematurely
* terminate collection of the current leaf.
* <p>Note: IndexSearcher swallows this exception and never re-throws it.
* As a consequence, you should not catch it when calling

View File

@ -20,20 +20,19 @@ package org.apache.lucene.search;
import java.io.IOException;
import org.apache.lucene.index.AtomicReaderContext;
import org.apache.lucene.index.IndexReaderContext;
/**
* <p>Expert: Collectors are primarily meant to be used to
* gather raw results from a search, and implement sorting
* or custom result filtering, collation, etc. </p>
*
* <p>Lucene's core collectors are derived from Collector.
* Likely your application can use one of these classes, or
* subclass {@link TopDocsCollector}, instead of
* implementing Collector directly:
* <p>Lucene's core collectors are derived from {@link Collector}
* and {@link SimpleCollector}. Likely your application can
* use one of these classes, or subclass {@link TopDocsCollector},
* instead of implementing Collector directly:
*
* <ul>
*
*
* <li>{@link TopDocsCollector} is an abstract base class
* that assumes you will retrieve the top N docs,
* according to some criteria, after collection is
@ -62,118 +61,16 @@ import org.apache.lucene.index.IndexReaderContext;
*
* </ul>
*
* <p>Collector decouples the score from the collected doc:
* the score computation is skipped entirely if it's not
* needed. Collectors that do need the score should
* implement the {@link #setScorer} method, to hold onto the
* passed {@link Scorer} instance, and call {@link
* Scorer#score()} within the collect method to compute the
* current hit's score. If your collector may request the
* score for a single hit multiple times, you should use
* {@link ScoreCachingWrappingScorer}. </p>
*
* <p><b>NOTE:</b> The doc that is passed to the collect
* method is relative to the current reader. If your
* collector needs to resolve this to the docID space of the
* Multi*Reader, you must re-base it by recording the
* docBase from the most recent setNextReader call. Here's
* a simple example showing how to collect docIDs into a
* BitSet:</p>
*
* <pre class="prettyprint">
* IndexSearcher searcher = new IndexSearcher(indexReader);
* final BitSet bits = new BitSet(indexReader.maxDoc());
* searcher.search(query, new Collector() {
* private int docBase;
*
* <em>// ignore scorer</em>
* public void setScorer(Scorer scorer) {
* }
*
* <em>// accept docs out of order (for a BitSet it doesn't matter)</em>
* public boolean acceptsDocsOutOfOrder() {
* return true;
* }
*
* public void collect(int doc) {
* bits.set(doc + docBase);
* }
*
* public void setNextReader(AtomicReaderContext context) {
* this.docBase = context.docBase;
* }
* });
* </pre>
*
* <p>Not all collectors will need to rebase the docID. For
* example, a collector that simply counts the total number
* of hits would skip it.</p>
*
* <p><b>NOTE:</b> Prior to 2.9, Lucene silently filtered
* out hits with score <= 0. As of 2.9, the core Collectors
* no longer do that. It's very unusual to have such hits
* (a negative query boost, or function query returning
* negative custom scores, could cause it to happen). If
* you need that behavior, use {@link
* PositiveScoresOnlyCollector}.</p>
*
* @lucene.experimental
*
* @since 2.9
*/
public abstract class Collector {
/**
* Called before successive calls to {@link #collect(int)}. Implementations
* that need the score of the current document (passed-in to
* {@link #collect(int)}), should save the passed-in Scorer and call
* scorer.score() when needed.
*/
public abstract void setScorer(Scorer scorer) throws IOException;
/**
* Called once for every document matching a query, with the unbased document
* number.
* <p>Note: The collection of the current segment can be terminated by throwing
* a {@link CollectionTerminatedException}. In this case, the last docs of the
* current {@link AtomicReaderContext} will be skipped and {@link IndexSearcher}
* will swallow the exception and continue collection with the next leaf.
* <p>
* Note: This is called in an inner search loop. For good search performance,
* implementations of this method should not call {@link IndexSearcher#doc(int)} or
* {@link org.apache.lucene.index.IndexReader#document(int)} on every hit.
* Doing so can slow searches by an order of magnitude or more.
*/
public abstract void collect(int doc) throws IOException;
public interface Collector {
/**
* Called before collecting from each {@link AtomicReaderContext}. All doc ids in
* {@link #collect(int)} will correspond to {@link IndexReaderContext#reader}.
*
* Add {@link AtomicReaderContext#docBase} to the current {@link IndexReaderContext#reader}'s
* internal document id to re-base ids in {@link #collect(int)}.
*
* Create a new {@link LeafCollector collector} to collect the given context.
*
* @param context
* next atomic reader context
*/
public abstract void setNextReader(AtomicReaderContext context) throws IOException;
LeafCollector getLeafCollector(AtomicReaderContext context) 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.
*/
public abstract boolean acceptsDocsOutOfOrder();
}

View File

@ -212,31 +212,16 @@ public class ConstantScoreQuery extends Query {
}
@Override
public boolean score(Collector collector, int max) throws IOException {
public boolean score(LeafCollector collector, int max) throws IOException {
return bulkScorer.score(wrapCollector(collector), max);
}
private Collector wrapCollector(final Collector collector) {
return new Collector() {
private LeafCollector wrapCollector(LeafCollector collector) {
return new FilterLeafCollector(collector) {
@Override
public void setScorer(Scorer scorer) throws IOException {
// we must wrap again here, but using the scorer passed in as parameter:
collector.setScorer(new ConstantScorer(scorer, weight, theScore));
}
@Override
public void collect(int doc) throws IOException {
collector.collect(doc);
}
@Override
public void setNextReader(AtomicReaderContext context) throws IOException {
collector.setNextReader(context);
}
@Override
public boolean acceptsDocsOutOfOrder() {
return collector.acceptsDocsOutOfOrder();
in.setScorer(new ConstantScorer(scorer, weight, theScore));
}
};
}

View File

@ -20,7 +20,7 @@ package org.apache.lucene.search;
import java.util.Collection;
/** Used by {@link BulkScorer}s that need to pass a {@link
* Scorer} to {@link Collector#setScorer}. */
* Scorer} to {@link LeafCollector#setScorer}. */
final class FakeScorer extends Scorer {
float score;
int doc = -1;

View File

@ -0,0 +1,48 @@
package org.apache.lucene.search;
import java.io.IOException;
import org.apache.lucene.index.AtomicReaderContext;
/*
* 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.
*/
/**
* {@link Collector} delegator.
*
* @lucene.experimental
*/
public class FilterCollector implements Collector {
protected final Collector in;
/** Sole constructor. */
public FilterCollector(Collector in) {
this.in = in;
}
@Override
public LeafCollector getLeafCollector(AtomicReaderContext context) throws IOException {
return in.getLeafCollector(context);
}
@Override
public String toString() {
return getClass().getSimpleName() + "(" + in + ")";
}
}

View File

@ -0,0 +1,56 @@
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;
/**
* {@link LeafCollector} delegator.
*
* @lucene.experimental
*/
public class FilterLeafCollector implements LeafCollector {
protected final LeafCollector in;
/** Sole constructor. */
public FilterLeafCollector(LeafCollector in) {
this.in = in;
}
@Override
public void setScorer(Scorer scorer) throws IOException {
in.setScorer(scorer);
}
@Override
public void collect(int doc) throws IOException {
in.collect(doc);
}
@Override
public boolean acceptsDocsOutOfOrder() {
return in.acceptsDocsOutOfOrder();
}
@Override
public String toString() {
return getClass().getSimpleName() + "(" + in + ")";
}
}

View File

@ -225,7 +225,7 @@ public class FilteredQuery extends Query {
}
@Override
public boolean score(Collector collector, int maxDoc) throws IOException {
public boolean score(LeafCollector collector, 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);

View File

@ -275,7 +275,7 @@ public class IndexSearcher {
/** Lower-level search API.
*
* <p>{@link Collector#collect(int)} is called for every matching
* <p>{@link LeafCollector#collect(int)} is called for every matching
* document.
*
* @param query to match documents
@ -291,7 +291,7 @@ public class IndexSearcher {
/** Lower-level search API.
*
* <p>{@link Collector#collect(int)} is called for every matching document.
* <p>{@link LeafCollector#collect(int)} is called for every matching document.
*
* @throws BooleanQuery.TooManyClauses If a query would exceed
* {@link BooleanQuery#getMaxClauseCount()} clauses.
@ -578,7 +578,7 @@ public class IndexSearcher {
* Lower-level search API.
*
* <p>
* {@link Collector#collect(int)} is called for every document. <br>
* {@link LeafCollector#collect(int)} is called for every document. <br>
*
* <p>
* NOTE: this method executes the searches on all given leaves exclusively.
@ -600,17 +600,18 @@ public class IndexSearcher {
// threaded...? the Collector could be sync'd?
// always use single thread:
for (AtomicReaderContext ctx : leaves) { // search each subreader
final LeafCollector leafCollector;
try {
collector.setNextReader(ctx);
leafCollector = collector.getLeafCollector(ctx);
} catch (CollectionTerminatedException e) {
// there is no doc of interest in this reader context
// continue with the following leaf
continue;
}
BulkScorer scorer = weight.bulkScorer(ctx, !collector.acceptsDocsOutOfOrder(), ctx.reader().getLiveDocs());
BulkScorer scorer = weight.bulkScorer(ctx, !leafCollector.acceptsDocsOutOfOrder(), ctx.reader().getLiveDocs());
if (scorer != null) {
try {
scorer.score(collector);
scorer.score(leafCollector);
} catch (CollectionTerminatedException e) {
// collection was terminated prematurely
// continue with the following leaf
@ -779,12 +780,12 @@ public class IndexSearcher {
try {
final AtomicReaderContext ctx = slice.leaves[0];
final int base = ctx.docBase;
hq.setNextReader(ctx);
hq.setScorer(fakeScorer);
final LeafCollector collector = hq.getLeafCollector(ctx);
collector.setScorer(fakeScorer);
for(ScoreDoc scoreDoc : docs.scoreDocs) {
fakeScorer.doc = scoreDoc.doc - base;
fakeScorer.score = scoreDoc.score;
hq.collect(scoreDoc.doc-base);
collector.collect(scoreDoc.doc-base);
}
// Carry over maxScore from sub:

View File

@ -0,0 +1,121 @@
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.AtomicReaderContext;
/**
* <p>Collector decouples the score from the collected doc:
* the score computation is skipped entirely if it's not
* needed. Collectors that do need the score should
* implement the {@link #setScorer} method, to hold onto the
* passed {@link Scorer} instance, and call {@link
* Scorer#score()} within the collect method to compute the
* current hit's score. If your collector may request the
* score for a single hit multiple times, you should use
* {@link ScoreCachingWrappingScorer}. </p>
*
* <p><b>NOTE:</b> The doc that is passed to the collect
* method is relative to the current reader. If your
* collector needs to resolve this to the docID space of the
* Multi*Reader, you must re-base it by recording the
* docBase from the most recent setNextReader call. Here's
* a simple example showing how to collect docIDs into a
* BitSet:</p>
*
* <pre class="prettyprint">
* IndexSearcher searcher = new IndexSearcher(indexReader);
* final BitSet bits = new BitSet(indexReader.maxDoc());
* searcher.search(query, new Collector() {
*
* public LeafCollector getLeafCollector(AtomicReaderContext context)
* throws IOException {
* final int docBase = context.docBase;
* return new LeafCollector() {
*
* <em>// ignore scorer</em>
* public void setScorer(Scorer scorer) throws IOException {
* }
*
* public void collect(int doc) throws IOException {
* bits.set(docBase + doc);
* }
*
* // accept docs out of order (for a BitSet it doesn't matter)
* public boolean acceptsDocsOutOfOrder() {
* return true;
* }
*
* };
* }
*
* });
* </pre>
*
* <p>Not all collectors will need to rebase the docID. For
* example, a collector that simply counts the total number
* of hits would skip it.</p>
*
* @lucene.experimental
*/
public interface LeafCollector {
/**
* Called before successive calls to {@link #collect(int)}. Implementations
* that need the score of the current document (passed-in to
* {@link #collect(int)}), should save the passed-in Scorer and call
* scorer.score() when needed.
*/
void setScorer(Scorer scorer) throws IOException;
/**
* Called once for every document matching a query, with the unbased document
* number.
* <p>Note: The collection of the current segment can be terminated by throwing
* a {@link CollectionTerminatedException}. In this case, the last docs of the
* current {@link AtomicReaderContext} will be skipped and {@link IndexSearcher}
* will swallow the exception and continue collection with the next leaf.
* <p>
* Note: This is called in an inner search loop. For good search performance,
* implementations of this method should not call {@link IndexSearcher#doc(int)} or
* {@link org.apache.lucene.index.IndexReader#document(int)} on every hit.
* Doing so can slow searches by an order of magnitude or more.
*/
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();
}

View File

@ -18,6 +18,7 @@ package org.apache.lucene.search;
*/
import java.io.IOException;
import java.util.Arrays;
import org.apache.lucene.index.AtomicReaderContext;
import org.apache.lucene.search.Collector;
@ -29,7 +30,12 @@ import org.apache.lucene.search.Scorer;
* list of collectors and wraps them with {@link MultiCollector}, while
* filtering out the <code>null</code> null ones.
*/
public class MultiCollector extends Collector {
public class MultiCollector implements Collector {
/** See {@link #wrap(Iterable)}. */
public static Collector wrap(Collector... collectors) {
return wrap(Arrays.asList(collectors));
}
/**
* Wraps a list of {@link Collector}s with a {@link MultiCollector}. This
@ -47,7 +53,7 @@ public class MultiCollector extends Collector {
* if either 0 collectors were input, or all collectors are
* <code>null</code>.
*/
public static Collector wrap(Collector... collectors) {
public static Collector wrap(Iterable<? extends Collector> collectors) {
// For the user's convenience, we allow null collectors to be passed.
// However, to improve performance, these null collectors are found
// and dropped from the array we save for actual collection time.
@ -70,8 +76,6 @@ public class MultiCollector extends Collector {
}
}
return col;
} else if (n == collectors.length) {
return new MultiCollector(collectors);
} else {
Collector[] colls = new Collector[n];
n = 0;
@ -91,34 +95,47 @@ public class MultiCollector extends Collector {
}
@Override
public boolean acceptsDocsOutOfOrder() {
for (Collector c : collectors) {
if (!c.acceptsDocsOutOfOrder()) {
return false;
public LeafCollector getLeafCollector(AtomicReaderContext context) throws IOException {
final LeafCollector[] leafCollectors = new LeafCollector[collectors.length];
for (int i = 0; i < collectors.length; ++i) {
leafCollectors[i] = collectors[i].getLeafCollector(context);
}
return new MultiLeafCollector(leafCollectors);
}
private static class MultiLeafCollector implements LeafCollector {
private final LeafCollector[] collectors;
private MultiLeafCollector(LeafCollector[] collectors) {
this.collectors = collectors;
}
@Override
public void setScorer(Scorer scorer) throws IOException {
for (LeafCollector c : collectors) {
c.setScorer(scorer);
}
}
return true;
}
@Override
public void collect(int doc) throws IOException {
for (Collector c : collectors) {
c.collect(doc);
@Override
public void collect(int doc) throws IOException {
for (LeafCollector c : collectors) {
c.collect(doc);
}
}
}
@Override
public void setNextReader(AtomicReaderContext context) throws IOException {
for (Collector c : collectors) {
c.setNextReader(context);
@Override
public boolean acceptsDocsOutOfOrder() {
for (LeafCollector c : collectors) {
if (!c.acceptsDocsOutOfOrder()) {
return false;
}
}
return true;
}
}
@Override
public void setScorer(Scorer s) throws IOException {
for (Collector c : collectors) {
c.setScorer(s);
}
}
}

View File

@ -26,38 +26,33 @@ import org.apache.lucene.index.AtomicReaderContext;
* {@link Collector} and makes sure only documents with
* scores &gt; 0 are collected.
*/
public class PositiveScoresOnlyCollector extends Collector {
public class PositiveScoresOnlyCollector extends FilterCollector {
final private Collector c;
private Scorer scorer;
public PositiveScoresOnlyCollector(Collector c) {
this.c = c;
}
@Override
public void collect(int doc) throws IOException {
if (scorer.score() > 0) {
c.collect(doc);
}
public PositiveScoresOnlyCollector(Collector in) {
super(in);
}
@Override
public void setNextReader(AtomicReaderContext context) throws IOException {
c.setNextReader(context);
}
public LeafCollector getLeafCollector(AtomicReaderContext context)
throws IOException {
return new FilterLeafCollector(super.getLeafCollector(context)) {
@Override
public void setScorer(Scorer scorer) throws IOException {
// Set a ScoreCachingWrappingScorer in case the wrapped Collector will call
// score() also.
this.scorer = new ScoreCachingWrappingScorer(scorer);
c.setScorer(this.scorer);
}
private Scorer scorer;
@Override
public boolean acceptsDocsOutOfOrder() {
return c.acceptsDocsOutOfOrder();
@Override
public void setScorer(Scorer scorer) throws IOException {
this.scorer = new ScoreCachingWrappingScorer(scorer);
in.setScorer(this.scorer);
}
@Override
public void collect(int doc) throws IOException {
if (scorer.score() > 0) {
in.collect(doc);
}
}
};
}
}

View File

@ -37,7 +37,7 @@ public class ScoreCachingWrappingScorer extends Scorer {
private final Scorer scorer;
private int curDoc = -1;
private float curScore;
/** Creates a new instance by wrapping the given scorer. */
public ScoreCachingWrappingScorer(Scorer scorer) {
super(scorer.weight);
@ -51,7 +51,7 @@ public class ScoreCachingWrappingScorer extends Scorer {
curScore = scorer.score();
curDoc = doc;
}
return curScore;
}

View File

@ -57,7 +57,7 @@ public abstract class Scorer extends DocsEnum {
/** Returns the score of the current document matching the query.
* Initially invalid, until {@link #nextDoc()} or {@link #advance(int)}
* is called the first time, or when called from within
* {@link Collector#collect}.
* {@link LeafCollector#collect}.
*/
public abstract float score() throws IOException;

View File

@ -0,0 +1,53 @@
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.AtomicReaderContext;
/**
* Base {@link Collector} implementation that is used to collect all contexts.
*
* @lucene.experimental
*/
public abstract class SimpleCollector implements Collector, LeafCollector {
@Override
public final LeafCollector getLeafCollector(AtomicReaderContext context) throws IOException {
doSetNextReader(context);
return this;
}
/** This method is called before collecting <code>context</code>. */
protected void doSetNextReader(AtomicReaderContext context) throws IOException {}
@Override
public void setScorer(Scorer scorer) throws IOException {
// no-op by default
}
// redeclare methods so that javadocs are inherited on sub-classes
@Override
public abstract boolean acceptsDocsOutOfOrder();
@Override
public abstract void collect(int doc) throws IOException;
}

View File

@ -75,7 +75,7 @@ public class SortRescorer extends Rescorer {
if (readerContext != null) {
// We advanced to another segment:
collector.setNextReader(readerContext);
collector.getLeafCollector(readerContext);
collector.setScorer(fakeScorer);
docBase = readerContext.docBase;
}

View File

@ -29,7 +29,7 @@ import java.io.IOException;
* exceeded, the search thread is stopped by throwing a
* {@link TimeExceededException}.
*/
public class TimeLimitingCollector extends Collector {
public class TimeLimitingCollector implements Collector {
/** Thrown when elapsed search time exceeds allowed search time. */
@ -131,45 +131,30 @@ public class TimeLimitingCollector extends Collector {
this.greedy = greedy;
}
/**
* Calls {@link Collector#collect(int)} on the decorated {@link Collector}
* unless the allowed time has passed, in which case it throws an exception.
*
* @throws TimeExceededException
* if the time allowed has exceeded.
*/
@Override
public void collect(final int doc) throws IOException {
final long time = clock.get();
if (timeout < time) {
if (greedy) {
//System.out.println(this+" greedy: before failing, collecting doc: "+(docBase + doc)+" "+(time-t0));
collector.collect(doc);
}
//System.out.println(this+" failing on: "+(docBase + doc)+" "+(time-t0));
throw new TimeExceededException( timeout-t0, time-t0, docBase + doc );
}
//System.out.println(this+" collecting: "+(docBase + doc)+" "+(time-t0));
collector.collect(doc);
}
@Override
public void setNextReader(AtomicReaderContext context) throws IOException {
collector.setNextReader(context);
public LeafCollector getLeafCollector(AtomicReaderContext context) throws IOException {
this.docBase = context.docBase;
if (Long.MIN_VALUE == t0) {
setBaseline();
}
}
@Override
public void setScorer(Scorer scorer) throws IOException {
collector.setScorer(scorer);
}
@Override
public boolean acceptsDocsOutOfOrder() {
return collector.acceptsDocsOutOfOrder();
return new FilterLeafCollector(collector.getLeafCollector(context)) {
@Override
public void collect(int doc) throws IOException {
final long time = clock.get();
if (timeout < time) {
if (greedy) {
//System.out.println(this+" greedy: before failing, collecting doc: "+(docBase + doc)+" "+(time-t0));
in.collect(doc);
}
//System.out.println(this+" failing on: "+(docBase + doc)+" "+(time-t0));
throw new TimeExceededException( timeout-t0, time-t0, docBase + doc );
}
//System.out.println(this+" collecting: "+(docBase + doc)+" "+(time-t0));
in.collect(doc);
}
};
}
/**

View File

@ -31,7 +31,7 @@ import org.apache.lucene.util.PriorityQueue;
* however, you might want to consider overriding all methods, in order to avoid
* a NullPointerException.
*/
public abstract class TopDocsCollector<T extends ScoreDoc> extends Collector {
public abstract class TopDocsCollector<T extends ScoreDoc> extends SimpleCollector {
/** This is used in case topDocs() is called with illegal parameters, or there
* simply aren't (enough) results. */

View File

@ -92,7 +92,7 @@ public abstract class TopFieldCollector extends TopDocsCollector<Entry> {
}
@Override
public void setNextReader(AtomicReaderContext context) throws IOException {
protected void doSetNextReader(AtomicReaderContext context) throws IOException {
this.docBase = context.docBase;
queue.setComparator(0, comparator.setNextReader(context));
comparator = queue.firstComparator;
@ -446,7 +446,7 @@ public abstract class TopFieldCollector extends TopDocsCollector<Entry> {
}
@Override
public void setNextReader(AtomicReaderContext context) throws IOException {
protected void doSetNextReader(AtomicReaderContext context) throws IOException {
docBase = context.docBase;
for (int i = 0; i < comparators.length; i++) {
queue.setComparator(i, comparators[i].setNextReader(context));
@ -1001,7 +1001,7 @@ public abstract class TopFieldCollector extends TopDocsCollector<Entry> {
}
@Override
public void setNextReader(AtomicReaderContext context) throws IOException {
protected void doSetNextReader(AtomicReaderContext context) throws IOException {
docBase = context.docBase;
afterDoc = after.doc - docBase;
for (int i = 0; i < comparators.length; i++) {

View File

@ -113,9 +113,9 @@ public abstract class TopScoreDocCollector extends TopDocsCollector<ScoreDoc> {
}
@Override
public void setNextReader(AtomicReaderContext context) {
super.setNextReader(context);
afterDoc = after.doc - docBase;
protected void doSetNextReader(AtomicReaderContext context) throws IOException {
super.doSetNextReader(context);
afterDoc = after.doc - context.docBase;
}
@Override
@ -208,9 +208,9 @@ public abstract class TopScoreDocCollector extends TopDocsCollector<ScoreDoc> {
}
@Override
public void setNextReader(AtomicReaderContext context) {
super.setNextReader(context);
afterDoc = after.doc - docBase;
protected void doSetNextReader(AtomicReaderContext context) throws IOException {
super.doSetNextReader(context);
afterDoc = after.doc - context.docBase;
}
@Override
@ -300,7 +300,7 @@ public abstract class TopScoreDocCollector extends TopDocsCollector<ScoreDoc> {
}
@Override
public void setNextReader(AtomicReaderContext context) {
protected void doSetNextReader(AtomicReaderContext context) throws IOException {
docBase = context.docBase;
}

View File

@ -17,13 +17,12 @@ package org.apache.lucene.search;
* limitations under the License.
*/
import org.apache.lucene.index.AtomicReaderContext;
/**
* Just counts the total number of hits.
*/
public class TotalHitCountCollector extends Collector {
public class TotalHitCountCollector extends SimpleCollector {
private int totalHits;
/** Returns how many hits matched the search. */
@ -31,19 +30,11 @@ public class TotalHitCountCollector extends Collector {
return totalHits;
}
@Override
public void setScorer(Scorer scorer) {
}
@Override
public void collect(int doc) {
totalHits++;
}
@Override
public void setNextReader(AtomicReaderContext context) {
}
@Override
public boolean acceptsDocsOutOfOrder() {
return true;

View File

@ -150,7 +150,7 @@ public abstract class Weight {
}
@Override
public boolean score(Collector collector, int max) throws IOException {
public boolean score(LeafCollector collector, int max) throws IOException {
// TODO: this may be sort of weird, when we are
// embedded in a BooleanScorer, because we are
// called for every chunk of 2048 documents. But,
@ -172,7 +172,7 @@ public abstract class Weight {
/**
* Returns true iff this implementation scores docs only out of order. This
* method is used in conjunction with {@link Collector}'s
* {@link Collector#acceptsDocsOutOfOrder() acceptsDocsOutOfOrder} and
* {@link LeafCollector#acceptsDocsOutOfOrder() acceptsDocsOutOfOrder} and
* {@link #bulkScorer(AtomicReaderContext, boolean, Bits)} to
* create a matching {@link Scorer} instance for a given {@link Collector}, or
* vice versa.

View File

@ -508,7 +508,7 @@ on the built-in available scoring models and extending or changing Similarity.
abstract method:
<ol>
<li>
{@link org.apache.lucene.search.BulkScorer#score(org.apache.lucene.search.Collector,int) score(Collector,int)} &mdash;
{@link org.apache.lucene.search.BulkScorer#score(org.apache.lucene.search.LeafCollector,int) score(LeafCollector,int)} &mdash;
Score all documents up to but not including the specified max document.
</li>
</ol>
@ -563,7 +563,7 @@ on the built-in available scoring models and extending or changing Similarity.
<p>If a Filter is being used, some initial setup is done to determine which docs to include.
Otherwise, we ask the Weight for a {@link org.apache.lucene.search.Scorer Scorer} for each
{@link org.apache.lucene.index.IndexReader IndexReader} segment and proceed by calling
{@link org.apache.lucene.search.BulkScorer#score(org.apache.lucene.search.Collector) BulkScorer.score(Collector)}.
{@link org.apache.lucene.search.BulkScorer#score(org.apache.lucene.search.LeafCollector) BulkScorer.score(LeafCollector)}.
</p>
<p>At last, we are actually going to score some documents. The score method takes in the Collector
(most likely the TopScoreDocCollector or TopFieldCollector) and does its business.Of course, here

View File

@ -22,6 +22,8 @@ import java.io.FileNotFoundException;
import java.io.FilenameFilter;
import java.io.IOException;
import java.io.RandomAccessFile;
import java.nio.channels.FileChannel;
import java.nio.file.StandardOpenOption;
import java.util.Collection;
import static java.util.Collections.synchronizedSet;
@ -402,11 +404,11 @@ public abstract class FSDirectory extends BaseDirectory {
IOException exc = null;
while (!success && retryCount < 5) {
retryCount++;
RandomAccessFile file = null;
FileChannel file = null;
try {
try {
file = new RandomAccessFile(fullFile, "rw");
file.getFD().sync();
file = FileChannel.open(fullFile.toPath(), StandardOpenOption.WRITE);
file.force(true); // TODO: we probably dont care about metadata, but this is what we did before...
success = true;
} finally {
if (file != null)

View File

@ -67,6 +67,7 @@ import org.apache.lucene.store.SimpleFSLockFactory;
import org.apache.lucene.store.SingleInstanceLockFactory;
import org.apache.lucene.util.Bits;
import org.apache.lucene.util.BytesRef;
import org.apache.lucene.util.Constants;
import org.apache.lucene.util.IOUtils;
import org.apache.lucene.util.LuceneTestCase;
import org.apache.lucene.util.SetOnce;
@ -2371,4 +2372,47 @@ public class TestIndexWriter extends LuceneTestCase {
r.close();
dir.close();
}
// LUCENE-5574
public void testClosingNRTReaderDoesNotCorruptYourIndex() throws IOException {
// Windows disallows deleting & overwriting files still
// open for reading:
assumeFalse("this test can't run on Windows", Constants.WINDOWS);
MockDirectoryWrapper dir = newMockDirectory();
// Allow deletion of still open files:
dir.setNoDeleteOpenFile(false);
// Allow writing to same file more than once:
dir.setPreventDoubleWrite(false);
IndexWriterConfig iwc = newIndexWriterConfig(TEST_VERSION_CURRENT, new MockAnalyzer(random()));
LogMergePolicy lmp = new LogDocMergePolicy();
lmp.setMergeFactor(2);
iwc.setMergePolicy(lmp);
RandomIndexWriter w = new RandomIndexWriter(random(), dir, iwc);
Document doc = new Document();
doc.add(new TextField("a", "foo", Field.Store.NO));
w.addDocument(doc);
w.commit();
w.addDocument(doc);
// Get a new reader, but this also sets off a merge:
IndexReader r = w.getReader();
w.close();
// Blow away index and make a new writer:
for(String fileName : dir.listAll()) {
dir.deleteFile(fileName);
}
w = new RandomIndexWriter(random(), dir);
w.addDocument(doc);
w.close();
r.close();
dir.close();
}
}

View File

@ -26,6 +26,7 @@ import org.apache.lucene.document.Field;
import org.apache.lucene.document.FieldType;
import org.apache.lucene.document.TextField;
import org.apache.lucene.index.FieldInfo.IndexOptions;
import org.apache.lucene.search.LeafCollector;
import org.apache.lucene.search.BooleanClause.Occur;
import org.apache.lucene.search.BooleanQuery;
import org.apache.lucene.search.CollectionStatistics;
@ -34,6 +35,7 @@ import org.apache.lucene.search.Explanation;
import org.apache.lucene.search.IndexSearcher;
import org.apache.lucene.search.PhraseQuery;
import org.apache.lucene.search.Scorer;
import org.apache.lucene.search.SimpleCollector;
import org.apache.lucene.search.TermQuery;
import org.apache.lucene.search.TermStatistics;
import org.apache.lucene.search.similarities.TFIDFSimilarity;
@ -414,14 +416,12 @@ public class TestOmitTf extends LuceneTestCase {
dir.close();
}
public static class CountingHitCollector extends Collector {
public static class CountingHitCollector extends SimpleCollector {
static int count=0;
static int sum=0;
private int docBase = -1;
CountingHitCollector(){count=0;sum=0;}
@Override
public void setScorer(Scorer scorer) throws IOException {}
@Override
public void collect(int doc) throws IOException {
count++;
sum += doc + docBase; // use it to avoid any possibility of being merged away
@ -431,7 +431,7 @@ public class TestOmitTf extends LuceneTestCase {
public static int getSum() { return sum; }
@Override
public void setNextReader(AtomicReaderContext context) {
protected void doSetNextReader(AtomicReaderContext context) throws IOException {
docBase = context.docBase;
}
@Override

View File

@ -17,6 +17,8 @@ package org.apache.lucene.search;
* limitations under the License.
*/
import java.io.IOException;
import org.apache.lucene.index.AtomicReaderContext;
import org.apache.lucene.index.Terms;
import org.apache.lucene.index.TermsEnum;
@ -37,7 +39,7 @@ final class JustCompileSearch {
private static final String UNSUPPORTED_MSG = "unsupported: used for back-compat testing only !";
static final class JustCompileCollector extends Collector {
static final class JustCompileCollector extends SimpleCollector {
@Override
public void collect(int doc) {
@ -45,7 +47,7 @@ final class JustCompileSearch {
}
@Override
public void setNextReader(AtomicReaderContext context) {
protected void doSetNextReader(AtomicReaderContext context) throws IOException {
throw new UnsupportedOperationException(UNSUPPORTED_MSG);
}
@ -290,7 +292,7 @@ final class JustCompileSearch {
}
@Override
public void setNextReader(AtomicReaderContext context) {
protected void doSetNextReader(AtomicReaderContext context) throws IOException {
throw new UnsupportedOperationException(UNSUPPORTED_MSG);
}

View File

@ -27,7 +27,7 @@ import org.junit.Test;
public class MultiCollectorTest extends LuceneTestCase {
private static class DummyCollector extends Collector {
private static class DummyCollector extends SimpleCollector {
boolean acceptsDocsOutOfOrderCalled = false;
boolean collectCalled = false;
@ -46,7 +46,7 @@ public class MultiCollectorTest extends LuceneTestCase {
}
@Override
public void setNextReader(AtomicReaderContext context) throws IOException {
protected void doSetNextReader(AtomicReaderContext context) throws IOException {
setNextReaderCalled = true;
}
@ -71,10 +71,11 @@ public class MultiCollectorTest extends LuceneTestCase {
// doesn't, an NPE would be thrown.
Collector c = MultiCollector.wrap(new DummyCollector(), null, new DummyCollector());
assertTrue(c instanceof MultiCollector);
assertTrue(c.acceptsDocsOutOfOrder());
c.collect(1);
c.setNextReader(null);
c.setScorer(null);
final LeafCollector ac = c.getLeafCollector(null);
assertTrue(ac.acceptsDocsOutOfOrder());
ac.collect(1);
c.getLeafCollector(null);
c.getLeafCollector(null).setScorer(null);
}
@Test
@ -93,10 +94,11 @@ public class MultiCollectorTest extends LuceneTestCase {
// doesn't, an NPE would be thrown.
DummyCollector[] dcs = new DummyCollector[] { new DummyCollector(), new DummyCollector() };
Collector c = MultiCollector.wrap(dcs);
assertTrue(c.acceptsDocsOutOfOrder());
c.collect(1);
c.setNextReader(null);
c.setScorer(null);
LeafCollector ac = c.getLeafCollector(null);
assertTrue(ac.acceptsDocsOutOfOrder());
ac.collect(1);
ac = c.getLeafCollector(null);
ac.setScorer(null);
for (DummyCollector dc : dcs) {
assertTrue(dc.acceptsDocsOutOfOrderCalled);

View File

@ -187,10 +187,7 @@ public class TestBooleanOr extends LuceneTestCase {
final FixedBitSet hits = new FixedBitSet(docCount);
final AtomicInteger end = new AtomicInteger();
Collector c = new Collector() {
@Override
public void setNextReader(AtomicReaderContext sub) {
}
LeafCollector c = new SimpleCollector() {
@Override
public void collect(int doc) {
@ -198,10 +195,6 @@ public class TestBooleanOr extends LuceneTestCase {
hits.set(doc);
}
@Override
public void setScorer(Scorer scorer) {
}
@Override
public boolean acceptsDocsOutOfOrder() {
return true;

View File

@ -121,46 +121,45 @@ public class TestBooleanQueryVisitSubscorers extends LuceneTestCase {
return collector.docCounts;
}
static class MyCollector extends Collector {
private TopDocsCollector<ScoreDoc> collector;
private int docBase;
static class MyCollector extends FilterCollector {
public final Map<Integer,Integer> docCounts = new HashMap<>();
private final Set<Scorer> tqsSet = new HashSet<>();
MyCollector() {
collector = TopScoreDocCollector.create(10, true);
super(TopScoreDocCollector.create(10, true));
}
@Override
public boolean acceptsDocsOutOfOrder() {
return false;
}
@Override
public void collect(int doc) throws IOException {
int freq = 0;
for(Scorer scorer : tqsSet) {
if (doc == scorer.docID()) {
freq += scorer.freq();
public LeafCollector getLeafCollector(AtomicReaderContext context)
throws IOException {
final int docBase = context.docBase;
return new FilterLeafCollector(super.getLeafCollector(context)) {
@Override
public boolean acceptsDocsOutOfOrder() {
return false;
}
}
docCounts.put(doc + docBase, freq);
collector.collect(doc);
}
@Override
public void setNextReader(AtomicReaderContext context) throws IOException {
this.docBase = context.docBase;
collector.setNextReader(context);
}
@Override
public void setScorer(Scorer scorer) throws IOException {
collector.setScorer(scorer);
tqsSet.clear();
fillLeaves(scorer, tqsSet);
@Override
public void setScorer(Scorer scorer) throws IOException {
super.setScorer(scorer);
tqsSet.clear();
fillLeaves(scorer, tqsSet);
}
@Override
public void collect(int doc) throws IOException {
int freq = 0;
for(Scorer scorer : tqsSet) {
if (doc == scorer.docID()) {
freq += scorer.freq();
}
}
docCounts.put(doc + docBase, freq);
super.collect(doc);
}
};
}
private void fillLeaves(Scorer scorer, Set<Scorer> set) {
@ -174,11 +173,12 @@ public class TestBooleanQueryVisitSubscorers extends LuceneTestCase {
}
public TopDocs topDocs(){
return collector.topDocs();
return ((TopDocsCollector<?>) in).topDocs();
}
public int freq(int doc) throws IOException {
return docCounts.get(doc);
}
}
}

View File

@ -84,7 +84,7 @@ public class TestBooleanScorer extends LuceneTestCase {
private int doc = -1;
@Override
public boolean score(Collector c, int maxDoc) throws IOException {
public boolean score(LeafCollector c, int maxDoc) throws IOException {
assert doc == -1;
doc = 3000;
FakeScorer fs = new FakeScorer();
@ -99,7 +99,7 @@ public class TestBooleanScorer extends LuceneTestCase {
BooleanScorer bs = new BooleanScorer(weight, false, 1, Arrays.asList(scorers), Collections.<BulkScorer>emptyList(), scorers.length);
final List<Integer> hits = new ArrayList<>();
bs.score(new Collector() {
bs.score(new SimpleCollector() {
int docBase;
@Override
public void setScorer(Scorer scorer) {
@ -111,7 +111,7 @@ public class TestBooleanScorer extends LuceneTestCase {
}
@Override
public void setNextReader(AtomicReaderContext context) {
protected void doSetNextReader(AtomicReaderContext context) throws IOException {
docBase = context.docBase;
}
@ -138,7 +138,8 @@ public class TestBooleanScorer extends LuceneTestCase {
w.addDocument(doc);
final IndexReader r = w.getReader();
w.close();
final IndexSearcher s = newSearcher(r);
// we don't wrap with AssertingIndexSearcher in order to have the original scorer in setScorer.
final IndexSearcher s = newSearcher(r, true, false);
final BooleanQuery q = new BooleanQuery();
for(int term=0;term<33;term++) {
@ -149,12 +150,12 @@ public class TestBooleanScorer extends LuceneTestCase {
BooleanClause.Occur.SHOULD));
final int[] count = new int[1];
s.search(q, new Collector() {
s.search(q, new SimpleCollector() {
@Override
public void setScorer(Scorer scorer) {
// Make sure we got BooleanScorer:
final Class<?> clazz = scorer instanceof AssertingScorer ? ((AssertingScorer) scorer).getIn().getClass() : scorer.getClass();
final Class<?> clazz = scorer.getClass();
assertEquals("Scorer is implemented by wrong class", FakeScorer.class.getName(), clazz.getName());
}
@ -163,10 +164,6 @@ public class TestBooleanScorer extends LuceneTestCase {
count[0]++;
}
@Override
public void setNextReader(AtomicReaderContext context) {
}
@Override
public boolean acceptsDocsOutOfOrder() {
return true;
@ -219,7 +216,7 @@ public class TestBooleanScorer extends LuceneTestCase {
return new BulkScorer() {
@Override
public boolean score(Collector collector, int max) throws IOException {
public boolean score(LeafCollector collector, int max) throws IOException {
collector.setScorer(new FakeScorer());
collector.collect(0);
return false;

View File

@ -17,11 +17,10 @@ package org.apache.lucene.search;
* limitations under the License.
*/
import org.apache.lucene.index.AtomicReaderContext;
import org.apache.lucene.util.LuceneTestCase;
import java.io.IOException;
import org.apache.lucene.util.LuceneTestCase;
public class TestCachingCollector extends LuceneTestCase {
private static final double ONE_BYTE = 1.0 / (1024 * 1024); // 1 byte out of MB
@ -53,23 +52,17 @@ public class TestCachingCollector extends LuceneTestCase {
}
}
private static class NoOpCollector extends Collector {
private static class NoOpCollector extends SimpleCollector {
private final boolean acceptDocsOutOfOrder;
public NoOpCollector(boolean acceptDocsOutOfOrder) {
this.acceptDocsOutOfOrder = acceptDocsOutOfOrder;
}
@Override
public void setScorer(Scorer scorer) throws IOException {}
@Override
public void collect(int doc) throws IOException {}
@Override
public void setNextReader(AtomicReaderContext context) throws IOException {}
@Override
public boolean acceptsDocsOutOfOrder() {
return acceptDocsOutOfOrder;
@ -80,23 +73,18 @@ public class TestCachingCollector extends LuceneTestCase {
public void testBasic() throws Exception {
for (boolean cacheScores : new boolean[] { false, true }) {
CachingCollector cc = CachingCollector.create(new NoOpCollector(false), cacheScores, 1.0);
cc.setScorer(new MockScorer());
LeafCollector acc = cc.getLeafCollector(null);
acc.setScorer(new MockScorer());
// collect 1000 docs
for (int i = 0; i < 1000; i++) {
cc.collect(i);
acc.collect(i);
}
// now replay them
cc.replay(new Collector() {
cc.replay(new SimpleCollector() {
int prevDocID = -1;
@Override
public void setScorer(Scorer scorer) {}
@Override
public void setNextReader(AtomicReaderContext context) {}
@Override
public void collect(int doc) {
assertEquals(prevDocID + 1, doc);
@ -113,11 +101,12 @@ public class TestCachingCollector extends LuceneTestCase {
public void testIllegalStateOnReplay() throws Exception {
CachingCollector cc = CachingCollector.create(new NoOpCollector(false), true, 50 * ONE_BYTE);
cc.setScorer(new MockScorer());
LeafCollector acc = cc.getLeafCollector(null);
acc.setScorer(new MockScorer());
// collect 130 docs, this should be enough for triggering cache abort.
for (int i = 0; i < 130; i++) {
cc.collect(i);
acc.collect(i);
}
assertFalse("CachingCollector should not be cached due to low memory limit", cc.isCached());
@ -135,16 +124,18 @@ public class TestCachingCollector extends LuceneTestCase {
// is valid with the Collector passed to the ctor
// 'src' Collector does not support out-of-order
CachingCollector cc = CachingCollector.create(new NoOpCollector(false), true, 50 * ONE_BYTE);
cc.setScorer(new MockScorer());
for (int i = 0; i < 10; i++) cc.collect(i);
CachingCollector cc = CachingCollector.create(new NoOpCollector(false), true, 100 * ONE_BYTE);
LeafCollector acc = cc.getLeafCollector(null);
acc.setScorer(new MockScorer());
for (int i = 0; i < 10; i++) acc.collect(i);
cc.replay(new NoOpCollector(true)); // this call should not fail
cc.replay(new NoOpCollector(false)); // this call should not fail
// 'src' Collector supports out-of-order
cc = CachingCollector.create(new NoOpCollector(true), true, 50 * ONE_BYTE);
cc.setScorer(new MockScorer());
for (int i = 0; i < 10; i++) cc.collect(i);
cc = CachingCollector.create(new NoOpCollector(true), true, 100 * ONE_BYTE);
acc = cc.getLeafCollector(null);
acc.setScorer(new MockScorer());
for (int i = 0; i < 10; i++) acc.collect(i);
cc.replay(new NoOpCollector(true)); // this call should not fail
try {
cc.replay(new NoOpCollector(false)); // this call should fail
@ -165,12 +156,13 @@ public class TestCachingCollector extends LuceneTestCase {
int bytesPerDoc = cacheScores ? 8 : 4;
CachingCollector cc = CachingCollector.create(new NoOpCollector(false),
cacheScores, bytesPerDoc * ONE_BYTE * numDocs);
cc.setScorer(new MockScorer());
for (int i = 0; i < numDocs; i++) cc.collect(i);
LeafCollector acc = cc.getLeafCollector(null);
acc.setScorer(new MockScorer());
for (int i = 0; i < numDocs; i++) acc.collect(i);
assertTrue(cc.isCached());
// The 151's document should terminate caching
cc.collect(numDocs);
acc.collect(numDocs);
assertFalse(cc.isCached());
}
}
@ -179,9 +171,9 @@ public class TestCachingCollector extends LuceneTestCase {
for (boolean cacheScores : new boolean[] { false, true }) {
// create w/ null wrapped collector, and test that the methods work
CachingCollector cc = CachingCollector.create(true, cacheScores, 50 * ONE_BYTE);
cc.setNextReader(null);
cc.setScorer(new MockScorer());
cc.collect(0);
LeafCollector acc = cc.getLeafCollector(null);
acc.setScorer(new MockScorer());
acc.collect(0);
assertTrue(cc.isCached());
cc.replay(new NoOpCollector(true));

View File

@ -50,7 +50,7 @@ public class TestConstantScoreQuery extends LuceneTestCase {
private void checkHits(IndexSearcher searcher, Query q, final float expectedScore, final String scorerClassName, final String innerScorerClassName) throws IOException {
final int[] count = new int[1];
searcher.search(q, new Collector() {
searcher.search(q, new SimpleCollector() {
private Scorer scorer;
@Override
@ -69,10 +69,6 @@ public class TestConstantScoreQuery extends LuceneTestCase {
count[0]++;
}
@Override
public void setNextReader(AtomicReaderContext context) {
}
@Override
public boolean acceptsDocsOutOfOrder() {
return true;
@ -95,7 +91,8 @@ public class TestConstantScoreQuery extends LuceneTestCase {
reader = writer.getReader();
writer.close();
searcher = newSearcher(reader);
// we don't wrap with AssertingIndexSearcher in order to have the original scorer in setScorer.
searcher = newSearcher(reader, true, false);
// set a similarity that does not normalize our boost away
searcher.setSimilarity(new DefaultSimilarity() {

View File

@ -59,7 +59,7 @@ public class TestDocBoost extends LuceneTestCase {
IndexSearcher searcher = newSearcher(reader);
searcher.search
(new TermQuery(new Term("field", "word")),
new Collector() {
new SimpleCollector() {
private int base = 0;
private Scorer scorer;
@Override
@ -71,7 +71,7 @@ public class TestDocBoost extends LuceneTestCase {
scores[doc + base] = scorer.score();
}
@Override
public void setNextReader(AtomicReaderContext context) {
protected void doSetNextReader(AtomicReaderContext context) throws IOException {
base = context.docBase;
}
@Override

View File

@ -58,14 +58,11 @@ public class TestEarlyTermination extends LuceneTestCase {
for (int i = 0; i < iters; ++i) {
final IndexSearcher searcher = newSearcher(reader);
final Collector collector = new Collector() {
final Collector collector = new SimpleCollector() {
final boolean outOfOrder = random().nextBoolean();
boolean collectionTerminated = true;
@Override
public void setScorer(Scorer scorer) throws IOException {}
@Override
public void collect(int doc) throws IOException {
assertFalse(collectionTerminated);
@ -76,7 +73,7 @@ public class TestEarlyTermination extends LuceneTestCase {
}
@Override
public void setNextReader(AtomicReaderContext context) throws IOException {
protected void doSetNextReader(AtomicReaderContext context) throws IOException {
if (random().nextBoolean()) {
collectionTerminated = true;
throw new CollectionTerminatedException();

View File

@ -226,7 +226,7 @@ public class TestMultiTermConstantScore extends BaseTestRangeFilter {
search.setSimilarity(new DefaultSimilarity());
Query q = csrq("data", "1", "6", T, T);
q.setBoost(100);
search.search(q, null, new Collector() {
search.search(q, null, new SimpleCollector() {
private int base = 0;
private Scorer scorer;
@Override
@ -238,7 +238,7 @@ public class TestMultiTermConstantScore extends BaseTestRangeFilter {
assertEquals("score for doc " + (doc + base) + " was not correct", 1.0f, scorer.score(), SCORE_COMP_THRESH);
}
@Override
public void setNextReader(AtomicReaderContext context) {
protected void doSetNextReader(AtomicReaderContext context) throws IOException {
base = context.docBase;
}
@Override

View File

@ -22,6 +22,7 @@ import org.apache.lucene.index.RandomIndexWriter;
import org.apache.lucene.index.Term;
import org.apache.lucene.store.Directory;
import org.apache.lucene.util.LuceneTestCase;
import org.apache.lucene.document.Document;
public class TestPositiveScoresOnlyCollector extends LuceneTestCase {
@ -78,6 +79,7 @@ public class TestPositiveScoresOnlyCollector extends LuceneTestCase {
Directory directory = newDirectory();
RandomIndexWriter writer = new RandomIndexWriter(random(), directory);
writer.addDocument(new Document());
writer.commit();
IndexReader ir = writer.getReader();
writer.close();
@ -86,9 +88,10 @@ public class TestPositiveScoresOnlyCollector extends LuceneTestCase {
Scorer s = new SimpleScorer(fake);
TopDocsCollector<ScoreDoc> tdc = TopScoreDocCollector.create(scores.length, true);
Collector c = new PositiveScoresOnlyCollector(tdc);
c.setScorer(s);
LeafCollector ac = c.getLeafCollector(ir.leaves().get(0));
ac.setScorer(s);
while (s.nextDoc() != DocIdSetIterator.NO_MORE_DOCS) {
c.collect(0);
ac.collect(0);
}
TopDocs td = tdc.topDocs();
ScoreDoc[] sd = td.scoreDocs;

View File

@ -65,7 +65,7 @@ public class TestScoreCachingWrappingScorer extends LuceneTestCase {
}
}
private static final class ScoreCachingCollector extends Collector {
private static final class ScoreCachingCollector extends SimpleCollector {
private int idx = 0;
private Scorer scorer;
@ -88,9 +88,6 @@ public class TestScoreCachingWrappingScorer extends LuceneTestCase {
++idx;
}
@Override public void setNextReader(AtomicReaderContext context) {
}
@Override public void setScorer(Scorer scorer) {
this.scorer = new ScoreCachingWrappingScorer(scorer);
}

View File

@ -97,13 +97,10 @@ public class TestScorerPerf extends LuceneTestCase {
return sets;
}
public static class CountingHitCollector extends Collector {
public static class CountingHitCollector extends SimpleCollector {
int count=0;
int sum=0;
protected int docBase = 0;
@Override
public void setScorer(Scorer scorer) throws IOException {}
@Override
public void collect(int doc) {
@ -115,7 +112,7 @@ public class TestScorerPerf extends LuceneTestCase {
public int getSum() { return sum; }
@Override
public void setNextReader(AtomicReaderContext context) {
protected void doSetNextReader(AtomicReaderContext context) throws IOException {
docBase = context.docBase;
}
@Override

View File

@ -76,7 +76,7 @@ public class TestSimilarity extends LuceneTestCase {
Term b = new Term("field", "b");
Term c = new Term("field", "c");
searcher.search(new TermQuery(b), new Collector() {
searcher.search(new TermQuery(b), new SimpleCollector() {
private Scorer scorer;
@Override
public void setScorer(Scorer scorer) {
@ -86,9 +86,6 @@ public class TestSimilarity extends LuceneTestCase {
public final void collect(int doc) throws IOException {
assertEquals(1.0f, scorer.score(), 0);
}
@Override
public void setNextReader(AtomicReaderContext context) {}
@Override
public boolean acceptsDocsOutOfOrder() {
return true;
}
@ -98,7 +95,7 @@ public class TestSimilarity extends LuceneTestCase {
bq.add(new TermQuery(a), BooleanClause.Occur.SHOULD);
bq.add(new TermQuery(b), BooleanClause.Occur.SHOULD);
//System.out.println(bq.toString("field"));
searcher.search(bq, new Collector() {
searcher.search(bq, new SimpleCollector() {
private int base = 0;
private Scorer scorer;
@Override
@ -111,7 +108,7 @@ public class TestSimilarity extends LuceneTestCase {
assertEquals((float)doc+base+1, scorer.score(), 0);
}
@Override
public void setNextReader(AtomicReaderContext context) {
protected void doSetNextReader(AtomicReaderContext context) throws IOException {
base = context.docBase;
}
@Override
@ -125,7 +122,7 @@ public class TestSimilarity extends LuceneTestCase {
pq.add(c);
//System.out.println(pq.toString("field"));
searcher.search(pq,
new Collector() {
new SimpleCollector() {
private Scorer scorer;
@Override
public void setScorer(Scorer scorer) {
@ -137,8 +134,6 @@ public class TestSimilarity extends LuceneTestCase {
assertEquals(1.0f, scorer.score(), 0);
}
@Override
public void setNextReader(AtomicReaderContext context) {}
@Override
public boolean acceptsDocsOutOfOrder() {
return true;
}
@ -146,7 +141,7 @@ public class TestSimilarity extends LuceneTestCase {
pq.setSlop(2);
//System.out.println(pq.toString("field"));
searcher.search(pq, new Collector() {
searcher.search(pq, new SimpleCollector() {
private Scorer scorer;
@Override
public void setScorer(Scorer scorer) {
@ -158,8 +153,6 @@ public class TestSimilarity extends LuceneTestCase {
assertEquals(2.0f, scorer.score(), 0);
}
@Override
public void setNextReader(AtomicReaderContext context) {}
@Override
public boolean acceptsDocsOutOfOrder() {
return true;
}

View File

@ -175,7 +175,7 @@ public class TestSloppyPhraseQuery extends LuceneTestCase {
return query;
}
static class MaxFreqCollector extends Collector {
static class MaxFreqCollector extends SimpleCollector {
float max;
int totalHits;
Scorer scorer;
@ -191,10 +191,6 @@ public class TestSloppyPhraseQuery extends LuceneTestCase {
max = Math.max(max, scorer.freq());
}
@Override
public void setNextReader(AtomicReaderContext context) throws IOException {
}
@Override
public boolean acceptsDocsOutOfOrder() {
return false;
@ -203,7 +199,7 @@ public class TestSloppyPhraseQuery extends LuceneTestCase {
/** checks that no scores or freqs are infinite */
private void assertSaneScoring(PhraseQuery pq, IndexSearcher searcher) throws Exception {
searcher.search(pq, new Collector() {
searcher.search(pq, new SimpleCollector() {
Scorer scorer;
@Override
@ -217,11 +213,6 @@ public class TestSloppyPhraseQuery extends LuceneTestCase {
assertFalse(Float.isInfinite(scorer.score()));
}
@Override
public void setNextReader(AtomicReaderContext context) {
// do nothing
}
@Override
public boolean acceptsDocsOutOfOrder() {
return false;

View File

@ -65,10 +65,7 @@ public class TestSubScorerFreqs extends LuceneTestCase {
dir = null;
}
private static class CountingCollector extends Collector {
private final Collector other;
private int docBase;
private static class CountingCollector extends FilterCollector {
public final Map<Integer, Map<Query, Float>> docCounts = new HashMap<>();
private final Map<Query, Scorer> subScorers = new HashMap<>();
@ -79,16 +76,9 @@ public class TestSubScorerFreqs extends LuceneTestCase {
}
public CountingCollector(Collector other, Set<String> relationships) {
this.other = other;
super(other);
this.relationships = relationships;
}
@Override
public void setScorer(Scorer scorer) throws IOException {
other.setScorer(scorer);
subScorers.clear();
setSubScorers(scorer, "TOP");
}
public void setSubScorers(Scorer scorer, String relationship) {
for (ChildScorer child : scorer.getChildren()) {
@ -98,30 +88,34 @@ public class TestSubScorerFreqs extends LuceneTestCase {
}
subScorers.put(scorer.getWeight().getQuery(), scorer);
}
@Override
public void collect(int doc) throws IOException {
final Map<Query, Float> freqs = new HashMap<>();
for (Map.Entry<Query, Scorer> ent : subScorers.entrySet()) {
Scorer value = ent.getValue();
int matchId = value.docID();
freqs.put(ent.getKey(), matchId == doc ? value.freq() : 0.0f);
}
docCounts.put(doc + docBase, freqs);
other.collect(doc);
}
@Override
public void setNextReader(AtomicReaderContext context)
public LeafCollector getLeafCollector(AtomicReaderContext context)
throws IOException {
docBase = context.docBase;
other.setNextReader(context);
final int docBase = context.docBase;
return new FilterLeafCollector(super.getLeafCollector(context)) {
@Override
public void collect(int doc) throws IOException {
final Map<Query, Float> freqs = new HashMap<Query, Float>();
for (Map.Entry<Query, Scorer> ent : subScorers.entrySet()) {
Scorer value = ent.getValue();
int matchId = value.docID();
freqs.put(ent.getKey(), matchId == doc ? value.freq() : 0.0f);
}
docCounts.put(doc + docBase, freqs);
super.collect(doc);
}
@Override
public void setScorer(Scorer scorer) throws IOException {
super.setScorer(scorer);
subScorers.clear();
setSubScorers(scorer, "TOP");
}
};
}
@Override
public boolean acceptsDocsOutOfOrder() {
return other.acceptsDocsOutOfOrder();
}
}
private static final float FLOAT_TOLERANCE = 0.00001F;

View File

@ -84,7 +84,7 @@ public class TestTermScorer extends LuceneTestCase {
final List<TestHit> docs = new ArrayList<>();
// must call next first
ts.score(new Collector() {
ts.score(new SimpleCollector() {
private int base = 0;
private Scorer scorer;
@ -104,7 +104,7 @@ public class TestTermScorer extends LuceneTestCase {
}
@Override
public void setNextReader(AtomicReaderContext context) {
protected void doSetNextReader(AtomicReaderContext context) throws IOException {
base = context.docBase;
}

View File

@ -307,7 +307,7 @@ public class TestTimeLimitingCollector extends LuceneTestCase {
}
// counting collector that can slow down at collect().
private class MyHitCollector extends Collector {
private class MyHitCollector extends SimpleCollector {
private final BitSet bits = new BitSet();
private int slowdown = 0;
private int lastDocCollected = -1;
@ -349,7 +349,7 @@ public class TestTimeLimitingCollector extends LuceneTestCase {
}
@Override
public void setNextReader(AtomicReaderContext context) {
protected void doSetNextReader(AtomicReaderContext context) throws IOException {
docBase = context.docBase;
}

View File

@ -61,7 +61,7 @@ public class TestTopDocsCollector extends LuceneTestCase {
}
@Override
public void setNextReader(AtomicReaderContext context) {
protected void doSetNextReader(AtomicReaderContext context) throws IOException {
base = context.docBase;
}

View File

@ -22,6 +22,7 @@ import java.io.FileNotFoundException;
import java.io.IOException;
import java.nio.file.NoSuchFileException;
import java.util.Arrays;
import java.util.Collections;
import org.apache.lucene.store.MockDirectoryWrapper.Throttling;
import org.apache.lucene.util.LuceneTestCase;
@ -288,5 +289,34 @@ public class TestDirectory extends LuceneTestCase {
TestUtil.rm(path);
}
}
public void testFsyncDoesntCreateNewFiles() throws Exception {
File path = TestUtil.getTempDir("nocreate");
Directory fsdir = new SimpleFSDirectory(path);
// write a file
IndexOutput out = fsdir.createOutput("afile", newIOContext(random()));
out.writeString("boo");
out.close();
// delete it
assertTrue(new File(path, "afile").delete());
// directory is empty
assertEquals(0, fsdir.listAll().length);
// fsync it
try {
fsdir.sync(Collections.singleton("afile"));
fail("didn't get expected exception, instead fsync created new files: " + Arrays.asList(fsdir.listAll()));
} catch (FileNotFoundException | NoSuchFileException expected) {
// ok
}
// directory is still empty
assertEquals(0, fsdir.listAll().length);
fsdir.close();
}
}

View File

@ -22,15 +22,14 @@ import java.util.Collections;
import org.apache.lucene.document.Document;
import org.apache.lucene.index.IndexWriter;
import org.apache.lucene.index.IndexWriterConfig;
import org.apache.lucene.store.Directory;
import org.apache.lucene.store.IOContext;
import org.apache.lucene.store.IndexOutput;
import org.apache.lucene.store.MockDirectoryWrapper;
import org.junit.Assert;
import org.junit.Test;
import org.junit.runner.JUnitCore;
import org.junit.runner.Result;
import org.junit.runner.notification.Failure;
import com.carrotsearch.randomizedtesting.RandomizedTest;
// LUCENE-4456: Test that we fail if there are unreferenced files
@ -41,7 +40,8 @@ public class TestFailIfUnreferencedFiles extends WithNestedTests {
public static class Nested1 extends WithNestedTests.AbstractNestedTest {
public void testDummy() throws Exception {
Directory dir = newMockDirectory();
MockDirectoryWrapper dir = newMockDirectory();
dir.setAssertNoUnrefencedFilesOnClose(true);
IndexWriter iw = new IndexWriter(dir, new IndexWriterConfig(TEST_VERSION_CURRENT, null));
iw.addDocument(new Document());
iw.close();

View File

@ -26,6 +26,7 @@ import org.apache.lucene.facet.sortedset.SortedSetDocValuesFacetField;
import org.apache.lucene.facet.sortedset.SortedSetDocValuesReaderState;
import org.apache.lucene.facet.taxonomy.FastTaxonomyFacetCounts;
import org.apache.lucene.facet.taxonomy.TaxonomyReader;
import org.apache.lucene.search.LeafCollector;
import org.apache.lucene.search.BooleanClause;
import org.apache.lucene.search.BooleanQuery;
import org.apache.lucene.search.Collector;
@ -238,7 +239,7 @@ public class DrillSideways {
* default is false. Note that if you return true from
* this method (in a subclass) be sure your collector
* also returns false from {@link
* Collector#acceptsDocsOutOfOrder}: this will trick
* LeafCollector#acceptsDocsOutOfOrder}: this will trick
* {@code BooleanQuery} into also scoring all subDocs at
* once. */
protected boolean scoreSubDocsAtOnce() {

View File

@ -23,6 +23,7 @@ import java.util.Collections;
import org.apache.lucene.index.AtomicReaderContext;
import org.apache.lucene.index.DocsEnum;
import org.apache.lucene.search.LeafCollector;
import org.apache.lucene.search.Collector;
import org.apache.lucene.search.DocIdSetIterator;
import org.apache.lucene.search.Scorer;
@ -36,6 +37,7 @@ class DrillSidewaysScorer extends BulkScorer {
//private static boolean DEBUG = false;
private final Collector drillDownCollector;
private LeafCollector drillDownLeafCollector;
private final DocsAndCost[] dims;
@ -62,7 +64,7 @@ class DrillSidewaysScorer extends BulkScorer {
}
@Override
public boolean score(Collector collector, int maxDoc) throws IOException {
public boolean score(LeafCollector collector, int maxDoc) throws IOException {
if (maxDoc != Integer.MAX_VALUE) {
throw new IllegalArgumentException("maxDoc must be Integer.MAX_VALUE");
}
@ -73,12 +75,14 @@ class DrillSidewaysScorer extends BulkScorer {
FakeScorer scorer = new FakeScorer();
collector.setScorer(scorer);
if (drillDownCollector != null) {
drillDownCollector.setScorer(scorer);
drillDownCollector.setNextReader(context);
drillDownLeafCollector = drillDownCollector.getLeafCollector(context);
drillDownLeafCollector.setScorer(scorer);
} else {
drillDownLeafCollector = null;
}
for (DocsAndCost dim : dims) {
dim.sidewaysCollector.setScorer(scorer);
dim.sidewaysCollector.setNextReader(context);
dim.sidewaysLeafCollector = dim.sidewaysCollector.getLeafCollector(context);
dim.sidewaysLeafCollector.setScorer(scorer);
}
// TODO: if we ever allow null baseScorer ... it will
@ -100,10 +104,10 @@ class DrillSidewaysScorer extends BulkScorer {
final int numDims = dims.length;
Bits[] bits = new Bits[numBits];
Collector[] bitsSidewaysCollectors = new Collector[numBits];
LeafCollector[] bitsSidewaysCollectors = new LeafCollector[numBits];
DocIdSetIterator[] disis = new DocIdSetIterator[numDims-numBits];
Collector[] sidewaysCollectors = new Collector[numDims-numBits];
LeafCollector[] sidewaysCollectors = new LeafCollector[numDims-numBits];
long drillDownCost = 0;
int disiUpto = 0;
int bitsUpto = 0;
@ -111,14 +115,14 @@ class DrillSidewaysScorer extends BulkScorer {
DocIdSetIterator disi = dims[dim].disi;
if (dims[dim].bits == null) {
disis[disiUpto] = disi;
sidewaysCollectors[disiUpto] = dims[dim].sidewaysCollector;
sidewaysCollectors[disiUpto] = dims[dim].sidewaysLeafCollector;
disiUpto++;
if (disi != null) {
drillDownCost += disi.cost();
}
} else {
bits[bitsUpto] = dims[dim].bits;
bitsSidewaysCollectors[bitsUpto] = dims[dim].sidewaysCollector;
bitsSidewaysCollectors[bitsUpto] = dims[dim].sidewaysLeafCollector;
bitsUpto++;
}
}
@ -154,15 +158,15 @@ class DrillSidewaysScorer extends BulkScorer {
* (i.e., like BooleanScorer2, not BooleanScorer). In
* this case we just .next() on base and .advance() on
* the dim filters. */
private void doQueryFirstScoring(Collector collector, DocIdSetIterator[] disis, Collector[] sidewaysCollectors,
Bits[] bits, Collector[] bitsSidewaysCollectors) throws IOException {
private void doQueryFirstScoring(LeafCollector collector, DocIdSetIterator[] disis, LeafCollector[] sidewaysCollectors,
Bits[] bits, LeafCollector[] bitsSidewaysCollectors) throws IOException {
//if (DEBUG) {
// System.out.println(" doQueryFirstScoring");
//}
int docID = baseScorer.docID();
nextDoc: while (docID != DocsEnum.NO_MORE_DOCS) {
Collector failedCollector = null;
LeafCollector failedCollector = null;
for (int i=0;i<disis.length;i++) {
// TODO: should we sort this 2nd dimension of
// docsEnums from most frequent to least?
@ -225,7 +229,7 @@ class DrillSidewaysScorer extends BulkScorer {
/** Used when drill downs are highly constraining vs
* baseQuery. */
private void doDrillDownAdvanceScoring(Collector collector, DocIdSetIterator[] disis, Collector[] sidewaysCollectors) throws IOException {
private void doDrillDownAdvanceScoring(LeafCollector collector, DocIdSetIterator[] disis, LeafCollector[] sidewaysCollectors) throws IOException {
final int maxDoc = context.reader().maxDoc();
final int numDims = dims.length;
@ -423,7 +427,7 @@ class DrillSidewaysScorer extends BulkScorer {
}
}
private void doUnionScoring(Collector collector, DocIdSetIterator[] disis, Collector[] sidewaysCollectors) throws IOException {
private void doUnionScoring(LeafCollector collector, DocIdSetIterator[] disis, LeafCollector[] sidewaysCollectors) throws IOException {
//if (DEBUG) {
// System.out.println(" doUnionScoring");
//}
@ -569,14 +573,14 @@ class DrillSidewaysScorer extends BulkScorer {
}
}
private void collectHit(Collector collector, Collector[] sidewaysCollectors) throws IOException {
private void collectHit(LeafCollector collector, LeafCollector[] sidewaysCollectors) throws IOException {
//if (DEBUG) {
// System.out.println(" hit");
//}
collector.collect(collectDocID);
if (drillDownCollector != null) {
drillDownCollector.collect(collectDocID);
drillDownLeafCollector.collect(collectDocID);
}
// TODO: we could "fix" faceting of the sideways counts
@ -589,14 +593,14 @@ class DrillSidewaysScorer extends BulkScorer {
}
}
private void collectHit(Collector collector, Collector[] sidewaysCollectors, Collector[] sidewaysCollectors2) throws IOException {
private void collectHit(LeafCollector collector, LeafCollector[] sidewaysCollectors, LeafCollector[] sidewaysCollectors2) throws IOException {
//if (DEBUG) {
// System.out.println(" hit");
//}
collector.collect(collectDocID);
if (drillDownCollector != null) {
drillDownCollector.collect(collectDocID);
drillDownLeafCollector.collect(collectDocID);
}
// TODO: we could "fix" faceting of the sideways counts
@ -612,7 +616,7 @@ class DrillSidewaysScorer extends BulkScorer {
}
}
private void collectNearMiss(Collector sidewaysCollector) throws IOException {
private void collectNearMiss(LeafCollector sidewaysCollector) throws IOException {
//if (DEBUG) {
// System.out.println(" missingDim=" + dim);
//}
@ -620,8 +624,6 @@ class DrillSidewaysScorer extends BulkScorer {
}
private final class FakeScorer extends Scorer {
float score;
int doc;
public FakeScorer() {
super(null);
@ -674,6 +676,7 @@ class DrillSidewaysScorer extends BulkScorer {
// Random access bits:
Bits bits;
Collector sidewaysCollector;
LeafCollector sidewaysLeafCollector;
String dim;
@Override

View File

@ -32,6 +32,7 @@ import org.apache.lucene.search.MultiCollector;
import org.apache.lucene.search.Query;
import org.apache.lucene.search.ScoreDoc;
import org.apache.lucene.search.Scorer;
import org.apache.lucene.search.SimpleCollector;
import org.apache.lucene.search.Sort;
import org.apache.lucene.search.TopDocs;
import org.apache.lucene.search.TopDocsCollector;
@ -47,7 +48,7 @@ import org.apache.lucene.util.FixedBitSet;
* counting. Use the {@code search} utility methods to
* perform an "ordinary" search but also collect into a
* {@link Collector}. */
public class FacetsCollector extends Collector {
public class FacetsCollector extends SimpleCollector {
private AtomicReaderContext context;
private Scorer scorer;
@ -151,7 +152,7 @@ public class FacetsCollector extends Collector {
return matchingDocs;
}
@Override
public final boolean acceptsDocsOutOfOrder() {
// If we are keeping scores then we require in-order
@ -180,7 +181,7 @@ public class FacetsCollector extends Collector {
}
@Override
public final void setNextReader(AtomicReaderContext context) throws IOException {
protected void doSetNextReader(AtomicReaderContext context) throws IOException {
if (docs != null) {
matchingDocs.add(new MatchingDocs(this.context, docs.getDocIdSet(), totalHits, scores));
}

View File

@ -20,14 +20,13 @@ package org.apache.lucene.facet;
import java.util.ArrayList;
import java.util.List;
import org.apache.lucene.index.AtomicReaderContext;
import org.apache.lucene.search.Collector;
import org.apache.lucene.search.Scorer.ChildScorer;
import org.apache.lucene.search.Scorer;
import org.apache.lucene.search.Scorer.ChildScorer;
import org.apache.lucene.search.SimpleCollector;
/** Verifies in collect() that all child subScorers are on
* the collected doc. */
class AssertingSubDocsAtOnceCollector extends Collector {
class AssertingSubDocsAtOnceCollector extends SimpleCollector {
// TODO: allow wrapping another Collector
@ -56,10 +55,6 @@ class AssertingSubDocsAtOnceCollector extends Collector {
}
}
@Override
public void setNextReader(AtomicReaderContext context) {
}
@Override
public boolean acceptsDocsOutOfOrder() {
return false;

View File

@ -43,6 +43,7 @@ import org.apache.lucene.index.IndexReader;
import org.apache.lucene.index.IndexWriterConfig;
import org.apache.lucene.index.RandomIndexWriter;
import org.apache.lucene.index.Term;
import org.apache.lucene.search.LeafCollector;
import org.apache.lucene.search.Collector;
import org.apache.lucene.search.DocIdSet;
import org.apache.lucene.search.Filter;
@ -51,6 +52,7 @@ import org.apache.lucene.search.MatchAllDocsQuery;
import org.apache.lucene.search.Query;
import org.apache.lucene.search.ScoreDoc;
import org.apache.lucene.search.Scorer;
import org.apache.lucene.search.SimpleCollector;
import org.apache.lucene.search.Sort;
import org.apache.lucene.search.SortField;
import org.apache.lucene.search.TermQuery;
@ -666,13 +668,9 @@ public class TestDrillSideways extends FacetTestCase {
// had an AssertingScorer it could catch it when
// Weight.scoresDocsOutOfOrder lies!:
new DrillSideways(s, config, tr).search(ddq,
new Collector() {
new SimpleCollector() {
int lastDocID;
@Override
public void setScorer(Scorer s) {
}
@Override
public void collect(int doc) {
assert doc > lastDocID;
@ -680,7 +678,7 @@ public class TestDrillSideways extends FacetTestCase {
}
@Override
public void setNextReader(AtomicReaderContext context) {
protected void doSetNextReader(AtomicReaderContext context) throws IOException {
lastDocID = -1;
}

View File

@ -17,20 +17,20 @@ package org.apache.lucene.search.grouping;
* limitations under the License.
*/
import org.apache.lucene.index.IndexReader;
import org.apache.lucene.search.Collector;
import org.apache.lucene.util.FixedBitSet;
import java.io.IOException;
import java.util.Collection;
import org.apache.lucene.index.IndexReader;
import org.apache.lucene.search.SimpleCollector;
import org.apache.lucene.util.FixedBitSet;
/**
* This collector specializes in collecting the most relevant document (group head) for each group that match the query.
*
* @lucene.experimental
*/
@SuppressWarnings({"unchecked","rawtypes"})
public abstract class AbstractAllGroupHeadsCollector<GH extends AbstractAllGroupHeadsCollector.GroupHead> extends Collector {
public abstract class AbstractAllGroupHeadsCollector<GH extends AbstractAllGroupHeadsCollector.GroupHead> extends SimpleCollector {
protected final int[] reversed;
protected final int compIDXEnd;

View File

@ -17,13 +17,13 @@ package org.apache.lucene.search.grouping;
* limitations under the License.
*/
import org.apache.lucene.search.Collector;
import org.apache.lucene.search.Scorer;
import org.apache.lucene.util.BytesRef;
import java.io.IOException;
import java.util.Collection;
import org.apache.lucene.search.Scorer;
import org.apache.lucene.search.SimpleCollector;
import org.apache.lucene.util.BytesRef;
/**
* A collector that collects all groups that match the
* query. Only the group value is collected, and the order
@ -36,7 +36,7 @@ import java.util.Collection;
*
* @lucene.experimental
*/
public abstract class AbstractAllGroupsCollector<GROUP_VALUE_TYPE> extends Collector {
public abstract class AbstractAllGroupsCollector<GROUP_VALUE_TYPE> extends SimpleCollector {
/**
* Returns the total number of groups for the executed search.

View File

@ -17,18 +17,18 @@ package org.apache.lucene.search.grouping;
* limitations under the License.
*/
import org.apache.lucene.search.Collector;
import org.apache.lucene.search.Scorer;
import java.util.HashSet;
import java.util.List;
import java.util.Set;
import java.io.IOException;
import java.util.*;
import org.apache.lucene.search.SimpleCollector;
/**
* A second pass grouping collector that keeps track of distinct values for a specified field for the top N group.
*
* @lucene.experimental
*/
public abstract class AbstractDistinctValuesCollector<GC extends AbstractDistinctValuesCollector.GroupCount<?>> extends Collector {
public abstract class AbstractDistinctValuesCollector<GC extends AbstractDistinctValuesCollector.GroupCount<?>> extends SimpleCollector {
/**
* Returns all unique values for each top N group.
@ -42,10 +42,6 @@ public abstract class AbstractDistinctValuesCollector<GC extends AbstractDistinc
return true;
}
@Override
public void setScorer(Scorer scorer) throws IOException {
}
/**
* Returned by {@link AbstractDistinctValuesCollector#getGroups()},
* representing the value and set of distinct values for the group.

View File

@ -33,7 +33,7 @@ import java.util.*;
*
* @lucene.experimental
*/
abstract public class AbstractFirstPassGroupingCollector<GROUP_VALUE_TYPE> extends Collector {
abstract public class AbstractFirstPassGroupingCollector<GROUP_VALUE_TYPE> extends SimpleCollector {
private final Sort groupSort;
private final FieldComparator<?>[] comparators;
@ -326,7 +326,7 @@ abstract public class AbstractFirstPassGroupingCollector<GROUP_VALUE_TYPE> exten
}
@Override
public void setNextReader(AtomicReaderContext readerContext) throws IOException {
protected void doSetNextReader(AtomicReaderContext readerContext) throws IOException {
docBase = readerContext.docBase;
for (int i=0; i<comparators.length; i++) {
comparators[i] = comparators[i].setNextReader(readerContext);

View File

@ -19,6 +19,7 @@ package org.apache.lucene.search.grouping;
import org.apache.lucene.search.Collector;
import org.apache.lucene.search.Scorer;
import org.apache.lucene.search.SimpleCollector;
import org.apache.lucene.util.BytesRef;
import org.apache.lucene.util.PriorityQueue;
@ -30,7 +31,7 @@ import java.util.*;
*
* @lucene.experimental
*/
public abstract class AbstractGroupFacetCollector extends Collector {
public abstract class AbstractGroupFacetCollector extends SimpleCollector {
protected final String groupField;
protected final String facetField;

View File

@ -37,7 +37,7 @@ import java.util.Map;
*
* @lucene.experimental
*/
public abstract class AbstractSecondPassGroupingCollector<GROUP_VALUE_TYPE> extends Collector {
public abstract class AbstractSecondPassGroupingCollector<GROUP_VALUE_TYPE> extends SimpleCollector {
protected final Map<GROUP_VALUE_TYPE, SearchGroupDocs<GROUP_VALUE_TYPE>> groupMap;
private final int maxDocsPerGroup;
@ -107,10 +107,10 @@ public abstract class AbstractSecondPassGroupingCollector<GROUP_VALUE_TYPE> exte
protected abstract SearchGroupDocs<GROUP_VALUE_TYPE> retrieveGroup(int doc) throws IOException;
@Override
public void setNextReader(AtomicReaderContext readerContext) throws IOException {
protected void doSetNextReader(AtomicReaderContext readerContext) throws IOException {
//System.out.println("SP.setNextReader");
for (SearchGroupDocs<GROUP_VALUE_TYPE> group : groupMap.values()) {
group.collector.setNextReader(readerContext);
group.collector.getLeafCollector(readerContext);
}
}

View File

@ -55,7 +55,7 @@ import org.apache.lucene.util.PriorityQueue;
* @lucene.experimental
*/
public class BlockGroupingCollector extends Collector {
public class BlockGroupingCollector extends SimpleCollector {
private int[] pendingSubDocs;
private float[] pendingSubScores;
@ -350,7 +350,7 @@ public class BlockGroupingCollector extends Collector {
}
collector.setScorer(fakeScorer);
collector.setNextReader(og.readerContext);
collector.getLeafCollector(og.readerContext);
for(int docIDX=0;docIDX<og.count;docIDX++) {
final int doc = og.docs[docIDX];
fakeScorer.doc = doc;
@ -516,7 +516,7 @@ public class BlockGroupingCollector extends Collector {
}
@Override
public void setNextReader(AtomicReaderContext readerContext) throws IOException {
protected void doSetNextReader(AtomicReaderContext readerContext) throws IOException {
if (subDocUpto != 0) {
processGroup();
}

View File

@ -20,6 +20,7 @@ package org.apache.lucene.search.grouping.function;
import org.apache.lucene.index.AtomicReaderContext;
import org.apache.lucene.queries.function.FunctionValues;
import org.apache.lucene.queries.function.ValueSource;
import org.apache.lucene.search.LeafCollector;
import org.apache.lucene.search.FieldComparator;
import org.apache.lucene.search.Scorer;
import org.apache.lucene.search.Sort;
@ -101,7 +102,7 @@ public class FunctionAllGroupHeadsCollector extends AbstractAllGroupHeadsCollect
}
@Override
public void setNextReader(AtomicReaderContext context) throws IOException {
protected void doSetNextReader(AtomicReaderContext context) throws IOException {
this.readerContext = context;
FunctionValues values = groupBy.getValues(vsContext, context);
filler = values.getValueFiller();

View File

@ -20,6 +20,7 @@ package org.apache.lucene.search.grouping.function;
import org.apache.lucene.index.AtomicReaderContext;
import org.apache.lucene.queries.function.FunctionValues;
import org.apache.lucene.queries.function.ValueSource;
import org.apache.lucene.search.LeafCollector;
import org.apache.lucene.search.grouping.AbstractAllGroupsCollector;
import org.apache.lucene.util.mutable.MutableValue;
@ -75,7 +76,7 @@ public class FunctionAllGroupsCollector extends AbstractAllGroupsCollector<Mutab
}
@Override
public void setNextReader(AtomicReaderContext context) throws IOException {
protected void doSetNextReader(AtomicReaderContext context) throws IOException {
FunctionValues values = groupBy.getValues(vsContext, context);
filler = values.getValueFiller();
mval = filler.getValue();

View File

@ -20,6 +20,7 @@ package org.apache.lucene.search.grouping.function;
import org.apache.lucene.index.AtomicReaderContext;
import org.apache.lucene.queries.function.FunctionValues;
import org.apache.lucene.queries.function.ValueSource;
import org.apache.lucene.search.LeafCollector;
import org.apache.lucene.search.grouping.AbstractDistinctValuesCollector;
import org.apache.lucene.search.grouping.SearchGroup;
import org.apache.lucene.util.mutable.MutableValue;
@ -70,7 +71,7 @@ public class FunctionDistinctValuesCollector extends AbstractDistinctValuesColle
}
@Override
public void setNextReader(AtomicReaderContext context) throws IOException {
protected void doSetNextReader(AtomicReaderContext context) throws IOException {
FunctionValues values = groupSource.getValues(vsContext, context);
groupFiller = values.getValueFiller();
groupMval = groupFiller.getValue();

View File

@ -20,6 +20,7 @@ package org.apache.lucene.search.grouping.function;
import org.apache.lucene.index.AtomicReaderContext;
import org.apache.lucene.queries.function.FunctionValues;
import org.apache.lucene.queries.function.ValueSource;
import org.apache.lucene.search.LeafCollector;
import org.apache.lucene.search.Sort;
import org.apache.lucene.search.grouping.AbstractFirstPassGroupingCollector;
import org.apache.lucene.util.mutable.MutableValue;
@ -77,8 +78,8 @@ public class FunctionFirstPassGroupingCollector extends AbstractFirstPassGroupin
}
@Override
public void setNextReader(AtomicReaderContext readerContext) throws IOException {
super.setNextReader(readerContext);
protected void doSetNextReader(AtomicReaderContext readerContext) throws IOException {
super.doSetNextReader(readerContext);
FunctionValues values = groupByVS.getValues(vsContext, readerContext);
filler = values.getValueFiller();
mval = filler.getValue();

View File

@ -20,6 +20,7 @@ package org.apache.lucene.search.grouping.function;
import org.apache.lucene.index.AtomicReaderContext;
import org.apache.lucene.queries.function.FunctionValues;
import org.apache.lucene.queries.function.ValueSource;
import org.apache.lucene.search.LeafCollector;
import org.apache.lucene.search.Sort;
import org.apache.lucene.search.grouping.AbstractSecondPassGroupingCollector;
import org.apache.lucene.search.grouping.SearchGroup;
@ -71,8 +72,8 @@ public class FunctionSecondPassGroupingCollector extends AbstractSecondPassGroup
}
@Override
public void setNextReader(AtomicReaderContext readerContext) throws IOException {
super.setNextReader(readerContext);
protected void doSetNextReader(AtomicReaderContext readerContext) throws IOException {
super.doSetNextReader(readerContext);
FunctionValues values = groupByVS.getValues(vsContext, readerContext);
filler = values.getValueFiller();
mval = filler.getValue();

View File

@ -19,6 +19,7 @@ package org.apache.lucene.search.grouping.term;
import org.apache.lucene.index.AtomicReaderContext;
import org.apache.lucene.index.SortedDocValues;
import org.apache.lucene.search.LeafCollector;
import org.apache.lucene.search.FieldCache;
import org.apache.lucene.search.FieldComparator;
import org.apache.lucene.search.Scorer;
@ -158,7 +159,7 @@ public abstract class TermAllGroupHeadsCollector<GH extends AbstractAllGroupHead
}
@Override
public void setNextReader(AtomicReaderContext context) throws IOException {
protected void doSetNextReader(AtomicReaderContext context) throws IOException {
this.readerContext = context;
groupIndex = FieldCache.DEFAULT.getTermsIndex(context.reader(), groupField);
@ -273,7 +274,7 @@ public abstract class TermAllGroupHeadsCollector<GH extends AbstractAllGroupHead
}
@Override
public void setNextReader(AtomicReaderContext context) throws IOException {
protected void doSetNextReader(AtomicReaderContext context) throws IOException {
this.readerContext = context;
groupIndex = FieldCache.DEFAULT.getTermsIndex(context.reader(), groupField);
for (int i = 0; i < fields.length; i++) {
@ -441,7 +442,7 @@ public abstract class TermAllGroupHeadsCollector<GH extends AbstractAllGroupHead
}
@Override
public void setNextReader(AtomicReaderContext context) throws IOException {
protected void doSetNextReader(AtomicReaderContext context) throws IOException {
this.readerContext = context;
groupIndex = FieldCache.DEFAULT.getTermsIndex(context.reader(), groupField);
for (int i = 0; i < fields.length; i++) {
@ -584,7 +585,7 @@ public abstract class TermAllGroupHeadsCollector<GH extends AbstractAllGroupHead
}
@Override
public void setNextReader(AtomicReaderContext context) throws IOException {
protected void doSetNextReader(AtomicReaderContext context) throws IOException {
this.readerContext = context;
groupIndex = FieldCache.DEFAULT.getTermsIndex(context.reader(), groupField);

View File

@ -19,6 +19,7 @@ package org.apache.lucene.search.grouping.term;
import org.apache.lucene.index.AtomicReaderContext;
import org.apache.lucene.index.SortedDocValues;
import org.apache.lucene.search.LeafCollector;
import org.apache.lucene.search.FieldCache;
import org.apache.lucene.search.grouping.AbstractAllGroupsCollector;
import org.apache.lucene.util.BytesRef;
@ -103,7 +104,7 @@ public class TermAllGroupsCollector extends AbstractAllGroupsCollector<BytesRef>
}
@Override
public void setNextReader(AtomicReaderContext context) throws IOException {
protected void doSetNextReader(AtomicReaderContext context) throws IOException {
index = FieldCache.DEFAULT.getTermsIndex(context.reader(), groupField);
// Clear ordSet and fill it with previous encountered groups that can occur in the current segment.

View File

@ -19,6 +19,7 @@ package org.apache.lucene.search.grouping.term;
import org.apache.lucene.index.AtomicReaderContext;
import org.apache.lucene.index.SortedDocValues;
import org.apache.lucene.search.LeafCollector;
import org.apache.lucene.search.FieldCache;
import org.apache.lucene.search.grouping.AbstractDistinctValuesCollector;
import org.apache.lucene.search.grouping.SearchGroup;
@ -107,7 +108,7 @@ public class TermDistinctValuesCollector extends AbstractDistinctValuesCollector
}
@Override
public void setNextReader(AtomicReaderContext context) throws IOException {
protected void doSetNextReader(AtomicReaderContext context) throws IOException {
groupFieldTermIndex = FieldCache.DEFAULT.getTermsIndex(context.reader(), groupField);
countFieldTermIndex = FieldCache.DEFAULT.getTermsIndex(context.reader(), countField);
ordSet.clear();

View File

@ -21,6 +21,7 @@ import java.io.IOException;
import org.apache.lucene.index.AtomicReaderContext;
import org.apache.lucene.index.SortedDocValues;
import org.apache.lucene.search.LeafCollector;
import org.apache.lucene.search.FieldCache;
import org.apache.lucene.search.Sort;
import org.apache.lucene.search.grouping.AbstractFirstPassGroupingCollector;
@ -85,8 +86,8 @@ public class TermFirstPassGroupingCollector extends AbstractFirstPassGroupingCol
}
@Override
public void setNextReader(AtomicReaderContext readerContext) throws IOException {
super.setNextReader(readerContext);
protected void doSetNextReader(AtomicReaderContext readerContext) throws IOException {
super.doSetNextReader(readerContext);
index = FieldCache.DEFAULT.getTermsIndex(readerContext.reader(), groupField);
}
}

View File

@ -21,6 +21,7 @@ import org.apache.lucene.index.AtomicReaderContext;
import org.apache.lucene.index.SortedDocValues;
import org.apache.lucene.index.SortedSetDocValues;
import org.apache.lucene.index.TermsEnum;
import org.apache.lucene.search.LeafCollector;
import org.apache.lucene.search.FieldCache;
import org.apache.lucene.search.grouping.AbstractGroupFacetCollector;
import org.apache.lucene.util.BytesRef;
@ -122,7 +123,7 @@ public abstract class TermGroupFacetCollector extends AbstractGroupFacetCollecto
}
@Override
public void setNextReader(AtomicReaderContext context) throws IOException {
protected void doSetNextReader(AtomicReaderContext context) throws IOException {
if (segmentFacetCounts != null) {
segmentResults.add(createSegmentResult());
}
@ -277,7 +278,7 @@ public abstract class TermGroupFacetCollector extends AbstractGroupFacetCollecto
}
@Override
public void setNextReader(AtomicReaderContext context) throws IOException {
protected void doSetNextReader(AtomicReaderContext context) throws IOException {
if (segmentFacetCounts != null) {
segmentResults.add(createSegmentResult());
}

View File

@ -22,6 +22,7 @@ import java.util.Collection;
import org.apache.lucene.index.AtomicReaderContext;
import org.apache.lucene.index.SortedDocValues;
import org.apache.lucene.search.LeafCollector;
import org.apache.lucene.search.FieldCache;
import org.apache.lucene.search.Sort;
import org.apache.lucene.search.grouping.AbstractSecondPassGroupingCollector;
@ -53,8 +54,8 @@ public class TermSecondPassGroupingCollector extends AbstractSecondPassGroupingC
}
@Override
public void setNextReader(AtomicReaderContext readerContext) throws IOException {
super.setNextReader(readerContext);
protected void doSetNextReader(AtomicReaderContext readerContext) throws IOException {
super.doSetNextReader(readerContext);
index = FieldCache.DEFAULT.getTermsIndex(readerContext.reader(), groupField);
// Rebuild ordSet

View File

@ -35,12 +35,13 @@ import org.apache.lucene.index.DirectoryReader;
import org.apache.lucene.index.IndexReader;
import org.apache.lucene.index.IndexWriter;
import org.apache.lucene.index.Term;
import org.apache.lucene.search.LeafCollector;
import org.apache.lucene.search.Collector;
import org.apache.lucene.search.IndexSearcher;
import org.apache.lucene.search.PhraseQuery;
import org.apache.lucene.search.Query;
import org.apache.lucene.search.SimpleCollector;
import org.apache.lucene.search.TopDocs;
import org.apache.lucene.search.spans.SpanNearQuery;
import org.apache.lucene.search.spans.SpanQuery;
import org.apache.lucene.search.spans.SpanTermQuery;
@ -116,7 +117,7 @@ public class HighlighterPhraseTest extends LuceneTestCase {
new SpanTermQuery(new Term(FIELD, "fox")),
new SpanTermQuery(new Term(FIELD, "jumped")) }, 0, true);
final FixedBitSet bitset = new FixedBitSet(indexReader.maxDoc());
indexSearcher.search(phraseQuery, new Collector() {
indexSearcher.search(phraseQuery, new SimpleCollector() {
private int baseDoc;
@Override
@ -130,7 +131,7 @@ public class HighlighterPhraseTest extends LuceneTestCase {
}
@Override
public void setNextReader(AtomicReaderContext context) {
protected void doSetNextReader(AtomicReaderContext context) throws IOException {
this.baseDoc = context.docBase;
}

View File

@ -19,11 +19,11 @@ package org.apache.lucene.search.join;
import java.util.Collection;
import org.apache.lucene.search.Collector;
import org.apache.lucene.search.LeafCollector;
import org.apache.lucene.search.Scorer;
import org.apache.lucene.search.Weight;
/** Passed to {@link Collector#setScorer} during join collection. */
/** Passed to {@link LeafCollector#setScorer} during join collection. */
final class FakeScorer extends Scorer {
float score;
int doc = -1;

View File

@ -22,9 +22,11 @@ import java.io.IOException;
import org.apache.lucene.index.AtomicReaderContext;
import org.apache.lucene.index.BinaryDocValues;
import org.apache.lucene.index.SortedSetDocValues;
import org.apache.lucene.search.LeafCollector;
import org.apache.lucene.search.Collector;
import org.apache.lucene.search.FieldCache;
import org.apache.lucene.search.Scorer;
import org.apache.lucene.search.SimpleCollector;
import org.apache.lucene.util.BytesRef;
import org.apache.lucene.util.BytesRefHash;
@ -33,7 +35,7 @@ import org.apache.lucene.util.BytesRefHash;
*
* @lucene.experimental
*/
abstract class TermsCollector extends Collector {
abstract class TermsCollector extends SimpleCollector {
final String field;
final BytesRefHash collectorTerms = new BytesRefHash();
@ -46,10 +48,6 @@ abstract class TermsCollector extends Collector {
return collectorTerms;
}
@Override
public void setScorer(Scorer scorer) throws IOException {
}
@Override
public boolean acceptsDocsOutOfOrder() {
return true;
@ -86,7 +84,7 @@ abstract class TermsCollector extends Collector {
}
@Override
public void setNextReader(AtomicReaderContext context) throws IOException {
protected void doSetNextReader(AtomicReaderContext context) throws IOException {
docTermOrds = FieldCache.DEFAULT.getDocTermOrds(context.reader(), field);
}
}
@ -108,7 +106,7 @@ abstract class TermsCollector extends Collector {
}
@Override
public void setNextReader(AtomicReaderContext context) throws IOException {
protected void doSetNextReader(AtomicReaderContext context) throws IOException {
fromDocTerms = FieldCache.DEFAULT.getTerms(context.reader(), field, false);
}
}

View File

@ -27,6 +27,7 @@ import org.apache.lucene.index.IndexReader;
import org.apache.lucene.index.Term;
import org.apache.lucene.index.Terms;
import org.apache.lucene.index.TermsEnum;
import org.apache.lucene.search.LeafCollector;
import org.apache.lucene.search.Collector;
import org.apache.lucene.search.ComplexExplanation;
import org.apache.lucene.search.DocIdSetIterator;
@ -227,7 +228,7 @@ class TermsIncludingScoreQuery extends Query {
}
@Override
public boolean score(Collector collector, int max) throws IOException {
public boolean score(LeafCollector collector, int max) throws IOException {
FakeScorer fakeScorer = new FakeScorer();
collector.setScorer(fakeScorer);
if (doc == -1) {

View File

@ -22,14 +22,16 @@ import java.io.IOException;
import org.apache.lucene.index.AtomicReaderContext;
import org.apache.lucene.index.BinaryDocValues;
import org.apache.lucene.index.SortedSetDocValues;
import org.apache.lucene.search.LeafCollector;
import org.apache.lucene.search.Collector;
import org.apache.lucene.search.FieldCache;
import org.apache.lucene.search.Scorer;
import org.apache.lucene.search.SimpleCollector;
import org.apache.lucene.util.ArrayUtil;
import org.apache.lucene.util.BytesRef;
import org.apache.lucene.util.BytesRefHash;
abstract class TermsWithScoreCollector extends Collector {
abstract class TermsWithScoreCollector extends SimpleCollector {
private final static int INITIAL_ARRAY_SIZE = 256;
@ -128,7 +130,7 @@ abstract class TermsWithScoreCollector extends Collector {
}
@Override
public void setNextReader(AtomicReaderContext context) throws IOException {
protected void doSetNextReader(AtomicReaderContext context) throws IOException {
fromDocTerms = FieldCache.DEFAULT.getTerms(context.reader(), field, false);
}
@ -214,7 +216,7 @@ abstract class TermsWithScoreCollector extends Collector {
}
@Override
public void setNextReader(AtomicReaderContext context) throws IOException {
protected void doSetNextReader(AtomicReaderContext context) throws IOException {
fromDocTermOrds = FieldCache.DEFAULT.getDocTermOrds(context.reader(), field);
}

View File

@ -74,7 +74,7 @@ import java.util.*;
*
* @lucene.experimental
*/
public class ToParentBlockJoinCollector extends Collector {
public class ToParentBlockJoinCollector extends SimpleCollector {
private final Sort sort;
@ -269,7 +269,7 @@ public class ToParentBlockJoinCollector extends Collector {
}
@Override
public void setNextReader(AtomicReaderContext context) throws IOException {
protected void doSetNextReader(AtomicReaderContext context) throws IOException {
currentReaderContext = context;
docBase = context.docBase;
for (int compIDX = 0; compIDX < comparators.length; compIDX++) {
@ -421,7 +421,7 @@ public class ToParentBlockJoinCollector extends Collector {
}
collector.setScorer(fakeScorer);
collector.setNextReader(og.readerContext);
collector.getLeafCollector(og.readerContext);
for(int docIDX=0;docIDX<numChildDocs;docIDX++) {
//System.out.println("docIDX=" + docIDX + " vs " + og.docs[slot].length);
final int doc = og.docs[slot][docIDX];

View File

@ -47,6 +47,7 @@ import org.apache.lucene.index.SortedSetDocValues;
import org.apache.lucene.index.Term;
import org.apache.lucene.index.Terms;
import org.apache.lucene.index.TermsEnum;
import org.apache.lucene.search.LeafCollector;
import org.apache.lucene.search.BooleanClause;
import org.apache.lucene.search.BooleanQuery;
import org.apache.lucene.search.Collector;
@ -58,6 +59,7 @@ import org.apache.lucene.search.MatchAllDocsQuery;
import org.apache.lucene.search.Query;
import org.apache.lucene.search.ScoreDoc;
import org.apache.lucene.search.Scorer;
import org.apache.lucene.search.SimpleCollector;
import org.apache.lucene.search.TermQuery;
import org.apache.lucene.search.TopDocs;
import org.apache.lucene.search.TopScoreDocCollector;
@ -222,12 +224,9 @@ public class TestJoinUtil extends LuceneTestCase {
bq.add(joinQuery, BooleanClause.Occur.SHOULD);
bq.add(new TermQuery(new Term("id", "3")), BooleanClause.Occur.SHOULD);
indexSearcher.search(bq, new Collector() {
indexSearcher.search(bq, new SimpleCollector() {
boolean sawFive;
@Override
public void setNextReader(AtomicReaderContext context) {
}
@Override
public void collect(int docID) {
// Hairy / evil (depends on how BooleanScorer
// stores temporarily collected docIDs by
@ -239,9 +238,6 @@ public class TestJoinUtil extends LuceneTestCase {
}
}
@Override
public void setScorer(Scorer scorer) {
}
@Override
public boolean acceptsDocsOutOfOrder() {
return true;
}
@ -407,7 +403,7 @@ public class TestJoinUtil extends LuceneTestCase {
// Need to know all documents that have matches. TopDocs doesn't give me that and then I'd be also testing TopDocsCollector...
final FixedBitSet actualResult = new FixedBitSet(indexSearcher.getIndexReader().maxDoc());
final TopScoreDocCollector topScoreDocCollector = TopScoreDocCollector.create(10, false);
indexSearcher.search(joinQuery, new Collector() {
indexSearcher.search(joinQuery, new SimpleCollector() {
int docBase;
@ -418,9 +414,9 @@ public class TestJoinUtil extends LuceneTestCase {
}
@Override
public void setNextReader(AtomicReaderContext context) {
protected void doSetNextReader(AtomicReaderContext context) throws IOException {
docBase = context.docBase;
topScoreDocCollector.setNextReader(context);
topScoreDocCollector.getLeafCollector(context);
}
@Override
@ -572,7 +568,7 @@ public class TestJoinUtil extends LuceneTestCase {
}
final Map<BytesRef, JoinScore> joinValueToJoinScores = new HashMap<>();
if (multipleValuesPerDocument) {
fromSearcher.search(new TermQuery(new Term("value", uniqueRandomValue)), new Collector() {
fromSearcher.search(new TermQuery(new Term("value", uniqueRandomValue)), new SimpleCollector() {
private Scorer scorer;
private SortedSetDocValues docTermOrds;
@ -593,7 +589,7 @@ public class TestJoinUtil extends LuceneTestCase {
}
@Override
public void setNextReader(AtomicReaderContext context) throws IOException {
protected void doSetNextReader(AtomicReaderContext context) throws IOException {
docTermOrds = FieldCache.DEFAULT.getDocTermOrds(context.reader(), fromField);
}
@ -608,7 +604,7 @@ public class TestJoinUtil extends LuceneTestCase {
}
});
} else {
fromSearcher.search(new TermQuery(new Term("value", uniqueRandomValue)), new Collector() {
fromSearcher.search(new TermQuery(new Term("value", uniqueRandomValue)), new SimpleCollector() {
private Scorer scorer;
private BinaryDocValues terms;
@ -631,7 +627,7 @@ public class TestJoinUtil extends LuceneTestCase {
}
@Override
public void setNextReader(AtomicReaderContext context) throws IOException {
protected void doSetNextReader(AtomicReaderContext context) throws IOException {
terms = FieldCache.DEFAULT.getTerms(context.reader(), fromField, true);
docsWithField = FieldCache.DEFAULT.getDocsWithField(context.reader(), fromField);
}
@ -675,7 +671,7 @@ public class TestJoinUtil extends LuceneTestCase {
}
}
} else {
toSearcher.search(new MatchAllDocsQuery(), new Collector() {
toSearcher.search(new MatchAllDocsQuery(), new SimpleCollector() {
private SortedSetDocValues docTermOrds;
private final BytesRef scratch = new BytesRef();
@ -701,7 +697,7 @@ public class TestJoinUtil extends LuceneTestCase {
}
@Override
public void setNextReader(AtomicReaderContext context) throws IOException {
protected void doSetNextReader(AtomicReaderContext context) throws IOException {
docBase = context.docBase;
docTermOrds = FieldCache.DEFAULT.getDocTermOrds(context.reader(), toField);
}
@ -713,7 +709,7 @@ public class TestJoinUtil extends LuceneTestCase {
});
}
} else {
toSearcher.search(new MatchAllDocsQuery(), new Collector() {
toSearcher.search(new MatchAllDocsQuery(), new SimpleCollector() {
private BinaryDocValues terms;
private int docBase;
@ -730,7 +726,7 @@ public class TestJoinUtil extends LuceneTestCase {
}
@Override
public void setNextReader(AtomicReaderContext context) throws IOException {
protected void doSetNextReader(AtomicReaderContext context) throws IOException {
terms = FieldCache.DEFAULT.getTerms(context.reader(), toField, false);
docBase = context.docBase;
}

View File

@ -54,6 +54,7 @@ import org.apache.lucene.search.Collector;
import org.apache.lucene.search.IndexSearcher;
import org.apache.lucene.search.Query;
import org.apache.lucene.search.Scorer;
import org.apache.lucene.search.SimpleCollector;
import org.apache.lucene.search.similarities.Similarity;
import org.apache.lucene.store.RAMDirectory; // for javadocs
import org.apache.lucene.util.ArrayUtil;
@ -532,7 +533,7 @@ public class MemoryIndex {
IndexSearcher searcher = createSearcher();
try {
final float[] scores = new float[1]; // inits to 0.0f (no match)
searcher.search(query, new Collector() {
searcher.search(query, new SimpleCollector() {
private Scorer scorer;
@Override
@ -550,8 +551,6 @@ public class MemoryIndex {
return true;
}
@Override
public void setNextReader(AtomicReaderContext context) { }
});
float score = scores[0];
return score;

View File

@ -21,9 +21,11 @@ import java.io.IOException;
import org.apache.lucene.index.AtomicReaderContext;
import org.apache.lucene.index.IndexWriter;
import org.apache.lucene.search.LeafCollector;
import org.apache.lucene.search.CollectionTerminatedException;
import org.apache.lucene.search.Collector;
import org.apache.lucene.search.Scorer;
import org.apache.lucene.search.FilterLeafCollector;
import org.apache.lucene.search.FilterCollector;
import org.apache.lucene.search.Sort;
import org.apache.lucene.search.TopDocsCollector;
import org.apache.lucene.search.TotalHitCountCollector;
@ -32,11 +34,11 @@ import org.apache.lucene.search.TotalHitCountCollector;
* A {@link Collector} that early terminates collection of documents on a
* per-segment basis, if the segment was sorted according to the given
* {@link Sort}.
*
*
* <p>
* <b>NOTE:</b> the {@code Collector} detects sorted segments according to
* {@link SortingMergePolicy}, so it's best used in conjunction with it. Also,
* it collects up to a specified {@code numDocsToCollect} from each segment,
* it collects up to a specified {@code numDocsToCollect} from each segment,
* and therefore is mostly suitable for use in conjunction with collectors such as
* {@link TopDocsCollector}, and not e.g. {@link TotalHitCountCollector}.
* <p>
@ -58,26 +60,21 @@ import org.apache.lucene.search.TotalHitCountCollector;
* the old and the new {@code Sort}s have the same identifier, this
* {@code Collector} will incorrectly detect sorted segments.</li>
* </ul>
*
*
* @lucene.experimental
*/
public class EarlyTerminatingSortingCollector extends Collector {
/** The wrapped Collector */
protected final Collector in;
public class EarlyTerminatingSortingCollector extends FilterCollector {
/** Sort used to sort the search results */
protected final Sort sort;
/** Number of documents to collect in each segment */
protected final int numDocsToCollect;
/** Number of documents to collect in the current segment being processed */
protected int segmentTotalCollect;
/** True if the current segment being processed is sorted by {@link #sort} */
protected boolean segmentSorted;
private int numCollected;
/**
* Create a new {@link EarlyTerminatingSortingCollector} instance.
*
*
* @param in
* the collector to wrap
* @param sort
@ -88,38 +85,37 @@ public class EarlyTerminatingSortingCollector extends Collector {
* hits.
*/
public EarlyTerminatingSortingCollector(Collector in, Sort sort, int numDocsToCollect) {
super(in);
if (numDocsToCollect <= 0) {
throw new IllegalStateException("numDocsToCollect must always be > 0, got " + segmentTotalCollect);
throw new IllegalStateException("numDocsToCollect must always be > 0, got " + numDocsToCollect);
}
this.in = in;
this.sort = sort;
this.numDocsToCollect = numDocsToCollect;
}
@Override
public void setScorer(Scorer scorer) throws IOException {
in.setScorer(scorer);
}
public LeafCollector getLeafCollector(AtomicReaderContext context) throws IOException {
if (SortingMergePolicy.isSorted(context.reader(), sort)) {
// segment is sorted, can early-terminate
return new FilterLeafCollector(super.getLeafCollector(context)) {
@Override
public void collect(int doc) throws IOException {
in.collect(doc);
if (++numCollected >= segmentTotalCollect) {
throw new CollectionTerminatedException();
@Override
public void collect(int doc) throws IOException {
super.collect(doc);
if (++numCollected >= numDocsToCollect) {
throw new CollectionTerminatedException();
}
}
@Override
public boolean acceptsDocsOutOfOrder() {
return false;
}
};
} else {
return super.getLeafCollector(context);
}
}
@Override
public void setNextReader(AtomicReaderContext context) throws IOException {
in.setNextReader(context);
segmentSorted = SortingMergePolicy.isSorted(context.reader(), sort);
segmentTotalCollect = segmentSorted ? numDocsToCollect : Integer.MAX_VALUE;
numCollected = 0;
}
@Override
public boolean acceptsDocsOutOfOrder() {
return !segmentSorted && in.acceptsDocsOutOfOrder();
}
}

View File

@ -34,6 +34,7 @@ import org.apache.lucene.index.IndexReader;
import org.apache.lucene.index.IndexWriterConfig;
import org.apache.lucene.index.RandomIndexWriter;
import org.apache.lucene.index.Term;
import org.apache.lucene.search.LeafCollector;
import org.apache.lucene.search.IndexSearcher;
import org.apache.lucene.search.ScoreDoc;
import org.apache.lucene.search.Sort;
@ -147,9 +148,10 @@ public class TestEarlyTermination extends LuceneTestCase {
Sort different = new Sort(new SortField("ndv2", SortField.Type.LONG));
searcher.search(query, new EarlyTerminatingSortingCollector(collector2, different, numHits) {
@Override
public void setNextReader(AtomicReaderContext context) throws IOException {
super.setNextReader(context);
assertFalse("segment should not be recognized as sorted as different sorter was used", segmentSorted);
public LeafCollector getLeafCollector(AtomicReaderContext context) throws IOException {
final LeafCollector ret = super.getLeafCollector(context);
assertTrue("segment should not be recognized as sorted as different sorter was used", ret.getClass() == in.getLeafCollector(context).getClass());
return ret;
}
});
}

View File

@ -22,13 +22,13 @@ import java.io.IOException;
import org.apache.lucene.index.AtomicReaderContext;
import org.apache.lucene.index.DirectoryReader;
import org.apache.lucene.index.IndexReader;
import org.apache.lucene.search.LeafCollector;
import org.apache.lucene.search.IndexSearcher;
import org.apache.lucene.search.Collector;
import org.apache.lucene.search.Scorer;
import org.apache.lucene.search.Query;
import org.apache.lucene.search.SimpleCollector;
import org.apache.lucene.queryparser.surround.parser.QueryParser;
import org.junit.Assert;
public class BooleanQueryTst {
@ -57,7 +57,7 @@ public class BooleanQueryTst {
public void setVerbose(boolean verbose) {this.verbose = verbose;}
class TestCollector extends Collector { // FIXME: use check hits from Lucene tests
class TestCollector extends SimpleCollector { // FIXME: use check hits from Lucene tests
int totalMatched;
boolean[] encountered;
private Scorer scorer = null;
@ -79,7 +79,7 @@ public class BooleanQueryTst {
}
@Override
public void setNextReader(AtomicReaderContext context) throws IOException {
protected void doSetNextReader(AtomicReaderContext context) throws IOException {
docBase = context.docBase;
}

View File

@ -25,7 +25,6 @@ import org.apache.lucene.spatial.prefix.tree.Cell;
import org.apache.lucene.spatial.prefix.tree.SpatialPrefixTree;
import org.apache.lucene.util.Bits;
import org.apache.lucene.util.BytesRef;
import org.apache.lucene.util.StringHelper;
import java.io.IOException;
import java.util.Iterator;
@ -117,7 +116,7 @@ public abstract class AbstractVisitingPrefixTreeFilter extends AbstractPrefixTre
protected final boolean hasIndexedLeaves;//if false then we can skip looking for them
private VNode curVNode;//current pointer, derived from query shape
private BytesRef curVNodeTerm = new BytesRef();//curVNode.cell's term.
private BytesRef curVNodeTerm = new BytesRef();//curVNode.cell's term, without leaf
private Cell scanCell;
private BytesRef thisTerm;//the result of termsEnum.term()
@ -171,8 +170,7 @@ public abstract class AbstractVisitingPrefixTreeFilter extends AbstractPrefixTre
}
//Seek to curVNode's cell (or skip if termsEnum has moved beyond)
curVNodeTerm.bytes = curVNode.cell.getTokenBytes();
curVNodeTerm.length = curVNodeTerm.bytes.length;
curVNode.cell.getTokenBytesNoLeaf(curVNodeTerm);
int compare = thisTerm.compareTo(curVNodeTerm);
if (compare > 0) {
// leap frog (termsEnum is beyond where we would otherwise seek)
@ -215,7 +213,7 @@ public abstract class AbstractVisitingPrefixTreeFilter extends AbstractPrefixTre
if (hasIndexedLeaves && cell.getLevel() != 0) {
//If the next indexed term just adds a leaf marker ('+') to cell,
// then add all of those docs
assert StringHelper.startsWith(thisTerm, curVNodeTerm);//TODO refactor to use method on curVNode.cell
assert curVNode.cell.isWithin(curVNodeTerm, thisTerm);
scanCell = grid.getCell(thisTerm.bytes, thisTerm.offset, thisTerm.length, scanCell);
if (scanCell.getLevel() == cell.getLevel() && scanCell.isLeaf()) {
visitLeaf(scanCell);
@ -265,7 +263,7 @@ public abstract class AbstractVisitingPrefixTreeFilter extends AbstractPrefixTre
*/
protected void scan(int scanDetailLevel) throws IOException {
for (;
thisTerm != null && StringHelper.startsWith(thisTerm, curVNodeTerm);//TODO refactor to use method on curVNode.cell
thisTerm != null && curVNode.cell.isWithin(curVNodeTerm, thisTerm);
thisTerm = termsEnum.next()) {
scanCell = grid.getCell(thisTerm.bytes, thisTerm.offset, thisTerm.length, scanCell);

View File

@ -0,0 +1,183 @@
package org.apache.lucene.spatial.prefix;
/*
* Licensed to the Apache Software Foundation (ASF) under one or more
* contributor license agreements. See the NOTICE file distributed with
* this work for additional information regarding copyright ownership.
* The ASF licenses this file to You under the Apache License, Version 2.0
* (the "License"); you may not use this file except in compliance with
* the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
import org.apache.lucene.analysis.TokenStream;
import org.apache.lucene.analysis.tokenattributes.CharTermAttribute;
import org.apache.lucene.analysis.tokenattributes.TermToBytesRefAttribute;
import org.apache.lucene.spatial.prefix.tree.Cell;
import org.apache.lucene.util.Attribute;
import org.apache.lucene.util.AttributeImpl;
import org.apache.lucene.util.AttributeReflector;
import org.apache.lucene.util.AttributeSource;
import org.apache.lucene.util.BytesRef;
import java.io.IOException;
import java.util.Iterator;
/**
* A TokenStream used internally by {@link org.apache.lucene.spatial.prefix.PrefixTreeStrategy}.
*
* This is highly modelled after {@link org.apache.lucene.analysis.NumericTokenStream}.
*
* If there is demand for it to be public; it could be made to be.
*
* @lucene.internal
*/
class CellTokenStream extends TokenStream {
private interface CellTermAttribute extends Attribute {
Cell getCell();
void setCell(Cell cell);
//TODO one day deprecate this once we have better encodings
boolean getOmitLeafByte();
void setOmitLeafByte(boolean b);
}
// just a wrapper to prevent adding CTA
private static final class CellAttributeFactory extends AttributeSource.AttributeFactory {
private final AttributeSource.AttributeFactory delegate;
CellAttributeFactory(AttributeSource.AttributeFactory delegate) {
this.delegate = delegate;
}
@Override
public AttributeImpl createAttributeInstance(Class<? extends Attribute> attClass) {
if (CharTermAttribute.class.isAssignableFrom(attClass))
throw new IllegalArgumentException("CellTokenStream does not support CharTermAttribute.");
return delegate.createAttributeInstance(attClass);
}
}
private static final class CellTermAttributeImpl extends AttributeImpl
implements CellTermAttribute, TermToBytesRefAttribute {
private BytesRef bytes = new BytesRef();
private Cell cell;
private boolean omitLeafByte;//false by default (whether there's a leaf byte or not)
@Override
public Cell getCell() {
return cell;
}
@Override
public boolean getOmitLeafByte() {
return omitLeafByte;
}
@Override
public void setCell(Cell cell) {
this.cell = cell;
omitLeafByte = false;//reset
}
@Override
public void setOmitLeafByte(boolean b) {
omitLeafByte = b;
}
@Override
public void clear() {
// this attribute has no contents to clear!
// we keep it untouched as it's fully controlled by outer class.
}
@Override
public void copyTo(AttributeImpl target) {
final CellTermAttribute a = (CellTermAttribute) target;
a.setCell(cell);
a.setOmitLeafByte(omitLeafByte);
}
@Override
public int fillBytesRef() {
if (omitLeafByte)
cell.getTokenBytesNoLeaf(bytes);
else
cell.getTokenBytes(bytes);
return bytes.hashCode();
}
@Override
public BytesRef getBytesRef() {
return bytes;
}
@Override
public void reflectWith(AttributeReflector reflector) {
fillBytesRef();
reflector.reflect(TermToBytesRefAttribute.class, "bytes", BytesRef.deepCopyOf(bytes));
}
}
public CellTokenStream() {
this(AttributeFactory.DEFAULT_ATTRIBUTE_FACTORY);
}
public CellTokenStream(AttributeFactory factory) {
super(new CellAttributeFactory(factory));
}
public CellTokenStream setCells(Iterator<Cell> iter) {
this.iter = iter;
return this;
}
@Override
public void reset() throws IOException {
if (iter == null)
throw new IllegalStateException("call setCells() before usage");
cellAtt.setCell(null);
cellAtt.setOmitLeafByte(false);
}
/** Outputs the token of a cell, and if its a leaf, outputs it again with the leaf byte. */
@Override
public final boolean incrementToken() {
if (iter == null)
throw new IllegalStateException("call setCells() before usage");
// this will only clear all other attributes in this TokenStream
clearAttributes();
if (cellAtt.getOmitLeafByte()) {
cellAtt.setOmitLeafByte(false);
return true;
}
//get next
if (!iter.hasNext())
return false;
cellAtt.setCell(iter.next());
if (cellAtt.getCell().isLeaf())
cellAtt.setOmitLeafByte(true);
return true;
}
{
addAttributeImpl(new CellTermAttributeImpl());//because non-public constructor
}
//members
private final CellTermAttribute cellAtt = addAttribute(CellTermAttribute.class);
//TODO support position increment, and TypeAttribute
private Iterator<Cell> iter = null; // null means not initialized
}

View File

@ -83,7 +83,7 @@ public class ContainsPrefixTreeFilter extends AbstractPrefixTreeFilter {
super(context, acceptDocs);
}
BytesRef termBytes = new BytesRef();
BytesRef termBytes = new BytesRef();//no leaf
Cell nextCell;//see getLeafDocs
/** This is the primary algorithm; recursive. Returns null if finds none. */
@ -130,16 +130,15 @@ public class ContainsPrefixTreeFilter extends AbstractPrefixTreeFilter {
}
private boolean seekExact(Cell cell) throws IOException {
assert new BytesRef(cell.getTokenBytes()).compareTo(termBytes) > 0;
termBytes.bytes = cell.getTokenBytes();
termBytes.length = termBytes.bytes.length;
assert cell.getTokenBytesNoLeaf(null).compareTo(termBytes) > 0;
cell.getTokenBytesNoLeaf(termBytes);
if (termsEnum == null)
return false;
return termsEnum.seekExact(termBytes);
}
private SmallDocSet getDocs(Cell cell, Bits acceptContains) throws IOException {
assert new BytesRef(cell.getTokenBytes()).equals(termBytes);
assert cell.getTokenBytesNoLeaf(null).equals(termBytes);
return collectDocs(acceptContains);
}
@ -147,7 +146,7 @@ public class ContainsPrefixTreeFilter extends AbstractPrefixTreeFilter {
private Cell lastLeaf = null;//just for assertion
private SmallDocSet getLeafDocs(Cell leafCell, Bits acceptContains) throws IOException {
assert new BytesRef(leafCell.getTokenBytes()).equals(termBytes);
assert leafCell.getTokenBytesNoLeaf(null).equals(termBytes);
assert ! leafCell.equals(lastLeaf);//don't call for same leaf again
lastLeaf = leafCell;

View File

@ -46,7 +46,7 @@ public class PointPrefixTreeFieldCacheProvider extends ShapeFieldCacheProvider<P
protected Point readShape(BytesRef term) {
scanCell = grid.getCell(term.bytes, term.offset, term.length, scanCell);
if (scanCell.getLevel() == grid.getMaxLevels() && !scanCell.isLeaf())
return scanCell.getCenter();
return scanCell.getShape().getCenter();
return null;
}
}

View File

@ -19,8 +19,6 @@ package org.apache.lucene.spatial.prefix;
import com.spatial4j.core.shape.Point;
import com.spatial4j.core.shape.Shape;
import org.apache.lucene.analysis.TokenStream;
import org.apache.lucene.analysis.tokenattributes.CharTermAttribute;
import org.apache.lucene.document.Field;
import org.apache.lucene.document.FieldType;
import org.apache.lucene.index.FieldInfo;
@ -31,7 +29,6 @@ import org.apache.lucene.spatial.prefix.tree.SpatialPrefixTree;
import org.apache.lucene.spatial.query.SpatialArgs;
import org.apache.lucene.spatial.util.ShapeFieldCacheDistanceValueSource;
import java.util.Iterator;
import java.util.List;
import java.util.Map;
import java.util.concurrent.ConcurrentHashMap;
@ -125,13 +122,12 @@ public abstract class PrefixTreeStrategy extends SpatialStrategy {
public Field[] createIndexableFields(Shape shape, double distErr) {
int detailLevel = grid.getLevelForDistance(distErr);
// note: maybe CellTokenStream should do this line, but it doesn't matter and it would create extra
// coupling
List<Cell> cells = grid.getCells(shape, detailLevel, true, simplifyIndexedCells);//intermediates cells
//TODO is CellTokenStream supposed to be re-used somehow? see Uwe's comments:
// http://code.google.com/p/lucene-spatial-playground/issues/detail?id=4
Field field = new Field(getFieldName(),
new CellTokenStream(cells.iterator()), FIELD_TYPE);
new CellTokenStream().setCells(cells.iterator()), FIELD_TYPE);
return new Field[]{field};
}
@ -146,41 +142,6 @@ public abstract class PrefixTreeStrategy extends SpatialStrategy {
FIELD_TYPE.freeze();
}
/** Outputs the tokenString of a cell, and if its a leaf, outputs it again with the leaf byte. */
final static class CellTokenStream extends TokenStream {
private final CharTermAttribute termAtt = addAttribute(CharTermAttribute.class);
private Iterator<Cell> iter = null;
public CellTokenStream(Iterator<Cell> tokens) {
this.iter = tokens;
}
CharSequence nextTokenStringNeedingLeaf = null;
@Override
public boolean incrementToken() {
clearAttributes();
if (nextTokenStringNeedingLeaf != null) {
termAtt.append(nextTokenStringNeedingLeaf);
termAtt.append((char) Cell.LEAF_BYTE);
nextTokenStringNeedingLeaf = null;
return true;
}
if (iter.hasNext()) {
Cell cell = iter.next();
CharSequence token = cell.getTokenString();
termAtt.append(token);
if (cell.isLeaf())
nextTokenStringNeedingLeaf = token;
return true;
}
return false;
}
}
@Override
public ValueSource makeDistanceValueSource(Point queryPoint, double multiplier) {
PointPrefixTreeFieldCacheProvider p = provider.get( getFieldName() );

View File

@ -61,7 +61,7 @@ public class TermQueryPrefixTreeStrategy extends PrefixTreeStrategy {
BytesRef[] terms = new BytesRef[cells.size()];
int i = 0;
for (Cell cell : cells) {
terms[i++] = new BytesRef(cell.getTokenString());//TODO use cell.getTokenBytes()
terms[i++] = cell.getTokenBytesNoLeaf(null);
}
return new TermsFilter(getFieldName(), terms);
}

View File

@ -20,6 +20,8 @@ package org.apache.lucene.spatial.prefix.tree;
import com.spatial4j.core.shape.Point;
import com.spatial4j.core.shape.Shape;
import com.spatial4j.core.shape.SpatialRelation;
import org.apache.lucene.util.BytesRef;
import org.apache.lucene.util.StringHelper;
import java.util.ArrayList;
import java.util.Collection;
@ -27,74 +29,45 @@ import java.util.Collections;
import java.util.List;
/**
* Represents a grid cell. These are not necessarily thread-safe, although new
* Cell("") (world cell) must be.
* Represents a grid cell. These are not necessarily thread-safe, although calling {@link #getShape()} will
* sufficiently prepare it to be so, if needed.
*
* @lucene.experimental
*/
public abstract class Cell implements Comparable<Cell> {
public static final byte LEAF_BYTE = '+';//NOTE: must sort before letters & numbers
public abstract class Cell {
/*
Holds a byte[] and/or String representation of the cell. Both are lazy constructed from the other.
Neither contains the trailing leaf byte.
*/
private static final byte LEAF_BYTE = '+';//NOTE: must sort before letters & numbers
//Arguably we could simply use a BytesRef, using an extra Object.
private byte[] bytes;
private int b_off;
private int b_len;
private String token;//this is the only part of equality
/**
* When set via getSubCells(filter), it is the relationship between this cell
* and the given shape filter.
* and the given shape filter. Doesn't participate in shape equality.
*/
protected SpatialRelation shapeRel;
/**
* Always false for points. Otherwise, indicate no further sub-cells are going
* to be provided because shapeRel is WITHIN or maxLevels or a detailLevel is
* hit.
*/
protected boolean leaf;
protected Cell(String token) {
this.token = token;
if (token.length() > 0 && token.charAt(token.length() - 1) == (char) LEAF_BYTE) {
this.token = token.substring(0, token.length() - 1);
setLeaf();
}
if (getLevel() == 0)
getShape();//ensure any lazy instantiation completes to make this threadsafe
}
/** Warning: Refers to the same bytes (no copy). If {@link #setLeaf()} is subsequently called then it
* may modify bytes. */
protected Cell(byte[] bytes, int off, int len) {
this.bytes = bytes;
this.b_off = off;
this.b_len = len;
b_fixLeaf();
}
/** Warning: Refers to the same bytes (no copy). If {@link #setLeaf()} is subsequently called then it
* may modify bytes. */
public void reset(byte[] bytes, int off, int len) {
assert getLevel() != 0;
token = null;
shapeRel = null;
this.bytes = bytes;
this.b_off = off;
this.b_len = len;
b_fixLeaf();
}
private void b_fixLeaf() {
//note that non-point shapes always have the maxLevels cell set with setLeaf
if (bytes[b_off + b_len - 1] == LEAF_BYTE) {
b_len--;
setLeaf();
} else {
leaf = false;
}
}
protected abstract SpatialPrefixTree getGrid();
public SpatialRelation getShapeRel() {
return shapeRel;
@ -105,47 +78,68 @@ public abstract class Cell implements Comparable<Cell> {
* further cells with this prefix for the shape (always true at maxLevels).
*/
public boolean isLeaf() {
return leaf;
return (b_len > 0 && bytes[b_off + b_len - 1] == LEAF_BYTE);
}
/** Note: not supported at level 0. */
/** Modifies the bytes to reflect that this is a leaf. Warning: never invoke from a cell
* initialized to reference the same bytes from termsEnum, which should be treated as immutable.
* Note: not supported at level 0. */
public void setLeaf() {
assert getLevel() != 0;
leaf = true;
}
/**
* Note: doesn't contain a trailing leaf byte.
*/
public String getTokenString() {
if (token == null) {
token = new String(bytes, b_off, b_len, SpatialPrefixTree.UTF8);
}
return token;
}
/**
* Note: doesn't contain a trailing leaf byte.
*/
public byte[] getTokenBytes() {
if (bytes != null) {
if (b_off != 0 || b_len != bytes.length) {
throw new IllegalStateException("Not supported if byte[] needs to be recreated.");
}
} else {
bytes = token.getBytes(SpatialPrefixTree.UTF8);
if (isLeaf())
return;
//if isn't big enough, we have to copy
if (bytes.length < b_off + b_len) {
//hopefully this copying doesn't happen too much (DWS: I checked and it doesn't seem to happen)
byte[] copy = new byte[b_len + 1];
System.arraycopy(bytes, b_off, copy, 0, b_len);
copy[b_len++] = LEAF_BYTE;
bytes = copy;
b_off = 0;
b_len = bytes.length;
} else {
bytes[b_off + b_len++] = LEAF_BYTE;
}
return bytes;
}
/**
* Returns the bytes for this cell.
* The result param is used to save object allocation, though it's bytes aren't used.
* @param result where the result goes, or null to create new
*/
public BytesRef getTokenBytes(BytesRef result) {
if (result == null)
result = new BytesRef();
result.bytes = bytes;
result.offset = b_off;
result.length = b_len;
return result;
}
/**
* Returns the bytes for this cell, without leaf set. The bytes should sort before any
* cells that have the leaf set for the spatial location.
* The result param is used to save object allocation, though it's bytes aren't used.
* @param result where the result goes, or null to create new
*/
public BytesRef getTokenBytesNoLeaf(BytesRef result) {
result = getTokenBytes(result);
if (isLeaf())
result.length--;
return result;
}
/** Level 0 is the world (and has no parent), from then on a higher level means a smaller
* cell than the level before it.
*/
public int getLevel() {
return token != null ? token.length() : b_len;
return isLeaf() ? b_len - 1 : b_len;
}
//TODO add getParent() and update some algorithms to use this?
//public Cell getParent();
/** Gets the parent cell that contains this one. Don't call on the world cell. */
public Cell getParent() {
assert getLevel() > 0;
return getGrid().getCell(bytes, b_off, b_len - (isLeaf() ? 2 : 1));
}
/**
* Like {@link #getSubCells()} but with the results filtered by a shape. If
@ -196,8 +190,6 @@ public abstract class Cell implements Comparable<Cell> {
*/
public abstract Cell getSubCell(Point p);
//TODO Cell getSubCell(byte b)
/**
* Gets the cells at the next grid cell level that cover this cell.
* Precondition: Never called when getLevel() == maxLevel.
@ -211,30 +203,45 @@ public abstract class Cell implements Comparable<Cell> {
*/
public abstract int getSubCellsSize();
/** Gets the shape for this cell; typically a Rectangle. This method also serves to trigger any lazy
* loading needed to make the cell instance thread-safe.
*/
public abstract Shape getShape();
/** TODO remove once no longer used. */
public Point getCenter() {
return getShape().getCenter();
}
@Override
public int compareTo(Cell o) {
return getTokenString().compareTo(o.getTokenString());
}
@Override
public boolean equals(Object obj) {
return !(obj == null || !(obj instanceof Cell)) && getTokenString().equals(((Cell) obj).getTokenString());
//this method isn't "normally" called; just in asserts/tests
if (obj instanceof Cell) {
Cell cell = (Cell) obj;
return getTokenBytes(null).equals(cell.getTokenBytes(null));
} else {
return false;
}
}
@Override
public int hashCode() {
return getTokenString().hashCode();
return getTokenBytesNoLeaf(null).hashCode();
}
@Override
public String toString() {
return getTokenString() + (isLeaf() ? (char) LEAF_BYTE : "");
//this method isn't "normally" called; just in asserts/tests
return getTokenBytes(null).utf8ToString();
}
/**
* Returns if the target term is within/underneath this cell; not necessarily a direct descendant.
* @param bytesNoLeaf must be getTokenBytesNoLeaf
* @param term the term
*/
public boolean isWithin(BytesRef bytesNoLeaf, BytesRef term) {
assert bytesNoLeaf.equals(getTokenBytesNoLeaf(null));
return StringHelper.startsWith(term, bytesNoLeaf);
}
}

View File

@ -83,28 +83,43 @@ public class GeohashPrefixTree extends SpatialPrefixTree {
return new GhCell(GeohashUtils.encodeLatLon(p.getY(), p.getX(), level));//args are lat,lon (y,x)
}
@Override
public Cell getCell(String token) {
return new GhCell(token);
}
@Override
public Cell getCell(byte[] bytes, int offset, int len) {
return new GhCell(bytes, offset, len);
}
private static byte[] stringToBytesPlus1(String token) {
//copy ASCII token to byte array with one extra spot for eventual LEAF_BYTE if needed
byte[] bytes = new byte[token.length() + 1];
for (int i = 0; i < token.length(); i++) {
bytes[i] = (byte) token.charAt(i);
}
return bytes;
}
class GhCell extends Cell {
GhCell(String token) {
super(token);
private Shape shape;//cache
private String geohash;//cache; never has leaf byte, simply a geohash
GhCell(String geohash) {
super(stringToBytesPlus1(geohash), 0, geohash.length());
this.geohash = geohash;
if (isLeaf())
this.geohash = geohash.substring(0, geohash.length() - 1);
}
GhCell(byte[] bytes, int off, int len) {
super(bytes, off, len);
}
@Override
protected SpatialPrefixTree getGrid() { return GeohashPrefixTree.this; }
@Override
public void reset(byte[] bytes, int off, int len) {
super.reset(bytes, off, len);
geohash = null;
shape = null;
}
@ -125,26 +140,26 @@ public class GeohashPrefixTree extends SpatialPrefixTree {
@Override
public Cell getSubCell(Point p) {
return GeohashPrefixTree.this.getCell(p, getLevel() + 1);//not performant!
return getGrid().getCell(p, getLevel() + 1);//not performant!
}
private Shape shape;//cache
@Override
public Shape getShape() {
if (shape == null) {
shape = GeohashUtils.decodeBoundary(getGeohash(), ctx);
shape = GeohashUtils.decodeBoundary(getGeohash(), getGrid().getSpatialContext());
}
return shape;
}
@Override
public Point getCenter() {
return GeohashUtils.decode(getGeohash(), ctx);
return GeohashUtils.decode(getGeohash(), getGrid().getSpatialContext());
}
private String getGeohash() {
return getTokenString();
if (geohash == null)
geohash = getTokenBytesNoLeaf(null).utf8ToString();
return geohash;
}
}//class GhCell

View File

@ -22,6 +22,7 @@ import com.spatial4j.core.shape.Point;
import com.spatial4j.core.shape.Rectangle;
import com.spatial4j.core.shape.Shape;
import com.spatial4j.core.shape.SpatialRelation;
import org.apache.lucene.util.BytesRef;
import java.io.PrintStream;
import java.text.NumberFormat;
@ -142,15 +143,10 @@ public class QuadPrefixTree extends SpatialPrefixTree {
@Override
public Cell getCell(Point p, int level) {
List<Cell> cells = new ArrayList<>(1);
build(xmid, ymid, 0, cells, new StringBuilder(), ctx.makePoint(p.getX(),p.getY()), level);
build(xmid, ymid, 0, cells, new BytesRef(maxLevels+1), ctx.makePoint(p.getX(),p.getY()), level);
return cells.get(0);//note cells could be longer if p on edge
}
@Override
public Cell getCell(String token) {
return new QuadCell(token);
}
@Override
public Cell getCell(byte[] bytes, int offset, int len) {
return new QuadCell(bytes, offset, len);
@ -161,10 +157,10 @@ public class QuadPrefixTree extends SpatialPrefixTree {
double y,
int level,
List<Cell> matches,
StringBuilder str,
BytesRef str,
Shape shape,
int maxLevel) {
assert str.length() == level;
assert str.length == level;
double w = levelW[level] / 2;
double h = levelH[level] / 2;
@ -187,51 +183,51 @@ public class QuadPrefixTree extends SpatialPrefixTree {
double cy,
int level,
List<Cell> matches,
StringBuilder str,
BytesRef str,
Shape shape,
int maxLevel) {
assert str.length() == level;
assert str.length == level;
assert str.offset == 0;
double w = levelW[level] / 2;
double h = levelH[level] / 2;
int strlen = str.length();
int strlen = str.length;
Rectangle rectangle = ctx.makeRectangle(cx - w, cx + w, cy - h, cy + h);
SpatialRelation v = shape.relate(rectangle);
if (SpatialRelation.CONTAINS == v) {
str.append(c);
str.bytes[str.length++] = (byte)c;//append
//str.append(SpatialPrefixGrid.COVER);
matches.add(new QuadCell(str.toString(),v.transpose()));
matches.add(new QuadCell(BytesRef.deepCopyOf(str), v.transpose()));
} else if (SpatialRelation.DISJOINT == v) {
// nothing
} else { // SpatialRelation.WITHIN, SpatialRelation.INTERSECTS
str.append(c);
str.bytes[str.length++] = (byte)c;//append
int nextLevel = level+1;
if (nextLevel >= maxLevel) {
//str.append(SpatialPrefixGrid.INTERSECTS);
matches.add(new QuadCell(str.toString(),v.transpose()));
matches.add(new QuadCell(BytesRef.deepCopyOf(str), v.transpose()));
} else {
build(cx, cy, nextLevel, matches, str, shape, maxLevel);
}
}
str.setLength(strlen);
str.length = strlen;
}
class QuadCell extends Cell {
public QuadCell(String token) {
super(token);
}
public QuadCell(String token, SpatialRelation shapeRel) {
super(token);
this.shapeRel = shapeRel;
}
class QuadCell extends Cell{
QuadCell(byte[] bytes, int off, int len) {
super(bytes, off, len);
}
QuadCell(BytesRef str, SpatialRelation shapeRel) {
this(str.bytes, str.offset, str.length);
this.shapeRel = shapeRel;
}
@Override
protected SpatialPrefixTree getGrid() { return QuadPrefixTree.this; }
@Override
public void reset(byte[] bytes, int off, int len) {
super.reset(bytes, off, len);
@ -240,14 +236,26 @@ public class QuadPrefixTree extends SpatialPrefixTree {
@Override
public Collection<Cell> getSubCells() {
BytesRef source = getTokenBytesNoLeaf(null);
BytesRef target = new BytesRef();
List<Cell> cells = new ArrayList<>(4);
cells.add(new QuadCell(getTokenString()+"A"));
cells.add(new QuadCell(getTokenString()+"B"));
cells.add(new QuadCell(getTokenString()+"C"));
cells.add(new QuadCell(getTokenString()+"D"));
cells.add(new QuadCell(concat(source, (byte)'A', target), null));
cells.add(new QuadCell(concat(source, (byte)'B', target), null));
cells.add(new QuadCell(concat(source, (byte)'C', target), null));
cells.add(new QuadCell(concat(source, (byte)'D', target), null));
return cells;
}
private BytesRef concat(BytesRef source, byte b, BytesRef target) {
assert target.offset == 0;
target.bytes = new byte[source.length + 2];//+2 for new char + potential leaf
target.length = 0;
target.copyBytes(source);
target.bytes[target.length++] = b;
return target;
}
@Override
public int getSubCellsSize() {
return 4;
@ -268,27 +276,30 @@ public class QuadPrefixTree extends SpatialPrefixTree {
}
private Rectangle makeShape() {
String token = getTokenString();
BytesRef token = getTokenBytesNoLeaf(null);
double xmin = QuadPrefixTree.this.xmin;
double ymin = QuadPrefixTree.this.ymin;
for (int i = 0; i < token.length(); i++) {
char c = token.charAt(i);
if ('A' == c || 'a' == c) {
ymin += levelH[i];
} else if ('B' == c || 'b' == c) {
xmin += levelW[i];
ymin += levelH[i];
} else if ('C' == c || 'c' == c) {
// nothing really
}
else if('D' == c || 'd' == c) {
xmin += levelW[i];
} else {
throw new RuntimeException("unexpected char: " + c);
for (int i = 0; i < token.length; i++) {
byte c = token.bytes[token.offset + i];
switch (c) {
case 'A':
ymin += levelH[i];
break;
case 'B':
xmin += levelW[i];
ymin += levelH[i];
break;
case 'C':
break;//nothing really
case 'D':
xmin += levelW[i];
break;
default:
throw new RuntimeException("unexpected char: " + c);
}
}
int len = token.length();
int len = token.length;
double width, height;
if (len > 0) {
width = levelW[len-1];

View File

@ -21,10 +21,12 @@ import com.spatial4j.core.context.SpatialContext;
import com.spatial4j.core.shape.Point;
import com.spatial4j.core.shape.Rectangle;
import com.spatial4j.core.shape.Shape;
import org.apache.lucene.util.BytesRef;
import java.nio.charset.Charset;
import java.nio.charset.StandardCharsets;
import java.util.ArrayList;
import java.util.Arrays;
import java.util.Collection;
import java.util.Collections;
import java.util.List;
@ -103,14 +105,14 @@ public abstract class SpatialPrefixTree {
private transient Cell worldCell;//cached
/**
* Returns the level 0 cell which encompasses all spatial data. Equivalent to {@link #getCell(String)} with "".
* This cell is threadsafe, just like a spatial prefix grid is, although cells aren't
* generally threadsafe.
* TODO rename to getTopCell or is this fine?
* Returns the level 0 cell which encompasses all spatial data. Equivalent to {@link #getCell(byte[], int, int)} with
* no bytes. This cell is thread-safe, just like a spatial prefix grid is, although cells aren't
* generally thread-safe.
*/
public Cell getWorldCell() {
public Cell getWorldCell() {//another possible name: getTopCell
if (worldCell == null) {
worldCell = getCell("");
worldCell = getCell(BytesRef.EMPTY_BYTES, 0, 0);
worldCell.getShape();//lazy load; make thread-safe
}
return worldCell;
}
@ -119,8 +121,6 @@ public abstract class SpatialPrefixTree {
* The cell for the specified token. The empty string should be equal to {@link #getWorldCell()}.
* Precondition: Never called when token length > maxLevel.
*/
public abstract Cell getCell(String token);
public abstract Cell getCell(byte[] bytes, int offset, int len);
public final Cell getCell(byte[] bytes, int offset, int len, Cell target) {
@ -215,40 +215,23 @@ public abstract class SpatialPrefixTree {
* A Point-optimized implementation of
* {@link #getCells(com.spatial4j.core.shape.Shape, int, boolean, boolean)}. That
* method in facts calls this for points.
* <p/>
* This implementation depends on {@link #getCell(String)} being fast, as its
* called repeatedly when incPlarents is true.
*/
public List<Cell> getCells(Point p, int detailLevel, boolean inclParents) {
Cell cell = getCell(p, detailLevel);
if (!inclParents) {
assert !cell.isLeaf();
if (!inclParents || detailLevel == 1) {
return Collections.singletonList(cell);
}
String endToken = cell.getTokenString();
assert endToken.length() == detailLevel;
List<Cell> cells = new ArrayList<>(detailLevel);
for (int i = 1; i < detailLevel; i++) {
cells.add(getCell(endToken.substring(0, i)));//TODO refactor: add a cell.getParent()
//fill in reverse order to be sorted
Cell[] cells = new Cell[detailLevel];
for (int i = detailLevel-1; true; i--) {
cells[i] = cell;
if (i == 0)
break;
cell = cell.getParent();
}
cells.add(cell);
return cells;
return Arrays.asList(cells);
}
/**
* Will add the trailing leaf byte for leaves. This isn't particularly efficient.
* @deprecated TODO remove; not used and not interesting, don't need collection in & out
*/
public static List<String> cellsToTokenStrings(Collection<Cell> cells) {
List<String> tokens = new ArrayList<>((cells.size()));
for (Cell cell : cells) {
final String token = cell.getTokenString();
if (cell.isLeaf()) {
tokens.add(token + (char) Cell.LEAF_BYTE);
} else {
tokens.add(token);
}
}
return tokens;
}
}

View File

@ -35,6 +35,9 @@ import org.apache.lucene.spatial.query.SpatialOperation;
import org.junit.Before;
import org.junit.Test;
import java.util.ArrayList;
import java.util.List;
public class SpatialPrefixTreeTest extends SpatialTestCase {
//TODO plug in others and test them
@ -56,9 +59,10 @@ public class SpatialPrefixTreeTest extends SpatialTestCase {
Cell c = trie.getWorldCell();
assertEquals(0, c.getLevel());
assertEquals(ctx.getWorldBounds(), c.getShape());
while(c.getLevel() < trie.getMaxLevels()) {
while (c.getLevel() < trie.getMaxLevels()) {
prevC = c;
c = c.getSubCells().iterator().next();//TODO random which one?
List<Cell> subCells = new ArrayList<>(c.getSubCells());
c = subCells.get(random().nextInt(subCells.size()-1));
assertEquals(prevC.getLevel()+1,c.getLevel());
Rectangle prevNShape = (Rectangle) prevC.getShape();

Some files were not shown because too many files have changed in this diff Show More