mirror of https://github.com/apache/lucene.git
LUCENE-7983: IndexWriter.IndexReaderWarmer is now a functional interface instead of an abstract class with a single method.
This commit is contained in:
parent
b09c99f516
commit
46cd679e91
|
@ -106,6 +106,10 @@ Other
|
||||||
* LUCENE-7978: Add some more documentation about setting up build
|
* LUCENE-7978: Add some more documentation about setting up build
|
||||||
environment. (Anton R. Yuste via Uwe Schindler)
|
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 =======================
|
======================= Lucene 7.0.1 =======================
|
||||||
(No Changes)
|
(No Changes)
|
||||||
|
|
||||||
|
|
|
@ -4804,19 +4804,16 @@ public class IndexWriter implements Closeable, TwoPhaseCommit, Accountable {
|
||||||
*
|
*
|
||||||
* @lucene.experimental
|
* @lucene.experimental
|
||||||
*
|
*
|
||||||
* <p><b>NOTE</b>: warm is called before any deletes have
|
* <p><b>NOTE</b>: {@link #warm(LeafReader)} is called before any
|
||||||
* been carried over to the merged segment. */
|
* deletes have been carried over to the merged segment. */
|
||||||
public static abstract class IndexReaderWarmer {
|
@FunctionalInterface
|
||||||
|
public static interface IndexReaderWarmer {
|
||||||
/** Sole constructor. (For invocation by subclass
|
/**
|
||||||
* constructors, typically implicit.) */
|
* Invoked on the {@link LeafReader} for the newly
|
||||||
protected IndexReaderWarmer() {
|
* merged segment, before that segment is made visible
|
||||||
}
|
* to near-real-time readers.
|
||||||
|
*/
|
||||||
/** Invoked on the {@link LeafReader} for the newly
|
void warm(LeafReader reader) throws IOException;
|
||||||
* merged segment, before that segment is made visible
|
|
||||||
* to near-real-time readers. */
|
|
||||||
public abstract void warm(LeafReader reader) throws IOException;
|
|
||||||
}
|
}
|
||||||
|
|
||||||
void tragicEvent(Throwable tragedy, String location) throws IOException {
|
void tragicEvent(Throwable tragedy, String location) throws IOException {
|
||||||
|
|
|
@ -26,7 +26,7 @@ import org.apache.lucene.util.InfoStream;
|
||||||
* A very simple merged segment warmer that just ensures
|
* A very simple merged segment warmer that just ensures
|
||||||
* data structures are initialized.
|
* data structures are initialized.
|
||||||
*/
|
*/
|
||||||
public class SimpleMergedSegmentWarmer extends IndexReaderWarmer {
|
public class SimpleMergedSegmentWarmer implements IndexReaderWarmer {
|
||||||
private final InfoStream infoStream;
|
private final InfoStream infoStream;
|
||||||
|
|
||||||
/**
|
/**
|
||||||
|
|
|
@ -129,17 +129,14 @@ public class TestDemoParallelLeafReader extends LuceneTestCase {
|
||||||
}
|
}
|
||||||
w = new IndexWriter(indexDir, iwc);
|
w = new IndexWriter(indexDir, iwc);
|
||||||
|
|
||||||
w.getConfig().setMergedSegmentWarmer(new IndexWriter.IndexReaderWarmer() {
|
w.getConfig().setMergedSegmentWarmer((reader) -> {
|
||||||
@Override
|
// This will build the parallel index for the merged segment before the merge becomes visible, so reopen delay is only due to
|
||||||
public void warm(LeafReader reader) throws IOException {
|
// newly flushed segments:
|
||||||
// This will build the parallel index for the merged segment before the merge becomes visible, so reopen delay is only due to
|
if (DEBUG) System.out.println(Thread.currentThread().getName() +": TEST: now warm " + reader);
|
||||||
// newly flushed segments:
|
// 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
|
||||||
if (DEBUG) System.out.println(Thread.currentThread().getName() +": TEST: now warm " + reader);
|
// building the parallel index, if necessary
|
||||||
// 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
|
getParallelLeafReader(reader, false, getCurrentSchemaGen());
|
||||||
// building the parallel index, if necessary
|
});
|
||||||
getParallelLeafReader(reader, false, getCurrentSchemaGen());
|
|
||||||
}
|
|
||||||
});
|
|
||||||
|
|
||||||
// start with empty commit:
|
// start with empty commit:
|
||||||
w.commit();
|
w.commit();
|
||||||
|
|
|
@ -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 {
|
public void testMergeWarmer() throws Exception {
|
||||||
|
|
||||||
Directory dir1 = getAssertNoDeletesDirectory(newDirectory());
|
Directory dir1 = getAssertNoDeletesDirectory(newDirectory());
|
||||||
// Enroll warmer
|
// Enroll warmer
|
||||||
MyWarmer warmer = new MyWarmer();
|
AtomicInteger warmCount = new AtomicInteger();
|
||||||
IndexWriter writer = new IndexWriter(
|
IndexWriter writer = new IndexWriter(
|
||||||
dir1,
|
dir1,
|
||||||
newIndexWriterConfig(new MockAnalyzer(random()))
|
newIndexWriterConfig(new MockAnalyzer(random()))
|
||||||
.setMaxBufferedDocs(2)
|
.setMaxBufferedDocs(2)
|
||||||
.setMergedSegmentWarmer(warmer)
|
.setMergedSegmentWarmer((leafReader) -> warmCount.incrementAndGet())
|
||||||
.setMergeScheduler(new ConcurrentMergeScheduler())
|
.setMergeScheduler(new ConcurrentMergeScheduler())
|
||||||
.setMergePolicy(newLogMergePolicy())
|
.setMergePolicy(newLogMergePolicy())
|
||||||
);
|
);
|
||||||
|
@ -648,12 +639,12 @@ public class TestIndexWriterReader extends LuceneTestCase {
|
||||||
}
|
}
|
||||||
((ConcurrentMergeScheduler) writer.getConfig().getMergeScheduler()).sync();
|
((ConcurrentMergeScheduler) writer.getConfig().getMergeScheduler()).sync();
|
||||||
|
|
||||||
assertTrue(warmer.warmCount > 0);
|
assertTrue(warmCount.get() > 0);
|
||||||
final int count = warmer.warmCount;
|
final int count = warmCount.get();
|
||||||
|
|
||||||
writer.addDocument(DocHelper.createDocument(17, "test", 4));
|
writer.addDocument(DocHelper.createDocument(17, "test", 4));
|
||||||
writer.forceMerge(1);
|
writer.forceMerge(1);
|
||||||
assertTrue(warmer.warmCount > count);
|
assertTrue(warmCount.get() > count);
|
||||||
|
|
||||||
writer.close();
|
writer.close();
|
||||||
r1.close();
|
r1.close();
|
||||||
|
@ -969,24 +960,21 @@ public class TestIndexWriterReader extends LuceneTestCase {
|
||||||
final AtomicBoolean didWarm = new AtomicBoolean();
|
final AtomicBoolean didWarm = new AtomicBoolean();
|
||||||
IndexWriter w = new IndexWriter(
|
IndexWriter w = new IndexWriter(
|
||||||
dir,
|
dir,
|
||||||
newIndexWriterConfig(new MockAnalyzer(random())).
|
newIndexWriterConfig(new MockAnalyzer(random()))
|
||||||
setMaxBufferedDocs(2).
|
.setMaxBufferedDocs(2)
|
||||||
setReaderPooling(true).
|
.setReaderPooling(true)
|
||||||
setMergedSegmentWarmer(new IndexWriter.IndexReaderWarmer() {
|
.setMergedSegmentWarmer((r) -> {
|
||||||
@Override
|
IndexSearcher s = newSearcher(r);
|
||||||
public void warm(LeafReader r) throws IOException {
|
TopDocs hits = s.search(new TermQuery(new Term("foo", "bar")), 10);
|
||||||
IndexSearcher s = newSearcher(r);
|
assertEquals(20, hits.totalHits);
|
||||||
TopDocs hits = s.search(new TermQuery(new Term("foo", "bar")), 10);
|
didWarm.set(true);
|
||||||
assertEquals(20, hits.totalHits);
|
})
|
||||||
didWarm.set(true);
|
.setMergePolicy(newLogMergePolicy(10))
|
||||||
}
|
|
||||||
}).
|
|
||||||
setMergePolicy(newLogMergePolicy(10))
|
|
||||||
);
|
);
|
||||||
|
|
||||||
Document doc = new Document();
|
Document doc = new Document();
|
||||||
doc.add(newStringField("foo", "bar", Field.Store.NO));
|
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.addDocument(doc);
|
||||||
}
|
}
|
||||||
w.waitForMerges();
|
w.waitForMerges();
|
||||||
|
|
|
@ -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...
|
// 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;
|
private final PrimaryNode primary;
|
||||||
|
|
||||||
|
@ -48,7 +48,7 @@ class PreCopyMergedSegmentWarmer extends IndexReaderWarmer {
|
||||||
final SegmentCommitInfo info = ((SegmentReader) reader).getSegmentInfo();
|
final SegmentCommitInfo info = ((SegmentReader) reader).getSegmentInfo();
|
||||||
//System.out.println("TEST: warm merged segment files " + info);
|
//System.out.println("TEST: warm merged segment files " + info);
|
||||||
Map<String,FileMetaData> filesMetaData = new HashMap<>();
|
Map<String,FileMetaData> filesMetaData = new HashMap<>();
|
||||||
for(String fileName : info.files()) {
|
for (String fileName : info.files()) {
|
||||||
FileMetaData metaData = primary.readLocalFileMetaData(fileName);
|
FileMetaData metaData = primary.readLocalFileMetaData(fileName);
|
||||||
assert metaData != null;
|
assert metaData != null;
|
||||||
assert filesMetaData.containsKey(fileName) == false;
|
assert filesMetaData.containsKey(fileName) == false;
|
||||||
|
|
|
@ -16,7 +16,6 @@
|
||||||
*/
|
*/
|
||||||
package org.apache.lucene.index;
|
package org.apache.lucene.index;
|
||||||
|
|
||||||
import java.io.IOException;
|
|
||||||
import java.nio.file.Path;
|
import java.nio.file.Path;
|
||||||
import java.util.ArrayList;
|
import java.util.ArrayList;
|
||||||
import java.util.Collections;
|
import java.util.Collections;
|
||||||
|
@ -468,32 +467,29 @@ public abstract class ThreadedIndexingAndSearchingTestCase extends LuceneTestCas
|
||||||
mp.setNoCFSRatio(Math.max(0.25d, mp.getNoCFSRatio()));
|
mp.setNoCFSRatio(Math.max(0.25d, mp.getNoCFSRatio()));
|
||||||
}
|
}
|
||||||
|
|
||||||
conf.setMergedSegmentWarmer(new IndexWriter.IndexReaderWarmer() {
|
conf.setMergedSegmentWarmer((reader) -> {
|
||||||
@Override
|
if (VERBOSE) {
|
||||||
public void warm(LeafReader reader) throws IOException {
|
System.out.println("TEST: now warm merged reader=" + reader);
|
||||||
if (VERBOSE) {
|
}
|
||||||
System.out.println("TEST: now warm merged reader=" + reader);
|
warmed.put(((SegmentReader) reader).core, Boolean.TRUE);
|
||||||
}
|
final int maxDoc = reader.maxDoc();
|
||||||
warmed.put(((SegmentReader) reader).core, Boolean.TRUE);
|
final Bits liveDocs = reader.getLiveDocs();
|
||||||
final int maxDoc = reader.maxDoc();
|
int sum = 0;
|
||||||
final Bits liveDocs = reader.getLiveDocs();
|
final int inc = Math.max(1, maxDoc/50);
|
||||||
int sum = 0;
|
for(int docID=0;docID<maxDoc;docID += inc) {
|
||||||
final int inc = Math.max(1, maxDoc/50);
|
if (liveDocs == null || liveDocs.get(docID)) {
|
||||||
for(int docID=0;docID<maxDoc;docID += inc) {
|
final Document doc = reader.document(docID);
|
||||||
if (liveDocs == null || liveDocs.get(docID)) {
|
sum += doc.getFields().size();
|
||||||
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");
|
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
});
|
|
||||||
|
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) {
|
if (VERBOSE) {
|
||||||
conf.setInfoStream(new PrintStreamInfoStream(System.out) {
|
conf.setInfoStream(new PrintStreamInfoStream(System.out) {
|
||||||
|
|
Loading…
Reference in New Issue