LUCENE-7983: IndexWriter.IndexReaderWarmer is now a functional interface instead of an abstract class with a single method.

This commit is contained in:
Dawid Weiss 2017-10-04 10:54:13 +02:00
parent b09c99f516
commit 46cd679e91
7 changed files with 64 additions and 82 deletions

View File

@ -106,6 +106,10 @@ Other
* LUCENE-7978: Add some more documentation about setting up build
environment. (Anton R. Yuste via Uwe Schindler)
* LUCENE-7983: IndexWriter.IndexReaderWarmer is now a functional interface
instead of an abstract class with a single method (Dawid Weiss)
======================= Lucene 7.0.1 =======================
(No Changes)

View File

@ -4804,19 +4804,16 @@ public class IndexWriter implements Closeable, TwoPhaseCommit, Accountable {
*
* @lucene.experimental
*
* <p><b>NOTE</b>: warm is called before any deletes have
* been carried over to the merged segment. */
public static abstract class IndexReaderWarmer {
/** Sole constructor. (For invocation by subclass
* constructors, typically implicit.) */
protected IndexReaderWarmer() {
}
/** Invoked on the {@link LeafReader} for the newly
* merged segment, before that segment is made visible
* to near-real-time readers. */
public abstract void warm(LeafReader reader) throws IOException;
* <p><b>NOTE</b>: {@link #warm(LeafReader)} is called before any
* deletes have been carried over to the merged segment. */
@FunctionalInterface
public static interface IndexReaderWarmer {
/**
* Invoked on the {@link LeafReader} for the newly
* merged segment, before that segment is made visible
* to near-real-time readers.
*/
void warm(LeafReader reader) throws IOException;
}
void tragicEvent(Throwable tragedy, String location) throws IOException {

View File

@ -26,9 +26,9 @@ import org.apache.lucene.util.InfoStream;
* A very simple merged segment warmer that just ensures
* data structures are initialized.
*/
public class SimpleMergedSegmentWarmer extends IndexReaderWarmer {
public class SimpleMergedSegmentWarmer implements IndexReaderWarmer {
private final InfoStream infoStream;
/**
* Creates a new SimpleMergedSegmentWarmer
* @param infoStream InfoStream to log statistics about warming.
@ -53,7 +53,7 @@ public class SimpleMergedSegmentWarmer extends IndexReaderWarmer {
normsCount++;
}
}
if (info.getDocValuesType() != DocValuesType.NONE) {
switch(info.getDocValuesType()) {
case NUMERIC:

View File

@ -129,17 +129,14 @@ public class TestDemoParallelLeafReader extends LuceneTestCase {
}
w = new IndexWriter(indexDir, iwc);
w.getConfig().setMergedSegmentWarmer(new IndexWriter.IndexReaderWarmer() {
@Override
public void warm(LeafReader reader) throws IOException {
// This will build the parallel index for the merged segment before the merge becomes visible, so reopen delay is only due to
// newly flushed segments:
if (DEBUG) System.out.println(Thread.currentThread().getName() +": TEST: now warm " + reader);
// TODO: it's not great that we pass false here; it means we close the reader & reopen again for NRT reader; still we did "warm" by
// building the parallel index, if necessary
getParallelLeafReader(reader, false, getCurrentSchemaGen());
}
});
w.getConfig().setMergedSegmentWarmer((reader) -> {
// This will build the parallel index for the merged segment before the merge becomes visible, so reopen delay is only due to
// newly flushed segments:
if (DEBUG) System.out.println(Thread.currentThread().getName() +": TEST: now warm " + reader);
// TODO: it's not great that we pass false here; it means we close the reader & reopen again for NRT reader; still we did "warm" by
// building the parallel index, if necessary
getParallelLeafReader(reader, false, getCurrentSchemaGen());
});
// start with empty commit:
w.commit();

View File

@ -612,24 +612,15 @@ public class TestIndexWriterReader extends LuceneTestCase {
}
}
private static class MyWarmer extends IndexWriter.IndexReaderWarmer {
int warmCount;
@Override
public void warm(LeafReader reader) throws IOException {
warmCount++;
}
}
public void testMergeWarmer() throws Exception {
Directory dir1 = getAssertNoDeletesDirectory(newDirectory());
// Enroll warmer
MyWarmer warmer = new MyWarmer();
AtomicInteger warmCount = new AtomicInteger();
IndexWriter writer = new IndexWriter(
dir1,
newIndexWriterConfig(new MockAnalyzer(random()))
.setMaxBufferedDocs(2)
.setMergedSegmentWarmer(warmer)
.setMergedSegmentWarmer((leafReader) -> warmCount.incrementAndGet())
.setMergeScheduler(new ConcurrentMergeScheduler())
.setMergePolicy(newLogMergePolicy())
);
@ -648,12 +639,12 @@ public class TestIndexWriterReader extends LuceneTestCase {
}
((ConcurrentMergeScheduler) writer.getConfig().getMergeScheduler()).sync();
assertTrue(warmer.warmCount > 0);
final int count = warmer.warmCount;
assertTrue(warmCount.get() > 0);
final int count = warmCount.get();
writer.addDocument(DocHelper.createDocument(17, "test", 4));
writer.forceMerge(1);
assertTrue(warmer.warmCount > count);
assertTrue(warmCount.get() > count);
writer.close();
r1.close();
@ -969,24 +960,21 @@ public class TestIndexWriterReader extends LuceneTestCase {
final AtomicBoolean didWarm = new AtomicBoolean();
IndexWriter w = new IndexWriter(
dir,
newIndexWriterConfig(new MockAnalyzer(random())).
setMaxBufferedDocs(2).
setReaderPooling(true).
setMergedSegmentWarmer(new IndexWriter.IndexReaderWarmer() {
@Override
public void warm(LeafReader r) throws IOException {
IndexSearcher s = newSearcher(r);
TopDocs hits = s.search(new TermQuery(new Term("foo", "bar")), 10);
assertEquals(20, hits.totalHits);
didWarm.set(true);
}
}).
setMergePolicy(newLogMergePolicy(10))
newIndexWriterConfig(new MockAnalyzer(random()))
.setMaxBufferedDocs(2)
.setReaderPooling(true)
.setMergedSegmentWarmer((r) -> {
IndexSearcher s = newSearcher(r);
TopDocs hits = s.search(new TermQuery(new Term("foo", "bar")), 10);
assertEquals(20, hits.totalHits);
didWarm.set(true);
})
.setMergePolicy(newLogMergePolicy(10))
);
Document doc = new Document();
doc.add(newStringField("foo", "bar", Field.Store.NO));
for(int i=0;i<20;i++) {
for (int i = 0; i < 20; i++) {
w.addDocument(doc);
}
w.waitForMerges();

View File

@ -34,7 +34,7 @@ import org.apache.lucene.index.SegmentReader;
// TODO: or ... replica node can do merging locally? tricky to keep things in sync, when one node merges more slowly than others...
class PreCopyMergedSegmentWarmer extends IndexReaderWarmer {
class PreCopyMergedSegmentWarmer implements IndexReaderWarmer {
private final PrimaryNode primary;
@ -48,7 +48,7 @@ class PreCopyMergedSegmentWarmer extends IndexReaderWarmer {
final SegmentCommitInfo info = ((SegmentReader) reader).getSegmentInfo();
//System.out.println("TEST: warm merged segment files " + info);
Map<String,FileMetaData> filesMetaData = new HashMap<>();
for(String fileName : info.files()) {
for (String fileName : info.files()) {
FileMetaData metaData = primary.readLocalFileMetaData(fileName);
assert metaData != null;
assert filesMetaData.containsKey(fileName) == false;

View File

@ -16,7 +16,6 @@
*/
package org.apache.lucene.index;
import java.io.IOException;
import java.nio.file.Path;
import java.util.ArrayList;
import java.util.Collections;
@ -468,32 +467,29 @@ public abstract class ThreadedIndexingAndSearchingTestCase extends LuceneTestCas
mp.setNoCFSRatio(Math.max(0.25d, mp.getNoCFSRatio()));
}
conf.setMergedSegmentWarmer(new IndexWriter.IndexReaderWarmer() {
@Override
public void warm(LeafReader reader) throws IOException {
if (VERBOSE) {
System.out.println("TEST: now warm merged reader=" + reader);
}
warmed.put(((SegmentReader) reader).core, Boolean.TRUE);
final int maxDoc = reader.maxDoc();
final Bits liveDocs = reader.getLiveDocs();
int sum = 0;
final int inc = Math.max(1, maxDoc/50);
for(int docID=0;docID<maxDoc;docID += inc) {
if (liveDocs == null || liveDocs.get(docID)) {
final Document doc = reader.document(docID);
sum += doc.getFields().size();
}
}
IndexSearcher searcher = newSearcher(reader, false);
sum += searcher.search(new TermQuery(new Term("body", "united")), 10).totalHits;
if (VERBOSE) {
System.out.println("TEST: warm visited " + sum + " fields");
conf.setMergedSegmentWarmer((reader) -> {
if (VERBOSE) {
System.out.println("TEST: now warm merged reader=" + reader);
}
warmed.put(((SegmentReader) reader).core, Boolean.TRUE);
final int maxDoc = reader.maxDoc();
final Bits liveDocs = reader.getLiveDocs();
int sum = 0;
final int inc = Math.max(1, maxDoc/50);
for(int docID=0;docID<maxDoc;docID += inc) {
if (liveDocs == null || liveDocs.get(docID)) {
final Document doc = reader.document(docID);
sum += doc.getFields().size();
}
}
});
IndexSearcher searcher = newSearcher(reader, false);
sum += searcher.search(new TermQuery(new Term("body", "united")), 10).totalHits;
if (VERBOSE) {
System.out.println("TEST: warm visited " + sum + " fields");
}
});
if (VERBOSE) {
conf.setInfoStream(new PrintStreamInfoStream(System.out) {