mirror of https://github.com/apache/lucene.git
LUCENE-3622: merge trunk (1212830:1213323)
git-svn-id: https://svn.apache.org/repos/asf/lucene/dev/branches/lucene3622@1213405 13f79535-47bb-0310-9956-ffa450edef68
This commit is contained in:
commit
b15b981081
|
@ -223,6 +223,10 @@ Changes in backwards compatibility policy
|
|||
changes on disk, copy the FilteredIndexReader to a new index using
|
||||
IndexWriter.addIndexes(). (Uwe Schindler, Robert Muir)
|
||||
|
||||
* LUCENE-3640: Removed IndexSearcher.close(), because IndexSearcher no longer
|
||||
takes a Directory and no longer "manages" IndexReaders, it is a no-op.
|
||||
(Robert Muir)
|
||||
|
||||
Changes in Runtime Behavior
|
||||
|
||||
* LUCENE-2846: omitNorms now behaves like omitTermFrequencyAndPositions, if you
|
||||
|
@ -672,6 +676,13 @@ Changes in backwards compatibility policy
|
|||
already delegate to abstract methods, which you had to already override
|
||||
anyway. (Shai Erera)
|
||||
|
||||
* LUCENE-3636: Added SearcherFactory, used by SearcherManager and NRTManager
|
||||
to create new IndexSearchers. You can provide your own implementation to
|
||||
warm new searchers, set an ExecutorService, set a custom Similarity, or
|
||||
even return your own subclass of IndexSearcher. The SearcherWarmer and
|
||||
ExecutorService parameters on these classes were removed, as they are
|
||||
subsumed by SearcherFactory. (Shai Erera, Mike McCandless, Robert Muir)
|
||||
|
||||
Security fixes
|
||||
|
||||
* LUCENE-3588: Try harder to prevent SIGSEGV on cloned MMapIndexInputs:
|
||||
|
@ -717,6 +728,9 @@ Bug fixes
|
|||
In ParallelReader the bug was not existent, but the implementation method
|
||||
was also made private. (Uwe Schindler)
|
||||
|
||||
* LUCENE-3641: Fixed MultiReader to correctly propagate readerFinishedListeners
|
||||
to clones/reopened readers. (Uwe Schindler)
|
||||
|
||||
Documentation
|
||||
|
||||
* LUCENE-3597: Fixed incorrect grouping documentation. (Martijn van Groningen, Robert Muir)
|
||||
|
|
|
@ -88,6 +88,32 @@ Bug Fixes
|
|||
|
||||
======================= Lucene 3.6.0 ================
|
||||
|
||||
Changes in backwards compatibility policy
|
||||
|
||||
* LUCENE-3626: The internal implementation classes in PKIndexSplitter
|
||||
and MultiPassIndexSplitter were made private as they now work
|
||||
per segment. (Uwe Schindler)
|
||||
|
||||
New Features
|
||||
|
||||
* LUCENE-3596: DirectoryTaxonomyWriter extensions can override createIndexWriterConfig()
|
||||
and modify how its internal index writer is opened. (Doron Cohen)
|
||||
|
||||
API Changes
|
||||
|
||||
* LUCENE-3596: DirectoryTaxonomyWriter.openIndexWriter() now takes an
|
||||
openIndexWriter parameter rather than just an open-mode. (Doron Cohen)
|
||||
|
||||
* LUCENE-3606: FieldNormModifier was deprecated, because IndexReader's
|
||||
setNorm() was deprecated. Furthermore, this class is broken, as it does
|
||||
not take position overlaps into account while recalculating norms.
|
||||
(Uwe Schindler, Robert Muir)
|
||||
|
||||
Changes in runtime behavior:
|
||||
|
||||
* LUCENE-3626: PKIndexSplitter and MultiPassIndexSplitter now work
|
||||
per segment. (Uwe Schindler)
|
||||
|
||||
Bug Fixes
|
||||
|
||||
* LUCENE-3600: BlockJoinQuery now supports parent docs that have no
|
||||
|
|
|
@ -131,7 +131,6 @@ public class SearchFiles {
|
|||
break;
|
||||
}
|
||||
}
|
||||
searcher.close();
|
||||
reader.close();
|
||||
}
|
||||
|
||||
|
|
|
@ -72,26 +72,21 @@ public class HighlighterPhraseTest extends LuceneTestCase {
|
|||
try {
|
||||
assertEquals(1, indexReader.numDocs());
|
||||
final IndexSearcher indexSearcher = newSearcher(indexReader);
|
||||
try {
|
||||
final PhraseQuery phraseQuery = new PhraseQuery();
|
||||
phraseQuery.add(new Term(FIELD, "fox"));
|
||||
phraseQuery.add(new Term(FIELD, "jumped"));
|
||||
phraseQuery.setSlop(0);
|
||||
TopDocs hits = indexSearcher.search(phraseQuery, 1);
|
||||
assertEquals(1, hits.totalHits);
|
||||
final Highlighter highlighter = new Highlighter(
|
||||
new SimpleHTMLFormatter(), new SimpleHTMLEncoder(),
|
||||
new QueryScorer(phraseQuery));
|
||||
final PhraseQuery phraseQuery = new PhraseQuery();
|
||||
phraseQuery.add(new Term(FIELD, "fox"));
|
||||
phraseQuery.add(new Term(FIELD, "jumped"));
|
||||
phraseQuery.setSlop(0);
|
||||
TopDocs hits = indexSearcher.search(phraseQuery, 1);
|
||||
assertEquals(1, hits.totalHits);
|
||||
final Highlighter highlighter = new Highlighter(
|
||||
new SimpleHTMLFormatter(), new SimpleHTMLEncoder(),
|
||||
new QueryScorer(phraseQuery));
|
||||
|
||||
final TokenStream tokenStream = TokenSources
|
||||
.getTokenStream(indexReader.getTermVector(
|
||||
0, FIELD), false);
|
||||
assertEquals(highlighter.getBestFragment(new TokenStreamConcurrent(),
|
||||
TEXT), highlighter.getBestFragment(tokenStream, TEXT));
|
||||
|
||||
} finally {
|
||||
indexSearcher.close();
|
||||
}
|
||||
final TokenStream tokenStream = TokenSources
|
||||
.getTokenStream(indexReader.getTermVector(
|
||||
0, FIELD), false);
|
||||
assertEquals(highlighter.getBestFragment(new TokenStreamConcurrent(),
|
||||
TEXT), highlighter.getBestFragment(tokenStream, TEXT));
|
||||
} finally {
|
||||
indexReader.close();
|
||||
directory.close();
|
||||
|
@ -120,52 +115,48 @@ public class HighlighterPhraseTest extends LuceneTestCase {
|
|||
try {
|
||||
assertEquals(1, indexReader.numDocs());
|
||||
final IndexSearcher indexSearcher = newSearcher(indexReader);
|
||||
try {
|
||||
final Query phraseQuery = new SpanNearQuery(new SpanQuery[] {
|
||||
new SpanTermQuery(new Term(FIELD, "fox")),
|
||||
new SpanTermQuery(new Term(FIELD, "jumped")) }, 0, true);
|
||||
final FixedBitSet bitset = new FixedBitSet(indexReader.maxDoc());
|
||||
indexSearcher.search(phraseQuery, new Collector() {
|
||||
private int baseDoc;
|
||||
final Query phraseQuery = new SpanNearQuery(new SpanQuery[] {
|
||||
new SpanTermQuery(new Term(FIELD, "fox")),
|
||||
new SpanTermQuery(new Term(FIELD, "jumped")) }, 0, true);
|
||||
final FixedBitSet bitset = new FixedBitSet(indexReader.maxDoc());
|
||||
indexSearcher.search(phraseQuery, new Collector() {
|
||||
private int baseDoc;
|
||||
|
||||
@Override
|
||||
public boolean acceptsDocsOutOfOrder() {
|
||||
return true;
|
||||
}
|
||||
|
||||
@Override
|
||||
public void collect(int i) throws IOException {
|
||||
bitset.set(this.baseDoc + i);
|
||||
}
|
||||
|
||||
@Override
|
||||
public void setNextReader(AtomicReaderContext context)
|
||||
throws IOException {
|
||||
this.baseDoc = context.docBase;
|
||||
}
|
||||
|
||||
@Override
|
||||
public void setScorer(org.apache.lucene.search.Scorer scorer)
|
||||
throws IOException {
|
||||
// Do Nothing
|
||||
}
|
||||
});
|
||||
assertEquals(1, bitset.cardinality());
|
||||
final int maxDoc = indexReader.maxDoc();
|
||||
final Highlighter highlighter = new Highlighter(
|
||||
new SimpleHTMLFormatter(), new SimpleHTMLEncoder(),
|
||||
new QueryScorer(phraseQuery));
|
||||
for (int position = bitset.nextSetBit(0); position >= 0 && position < maxDoc-1; position = bitset
|
||||
.nextSetBit(position + 1)) {
|
||||
assertEquals(0, position);
|
||||
final TokenStream tokenStream = TokenSources.getTokenStream(
|
||||
indexReader.getTermVector(position,
|
||||
FIELD), false);
|
||||
assertEquals(highlighter.getBestFragment(new TokenStreamConcurrent(),
|
||||
TEXT), highlighter.getBestFragment(tokenStream, TEXT));
|
||||
@Override
|
||||
public boolean acceptsDocsOutOfOrder() {
|
||||
return true;
|
||||
}
|
||||
} finally {
|
||||
indexSearcher.close();
|
||||
|
||||
@Override
|
||||
public void collect(int i) throws IOException {
|
||||
bitset.set(this.baseDoc + i);
|
||||
}
|
||||
|
||||
@Override
|
||||
public void setNextReader(AtomicReaderContext context)
|
||||
throws IOException {
|
||||
this.baseDoc = context.docBase;
|
||||
}
|
||||
|
||||
@Override
|
||||
public void setScorer(org.apache.lucene.search.Scorer scorer)
|
||||
throws IOException {
|
||||
// Do Nothing
|
||||
}
|
||||
});
|
||||
assertEquals(1, bitset.cardinality());
|
||||
final int maxDoc = indexReader.maxDoc();
|
||||
final Highlighter highlighter = new Highlighter(
|
||||
new SimpleHTMLFormatter(), new SimpleHTMLEncoder(),
|
||||
new QueryScorer(phraseQuery));
|
||||
for (int position = bitset.nextSetBit(0); position >= 0 && position < maxDoc-1; position = bitset
|
||||
.nextSetBit(position + 1)) {
|
||||
assertEquals(0, position);
|
||||
final TokenStream tokenStream = TokenSources.getTokenStream(
|
||||
indexReader.getTermVector(position,
|
||||
FIELD), false);
|
||||
assertEquals(highlighter.getBestFragment(new TokenStreamConcurrent(),
|
||||
TEXT), highlighter.getBestFragment(tokenStream, TEXT));
|
||||
}
|
||||
} finally {
|
||||
indexReader.close();
|
||||
|
@ -195,25 +186,21 @@ public class HighlighterPhraseTest extends LuceneTestCase {
|
|||
try {
|
||||
assertEquals(1, indexReader.numDocs());
|
||||
final IndexSearcher indexSearcher = newSearcher(indexReader);
|
||||
try {
|
||||
final PhraseQuery phraseQuery = new PhraseQuery();
|
||||
phraseQuery.add(new Term(FIELD, "did"));
|
||||
phraseQuery.add(new Term(FIELD, "jump"));
|
||||
phraseQuery.setSlop(0);
|
||||
TopDocs hits = indexSearcher.search(phraseQuery, 1);
|
||||
assertEquals(0, hits.totalHits);
|
||||
final Highlighter highlighter = new Highlighter(
|
||||
new SimpleHTMLFormatter(), new SimpleHTMLEncoder(),
|
||||
new QueryScorer(phraseQuery));
|
||||
final TokenStream tokenStream = TokenSources
|
||||
.getTokenStream(indexReader.getTermVector(
|
||||
0, FIELD), false);
|
||||
assertEquals(
|
||||
highlighter.getBestFragment(new TokenStreamSparse(), TEXT),
|
||||
highlighter.getBestFragment(tokenStream, TEXT));
|
||||
} finally {
|
||||
indexSearcher.close();
|
||||
}
|
||||
final PhraseQuery phraseQuery = new PhraseQuery();
|
||||
phraseQuery.add(new Term(FIELD, "did"));
|
||||
phraseQuery.add(new Term(FIELD, "jump"));
|
||||
phraseQuery.setSlop(0);
|
||||
TopDocs hits = indexSearcher.search(phraseQuery, 1);
|
||||
assertEquals(0, hits.totalHits);
|
||||
final Highlighter highlighter = new Highlighter(
|
||||
new SimpleHTMLFormatter(), new SimpleHTMLEncoder(),
|
||||
new QueryScorer(phraseQuery));
|
||||
final TokenStream tokenStream = TokenSources
|
||||
.getTokenStream(indexReader.getTermVector(
|
||||
0, FIELD), false);
|
||||
assertEquals(
|
||||
highlighter.getBestFragment(new TokenStreamSparse(), TEXT),
|
||||
highlighter.getBestFragment(tokenStream, TEXT));
|
||||
} finally {
|
||||
indexReader.close();
|
||||
directory.close();
|
||||
|
@ -241,23 +228,19 @@ public class HighlighterPhraseTest extends LuceneTestCase {
|
|||
try {
|
||||
assertEquals(1, indexReader.numDocs());
|
||||
final IndexSearcher indexSearcher = newSearcher(indexReader);
|
||||
try {
|
||||
final PhraseQuery phraseQuery = new PhraseQuery();
|
||||
phraseQuery.add(new Term(FIELD, "did"));
|
||||
phraseQuery.add(new Term(FIELD, "jump"));
|
||||
phraseQuery.setSlop(1);
|
||||
TopDocs hits = indexSearcher.search(phraseQuery, 1);
|
||||
assertEquals(1, hits.totalHits);
|
||||
final Highlighter highlighter = new Highlighter(
|
||||
new SimpleHTMLFormatter(), new SimpleHTMLEncoder(),
|
||||
new QueryScorer(phraseQuery));
|
||||
final TokenStream tokenStream = TokenSources.getTokenStream(
|
||||
indexReader.getTermVector(0, FIELD), true);
|
||||
assertEquals("the fox <B>did</B> not <B>jump</B>", highlighter
|
||||
.getBestFragment(tokenStream, TEXT));
|
||||
} finally {
|
||||
indexSearcher.close();
|
||||
}
|
||||
final PhraseQuery phraseQuery = new PhraseQuery();
|
||||
phraseQuery.add(new Term(FIELD, "did"));
|
||||
phraseQuery.add(new Term(FIELD, "jump"));
|
||||
phraseQuery.setSlop(1);
|
||||
TopDocs hits = indexSearcher.search(phraseQuery, 1);
|
||||
assertEquals(1, hits.totalHits);
|
||||
final Highlighter highlighter = new Highlighter(
|
||||
new SimpleHTMLFormatter(), new SimpleHTMLEncoder(),
|
||||
new QueryScorer(phraseQuery));
|
||||
final TokenStream tokenStream = TokenSources.getTokenStream(
|
||||
indexReader.getTermVector(0, FIELD), true);
|
||||
assertEquals("the fox <B>did</B> not <B>jump</B>", highlighter
|
||||
.getBestFragment(tokenStream, TEXT));
|
||||
} finally {
|
||||
indexReader.close();
|
||||
directory.close();
|
||||
|
@ -285,25 +268,21 @@ public class HighlighterPhraseTest extends LuceneTestCase {
|
|||
try {
|
||||
assertEquals(1, indexReader.numDocs());
|
||||
final IndexSearcher indexSearcher = newSearcher(indexReader);
|
||||
try {
|
||||
final Query phraseQuery = new SpanNearQuery(new SpanQuery[] {
|
||||
new SpanTermQuery(new Term(FIELD, "did")),
|
||||
new SpanTermQuery(new Term(FIELD, "jump")) }, 0, true);
|
||||
final Query phraseQuery = new SpanNearQuery(new SpanQuery[] {
|
||||
new SpanTermQuery(new Term(FIELD, "did")),
|
||||
new SpanTermQuery(new Term(FIELD, "jump")) }, 0, true);
|
||||
|
||||
TopDocs hits = indexSearcher.search(phraseQuery, 1);
|
||||
assertEquals(0, hits.totalHits);
|
||||
final Highlighter highlighter = new Highlighter(
|
||||
new SimpleHTMLFormatter(), new SimpleHTMLEncoder(),
|
||||
new QueryScorer(phraseQuery));
|
||||
final TokenStream tokenStream = TokenSources
|
||||
.getTokenStream(indexReader.getTermVector(
|
||||
0, FIELD), false);
|
||||
assertEquals(
|
||||
highlighter.getBestFragment(new TokenStreamSparse(), TEXT),
|
||||
highlighter.getBestFragment(tokenStream, TEXT));
|
||||
} finally {
|
||||
indexSearcher.close();
|
||||
}
|
||||
TopDocs hits = indexSearcher.search(phraseQuery, 1);
|
||||
assertEquals(0, hits.totalHits);
|
||||
final Highlighter highlighter = new Highlighter(
|
||||
new SimpleHTMLFormatter(), new SimpleHTMLEncoder(),
|
||||
new QueryScorer(phraseQuery));
|
||||
final TokenStream tokenStream = TokenSources
|
||||
.getTokenStream(indexReader.getTermVector(
|
||||
0, FIELD), false);
|
||||
assertEquals(
|
||||
highlighter.getBestFragment(new TokenStreamSparse(), TEXT),
|
||||
highlighter.getBestFragment(tokenStream, TEXT));
|
||||
} finally {
|
||||
indexReader.close();
|
||||
directory.close();
|
||||
|
|
|
@ -109,7 +109,6 @@ public class HighlighterTest extends BaseTokenStreamTestCase implements Formatte
|
|||
|
||||
if (VERBOSE) System.out.println(fragment);
|
||||
}
|
||||
searcher.close();
|
||||
}
|
||||
|
||||
public void testHighlightingWithDefaultField() throws Exception {
|
||||
|
@ -1271,7 +1270,6 @@ public class HighlighterTest extends BaseTokenStreamTestCase implements Formatte
|
|||
public void run() throws Exception {
|
||||
numHighlights = 0;
|
||||
// test to show how rewritten query can still be used
|
||||
if (searcher != null) searcher.close();
|
||||
searcher = new IndexSearcher(reader);
|
||||
Analyzer analyzer = new MockAnalyzer(random, MockTokenizer.SIMPLE, true, MockTokenFilter.ENGLISH_STOPSET, true);
|
||||
|
||||
|
@ -1664,7 +1662,6 @@ public class HighlighterTest extends BaseTokenStreamTestCase implements Formatte
|
|||
if (VERBOSE) System.out.println("result:" + result);
|
||||
assertEquals("more <B>random</B> words for second field", result);
|
||||
}
|
||||
searcher.close();
|
||||
reader.close();
|
||||
}
|
||||
|
||||
|
@ -1703,7 +1700,6 @@ public class HighlighterTest extends BaseTokenStreamTestCase implements Formatte
|
|||
}
|
||||
|
||||
public void doSearching(Query unReWrittenQuery) throws Exception {
|
||||
if (searcher != null) searcher.close();
|
||||
searcher = new IndexSearcher(reader);
|
||||
// for any multi-term queries to work (prefix, wildcard, range,fuzzy etc)
|
||||
// you must use a rewritten query!
|
||||
|
@ -1769,7 +1765,6 @@ public class HighlighterTest extends BaseTokenStreamTestCase implements Formatte
|
|||
|
||||
@Override
|
||||
public void tearDown() throws Exception {
|
||||
if (searcher != null) searcher.close();
|
||||
reader.close();
|
||||
dir.close();
|
||||
ramDir.close();
|
||||
|
|
|
@ -114,31 +114,27 @@ public class TokenSourcesTest extends LuceneTestCase {
|
|||
indexWriter.close();
|
||||
}
|
||||
final IndexReader indexReader = IndexReader.open(directory);
|
||||
assertEquals(1, indexReader.numDocs());
|
||||
final IndexSearcher indexSearcher = newSearcher(indexReader);
|
||||
try {
|
||||
assertEquals(1, indexReader.numDocs());
|
||||
final IndexSearcher indexSearcher = newSearcher(indexReader);
|
||||
try {
|
||||
final DisjunctionMaxQuery query = new DisjunctionMaxQuery(1);
|
||||
query.add(new SpanTermQuery(new Term(FIELD, "{fox}")));
|
||||
query.add(new SpanTermQuery(new Term(FIELD, "fox")));
|
||||
final DisjunctionMaxQuery query = new DisjunctionMaxQuery(1);
|
||||
query.add(new SpanTermQuery(new Term(FIELD, "{fox}")));
|
||||
query.add(new SpanTermQuery(new Term(FIELD, "fox")));
|
||||
// final Query phraseQuery = new SpanNearQuery(new SpanQuery[] {
|
||||
// new SpanTermQuery(new Term(FIELD, "{fox}")),
|
||||
// new SpanTermQuery(new Term(FIELD, "fox")) }, 0, true);
|
||||
|
||||
TopDocs hits = indexSearcher.search(query, 1);
|
||||
assertEquals(1, hits.totalHits);
|
||||
final Highlighter highlighter = new Highlighter(
|
||||
new SimpleHTMLFormatter(), new SimpleHTMLEncoder(),
|
||||
new QueryScorer(query));
|
||||
final TokenStream tokenStream = TokenSources
|
||||
.getTokenStream(
|
||||
indexReader.getTermVector(0, FIELD),
|
||||
false);
|
||||
assertEquals("<B>the fox</B> did not jump",
|
||||
highlighter.getBestFragment(tokenStream, TEXT));
|
||||
} finally {
|
||||
indexSearcher.close();
|
||||
}
|
||||
TopDocs hits = indexSearcher.search(query, 1);
|
||||
assertEquals(1, hits.totalHits);
|
||||
final Highlighter highlighter = new Highlighter(
|
||||
new SimpleHTMLFormatter(), new SimpleHTMLEncoder(),
|
||||
new QueryScorer(query));
|
||||
final TokenStream tokenStream = TokenSources
|
||||
.getTokenStream(
|
||||
indexReader.getTermVector(0, FIELD),
|
||||
false);
|
||||
assertEquals("<B>the fox</B> did not jump",
|
||||
highlighter.getBestFragment(tokenStream, TEXT));
|
||||
} finally {
|
||||
indexReader.close();
|
||||
directory.close();
|
||||
|
@ -166,28 +162,24 @@ public class TokenSourcesTest extends LuceneTestCase {
|
|||
try {
|
||||
assertEquals(1, indexReader.numDocs());
|
||||
final IndexSearcher indexSearcher = newSearcher(indexReader);
|
||||
try {
|
||||
final DisjunctionMaxQuery query = new DisjunctionMaxQuery(1);
|
||||
query.add(new SpanTermQuery(new Term(FIELD, "{fox}")));
|
||||
query.add(new SpanTermQuery(new Term(FIELD, "fox")));
|
||||
// final Query phraseQuery = new SpanNearQuery(new SpanQuery[] {
|
||||
// new SpanTermQuery(new Term(FIELD, "{fox}")),
|
||||
// new SpanTermQuery(new Term(FIELD, "fox")) }, 0, true);
|
||||
final DisjunctionMaxQuery query = new DisjunctionMaxQuery(1);
|
||||
query.add(new SpanTermQuery(new Term(FIELD, "{fox}")));
|
||||
query.add(new SpanTermQuery(new Term(FIELD, "fox")));
|
||||
// final Query phraseQuery = new SpanNearQuery(new SpanQuery[] {
|
||||
// new SpanTermQuery(new Term(FIELD, "{fox}")),
|
||||
// new SpanTermQuery(new Term(FIELD, "fox")) }, 0, true);
|
||||
|
||||
TopDocs hits = indexSearcher.search(query, 1);
|
||||
assertEquals(1, hits.totalHits);
|
||||
final Highlighter highlighter = new Highlighter(
|
||||
new SimpleHTMLFormatter(), new SimpleHTMLEncoder(),
|
||||
new QueryScorer(query));
|
||||
final TokenStream tokenStream = TokenSources
|
||||
.getTokenStream(
|
||||
indexReader.getTermVector(0, FIELD),
|
||||
false);
|
||||
assertEquals("<B>the fox</B> did not jump",
|
||||
highlighter.getBestFragment(tokenStream, TEXT));
|
||||
} finally {
|
||||
indexSearcher.close();
|
||||
}
|
||||
TopDocs hits = indexSearcher.search(query, 1);
|
||||
assertEquals(1, hits.totalHits);
|
||||
final Highlighter highlighter = new Highlighter(
|
||||
new SimpleHTMLFormatter(), new SimpleHTMLEncoder(),
|
||||
new QueryScorer(query));
|
||||
final TokenStream tokenStream = TokenSources
|
||||
.getTokenStream(
|
||||
indexReader.getTermVector(0, FIELD),
|
||||
false);
|
||||
assertEquals("<B>the fox</B> did not jump",
|
||||
highlighter.getBestFragment(tokenStream, TEXT));
|
||||
} finally {
|
||||
indexReader.close();
|
||||
directory.close();
|
||||
|
@ -214,28 +206,24 @@ public class TokenSourcesTest extends LuceneTestCase {
|
|||
try {
|
||||
assertEquals(1, indexReader.numDocs());
|
||||
final IndexSearcher indexSearcher = newSearcher(indexReader);
|
||||
try {
|
||||
// final DisjunctionMaxQuery query = new DisjunctionMaxQuery(1);
|
||||
// query.add(new SpanTermQuery(new Term(FIELD, "{fox}")));
|
||||
// query.add(new SpanTermQuery(new Term(FIELD, "fox")));
|
||||
final Query phraseQuery = new SpanNearQuery(new SpanQuery[] {
|
||||
new SpanTermQuery(new Term(FIELD, "the")),
|
||||
new SpanTermQuery(new Term(FIELD, "fox"))}, 0, true);
|
||||
// final DisjunctionMaxQuery query = new DisjunctionMaxQuery(1);
|
||||
// query.add(new SpanTermQuery(new Term(FIELD, "{fox}")));
|
||||
// query.add(new SpanTermQuery(new Term(FIELD, "fox")));
|
||||
final Query phraseQuery = new SpanNearQuery(new SpanQuery[] {
|
||||
new SpanTermQuery(new Term(FIELD, "the")),
|
||||
new SpanTermQuery(new Term(FIELD, "fox"))}, 0, true);
|
||||
|
||||
TopDocs hits = indexSearcher.search(phraseQuery, 1);
|
||||
assertEquals(1, hits.totalHits);
|
||||
final Highlighter highlighter = new Highlighter(
|
||||
new SimpleHTMLFormatter(), new SimpleHTMLEncoder(),
|
||||
new QueryScorer(phraseQuery));
|
||||
final TokenStream tokenStream = TokenSources
|
||||
.getTokenStream(
|
||||
indexReader.getTermVector(0, FIELD),
|
||||
false);
|
||||
assertEquals("<B>the fox</B> did not jump",
|
||||
highlighter.getBestFragment(tokenStream, TEXT));
|
||||
} finally {
|
||||
indexSearcher.close();
|
||||
}
|
||||
TopDocs hits = indexSearcher.search(phraseQuery, 1);
|
||||
assertEquals(1, hits.totalHits);
|
||||
final Highlighter highlighter = new Highlighter(
|
||||
new SimpleHTMLFormatter(), new SimpleHTMLEncoder(),
|
||||
new QueryScorer(phraseQuery));
|
||||
final TokenStream tokenStream = TokenSources
|
||||
.getTokenStream(
|
||||
indexReader.getTermVector(0, FIELD),
|
||||
false);
|
||||
assertEquals("<B>the fox</B> did not jump",
|
||||
highlighter.getBestFragment(tokenStream, TEXT));
|
||||
} finally {
|
||||
indexReader.close();
|
||||
directory.close();
|
||||
|
@ -263,28 +251,24 @@ public class TokenSourcesTest extends LuceneTestCase {
|
|||
try {
|
||||
assertEquals(1, indexReader.numDocs());
|
||||
final IndexSearcher indexSearcher = newSearcher(indexReader);
|
||||
try {
|
||||
// final DisjunctionMaxQuery query = new DisjunctionMaxQuery(1);
|
||||
// query.add(new SpanTermQuery(new Term(FIELD, "the")));
|
||||
// query.add(new SpanTermQuery(new Term(FIELD, "fox")));
|
||||
final Query phraseQuery = new SpanNearQuery(new SpanQuery[] {
|
||||
new SpanTermQuery(new Term(FIELD, "the")),
|
||||
new SpanTermQuery(new Term(FIELD, "fox"))}, 0, true);
|
||||
// final DisjunctionMaxQuery query = new DisjunctionMaxQuery(1);
|
||||
// query.add(new SpanTermQuery(new Term(FIELD, "the")));
|
||||
// query.add(new SpanTermQuery(new Term(FIELD, "fox")));
|
||||
final Query phraseQuery = new SpanNearQuery(new SpanQuery[] {
|
||||
new SpanTermQuery(new Term(FIELD, "the")),
|
||||
new SpanTermQuery(new Term(FIELD, "fox"))}, 0, true);
|
||||
|
||||
TopDocs hits = indexSearcher.search(phraseQuery, 1);
|
||||
assertEquals(1, hits.totalHits);
|
||||
final Highlighter highlighter = new Highlighter(
|
||||
new SimpleHTMLFormatter(), new SimpleHTMLEncoder(),
|
||||
new QueryScorer(phraseQuery));
|
||||
final TokenStream tokenStream = TokenSources
|
||||
.getTokenStream(
|
||||
indexReader.getTermVector(0, FIELD),
|
||||
false);
|
||||
assertEquals("<B>the fox</B> did not jump",
|
||||
highlighter.getBestFragment(tokenStream, TEXT));
|
||||
} finally {
|
||||
indexSearcher.close();
|
||||
}
|
||||
TopDocs hits = indexSearcher.search(phraseQuery, 1);
|
||||
assertEquals(1, hits.totalHits);
|
||||
final Highlighter highlighter = new Highlighter(
|
||||
new SimpleHTMLFormatter(), new SimpleHTMLEncoder(),
|
||||
new QueryScorer(phraseQuery));
|
||||
final TokenStream tokenStream = TokenSources
|
||||
.getTokenStream(
|
||||
indexReader.getTermVector(0, FIELD),
|
||||
false);
|
||||
assertEquals("<B>the fox</B> did not jump",
|
||||
highlighter.getBestFragment(tokenStream, TEXT));
|
||||
} finally {
|
||||
indexReader.close();
|
||||
directory.close();
|
||||
|
|
|
@ -143,9 +143,7 @@ public class MemoryIndexTest extends BaseTokenStreamTestCase {
|
|||
TopDocs memDocs = mem.search(qp.parse(query), 1);
|
||||
assertEquals(ramDocs.totalHits, memDocs.totalHits);
|
||||
}
|
||||
ram.close();
|
||||
reader.close();
|
||||
mem.close();
|
||||
}
|
||||
|
||||
/**
|
||||
|
|
|
@ -21,12 +21,12 @@ import java.io.File;
|
|||
import java.io.IOException;
|
||||
import java.util.ArrayList;
|
||||
|
||||
import org.apache.lucene.index.IndexWriter; // javadoc
|
||||
import org.apache.lucene.index.IndexWriterConfig.OpenMode;
|
||||
import org.apache.lucene.store.Directory;
|
||||
import org.apache.lucene.store.FSDirectory;
|
||||
import org.apache.lucene.util.FixedBitSet;
|
||||
import org.apache.lucene.util.Bits;
|
||||
import org.apache.lucene.util.ReaderUtil;
|
||||
import org.apache.lucene.util.Version;
|
||||
|
||||
/**
|
||||
|
@ -174,15 +174,60 @@ public class MultiPassIndexSplitter {
|
|||
}
|
||||
|
||||
/**
|
||||
* This class pretends that it can write deletions to the underlying index.
|
||||
* Instead, deletions are buffered in a bitset and overlaid with the original
|
||||
* list of deletions.
|
||||
* This class emulates deletions on the underlying index.
|
||||
*/
|
||||
public static final class FakeDeleteIndexReader extends FilterIndexReader {
|
||||
private static final class FakeDeleteIndexReader extends MultiReader {
|
||||
|
||||
public FakeDeleteIndexReader(IndexReader reader) throws IOException {
|
||||
super(initSubReaders(reader), false /* dont close */);
|
||||
}
|
||||
|
||||
private static IndexReader[] initSubReaders(IndexReader reader) throws IOException {
|
||||
final ArrayList<IndexReader> subs = new ArrayList<IndexReader>();
|
||||
new ReaderUtil.Gather(reader) {
|
||||
@Override
|
||||
protected void add(int base, IndexReader r) {
|
||||
subs.add(new FakeDeleteAtomicIndexReader(r));
|
||||
}
|
||||
}.run();
|
||||
return subs.toArray(new IndexReader[subs.size()]);
|
||||
}
|
||||
|
||||
public void deleteDocument(int docID) {
|
||||
final int i = readerIndex(docID);
|
||||
((FakeDeleteAtomicIndexReader) subReaders[i]).deleteDocument(docID - starts[i]);
|
||||
}
|
||||
|
||||
public void undeleteAll() {
|
||||
for (IndexReader r : subReaders) {
|
||||
((FakeDeleteAtomicIndexReader) r).undeleteAll();
|
||||
}
|
||||
}
|
||||
|
||||
// override this as MultiReader precalculates the number of deletions
|
||||
// (this method is never used by MultiPassIndexSplitter)
|
||||
@Override
|
||||
public int numDocs() {
|
||||
int n = 0;
|
||||
for (int i = 0; i < subReaders.length; i++)
|
||||
n += subReaders[i].numDocs();
|
||||
return n;
|
||||
}
|
||||
|
||||
// override this as MultiReader precalculates the number of deletions
|
||||
// (this method is never used by MultiPassIndexSplitter)
|
||||
@Override
|
||||
public boolean hasDeletions() {
|
||||
return (maxDoc() != numDocs());
|
||||
}
|
||||
|
||||
}
|
||||
|
||||
private static final class FakeDeleteAtomicIndexReader extends FilterIndexReader {
|
||||
FixedBitSet liveDocs;
|
||||
|
||||
public FakeDeleteIndexReader(IndexReader in) {
|
||||
super(new SlowMultiReaderWrapper(in));
|
||||
public FakeDeleteAtomicIndexReader(IndexReader reader) {
|
||||
super(reader);
|
||||
undeleteAll(); // initialize main bitset
|
||||
}
|
||||
|
||||
|
@ -191,7 +236,7 @@ public class MultiPassIndexSplitter {
|
|||
return liveDocs.cardinality();
|
||||
}
|
||||
|
||||
void undeleteAll() {
|
||||
public void undeleteAll() {
|
||||
final int maxDoc = in.maxDoc();
|
||||
liveDocs = new FixedBitSet(in.maxDoc());
|
||||
if (in.hasDeletions()) {
|
||||
|
@ -207,7 +252,7 @@ public class MultiPassIndexSplitter {
|
|||
}
|
||||
}
|
||||
|
||||
void deleteDocument(int n) {
|
||||
public void deleteDocument(int n) {
|
||||
liveDocs.clear(n);
|
||||
}
|
||||
|
||||
|
|
|
@ -20,6 +20,7 @@ package org.apache.lucene.index;
|
|||
import java.io.IOException;
|
||||
|
||||
import org.apache.lucene.index.IndexWriterConfig.OpenMode;
|
||||
import org.apache.lucene.index.IndexReader.AtomicReaderContext;
|
||||
import org.apache.lucene.search.DocIdSet;
|
||||
import org.apache.lucene.search.DocIdSetIterator;
|
||||
import org.apache.lucene.search.Filter;
|
||||
|
@ -28,6 +29,7 @@ import org.apache.lucene.store.Directory;
|
|||
import org.apache.lucene.util.Bits;
|
||||
import org.apache.lucene.util.FixedBitSet;
|
||||
import org.apache.lucene.util.IOUtils;
|
||||
import org.apache.lucene.util.ReaderUtil;
|
||||
import org.apache.lucene.util.Version;
|
||||
|
||||
/**
|
||||
|
@ -99,9 +101,14 @@ public class PKIndexSplitter {
|
|||
|
||||
private void createIndex(IndexWriterConfig config, Directory target, IndexReader reader, Filter preserveFilter, boolean negateFilter) throws IOException {
|
||||
boolean success = false;
|
||||
IndexWriter w = new IndexWriter(target, config);
|
||||
final IndexWriter w = new IndexWriter(target, config);
|
||||
try {
|
||||
w.addIndexes(new DocumentFilteredIndexReader(reader, preserveFilter, negateFilter));
|
||||
final AtomicReaderContext[] leaves = ReaderUtil.leaves(reader.getTopReaderContext());
|
||||
final IndexReader[] subReaders = new IndexReader[leaves.length];
|
||||
for (int i = 0; i < leaves.length; i++) {
|
||||
subReaders[i] = new DocumentFilteredAtomicIndexReader(leaves[i], preserveFilter, negateFilter);
|
||||
}
|
||||
w.addIndexes(subReaders);
|
||||
success = true;
|
||||
} finally {
|
||||
if (success) {
|
||||
|
@ -112,17 +119,16 @@ public class PKIndexSplitter {
|
|||
}
|
||||
}
|
||||
|
||||
public static class DocumentFilteredIndexReader extends FilterIndexReader {
|
||||
private static class DocumentFilteredAtomicIndexReader extends FilterIndexReader {
|
||||
final Bits liveDocs;
|
||||
final int numDocs;
|
||||
|
||||
public DocumentFilteredIndexReader(IndexReader reader, Filter preserveFilter, boolean negateFilter) throws IOException {
|
||||
super(new SlowMultiReaderWrapper(reader));
|
||||
|
||||
public DocumentFilteredAtomicIndexReader(AtomicReaderContext context, Filter preserveFilter, boolean negateFilter) throws IOException {
|
||||
super(context.reader);
|
||||
final int maxDoc = in.maxDoc();
|
||||
final FixedBitSet bits = new FixedBitSet(maxDoc);
|
||||
// ignore livedocs here, as we filter them later:
|
||||
final DocIdSet docs = preserveFilter.getDocIdSet((AtomicReaderContext) in.getTopReaderContext(), null);
|
||||
final DocIdSet docs = preserveFilter.getDocIdSet(context, null);
|
||||
if (docs != null) {
|
||||
final DocIdSetIterator it = docs.iterator();
|
||||
if (it != null) {
|
||||
|
|
|
@ -23,7 +23,6 @@ import org.apache.lucene.index.Fields;
|
|||
import org.apache.lucene.index.TermsEnum;
|
||||
import org.apache.lucene.index.FieldsEnum;
|
||||
import org.apache.lucene.index.Terms;
|
||||
import org.apache.lucene.index.FieldReaderException;
|
||||
import org.apache.lucene.index.DocsEnum;
|
||||
import org.apache.lucene.search.DocIdSetIterator;
|
||||
import org.apache.lucene.store.FSDirectory;
|
||||
|
@ -117,7 +116,7 @@ public class HighFreqTerms {
|
|||
if (field != null) {
|
||||
Fields fields = MultiFields.getFields(reader);
|
||||
if (fields == null) {
|
||||
throw new FieldReaderException("field " + field + " not found");
|
||||
throw new RuntimeException("field " + field + " not found");
|
||||
}
|
||||
Terms terms = fields.terms(field);
|
||||
if (terms != null) {
|
||||
|
@ -128,7 +127,7 @@ public class HighFreqTerms {
|
|||
} else {
|
||||
Fields fields = MultiFields.getFields(reader);
|
||||
if (fields == null) {
|
||||
throw new FieldReaderException("no fields found for this index");
|
||||
throw new RuntimeException("no fields found for this index");
|
||||
}
|
||||
tiq = new TermStatsQueue(numTerms);
|
||||
FieldsEnum fieldsEnum = fields.iterator();
|
||||
|
|
|
@ -33,13 +33,14 @@ public class TestMultiPassIndexSplitter extends LuceneTestCase {
|
|||
public void setUp() throws Exception {
|
||||
super.setUp();
|
||||
dir = newDirectory();
|
||||
IndexWriter w = new IndexWriter(dir, newIndexWriterConfig(TEST_VERSION_CURRENT, new MockAnalyzer(random)).setMergePolicy(newLogMergePolicy()));
|
||||
IndexWriter w = new IndexWriter(dir, newIndexWriterConfig(TEST_VERSION_CURRENT, new MockAnalyzer(random)).setMergePolicy(NoMergePolicy.COMPOUND_FILES));
|
||||
Document doc;
|
||||
for (int i = 0; i < NUM_DOCS; i++) {
|
||||
doc = new Document();
|
||||
doc.add(newField("id", i + "", StringField.TYPE_STORED));
|
||||
doc.add(newField("f", i + " " + i, TextField.TYPE_STORED));
|
||||
w.addDocument(doc);
|
||||
if (i%3==0) w.commit();
|
||||
}
|
||||
w.commit();
|
||||
w.deleteDocuments(new Term("id", "" + (NUM_DOCS-1)));
|
||||
|
|
|
@ -37,14 +37,16 @@ public class TestPKIndexSplitter extends LuceneTestCase {
|
|||
Directory dir = newDirectory();
|
||||
IndexWriter w = new IndexWriter(dir, newIndexWriterConfig(
|
||||
TEST_VERSION_CURRENT, new MockAnalyzer(random, MockTokenizer.WHITESPACE, false))
|
||||
.setOpenMode(OpenMode.CREATE));
|
||||
.setOpenMode(OpenMode.CREATE).setMergePolicy(NoMergePolicy.COMPOUND_FILES));
|
||||
for (int x = 0; x < 11; x++) {
|
||||
Document doc = createDocument(x, "1", 3, format);
|
||||
w.addDocument(doc);
|
||||
if (x%3==0) w.commit();
|
||||
}
|
||||
for (int x = 11; x < 20; x++) {
|
||||
Document doc = createDocument(x, "2", 3, format);
|
||||
w.addDocument(doc);
|
||||
if (x%3==0) w.commit();
|
||||
}
|
||||
w.close();
|
||||
|
||||
|
@ -55,7 +57,7 @@ public class TestPKIndexSplitter extends LuceneTestCase {
|
|||
// delete some documents
|
||||
w = new IndexWriter(dir, newIndexWriterConfig(
|
||||
TEST_VERSION_CURRENT, new MockAnalyzer(random, MockTokenizer.WHITESPACE, false))
|
||||
.setOpenMode(OpenMode.APPEND));
|
||||
.setOpenMode(OpenMode.APPEND).setMergePolicy(NoMergePolicy.COMPOUND_FILES));
|
||||
w.deleteDocuments(midTerm);
|
||||
w.deleteDocuments(new Term("id", format.format(2)));
|
||||
w.close();
|
||||
|
|
|
@ -69,7 +69,6 @@ public class DuplicateFilterTest extends LuceneTestCase {
|
|||
@Override
|
||||
public void tearDown() throws Exception {
|
||||
reader.close();
|
||||
searcher.close();
|
||||
directory.close();
|
||||
super.tearDown();
|
||||
}
|
||||
|
|
|
@ -60,7 +60,6 @@ public class FuzzyLikeThisQueryTest extends LuceneTestCase {
|
|||
|
||||
@Override
|
||||
public void tearDown() throws Exception {
|
||||
searcher.close();
|
||||
reader.close();
|
||||
directory.close();
|
||||
super.tearDown();
|
||||
|
|
|
@ -71,7 +71,6 @@ public class TestSlowCollationMethods extends LuceneTestCase {
|
|||
|
||||
@AfterClass
|
||||
public static void afterClass() throws Exception {
|
||||
searcher.close();
|
||||
reader.close();
|
||||
dir.close();
|
||||
collator = null;
|
||||
|
|
|
@ -56,7 +56,6 @@ public class TestRegexQuery extends LuceneTestCase {
|
|||
|
||||
@Override
|
||||
public void tearDown() throws Exception {
|
||||
searcher.close();
|
||||
reader.close();
|
||||
directory.close();
|
||||
super.tearDown();
|
||||
|
|
|
@ -80,7 +80,6 @@ public class TestSpanRegexQuery extends LuceneTestCase {
|
|||
// true);
|
||||
int numHits = searcher.search(sfq, null, 1000).totalHits;
|
||||
assertEquals(1, numHits);
|
||||
searcher.close();
|
||||
reader.close();
|
||||
directory.close();
|
||||
}
|
||||
|
|
|
@ -260,7 +260,6 @@ public class TestCartesian extends LuceneTestCase {
|
|||
assertTrue(geo_distance >= lastDistance);
|
||||
lastDistance = geo_distance;
|
||||
}
|
||||
searcher.close();
|
||||
reader.close();
|
||||
}
|
||||
|
||||
|
@ -332,7 +331,6 @@ public class TestCartesian extends LuceneTestCase {
|
|||
assertTrue(geo_distance >= lastDistance);
|
||||
lastDistance = geo_distance;
|
||||
}
|
||||
searcher.close();
|
||||
reader.close();
|
||||
}
|
||||
|
||||
|
@ -404,7 +402,6 @@ public class TestCartesian extends LuceneTestCase {
|
|||
lastDistance = geo_distance;
|
||||
}
|
||||
}
|
||||
searcher.close();
|
||||
reader.close();
|
||||
}
|
||||
|
||||
|
@ -476,7 +473,6 @@ public class TestCartesian extends LuceneTestCase {
|
|||
|
||||
}
|
||||
}
|
||||
searcher.close();
|
||||
reader.close();
|
||||
}
|
||||
}
|
||||
|
|
|
@ -0,0 +1,163 @@
|
|||
package org.apache.lucene.index;
|
||||
|
||||
/**
|
||||
* Licensed to the Apache Software Foundation (ASF) under one or more
|
||||
* contributor license agreements. See the NOTICE file distributed with
|
||||
* this work for additional information regarding copyright ownership.
|
||||
* The ASF licenses this file to You under the Apache License, Version 2.0
|
||||
* (the "License"); you may not use this file except in compliance with
|
||||
* the License. You may obtain a copy of the License at
|
||||
*
|
||||
* http://www.apache.org/licenses/LICENSE-2.0
|
||||
*
|
||||
* Unless required by applicable law or agreed to in writing, software
|
||||
* distributed under the License is distributed on an "AS IS" BASIS,
|
||||
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
|
||||
* See the License for the specific language governing permissions and
|
||||
* limitations under the License.
|
||||
*/
|
||||
|
||||
import java.io.IOException;
|
||||
import java.util.Collection;
|
||||
import java.util.HashSet;
|
||||
import java.util.Set;
|
||||
|
||||
import org.apache.lucene.util.Bits;
|
||||
import org.apache.lucene.util.BytesRef;
|
||||
import org.apache.lucene.util.ReaderUtil;
|
||||
|
||||
abstract class BaseMultiReader<R extends IndexReader> extends IndexReader implements Cloneable {
|
||||
protected final R[] subReaders;
|
||||
protected final int[] starts; // 1st docno for each segment
|
||||
private final ReaderContext topLevelContext;
|
||||
private final int maxDoc;
|
||||
private final int numDocs;
|
||||
private final boolean hasDeletions;
|
||||
|
||||
protected BaseMultiReader(R[] subReaders) throws IOException {
|
||||
this.subReaders = subReaders;
|
||||
starts = new int[subReaders.length + 1]; // build starts array
|
||||
int maxDoc = 0, numDocs = 0;
|
||||
boolean hasDeletions = false;
|
||||
for (int i = 0; i < subReaders.length; i++) {
|
||||
starts[i] = maxDoc;
|
||||
maxDoc += subReaders[i].maxDoc(); // compute maxDocs
|
||||
numDocs += subReaders[i].numDocs(); // compute numDocs
|
||||
|
||||
if (subReaders[i].hasDeletions()) {
|
||||
hasDeletions = true;
|
||||
}
|
||||
}
|
||||
starts[subReaders.length] = maxDoc;
|
||||
this.maxDoc = maxDoc;
|
||||
this.numDocs = numDocs;
|
||||
this.hasDeletions = hasDeletions;
|
||||
topLevelContext = ReaderUtil.buildReaderContext(this);
|
||||
}
|
||||
|
||||
@Override
|
||||
public Fields fields() throws IOException {
|
||||
throw new UnsupportedOperationException("please use MultiFields.getFields, or wrap your IndexReader with SlowMultiReaderWrapper, if you really need a top level Fields");
|
||||
}
|
||||
|
||||
@Override
|
||||
protected abstract IndexReader doOpenIfChanged() throws CorruptIndexException, IOException;
|
||||
|
||||
@Override
|
||||
public abstract Object clone();
|
||||
|
||||
@Override
|
||||
public Bits getLiveDocs() {
|
||||
throw new UnsupportedOperationException("please use MultiFields.getLiveDocs, or wrap your IndexReader with SlowMultiReaderWrapper, if you really need a top level Bits liveDocs");
|
||||
}
|
||||
|
||||
@Override
|
||||
public Fields getTermVectors(int docID) throws IOException {
|
||||
ensureOpen();
|
||||
final int i = readerIndex(docID); // find segment num
|
||||
return subReaders[i].getTermVectors(docID - starts[i]); // dispatch to segment
|
||||
}
|
||||
|
||||
@Override
|
||||
public int numDocs() {
|
||||
// Don't call ensureOpen() here (it could affect performance)
|
||||
return numDocs;
|
||||
}
|
||||
|
||||
@Override
|
||||
public final int maxDoc() {
|
||||
// Don't call ensureOpen() here (it could affect performance)
|
||||
return maxDoc;
|
||||
}
|
||||
|
||||
@Override
|
||||
public void document(int docID, StoredFieldVisitor visitor) throws CorruptIndexException, IOException {
|
||||
ensureOpen();
|
||||
final int i = readerIndex(docID); // find segment num
|
||||
subReaders[i].document(docID - starts[i], visitor); // dispatch to segment reader
|
||||
}
|
||||
|
||||
@Override
|
||||
public boolean hasDeletions() {
|
||||
// Don't call ensureOpen() here (it could affect performance)
|
||||
return hasDeletions;
|
||||
}
|
||||
|
||||
/** Helper method for subclasses to get the corresponding reader for a doc ID */
|
||||
protected final int readerIndex(int docID) {
|
||||
if (docID < 0 || docID >= maxDoc) {
|
||||
throw new IllegalArgumentException("docID must be >= 0 and < maxDoc=" + maxDoc + " (got docID=" + docID + ")");
|
||||
}
|
||||
return ReaderUtil.subIndex(docID, this.starts);
|
||||
}
|
||||
|
||||
@Override
|
||||
public boolean hasNorms(String field) throws IOException {
|
||||
ensureOpen();
|
||||
for (int i = 0; i < subReaders.length; i++) {
|
||||
if (subReaders[i].hasNorms(field)) return true;
|
||||
}
|
||||
return false;
|
||||
}
|
||||
|
||||
@Override
|
||||
public synchronized byte[] norms(String field) throws IOException {
|
||||
throw new UnsupportedOperationException("please use MultiNorms.norms, or wrap your IndexReader with SlowMultiReaderWrapper, if you really need a top level norms");
|
||||
}
|
||||
|
||||
@Override
|
||||
public int docFreq(String field, BytesRef t) throws IOException {
|
||||
ensureOpen();
|
||||
int total = 0; // sum freqs in segments
|
||||
for (int i = 0; i < subReaders.length; i++) {
|
||||
total += subReaders[i].docFreq(field, t);
|
||||
}
|
||||
return total;
|
||||
}
|
||||
|
||||
@Override
|
||||
public Collection<String> getFieldNames (IndexReader.FieldOption fieldNames) {
|
||||
ensureOpen();
|
||||
// maintain a unique set of field names
|
||||
final Set<String> fieldSet = new HashSet<String>();
|
||||
for (IndexReader reader : subReaders) {
|
||||
fieldSet.addAll(reader.getFieldNames(fieldNames));
|
||||
}
|
||||
return fieldSet;
|
||||
}
|
||||
|
||||
@Override
|
||||
public IndexReader[] getSequentialSubReaders() {
|
||||
return subReaders;
|
||||
}
|
||||
|
||||
@Override
|
||||
public ReaderContext getTopReaderContext() {
|
||||
return topLevelContext;
|
||||
}
|
||||
|
||||
@Override
|
||||
public DocValues docValues(String field) throws IOException {
|
||||
throw new UnsupportedOperationException("please use MultiDocValues#getDocValues, or wrap your IndexReader with SlowMultiReaderWrapper, if you really need a top level DocValues");
|
||||
}
|
||||
}
|
|
@ -18,7 +18,6 @@ package org.apache.lucene.index;
|
|||
*/
|
||||
|
||||
import org.apache.lucene.store.Directory;
|
||||
import org.apache.lucene.util.InfoStream;
|
||||
import org.apache.lucene.util.ThreadInterruptedException;
|
||||
import org.apache.lucene.util.CollectionUtil;
|
||||
|
||||
|
@ -336,8 +335,9 @@ public class ConcurrentMergeScheduler extends MergeScheduler {
|
|||
// merges to do first (the easiest ones?), etc.
|
||||
MergePolicy.OneMerge merge = writer.getNextMerge();
|
||||
if (merge == null) {
|
||||
if (verbose())
|
||||
if (verbose()) {
|
||||
message(" no more merges pending; now return");
|
||||
}
|
||||
return;
|
||||
}
|
||||
|
||||
|
@ -348,7 +348,9 @@ public class ConcurrentMergeScheduler extends MergeScheduler {
|
|||
boolean success = false;
|
||||
try {
|
||||
synchronized(this) {
|
||||
message(" consider merge " + merge.segString(dir));
|
||||
if (verbose()) {
|
||||
message(" consider merge " + merge.segString(dir));
|
||||
}
|
||||
|
||||
// OK to spawn a new merge thread to handle this
|
||||
// merge:
|
||||
|
@ -440,8 +442,9 @@ public class ConcurrentMergeScheduler extends MergeScheduler {
|
|||
|
||||
try {
|
||||
|
||||
if (verbose())
|
||||
if (verbose()) {
|
||||
message(" merge thread: start");
|
||||
}
|
||||
|
||||
while(true) {
|
||||
setRunningMerge(merge);
|
||||
|
@ -453,15 +456,17 @@ public class ConcurrentMergeScheduler extends MergeScheduler {
|
|||
if (merge != null) {
|
||||
tWriter.mergeInit(merge);
|
||||
updateMergeThreads();
|
||||
if (verbose())
|
||||
if (verbose()) {
|
||||
message(" merge thread: do another merge " + merge.segString(dir));
|
||||
}
|
||||
} else {
|
||||
break;
|
||||
}
|
||||
}
|
||||
|
||||
if (verbose())
|
||||
if (verbose()) {
|
||||
message(" merge thread: done");
|
||||
}
|
||||
|
||||
} catch (Throwable exc) {
|
||||
|
||||
|
|
|
@ -23,95 +23,70 @@ import java.util.ArrayList;
|
|||
import java.util.Collection;
|
||||
import java.util.Collections;
|
||||
import java.util.HashMap;
|
||||
import java.util.HashSet;
|
||||
import java.util.List;
|
||||
import java.util.Map;
|
||||
import java.util.Set;
|
||||
import java.util.concurrent.ConcurrentHashMap;
|
||||
|
||||
import org.apache.lucene.store.Directory;
|
||||
import org.apache.lucene.store.IOContext;
|
||||
import org.apache.lucene.util.Bits;
|
||||
import org.apache.lucene.util.BytesRef;
|
||||
import org.apache.lucene.util.IOUtils;
|
||||
import org.apache.lucene.util.MapBackedSet;
|
||||
|
||||
/**
|
||||
* An IndexReader which reads indexes with multiple segments.
|
||||
*/
|
||||
class DirectoryReader extends IndexReader implements Cloneable {
|
||||
protected Directory directory;
|
||||
|
||||
IndexWriter writer;
|
||||
|
||||
final class DirectoryReader extends BaseMultiReader<SegmentReader> {
|
||||
protected final Directory directory;
|
||||
private final IndexWriter writer;
|
||||
private final SegmentInfos segmentInfos;
|
||||
private final int termInfosIndexDivisor;
|
||||
|
||||
private SegmentReader[] subReaders;
|
||||
private ReaderContext topLevelReaderContext;
|
||||
private int[] starts; // 1st docno for each segment
|
||||
private int maxDoc = 0;
|
||||
private int numDocs = -1;
|
||||
private boolean hasDeletions = false;
|
||||
|
||||
private final boolean applyAllDeletes;
|
||||
|
||||
DirectoryReader(SegmentReader[] readers, Directory directory, IndexWriter writer,
|
||||
SegmentInfos sis, int termInfosIndexDivisor, boolean applyAllDeletes,
|
||||
Collection<ReaderFinishedListener> readerFinishedListeners
|
||||
) throws IOException {
|
||||
super(readers);
|
||||
this.directory = directory;
|
||||
this.writer = writer;
|
||||
this.segmentInfos = sis;
|
||||
this.termInfosIndexDivisor = termInfosIndexDivisor;
|
||||
this.readerFinishedListeners = readerFinishedListeners;
|
||||
this.applyAllDeletes = applyAllDeletes;
|
||||
}
|
||||
|
||||
static IndexReader open(final Directory directory, final IndexCommit commit,
|
||||
final int termInfosIndexDivisor) throws CorruptIndexException, IOException {
|
||||
return (IndexReader) new SegmentInfos.FindSegmentsFile(directory) {
|
||||
@Override
|
||||
protected Object doBody(String segmentFileName) throws CorruptIndexException, IOException {
|
||||
SegmentInfos infos = new SegmentInfos();
|
||||
infos.read(directory, segmentFileName);
|
||||
return new DirectoryReader(directory, infos, termInfosIndexDivisor);
|
||||
final Collection<ReaderFinishedListener> readerFinishedListeners =
|
||||
new MapBackedSet<ReaderFinishedListener>(new ConcurrentHashMap<ReaderFinishedListener,Boolean>());
|
||||
SegmentInfos sis = new SegmentInfos();
|
||||
sis.read(directory, segmentFileName);
|
||||
final SegmentReader[] readers = new SegmentReader[sis.size()];
|
||||
for (int i = sis.size()-1; i >= 0; i--) {
|
||||
IOException prior = null;
|
||||
boolean success = false;
|
||||
try {
|
||||
readers[i] = SegmentReader.get(sis.info(i), termInfosIndexDivisor, IOContext.READ);
|
||||
readers[i].readerFinishedListeners = readerFinishedListeners;
|
||||
success = true;
|
||||
} catch(IOException ex) {
|
||||
prior = ex;
|
||||
} finally {
|
||||
if (!success)
|
||||
IOUtils.closeWhileHandlingException(prior, readers);
|
||||
}
|
||||
}
|
||||
return new DirectoryReader(readers, directory, null, sis, termInfosIndexDivisor,
|
||||
false, readerFinishedListeners);
|
||||
}
|
||||
}.run(commit);
|
||||
}
|
||||
|
||||
/** Construct reading the named set of readers. */
|
||||
DirectoryReader(Directory directory, SegmentInfos sis, int termInfosIndexDivisor) throws IOException {
|
||||
this.directory = directory;
|
||||
this.segmentInfos = sis;
|
||||
this.termInfosIndexDivisor = termInfosIndexDivisor;
|
||||
readerFinishedListeners = new MapBackedSet<ReaderFinishedListener>(new ConcurrentHashMap<ReaderFinishedListener,Boolean>());
|
||||
applyAllDeletes = false;
|
||||
|
||||
// To reduce the chance of hitting FileNotFound
|
||||
// (and having to retry), we open segments in
|
||||
// reverse because IndexWriter merges & deletes
|
||||
// the newest segments first.
|
||||
|
||||
SegmentReader[] readers = new SegmentReader[sis.size()];
|
||||
for (int i = sis.size()-1; i >= 0; i--) {
|
||||
boolean success = false;
|
||||
try {
|
||||
readers[i] = SegmentReader.get(sis.info(i), termInfosIndexDivisor, IOContext.READ);
|
||||
readers[i].readerFinishedListeners = readerFinishedListeners;
|
||||
success = true;
|
||||
} finally {
|
||||
if (!success) {
|
||||
// Close all readers we had opened:
|
||||
for(i++;i<sis.size();i++) {
|
||||
try {
|
||||
readers[i].close();
|
||||
} catch (Throwable ignore) {
|
||||
// keep going - we want to clean up as much as possible
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
initialize(readers);
|
||||
}
|
||||
|
||||
// Used by near real-time search
|
||||
DirectoryReader(IndexWriter writer, SegmentInfos infos, boolean applyAllDeletes) throws IOException {
|
||||
this.directory = writer.getDirectory();
|
||||
this.applyAllDeletes = applyAllDeletes; // saved for reopen
|
||||
|
||||
this.termInfosIndexDivisor = writer.getConfig().getReaderTermsIndexDivisor();
|
||||
readerFinishedListeners = writer.getReaderFinishedListeners();
|
||||
|
||||
static DirectoryReader open(IndexWriter writer, SegmentInfos infos, boolean applyAllDeletes) throws IOException {
|
||||
// IndexWriter synchronizes externally before calling
|
||||
// us, which ensures infos will not change; so there's
|
||||
// no need to process segments in reverse order
|
||||
|
@ -120,16 +95,17 @@ class DirectoryReader extends IndexReader implements Cloneable {
|
|||
List<SegmentReader> readers = new ArrayList<SegmentReader>();
|
||||
final Directory dir = writer.getDirectory();
|
||||
|
||||
segmentInfos = (SegmentInfos) infos.clone();
|
||||
final SegmentInfos segmentInfos = (SegmentInfos) infos.clone();
|
||||
int infosUpto = 0;
|
||||
for (int i=0;i<numSegments;i++) {
|
||||
IOException prior = null;
|
||||
boolean success = false;
|
||||
try {
|
||||
final SegmentInfo info = infos.info(i);
|
||||
assert info.dir == dir;
|
||||
final SegmentReader reader = writer.readerPool.getReadOnlyClone(info, IOContext.READ);
|
||||
if (reader.numDocs() > 0 || writer.getKeepFullyDeletedSegments()) {
|
||||
reader.readerFinishedListeners = readerFinishedListeners;
|
||||
reader.readerFinishedListeners = writer.getReaderFinishedListeners();
|
||||
readers.add(reader);
|
||||
infosUpto++;
|
||||
} else {
|
||||
|
@ -137,37 +113,25 @@ class DirectoryReader extends IndexReader implements Cloneable {
|
|||
segmentInfos.remove(infosUpto);
|
||||
}
|
||||
success = true;
|
||||
} catch(IOException ex) {
|
||||
prior = ex;
|
||||
} finally {
|
||||
if (!success) {
|
||||
// Close all readers we had opened:
|
||||
for(SegmentReader reader : readers) {
|
||||
try {
|
||||
reader.close();
|
||||
} catch (Throwable ignore) {
|
||||
// keep going - we want to clean up as much as possible
|
||||
}
|
||||
}
|
||||
}
|
||||
if (!success)
|
||||
IOUtils.closeWhileHandlingException(prior, readers);
|
||||
}
|
||||
}
|
||||
|
||||
this.writer = writer;
|
||||
|
||||
initialize(readers.toArray(new SegmentReader[readers.size()]));
|
||||
return new DirectoryReader(readers.toArray(new SegmentReader[readers.size()]),
|
||||
dir, writer, segmentInfos, writer.getConfig().getReaderTermsIndexDivisor(),
|
||||
applyAllDeletes, writer.getReaderFinishedListeners());
|
||||
}
|
||||
|
||||
/** This constructor is only used for {@link #doOpenIfChanged()} */
|
||||
DirectoryReader(Directory directory, SegmentInfos infos, SegmentReader[] oldReaders,
|
||||
boolean doClone, int termInfosIndexDivisor, Collection<ReaderFinishedListener> readerFinishedListeners) throws IOException {
|
||||
this.directory = directory;
|
||||
this.segmentInfos = infos;
|
||||
this.termInfosIndexDivisor = termInfosIndexDivisor;
|
||||
this.readerFinishedListeners = readerFinishedListeners;
|
||||
applyAllDeletes = false;
|
||||
|
||||
static DirectoryReader open(Directory directory, IndexWriter writer, SegmentInfos infos, SegmentReader[] oldReaders,
|
||||
boolean doClone, int termInfosIndexDivisor, Collection<ReaderFinishedListener> readerFinishedListeners
|
||||
) throws IOException {
|
||||
// we put the old SegmentReaders in a map, that allows us
|
||||
// to lookup a reader using its segment name
|
||||
Map<String,Integer> segmentReaders = new HashMap<String,Integer>();
|
||||
final Map<String,Integer> segmentReaders = new HashMap<String,Integer>();
|
||||
|
||||
if (oldReaders != null) {
|
||||
// create a Map SegmentName->SegmentReader
|
||||
|
@ -194,6 +158,7 @@ class DirectoryReader extends IndexReader implements Cloneable {
|
|||
}
|
||||
|
||||
boolean success = false;
|
||||
IOException prior = null;
|
||||
try {
|
||||
SegmentReader newReader;
|
||||
if (newReaders[i] == null || infos.info(i).getUseCompoundFile() != newReaders[i].getSegmentInfo().getUseCompoundFile()) {
|
||||
|
@ -221,6 +186,8 @@ class DirectoryReader extends IndexReader implements Cloneable {
|
|||
}
|
||||
}
|
||||
success = true;
|
||||
} catch (IOException ex) {
|
||||
prior = ex;
|
||||
} finally {
|
||||
if (!success) {
|
||||
for (i++; i < infos.size(); i++) {
|
||||
|
@ -235,17 +202,19 @@ class DirectoryReader extends IndexReader implements Cloneable {
|
|||
// closing we must decRef it
|
||||
newReaders[i].decRef();
|
||||
}
|
||||
} catch (IOException ignore) {
|
||||
// keep going - we want to clean up as much as possible
|
||||
} catch (IOException ex) {
|
||||
if (prior == null) prior = ex;
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
||||
// throw the first exception
|
||||
if (prior != null) throw prior;
|
||||
}
|
||||
}
|
||||
|
||||
// initialize the readers to calculate maxDoc before we try to reuse the old normsCache
|
||||
initialize(newReaders);
|
||||
return new DirectoryReader(newReaders,
|
||||
directory, writer, infos, termInfosIndexDivisor,
|
||||
false, readerFinishedListeners);
|
||||
}
|
||||
|
||||
/** {@inheritDoc} */
|
||||
|
@ -269,41 +238,10 @@ class DirectoryReader extends IndexReader implements Cloneable {
|
|||
return buffer.toString();
|
||||
}
|
||||
|
||||
private void initialize(SegmentReader[] subReaders) throws IOException {
|
||||
this.subReaders = subReaders;
|
||||
starts = new int[subReaders.length + 1]; // build starts array
|
||||
final AtomicReaderContext[] subReaderCtx = new AtomicReaderContext[subReaders.length];
|
||||
topLevelReaderContext = new CompositeReaderContext(this, subReaderCtx, subReaderCtx);
|
||||
final List<Fields> subFields = new ArrayList<Fields>();
|
||||
|
||||
for (int i = 0; i < subReaders.length; i++) {
|
||||
starts[i] = maxDoc;
|
||||
subReaderCtx[i] = new AtomicReaderContext(topLevelReaderContext, subReaders[i], i, maxDoc, i, maxDoc);
|
||||
maxDoc += subReaders[i].maxDoc(); // compute maxDocs
|
||||
|
||||
if (subReaders[i].hasDeletions()) {
|
||||
hasDeletions = true;
|
||||
}
|
||||
|
||||
final Fields f = subReaders[i].fields();
|
||||
if (f != null) {
|
||||
subFields.add(f);
|
||||
}
|
||||
}
|
||||
starts[subReaders.length] = maxDoc;
|
||||
}
|
||||
|
||||
@Override
|
||||
public Bits getLiveDocs() {
|
||||
throw new UnsupportedOperationException("please use MultiFields.getLiveDocs, or wrap your IndexReader with SlowMultiReaderWrapper, if you really need a top level Bits liveDocs");
|
||||
}
|
||||
|
||||
@Override
|
||||
public final synchronized Object clone() {
|
||||
try {
|
||||
DirectoryReader newReader = doOpenIfChanged((SegmentInfos) segmentInfos.clone(), true);
|
||||
newReader.writer = writer;
|
||||
newReader.hasDeletions = hasDeletions;
|
||||
DirectoryReader newReader = doOpenIfChanged((SegmentInfos) segmentInfos.clone(), true, writer);
|
||||
assert newReader.readerFinishedListeners != null;
|
||||
return newReader;
|
||||
} catch (Exception ex) {
|
||||
|
@ -381,13 +319,13 @@ class DirectoryReader extends IndexReader implements Cloneable {
|
|||
protected Object doBody(String segmentFileName) throws CorruptIndexException, IOException {
|
||||
final SegmentInfos infos = new SegmentInfos();
|
||||
infos.read(directory, segmentFileName);
|
||||
return doOpenIfChanged(infos, false);
|
||||
return doOpenIfChanged(infos, false, null);
|
||||
}
|
||||
}.run(commit);
|
||||
}
|
||||
|
||||
private synchronized DirectoryReader doOpenIfChanged(SegmentInfos infos, boolean doClone) throws CorruptIndexException, IOException {
|
||||
return new DirectoryReader(directory, infos, subReaders, doClone, termInfosIndexDivisor, readerFinishedListeners);
|
||||
private synchronized DirectoryReader doOpenIfChanged(SegmentInfos infos, boolean doClone, IndexWriter writer) throws CorruptIndexException, IOException {
|
||||
return DirectoryReader.open(directory, writer, infos, subReaders, doClone, termInfosIndexDivisor, readerFinishedListeners);
|
||||
}
|
||||
|
||||
/** Version number when this IndexReader was opened. */
|
||||
|
@ -397,118 +335,12 @@ class DirectoryReader extends IndexReader implements Cloneable {
|
|||
return segmentInfos.getVersion();
|
||||
}
|
||||
|
||||
@Override
|
||||
public Fields getTermVectors(int docID) throws IOException {
|
||||
ensureOpen();
|
||||
int i = readerIndex(docID); // find segment num
|
||||
return subReaders[i].getTermVectors(docID - starts[i]); // dispatch to segment
|
||||
}
|
||||
|
||||
@Override
|
||||
public int numDocs() {
|
||||
// Don't call ensureOpen() here (it could affect performance)
|
||||
|
||||
// NOTE: multiple threads may wind up init'ing
|
||||
// numDocs... but that's harmless
|
||||
if (numDocs == -1) { // check cache
|
||||
int n = 0; // cache miss--recompute
|
||||
for (int i = 0; i < subReaders.length; i++)
|
||||
n += subReaders[i].numDocs(); // sum from readers
|
||||
numDocs = n;
|
||||
}
|
||||
return numDocs;
|
||||
}
|
||||
|
||||
@Override
|
||||
public int maxDoc() {
|
||||
// Don't call ensureOpen() here (it could affect performance)
|
||||
return maxDoc;
|
||||
}
|
||||
|
||||
@Override
|
||||
public void document(int docID, StoredFieldVisitor visitor) throws CorruptIndexException, IOException {
|
||||
ensureOpen();
|
||||
int i = readerIndex(docID); // find segment num
|
||||
subReaders[i].document(docID - starts[i], visitor); // dispatch to segment reader
|
||||
}
|
||||
|
||||
@Override
|
||||
public boolean hasDeletions() {
|
||||
ensureOpen();
|
||||
return hasDeletions;
|
||||
}
|
||||
|
||||
private int readerIndex(int n) { // find reader for doc n:
|
||||
return readerIndex(n, this.starts, this.subReaders.length);
|
||||
}
|
||||
|
||||
final static int readerIndex(int n, int[] starts, int numSubReaders) { // find reader for doc n:
|
||||
int lo = 0; // search starts array
|
||||
int hi = numSubReaders - 1; // for first element less
|
||||
|
||||
while (hi >= lo) {
|
||||
int mid = (lo + hi) >>> 1;
|
||||
int midValue = starts[mid];
|
||||
if (n < midValue)
|
||||
hi = mid - 1;
|
||||
else if (n > midValue)
|
||||
lo = mid + 1;
|
||||
else { // found a match
|
||||
while (mid+1 < numSubReaders && starts[mid+1] == midValue) {
|
||||
mid++; // scan to last match
|
||||
}
|
||||
return mid;
|
||||
}
|
||||
}
|
||||
return hi;
|
||||
}
|
||||
|
||||
@Override
|
||||
public boolean hasNorms(String field) throws IOException {
|
||||
ensureOpen();
|
||||
for (int i = 0; i < subReaders.length; i++) {
|
||||
if (subReaders[i].hasNorms(field)) return true;
|
||||
}
|
||||
return false;
|
||||
}
|
||||
|
||||
@Override
|
||||
public synchronized byte[] norms(String field) throws IOException {
|
||||
ensureOpen();
|
||||
throw new UnsupportedOperationException("please use MultiNorms.norms, or wrap your IndexReader with SlowMultiReaderWrapper, if you really need a top level norms");
|
||||
}
|
||||
|
||||
@Override
|
||||
public int docFreq(String field, BytesRef term) throws IOException {
|
||||
ensureOpen();
|
||||
int total = 0; // sum freqs in segments
|
||||
for (int i = 0; i < subReaders.length; i++) {
|
||||
total += subReaders[i].docFreq(field, term);
|
||||
}
|
||||
return total;
|
||||
}
|
||||
|
||||
@Override
|
||||
public Fields fields() throws IOException {
|
||||
throw new UnsupportedOperationException("please use MultiFields.getFields, or wrap your IndexReader with SlowMultiReaderWrapper, if you really need a top level Fields");
|
||||
}
|
||||
|
||||
@Override
|
||||
public Map<String,String> getCommitUserData() {
|
||||
ensureOpen();
|
||||
return segmentInfos.getUserData();
|
||||
}
|
||||
|
||||
/**
|
||||
* Check whether this IndexReader is still using the current (i.e., most recently committed) version of the index. If
|
||||
* a writer has committed any changes to the index since this reader was opened, this will return <code>false</code>,
|
||||
* in which case you must open a new IndexReader in order
|
||||
* to see the changes. Use {@link IndexWriter#commit} to
|
||||
* commit changes to the index.
|
||||
*
|
||||
* @throws CorruptIndexException if the index is corrupt
|
||||
* @throws IOException if there is a low-level IO error
|
||||
*/
|
||||
@Override
|
||||
public boolean isCurrent() throws CorruptIndexException, IOException {
|
||||
ensureOpen();
|
||||
|
@ -542,33 +374,6 @@ class DirectoryReader extends IndexReader implements Cloneable {
|
|||
if (ioe != null) throw ioe;
|
||||
}
|
||||
|
||||
@Override
|
||||
public Collection<String> getFieldNames (IndexReader.FieldOption fieldNames) {
|
||||
ensureOpen();
|
||||
return getFieldNames(fieldNames, this.subReaders);
|
||||
}
|
||||
|
||||
static Collection<String> getFieldNames (IndexReader.FieldOption fieldNames, IndexReader[] subReaders) {
|
||||
// maintain a unique set of field names
|
||||
Set<String> fieldSet = new HashSet<String>();
|
||||
for (IndexReader reader : subReaders) {
|
||||
Collection<String> names = reader.getFieldNames(fieldNames);
|
||||
fieldSet.addAll(names);
|
||||
}
|
||||
return fieldSet;
|
||||
}
|
||||
|
||||
@Override
|
||||
public ReaderContext getTopReaderContext() {
|
||||
ensureOpen();
|
||||
return topLevelReaderContext;
|
||||
}
|
||||
|
||||
@Override
|
||||
public IndexReader[] getSequentialSubReaders() {
|
||||
return subReaders;
|
||||
}
|
||||
|
||||
/** Returns the directory this index resides in. */
|
||||
@Override
|
||||
public Directory directory() {
|
||||
|
@ -711,9 +516,4 @@ class DirectoryReader extends IndexReader implements Cloneable {
|
|||
throw new UnsupportedOperationException("This IndexCommit does not support deletions");
|
||||
}
|
||||
}
|
||||
|
||||
@Override
|
||||
public DocValues docValues(String field) throws IOException {
|
||||
throw new UnsupportedOperationException("please use MultiDocValues#getDocValues, or wrap your IndexReader with SlowMultiReaderWrapper, if you really need a top level DocValues");
|
||||
}
|
||||
}
|
||||
|
|
|
@ -1,90 +0,0 @@
|
|||
package org.apache.lucene.index;
|
||||
|
||||
/**
|
||||
* Licensed to the Apache Software Foundation (ASF) under one or more
|
||||
* contributor license agreements. See the NOTICE file distributed with
|
||||
* this work for additional information regarding copyright ownership.
|
||||
* The ASF licenses this file to You under the Apache License, Version 2.0
|
||||
* (the "License"); you may not use this file except in compliance with
|
||||
* the License. You may obtain a copy of the License at
|
||||
*
|
||||
* http://www.apache.org/licenses/LICENSE-2.0
|
||||
*
|
||||
* Unless required by applicable law or agreed to in writing, software
|
||||
* distributed under the License is distributed on an "AS IS" BASIS,
|
||||
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
|
||||
* See the License for the specific language governing permissions and
|
||||
* limitations under the License.
|
||||
*/
|
||||
|
||||
import java.io.IOException;
|
||||
import java.util.HashMap;
|
||||
import java.util.Map;
|
||||
|
||||
/** This is just a "splitter" class: it lets you wrap two
|
||||
* DocFieldConsumer instances as a single consumer. */
|
||||
|
||||
final class DocFieldConsumers extends DocFieldConsumer {
|
||||
final DocFieldConsumer one;
|
||||
final DocFieldConsumer two;
|
||||
final DocumentsWriterPerThread.DocState docState;
|
||||
|
||||
public DocFieldConsumers(DocFieldProcessor processor, DocFieldConsumer one, DocFieldConsumer two) {
|
||||
this.one = one;
|
||||
this.two = two;
|
||||
this.docState = processor.docState;
|
||||
}
|
||||
|
||||
@Override
|
||||
public void flush(Map<FieldInfo, DocFieldConsumerPerField> fieldsToFlush, SegmentWriteState state) throws IOException {
|
||||
|
||||
Map<FieldInfo, DocFieldConsumerPerField> oneFieldsToFlush = new HashMap<FieldInfo, DocFieldConsumerPerField>();
|
||||
Map<FieldInfo, DocFieldConsumerPerField> twoFieldsToFlush = new HashMap<FieldInfo, DocFieldConsumerPerField>();
|
||||
|
||||
for (Map.Entry<FieldInfo, DocFieldConsumerPerField> fieldToFlush : fieldsToFlush.entrySet()) {
|
||||
DocFieldConsumersPerField perField = (DocFieldConsumersPerField) fieldToFlush.getValue();
|
||||
oneFieldsToFlush.put(fieldToFlush.getKey(), perField.one);
|
||||
twoFieldsToFlush.put(fieldToFlush.getKey(), perField.two);
|
||||
}
|
||||
|
||||
one.flush(oneFieldsToFlush, state);
|
||||
two.flush(twoFieldsToFlush, state);
|
||||
}
|
||||
|
||||
@Override
|
||||
public void abort() {
|
||||
try {
|
||||
one.abort();
|
||||
} finally {
|
||||
two.abort();
|
||||
}
|
||||
}
|
||||
|
||||
@Override
|
||||
public boolean freeRAM() {
|
||||
boolean any = one.freeRAM();
|
||||
any |= two.freeRAM();
|
||||
return any;
|
||||
}
|
||||
|
||||
@Override
|
||||
public void finishDocument() throws IOException {
|
||||
try {
|
||||
one.finishDocument();
|
||||
} finally {
|
||||
two.finishDocument();
|
||||
}
|
||||
}
|
||||
|
||||
@Override
|
||||
public void startDocument() throws IOException {
|
||||
one.startDocument();
|
||||
two.startDocument();
|
||||
}
|
||||
|
||||
@Override
|
||||
public DocFieldConsumerPerField addField(FieldInfo fi) {
|
||||
return new DocFieldConsumersPerField(this, fi, one.addField(fi), two.addField(fi));
|
||||
}
|
||||
|
||||
}
|
|
@ -1,55 +0,0 @@
|
|||
package org.apache.lucene.index;
|
||||
|
||||
/**
|
||||
* Licensed to the Apache Software Foundation (ASF) under one or more
|
||||
* contributor license agreements. See the NOTICE file distributed with
|
||||
* this work for additional information regarding copyright ownership.
|
||||
* The ASF licenses this file to You under the Apache License, Version 2.0
|
||||
* (the "License"); you may not use this file except in compliance with
|
||||
* the License. You may obtain a copy of the License at
|
||||
*
|
||||
* http://www.apache.org/licenses/LICENSE-2.0
|
||||
*
|
||||
* Unless required by applicable law or agreed to in writing, software
|
||||
* distributed under the License is distributed on an "AS IS" BASIS,
|
||||
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
|
||||
* See the License for the specific language governing permissions and
|
||||
* limitations under the License.
|
||||
*/
|
||||
|
||||
import java.io.IOException;
|
||||
|
||||
final class DocFieldConsumersPerField extends DocFieldConsumerPerField {
|
||||
|
||||
final DocFieldConsumerPerField one;
|
||||
final DocFieldConsumerPerField two;
|
||||
final DocFieldConsumers parent;
|
||||
final FieldInfo fieldInfo;
|
||||
|
||||
public DocFieldConsumersPerField(DocFieldConsumers parent, FieldInfo fi, DocFieldConsumerPerField one, DocFieldConsumerPerField two) {
|
||||
this.parent = parent;
|
||||
this.one = one;
|
||||
this.two = two;
|
||||
this.fieldInfo = fi;
|
||||
}
|
||||
|
||||
@Override
|
||||
public void processFields(IndexableField[] fields, int count) throws IOException {
|
||||
one.processFields(fields, count);
|
||||
two.processFields(fields, count);
|
||||
}
|
||||
|
||||
@Override
|
||||
public void abort() {
|
||||
try {
|
||||
one.abort();
|
||||
} finally {
|
||||
two.abort();
|
||||
}
|
||||
}
|
||||
|
||||
@Override
|
||||
FieldInfo getFieldInfo() {
|
||||
return fieldInfo;
|
||||
}
|
||||
}
|
|
@ -277,8 +277,8 @@ final class DocFieldProcessor extends DocConsumer {
|
|||
perField.consumer.processFields(perField.fields, perField.fieldCount);
|
||||
}
|
||||
|
||||
if (docState.maxTermPrefix != null && docState.infoStream != null) {
|
||||
docState.infoStream.println("WARNING: document contains at least one immense term (whose UTF8 encoding is longer than the max length " + DocumentsWriterPerThread.MAX_TERM_LENGTH_UTF8 + "), all of which were skipped. Please correct the analyzer to not produce such terms. The prefix of the first immense term is: '" + docState.maxTermPrefix + "...'");
|
||||
if (docState.maxTermPrefix != null && docState.infoStream.isEnabled("IW")) {
|
||||
docState.infoStream.message("IW", "WARNING: document contains at least one immense term (whose UTF8 encoding is longer than the max length " + DocumentsWriterPerThread.MAX_TERM_LENGTH_UTF8 + "), all of which were skipped. Please correct the analyzer to not produce such terms. The prefix of the first immense term is: '" + docState.maxTermPrefix + "...'");
|
||||
docState.maxTermPrefix = null;
|
||||
}
|
||||
}
|
||||
|
|
|
@ -56,9 +56,6 @@ final class DocInverter extends DocFieldConsumer {
|
|||
}
|
||||
}
|
||||
|
||||
// Used to read a string value for a field
|
||||
final ReusableStringReader stringReader = new ReusableStringReader();
|
||||
|
||||
public DocInverter(DocumentsWriterPerThread.DocState docState, InvertedDocConsumer consumer, InvertedDocEndConsumer endConsumer) {
|
||||
this.docState = docState;
|
||||
this.consumer = consumer;
|
||||
|
|
|
@ -18,7 +18,6 @@ package org.apache.lucene.index;
|
|||
*/
|
||||
|
||||
import java.io.IOException;
|
||||
import java.io.PrintStream;
|
||||
import java.util.Collection;
|
||||
import java.util.Iterator;
|
||||
import java.util.LinkedList;
|
||||
|
@ -216,7 +215,9 @@ final class DocumentsWriter {
|
|||
}
|
||||
|
||||
try {
|
||||
infoStream.message("DW", "abort");
|
||||
if (infoStream.isEnabled("DW")) {
|
||||
infoStream.message("DW", "abort");
|
||||
}
|
||||
|
||||
final Iterator<ThreadState> threadsIterator = perThreadPool.getActivePerThreadsIterator();
|
||||
while (threadsIterator.hasNext()) {
|
||||
|
@ -298,8 +299,10 @@ final class DocumentsWriter {
|
|||
maybeMerge |= doFlush(flushingDWPT);
|
||||
}
|
||||
|
||||
if (infoStream.isEnabled("DW") && flushControl.anyStalledThreads()) {
|
||||
infoStream.message("DW", "WARNING DocumentsWriter has stalled threads; waiting");
|
||||
if (infoStream.isEnabled("DW")) {
|
||||
if (flushControl.anyStalledThreads()) {
|
||||
infoStream.message("DW", "WARNING DocumentsWriter has stalled threads; waiting");
|
||||
}
|
||||
}
|
||||
|
||||
flushControl.waitIfStalled(); // block if stalled
|
||||
|
|
|
@ -21,7 +21,6 @@ import static org.apache.lucene.util.ByteBlockPool.BYTE_BLOCK_MASK;
|
|||
import static org.apache.lucene.util.ByteBlockPool.BYTE_BLOCK_SIZE;
|
||||
|
||||
import java.io.IOException;
|
||||
import java.io.PrintStream;
|
||||
import java.text.NumberFormat;
|
||||
|
||||
import org.apache.lucene.analysis.Analyzer;
|
||||
|
@ -88,14 +87,15 @@ public class DocumentsWriterPerThread {
|
|||
static class DocState {
|
||||
final DocumentsWriterPerThread docWriter;
|
||||
Analyzer analyzer;
|
||||
PrintStream infoStream;
|
||||
InfoStream infoStream;
|
||||
SimilarityProvider similarityProvider;
|
||||
int docID;
|
||||
Iterable<? extends IndexableField> doc;
|
||||
String maxTermPrefix;
|
||||
|
||||
DocState(DocumentsWriterPerThread docWriter) {
|
||||
DocState(DocumentsWriterPerThread docWriter, InfoStream infoStream) {
|
||||
this.docWriter = docWriter;
|
||||
this.infoStream = infoStream;
|
||||
}
|
||||
|
||||
// Only called by asserts
|
||||
|
@ -131,7 +131,9 @@ public class DocumentsWriterPerThread {
|
|||
void abort() throws IOException {
|
||||
hasAborted = aborting = true;
|
||||
try {
|
||||
infoStream.message("DWPT", "now abort");
|
||||
if (infoStream.isEnabled("DWPT")) {
|
||||
infoStream.message("DWPT", "now abort");
|
||||
}
|
||||
try {
|
||||
consumer.abort();
|
||||
} catch (Throwable t) {
|
||||
|
@ -144,7 +146,9 @@ public class DocumentsWriterPerThread {
|
|||
|
||||
} finally {
|
||||
aborting = false;
|
||||
infoStream.message("DWPT", "done abort");
|
||||
if (infoStream.isEnabled("DWPT")) {
|
||||
infoStream.message("DWPT", "done abort");
|
||||
}
|
||||
}
|
||||
}
|
||||
private final static boolean INFO_VERBOSE = false;
|
||||
|
@ -181,7 +185,7 @@ public class DocumentsWriterPerThread {
|
|||
this.writer = parent.indexWriter;
|
||||
this.infoStream = parent.infoStream;
|
||||
this.codec = parent.codec;
|
||||
this.docState = new DocState(this);
|
||||
this.docState = new DocState(this, infoStream);
|
||||
this.docState.similarityProvider = parent.indexWriter.getConfig()
|
||||
.getSimilarityProvider();
|
||||
bytesUsed = Counter.newCounter();
|
||||
|
|
|
@ -1,79 +0,0 @@
|
|||
package org.apache.lucene.index;
|
||||
/**
|
||||
* Copyright 2004 The Apache Software Foundation
|
||||
*
|
||||
* Licensed under the Apache License, Version 2.0 (the "License");
|
||||
* you may not use this file except in compliance with the License.
|
||||
* You may obtain a copy of the License at
|
||||
*
|
||||
* http://www.apache.org/licenses/LICENSE-2.0
|
||||
*
|
||||
* Unless required by applicable law or agreed to in writing, software
|
||||
* distributed under the License is distributed on an "AS IS" BASIS,
|
||||
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
|
||||
* See the License for the specific language governing permissions and
|
||||
* limitations under the License.
|
||||
*/
|
||||
|
||||
/**
|
||||
*
|
||||
*
|
||||
**/
|
||||
public class FieldReaderException extends RuntimeException{
|
||||
/**
|
||||
* Constructs a new runtime exception with <code>null</code> as its
|
||||
* detail message. The cause is not initialized, and may subsequently be
|
||||
* initialized by a call to {@link #initCause}.
|
||||
*/
|
||||
public FieldReaderException() {
|
||||
}
|
||||
|
||||
/**
|
||||
* Constructs a new runtime exception with the specified cause and a
|
||||
* detail message of <tt>(cause==null ? null : cause.toString())</tt>
|
||||
* (which typically contains the class and detail message of
|
||||
* <tt>cause</tt>).
|
||||
* <p>
|
||||
* This constructor is useful for runtime exceptions
|
||||
* that are little more than wrappers for other throwables.
|
||||
*
|
||||
* @param cause the cause (which is saved for later retrieval by the
|
||||
* {@link #getCause()} method). (A <tt>null</tt> value is
|
||||
* permitted, and indicates that the cause is nonexistent or
|
||||
* unknown.)
|
||||
* @since 1.4
|
||||
*/
|
||||
public FieldReaderException(Throwable cause) {
|
||||
super(cause);
|
||||
}
|
||||
|
||||
/**
|
||||
* Constructs a new runtime exception with the specified detail message.
|
||||
* The cause is not initialized, and may subsequently be initialized by a
|
||||
* call to {@link #initCause}.
|
||||
*
|
||||
* @param message the detail message. The detail message is saved for
|
||||
* later retrieval by the {@link #getMessage()} method.
|
||||
*/
|
||||
public FieldReaderException(String message) {
|
||||
super(message);
|
||||
}
|
||||
|
||||
/**
|
||||
* Constructs a new runtime exception with the specified detail message and
|
||||
* cause. <p>Note that the detail message associated with
|
||||
* <code>cause</code> is <i>not</i> automatically incorporated in
|
||||
* this runtime exception's detail message.
|
||||
*
|
||||
* @param message the detail message (which is saved for later retrieval
|
||||
* by the {@link #getMessage()} method).
|
||||
* @param cause the cause (which is saved for later retrieval by the
|
||||
* {@link #getCause()} method). (A <tt>null</tt> value is
|
||||
* permitted, and indicates that the cause is nonexistent or
|
||||
* unknown.)
|
||||
* @since 1.4
|
||||
*/
|
||||
public FieldReaderException(String message, Throwable cause) {
|
||||
super(message, cause);
|
||||
}
|
||||
}
|
|
@ -20,7 +20,6 @@ import java.util.Iterator;
|
|||
|
||||
import org.apache.lucene.index.DocumentsWriterPerThreadPool.ThreadState;
|
||||
import org.apache.lucene.store.Directory;
|
||||
import org.apache.lucene.util.InfoStream;
|
||||
import org.apache.lucene.util.SetOnce;
|
||||
|
||||
/**
|
||||
|
@ -129,7 +128,9 @@ public abstract class FlushPolicy {
|
|||
}
|
||||
|
||||
private boolean assertMessage(String s) {
|
||||
writer.get().infoStream.message("FP", s);
|
||||
if (writer.get().infoStream.isEnabled("FP")) {
|
||||
writer.get().infoStream.message("FP", s);
|
||||
}
|
||||
return true;
|
||||
}
|
||||
|
||||
|
|
|
@ -501,8 +501,10 @@ final class IndexFileDeleter {
|
|||
void incRef(String fileName) throws IOException {
|
||||
assert locked();
|
||||
RefCount rc = getRefCount(fileName);
|
||||
if (infoStream.isEnabled("IFD") && VERBOSE_REF_COUNTS) {
|
||||
infoStream.message("IFD", " IncRef \"" + fileName + "\": pre-incr count is " + rc.count);
|
||||
if (infoStream.isEnabled("IFD")) {
|
||||
if (VERBOSE_REF_COUNTS) {
|
||||
infoStream.message("IFD", " IncRef \"" + fileName + "\": pre-incr count is " + rc.count);
|
||||
}
|
||||
}
|
||||
rc.IncRef();
|
||||
}
|
||||
|
@ -517,8 +519,10 @@ final class IndexFileDeleter {
|
|||
void decRef(String fileName) throws IOException {
|
||||
assert locked();
|
||||
RefCount rc = getRefCount(fileName);
|
||||
if (infoStream.isEnabled("IFD") && VERBOSE_REF_COUNTS) {
|
||||
infoStream.message("IFD", " DecRef \"" + fileName + "\": pre-decr count is " + rc.count);
|
||||
if (infoStream.isEnabled("IFD")) {
|
||||
if (VERBOSE_REF_COUNTS) {
|
||||
infoStream.message("IFD", " DecRef \"" + fileName + "\": pre-decr count is " + rc.count);
|
||||
}
|
||||
}
|
||||
if (0 == rc.DecRef()) {
|
||||
// This file is no longer referenced by any past
|
||||
|
|
|
@ -244,19 +244,18 @@ public abstract class IndexReader implements Cloneable,Closeable {
|
|||
|
||||
/**
|
||||
* Expert: decreases the refCount of this IndexReader
|
||||
* instance. If the refCount drops to 0, then pending
|
||||
* changes (if any) are committed to the index and this
|
||||
* instance. If the refCount drops to 0, then this
|
||||
* reader is closed. If an exception is hit, the refCount
|
||||
* is unchanged.
|
||||
*
|
||||
* @throws IOException in case an IOException occurs in commit() or doClose()
|
||||
* @throws IOException in case an IOException occurs in doClose()
|
||||
*
|
||||
* @see #incRef
|
||||
*/
|
||||
public final void decRef() throws IOException {
|
||||
ensureOpen();
|
||||
final int rc = refCount.getAndDecrement();
|
||||
if (rc == 1) {
|
||||
final int rc = refCount.decrementAndGet();
|
||||
if (rc == 0) {
|
||||
boolean success = false;
|
||||
try {
|
||||
doClose();
|
||||
|
@ -268,8 +267,8 @@ public abstract class IndexReader implements Cloneable,Closeable {
|
|||
}
|
||||
}
|
||||
readerFinished();
|
||||
} else if (rc <= 0) {
|
||||
throw new IllegalStateException("too many decRef calls: refCount was " + rc + " before decrement");
|
||||
} else if (rc < 0) {
|
||||
throw new IllegalStateException("too many decRef calls: refCount is " + rc + " after decrement");
|
||||
}
|
||||
}
|
||||
|
||||
|
@ -731,9 +730,6 @@ public abstract class IndexReader implements Cloneable,Closeable {
|
|||
// IndexableField
|
||||
public final Document document(int docID) throws CorruptIndexException, IOException {
|
||||
ensureOpen();
|
||||
if (docID < 0 || docID >= maxDoc()) {
|
||||
throw new IllegalArgumentException("docID must be >= 0 and < maxDoc=" + maxDoc() + " (got docID=" + docID + ")");
|
||||
}
|
||||
final DocumentStoredFieldVisitor visitor = new DocumentStoredFieldVisitor();
|
||||
document(docID, visitor);
|
||||
return visitor.getDocument();
|
||||
|
|
|
@ -332,7 +332,9 @@ public class IndexWriter implements Closeable, TwoPhaseCommit {
|
|||
|
||||
final long tStart = System.currentTimeMillis();
|
||||
|
||||
infoStream.message("IW", "flush at getReader");
|
||||
if (infoStream.isEnabled("IW")) {
|
||||
infoStream.message("IW", "flush at getReader");
|
||||
}
|
||||
// Do this up front before flushing so that the readers
|
||||
// obtained during this flush are pooled, the first time
|
||||
// this method is called:
|
||||
|
@ -362,7 +364,7 @@ public class IndexWriter implements Closeable, TwoPhaseCommit {
|
|||
// just like we do when loading segments_N
|
||||
synchronized(this) {
|
||||
maybeApplyDeletes(applyAllDeletes);
|
||||
r = new DirectoryReader(this, segmentInfos, applyAllDeletes);
|
||||
r = DirectoryReader.open(this, segmentInfos, applyAllDeletes);
|
||||
if (infoStream.isEnabled("IW")) {
|
||||
infoStream.message("IW", "return reader version=" + r.getVersion() + " reader=" + r);
|
||||
}
|
||||
|
@ -372,8 +374,10 @@ public class IndexWriter implements Closeable, TwoPhaseCommit {
|
|||
// never reached but javac disagrees:
|
||||
return null;
|
||||
} finally {
|
||||
if (!success && infoStream.isEnabled("IW")) {
|
||||
infoStream.message("IW", "hit exception during NRT reader");
|
||||
if (!success) {
|
||||
if (infoStream.isEnabled("IW")) {
|
||||
infoStream.message("IW", "hit exception during NRT reader");
|
||||
}
|
||||
}
|
||||
// Done: finish the full flush!
|
||||
docWriter.finishFullFlush(success);
|
||||
|
@ -873,8 +877,9 @@ public class IndexWriter implements Closeable, TwoPhaseCommit {
|
|||
segmentInfos.replace(oldInfos);
|
||||
changeCount++;
|
||||
segmentInfos.changed();
|
||||
if (infoStream.isEnabled("IW"))
|
||||
if (infoStream.isEnabled("IW")) {
|
||||
infoStream.message("IW", "init: loaded commit \"" + commit.getSegmentsFileName() + "\"");
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
|
@ -910,7 +915,9 @@ public class IndexWriter implements Closeable, TwoPhaseCommit {
|
|||
|
||||
} finally {
|
||||
if (!success) {
|
||||
infoStream.message("IW", "init: hit exception on init; releasing write lock");
|
||||
if (infoStream.isEnabled("IW")) {
|
||||
infoStream.message("IW", "init: hit exception on init; releasing write lock");
|
||||
}
|
||||
try {
|
||||
writeLock.release();
|
||||
} catch (Throwable t) {
|
||||
|
@ -1078,14 +1085,17 @@ public class IndexWriter implements Closeable, TwoPhaseCommit {
|
|||
|
||||
mergeScheduler.close();
|
||||
|
||||
infoStream.message("IW", "now call final commit()");
|
||||
if (infoStream.isEnabled("IW")) {
|
||||
infoStream.message("IW", "now call final commit()");
|
||||
}
|
||||
|
||||
if (!hitOOM) {
|
||||
commitInternal(null);
|
||||
}
|
||||
|
||||
if (infoStream.isEnabled("IW"))
|
||||
if (infoStream.isEnabled("IW")) {
|
||||
infoStream.message("IW", "at close: " + segString());
|
||||
}
|
||||
// used by assert below
|
||||
final DocumentsWriter oldWriter = docWriter;
|
||||
synchronized(this) {
|
||||
|
@ -1109,7 +1119,9 @@ public class IndexWriter implements Closeable, TwoPhaseCommit {
|
|||
closing = false;
|
||||
notifyAll();
|
||||
if (!closed) {
|
||||
infoStream.message("IW", "hit exception while closing");
|
||||
if (infoStream.isEnabled("IW")) {
|
||||
infoStream.message("IW", "hit exception while closing");
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
||||
|
@ -1338,8 +1350,10 @@ public class IndexWriter implements Closeable, TwoPhaseCommit {
|
|||
anySegmentFlushed = docWriter.updateDocuments(docs, analyzer, delTerm);
|
||||
success = true;
|
||||
} finally {
|
||||
if (!success && infoStream.isEnabled("IW")) {
|
||||
infoStream.message("IW", "hit exception updating document");
|
||||
if (!success) {
|
||||
if (infoStream.isEnabled("IW")) {
|
||||
infoStream.message("IW", "hit exception updating document");
|
||||
}
|
||||
}
|
||||
}
|
||||
if (anySegmentFlushed) {
|
||||
|
@ -1485,8 +1499,10 @@ public class IndexWriter implements Closeable, TwoPhaseCommit {
|
|||
anySegmentFlushed = docWriter.updateDocument(doc, analyzer, term);
|
||||
success = true;
|
||||
} finally {
|
||||
if (!success && infoStream.isEnabled("IW")) {
|
||||
infoStream.message("IW", "hit exception updating document");
|
||||
if (!success) {
|
||||
if (infoStream.isEnabled("IW")) {
|
||||
infoStream.message("IW", "hit exception updating document");
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
|
@ -1744,8 +1760,9 @@ public class IndexWriter implements Closeable, TwoPhaseCommit {
|
|||
|
||||
flush(true, true);
|
||||
|
||||
if (infoStream.isEnabled("IW"))
|
||||
if (infoStream.isEnabled("IW")) {
|
||||
infoStream.message("IW", "forceMergeDeletes: index now " + segString());
|
||||
}
|
||||
|
||||
MergePolicy.MergeSpecification spec;
|
||||
|
||||
|
@ -1938,7 +1955,9 @@ public class IndexWriter implements Closeable, TwoPhaseCommit {
|
|||
|
||||
boolean success = false;
|
||||
|
||||
infoStream.message("IW", "rollback");
|
||||
if (infoStream.isEnabled("IW")) {
|
||||
infoStream.message("IW", "rollback");
|
||||
}
|
||||
|
||||
try {
|
||||
synchronized(this) {
|
||||
|
@ -1946,7 +1965,9 @@ public class IndexWriter implements Closeable, TwoPhaseCommit {
|
|||
stopMerges = true;
|
||||
}
|
||||
|
||||
infoStream.message("IW", "rollback: done finish merges");
|
||||
if (infoStream.isEnabled("IW")) {
|
||||
infoStream.message("IW", "rollback: done finish merges");
|
||||
}
|
||||
|
||||
// Must pre-close these two, in case they increment
|
||||
// changeCount so that we can then set it to false
|
||||
|
@ -1998,7 +2019,9 @@ public class IndexWriter implements Closeable, TwoPhaseCommit {
|
|||
if (!success) {
|
||||
closing = false;
|
||||
notifyAll();
|
||||
infoStream.message("IW", "hit exception during rollback");
|
||||
if (infoStream.isEnabled("IW")) {
|
||||
infoStream.message("IW", "hit exception during rollback");
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
||||
|
@ -2050,8 +2073,10 @@ public class IndexWriter implements Closeable, TwoPhaseCommit {
|
|||
} catch (OutOfMemoryError oom) {
|
||||
handleOOM(oom, "deleteAll");
|
||||
} finally {
|
||||
if (!success && infoStream.isEnabled("IW")) {
|
||||
infoStream.message("IW", "hit exception during deleteAll");
|
||||
if (!success) {
|
||||
if (infoStream.isEnabled("IW")) {
|
||||
infoStream.message("IW", "hit exception during deleteAll");
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
||||
|
@ -2063,16 +2088,18 @@ public class IndexWriter implements Closeable, TwoPhaseCommit {
|
|||
|
||||
// Abort all pending & running merges:
|
||||
for (final MergePolicy.OneMerge merge : pendingMerges) {
|
||||
if (infoStream.isEnabled("IW"))
|
||||
if (infoStream.isEnabled("IW")) {
|
||||
infoStream.message("IW", "now abort pending merge " + merge.segString(directory));
|
||||
}
|
||||
merge.abort();
|
||||
mergeFinish(merge);
|
||||
}
|
||||
pendingMerges.clear();
|
||||
|
||||
for (final MergePolicy.OneMerge merge : runningMerges) {
|
||||
if (infoStream.isEnabled("IW"))
|
||||
if (infoStream.isEnabled("IW")) {
|
||||
infoStream.message("IW", "now abort running merge " + merge.segString(directory));
|
||||
}
|
||||
merge.abort();
|
||||
}
|
||||
|
||||
|
@ -2082,8 +2109,9 @@ public class IndexWriter implements Closeable, TwoPhaseCommit {
|
|||
// because the merge threads periodically check if
|
||||
// they are aborted.
|
||||
while(runningMerges.size() > 0) {
|
||||
if (infoStream.isEnabled("IW"))
|
||||
if (infoStream.isEnabled("IW")) {
|
||||
infoStream.message("IW", "now wait for " + runningMerges.size() + " running merge to abort");
|
||||
}
|
||||
doWait();
|
||||
}
|
||||
|
||||
|
@ -2092,7 +2120,9 @@ public class IndexWriter implements Closeable, TwoPhaseCommit {
|
|||
|
||||
assert 0 == mergingSegments.size();
|
||||
|
||||
infoStream.message("IW", "all running merges have aborted");
|
||||
if (infoStream.isEnabled("IW")) {
|
||||
infoStream.message("IW", "all running merges have aborted");
|
||||
}
|
||||
|
||||
} else {
|
||||
// waitForMerges() will ensure any running addIndexes finishes.
|
||||
|
@ -2112,7 +2142,9 @@ public class IndexWriter implements Closeable, TwoPhaseCommit {
|
|||
*/
|
||||
public synchronized void waitForMerges() {
|
||||
ensureOpen(false);
|
||||
infoStream.message("IW", "waitForMerges");
|
||||
if (infoStream.isEnabled("IW")) {
|
||||
infoStream.message("IW", "waitForMerges");
|
||||
}
|
||||
while(pendingMerges.size() > 0 || runningMerges.size() > 0) {
|
||||
doWait();
|
||||
}
|
||||
|
@ -2120,7 +2152,9 @@ public class IndexWriter implements Closeable, TwoPhaseCommit {
|
|||
// sanity check
|
||||
assert 0 == mergingSegments.size();
|
||||
|
||||
infoStream.message("IW", "waitForMerges done");
|
||||
if (infoStream.isEnabled("IW")) {
|
||||
infoStream.message("IW", "waitForMerges done");
|
||||
}
|
||||
}
|
||||
|
||||
/**
|
||||
|
@ -2247,7 +2281,9 @@ public class IndexWriter implements Closeable, TwoPhaseCommit {
|
|||
FrozenBufferedDeletes packet, FrozenBufferedDeletes globalPacket) throws IOException {
|
||||
// Lock order IW -> BDS
|
||||
synchronized (bufferedDeletesStream) {
|
||||
infoStream.message("IW", "publishFlushedSegment");
|
||||
if (infoStream.isEnabled("IW")) {
|
||||
infoStream.message("IW", "publishFlushedSegment");
|
||||
}
|
||||
|
||||
if (globalPacket != null && globalPacket.any()) {
|
||||
bufferedDeletesStream.push(globalPacket);
|
||||
|
@ -2341,7 +2377,10 @@ public class IndexWriter implements Closeable, TwoPhaseCommit {
|
|||
noDupDirs(dirs);
|
||||
|
||||
try {
|
||||
infoStream.message("IW", "flush at addIndexes(Directory...)");
|
||||
if (infoStream.isEnabled("IW")) {
|
||||
infoStream.message("IW", "flush at addIndexes(Directory...)");
|
||||
}
|
||||
|
||||
flush(false, true);
|
||||
|
||||
List<SegmentInfo> infos = new ArrayList<SegmentInfo>();
|
||||
|
@ -2410,7 +2449,9 @@ public class IndexWriter implements Closeable, TwoPhaseCommit {
|
|||
int numDocs = 0;
|
||||
|
||||
try {
|
||||
infoStream.message("IW", "flush at addIndexes(IndexReader...)");
|
||||
if (infoStream.isEnabled("IW")) {
|
||||
infoStream.message("IW", "flush at addIndexes(IndexReader...)");
|
||||
}
|
||||
flush(false, true);
|
||||
|
||||
String mergedName = newSegmentName();
|
||||
|
@ -2638,7 +2679,9 @@ public class IndexWriter implements Closeable, TwoPhaseCommit {
|
|||
success = true;
|
||||
} finally {
|
||||
if (!success) {
|
||||
infoStream.message("IW", "hit exception during prepareCommit");
|
||||
if (infoStream.isEnabled("IW")) {
|
||||
infoStream.message("IW", "hit exception during prepareCommit");
|
||||
}
|
||||
}
|
||||
// Done: finish the full flush!
|
||||
docWriter.finishFullFlush(flushSuccess);
|
||||
|
@ -2722,16 +2765,24 @@ public class IndexWriter implements Closeable, TwoPhaseCommit {
|
|||
|
||||
private final void commitInternal(Map<String,String> commitUserData) throws CorruptIndexException, IOException {
|
||||
|
||||
infoStream.message("IW", "commit: start");
|
||||
if (infoStream.isEnabled("IW")) {
|
||||
infoStream.message("IW", "commit: start");
|
||||
}
|
||||
|
||||
synchronized(commitLock) {
|
||||
infoStream.message("IW", "commit: enter lock");
|
||||
if (infoStream.isEnabled("IW")) {
|
||||
infoStream.message("IW", "commit: enter lock");
|
||||
}
|
||||
|
||||
if (pendingCommit == null) {
|
||||
infoStream.message("IW", "commit: now prepare");
|
||||
if (infoStream.isEnabled("IW")) {
|
||||
infoStream.message("IW", "commit: now prepare");
|
||||
}
|
||||
prepareCommit(commitUserData);
|
||||
} else {
|
||||
infoStream.message("IW", "commit: already prepared");
|
||||
if (infoStream.isEnabled("IW")) {
|
||||
infoStream.message("IW", "commit: already prepared");
|
||||
}
|
||||
}
|
||||
|
||||
finishCommit();
|
||||
|
@ -2742,10 +2793,13 @@ public class IndexWriter implements Closeable, TwoPhaseCommit {
|
|||
|
||||
if (pendingCommit != null) {
|
||||
try {
|
||||
infoStream.message("IW", "commit: pendingCommit != null");
|
||||
if (infoStream.isEnabled("IW")) {
|
||||
infoStream.message("IW", "commit: pendingCommit != null");
|
||||
}
|
||||
pendingCommit.finishCommit(directory, codec);
|
||||
if (infoStream.isEnabled("IW"))
|
||||
if (infoStream.isEnabled("IW")) {
|
||||
infoStream.message("IW", "commit: wrote segments file \"" + pendingCommit.getCurrentSegmentFileName() + "\"");
|
||||
}
|
||||
lastCommitChangeCount = pendingCommitChangeCount;
|
||||
segmentInfos.updateGeneration(pendingCommit);
|
||||
segmentInfos.setUserData(pendingCommit.getUserData());
|
||||
|
@ -2760,10 +2814,14 @@ public class IndexWriter implements Closeable, TwoPhaseCommit {
|
|||
}
|
||||
|
||||
} else {
|
||||
infoStream.message("IW", "commit: pendingCommit == null; skip");
|
||||
if (infoStream.isEnabled("IW")) {
|
||||
infoStream.message("IW", "commit: pendingCommit == null; skip");
|
||||
}
|
||||
}
|
||||
|
||||
infoStream.message("IW", "commit: done");
|
||||
if (infoStream.isEnabled("IW")) {
|
||||
infoStream.message("IW", "commit: done");
|
||||
}
|
||||
}
|
||||
|
||||
// Ensures only one flush() is actually flushing segments
|
||||
|
@ -2833,14 +2891,19 @@ public class IndexWriter implements Closeable, TwoPhaseCommit {
|
|||
// never hit
|
||||
return false;
|
||||
} finally {
|
||||
if (!success)
|
||||
infoStream.message("IW", "hit exception during flush");
|
||||
if (!success) {
|
||||
if (infoStream.isEnabled("IW")) {
|
||||
infoStream.message("IW", "hit exception during flush");
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
final synchronized void maybeApplyDeletes(boolean applyAllDeletes) throws IOException {
|
||||
if (applyAllDeletes) {
|
||||
infoStream.message("IW", "apply all deletes during flush");
|
||||
if (infoStream.isEnabled("IW")) {
|
||||
infoStream.message("IW", "apply all deletes during flush");
|
||||
}
|
||||
applyAllDeletes();
|
||||
} else if (infoStream.isEnabled("IW")) {
|
||||
infoStream.message("IW", "don't apply deletes now delTermCount=" + bufferedDeletesStream.numTerms() + " bytesUsed=" + bufferedDeletesStream.bytesUsed());
|
||||
|
@ -2920,8 +2983,9 @@ public class IndexWriter implements Closeable, TwoPhaseCommit {
|
|||
|
||||
final List<SegmentInfo> sourceSegments = merge.segments;
|
||||
|
||||
if (infoStream.isEnabled("IW"))
|
||||
if (infoStream.isEnabled("IW")) {
|
||||
infoStream.message("IW", "commitMergeDeletes " + merge.segString(directory));
|
||||
}
|
||||
|
||||
// Carefully merge deletes that occurred after we
|
||||
// started merging:
|
||||
|
@ -3018,8 +3082,9 @@ public class IndexWriter implements Closeable, TwoPhaseCommit {
|
|||
throw new IllegalStateException("this writer hit an OutOfMemoryError; cannot complete merge");
|
||||
}
|
||||
|
||||
if (infoStream.isEnabled("IW"))
|
||||
if (infoStream.isEnabled("IW")) {
|
||||
infoStream.message("IW", "commitMerge: " + merge.segString(directory) + " index=" + segString());
|
||||
}
|
||||
|
||||
assert merge.registerDone;
|
||||
|
||||
|
@ -3030,8 +3095,9 @@ public class IndexWriter implements Closeable, TwoPhaseCommit {
|
|||
// file that current segments does not reference), we
|
||||
// abort this merge
|
||||
if (merge.isAborted()) {
|
||||
if (infoStream.isEnabled("IW"))
|
||||
if (infoStream.isEnabled("IW")) {
|
||||
infoStream.message("IW", "commitMerge: skipping merge " + merge.segString(directory) + ": it was aborted");
|
||||
}
|
||||
return false;
|
||||
}
|
||||
|
||||
|
@ -3046,8 +3112,10 @@ public class IndexWriter implements Closeable, TwoPhaseCommit {
|
|||
|
||||
final boolean allDeleted = mergedReader.numDocs() == 0;
|
||||
|
||||
if (allDeleted && infoStream.isEnabled("IW")) {
|
||||
infoStream.message("IW", "merged segment " + merge.info + " is 100% deleted" + (keepFullyDeletedSegments ? "" : "; skipping insert"));
|
||||
if (allDeleted) {
|
||||
if (infoStream.isEnabled("IW")) {
|
||||
infoStream.message("IW", "merged segment " + merge.info + " is 100% deleted" + (keepFullyDeletedSegments ? "" : "; skipping insert"));
|
||||
}
|
||||
}
|
||||
|
||||
final boolean dropSegment = allDeleted && !keepFullyDeletedSegments;
|
||||
|
@ -3133,8 +3201,9 @@ public class IndexWriter implements Closeable, TwoPhaseCommit {
|
|||
try {
|
||||
mergeInit(merge);
|
||||
|
||||
if (infoStream.isEnabled("IW"))
|
||||
if (infoStream.isEnabled("IW")) {
|
||||
infoStream.message("IW", "now merge\n merge=" + merge.segString(directory) + "\n index=" + segString());
|
||||
}
|
||||
|
||||
mergeMiddle(merge);
|
||||
mergeSuccess(merge);
|
||||
|
@ -3147,9 +3216,12 @@ public class IndexWriter implements Closeable, TwoPhaseCommit {
|
|||
mergeFinish(merge);
|
||||
|
||||
if (!success) {
|
||||
infoStream.message("IW", "hit exception during merge");
|
||||
if (merge.info != null && !segmentInfos.contains(merge.info))
|
||||
if (infoStream.isEnabled("IW")) {
|
||||
infoStream.message("IW", "hit exception during merge");
|
||||
}
|
||||
if (merge.info != null && !segmentInfos.contains(merge.info)) {
|
||||
deleter.refresh(merge.info.name);
|
||||
}
|
||||
}
|
||||
|
||||
// This merge (and, generally, any change to the
|
||||
|
@ -3163,8 +3235,10 @@ public class IndexWriter implements Closeable, TwoPhaseCommit {
|
|||
} catch (OutOfMemoryError oom) {
|
||||
handleOOM(oom, "merge");
|
||||
}
|
||||
if (merge.info != null && infoStream.isEnabled("IW")) {
|
||||
infoStream.message("IW", "merge time " + (System.currentTimeMillis()-t0) + " msec for " + merge.info.docCount + " docs");
|
||||
if (merge.info != null) {
|
||||
if (infoStream.isEnabled("IW")) {
|
||||
infoStream.message("IW", "merge time " + (System.currentTimeMillis()-t0) + " msec for " + merge.info.docCount + " docs");
|
||||
}
|
||||
}
|
||||
//System.out.println(Thread.currentThread().getName() + ": merge end");
|
||||
}
|
||||
|
@ -3209,8 +3283,9 @@ public class IndexWriter implements Closeable, TwoPhaseCommit {
|
|||
|
||||
pendingMerges.add(merge);
|
||||
|
||||
if (infoStream.isEnabled("IW"))
|
||||
if (infoStream.isEnabled("IW")) {
|
||||
infoStream.message("IW", "add merge to pendingMerges: " + merge.segString(directory) + " [total " + pendingMerges.size() + " pending]");
|
||||
}
|
||||
|
||||
merge.mergeGen = mergeGen;
|
||||
merge.isExternal = isExternal;
|
||||
|
@ -3227,7 +3302,9 @@ public class IndexWriter implements Closeable, TwoPhaseCommit {
|
|||
builder.append("]");
|
||||
// don't call mergingSegments.toString() could lead to ConcurrentModException
|
||||
// since merge updates the segments FieldInfos
|
||||
infoStream.message("IW", builder.toString());
|
||||
if (infoStream.isEnabled("IW")) {
|
||||
infoStream.message("IW", builder.toString());
|
||||
}
|
||||
}
|
||||
for(SegmentInfo info : merge.segments) {
|
||||
if (infoStream.isEnabled("IW")) {
|
||||
|
@ -3251,7 +3328,9 @@ public class IndexWriter implements Closeable, TwoPhaseCommit {
|
|||
success = true;
|
||||
} finally {
|
||||
if (!success) {
|
||||
infoStream.message("IW", "hit exception in mergeInit");
|
||||
if (infoStream.isEnabled("IW")) {
|
||||
infoStream.message("IW", "hit exception in mergeInit");
|
||||
}
|
||||
mergeFinish(merge);
|
||||
}
|
||||
}
|
||||
|
@ -3544,7 +3623,9 @@ public class IndexWriter implements Closeable, TwoPhaseCommit {
|
|||
handleMergeException(t, merge);
|
||||
} finally {
|
||||
if (!success) {
|
||||
infoStream.message("IW", "hit exception creating compound file during merge");
|
||||
if (infoStream.isEnabled("IW")) {
|
||||
infoStream.message("IW", "hit exception creating compound file during merge");
|
||||
}
|
||||
|
||||
synchronized(this) {
|
||||
deleter.deleteFile(compoundFileName);
|
||||
|
@ -3563,7 +3644,9 @@ public class IndexWriter implements Closeable, TwoPhaseCommit {
|
|||
deleter.deleteNewFiles(merge.info.files());
|
||||
|
||||
if (merge.isAborted()) {
|
||||
infoStream.message("IW", "abort merge after building CFS");
|
||||
if (infoStream.isEnabled("IW")) {
|
||||
infoStream.message("IW", "abort merge after building CFS");
|
||||
}
|
||||
deleter.deleteFile(compoundFileName);
|
||||
return 0;
|
||||
}
|
||||
|
@ -3746,14 +3829,18 @@ public class IndexWriter implements Closeable, TwoPhaseCommit {
|
|||
|
||||
try {
|
||||
|
||||
infoStream.message("IW", "startCommit(): start");
|
||||
if (infoStream.isEnabled("IW")) {
|
||||
infoStream.message("IW", "startCommit(): start");
|
||||
}
|
||||
|
||||
synchronized(this) {
|
||||
|
||||
assert lastCommitChangeCount <= changeCount;
|
||||
|
||||
if (pendingCommitChangeCount == lastCommitChangeCount) {
|
||||
infoStream.message("IW", " skip startCommit(): no changes pending");
|
||||
if (infoStream.isEnabled("IW")) {
|
||||
infoStream.message("IW", " skip startCommit(): no changes pending");
|
||||
}
|
||||
deleter.decRef(filesToCommit);
|
||||
filesToCommit = null;
|
||||
return;
|
||||
|
@ -3796,7 +3883,9 @@ public class IndexWriter implements Closeable, TwoPhaseCommit {
|
|||
pendingCommit = toSync;
|
||||
}
|
||||
|
||||
infoStream.message("IW", "done all syncs");
|
||||
if (infoStream.isEnabled("IW")) {
|
||||
infoStream.message("IW", "done all syncs");
|
||||
}
|
||||
|
||||
assert testPoint("midStartCommitSuccess");
|
||||
|
||||
|
@ -3809,7 +3898,9 @@ public class IndexWriter implements Closeable, TwoPhaseCommit {
|
|||
segmentInfos.updateGeneration(toSync);
|
||||
|
||||
if (!pendingCommitSet) {
|
||||
infoStream.message("IW", "hit exception committing segments file");
|
||||
if (infoStream.isEnabled("IW")) {
|
||||
infoStream.message("IW", "hit exception committing segments file");
|
||||
}
|
||||
|
||||
// Hit exception
|
||||
deleter.decRef(filesToCommit);
|
||||
|
|
|
@ -23,7 +23,6 @@ import java.util.Collection;
|
|||
import java.util.List;
|
||||
import java.util.Map;
|
||||
|
||||
import org.apache.lucene.util.InfoStream;
|
||||
|
||||
/** <p>This class implements a {@link MergePolicy} that tries
|
||||
* to merge segments into levels of exponentially
|
||||
|
@ -424,8 +423,9 @@ public abstract class LogMergePolicy extends MergePolicy {
|
|||
final List<SegmentInfo> segments = segmentInfos.asList();
|
||||
final int numSegments = segments.size();
|
||||
|
||||
if (verbose())
|
||||
if (verbose()) {
|
||||
message("findForcedDeleteMerges: " + numSegments + " segments");
|
||||
}
|
||||
|
||||
MergeSpecification spec = new MergeSpecification();
|
||||
int firstSegmentWithDeletions = -1;
|
||||
|
@ -435,15 +435,17 @@ public abstract class LogMergePolicy extends MergePolicy {
|
|||
final SegmentInfo info = segmentInfos.info(i);
|
||||
int delCount = w.numDeletedDocs(info);
|
||||
if (delCount > 0) {
|
||||
if (verbose())
|
||||
if (verbose()) {
|
||||
message(" segment " + info.name + " has deletions");
|
||||
}
|
||||
if (firstSegmentWithDeletions == -1)
|
||||
firstSegmentWithDeletions = i;
|
||||
else if (i - firstSegmentWithDeletions == mergeFactor) {
|
||||
// We've seen mergeFactor segments in a row with
|
||||
// deletions, so force a merge now:
|
||||
if (verbose())
|
||||
if (verbose()) {
|
||||
message(" add merge " + firstSegmentWithDeletions + " to " + (i-1) + " inclusive");
|
||||
}
|
||||
spec.add(new OneMerge(segments.subList(firstSegmentWithDeletions, i)));
|
||||
firstSegmentWithDeletions = i;
|
||||
}
|
||||
|
@ -451,16 +453,18 @@ public abstract class LogMergePolicy extends MergePolicy {
|
|||
// End of a sequence of segments with deletions, so,
|
||||
// merge those past segments even if it's fewer than
|
||||
// mergeFactor segments
|
||||
if (verbose())
|
||||
if (verbose()) {
|
||||
message(" add merge " + firstSegmentWithDeletions + " to " + (i-1) + " inclusive");
|
||||
}
|
||||
spec.add(new OneMerge(segments.subList(firstSegmentWithDeletions, i)));
|
||||
firstSegmentWithDeletions = -1;
|
||||
}
|
||||
}
|
||||
|
||||
if (firstSegmentWithDeletions != -1) {
|
||||
if (verbose())
|
||||
if (verbose()) {
|
||||
message(" add merge " + firstSegmentWithDeletions + " to " + (numSegments-1) + " inclusive");
|
||||
}
|
||||
spec.add(new OneMerge(segments.subList(firstSegmentWithDeletions, numSegments)));
|
||||
}
|
||||
|
||||
|
@ -500,8 +504,9 @@ public abstract class LogMergePolicy extends MergePolicy {
|
|||
public MergeSpecification findMerges(SegmentInfos infos) throws IOException {
|
||||
|
||||
final int numSegments = infos.size();
|
||||
if (verbose())
|
||||
if (verbose()) {
|
||||
message("findMerges: " + numSegments + " segments");
|
||||
}
|
||||
|
||||
// Compute levels, which is just log (base mergeFactor)
|
||||
// of the size of each segment
|
||||
|
@ -582,8 +587,9 @@ public abstract class LogMergePolicy extends MergePolicy {
|
|||
}
|
||||
upto--;
|
||||
}
|
||||
if (verbose())
|
||||
if (verbose()) {
|
||||
message(" level " + levelBottom + " to " + maxLevel + ": " + (1+upto-start) + " segments");
|
||||
}
|
||||
|
||||
// Finally, record all merges that are viable at this level:
|
||||
int end = start + mergeFactor;
|
||||
|
|
|
@ -19,25 +19,14 @@ package org.apache.lucene.index;
|
|||
|
||||
import java.io.IOException;
|
||||
import java.util.Collection;
|
||||
import java.util.Map;
|
||||
import java.util.concurrent.ConcurrentHashMap;
|
||||
|
||||
import org.apache.lucene.index.codecs.PerDocProducer;
|
||||
import org.apache.lucene.util.Bits;
|
||||
import org.apache.lucene.util.BytesRef;
|
||||
import org.apache.lucene.util.ReaderUtil;
|
||||
import org.apache.lucene.util.MapBackedSet;
|
||||
|
||||
/** An IndexReader which reads multiple indexes, appending
|
||||
* their content. */
|
||||
public class MultiReader extends IndexReader implements Cloneable {
|
||||
protected IndexReader[] subReaders;
|
||||
private final ReaderContext topLevelContext;
|
||||
private int[] starts; // 1st docno for each segment
|
||||
private boolean[] decrefOnClose; // remember which subreaders to decRef on close
|
||||
private int maxDoc = 0;
|
||||
private int numDocs = -1;
|
||||
private boolean hasDeletions = false;
|
||||
public class MultiReader extends BaseMultiReader<IndexReader> {
|
||||
private final boolean[] decrefOnClose; // remember which subreaders to decRef on close
|
||||
|
||||
/**
|
||||
* <p>Construct a MultiReader aggregating the named set of (sub)readers.
|
||||
|
@ -45,81 +34,43 @@ public class MultiReader extends IndexReader implements Cloneable {
|
|||
* @param subReaders set of (sub)readers
|
||||
*/
|
||||
public MultiReader(IndexReader... subReaders) throws IOException {
|
||||
topLevelContext = initialize(subReaders, true);
|
||||
this(subReaders, true);
|
||||
}
|
||||
|
||||
/**
|
||||
* <p>Construct a MultiReader aggregating the named set of (sub)readers.
|
||||
* @param subReaders set of (sub)readers
|
||||
* @param closeSubReaders indicates whether the subreaders should be closed
|
||||
* when this MultiReader is closed
|
||||
* @param subReaders set of (sub)readers
|
||||
*/
|
||||
public MultiReader(IndexReader[] subReaders, boolean closeSubReaders) throws IOException {
|
||||
topLevelContext = initialize(subReaders, closeSubReaders);
|
||||
}
|
||||
|
||||
private ReaderContext initialize(IndexReader[] subReaders, boolean closeSubReaders) throws IOException {
|
||||
this.subReaders = subReaders.clone();
|
||||
starts = new int[subReaders.length + 1]; // build starts array
|
||||
super(subReaders.clone());
|
||||
readerFinishedListeners = new MapBackedSet<ReaderFinishedListener>(new ConcurrentHashMap<ReaderFinishedListener,Boolean>());
|
||||
decrefOnClose = new boolean[subReaders.length];
|
||||
for (int i = 0; i < subReaders.length; i++) {
|
||||
starts[i] = maxDoc;
|
||||
maxDoc += subReaders[i].maxDoc(); // compute maxDocs
|
||||
|
||||
if (!closeSubReaders) {
|
||||
subReaders[i].incRef();
|
||||
decrefOnClose[i] = true;
|
||||
} else {
|
||||
decrefOnClose[i] = false;
|
||||
}
|
||||
|
||||
if (subReaders[i].hasDeletions()) {
|
||||
hasDeletions = true;
|
||||
}
|
||||
}
|
||||
starts[subReaders.length] = maxDoc;
|
||||
readerFinishedListeners = new MapBackedSet<ReaderFinishedListener>(new ConcurrentHashMap<ReaderFinishedListener,Boolean>());
|
||||
return ReaderUtil.buildReaderContext(this);
|
||||
}
|
||||
|
||||
@Override
|
||||
public Fields fields() throws IOException {
|
||||
throw new UnsupportedOperationException("please use MultiFields.getFields, or wrap your IndexReader with SlowMultiReaderWrapper, if you really need a top level Fields");
|
||||
// used only by openIfChaged
|
||||
private MultiReader(IndexReader[] subReaders, boolean[] decrefOnClose,
|
||||
Collection<ReaderFinishedListener> readerFinishedListeners)
|
||||
throws IOException {
|
||||
super(subReaders);
|
||||
this.decrefOnClose = decrefOnClose;
|
||||
this.readerFinishedListeners = readerFinishedListeners;
|
||||
}
|
||||
|
||||
/**
|
||||
* Tries to reopen the subreaders.
|
||||
* <br>
|
||||
* If one or more subreaders could be re-opened (i. e. IndexReader.openIfChanged(subReader)
|
||||
* returned a new instance), then a new MultiReader instance
|
||||
* is returned, otherwise this instance is returned.
|
||||
* <p>
|
||||
* A re-opened instance might share one or more subreaders with the old
|
||||
* instance. Index modification operations result in undefined behavior
|
||||
* when performed before the old instance is closed.
|
||||
* (see {@link IndexReader#openIfChanged}).
|
||||
* <p>
|
||||
* If subreaders are shared, then the reference count of those
|
||||
* readers is increased to ensure that the subreaders remain open
|
||||
* until the last referring reader is closed.
|
||||
*
|
||||
* @throws CorruptIndexException if the index is corrupt
|
||||
* @throws IOException if there is a low-level IO error
|
||||
*/
|
||||
@Override
|
||||
protected synchronized IndexReader doOpenIfChanged() throws CorruptIndexException, IOException {
|
||||
return doReopen(false);
|
||||
}
|
||||
|
||||
/**
|
||||
* Clones the subreaders.
|
||||
* (see {@link IndexReader#clone()}).
|
||||
* <br>
|
||||
* <p>
|
||||
* If subreaders are shared, then the reference count of those
|
||||
* readers is increased to ensure that the subreaders remain open
|
||||
* until the last referring reader is closed.
|
||||
*/
|
||||
@Override
|
||||
public synchronized Object clone() {
|
||||
try {
|
||||
|
@ -129,18 +80,6 @@ public class MultiReader extends IndexReader implements Cloneable {
|
|||
}
|
||||
}
|
||||
|
||||
@Override
|
||||
public Bits getLiveDocs() {
|
||||
throw new UnsupportedOperationException("please use MultiFields.getLiveDocs, or wrap your IndexReader with SlowMultiReaderWrapper, if you really need a top level Bits liveDocs");
|
||||
}
|
||||
|
||||
/**
|
||||
* If clone is true then we clone each of the subreaders
|
||||
* @param doClone
|
||||
* @return New IndexReader, or null if open/clone is not necessary
|
||||
* @throws CorruptIndexException
|
||||
* @throws IOException
|
||||
*/
|
||||
private IndexReader doReopen(boolean doClone) throws CorruptIndexException, IOException {
|
||||
ensureOpen();
|
||||
|
||||
|
@ -186,102 +125,30 @@ public class MultiReader extends IndexReader implements Cloneable {
|
|||
newDecrefOnClose[i] = true;
|
||||
}
|
||||
}
|
||||
MultiReader mr = new MultiReader(newSubReaders);
|
||||
mr.decrefOnClose = newDecrefOnClose;
|
||||
return mr;
|
||||
return new MultiReader(newSubReaders, newDecrefOnClose, readerFinishedListeners);
|
||||
} else {
|
||||
return null;
|
||||
}
|
||||
}
|
||||
|
||||
@Override
|
||||
public Fields getTermVectors(int docID) throws IOException {
|
||||
ensureOpen();
|
||||
int i = readerIndex(docID); // find segment num
|
||||
return subReaders[i].getTermVectors(docID - starts[i]); // dispatch to segment
|
||||
}
|
||||
|
||||
@Override
|
||||
public int numDocs() {
|
||||
// Don't call ensureOpen() here (it could affect performance)
|
||||
// NOTE: multiple threads may wind up init'ing
|
||||
// numDocs... but that's harmless
|
||||
if (numDocs == -1) { // check cache
|
||||
int n = 0; // cache miss--recompute
|
||||
for (int i = 0; i < subReaders.length; i++)
|
||||
n += subReaders[i].numDocs(); // sum from readers
|
||||
numDocs = n;
|
||||
}
|
||||
return numDocs;
|
||||
}
|
||||
|
||||
@Override
|
||||
public int maxDoc() {
|
||||
// Don't call ensureOpen() here (it could affect performance)
|
||||
return maxDoc;
|
||||
}
|
||||
|
||||
@Override
|
||||
public void document(int docID, StoredFieldVisitor visitor) throws CorruptIndexException, IOException {
|
||||
ensureOpen();
|
||||
int i = readerIndex(docID); // find segment num
|
||||
subReaders[i].document(docID - starts[i], visitor); // dispatch to segment reader
|
||||
}
|
||||
|
||||
@Override
|
||||
public boolean hasDeletions() {
|
||||
ensureOpen();
|
||||
return hasDeletions;
|
||||
}
|
||||
|
||||
private int readerIndex(int n) { // find reader for doc n:
|
||||
return DirectoryReader.readerIndex(n, this.starts, this.subReaders.length);
|
||||
}
|
||||
|
||||
@Override
|
||||
public boolean hasNorms(String field) throws IOException {
|
||||
ensureOpen();
|
||||
for (int i = 0; i < subReaders.length; i++) {
|
||||
if (subReaders[i].hasNorms(field)) return true;
|
||||
}
|
||||
return false;
|
||||
}
|
||||
|
||||
@Override
|
||||
public synchronized byte[] norms(String field) throws IOException {
|
||||
throw new UnsupportedOperationException("please use MultiNorms.norms, or wrap your IndexReader with SlowMultiReaderWrapper, if you really need a top level norms");
|
||||
}
|
||||
|
||||
@Override
|
||||
public int docFreq(String field, BytesRef t) throws IOException {
|
||||
ensureOpen();
|
||||
int total = 0; // sum freqs in segments
|
||||
for (int i = 0; i < subReaders.length; i++) {
|
||||
total += subReaders[i].docFreq(field, t);
|
||||
}
|
||||
return total;
|
||||
}
|
||||
|
||||
@Override
|
||||
protected synchronized void doClose() throws IOException {
|
||||
IOException ioe = null;
|
||||
for (int i = 0; i < subReaders.length; i++) {
|
||||
if (decrefOnClose[i]) {
|
||||
subReaders[i].decRef();
|
||||
} else {
|
||||
subReaders[i].close();
|
||||
try {
|
||||
if (decrefOnClose[i]) {
|
||||
subReaders[i].decRef();
|
||||
} else {
|
||||
subReaders[i].close();
|
||||
}
|
||||
} catch (IOException e) {
|
||||
if (ioe == null) ioe = e;
|
||||
}
|
||||
}
|
||||
// throw the first exception
|
||||
if (ioe != null) throw ioe;
|
||||
}
|
||||
|
||||
@Override
|
||||
public Collection<String> getFieldNames (IndexReader.FieldOption fieldNames) {
|
||||
ensureOpen();
|
||||
return DirectoryReader.getFieldNames(fieldNames, this.subReaders);
|
||||
}
|
||||
|
||||
/**
|
||||
* Checks recursively if all subreaders are up to date.
|
||||
*/
|
||||
@Override
|
||||
public boolean isCurrent() throws CorruptIndexException, IOException {
|
||||
ensureOpen();
|
||||
|
@ -303,17 +170,6 @@ public class MultiReader extends IndexReader implements Cloneable {
|
|||
throw new UnsupportedOperationException("MultiReader does not support this method.");
|
||||
}
|
||||
|
||||
@Override
|
||||
public IndexReader[] getSequentialSubReaders() {
|
||||
return subReaders;
|
||||
}
|
||||
|
||||
@Override
|
||||
public ReaderContext getTopReaderContext() {
|
||||
ensureOpen();
|
||||
return topLevelContext;
|
||||
}
|
||||
|
||||
@Override
|
||||
public void addReaderFinishedListener(ReaderFinishedListener listener) {
|
||||
super.addReaderFinishedListener(listener);
|
||||
|
@ -329,9 +185,4 @@ public class MultiReader extends IndexReader implements Cloneable {
|
|||
sub.removeReaderFinishedListener(listener);
|
||||
}
|
||||
}
|
||||
|
||||
@Override
|
||||
public DocValues docValues(String field) throws IOException {
|
||||
throw new UnsupportedOperationException("please use MultiDocValues#getDocValues, or wrap your IndexReader with SlowMultiReaderWrapper, if you really need a top level DocValues");
|
||||
}
|
||||
}
|
||||
|
|
|
@ -1,60 +0,0 @@
|
|||
package org.apache.lucene.index;
|
||||
|
||||
/**
|
||||
* Licensed to the Apache Software Foundation (ASF) under one or more
|
||||
* contributor license agreements. See the NOTICE file distributed with
|
||||
* this work for additional information regarding copyright ownership.
|
||||
* The ASF licenses this file to You under the Apache License, Version 2.0
|
||||
* (the "License"); you may not use this file except in compliance with
|
||||
* the License. You may obtain a copy of the License at
|
||||
*
|
||||
* http://www.apache.org/licenses/LICENSE-2.0
|
||||
*
|
||||
* Unless required by applicable law or agreed to in writing, software
|
||||
* distributed under the License is distributed on an "AS IS" BASIS,
|
||||
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
|
||||
* See the License for the specific language governing permissions and
|
||||
* limitations under the License.
|
||||
*/
|
||||
|
||||
import java.io.Reader;
|
||||
|
||||
/** Used by DocumentsWriter to implemented a StringReader
|
||||
* that can be reset to a new string; we use this when
|
||||
* tokenizing the string value from a Field. */
|
||||
final class ReusableStringReader extends Reader {
|
||||
int upto;
|
||||
int left;
|
||||
String s;
|
||||
void init(String s) {
|
||||
this.s = s;
|
||||
left = s.length();
|
||||
this.upto = 0;
|
||||
}
|
||||
@Override
|
||||
public int read(char[] c) {
|
||||
return read(c, 0, c.length);
|
||||
}
|
||||
@Override
|
||||
public int read(char[] c, int off, int len) {
|
||||
if (left > len) {
|
||||
s.getChars(upto, upto+len, c, off);
|
||||
upto += len;
|
||||
left -= len;
|
||||
return len;
|
||||
} else if (0 == left) {
|
||||
// don't keep a reference (s could have been very large)
|
||||
s = null;
|
||||
return -1;
|
||||
} else {
|
||||
s.getChars(upto, upto+left, c, off);
|
||||
int r = left;
|
||||
left = 0;
|
||||
upto = s.length();
|
||||
return r;
|
||||
}
|
||||
}
|
||||
@Override
|
||||
public void close() {}
|
||||
}
|
||||
|
|
@ -26,7 +26,6 @@ import java.util.Comparator;
|
|||
import java.util.List;
|
||||
import java.util.ArrayList;
|
||||
|
||||
import org.apache.lucene.util.InfoStream;
|
||||
|
||||
/**
|
||||
* Merges segments of approximately equal size, subject to
|
||||
|
@ -346,7 +345,9 @@ public class TieredMergePolicy extends MergePolicy {
|
|||
|
||||
final boolean maxMergeIsRunning = mergingBytes >= maxMergedSegmentBytes;
|
||||
|
||||
message(" allowedSegmentCount=" + allowedSegCountInt + " vs count=" + infosSorted.size() + " (eligible count=" + eligible.size() + ") tooBigCount=" + tooBigCount);
|
||||
if (verbose()) {
|
||||
message(" allowedSegmentCount=" + allowedSegCountInt + " vs count=" + infosSorted.size() + " (eligible count=" + eligible.size() + ") tooBigCount=" + tooBigCount);
|
||||
}
|
||||
|
||||
if (eligible.size() == 0) {
|
||||
return spec;
|
||||
|
@ -386,7 +387,9 @@ public class TieredMergePolicy extends MergePolicy {
|
|||
}
|
||||
|
||||
final MergeScore score = score(candidate, hitTooLarge, mergingBytes);
|
||||
message(" maybe=" + writer.get().segString(candidate) + " score=" + score.getScore() + " " + score.getExplanation() + " tooLarge=" + hitTooLarge + " size=" + String.format("%.3f MB", totAfterMergeBytes/1024./1024.));
|
||||
if (verbose()) {
|
||||
message(" maybe=" + writer.get().segString(candidate) + " score=" + score.getScore() + " " + score.getExplanation() + " tooLarge=" + hitTooLarge + " size=" + String.format("%.3f MB", totAfterMergeBytes/1024./1024.));
|
||||
}
|
||||
|
||||
// If we are already running a max sized merge
|
||||
// (maxMergeIsRunning), don't allow another max
|
||||
|
@ -649,9 +652,7 @@ public class TieredMergePolicy extends MergePolicy {
|
|||
}
|
||||
|
||||
private void message(String message) {
|
||||
if (verbose()) {
|
||||
writer.get().infoStream.message("TMP", message);
|
||||
}
|
||||
writer.get().infoStream.message("TMP", message);
|
||||
}
|
||||
|
||||
@Override
|
||||
|
|
|
@ -18,7 +18,6 @@ package org.apache.lucene.index;
|
|||
*/
|
||||
|
||||
import org.apache.lucene.util.Constants;
|
||||
import org.apache.lucene.util.InfoStream;
|
||||
|
||||
import java.io.IOException;
|
||||
import java.util.ArrayList;
|
||||
|
@ -90,7 +89,9 @@ public class UpgradeIndexMergePolicy extends MergePolicy {
|
|||
}
|
||||
}
|
||||
|
||||
if (verbose()) message("findForcedMerges: segmentsToUpgrade=" + oldSegments);
|
||||
if (verbose()) {
|
||||
message("findForcedMerges: segmentsToUpgrade=" + oldSegments);
|
||||
}
|
||||
|
||||
if (oldSegments.isEmpty())
|
||||
return null;
|
||||
|
@ -107,9 +108,10 @@ public class UpgradeIndexMergePolicy extends MergePolicy {
|
|||
}
|
||||
|
||||
if (!oldSegments.isEmpty()) {
|
||||
if (verbose())
|
||||
if (verbose()) {
|
||||
message("findForcedMerges: " + base.getClass().getSimpleName() +
|
||||
" does not want to merge all old segments, merge remaining ones into new segment: " + oldSegments);
|
||||
}
|
||||
final List<SegmentInfo> newInfos = new ArrayList<SegmentInfo>();
|
||||
for (final SegmentInfo si : segmentInfos) {
|
||||
if (oldSegments.containsKey(si)) {
|
||||
|
@ -152,9 +154,6 @@ public class UpgradeIndexMergePolicy extends MergePolicy {
|
|||
}
|
||||
|
||||
private void message(String message) {
|
||||
if (verbose()) {
|
||||
writer.get().infoStream.message("UPGMP", message);
|
||||
}
|
||||
writer.get().infoStream.message("UPGMP", message);
|
||||
}
|
||||
|
||||
}
|
||||
|
|
|
@ -22,7 +22,6 @@ import java.io.IOException;
|
|||
import org.apache.lucene.index.CorruptIndexException;
|
||||
import org.apache.lucene.index.FieldInfo;
|
||||
import org.apache.lucene.index.FieldInfos;
|
||||
import org.apache.lucene.index.FieldReaderException;
|
||||
import org.apache.lucene.index.IndexFileNames;
|
||||
import org.apache.lucene.index.IndexFormatTooNewException;
|
||||
import org.apache.lucene.index.IndexFormatTooOldException;
|
||||
|
@ -225,7 +224,7 @@ public final class Lucene40StoredFieldsReader extends StoredFieldsReader impleme
|
|||
visitor.doubleField(info, Double.longBitsToDouble(fieldsStream.readLong()));
|
||||
return;
|
||||
default:
|
||||
throw new FieldReaderException("Invalid numeric type: " + Integer.toHexString(numeric));
|
||||
throw new CorruptIndexException("Invalid numeric type: " + Integer.toHexString(numeric));
|
||||
}
|
||||
} else {
|
||||
final int length = fieldsStream.readVInt();
|
||||
|
@ -252,7 +251,7 @@ public final class Lucene40StoredFieldsReader extends StoredFieldsReader impleme
|
|||
fieldsStream.readLong();
|
||||
return;
|
||||
default:
|
||||
throw new FieldReaderException("Invalid numeric type: " + Integer.toHexString(numeric));
|
||||
throw new CorruptIndexException("Invalid numeric type: " + Integer.toHexString(numeric));
|
||||
}
|
||||
} else {
|
||||
final int length = fieldsStream.readVInt();
|
||||
|
|
|
@ -42,8 +42,6 @@ import org.apache.lucene.index.Terms;
|
|||
import org.apache.lucene.search.similarities.DefaultSimilarityProvider;
|
||||
import org.apache.lucene.search.similarities.SimilarityProvider;
|
||||
import org.apache.lucene.store.NIOFSDirectory; // javadoc
|
||||
import org.apache.lucene.util.Bits;
|
||||
import org.apache.lucene.util.BytesRef;
|
||||
import org.apache.lucene.util.ReaderUtil;
|
||||
import org.apache.lucene.util.TermContext;
|
||||
import org.apache.lucene.util.ThreadInterruptedException;
|
||||
|
@ -73,7 +71,7 @@ import org.apache.lucene.util.ThreadInterruptedException;
|
|||
* synchronize on the <code>IndexSearcher</code> instance;
|
||||
* use your own (non-Lucene) objects instead.</p>
|
||||
*/
|
||||
public class IndexSearcher implements Closeable {
|
||||
public class IndexSearcher {
|
||||
final IndexReader reader; // package private for testing!
|
||||
|
||||
// NOTE: these members might change in incompatible ways
|
||||
|
@ -199,10 +197,6 @@ public class IndexSearcher implements Closeable {
|
|||
return similarityProvider;
|
||||
}
|
||||
|
||||
@Override
|
||||
public void close() throws IOException {
|
||||
}
|
||||
|
||||
/** @lucene.internal */
|
||||
protected Query wrapFilter(Query query, Filter filter) {
|
||||
return (filter == null) ? query : new FilteredQuery(query, filter);
|
||||
|
|
|
@ -21,7 +21,6 @@ import java.io.Closeable;
|
|||
import java.io.IOException;
|
||||
import java.util.List;
|
||||
import java.util.concurrent.CopyOnWriteArrayList;
|
||||
import java.util.concurrent.ExecutorService;
|
||||
import java.util.concurrent.TimeUnit;
|
||||
import java.util.concurrent.atomic.AtomicLong;
|
||||
import java.util.concurrent.locks.Condition;
|
||||
|
@ -34,6 +33,7 @@ import org.apache.lucene.index.IndexWriter;
|
|||
import org.apache.lucene.index.IndexableField;
|
||||
import org.apache.lucene.index.Term;
|
||||
import org.apache.lucene.search.IndexSearcher; // javadocs
|
||||
import org.apache.lucene.search.SearcherFactory;
|
||||
import org.apache.lucene.store.Directory;
|
||||
import org.apache.lucene.util.IOUtils;
|
||||
import org.apache.lucene.util.ThreadInterruptedException;
|
||||
|
@ -49,6 +49,8 @@ import org.apache.lucene.util.ThreadInterruptedException;
|
|||
* #addWaitingListener} so your reopener is notified when a
|
||||
* caller is waiting for a specific generation searcher. </p>
|
||||
*
|
||||
* @see SearcherFactory
|
||||
*
|
||||
* @lucene.experimental
|
||||
*/
|
||||
|
||||
|
@ -65,60 +67,30 @@ public class NRTManager implements Closeable {
|
|||
/**
|
||||
* Create new NRTManager.
|
||||
*
|
||||
* @param writer IndexWriter to open near-real-time
|
||||
* readers
|
||||
* @param warmer optional {@link SearcherWarmer}. Pass
|
||||
* null if you don't require the searcher to warmed
|
||||
* before going live. If this is non-null then a
|
||||
* merged segment warmer is installed on the
|
||||
* provided IndexWriter's config.
|
||||
*
|
||||
* <p><b>NOTE</b>: the provided {@link SearcherWarmer} is
|
||||
* not invoked for the initial searcher; you should
|
||||
* warm it yourself if necessary.
|
||||
* @param writer IndexWriter to open near-real-time
|
||||
* readers
|
||||
* @param searcherFactory An optional {@link SearcherFactory}. Pass
|
||||
* <code>null</code> if you don't require the searcher to be warmed
|
||||
* before going live or other custom behavior.
|
||||
*/
|
||||
public NRTManager(IndexWriter writer, SearcherWarmer warmer) throws IOException {
|
||||
this(writer, null, warmer, true);
|
||||
}
|
||||
|
||||
/**
|
||||
* Create new NRTManager.
|
||||
*
|
||||
* @param writer IndexWriter to open near-real-time
|
||||
* readers
|
||||
* @param es optional ExecutorService so different segments can
|
||||
* be searched concurrently (see {@link IndexSearcher#IndexSearcher(IndexReader, ExecutorService)}.
|
||||
* Pass <code>null</code> to search segments sequentially.
|
||||
* @param warmer optional {@link SearcherWarmer}. Pass
|
||||
* null if you don't require the searcher to warmed
|
||||
* before going live. If this is non-null then a
|
||||
* merged segment warmer is installed on the
|
||||
* provided IndexWriter's config.
|
||||
*
|
||||
* <p><b>NOTE</b>: the provided {@link SearcherWarmer} is
|
||||
* not invoked for the initial searcher; you should
|
||||
* warm it yourself if necessary.
|
||||
*/
|
||||
public NRTManager(IndexWriter writer, ExecutorService es,
|
||||
SearcherWarmer warmer) throws IOException {
|
||||
this(writer, es, warmer, true);
|
||||
public NRTManager(IndexWriter writer, SearcherFactory searcherFactory) throws IOException {
|
||||
this(writer, searcherFactory, true);
|
||||
}
|
||||
|
||||
/**
|
||||
* Expert: just like {@link
|
||||
* #NRTManager(IndexWriter,ExecutorService,SearcherWarmer)},
|
||||
* #NRTManager(IndexWriter,SearcherFactory)},
|
||||
* but you can also specify whether every searcher must
|
||||
* apply deletes. This is useful for cases where certain
|
||||
* uses can tolerate seeing some deleted docs, since
|
||||
* reopen time is faster if deletes need not be applied. */
|
||||
public NRTManager(IndexWriter writer, ExecutorService es,
|
||||
SearcherWarmer warmer, boolean alwaysApplyDeletes) throws IOException {
|
||||
public NRTManager(IndexWriter writer, SearcherFactory searcherFactory, boolean alwaysApplyDeletes) throws IOException {
|
||||
this.writer = writer;
|
||||
if (alwaysApplyDeletes) {
|
||||
withoutDeletes = withDeletes = new SearcherManagerRef(true, 0, new SearcherManager(writer, true, warmer, es));
|
||||
withoutDeletes = withDeletes = new SearcherManagerRef(true, 0, new SearcherManager(writer, true, searcherFactory));
|
||||
} else {
|
||||
withDeletes = new SearcherManagerRef(true, 0, new SearcherManager(writer, true, warmer, es));
|
||||
withoutDeletes = new SearcherManagerRef(false, 0, new SearcherManager(writer, false, warmer, es));
|
||||
withDeletes = new SearcherManagerRef(true, 0, new SearcherManager(writer, true, searcherFactory));
|
||||
withoutDeletes = new SearcherManagerRef(false, 0, new SearcherManager(writer, false, searcherFactory));
|
||||
}
|
||||
indexingGen = new AtomicLong(1);
|
||||
}
|
||||
|
|
|
@ -0,0 +1,57 @@
|
|||
package org.apache.lucene.search;
|
||||
|
||||
/**
|
||||
* Licensed to the Apache Software Foundation (ASF) under one or more
|
||||
* contributor license agreements. See the NOTICE file distributed with
|
||||
* this work for additional information regarding copyright ownership.
|
||||
* The ASF licenses this file to You under the Apache License, Version 2.0
|
||||
* (the "License"); you may not use this file except in compliance with
|
||||
* the License. You may obtain a copy of the License at
|
||||
*
|
||||
* http://www.apache.org/licenses/LICENSE-2.0
|
||||
*
|
||||
* Unless required by applicable law or agreed to in writing, software
|
||||
* distributed under the License is distributed on an "AS IS" BASIS,
|
||||
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
|
||||
* See the License for the specific language governing permissions and
|
||||
* limitations under the License.
|
||||
*/
|
||||
|
||||
import java.io.IOException;
|
||||
import java.util.concurrent.ExecutorService; // javadocs
|
||||
|
||||
import org.apache.lucene.index.IndexReader;
|
||||
import org.apache.lucene.index.IndexWriter; // javadocs
|
||||
import org.apache.lucene.index.IndexWriterConfig; // javadocs
|
||||
import org.apache.lucene.search.similarities.SimilarityProvider; // javadocs
|
||||
|
||||
/**
|
||||
* Factory class used by {@link SearcherManager} and {@link NRTManager} to
|
||||
* create new IndexSearchers. The default implementation just creates
|
||||
* an IndexSearcher with no custom behavior:
|
||||
*
|
||||
* <pre class="prettyprint">
|
||||
* public IndexSearcher newSearcher(IndexReader r) throws IOException {
|
||||
* return new IndexSearcher(r);
|
||||
* }
|
||||
* </pre>
|
||||
*
|
||||
* You can pass your own factory instead if you want custom behavior, such as:
|
||||
* <ul>
|
||||
* <li>Setting a custom scoring model: {@link IndexSearcher#setSimilarityProvider(SimilarityProvider)}
|
||||
* <li>Parallel per-segment search: {@link IndexSearcher#IndexSearcher(IndexReader, ExecutorService)}
|
||||
* <li>Return custom subclasses of IndexSearcher (for example that implement distributed scoring)
|
||||
* <li>Run queries to warm your IndexSearcher before it is used. Note: when using near-realtime search
|
||||
* you may want to also {@link IndexWriterConfig#setMergedSegmentWarmer(IndexWriter.IndexReaderWarmer)} to warm
|
||||
* newly merged segments in the background, outside of the reopen path.
|
||||
* </ul>
|
||||
* @lucene.experimental
|
||||
*/
|
||||
public class SearcherFactory {
|
||||
/**
|
||||
* Returns a new IndexSearcher over the given reader.
|
||||
*/
|
||||
public IndexSearcher newSearcher(IndexReader reader) throws IOException {
|
||||
return new IndexSearcher(reader);
|
||||
}
|
||||
}
|
|
@ -17,8 +17,8 @@ package org.apache.lucene.search;
|
|||
* limitations under the License.
|
||||
*/
|
||||
|
||||
import java.io.Closeable;
|
||||
import java.io.IOException;
|
||||
import java.util.concurrent.ExecutorService;
|
||||
import java.util.concurrent.Semaphore;
|
||||
|
||||
import org.apache.lucene.index.CorruptIndexException;
|
||||
|
@ -61,14 +61,15 @@ import org.apache.lucene.store.Directory;
|
|||
* {@link NRTManager} since that class pulls near-real-time readers from the
|
||||
* IndexWriter.
|
||||
*
|
||||
* @see SearcherFactory
|
||||
*
|
||||
* @lucene.experimental
|
||||
*/
|
||||
|
||||
public final class SearcherManager {
|
||||
public final class SearcherManager implements Closeable {
|
||||
|
||||
private volatile IndexSearcher currentSearcher;
|
||||
private final ExecutorService es;
|
||||
private final SearcherWarmer warmer;
|
||||
private final SearcherFactory searcherFactory;
|
||||
private final Semaphore reopenLock = new Semaphore(1);
|
||||
|
||||
/**
|
||||
|
@ -81,60 +82,41 @@ public final class SearcherManager {
|
|||
* Applying deletes can be costly, so if your app can tolerate deleted documents
|
||||
* being returned you might gain some performance by passing <code>false</code>.
|
||||
* See {@link IndexReader#openIfChanged(IndexReader, IndexWriter, boolean)}.
|
||||
* @param warmer An optional {@link SearcherWarmer}. Pass
|
||||
* <code>null</code> if you don't require the searcher to warmed
|
||||
* before going live. If this is <code>non-null</code> then a
|
||||
* merged segment warmer is installed on the
|
||||
* provided IndexWriter's config.
|
||||
* @param es An optional {@link ExecutorService} so different segments can
|
||||
* be searched concurrently (see {@link
|
||||
* IndexSearcher#IndexSearcher(IndexReader,ExecutorService)}. Pass <code>null</code>
|
||||
* to search segments sequentially.
|
||||
* @param searcherFactory An optional {@link SearcherFactory}. Pass
|
||||
* <code>null</code> if you don't require the searcher to be warmed
|
||||
* before going live or other custom behavior.
|
||||
*
|
||||
* @throws IOException
|
||||
*/
|
||||
public SearcherManager(IndexWriter writer, boolean applyAllDeletes,
|
||||
final SearcherWarmer warmer, final ExecutorService es) throws IOException {
|
||||
this.es = es;
|
||||
this.warmer = warmer;
|
||||
currentSearcher = new IndexSearcher(IndexReader.open(writer, applyAllDeletes));
|
||||
if (warmer != null) {
|
||||
writer.getConfig().setMergedSegmentWarmer(
|
||||
new IndexWriter.IndexReaderWarmer() {
|
||||
@Override
|
||||
public void warm(IndexReader reader) throws IOException {
|
||||
warmer.warm(new IndexSearcher(reader, es));
|
||||
}
|
||||
});
|
||||
public SearcherManager(IndexWriter writer, boolean applyAllDeletes, SearcherFactory searcherFactory) throws IOException {
|
||||
if (searcherFactory == null) {
|
||||
searcherFactory = new SearcherFactory();
|
||||
}
|
||||
this.searcherFactory = searcherFactory;
|
||||
currentSearcher = searcherFactory.newSearcher(IndexReader.open(writer, applyAllDeletes));
|
||||
}
|
||||
|
||||
/**
|
||||
* Creates and returns a new SearcherManager from the given {@link Directory}.
|
||||
* @param dir the directory to open the IndexReader on.
|
||||
* @param warmer An optional {@link SearcherWarmer}. Pass
|
||||
* <code>null</code> if you don't require the searcher to warmed
|
||||
* before going live. If this is <code>non-null</code> then a
|
||||
* merged segment warmer is installed on the
|
||||
* provided IndexWriter's config.
|
||||
* @param es And optional {@link ExecutorService} so different segments can
|
||||
* be searched concurrently (see {@link
|
||||
* IndexSearcher#IndexSearcher(IndexReader,ExecutorService)}. Pass <code>null</code>
|
||||
* to search segments sequentially.
|
||||
* @param searcherFactory An optional {@link SearcherFactory}. Pass
|
||||
* <code>null</code> if you don't require the searcher to be warmed
|
||||
* before going live or other custom behavior.
|
||||
*
|
||||
* @throws IOException
|
||||
*/
|
||||
public SearcherManager(Directory dir, SearcherWarmer warmer,
|
||||
ExecutorService es) throws IOException {
|
||||
this.es = es;
|
||||
this.warmer = warmer;
|
||||
currentSearcher = new IndexSearcher(IndexReader.open(dir), es);
|
||||
public SearcherManager(Directory dir, SearcherFactory searcherFactory) throws IOException {
|
||||
if (searcherFactory == null) {
|
||||
searcherFactory = new SearcherFactory();
|
||||
}
|
||||
this.searcherFactory = searcherFactory;
|
||||
currentSearcher = searcherFactory.newSearcher(IndexReader.open(dir));
|
||||
}
|
||||
|
||||
/**
|
||||
* You must call this, periodically, to perform a reopen. This calls
|
||||
* {@link IndexReader#openIfChanged(IndexReader)} with the underlying reader, and if that returns a
|
||||
* new reader, it's warmed (if you provided a {@link SearcherWarmer} and then
|
||||
* new reader, it's warmed (if you provided a {@link SearcherFactory} and then
|
||||
* swapped into production.
|
||||
*
|
||||
* <p>
|
||||
|
@ -167,12 +149,9 @@ public final class SearcherManager {
|
|||
release(searcherToReopen);
|
||||
}
|
||||
if (newReader != null) {
|
||||
final IndexSearcher newSearcher = new IndexSearcher(newReader, es);
|
||||
final IndexSearcher newSearcher = searcherFactory.newSearcher(newReader);
|
||||
boolean success = false;
|
||||
try {
|
||||
if (warmer != null) {
|
||||
warmer.warm(newSearcher);
|
||||
}
|
||||
swapSearcher(newSearcher);
|
||||
success = true;
|
||||
} finally {
|
||||
|
@ -260,5 +239,4 @@ public final class SearcherManager {
|
|||
currentSearcher = newSearcher;
|
||||
release(oldSearcher);
|
||||
}
|
||||
|
||||
}
|
||||
|
|
|
@ -1,34 +0,0 @@
|
|||
package org.apache.lucene.search;
|
||||
|
||||
/**
|
||||
* Licensed to the Apache Software Foundation (ASF) under one or more
|
||||
* contributor license agreements. See the NOTICE file distributed with
|
||||
* this work for additional information regarding copyright ownership.
|
||||
* The ASF licenses this file to You under the Apache License, Version 2.0
|
||||
* (the "License"); you may not use this file except in compliance with
|
||||
* the License. You may obtain a copy of the License at
|
||||
*
|
||||
* http://www.apache.org/licenses/LICENSE-2.0
|
||||
*
|
||||
* Unless required by applicable law or agreed to in writing, software
|
||||
* distributed under the License is distributed on an "AS IS" BASIS,
|
||||
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
|
||||
* See the License for the specific language governing permissions and
|
||||
* limitations under the License.
|
||||
*/
|
||||
|
||||
import java.io.IOException;
|
||||
|
||||
import org.apache.lucene.search.NRTManager; // javadocs
|
||||
|
||||
/** Pass an implementation of this to {@link NRTManager} or
|
||||
* {@link SearcherManager} to warm a new {@link
|
||||
* IndexSearcher} before it's put into production.
|
||||
*
|
||||
* @lucene.experimental */
|
||||
|
||||
public interface SearcherWarmer {
|
||||
// TODO: can we somehow merge this w/ IW's
|
||||
// IndexReaderWarmer.... should IW switch to this?
|
||||
public void warm(IndexSearcher s) throws IOException;
|
||||
}
|
|
@ -96,7 +96,7 @@ public final class IOUtils {
|
|||
}
|
||||
|
||||
/** @see #closeWhileHandlingException(Exception, Closeable...) */
|
||||
public static <E extends Exception> void closeWhileHandlingException(E priorException, Iterable<Closeable> objects) throws E, IOException {
|
||||
public static <E extends Exception> void closeWhileHandlingException(E priorException, Iterable<? extends Closeable> objects) throws E, IOException {
|
||||
Throwable th = null;
|
||||
|
||||
for (Closeable object : objects) {
|
||||
|
|
|
@ -26,10 +26,14 @@ public abstract class InfoStream implements Closeable {
|
|||
public static final InfoStream NO_OUTPUT = new NoOutput();
|
||||
private static final class NoOutput extends InfoStream {
|
||||
@Override
|
||||
public void message(String component, String message) {}
|
||||
public void message(String component, String message) {
|
||||
assert false: "message() should not be called when isEnabled returns false";
|
||||
}
|
||||
|
||||
@Override
|
||||
public boolean isEnabled(String component) { return false; }
|
||||
public boolean isEnabled(String component) {
|
||||
return false;
|
||||
}
|
||||
|
||||
@Override
|
||||
public void close() {}
|
||||
|
|
|
@ -189,7 +189,7 @@ public final class ReaderUtil {
|
|||
|
||||
int newDocBase = 0;
|
||||
for (int i = 0; i < sequentialSubReaders.length; i++) {
|
||||
build(newParent, sequentialSubReaders[i], i, newDocBase);
|
||||
children[i] = build(newParent, sequentialSubReaders[i], i, newDocBase);
|
||||
newDocBase += sequentialSubReaders[i].maxDoc();
|
||||
}
|
||||
return newParent;
|
||||
|
|
|
@ -102,7 +102,6 @@ public abstract class CollationTestBase extends LuceneTestCase {
|
|||
(query, new TermRangeFilter("content", secondBeg, secondEnd, true, true), 1).scoreDocs;
|
||||
assertEquals("The index Term should be included.", 1, result.length);
|
||||
|
||||
searcher.close();
|
||||
reader.close();
|
||||
}
|
||||
|
||||
|
@ -131,7 +130,6 @@ public abstract class CollationTestBase extends LuceneTestCase {
|
|||
query = new TermRangeQuery("content", secondBeg, secondEnd, true, true);
|
||||
hits = searcher.search(query, null, 1000).scoreDocs;
|
||||
assertEquals("The index Term should be included.", 1, hits.length);
|
||||
searcher.close();
|
||||
reader.close();
|
||||
}
|
||||
|
||||
|
@ -164,7 +162,6 @@ public abstract class CollationTestBase extends LuceneTestCase {
|
|||
("content", secondBeg, secondEnd, true, true);
|
||||
result = search.search(csrq, null, 1000).scoreDocs;
|
||||
assertEquals("The index Term should be included.", 1, result.length);
|
||||
search.close();
|
||||
}
|
||||
|
||||
// Test using various international locales with accented characters (which
|
||||
|
@ -240,7 +237,6 @@ public abstract class CollationTestBase extends LuceneTestCase {
|
|||
|
||||
sort.setSort(new SortField("Denmark", SortField.Type.STRING));
|
||||
assertMatches(searcher, queryY, sort, dkResult);
|
||||
searcher.close();
|
||||
reader.close();
|
||||
}
|
||||
|
||||
|
|
|
@ -472,7 +472,6 @@ public abstract class ThreadedIndexingAndSearchingTestCase extends LuceneTestCas
|
|||
|
||||
IndexSearcher searcher = newSearcher(reader);
|
||||
sum += searcher.search(new TermQuery(new Term("body", "united")), 10).totalHits;
|
||||
searcher.close();
|
||||
|
||||
if (VERBOSE) {
|
||||
System.out.println("TEST: warm visited " + sum + " fields");
|
||||
|
|
|
@ -116,7 +116,6 @@ public class CheckHits {
|
|||
query.toString(defaultFieldName),
|
||||
correct, actual);
|
||||
FieldCache.DEFAULT.purge(s.getIndexReader()); // our wrapping can create insanity otherwise
|
||||
s.close();
|
||||
}
|
||||
}
|
||||
|
||||
|
|
|
@ -115,13 +115,10 @@ public class QueryUtils {
|
|||
IndexSearcher wrapped;
|
||||
check(random, q1, wrapped = wrapUnderlyingReader(random, s, -1), false);
|
||||
FieldCache.DEFAULT.purge(wrapped.getIndexReader()); // // our wrapping can create insanity otherwise
|
||||
wrapped.close();
|
||||
check(random, q1, wrapped = wrapUnderlyingReader(random, s, 0), false);
|
||||
FieldCache.DEFAULT.purge(wrapped.getIndexReader()); // // our wrapping can create insanity otherwise
|
||||
wrapped.close();
|
||||
check(random, q1, wrapped = wrapUnderlyingReader(random, s, +1), false);
|
||||
FieldCache.DEFAULT.purge(wrapped.getIndexReader()); // // our wrapping can create insanity otherwise
|
||||
wrapped.close();
|
||||
}
|
||||
checkExplanations(q1,s);
|
||||
|
||||
|
@ -309,7 +306,6 @@ public class QueryUtils {
|
|||
Assert.assertFalse("query's last doc was "+ lastDoc[0] +" but skipTo("+(lastDoc[0]+1)+") got to "+scorer.docID(),more);
|
||||
}
|
||||
leafPtr++;
|
||||
indexSearcher.close();
|
||||
}
|
||||
lastReader[0] = context.reader;
|
||||
assert readerContextArray[leafPtr].reader == context.reader;
|
||||
|
@ -335,7 +331,6 @@ public class QueryUtils {
|
|||
boolean more = scorer.advance(lastDoc[0] + 1) != DocIdSetIterator.NO_MORE_DOCS;
|
||||
Assert.assertFalse("query's last doc was "+ lastDoc[0] +" but skipTo("+(lastDoc[0]+1)+") got to "+scorer.docID(),more);
|
||||
}
|
||||
indexSearcher.close();
|
||||
}
|
||||
}
|
||||
}
|
||||
|
@ -394,7 +389,6 @@ public class QueryUtils {
|
|||
boolean more = scorer.advance(lastDoc[0] + 1) != DocIdSetIterator.NO_MORE_DOCS;
|
||||
Assert.assertFalse("query's last doc was "+ lastDoc[0] +" but skipTo("+(lastDoc[0]+1)+") got to "+scorer.docID(),more);
|
||||
}
|
||||
indexSearcher.close();
|
||||
leafPtr++;
|
||||
}
|
||||
|
||||
|
@ -419,7 +413,6 @@ public class QueryUtils {
|
|||
boolean more = scorer.advance(lastDoc[0] + 1) != DocIdSetIterator.NO_MORE_DOCS;
|
||||
Assert.assertFalse("query's last doc was "+ lastDoc[0] +" but skipTo("+(lastDoc[0]+1)+") got to "+scorer.docID(),more);
|
||||
}
|
||||
indexSearcher.close();
|
||||
}
|
||||
}
|
||||
}
|
||||
|
|
|
@ -32,12 +32,12 @@ import java.util.Map.Entry;
|
|||
import java.util.concurrent.ExecutorService;
|
||||
import java.util.concurrent.Executors;
|
||||
import java.util.concurrent.TimeUnit;
|
||||
import java.util.regex.Pattern;
|
||||
|
||||
import org.apache.lucene.analysis.Analyzer;
|
||||
import org.apache.lucene.document.Field;
|
||||
import org.apache.lucene.document.FieldType;
|
||||
import org.apache.lucene.index.*;
|
||||
import org.apache.lucene.index.IndexReader.ReaderFinishedListener;
|
||||
import org.apache.lucene.index.codecs.Codec;
|
||||
import org.apache.lucene.index.codecs.PostingsFormat;
|
||||
import org.apache.lucene.index.codecs.appending.AppendingCodec;
|
||||
|
@ -732,7 +732,8 @@ public abstract class LuceneTestCase extends Assert {
|
|||
rogueThreads.put(t, true);
|
||||
rogueCount++;
|
||||
if (t.getName().startsWith("LuceneTestCase")) {
|
||||
System.err.println("PLEASE CLOSE YOUR INDEXSEARCHERS IN YOUR TEST!!!!");
|
||||
// TODO: should we fail here now? really test should be failing?
|
||||
System.err.println("PLEASE CLOSE YOUR INDEXREADERS IN YOUR TEST!!!!");
|
||||
continue;
|
||||
} else {
|
||||
// wait on the thread to die of natural causes
|
||||
|
@ -1229,23 +1230,25 @@ public abstract class LuceneTestCase extends Assert {
|
|||
final ExecutorService ex = (random.nextBoolean()) ? null
|
||||
: Executors.newFixedThreadPool(threads = _TestUtil.nextInt(random, 1, 8),
|
||||
new NamedThreadFactory("LuceneTestCase"));
|
||||
if (ex != null && VERBOSE) {
|
||||
if (ex != null) {
|
||||
if (VERBOSE) {
|
||||
System.out.println("NOTE: newSearcher using ExecutorService with " + threads + " threads");
|
||||
}
|
||||
final IndexReader r0 = r;
|
||||
r.addReaderFinishedListener(new ReaderFinishedListener() {
|
||||
@Override
|
||||
public void finished(IndexReader reader) {
|
||||
// readerFinishedListener bogusly calls us with other random readers
|
||||
// so we must check that its *actually* the one we registered it on.
|
||||
if (reader == r0) {
|
||||
shutdownExecutorService(ex);
|
||||
}
|
||||
}
|
||||
});
|
||||
}
|
||||
IndexSearcher ret = random.nextBoolean() ?
|
||||
new AssertingIndexSearcher(random, r, ex) {
|
||||
@Override
|
||||
public void close() throws IOException {
|
||||
super.close();
|
||||
shutdownExecutorService(ex);
|
||||
}
|
||||
} : new AssertingIndexSearcher(random, r.getTopReaderContext(), ex) {
|
||||
@Override
|
||||
public void close() throws IOException {
|
||||
super.close();
|
||||
shutdownExecutorService(ex);
|
||||
}
|
||||
};
|
||||
IndexSearcher ret = random.nextBoolean()
|
||||
? new AssertingIndexSearcher(random, r, ex)
|
||||
: new AssertingIndexSearcher(random, r.getTopReaderContext(), ex);
|
||||
ret.setSimilarityProvider(similarityProvider);
|
||||
return ret;
|
||||
}
|
||||
|
|
|
@ -25,6 +25,7 @@ import java.io.IOException;
|
|||
* @lucene.experimental
|
||||
*/
|
||||
public class NullInfoStream extends InfoStream {
|
||||
|
||||
@Override
|
||||
public void message(String component, String message) {
|
||||
assert component != null;
|
||||
|
|
|
@ -73,7 +73,6 @@ public class TestDemo extends LuceneTestCase {
|
|||
phraseQuery.add(new Term("fieldname", "be"));
|
||||
assertEquals(1, isearcher.search(phraseQuery, null, 1).totalHits);
|
||||
|
||||
isearcher.close();
|
||||
ireader.close();
|
||||
directory.close();
|
||||
}
|
||||
|
|
|
@ -96,7 +96,6 @@ public class TestExternalCodecs extends LuceneTestCase {
|
|||
assertEquals(NUM_DOCS-1, s.search(new TermQuery(new Term("field1", "standard")), 1).totalHits);
|
||||
assertEquals(NUM_DOCS-1, s.search(new TermQuery(new Term("field2", "pulsing")), 1).totalHits);
|
||||
r.close();
|
||||
s.close();
|
||||
|
||||
if (VERBOSE) {
|
||||
System.out.println("\nTEST: now delete 2nd doc");
|
||||
|
@ -114,7 +113,6 @@ public class TestExternalCodecs extends LuceneTestCase {
|
|||
assertEquals(0, s.search(new TermQuery(new Term("id", "44")), 1).totalHits);
|
||||
|
||||
r.close();
|
||||
s.close();
|
||||
|
||||
w.close();
|
||||
|
||||
|
|
|
@ -122,7 +122,6 @@ public class TestSearch extends LuceneTestCase {
|
|||
out.println(i + " " + hits[i].score + " " + d.get("contents"));
|
||||
}
|
||||
}
|
||||
searcher.close();
|
||||
reader.close();
|
||||
directory.close();
|
||||
}
|
||||
|
|
|
@ -115,8 +115,6 @@ public class TestSearchForDuplicates extends LuceneTestCase {
|
|||
printHits(out, hits, searcher);
|
||||
checkHits(hits, MAX_DOCS, searcher);
|
||||
|
||||
searcher.close();
|
||||
|
||||
// try a new search with OR
|
||||
searcher = new IndexSearcher(reader);
|
||||
hits = null;
|
||||
|
@ -130,7 +128,6 @@ public class TestSearchForDuplicates extends LuceneTestCase {
|
|||
printHits(out, hits, searcher);
|
||||
checkHits(hits, MAX_DOCS, searcher);
|
||||
|
||||
searcher.close();
|
||||
reader.close();
|
||||
directory.close();
|
||||
}
|
||||
|
|
|
@ -174,7 +174,6 @@ public class TestDocument extends LuceneTestCase {
|
|||
|
||||
doAssert(searcher.doc(hits[0].doc), true);
|
||||
writer.close();
|
||||
searcher.close();
|
||||
reader.close();
|
||||
dir.close();
|
||||
}
|
||||
|
@ -270,7 +269,6 @@ public class TestDocument extends LuceneTestCase {
|
|||
else fail("unexpected id field");
|
||||
}
|
||||
writer.close();
|
||||
searcher.close();
|
||||
reader.close();
|
||||
dir.close();
|
||||
assertEquals("did not see all IDs", 7, result);
|
||||
|
|
|
@ -336,7 +336,6 @@ public class TestBackwardsCompatibility extends LuceneTestCase {
|
|||
hits = searcher.search(new TermQuery(new Term("utf8", "ab\ud917\udc17cd")), null, 1000).scoreDocs;
|
||||
assertEquals(34, hits.length);
|
||||
|
||||
searcher.close();
|
||||
reader.close();
|
||||
dir.close();
|
||||
}
|
||||
|
@ -374,7 +373,6 @@ public class TestBackwardsCompatibility extends LuceneTestCase {
|
|||
Document d = searcher.getIndexReader().document(hits[0].doc);
|
||||
assertEquals("wrong first document", "21", d.get("id"));
|
||||
doTestHits(hits, 44, searcher.getIndexReader());
|
||||
searcher.close();
|
||||
reader.close();
|
||||
|
||||
// fully merge
|
||||
|
@ -389,7 +387,6 @@ public class TestBackwardsCompatibility extends LuceneTestCase {
|
|||
d = searcher.doc(hits[0].doc);
|
||||
doTestHits(hits, 44, searcher.getIndexReader());
|
||||
assertEquals("wrong first document", "21", d.get("id"));
|
||||
searcher.close();
|
||||
reader.close();
|
||||
|
||||
dir.close();
|
||||
|
@ -406,7 +403,6 @@ public class TestBackwardsCompatibility extends LuceneTestCase {
|
|||
assertEquals("wrong number of hits", 34, hits.length);
|
||||
Document d = searcher.doc(hits[0].doc);
|
||||
assertEquals("wrong first document", "21", d.get("id"));
|
||||
searcher.close();
|
||||
reader.close();
|
||||
|
||||
// fully merge
|
||||
|
@ -419,7 +415,6 @@ public class TestBackwardsCompatibility extends LuceneTestCase {
|
|||
hits = searcher.search(new TermQuery(new Term("content", "aaa")), null, 1000).scoreDocs;
|
||||
assertEquals("wrong number of hits", 34, hits.length);
|
||||
doTestHits(hits, 34, searcher.getIndexReader());
|
||||
searcher.close();
|
||||
reader.close();
|
||||
|
||||
dir.close();
|
||||
|
@ -664,7 +659,6 @@ public class TestBackwardsCompatibility extends LuceneTestCase {
|
|||
assertTrue("value in id bounds", val >= 0L && val < 35L);
|
||||
}
|
||||
|
||||
searcher.close();
|
||||
reader.close();
|
||||
dir.close();
|
||||
_TestUtil.rmDir(oldIndexDir);
|
||||
|
|
|
@ -70,7 +70,6 @@ public class TestBinaryTerms extends LuceneTestCase {
|
|||
assertEquals("" + i, is.doc(docs.scoreDocs[0].doc).get("id"));
|
||||
}
|
||||
|
||||
is.close();
|
||||
ir.close();
|
||||
dir.close();
|
||||
}
|
||||
|
|
|
@ -393,7 +393,6 @@ public class TestCodecs extends LuceneTestCase {
|
|||
return searcher.search(q, null, n).scoreDocs;
|
||||
}
|
||||
finally {
|
||||
searcher.close();
|
||||
reader.close();
|
||||
}
|
||||
}
|
||||
|
|
|
@ -126,7 +126,6 @@ public class TestCrashCausesCorruptIndex extends LuceneTestCase {
|
|||
TopDocs topDocs = indexSearcher.search(new TermQuery(new Term(TEXT_FIELD, "fleas")), 10);
|
||||
assertNotNull(topDocs);
|
||||
assertEquals(expectedTotalHits, topDocs.totalHits);
|
||||
indexSearcher.close();
|
||||
indexReader.close();
|
||||
realDirectory.close();
|
||||
}
|
||||
|
|
|
@ -660,7 +660,6 @@ public class TestDeletionPolicy extends LuceneTestCase {
|
|||
ScoreDoc[] hits = searcher.search(query, null, 1000).scoreDocs;
|
||||
assertEquals(16, hits.length);
|
||||
reader.close();
|
||||
searcher.close();
|
||||
|
||||
writer = new IndexWriter(dir, newIndexWriterConfig(
|
||||
TEST_VERSION_CURRENT, new MockAnalyzer(random))
|
||||
|
@ -685,7 +684,6 @@ public class TestDeletionPolicy extends LuceneTestCase {
|
|||
dir.deleteFile(IndexFileNames.SEGMENTS_GEN);
|
||||
int expectedCount = 0;
|
||||
|
||||
searcher.close();
|
||||
rwReader.close();
|
||||
|
||||
for(int i=0;i<N+1;i++) {
|
||||
|
@ -697,7 +695,6 @@ public class TestDeletionPolicy extends LuceneTestCase {
|
|||
searcher = newSearcher(reader);
|
||||
hits = searcher.search(query, null, 1000).scoreDocs;
|
||||
assertEquals(expectedCount, hits.length);
|
||||
searcher.close();
|
||||
if (expectedCount == 0) {
|
||||
expectedCount = 16;
|
||||
} else if (expectedCount == 16) {
|
||||
|
|
|
@ -75,7 +75,6 @@ public class TestForTooMuchCloning extends LuceneTestCase {
|
|||
final int queryCloneCount = dir.getInputCloneCount() - cloneCount;
|
||||
//System.out.println("query clone count=" + queryCloneCount);
|
||||
assertTrue("too many calls to IndexInput.clone during TermRangeQuery: " + queryCloneCount, queryCloneCount < 50);
|
||||
s.close();
|
||||
r.close();
|
||||
dir.close();
|
||||
}
|
||||
|
|
|
@ -515,7 +515,6 @@ public class TestIndexReaderReopen extends LuceneTestCase {
|
|||
if (hits.length > 0) {
|
||||
searcher.doc(hits[0].doc);
|
||||
}
|
||||
searcher.close();
|
||||
if (refreshed != r) {
|
||||
refreshed.close();
|
||||
}
|
||||
|
|
|
@ -456,7 +456,6 @@ public class TestIndexWriter extends LuceneTestCase {
|
|||
IndexSearcher searcher = new IndexSearcher(reader);
|
||||
ScoreDoc[] hits = searcher.search(new TermQuery(new Term("field", "aaa")), null, 1000).scoreDocs;
|
||||
assertEquals(300, hits.length);
|
||||
searcher.close();
|
||||
reader.close();
|
||||
|
||||
dir.close();
|
||||
|
@ -488,7 +487,6 @@ public class TestIndexWriter extends LuceneTestCase {
|
|||
IndexSearcher searcher = new IndexSearcher(reader);
|
||||
ScoreDoc[] hits = searcher.search(new TermQuery(searchTerm), null, 1000).scoreDocs;
|
||||
assertEquals(10, hits.length);
|
||||
searcher.close();
|
||||
reader.close();
|
||||
|
||||
writer = new IndexWriter(dir, newIndexWriterConfig( TEST_VERSION_CURRENT, new MockAnalyzer(random))
|
||||
|
@ -511,7 +509,6 @@ public class TestIndexWriter extends LuceneTestCase {
|
|||
searcher = new IndexSearcher(reader);
|
||||
hits = searcher.search(new TermQuery(searchTerm), null, 1000).scoreDocs;
|
||||
assertEquals(27, hits.length);
|
||||
searcher.close();
|
||||
reader.close();
|
||||
|
||||
reader = IndexReader.open(dir);
|
||||
|
@ -590,7 +587,6 @@ public class TestIndexWriter extends LuceneTestCase {
|
|||
IndexSearcher searcher = new IndexSearcher(reader);
|
||||
ScoreDoc[] hits = searcher.search(new TermQuery(searchTerm), null, 1000).scoreDocs;
|
||||
assertEquals("did not get right number of hits", 100, hits.length);
|
||||
searcher.close();
|
||||
reader.close();
|
||||
|
||||
writer = new IndexWriter(dir, newIndexWriterConfig( TEST_VERSION_CURRENT, new MockAnalyzer(random))
|
||||
|
@ -1013,7 +1009,6 @@ public class TestIndexWriter extends LuceneTestCase {
|
|||
assertEquals(0, tps.nextPosition());
|
||||
w.close();
|
||||
|
||||
s.close();
|
||||
r.close();
|
||||
dir.close();
|
||||
}
|
||||
|
@ -1708,7 +1703,6 @@ public class TestIndexWriter extends LuceneTestCase {
|
|||
assertEquals("doc " + testID + ", field f" + fieldCount + " is wrong", docExp.get("f"+i), doc.get("f"+i));
|
||||
}
|
||||
}
|
||||
s.close();
|
||||
r.close();
|
||||
w.forceMerge(1);
|
||||
}
|
||||
|
|
|
@ -55,7 +55,6 @@ public class TestIndexWriterCommit extends LuceneTestCase {
|
|||
IndexSearcher searcher = new IndexSearcher(reader);
|
||||
ScoreDoc[] hits = searcher.search(new TermQuery(searchTerm), null, 1000).scoreDocs;
|
||||
assertEquals("first number of hits", 14, hits.length);
|
||||
searcher.close();
|
||||
reader.close();
|
||||
|
||||
reader = IndexReader.open(dir);
|
||||
|
@ -69,7 +68,6 @@ public class TestIndexWriterCommit extends LuceneTestCase {
|
|||
searcher = new IndexSearcher(r);
|
||||
hits = searcher.search(new TermQuery(searchTerm), null, 1000).scoreDocs;
|
||||
assertEquals("reader incorrectly sees changes from writer", 14, hits.length);
|
||||
searcher.close();
|
||||
r.close();
|
||||
assertTrue("reader should have still been current", reader.isCurrent());
|
||||
}
|
||||
|
@ -82,7 +80,6 @@ public class TestIndexWriterCommit extends LuceneTestCase {
|
|||
searcher = new IndexSearcher(r);
|
||||
hits = searcher.search(new TermQuery(searchTerm), null, 1000).scoreDocs;
|
||||
assertEquals("reader did not see changes after writer was closed", 47, hits.length);
|
||||
searcher.close();
|
||||
r.close();
|
||||
reader.close();
|
||||
dir.close();
|
||||
|
@ -109,7 +106,6 @@ public class TestIndexWriterCommit extends LuceneTestCase {
|
|||
IndexSearcher searcher = new IndexSearcher(reader);
|
||||
ScoreDoc[] hits = searcher.search(new TermQuery(searchTerm), null, 1000).scoreDocs;
|
||||
assertEquals("first number of hits", 14, hits.length);
|
||||
searcher.close();
|
||||
reader.close();
|
||||
|
||||
writer = new IndexWriter(dir, newIndexWriterConfig( TEST_VERSION_CURRENT, new MockAnalyzer(random))
|
||||
|
@ -124,7 +120,6 @@ public class TestIndexWriterCommit extends LuceneTestCase {
|
|||
searcher = new IndexSearcher(reader);
|
||||
hits = searcher.search(new TermQuery(searchTerm), null, 1000).scoreDocs;
|
||||
assertEquals("reader incorrectly sees changes from writer", 14, hits.length);
|
||||
searcher.close();
|
||||
reader.close();
|
||||
|
||||
// Now, close the writer:
|
||||
|
@ -136,7 +131,6 @@ public class TestIndexWriterCommit extends LuceneTestCase {
|
|||
searcher = new IndexSearcher(reader);
|
||||
hits = searcher.search(new TermQuery(searchTerm), null, 1000).scoreDocs;
|
||||
assertEquals("saw changes after writer.abort", 14, hits.length);
|
||||
searcher.close();
|
||||
reader.close();
|
||||
|
||||
// Now make sure we can re-open the index, add docs,
|
||||
|
@ -156,7 +150,6 @@ public class TestIndexWriterCommit extends LuceneTestCase {
|
|||
searcher = new IndexSearcher(r);
|
||||
hits = searcher.search(new TermQuery(searchTerm), null, 1000).scoreDocs;
|
||||
assertEquals("reader incorrectly sees changes from writer", 14, hits.length);
|
||||
searcher.close();
|
||||
r.close();
|
||||
}
|
||||
|
||||
|
@ -165,7 +158,6 @@ public class TestIndexWriterCommit extends LuceneTestCase {
|
|||
searcher = new IndexSearcher(r);
|
||||
hits = searcher.search(new TermQuery(searchTerm), null, 1000).scoreDocs;
|
||||
assertEquals("didn't see changes after close", 218, hits.length);
|
||||
searcher.close();
|
||||
r.close();
|
||||
|
||||
dir.close();
|
||||
|
|
|
@ -403,7 +403,6 @@ public class TestIndexWriterDelete extends LuceneTestCase {
|
|||
IndexReader reader = IndexReader.open(dir);
|
||||
IndexSearcher searcher = new IndexSearcher(reader);
|
||||
int hitCount = searcher.search(new TermQuery(term), null, 1000).totalHits;
|
||||
searcher.close();
|
||||
reader.close();
|
||||
return hitCount;
|
||||
}
|
||||
|
@ -612,7 +611,6 @@ public class TestIndexWriterDelete extends LuceneTestCase {
|
|||
+ result2 + " instead of expected " + START_COUNT + " or " + END_COUNT);
|
||||
}
|
||||
}
|
||||
searcher.close();
|
||||
newReader.close();
|
||||
if (result2 == END_COUNT) {
|
||||
break;
|
||||
|
|
|
@ -195,7 +195,6 @@ public class TestIndexWriterOnDiskFull extends LuceneTestCase {
|
|||
IndexSearcher searcher = newSearcher(reader);
|
||||
ScoreDoc[] hits = searcher.search(new TermQuery(searchTerm), null, 1000).scoreDocs;
|
||||
assertEquals("first number of hits", 57, hits.length);
|
||||
searcher.close();
|
||||
reader.close();
|
||||
|
||||
// Iterate with larger and larger amounts of free
|
||||
|
@ -395,7 +394,6 @@ public class TestIndexWriterOnDiskFull extends LuceneTestCase {
|
|||
}
|
||||
}
|
||||
|
||||
searcher.close();
|
||||
reader.close();
|
||||
if (VERBOSE) {
|
||||
System.out.println(" count is " + result);
|
||||
|
|
|
@ -695,7 +695,6 @@ public class TestIndexWriterReader extends LuceneTestCase {
|
|||
Query q = new TermQuery(new Term("indexname", "test"));
|
||||
IndexSearcher searcher = newSearcher(r);
|
||||
assertEquals(100, searcher.search(q, 10).totalHits);
|
||||
searcher.close();
|
||||
try {
|
||||
IndexReader.openIfChanged(r);
|
||||
fail("failed to hit AlreadyClosedException");
|
||||
|
@ -761,7 +760,6 @@ public class TestIndexWriterReader extends LuceneTestCase {
|
|||
Query q = new TermQuery(new Term("indexname", "test"));
|
||||
IndexSearcher searcher = newSearcher(r);
|
||||
final int count = searcher.search(q, 10).totalHits;
|
||||
searcher.close();
|
||||
assertTrue(count >= lastCount);
|
||||
lastCount = count;
|
||||
}
|
||||
|
@ -778,7 +776,6 @@ public class TestIndexWriterReader extends LuceneTestCase {
|
|||
Query q = new TermQuery(new Term("indexname", "test"));
|
||||
IndexSearcher searcher = newSearcher(r);
|
||||
final int count = searcher.search(q, 10).totalHits;
|
||||
searcher.close();
|
||||
assertTrue(count >= lastCount);
|
||||
|
||||
assertEquals(0, excs.size());
|
||||
|
@ -850,7 +847,6 @@ public class TestIndexWriterReader extends LuceneTestCase {
|
|||
Query q = new TermQuery(new Term("indexname", "test"));
|
||||
IndexSearcher searcher = newSearcher(r);
|
||||
sum += searcher.search(q, 10).totalHits;
|
||||
searcher.close();
|
||||
}
|
||||
|
||||
for(int i=0;i<numThreads;i++) {
|
||||
|
@ -865,7 +861,6 @@ public class TestIndexWriterReader extends LuceneTestCase {
|
|||
Query q = new TermQuery(new Term("indexname", "test"));
|
||||
IndexSearcher searcher = newSearcher(r);
|
||||
sum += searcher.search(q, 10).totalHits;
|
||||
searcher.close();
|
||||
assertTrue("no documents found at all", sum > 0);
|
||||
|
||||
assertEquals(0, excs.size());
|
||||
|
@ -954,7 +949,6 @@ public class TestIndexWriterReader extends LuceneTestCase {
|
|||
TopDocs hits = s.search(new TermQuery(new Term("foo", "bar")), 10);
|
||||
assertEquals(20, hits.totalHits);
|
||||
didWarm.set(true);
|
||||
s.close();
|
||||
}
|
||||
}).
|
||||
setMergePolicy(newLogMergePolicy(10))
|
||||
|
|
|
@ -138,9 +138,7 @@ public class TestLazyProxSkipping extends LuceneTestCase {
|
|||
// test whether only the minimum amount of seeks()
|
||||
// are performed
|
||||
performTest(5);
|
||||
searcher.close();
|
||||
performTest(10);
|
||||
searcher.close();
|
||||
}
|
||||
|
||||
public void testSeek() throws IOException {
|
||||
|
|
|
@ -100,7 +100,6 @@ public class TestNRTThreads extends ThreadedIndexingAndSearchingTestCase {
|
|||
if (s != fixedSearcher) {
|
||||
// Final searcher:
|
||||
s.getIndexReader().close();
|
||||
s.close();
|
||||
}
|
||||
}
|
||||
|
||||
|
|
|
@ -401,7 +401,6 @@ public class TestOmitTf extends LuceneTestCase {
|
|||
});
|
||||
assertEquals(15, CountingHitCollector.getCount());
|
||||
|
||||
searcher.close();
|
||||
reader.close();
|
||||
dir.close();
|
||||
}
|
||||
|
|
|
@ -46,9 +46,7 @@ public class TestParallelReader extends LuceneTestCase {
|
|||
@Override
|
||||
public void tearDown() throws Exception {
|
||||
single.getIndexReader().close();
|
||||
single.close();
|
||||
parallel.getIndexReader().close();
|
||||
parallel.close();
|
||||
dir.close();
|
||||
dir1.close();
|
||||
dir2.close();
|
||||
|
|
|
@ -61,7 +61,6 @@ public class TestReaderClosed extends LuceneTestCase {
|
|||
public void test() throws Exception {
|
||||
TermRangeQuery query = TermRangeQuery.newStringRange("field", "a", "z", true, true);
|
||||
searcher.search(query, 5);
|
||||
searcher.close();
|
||||
reader.close();
|
||||
try {
|
||||
searcher.search(query, 5);
|
||||
|
|
|
@ -105,7 +105,6 @@ public class TestStressIndexing extends LuceneTestCase {
|
|||
for (int i=0; i<100; i++) {
|
||||
IndexReader ir = IndexReader.open(directory);
|
||||
IndexSearcher is = new IndexSearcher(ir);
|
||||
is.close();
|
||||
ir.close();
|
||||
}
|
||||
count += 100;
|
||||
|
|
|
@ -84,7 +84,6 @@ public class TestTermsEnum2 extends LuceneTestCase {
|
|||
}
|
||||
|
||||
public void tearDown() throws Exception {
|
||||
searcher.close();
|
||||
reader.close();
|
||||
dir.close();
|
||||
super.tearDown();
|
||||
|
|
|
@ -136,7 +136,6 @@ public class TestTermInfosReaderIndex extends LuceneTestCase {
|
|||
TopDocs topDocs = searcher.search(query, 10);
|
||||
assertTrue(topDocs.totalHits > 0);
|
||||
}
|
||||
searcher.close();
|
||||
}
|
||||
|
||||
private List<Term> sample(IndexReader reader, int size) throws IOException {
|
||||
|
|
|
@ -194,7 +194,6 @@ public class TestPerFieldPostingsFormat extends LuceneTestCase {
|
|||
IndexSearcher searcher = newSearcher(reader);
|
||||
TopDocs search = searcher.search(new TermQuery(t), num + 10);
|
||||
assertEquals(num, search.totalHits);
|
||||
searcher.close();
|
||||
reader.close();
|
||||
|
||||
}
|
||||
|
|
|
@ -67,7 +67,6 @@ public class TestAutomatonQuery extends LuceneTestCase {
|
|||
|
||||
@Override
|
||||
public void tearDown() throws Exception {
|
||||
searcher.close();
|
||||
reader.close();
|
||||
directory.close();
|
||||
super.tearDown();
|
||||
|
|
|
@ -86,7 +86,6 @@ public class TestAutomatonQueryUnicode extends LuceneTestCase {
|
|||
|
||||
@Override
|
||||
public void tearDown() throws Exception {
|
||||
searcher.close();
|
||||
reader.close();
|
||||
directory.close();
|
||||
super.tearDown();
|
||||
|
|
|
@ -101,12 +101,10 @@ public class TestBoolean2 extends LuceneTestCase {
|
|||
|
||||
@AfterClass
|
||||
public static void afterClass() throws Exception {
|
||||
searcher.close();
|
||||
reader.close();
|
||||
littleReader.close();
|
||||
dir2.close();
|
||||
directory.close();
|
||||
bigSearcher.close();
|
||||
searcher = null;
|
||||
reader = null;
|
||||
littleReader = null;
|
||||
|
|
|
@ -73,7 +73,6 @@ public class TestBooleanMinShouldMatch extends LuceneTestCase {
|
|||
|
||||
@AfterClass
|
||||
public static void afterClass() throws Exception {
|
||||
s.close();
|
||||
s = null;
|
||||
r.close();
|
||||
r = null;
|
||||
|
|
|
@ -158,7 +158,6 @@ public class TestBooleanOr extends LuceneTestCase {
|
|||
|
||||
@Override
|
||||
public void tearDown() throws Exception {
|
||||
searcher.close();
|
||||
reader.close();
|
||||
dir.close();
|
||||
super.tearDown();
|
||||
|
|
|
@ -140,7 +140,6 @@ public class TestBooleanQuery extends LuceneTestCase {
|
|||
dmq.add(pq);
|
||||
assertEquals(1, s.search(dmq, 10).totalHits);
|
||||
|
||||
s.close();
|
||||
r.close();
|
||||
w.close();
|
||||
dir.close();
|
||||
|
|
|
@ -62,7 +62,6 @@ public class TestBooleanScorer extends LuceneTestCase
|
|||
IndexSearcher indexSearcher = newSearcher(ir);
|
||||
ScoreDoc[] hits = indexSearcher.search(query, null, 1000).scoreDocs;
|
||||
assertEquals("Number of matched documents", 2, hits.length);
|
||||
indexSearcher.close();
|
||||
ir.close();
|
||||
directory.close();
|
||||
}
|
||||
|
@ -122,7 +121,6 @@ public class TestBooleanScorer extends LuceneTestCase
|
|||
|
||||
assertEquals("should have only 1 hit", 1, hits.size());
|
||||
assertEquals("hit should have been docID=3000", 3000, hits.get(0).intValue());
|
||||
searcher.close();
|
||||
ir.close();
|
||||
directory.close();
|
||||
}
|
||||
|
@ -176,7 +174,6 @@ public class TestBooleanScorer extends LuceneTestCase
|
|||
|
||||
assertEquals(1, count[0]);
|
||||
|
||||
s.close();
|
||||
r.close();
|
||||
d.close();
|
||||
}
|
||||
|
|
|
@ -179,7 +179,6 @@ public class TestCachingWrapperFilter extends LuceneTestCase {
|
|||
writer.addDocument(doc);
|
||||
|
||||
reader = refreshReader(reader);
|
||||
searcher.close();
|
||||
searcher = newSearcher(reader, false);
|
||||
|
||||
TopDocs docs = searcher.search(new MatchAllDocsQuery(), 1);
|
||||
|
@ -205,7 +204,6 @@ public class TestCachingWrapperFilter extends LuceneTestCase {
|
|||
|
||||
writer.addDocument(doc);
|
||||
reader = refreshReader(reader);
|
||||
searcher.close();
|
||||
searcher = newSearcher(reader, false);
|
||||
|
||||
docs = searcher.search(new MatchAllDocsQuery(), filter, 1);
|
||||
|
@ -227,7 +225,6 @@ public class TestCachingWrapperFilter extends LuceneTestCase {
|
|||
writer.deleteDocuments(new Term("id", "1"));
|
||||
|
||||
reader = refreshReader(reader);
|
||||
searcher.close();
|
||||
searcher = newSearcher(reader, false);
|
||||
|
||||
docs = searcher.search(new MatchAllDocsQuery(), filter, 1);
|
||||
|
@ -245,7 +242,6 @@ public class TestCachingWrapperFilter extends LuceneTestCase {
|
|||
assertTrue(oldReader != null);
|
||||
assertTrue(oldReader2 != null);
|
||||
|
||||
searcher.close();
|
||||
reader.close();
|
||||
writer.close();
|
||||
dir.close();
|
||||
|
|
|
@ -125,7 +125,6 @@ public class TestConstantScoreQuery extends LuceneTestCase {
|
|||
checkHits(searcher, bq, csq1.getBoost() + csq2.getBoost(), bucketScorerClass, null);
|
||||
checkHits(searcher, csqbq, csqbq.getBoost(), ConstantScoreQuery.ConstantScorer.class.getName(), bucketScorerClass);
|
||||
} finally {
|
||||
if (searcher != null) searcher.close();
|
||||
if (reader != null) reader.close();
|
||||
if (directory != null) directory.close();
|
||||
}
|
||||
|
|
|
@ -97,7 +97,6 @@ public class TestDateFilter extends LuceneTestCase {
|
|||
|
||||
result = searcher.search(query2, df2, 1000).scoreDocs;
|
||||
assertEquals(0, result.length);
|
||||
searcher.close();
|
||||
reader.close();
|
||||
indexStore.close();
|
||||
}
|
||||
|
@ -164,7 +163,6 @@ public class TestDateFilter extends LuceneTestCase {
|
|||
|
||||
result = searcher.search(query2, df2, 1000).scoreDocs;
|
||||
assertEquals(0, result.length);
|
||||
searcher.close();
|
||||
reader.close();
|
||||
indexStore.close();
|
||||
}
|
||||
|
|
|
@ -88,7 +88,6 @@ public class TestDateSort extends LuceneTestCase {
|
|||
String text = document.get(TEXT_FIELD);
|
||||
actualOrder[i] = text;
|
||||
}
|
||||
searcher.close();
|
||||
|
||||
// Set up the expected order (i.e. Document 5, 4, 3, 2, 1).
|
||||
String[] expectedOrder = new String[5];
|
||||
|
|
|
@ -163,7 +163,6 @@ public class TestDisjunctionMaxQuery extends LuceneTestCase {
|
|||
|
||||
@Override
|
||||
public void tearDown() throws Exception {
|
||||
s.close();
|
||||
r.close();
|
||||
index.close();
|
||||
super.tearDown();
|
||||
|
|
|
@ -121,7 +121,6 @@ public class TestDocIdSet extends LuceneTestCase {
|
|||
};
|
||||
|
||||
Assert.assertEquals(0, searcher.search(new MatchAllDocsQuery(), f, 10).totalHits);
|
||||
searcher.close();
|
||||
reader.close();
|
||||
dir.close();
|
||||
}
|
||||
|
|
|
@ -133,9 +133,6 @@ public class TestDocValuesScoring extends LuceneTestCase {
|
|||
|
||||
assertEquals(boost.scoreDocs[0].score, noboost.scoreDocs[0].score, SCORE_EPSILON);
|
||||
|
||||
|
||||
searcher1.close();
|
||||
searcher2.close();
|
||||
ir.close();
|
||||
dir.close();
|
||||
}
|
||||
|
|
|
@ -61,7 +61,6 @@ public class TestElevationComparator extends LuceneTestCase {
|
|||
runTest(searcher, true);
|
||||
runTest(searcher, false);
|
||||
|
||||
searcher.close();
|
||||
r.close();
|
||||
directory.close();
|
||||
}
|
||||
|
|
|
@ -61,7 +61,6 @@ public class TestExplanations extends LuceneTestCase {
|
|||
|
||||
@AfterClass
|
||||
public static void afterClassTestExplanations() throws Exception {
|
||||
searcher.close();
|
||||
searcher = null;
|
||||
reader.close();
|
||||
reader = null;
|
||||
|
|
|
@ -122,7 +122,6 @@ public class TestFieldCacheRangeFilter extends BaseTestRangeFilter {
|
|||
|
||||
result = search.search(q,FieldCacheRangeFilter.newStringRange("id",medIP,medIP,T,T), numDocs).scoreDocs;
|
||||
assertEquals("med,med,T,T", 1, result.length);
|
||||
search.close();
|
||||
}
|
||||
|
||||
@Test
|
||||
|
@ -185,7 +184,6 @@ public class TestFieldCacheRangeFilter extends BaseTestRangeFilter {
|
|||
assertEquals("max,max,T,T", 1, result.length);
|
||||
result = search.search(q,FieldCacheRangeFilter.newStringRange("rand",maxRP,null,T,F), numDocs).scoreDocs;
|
||||
assertEquals("max,nul,T,T", 1, result.length);
|
||||
search.close();
|
||||
}
|
||||
|
||||
// byte-ranges cannot be tested, because all ranges are too big for bytes, need an extra range for that
|
||||
|
@ -278,7 +276,6 @@ public class TestFieldCacheRangeFilter extends BaseTestRangeFilter {
|
|||
assertEquals("overflow special case", 0, result.length);
|
||||
result = search.search(q,FieldCacheRangeFilter.newShortRange("id",maxIdO,minIdO,T,T), numDocs).scoreDocs;
|
||||
assertEquals("inverse range", 0, result.length);
|
||||
search.close();
|
||||
}
|
||||
|
||||
@Test
|
||||
|
@ -370,7 +367,6 @@ public class TestFieldCacheRangeFilter extends BaseTestRangeFilter {
|
|||
assertEquals("overflow special case", 0, result.length);
|
||||
result = search.search(q,FieldCacheRangeFilter.newIntRange("id",maxIdO,minIdO,T,T), numDocs).scoreDocs;
|
||||
assertEquals("inverse range", 0, result.length);
|
||||
search.close();
|
||||
}
|
||||
|
||||
@Test
|
||||
|
@ -462,7 +458,6 @@ public class TestFieldCacheRangeFilter extends BaseTestRangeFilter {
|
|||
assertEquals("overflow special case", 0, result.length);
|
||||
result = search.search(q,FieldCacheRangeFilter.newLongRange("id",maxIdO,minIdO,T,T), numDocs).scoreDocs;
|
||||
assertEquals("inverse range", 0, result.length);
|
||||
search.close();
|
||||
}
|
||||
|
||||
// float and double tests are a bit minimalistic, but its complicated, because missing precision
|
||||
|
@ -494,7 +489,6 @@ public class TestFieldCacheRangeFilter extends BaseTestRangeFilter {
|
|||
assertEquals("infinity special case", 0, result.length);
|
||||
result = search.search(q,FieldCacheRangeFilter.newFloatRange("id",null,Float.valueOf(Float.NEGATIVE_INFINITY),F,F), numDocs).scoreDocs;
|
||||
assertEquals("infinity special case", 0, result.length);
|
||||
search.close();
|
||||
}
|
||||
|
||||
@Test
|
||||
|
@ -524,7 +518,6 @@ public class TestFieldCacheRangeFilter extends BaseTestRangeFilter {
|
|||
assertEquals("infinity special case", 0, result.length);
|
||||
result = search.search(q,FieldCacheRangeFilter.newDoubleRange("id",null, Double.valueOf(Double.NEGATIVE_INFINITY),F,F), numDocs).scoreDocs;
|
||||
assertEquals("infinity special case", 0, result.length);
|
||||
search.close();
|
||||
}
|
||||
|
||||
// test using a sparse index (with deleted docs).
|
||||
|
@ -565,7 +558,6 @@ public class TestFieldCacheRangeFilter extends BaseTestRangeFilter {
|
|||
|
||||
result = search.search(q,FieldCacheRangeFilter.newByteRange("id",Byte.valueOf((byte) -20),Byte.valueOf((byte) -10),T,T), 100).scoreDocs;
|
||||
assertEquals("find all", 11, result.length);
|
||||
search.close();
|
||||
reader.close();
|
||||
dir.close();
|
||||
}
|
||||
|
|
|
@ -68,7 +68,6 @@ public class TestFieldCacheTermsFilter extends LuceneTestCase {
|
|||
results = searcher.search(q, new FieldCacheTermsFilter(fieldName, terms.toArray(new String[0])), numDocs).scoreDocs;
|
||||
assertEquals("Must match 2", 2, results.length);
|
||||
|
||||
searcher.close();
|
||||
reader.close();
|
||||
rd.close();
|
||||
}
|
||||
|
|
|
@ -58,7 +58,6 @@ public class TestFieldValueFilter extends LuceneTestCase {
|
|||
}
|
||||
|
||||
reader.close();
|
||||
searcher.close();
|
||||
directory.close();
|
||||
}
|
||||
|
||||
|
@ -86,7 +85,6 @@ public class TestFieldValueFilter extends LuceneTestCase {
|
|||
}
|
||||
|
||||
reader.close();
|
||||
searcher.close();
|
||||
directory.close();
|
||||
}
|
||||
|
||||
|
|
|
@ -104,7 +104,6 @@ public class TestFilteredQuery extends LuceneTestCase {
|
|||
|
||||
@Override
|
||||
public void tearDown() throws Exception {
|
||||
searcher.close();
|
||||
reader.close();
|
||||
directory.close();
|
||||
super.tearDown();
|
||||
|
|
|
@ -82,7 +82,6 @@ public class TestFilteredSearch extends LuceneTestCase {
|
|||
IndexSearcher indexSearcher = new IndexSearcher(reader);
|
||||
ScoreDoc[] hits = indexSearcher.search(booleanQuery, filter, 1000).scoreDocs;
|
||||
assertEquals("Number of matched documents", 1, hits.length);
|
||||
indexSearcher.close();
|
||||
reader.close();
|
||||
}
|
||||
catch (IOException e) {
|
||||
|
|
|
@ -185,7 +185,6 @@ public class TestFuzzyQuery extends LuceneTestCase {
|
|||
hits = searcher.search(query, null, 1000).scoreDocs;
|
||||
assertEquals(0, hits.length);
|
||||
|
||||
searcher.close();
|
||||
reader.close();
|
||||
directory.close();
|
||||
}
|
||||
|
@ -275,7 +274,6 @@ public class TestFuzzyQuery extends LuceneTestCase {
|
|||
// expecting exception
|
||||
}
|
||||
|
||||
searcher.close();
|
||||
reader.close();
|
||||
directory.close();
|
||||
}
|
||||
|
@ -312,7 +310,6 @@ public class TestFuzzyQuery extends LuceneTestCase {
|
|||
FuzzyQuery fq = new FuzzyQuery(new Term("field", "z123456"), 1f, 0, 2);
|
||||
TopDocs docs = searcher.search(fq, 2);
|
||||
assertEquals(5, docs.totalHits); // 5 docs, from the a and b's
|
||||
searcher.close();
|
||||
mr.close();
|
||||
ir1.close();
|
||||
ir2.close();
|
||||
|
@ -353,7 +350,6 @@ public class TestFuzzyQuery extends LuceneTestCase {
|
|||
hits = searcher.search(query, null, 1000).scoreDocs;
|
||||
assertEquals(0, hits.length);
|
||||
|
||||
searcher.close();
|
||||
reader.close();
|
||||
directory.close();
|
||||
}
|
||||
|
@ -378,7 +374,6 @@ public class TestFuzzyQuery extends LuceneTestCase {
|
|||
assertEquals("Lucene", reader.document(hits[0].doc).get("field"));
|
||||
assertEquals("Lucene", reader.document(hits[1].doc).get("field"));
|
||||
assertEquals("Lucenne", reader.document(hits[2].doc).get("field"));
|
||||
searcher.close();
|
||||
reader.close();
|
||||
directory.close();
|
||||
}
|
||||
|
@ -416,7 +411,6 @@ public class TestFuzzyQuery extends LuceneTestCase {
|
|||
ScoreDoc[] hits = searcher.search(q, 10).scoreDocs;
|
||||
assertEquals(1, hits.length);
|
||||
assertEquals("Giga byte", searcher.doc(hits[0].doc).get("field"));
|
||||
searcher.close();
|
||||
r.close();
|
||||
index.close();
|
||||
}
|
||||
|
@ -457,7 +451,6 @@ public class TestFuzzyQuery extends LuceneTestCase {
|
|||
assertEquals("test", searcher.doc(hits[0].doc).get("field"));
|
||||
assertEquals("foobar", searcher.doc(hits[1].doc).get("field"));
|
||||
|
||||
searcher.close();
|
||||
reader.close();
|
||||
index.close();
|
||||
}
|
||||
|
|
Some files were not shown because too many files have changed in this diff Show More
Loading…
Reference in New Issue