diff --git .gitignore .gitignore index 443dd2f..da6c362 100644 --- .gitignore +++ .gitignore @@ -40,7 +40,6 @@ /solr/example/example-DIH/**/dataimport.properties /solr/example/example-DIH/solr/mail/lib/*.jar -solr/contrib/dataimporthandler/test-lib/ -solr/contrib/morphlines-core/test-lib/ +test-lib/ -solr/core/test-lib/ +compile.log diff --git lucene/core/src/java/org/apache/lucene/search/CachingCollector.java lucene/core/src/java/org/apache/lucene/search/CachingCollector.java index c5957d8..fbaff17 100644 --- lucene/core/src/java/org/apache/lucene/search/CachingCollector.java +++ lucene/core/src/java/org/apache/lucene/search/CachingCollector.java @@ -161,7 +161,9 @@ public abstract class CachingCollector extends FilterCollector { + "Therefore cached documents may be out-of-order."); } collect(collector, i); + collector.leafDone(); } + other.done(); } } diff --git lucene/core/src/java/org/apache/lucene/search/Collector.java lucene/core/src/java/org/apache/lucene/search/Collector.java index bb47394..78535d9 100644 --- lucene/core/src/java/org/apache/lucene/search/Collector.java +++ lucene/core/src/java/org/apache/lucene/search/Collector.java @@ -73,4 +73,11 @@ public interface Collector { */ LeafCollector getLeafCollector(AtomicReaderContext context) throws IOException; + /** + * Advise that collection is complete. + * + * @throws IOException + */ + void done() throws IOException; + } diff --git lucene/core/src/java/org/apache/lucene/search/FilterCollector.java lucene/core/src/java/org/apache/lucene/search/FilterCollector.java index 247bb03..e290170 100644 --- lucene/core/src/java/org/apache/lucene/search/FilterCollector.java +++ lucene/core/src/java/org/apache/lucene/search/FilterCollector.java @@ -41,6 +41,11 @@ public class FilterCollector implements Collector { } @Override + public void done() throws IOException { + in.done(); + } + + @Override public String toString() { return getClass().getSimpleName() + "(" + in + ")"; } diff --git lucene/core/src/java/org/apache/lucene/search/FilterLeafCollector.java lucene/core/src/java/org/apache/lucene/search/FilterLeafCollector.java index e3ae9a8..6b5aac8 100644 --- lucene/core/src/java/org/apache/lucene/search/FilterLeafCollector.java +++ lucene/core/src/java/org/apache/lucene/search/FilterLeafCollector.java @@ -49,6 +49,11 @@ public class FilterLeafCollector implements LeafCollector { } @Override + public void leafDone() throws IOException { + in.leafDone(); + } + + @Override public String toString() { return getClass().getSimpleName() + "(" + in + ")"; } diff --git lucene/core/src/java/org/apache/lucene/search/IndexSearcher.java lucene/core/src/java/org/apache/lucene/search/IndexSearcher.java index 8f1a5f6..471bc28 100644 --- lucene/core/src/java/org/apache/lucene/search/IndexSearcher.java +++ lucene/core/src/java/org/apache/lucene/search/IndexSearcher.java @@ -546,6 +546,8 @@ public class IndexSearcher { } } + topCollector.done(); + final TopFieldDocs topDocs = (TopFieldDocs) topCollector.topDocs(); return new TopFieldDocs(totalHits, topDocs.scoreDocs, topDocs.fields, topDocs.getMaxScore()); @@ -617,7 +619,10 @@ public class IndexSearcher { // continue with the following leaf } } + leafCollector.leafDone(); } + + collector.done(); } /** Expert: called to re-write queries into primitive queries. @@ -787,6 +792,7 @@ public class IndexSearcher { fakeScorer.score = scoreDoc.score; collector.collect(scoreDoc.doc-base); } + collector.leafDone(); // Carry over maxScore from sub: if (doMaxScore && docs.getMaxScore() > hq.maxScore) { diff --git lucene/core/src/java/org/apache/lucene/search/LeafCollector.java lucene/core/src/java/org/apache/lucene/search/LeafCollector.java index 562e76d..8ed84bd 100644 --- lucene/core/src/java/org/apache/lucene/search/LeafCollector.java +++ lucene/core/src/java/org/apache/lucene/search/LeafCollector.java @@ -118,4 +118,9 @@ public interface LeafCollector { */ boolean acceptsDocsOutOfOrder(); + /** + * Advise that collection on this leaf has completed. + */ + void leafDone() throws IOException; + } diff --git lucene/core/src/java/org/apache/lucene/search/MultiCollector.java lucene/core/src/java/org/apache/lucene/search/MultiCollector.java index 859b893..888950c 100644 --- lucene/core/src/java/org/apache/lucene/search/MultiCollector.java +++ lucene/core/src/java/org/apache/lucene/search/MultiCollector.java @@ -21,8 +21,6 @@ import java.io.IOException; import java.util.Arrays; import org.apache.lucene.index.AtomicReaderContext; -import org.apache.lucene.search.Collector; -import org.apache.lucene.search.Scorer; /** * A {@link Collector} which allows running a search with several @@ -103,6 +101,13 @@ public class MultiCollector implements Collector { return new MultiLeafCollector(leafCollectors); } + @Override + public void done() throws IOException { + for (Collector c: collectors) { + c.done(); + } + } + private static class MultiLeafCollector implements LeafCollector { @@ -136,6 +141,13 @@ public class MultiCollector implements Collector { return true; } + @Override + public void leafDone() throws IOException { + for (LeafCollector c : collectors) { + c.leafDone(); + } + } + } } diff --git lucene/core/src/java/org/apache/lucene/search/SimpleCollector.java lucene/core/src/java/org/apache/lucene/search/SimpleCollector.java index 5803b2e..6f47c87 100644 --- lucene/core/src/java/org/apache/lucene/search/SimpleCollector.java +++ lucene/core/src/java/org/apache/lucene/search/SimpleCollector.java @@ -50,4 +50,14 @@ public abstract class SimpleCollector implements Collector, LeafCollector { @Override public abstract void collect(int doc) throws IOException; + @Override + public void leafDone() throws IOException { + // no-op by default + } + + @Override + public void done() throws IOException { + // no-op by default + } + } diff --git lucene/core/src/java/org/apache/lucene/search/SortRescorer.java lucene/core/src/java/org/apache/lucene/search/SortRescorer.java index 6f125e8..5406aa2 100644 --- lucene/core/src/java/org/apache/lucene/search/SortRescorer.java +++ lucene/core/src/java/org/apache/lucene/search/SortRescorer.java @@ -63,6 +63,7 @@ public class SortRescorer extends Rescorer { FakeScorer fakeScorer = new FakeScorer(); + LeafCollector leafCollector = null; while (hitUpto < hits.length) { ScoreDoc hit = hits[hitUpto]; int docID = hit.doc; @@ -75,19 +76,28 @@ public class SortRescorer extends Rescorer { if (readerContext != null) { // We advanced to another segment: - collector.getLeafCollector(readerContext); - collector.setScorer(fakeScorer); + if (leafCollector != null) { + leafCollector.leafDone(); + } + leafCollector = collector.getLeafCollector(readerContext); + leafCollector.setScorer(fakeScorer); docBase = readerContext.docBase; } fakeScorer.score = hit.score; fakeScorer.doc = docID - docBase; - collector.collect(fakeScorer.doc); + leafCollector.collect(fakeScorer.doc); hitUpto++; } + if (leafCollector != null) { + leafCollector.leafDone(); + } + + collector.done(); + return collector.topDocs(); } diff --git lucene/core/src/java/org/apache/lucene/search/TimeLimitingCollector.java lucene/core/src/java/org/apache/lucene/search/TimeLimitingCollector.java index 9a08a2b..71cdbc6 100644 --- lucene/core/src/java/org/apache/lucene/search/TimeLimitingCollector.java +++ lucene/core/src/java/org/apache/lucene/search/TimeLimitingCollector.java @@ -156,7 +156,12 @@ public class TimeLimitingCollector implements Collector { }; } - + + @Override + public void done() throws IOException { + collector.done(); + } + /** * This is so the same timer can be used with a multi-phase search process such as grouping. * We don't want to create a new TimeLimitingCollector for each phase because that would diff --git lucene/core/src/test/org/apache/lucene/search/MultiCollectorTest.java lucene/core/src/test/org/apache/lucene/search/MultiCollectorTest.java index 5a7df3c..9ded428 100644 --- lucene/core/src/test/org/apache/lucene/search/MultiCollectorTest.java +++ lucene/core/src/test/org/apache/lucene/search/MultiCollectorTest.java @@ -20,8 +20,6 @@ package org.apache.lucene.search; import java.io.IOException; import org.apache.lucene.index.AtomicReaderContext; -import org.apache.lucene.search.Collector; -import org.apache.lucene.search.Scorer; import org.apache.lucene.util.LuceneTestCase; import org.junit.Test; @@ -33,6 +31,8 @@ public class MultiCollectorTest extends LuceneTestCase { boolean collectCalled = false; boolean setNextReaderCalled = false; boolean setScorerCalled = false; + boolean leafDoneCalled = false; + boolean doneCalled = false; @Override public boolean acceptsDocsOutOfOrder() { @@ -55,6 +55,16 @@ public class MultiCollectorTest extends LuceneTestCase { setScorerCalled = true; } + @Override + public void leafDone() throws IOException { + leafDoneCalled = true; + } + + @Override + public void done() throws IOException { + doneCalled = true; + } + } @Test @@ -97,14 +107,19 @@ public class MultiCollectorTest extends LuceneTestCase { LeafCollector ac = c.getLeafCollector(null); assertTrue(ac.acceptsDocsOutOfOrder()); ac.collect(1); + ac.leafDone(); ac = c.getLeafCollector(null); ac.setScorer(null); + ac.leafDone(); + c.done(); for (DummyCollector dc : dcs) { assertTrue(dc.acceptsDocsOutOfOrderCalled); assertTrue(dc.collectCalled); assertTrue(dc.setNextReaderCalled); assertTrue(dc.setScorerCalled); + assertTrue(dc.leafDoneCalled); + assertTrue(dc.doneCalled); } } diff --git lucene/core/src/test/org/apache/lucene/search/TestCachingCollector.java lucene/core/src/test/org/apache/lucene/search/TestCachingCollector.java index e842909..18bc79e 100644 --- lucene/core/src/test/org/apache/lucene/search/TestCachingCollector.java +++ lucene/core/src/test/org/apache/lucene/search/TestCachingCollector.java @@ -81,6 +81,9 @@ public class TestCachingCollector extends LuceneTestCase { acc.collect(i); } + acc.leafDone(); + cc.done(); + // now replay them cc.replay(new SimpleCollector() { int prevDocID = -1; @@ -108,6 +111,9 @@ public class TestCachingCollector extends LuceneTestCase { for (int i = 0; i < 130; i++) { acc.collect(i); } + + acc.leafDone(); + cc.done(); assertFalse("CachingCollector should not be cached due to low memory limit", cc.isCached()); @@ -128,6 +134,7 @@ public class TestCachingCollector extends LuceneTestCase { LeafCollector acc = cc.getLeafCollector(null); acc.setScorer(new MockScorer()); for (int i = 0; i < 10; i++) acc.collect(i); + cc.done(); cc.replay(new NoOpCollector(true)); // this call should not fail cc.replay(new NoOpCollector(false)); // this call should not fail @@ -136,6 +143,8 @@ public class TestCachingCollector extends LuceneTestCase { acc = cc.getLeafCollector(null); acc.setScorer(new MockScorer()); for (int i = 0; i < 10; i++) acc.collect(i); + acc.leafDone(); + cc.done(); cc.replay(new NoOpCollector(true)); // this call should not fail try { cc.replay(new NoOpCollector(false)); // this call should fail @@ -164,6 +173,9 @@ public class TestCachingCollector extends LuceneTestCase { // The 151's document should terminate caching acc.collect(numDocs); assertFalse(cc.isCached()); + + acc.leafDone(); + cc.done(); } } @@ -174,8 +186,10 @@ public class TestCachingCollector extends LuceneTestCase { LeafCollector acc = cc.getLeafCollector(null); acc.setScorer(new MockScorer()); acc.collect(0); + acc.leafDone(); assertTrue(cc.isCached()); + cc.done(); cc.replay(new NoOpCollector(true)); } } diff --git lucene/core/src/test/org/apache/lucene/search/TestPositiveScoresOnlyCollector.java lucene/core/src/test/org/apache/lucene/search/TestPositiveScoresOnlyCollector.java index c856c69..529f38d 100644 --- lucene/core/src/test/org/apache/lucene/search/TestPositiveScoresOnlyCollector.java +++ lucene/core/src/test/org/apache/lucene/search/TestPositiveScoresOnlyCollector.java @@ -93,6 +93,8 @@ public class TestPositiveScoresOnlyCollector extends LuceneTestCase { while (s.nextDoc() != DocIdSetIterator.NO_MORE_DOCS) { ac.collect(0); } + ac.leafDone(); + c.done(); TopDocs td = tdc.topDocs(); ScoreDoc[] sd = td.scoreDocs; assertEquals(numPositiveScores, td.totalHits); diff --git lucene/facet/src/java/org/apache/lucene/facet/DrillSideways.java lucene/facet/src/java/org/apache/lucene/facet/DrillSideways.java index c8614f3..44cc1ac 100644 --- lucene/facet/src/java/org/apache/lucene/facet/DrillSideways.java +++ lucene/facet/src/java/org/apache/lucene/facet/DrillSideways.java @@ -175,6 +175,11 @@ public class DrillSideways { DrillSidewaysQuery dsq = new DrillSidewaysQuery(baseQuery, drillDownCollector, drillSidewaysCollectors, drillDownQueries, scoreSubDocsAtOnce()); searcher.search(dsq, hitCollector); + drillDownCollector.done(); + for (Collector c: drillSidewaysCollectors) { + c.done(); + } + return new DrillSidewaysResult(buildFacetsResult(drillDownCollector, drillSidewaysCollectors, drillDownDims.keySet().toArray(new String[drillDownDims.size()])), null); } diff --git lucene/facet/src/java/org/apache/lucene/facet/DrillSidewaysScorer.java lucene/facet/src/java/org/apache/lucene/facet/DrillSidewaysScorer.java index 273b6b1..9269c37 100644 --- lucene/facet/src/java/org/apache/lucene/facet/DrillSidewaysScorer.java +++ lucene/facet/src/java/org/apache/lucene/facet/DrillSidewaysScorer.java @@ -150,6 +150,13 @@ class DrillSidewaysScorer extends BulkScorer { doUnionScoring(collector, disis, sidewaysCollectors); } + if (drillDownLeafCollector != null) { + drillDownLeafCollector.leafDone(); + } + for (DocsAndCost dim : dims) { + dim.sidewaysLeafCollector.leafDone(); + } + return false; } diff --git lucene/grouping/src/java/org/apache/lucene/search/grouping/AbstractSecondPassGroupingCollector.java lucene/grouping/src/java/org/apache/lucene/search/grouping/AbstractSecondPassGroupingCollector.java index aedfa9e..1841e78 100644 --- lucene/grouping/src/java/org/apache/lucene/search/grouping/AbstractSecondPassGroupingCollector.java +++ lucene/grouping/src/java/org/apache/lucene/search/grouping/AbstractSecondPassGroupingCollector.java @@ -115,6 +115,21 @@ public abstract class AbstractSecondPassGroupingCollector exte } @Override + public void leafDone() throws IOException { + for (SearchGroupDocs group : groupMap.values()) { + group.collector.leafDone(); + } + } + + + @Override + public void done() throws IOException { + for (SearchGroupDocs group : groupMap.values()) { + group.collector.done(); + } + } + + @Override public boolean acceptsDocsOutOfOrder() { return false; } @@ -143,7 +158,6 @@ public abstract class AbstractSecondPassGroupingCollector exte maxScore); } - // TODO: merge with SearchGroup or not? // ad: don't need to build a new hashmap // disad: blows up the size of SearchGroup if we need many of them, and couples implementations diff --git lucene/grouping/src/java/org/apache/lucene/search/grouping/BlockGroupingCollector.java lucene/grouping/src/java/org/apache/lucene/search/grouping/BlockGroupingCollector.java index 7c33583..3910a3f 100644 --- lucene/grouping/src/java/org/apache/lucene/search/grouping/BlockGroupingCollector.java +++ lucene/grouping/src/java/org/apache/lucene/search/grouping/BlockGroupingCollector.java @@ -350,15 +350,18 @@ public class BlockGroupingCollector extends SimpleCollector { } collector.setScorer(fakeScorer); - collector.getLeafCollector(og.readerContext); + final LeafCollector leafCollector = collector.getLeafCollector(og.readerContext); for(int docIDX=0;docIDX> iter = leafCollectors.iterator(); + iter.hasNext(); ) { + iter.next().value.leafDone(); + } + } }; } + @Override + public void done() throws IOException { + for (final Iterator> iter = groups.values().iterator(); + iter.hasNext(); ) { + iter.next().value.done(); + } + } + public IntObjectMap getGroups() { return groups; } diff --git solr/core/src/java/org/apache/solr/search/CollapsingQParserPlugin.java solr/core/src/java/org/apache/solr/search/CollapsingQParserPlugin.java index 367a6ca..6b98d3c 100644 --- solr/core/src/java/org/apache/solr/search/CollapsingQParserPlugin.java +++ solr/core/src/java/org/apache/solr/search/CollapsingQParserPlugin.java @@ -34,15 +34,11 @@ import org.apache.lucene.index.TermsEnum; import org.apache.lucene.queries.function.FunctionQuery; 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.Collector; import org.apache.lucene.search.DocIdSetIterator; import org.apache.lucene.search.FieldCache; -import org.apache.lucene.search.FilterCollector; import org.apache.lucene.search.IndexSearcher; import org.apache.lucene.search.Query; import org.apache.lucene.search.Scorer; -import org.apache.lucene.search.TopFieldCollector; import org.apache.lucene.util.Bits; import org.apache.lucene.util.BytesRef; import org.apache.lucene.util.FixedBitSet; @@ -574,6 +570,7 @@ public class CollapsingQParserPlugin extends QParserPlugin { currentContext++; currentDocBase = contexts[currentContext].docBase; nextDocBase = currentContext+1 < contexts.length ? contexts[currentContext+1].docBase : maxDoc; + leafDelegate.leafDone(); leafDelegate = delegate.getLeafCollector(contexts[currentContext]); leafDelegate.setScorer(dummy); } @@ -583,6 +580,10 @@ public class CollapsingQParserPlugin extends QParserPlugin { leafDelegate.collect(contextDoc); } + leafDelegate.leafDone(); + + delegate.done(); + if(delegate instanceof DelegatingCollector) { ((DelegatingCollector) delegate).finish(); } @@ -691,6 +692,7 @@ public class CollapsingQParserPlugin extends QParserPlugin { currentContext++; currentDocBase = contexts[currentContext].docBase; nextDocBase = currentContext+1 < contexts.length ? contexts[currentContext+1].docBase : maxDoc; + leafDelegate.leafDone(); leafDelegate = delegate.getLeafCollector(contexts[currentContext]); leafDelegate.setScorer(dummy); } @@ -700,6 +702,10 @@ public class CollapsingQParserPlugin extends QParserPlugin { leafDelegate.collect(contextDoc); } + leafDelegate.leafDone(); + + delegate.done(); + if(delegate instanceof DelegatingCollector) { ((DelegatingCollector) delegate).finish(); } diff --git solr/core/src/java/org/apache/solr/search/DelegatingCollector.java solr/core/src/java/org/apache/solr/search/DelegatingCollector.java index 06b9658..7d7950c 100644 --- solr/core/src/java/org/apache/solr/search/DelegatingCollector.java +++ solr/core/src/java/org/apache/solr/search/DelegatingCollector.java @@ -69,6 +69,13 @@ public class DelegatingCollector extends SimpleCollector { } @Override + public void leafDone() throws IOException { + if (leafDelegate != null) { + leafDelegate.leafDone(); + } + } + + @Override protected void doSetNextReader(AtomicReaderContext context) throws IOException { this.context = context; this.docBase = context.docBase; @@ -80,6 +87,11 @@ public class DelegatingCollector extends SimpleCollector { return leafDelegate.acceptsDocsOutOfOrder(); } + @Override + public void done() throws IOException { + delegate.done(); + } + public void finish() throws IOException { if(delegate instanceof DelegatingCollector) { ((DelegatingCollector) delegate).finish(); diff --git solr/core/src/java/org/apache/solr/search/SolrIndexSearcher.java solr/core/src/java/org/apache/solr/search/SolrIndexSearcher.java index 176c0df..c2ae56e 100644 --- solr/core/src/java/org/apache/solr/search/SolrIndexSearcher.java +++ solr/core/src/java/org/apache/solr/search/SolrIndexSearcher.java @@ -946,8 +946,12 @@ public class SolrIndexSearcher extends IndexSearcher implements Closeable,SolrIn leafCollector.collect(docid); } } + + leafCollector.leafDone(); } + collector.done(); + if(collector instanceof DelegatingCollector) { ((DelegatingCollector) collector).finish(); } @@ -2022,17 +2026,26 @@ public class SolrIndexSearcher extends IndexSearcher implements Closeable,SolrIn int end=0; int readerIndex = 0; + LeafCollector leafCollector = null; while (iter.hasNext()) { int doc = iter.nextDoc(); while (doc>=end) { AtomicReaderContext leaf = leafContexts.get(readerIndex++); base = leaf.docBase; end = base + leaf.reader().maxDoc(); - topCollector.getLeafCollector(leaf); + if (leafCollector != null) { + leafCollector.leafDone(); + } + leafCollector = topCollector.getLeafCollector(leaf); // we should never need to set the scorer given the settings for the collector } topCollector.collect(doc-base); } + if (leafCollector != null) { + leafCollector.leafDone(); + } + + topCollector.done(); TopDocs topDocs = topCollector.topDocs(0, nDocs);