You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@lucene.apache.org by dw...@apache.org on 2014/04/05 11:03:00 UTC
svn commit: r1585024 [1/4] - in /lucene/dev/branches/solr5914: ./ dev-tools/
dev-tools/idea/.idea/libraries/ lucene/ lucene/core/
lucene/core/src/java/org/apache/lucene/index/
lucene/core/src/java/org/apache/lucene/search/
lucene/core/src/java/org/apac...
Author: dweiss
Date: Sat Apr 5 09:02:57 2014
New Revision: 1585024
URL: http://svn.apache.org/r1585024
Log:
Merging with trunk.
Added:
lucene/dev/branches/solr5914/lucene/core/src/java/org/apache/lucene/search/FilterCollector.java
- copied unchanged from r1585022, lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/search/FilterCollector.java
lucene/dev/branches/solr5914/lucene/core/src/java/org/apache/lucene/search/FilterLeafCollector.java
- copied unchanged from r1585022, lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/search/FilterLeafCollector.java
lucene/dev/branches/solr5914/lucene/core/src/java/org/apache/lucene/search/LeafCollector.java
- copied unchanged from r1585022, lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/search/LeafCollector.java
lucene/dev/branches/solr5914/lucene/core/src/java/org/apache/lucene/search/SimpleCollector.java
- copied unchanged from r1585022, lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/search/SimpleCollector.java
lucene/dev/branches/solr5914/lucene/spatial/src/java/org/apache/lucene/spatial/prefix/CellTokenStream.java
- copied unchanged from r1585022, lucene/dev/trunk/lucene/spatial/src/java/org/apache/lucene/spatial/prefix/CellTokenStream.java
lucene/dev/branches/solr5914/lucene/test-framework/src/java/org/apache/lucene/search/RandomOrderCollector.java
- copied unchanged from r1585022, lucene/dev/trunk/lucene/test-framework/src/java/org/apache/lucene/search/RandomOrderCollector.java
Removed:
lucene/dev/branches/solr5914/solr/core/src/java/org/apache/solr/search/DocSetDelegateCollector.java
Modified:
lucene/dev/branches/solr5914/ (props changed)
lucene/dev/branches/solr5914/dev-tools/ (props changed)
lucene/dev/branches/solr5914/dev-tools/idea/.idea/libraries/JUnit.xml
lucene/dev/branches/solr5914/lucene/ (props changed)
lucene/dev/branches/solr5914/lucene/CHANGES.txt (contents, props changed)
lucene/dev/branches/solr5914/lucene/MIGRATE.txt (contents, props changed)
lucene/dev/branches/solr5914/lucene/core/ (props changed)
lucene/dev/branches/solr5914/lucene/core/src/java/org/apache/lucene/index/IndexFileDeleter.java
lucene/dev/branches/solr5914/lucene/core/src/java/org/apache/lucene/index/IndexWriter.java
lucene/dev/branches/solr5914/lucene/core/src/java/org/apache/lucene/index/StandardDirectoryReader.java
lucene/dev/branches/solr5914/lucene/core/src/java/org/apache/lucene/search/BooleanScorer.java
lucene/dev/branches/solr5914/lucene/core/src/java/org/apache/lucene/search/BulkScorer.java
lucene/dev/branches/solr5914/lucene/core/src/java/org/apache/lucene/search/CachingCollector.java
lucene/dev/branches/solr5914/lucene/core/src/java/org/apache/lucene/search/CollectionTerminatedException.java
lucene/dev/branches/solr5914/lucene/core/src/java/org/apache/lucene/search/Collector.java
lucene/dev/branches/solr5914/lucene/core/src/java/org/apache/lucene/search/ConstantScoreQuery.java
lucene/dev/branches/solr5914/lucene/core/src/java/org/apache/lucene/search/FakeScorer.java
lucene/dev/branches/solr5914/lucene/core/src/java/org/apache/lucene/search/FilteredQuery.java
lucene/dev/branches/solr5914/lucene/core/src/java/org/apache/lucene/search/IndexSearcher.java
lucene/dev/branches/solr5914/lucene/core/src/java/org/apache/lucene/search/MultiCollector.java
lucene/dev/branches/solr5914/lucene/core/src/java/org/apache/lucene/search/PositiveScoresOnlyCollector.java
lucene/dev/branches/solr5914/lucene/core/src/java/org/apache/lucene/search/ScoreCachingWrappingScorer.java
lucene/dev/branches/solr5914/lucene/core/src/java/org/apache/lucene/search/Scorer.java
lucene/dev/branches/solr5914/lucene/core/src/java/org/apache/lucene/search/SortRescorer.java
lucene/dev/branches/solr5914/lucene/core/src/java/org/apache/lucene/search/TimeLimitingCollector.java
lucene/dev/branches/solr5914/lucene/core/src/java/org/apache/lucene/search/TopDocsCollector.java
lucene/dev/branches/solr5914/lucene/core/src/java/org/apache/lucene/search/TopFieldCollector.java
lucene/dev/branches/solr5914/lucene/core/src/java/org/apache/lucene/search/TopScoreDocCollector.java
lucene/dev/branches/solr5914/lucene/core/src/java/org/apache/lucene/search/TotalHitCountCollector.java
lucene/dev/branches/solr5914/lucene/core/src/java/org/apache/lucene/search/Weight.java
lucene/dev/branches/solr5914/lucene/core/src/java/org/apache/lucene/search/package.html
lucene/dev/branches/solr5914/lucene/core/src/java/org/apache/lucene/store/FSDirectory.java
lucene/dev/branches/solr5914/lucene/core/src/test/org/apache/lucene/index/TestIndexWriter.java
lucene/dev/branches/solr5914/lucene/core/src/test/org/apache/lucene/index/TestOmitTf.java
lucene/dev/branches/solr5914/lucene/core/src/test/org/apache/lucene/search/JustCompileSearch.java
lucene/dev/branches/solr5914/lucene/core/src/test/org/apache/lucene/search/MultiCollectorTest.java
lucene/dev/branches/solr5914/lucene/core/src/test/org/apache/lucene/search/TestBooleanOr.java
lucene/dev/branches/solr5914/lucene/core/src/test/org/apache/lucene/search/TestBooleanQueryVisitSubscorers.java
lucene/dev/branches/solr5914/lucene/core/src/test/org/apache/lucene/search/TestBooleanScorer.java
lucene/dev/branches/solr5914/lucene/core/src/test/org/apache/lucene/search/TestCachingCollector.java
lucene/dev/branches/solr5914/lucene/core/src/test/org/apache/lucene/search/TestConstantScoreQuery.java
lucene/dev/branches/solr5914/lucene/core/src/test/org/apache/lucene/search/TestDocBoost.java
lucene/dev/branches/solr5914/lucene/core/src/test/org/apache/lucene/search/TestEarlyTermination.java
lucene/dev/branches/solr5914/lucene/core/src/test/org/apache/lucene/search/TestMultiTermConstantScore.java
lucene/dev/branches/solr5914/lucene/core/src/test/org/apache/lucene/search/TestPositiveScoresOnlyCollector.java
lucene/dev/branches/solr5914/lucene/core/src/test/org/apache/lucene/search/TestScoreCachingWrappingScorer.java
lucene/dev/branches/solr5914/lucene/core/src/test/org/apache/lucene/search/TestScorerPerf.java
lucene/dev/branches/solr5914/lucene/core/src/test/org/apache/lucene/search/TestSimilarity.java
lucene/dev/branches/solr5914/lucene/core/src/test/org/apache/lucene/search/TestSloppyPhraseQuery.java
lucene/dev/branches/solr5914/lucene/core/src/test/org/apache/lucene/search/TestSubScorerFreqs.java
lucene/dev/branches/solr5914/lucene/core/src/test/org/apache/lucene/search/TestTermScorer.java
lucene/dev/branches/solr5914/lucene/core/src/test/org/apache/lucene/search/TestTimeLimitingCollector.java
lucene/dev/branches/solr5914/lucene/core/src/test/org/apache/lucene/search/TestTopDocsCollector.java
lucene/dev/branches/solr5914/lucene/core/src/test/org/apache/lucene/store/TestDirectory.java
lucene/dev/branches/solr5914/lucene/core/src/test/org/apache/lucene/util/junitcompat/TestFailIfUnreferencedFiles.java
lucene/dev/branches/solr5914/lucene/facet/ (props changed)
lucene/dev/branches/solr5914/lucene/facet/src/java/org/apache/lucene/facet/DrillSideways.java
lucene/dev/branches/solr5914/lucene/facet/src/java/org/apache/lucene/facet/DrillSidewaysScorer.java
lucene/dev/branches/solr5914/lucene/facet/src/java/org/apache/lucene/facet/FacetsCollector.java
lucene/dev/branches/solr5914/lucene/facet/src/test/org/apache/lucene/facet/AssertingSubDocsAtOnceCollector.java
lucene/dev/branches/solr5914/lucene/facet/src/test/org/apache/lucene/facet/TestDrillSideways.java
lucene/dev/branches/solr5914/lucene/grouping/ (props changed)
lucene/dev/branches/solr5914/lucene/grouping/src/java/org/apache/lucene/search/grouping/AbstractAllGroupHeadsCollector.java
lucene/dev/branches/solr5914/lucene/grouping/src/java/org/apache/lucene/search/grouping/AbstractAllGroupsCollector.java
lucene/dev/branches/solr5914/lucene/grouping/src/java/org/apache/lucene/search/grouping/AbstractDistinctValuesCollector.java
lucene/dev/branches/solr5914/lucene/grouping/src/java/org/apache/lucene/search/grouping/AbstractFirstPassGroupingCollector.java
lucene/dev/branches/solr5914/lucene/grouping/src/java/org/apache/lucene/search/grouping/AbstractGroupFacetCollector.java
lucene/dev/branches/solr5914/lucene/grouping/src/java/org/apache/lucene/search/grouping/AbstractSecondPassGroupingCollector.java
lucene/dev/branches/solr5914/lucene/grouping/src/java/org/apache/lucene/search/grouping/BlockGroupingCollector.java
lucene/dev/branches/solr5914/lucene/grouping/src/java/org/apache/lucene/search/grouping/function/FunctionAllGroupHeadsCollector.java
lucene/dev/branches/solr5914/lucene/grouping/src/java/org/apache/lucene/search/grouping/function/FunctionAllGroupsCollector.java
lucene/dev/branches/solr5914/lucene/grouping/src/java/org/apache/lucene/search/grouping/function/FunctionDistinctValuesCollector.java
lucene/dev/branches/solr5914/lucene/grouping/src/java/org/apache/lucene/search/grouping/function/FunctionFirstPassGroupingCollector.java
lucene/dev/branches/solr5914/lucene/grouping/src/java/org/apache/lucene/search/grouping/function/FunctionSecondPassGroupingCollector.java
lucene/dev/branches/solr5914/lucene/grouping/src/java/org/apache/lucene/search/grouping/term/TermAllGroupHeadsCollector.java
lucene/dev/branches/solr5914/lucene/grouping/src/java/org/apache/lucene/search/grouping/term/TermAllGroupsCollector.java
lucene/dev/branches/solr5914/lucene/grouping/src/java/org/apache/lucene/search/grouping/term/TermDistinctValuesCollector.java
lucene/dev/branches/solr5914/lucene/grouping/src/java/org/apache/lucene/search/grouping/term/TermFirstPassGroupingCollector.java
lucene/dev/branches/solr5914/lucene/grouping/src/java/org/apache/lucene/search/grouping/term/TermGroupFacetCollector.java
lucene/dev/branches/solr5914/lucene/grouping/src/java/org/apache/lucene/search/grouping/term/TermSecondPassGroupingCollector.java
lucene/dev/branches/solr5914/lucene/highlighter/ (props changed)
lucene/dev/branches/solr5914/lucene/highlighter/src/test/org/apache/lucene/search/highlight/HighlighterPhraseTest.java
lucene/dev/branches/solr5914/lucene/join/ (props changed)
lucene/dev/branches/solr5914/lucene/join/src/java/org/apache/lucene/search/join/FakeScorer.java
lucene/dev/branches/solr5914/lucene/join/src/java/org/apache/lucene/search/join/TermsCollector.java
lucene/dev/branches/solr5914/lucene/join/src/java/org/apache/lucene/search/join/TermsIncludingScoreQuery.java
lucene/dev/branches/solr5914/lucene/join/src/java/org/apache/lucene/search/join/TermsWithScoreCollector.java
lucene/dev/branches/solr5914/lucene/join/src/java/org/apache/lucene/search/join/ToParentBlockJoinCollector.java
lucene/dev/branches/solr5914/lucene/join/src/test/org/apache/lucene/search/join/TestJoinUtil.java
lucene/dev/branches/solr5914/lucene/memory/ (props changed)
lucene/dev/branches/solr5914/lucene/memory/src/java/org/apache/lucene/index/memory/MemoryIndex.java
lucene/dev/branches/solr5914/lucene/misc/ (props changed)
lucene/dev/branches/solr5914/lucene/misc/src/java/org/apache/lucene/index/sorter/EarlyTerminatingSortingCollector.java
lucene/dev/branches/solr5914/lucene/misc/src/test/org/apache/lucene/index/sorter/TestEarlyTermination.java
lucene/dev/branches/solr5914/lucene/queryparser/ (props changed)
lucene/dev/branches/solr5914/lucene/queryparser/src/test/org/apache/lucene/queryparser/surround/query/BooleanQueryTst.java
lucene/dev/branches/solr5914/lucene/spatial/ (props changed)
lucene/dev/branches/solr5914/lucene/spatial/src/java/org/apache/lucene/spatial/prefix/AbstractVisitingPrefixTreeFilter.java
lucene/dev/branches/solr5914/lucene/spatial/src/java/org/apache/lucene/spatial/prefix/ContainsPrefixTreeFilter.java
lucene/dev/branches/solr5914/lucene/spatial/src/java/org/apache/lucene/spatial/prefix/PointPrefixTreeFieldCacheProvider.java
lucene/dev/branches/solr5914/lucene/spatial/src/java/org/apache/lucene/spatial/prefix/PrefixTreeStrategy.java
lucene/dev/branches/solr5914/lucene/spatial/src/java/org/apache/lucene/spatial/prefix/TermQueryPrefixTreeStrategy.java
lucene/dev/branches/solr5914/lucene/spatial/src/java/org/apache/lucene/spatial/prefix/tree/Cell.java
lucene/dev/branches/solr5914/lucene/spatial/src/java/org/apache/lucene/spatial/prefix/tree/GeohashPrefixTree.java
lucene/dev/branches/solr5914/lucene/spatial/src/java/org/apache/lucene/spatial/prefix/tree/QuadPrefixTree.java
lucene/dev/branches/solr5914/lucene/spatial/src/java/org/apache/lucene/spatial/prefix/tree/SpatialPrefixTree.java
lucene/dev/branches/solr5914/lucene/spatial/src/test/org/apache/lucene/spatial/prefix/tree/SpatialPrefixTreeTest.java
lucene/dev/branches/solr5914/lucene/test-framework/ (props changed)
lucene/dev/branches/solr5914/lucene/test-framework/src/java/org/apache/lucene/search/AssertingBulkOutOfOrderScorer.java
lucene/dev/branches/solr5914/lucene/test-framework/src/java/org/apache/lucene/search/AssertingBulkScorer.java
lucene/dev/branches/solr5914/lucene/test-framework/src/java/org/apache/lucene/search/AssertingCollector.java
lucene/dev/branches/solr5914/lucene/test-framework/src/java/org/apache/lucene/search/AssertingWeight.java
lucene/dev/branches/solr5914/lucene/test-framework/src/java/org/apache/lucene/search/CheckHits.java
lucene/dev/branches/solr5914/lucene/test-framework/src/java/org/apache/lucene/search/QueryUtils.java
lucene/dev/branches/solr5914/lucene/test-framework/src/java/org/apache/lucene/store/MockDirectoryWrapper.java
lucene/dev/branches/solr5914/lucene/test-framework/src/java/org/apache/lucene/util/LuceneTestCase.java
lucene/dev/branches/solr5914/solr/ (props changed)
lucene/dev/branches/solr5914/solr/core/ (props changed)
lucene/dev/branches/solr5914/solr/core/src/java/org/apache/solr/analytics/accumulator/BasicAccumulator.java
lucene/dev/branches/solr5914/solr/core/src/java/org/apache/solr/analytics/accumulator/FacetingAccumulator.java
lucene/dev/branches/solr5914/solr/core/src/java/org/apache/solr/analytics/accumulator/ValueAccumulator.java
lucene/dev/branches/solr5914/solr/core/src/java/org/apache/solr/analytics/accumulator/facet/FieldFacetAccumulator.java
lucene/dev/branches/solr5914/solr/core/src/java/org/apache/solr/analytics/accumulator/facet/QueryFacetAccumulator.java
lucene/dev/branches/solr5914/solr/core/src/java/org/apache/solr/analytics/accumulator/facet/RangeFacetAccumulator.java
lucene/dev/branches/solr5914/solr/core/src/java/org/apache/solr/analytics/request/AnalyticsStats.java
lucene/dev/branches/solr5914/solr/core/src/java/org/apache/solr/cloud/OverseerCollectionProcessor.java
lucene/dev/branches/solr5914/solr/core/src/java/org/apache/solr/handler/component/ExpandComponent.java
lucene/dev/branches/solr5914/solr/core/src/java/org/apache/solr/schema/LatLonType.java
lucene/dev/branches/solr5914/solr/core/src/java/org/apache/solr/search/CollapsingQParserPlugin.java
lucene/dev/branches/solr5914/solr/core/src/java/org/apache/solr/search/DelegatingCollector.java
lucene/dev/branches/solr5914/solr/core/src/java/org/apache/solr/search/DocSetCollector.java
lucene/dev/branches/solr5914/solr/core/src/java/org/apache/solr/search/EarlyTerminatingCollector.java
lucene/dev/branches/solr5914/solr/core/src/java/org/apache/solr/search/FunctionRangeQuery.java
lucene/dev/branches/solr5914/solr/core/src/java/org/apache/solr/search/Grouping.java
lucene/dev/branches/solr5914/solr/core/src/java/org/apache/solr/search/SolrIndexSearcher.java
lucene/dev/branches/solr5914/solr/core/src/java/org/apache/solr/search/grouping/CommandHandler.java
lucene/dev/branches/solr5914/solr/core/src/java/org/apache/solr/search/grouping/collector/FilterCollector.java
lucene/dev/branches/solr5914/solr/core/src/test/org/apache/solr/search/TestSort.java
Modified: lucene/dev/branches/solr5914/dev-tools/idea/.idea/libraries/JUnit.xml
URL: http://svn.apache.org/viewvc/lucene/dev/branches/solr5914/dev-tools/idea/.idea/libraries/JUnit.xml?rev=1585024&r1=1585023&r2=1585024&view=diff
==============================================================================
--- lucene/dev/branches/solr5914/dev-tools/idea/.idea/libraries/JUnit.xml (original)
+++ lucene/dev/branches/solr5914/dev-tools/idea/.idea/libraries/JUnit.xml Sat Apr 5 09:02:57 2014
@@ -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 />
Modified: lucene/dev/branches/solr5914/lucene/CHANGES.txt
URL: http://svn.apache.org/viewvc/lucene/dev/branches/solr5914/lucene/CHANGES.txt?rev=1585024&r1=1585023&r2=1585024&view=diff
==============================================================================
--- lucene/dev/branches/solr5914/lucene/CHANGES.txt (original)
+++ lucene/dev/branches/solr5914/lucene/CHANGES.txt Sat Apr 5 09:02:57 2014
@@ -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)
Modified: lucene/dev/branches/solr5914/lucene/MIGRATE.txt
URL: http://svn.apache.org/viewvc/lucene/dev/branches/solr5914/lucene/MIGRATE.txt?rev=1585024&r1=1585023&r2=1585024&view=diff
==============================================================================
--- lucene/dev/branches/solr5914/lucene/MIGRATE.txt (original)
+++ lucene/dev/branches/solr5914/lucene/MIGRATE.txt Sat Apr 5 09:02:57 2014
@@ -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.
Modified: lucene/dev/branches/solr5914/lucene/core/src/java/org/apache/lucene/index/IndexFileDeleter.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/solr5914/lucene/core/src/java/org/apache/lucene/index/IndexFileDeleter.java?rev=1585024&r1=1585023&r2=1585024&view=diff
==============================================================================
--- lucene/dev/branches/solr5914/lucene/core/src/java/org/apache/lucene/index/IndexFileDeleter.java (original)
+++ lucene/dev/branches/solr5914/lucene/core/src/java/org/apache/lucene/index/IndexFileDeleter.java Sat Apr 5 09:02:57 2014
@@ -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
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
void deleteFile(String fileName)
throws IOException {
assert locked();
+ ensureOpen();
try {
if (infoStream.isEnabled("IFD")) {
infoStream.message("IFD", "delete \"" + fileName + "\"");
Modified: lucene/dev/branches/solr5914/lucene/core/src/java/org/apache/lucene/index/IndexWriter.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/solr5914/lucene/core/src/java/org/apache/lucene/index/IndexWriter.java?rev=1585024&r1=1585023&r2=1585024&view=diff
==============================================================================
--- lucene/dev/branches/solr5914/lucene/core/src/java/org/apache/lucene/index/IndexWriter.java (original)
+++ lucene/dev/branches/solr5914/lucene/core/src/java/org/apache/lucene/index/IndexWriter.java Sat Apr 5 09:02:57 2014
@@ -4566,8 +4566,7 @@ public class IndexWriter implements Clos
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 Clos
}
synchronized void incRefDeleter(SegmentInfos segmentInfos) throws IOException {
+ ensureOpen();
deleter.incRef(segmentInfos, false);
}
synchronized void decRefDeleter(SegmentInfos segmentInfos) throws IOException {
+ ensureOpen();
deleter.decRef(segmentInfos);
}
Modified: lucene/dev/branches/solr5914/lucene/core/src/java/org/apache/lucene/index/StandardDirectoryReader.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/solr5914/lucene/core/src/java/org/apache/lucene/index/StandardDirectoryReader.java?rev=1585024&r1=1585023&r2=1585024&view=diff
==============================================================================
--- lucene/dev/branches/solr5914/lucene/core/src/java/org/apache/lucene/index/StandardDirectoryReader.java (original)
+++ lucene/dev/branches/solr5914/lucene/core/src/java/org/apache/lucene/index/StandardDirectoryReader.java Sat Apr 5 09:02:57 2014
@@ -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 exte
}
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
Modified: lucene/dev/branches/solr5914/lucene/core/src/java/org/apache/lucene/search/BooleanScorer.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/solr5914/lucene/core/src/java/org/apache/lucene/search/BooleanScorer.java?rev=1585024&r1=1585023&r2=1585024&view=diff
==============================================================================
--- lucene/dev/branches/solr5914/lucene/core/src/java/org/apache/lucene/search/BooleanScorer.java (original)
+++ lucene/dev/branches/solr5914/lucene/core/src/java/org/apache/lucene/search/BooleanScorer.java Sat Apr 5 09:02:57 2014
@@ -61,7 +61,7 @@ import org.apache.lucene.search.BooleanQ
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;
@@ -93,11 +93,6 @@ final class BooleanScorer extends BulkSc
}
@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 BulkSc
}
}
- public Collector newCollector(int mask) {
+ public LeafCollector newCollector(int mask) {
return new BooleanScorerCollector(mask, this);
}
@@ -148,12 +143,12 @@ final class BooleanScorer extends BulkSc
// 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 BulkSc
}
@Override
- public boolean score(Collector collector, int max) throws IOException {
+ public boolean score(LeafCollector collector, int max) throws IOException {
boolean more;
Bucket tmp;
Modified: lucene/dev/branches/solr5914/lucene/core/src/java/org/apache/lucene/search/BulkScorer.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/solr5914/lucene/core/src/java/org/apache/lucene/search/BulkScorer.java?rev=1585024&r1=1585023&r2=1585024&view=diff
==============================================================================
--- lucene/dev/branches/solr5914/lucene/core/src/java/org/apache/lucene/search/BulkScorer.java (original)
+++ lucene/dev/branches/solr5914/lucene/core/src/java/org/apache/lucene/search/BulkScorer.java Sat Apr 5 09:02:57 2014
@@ -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;
}
Modified: lucene/dev/branches/solr5914/lucene/core/src/java/org/apache/lucene/search/CachingCollector.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/solr5914/lucene/core/src/java/org/apache/lucene/search/CachingCollector.java?rev=1585024&r1=1585023&r2=1585024&view=diff
==============================================================================
--- lucene/dev/branches/solr5914/lucene/core/src/java/org/apache/lucene/search/CachingCollector.java (original)
+++ lucene/dev/branches/solr5914/lucene/core/src/java/org/apache/lucene/search/CachingCollector.java Sat Apr 5 09:02:57 2014
@@ -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; }
- }
+ }
- // A CachingCollector which caches scores
- private static final class ScoreCachingCollector extends CachingCollector {
+ private static class NoScoreCachingCollector extends CachingCollector {
- private final CachedScorer cachedScorer;
- private final List<float[]> cachedScores;
+ 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<>();
+ }
+
+ protected NoScoreCachingLeafCollector wrap(LeafCollector in, int maxDocsToCache) {
+ return new NoScoreCachingLeafCollector(in, maxDocsToCache);
+ }
+
+ 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;
+ }
+ }
- private Scorer scorer;
- private float[] curScores;
+ 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;
+ }
+ }
- ScoreCachingCollector(Collector other, double maxRAMMB) {
- super(other, maxRAMMB, true);
+ protected void collect(LeafCollector collector, int i) throws IOException {
+ final int[] docs = this.docs.get(i);
+ for (int doc : docs) {
+ collector.collect(doc);
+ }
+ }
- cachedScorer = new CachedScorer();
- cachedScores = new ArrayList<>();
- curScores = new float[INITIAL_ARRAY_SIZE];
- cachedScores.add(curScores);
+ 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);
+ }
}
- ScoreCachingCollector(Collector other, int maxDocsToCache) {
- super(other, maxDocsToCache);
+ }
+
+ private static class ScoreCachingCollector extends NoScoreCachingCollector {
- cachedScorer = new CachedScorer();
- cachedScores = new ArrayList<>();
- curScores = new float[INITIAL_ARRAY_SIZE];
- cachedScores.add(curScores);
+ 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
- public void collect(int doc) throws IOException {
+ protected void postCollect(NoScoreCachingLeafCollector collector) {
+ final ScoreCachingLeafCollector coll = (ScoreCachingLeafCollector) collector;
+ super.postCollect(coll);
+ scores.add(coll.cachedScores());
+ }
- if (curDocs == null) {
- // Cache was too large
- cachedScorer.score = scorer.score();
- cachedScorer.doc = doc;
- other.collect(doc);
- return;
+ 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);
}
+ }
- // 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;
- }
- }
-
- curDocs = new int[nextLength];
- cachedDocs.add(curDocs);
- curScores = new float[nextLength];
- cachedScores.add(curScores);
- upto = 0;
- }
-
- curDocs[upto] = doc;
- cachedScorer.score = curScores[upto] = scorer.score();
- upto++;
- cachedScorer.doc = doc;
- other.collect(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 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;
+ public void collect(int doc) throws IOException {
+ 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);
}
- cachedScorer.score = curScores[curUpto];
- cachedScorer.doc = curDocs[curUpto];
- other.collect(curDocs[curUpto++]);
+ }
+ if (docs != null) {
+ buffer(doc);
+ ++docCount;
}
}
+ super.collect(doc);
}
- @Override
- public void setScorer(Scorer scorer) throws IOException {
- this.scorer = scorer;
- other.setScorer(cachedScorer);
+ boolean hasCache() {
+ return docs != null;
}
- @Override
- public String toString() {
- if (isCached()) {
- return "CachingCollector (" + (base+upto) + " docs & scores cached)";
- } else {
- return "CachingCollector (cache was cleared)";
- }
+ int[] cachedDocs() {
+ return docs == null ? null : Arrays.copyOf(docs, docCount);
}
}
- // A CachingCollector which does not cache scores
- private static final class NoScoreCachingCollector extends CachingCollector {
-
- NoScoreCachingCollector(Collector other, double maxRAMMB) {
- super(other, maxRAMMB, false);
- }
+ private class ScoreCachingLeafCollector extends NoScoreCachingLeafCollector {
+
+ Scorer scorer;
+ float[] scores;
- NoScoreCachingCollector(Collector other, int maxDocsToCache) {
- super(other, maxDocsToCache);
+ ScoreCachingLeafCollector(LeafCollector in, int maxDocsToCache) {
+ super(in, maxDocsToCache);
+ scores = new float[docs.length];
}
@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);
+ public void setScorer(Scorer scorer) throws IOException {
+ this.scorer = scorer;
+ super.setScorer(scorer);
}
@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++]);
- }
- }
+ protected void grow(int newLen) {
+ super.grow(newLen);
+ scores = Arrays.copyOf(scores, newLen);
}
@Override
- public void setScorer(Scorer scorer) throws IOException {
- other.setScorer(scorer);
+ protected void invalidate() {
+ super.invalidate();
+ scores = null;
}
@Override
- public String toString() {
- if (isCached()) {
- return "CachingCollector (" + (base+upto) + " docs cached)";
- } else {
- return "CachingCollector (cache was cleared)";
- }
+ protected void buffer(int doc) throws IOException {
+ super.buffer(doc);
+ scores[docCount] = scorer.score();
}
+ float[] cachedScores() {
+ return docs == null ? null : Arrays.copyOf(scores, docCount);
+ }
}
- // 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 e
/**
* 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 e
* 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 e
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 CachingCollector(Collector other, int maxDocsToCache) {
- this.other = other;
+ private boolean cached;
- cachedDocs = new ArrayList<>();
- curDocs = new int[INITIAL_ARRAY_SIZE];
- cachedDocs.add(curDocs);
- this.maxDocsToCache = maxDocsToCache;
- }
-
- @Override
- public boolean acceptsDocsOutOfOrder() {
- return other.acceptsDocsOutOfOrder();
+ private CachingCollector(Collector in) {
+ super(in);
+ cached = true;
}
- 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 e
* while the collector passed to the ctor does.
*/
public abstract void replay(Collector other) throws IOException;
-
+
}
Modified: lucene/dev/branches/solr5914/lucene/core/src/java/org/apache/lucene/search/CollectionTerminatedException.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/solr5914/lucene/core/src/java/org/apache/lucene/search/CollectionTerminatedException.java?rev=1585024&r1=1585023&r2=1585024&view=diff
==============================================================================
--- lucene/dev/branches/solr5914/lucene/core/src/java/org/apache/lucene/search/CollectionTerminatedException.java (original)
+++ lucene/dev/branches/solr5914/lucene/core/src/java/org/apache/lucene/search/CollectionTerminatedException.java Sat Apr 5 09:02:57 2014
@@ -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
Modified: lucene/dev/branches/solr5914/lucene/core/src/java/org/apache/lucene/search/Collector.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/solr5914/lucene/core/src/java/org/apache/lucene/search/Collector.java?rev=1585024&r1=1585023&r2=1585024&view=diff
==============================================================================
--- lucene/dev/branches/solr5914/lucene/core/src/java/org/apache/lucene/search/Collector.java (original)
+++ lucene/dev/branches/solr5914/lucene/core/src/java/org/apache/lucene/search/Collector.java Sat Apr 5 09:02:57 2014
@@ -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.IndexRead
*
* </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();
-
}
Modified: lucene/dev/branches/solr5914/lucene/core/src/java/org/apache/lucene/search/ConstantScoreQuery.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/solr5914/lucene/core/src/java/org/apache/lucene/search/ConstantScoreQuery.java?rev=1585024&r1=1585023&r2=1585024&view=diff
==============================================================================
--- lucene/dev/branches/solr5914/lucene/core/src/java/org/apache/lucene/search/ConstantScoreQuery.java (original)
+++ lucene/dev/branches/solr5914/lucene/core/src/java/org/apache/lucene/search/ConstantScoreQuery.java Sat Apr 5 09:02:57 2014
@@ -212,31 +212,16 @@ public class ConstantScoreQuery extends
}
@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));
}
};
}
Modified: lucene/dev/branches/solr5914/lucene/core/src/java/org/apache/lucene/search/FakeScorer.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/solr5914/lucene/core/src/java/org/apache/lucene/search/FakeScorer.java?rev=1585024&r1=1585023&r2=1585024&view=diff
==============================================================================
--- lucene/dev/branches/solr5914/lucene/core/src/java/org/apache/lucene/search/FakeScorer.java (original)
+++ lucene/dev/branches/solr5914/lucene/core/src/java/org/apache/lucene/search/FakeScorer.java Sat Apr 5 09:02:57 2014
@@ -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;
Modified: lucene/dev/branches/solr5914/lucene/core/src/java/org/apache/lucene/search/FilteredQuery.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/solr5914/lucene/core/src/java/org/apache/lucene/search/FilteredQuery.java?rev=1585024&r1=1585023&r2=1585024&view=diff
==============================================================================
--- lucene/dev/branches/solr5914/lucene/core/src/java/org/apache/lucene/search/FilteredQuery.java (original)
+++ lucene/dev/branches/solr5914/lucene/core/src/java/org/apache/lucene/search/FilteredQuery.java Sat Apr 5 09:02:57 2014
@@ -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);
Modified: lucene/dev/branches/solr5914/lucene/core/src/java/org/apache/lucene/search/IndexSearcher.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/solr5914/lucene/core/src/java/org/apache/lucene/search/IndexSearcher.java?rev=1585024&r1=1585023&r2=1585024&view=diff
==============================================================================
--- lucene/dev/branches/solr5914/lucene/core/src/java/org/apache/lucene/search/IndexSearcher.java (original)
+++ lucene/dev/branches/solr5914/lucene/core/src/java/org/apache/lucene/search/IndexSearcher.java Sat Apr 5 09:02:57 2014
@@ -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:
Modified: lucene/dev/branches/solr5914/lucene/core/src/java/org/apache/lucene/search/MultiCollector.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/solr5914/lucene/core/src/java/org/apache/lucene/search/MultiCollector.java?rev=1585024&r1=1585023&r2=1585024&view=diff
==============================================================================
--- lucene/dev/branches/solr5914/lucene/core/src/java/org/apache/lucene/search/MultiCollector.java (original)
+++ lucene/dev/branches/solr5914/lucene/core/src/java/org/apache/lucene/search/MultiCollector.java Sat Apr 5 09:02:57 2014
@@ -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 Coll
* 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 Coll
}
}
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 Coll
}
@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 true;
+ return new MultiLeafCollector(leafCollectors);
}
- @Override
- public void collect(int doc) throws IOException {
- for (Collector c : collectors) {
- c.collect(doc);
+
+ private static class MultiLeafCollector implements LeafCollector {
+
+ private final LeafCollector[] collectors;
+
+ private MultiLeafCollector(LeafCollector[] collectors) {
+ this.collectors = collectors;
}
- }
- @Override
- public void setNextReader(AtomicReaderContext context) throws IOException {
- for (Collector c : collectors) {
- c.setNextReader(context);
+ @Override
+ public void setScorer(Scorer scorer) throws IOException {
+ for (LeafCollector c : collectors) {
+ c.setScorer(scorer);
+ }
}
- }
- @Override
- public void setScorer(Scorer s) throws IOException {
- for (Collector c : collectors) {
- c.setScorer(s);
+ @Override
+ public void collect(int doc) throws IOException {
+ for (LeafCollector c : collectors) {
+ c.collect(doc);
+ }
+ }
+
+ @Override
+ public boolean acceptsDocsOutOfOrder() {
+ for (LeafCollector c : collectors) {
+ if (!c.acceptsDocsOutOfOrder()) {
+ return false;
+ }
+ }
+ return true;
}
+
}
}
Modified: lucene/dev/branches/solr5914/lucene/core/src/java/org/apache/lucene/search/PositiveScoresOnlyCollector.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/solr5914/lucene/core/src/java/org/apache/lucene/search/PositiveScoresOnlyCollector.java?rev=1585024&r1=1585023&r2=1585024&view=diff
==============================================================================
--- lucene/dev/branches/solr5914/lucene/core/src/java/org/apache/lucene/search/PositiveScoresOnlyCollector.java (original)
+++ lucene/dev/branches/solr5914/lucene/core/src/java/org/apache/lucene/search/PositiveScoresOnlyCollector.java Sat Apr 5 09:02:57 2014
@@ -26,38 +26,33 @@ import org.apache.lucene.index.AtomicRea
* {@link Collector} and makes sure only documents with
* scores > 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);
+ }
+ }
+
+ };
}
}
Modified: lucene/dev/branches/solr5914/lucene/core/src/java/org/apache/lucene/search/ScoreCachingWrappingScorer.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/solr5914/lucene/core/src/java/org/apache/lucene/search/ScoreCachingWrappingScorer.java?rev=1585024&r1=1585023&r2=1585024&view=diff
==============================================================================
--- lucene/dev/branches/solr5914/lucene/core/src/java/org/apache/lucene/search/ScoreCachingWrappingScorer.java (original)
+++ lucene/dev/branches/solr5914/lucene/core/src/java/org/apache/lucene/search/ScoreCachingWrappingScorer.java Sat Apr 5 09:02:57 2014
@@ -37,7 +37,7 @@ public class ScoreCachingWrappingScorer
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
curScore = scorer.score();
curDoc = doc;
}
-
+
return curScore;
}
Modified: lucene/dev/branches/solr5914/lucene/core/src/java/org/apache/lucene/search/Scorer.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/solr5914/lucene/core/src/java/org/apache/lucene/search/Scorer.java?rev=1585024&r1=1585023&r2=1585024&view=diff
==============================================================================
--- lucene/dev/branches/solr5914/lucene/core/src/java/org/apache/lucene/search/Scorer.java (original)
+++ lucene/dev/branches/solr5914/lucene/core/src/java/org/apache/lucene/search/Scorer.java Sat Apr 5 09:02:57 2014
@@ -57,7 +57,7 @@ public abstract class Scorer extends Doc
/** 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;
Modified: lucene/dev/branches/solr5914/lucene/core/src/java/org/apache/lucene/search/SortRescorer.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/solr5914/lucene/core/src/java/org/apache/lucene/search/SortRescorer.java?rev=1585024&r1=1585023&r2=1585024&view=diff
==============================================================================
--- lucene/dev/branches/solr5914/lucene/core/src/java/org/apache/lucene/search/SortRescorer.java (original)
+++ lucene/dev/branches/solr5914/lucene/core/src/java/org/apache/lucene/search/SortRescorer.java Sat Apr 5 09:02:57 2014
@@ -75,7 +75,7 @@ public class SortRescorer extends Rescor
if (readerContext != null) {
// We advanced to another segment:
- collector.setNextReader(readerContext);
+ collector.getLeafCollector(readerContext);
collector.setScorer(fakeScorer);
docBase = readerContext.docBase;
}
Modified: lucene/dev/branches/solr5914/lucene/core/src/java/org/apache/lucene/search/TimeLimitingCollector.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/solr5914/lucene/core/src/java/org/apache/lucene/search/TimeLimitingCollector.java?rev=1585024&r1=1585023&r2=1585024&view=diff
==============================================================================
--- lucene/dev/branches/solr5914/lucene/core/src/java/org/apache/lucene/search/TimeLimitingCollector.java (original)
+++ lucene/dev/branches/solr5914/lucene/core/src/java/org/apache/lucene/search/TimeLimitingCollector.java Sat Apr 5 09:02:57 2014
@@ -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 exten
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);
+ }
+
+ };
}
/**
Modified: lucene/dev/branches/solr5914/lucene/core/src/java/org/apache/lucene/search/TopDocsCollector.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/solr5914/lucene/core/src/java/org/apache/lucene/search/TopDocsCollector.java?rev=1585024&r1=1585023&r2=1585024&view=diff
==============================================================================
--- lucene/dev/branches/solr5914/lucene/core/src/java/org/apache/lucene/search/TopDocsCollector.java (original)
+++ lucene/dev/branches/solr5914/lucene/core/src/java/org/apache/lucene/search/TopDocsCollector.java Sat Apr 5 09:02:57 2014
@@ -31,7 +31,7 @@ import org.apache.lucene.util.PriorityQu
* however, you might want to consider overriding all methods, in order to avoid
* a NullPointerException.
*/
-public abstract class TopDocsCollector<T extends ScoreDoc> extends 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. */
Modified: lucene/dev/branches/solr5914/lucene/core/src/java/org/apache/lucene/search/TopFieldCollector.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/solr5914/lucene/core/src/java/org/apache/lucene/search/TopFieldCollector.java?rev=1585024&r1=1585023&r2=1585024&view=diff
==============================================================================
--- lucene/dev/branches/solr5914/lucene/core/src/java/org/apache/lucene/search/TopFieldCollector.java (original)
+++ lucene/dev/branches/solr5914/lucene/core/src/java/org/apache/lucene/search/TopFieldCollector.java Sat Apr 5 09:02:57 2014
@@ -92,7 +92,7 @@ public abstract class TopFieldCollector
}
@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
}
@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
}
@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++) {
Modified: lucene/dev/branches/solr5914/lucene/core/src/java/org/apache/lucene/search/TopScoreDocCollector.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/solr5914/lucene/core/src/java/org/apache/lucene/search/TopScoreDocCollector.java?rev=1585024&r1=1585023&r2=1585024&view=diff
==============================================================================
--- lucene/dev/branches/solr5914/lucene/core/src/java/org/apache/lucene/search/TopScoreDocCollector.java (original)
+++ lucene/dev/branches/solr5914/lucene/core/src/java/org/apache/lucene/search/TopScoreDocCollector.java Sat Apr 5 09:02:57 2014
@@ -113,9 +113,9 @@ public abstract class TopScoreDocCollect
}
@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 TopScoreDocCollect
}
@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 TopScoreDocCollect
}
@Override
- public void setNextReader(AtomicReaderContext context) {
+ protected void doSetNextReader(AtomicReaderContext context) throws IOException {
docBase = context.docBase;
}
Modified: lucene/dev/branches/solr5914/lucene/core/src/java/org/apache/lucene/search/TotalHitCountCollector.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/solr5914/lucene/core/src/java/org/apache/lucene/search/TotalHitCountCollector.java?rev=1585024&r1=1585023&r2=1585024&view=diff
==============================================================================
--- lucene/dev/branches/solr5914/lucene/core/src/java/org/apache/lucene/search/TotalHitCountCollector.java (original)
+++ lucene/dev/branches/solr5914/lucene/core/src/java/org/apache/lucene/search/TotalHitCountCollector.java Sat Apr 5 09:02:57 2014
@@ -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. */
@@ -32,19 +31,11 @@ public class TotalHitCountCollector exte
}
@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;
}
Modified: lucene/dev/branches/solr5914/lucene/core/src/java/org/apache/lucene/search/Weight.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/solr5914/lucene/core/src/java/org/apache/lucene/search/Weight.java?rev=1585024&r1=1585023&r2=1585024&view=diff
==============================================================================
--- lucene/dev/branches/solr5914/lucene/core/src/java/org/apache/lucene/search/Weight.java (original)
+++ lucene/dev/branches/solr5914/lucene/core/src/java/org/apache/lucene/search/Weight.java Sat Apr 5 09:02:57 2014
@@ -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.
Modified: lucene/dev/branches/solr5914/lucene/core/src/java/org/apache/lucene/search/package.html
URL: http://svn.apache.org/viewvc/lucene/dev/branches/solr5914/lucene/core/src/java/org/apache/lucene/search/package.html?rev=1585024&r1=1585023&r2=1585024&view=diff
==============================================================================
--- lucene/dev/branches/solr5914/lucene/core/src/java/org/apache/lucene/search/package.html (original)
+++ lucene/dev/branches/solr5914/lucene/core/src/java/org/apache/lucene/search/package.html Sat Apr 5 09:02:57 2014
@@ -508,7 +508,7 @@ on the built-in available scoring models
abstract method:
<ol>
<li>
- {@link org.apache.lucene.search.BulkScorer#score(org.apache.lucene.search.Collector,int) score(Collector,int)} —
+ {@link org.apache.lucene.search.BulkScorer#score(org.apache.lucene.search.LeafCollector,int) score(LeafCollector,int)} —
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
<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
Modified: lucene/dev/branches/solr5914/lucene/core/src/java/org/apache/lucene/store/FSDirectory.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/solr5914/lucene/core/src/java/org/apache/lucene/store/FSDirectory.java?rev=1585024&r1=1585023&r2=1585024&view=diff
==============================================================================
--- lucene/dev/branches/solr5914/lucene/core/src/java/org/apache/lucene/store/FSDirectory.java (original)
+++ lucene/dev/branches/solr5914/lucene/core/src/java/org/apache/lucene/store/FSDirectory.java Sat Apr 5 09:02:57 2014
@@ -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 extend
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)
Modified: lucene/dev/branches/solr5914/lucene/core/src/test/org/apache/lucene/index/TestIndexWriter.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/solr5914/lucene/core/src/test/org/apache/lucene/index/TestIndexWriter.java?rev=1585024&r1=1585023&r2=1585024&view=diff
==============================================================================
--- lucene/dev/branches/solr5914/lucene/core/src/test/org/apache/lucene/index/TestIndexWriter.java (original)
+++ lucene/dev/branches/solr5914/lucene/core/src/test/org/apache/lucene/index/TestIndexWriter.java Sat Apr 5 09:02:57 2014
@@ -67,6 +67,7 @@ import org.apache.lucene.store.SimpleFSL
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 Luc
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();
+ }
}