Skip to content

Commit 8294e1a

Browse files
s1monwmikemccand
andauthored
LUCENE-8962: Merge segments on getReader (#1623)
Add IndexWriter merge-on-refresh feature to selectively merge small segments on getReader, subject to a configurable timeout, to improve search performance by reducing the number of small segments for searching. Co-authored-by: Mike McCandless <mikemccand@apache.org>
1 parent 96a853b commit 8294e1a

File tree

15 files changed

+478
-138
lines changed

15 files changed

+478
-138
lines changed

lucene/CHANGES.txt

Lines changed: 4 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -172,6 +172,10 @@ New Features
172172

173173
* LUCENE-9386: RegExpQuery added case insensitive matching option. (Mark Harwood)
174174

175+
* LUCENE-8962: Add IndexWriter merge-on-refresh feature to selectively merge
176+
small segments on getReader, subject to a configurable timeout, to improve
177+
search performance by reducing the number of small segments for searching. (Simon Willnauer)
178+
175179
Improvements
176180
---------------------
177181

lucene/core/src/java/org/apache/lucene/index/IndexWriter.java

Lines changed: 212 additions & 49 deletions
Large diffs are not rendered by default.

lucene/core/src/java/org/apache/lucene/index/IndexWriterConfig.java

Lines changed: 8 additions & 7 deletions
Original file line numberDiff line numberDiff line change
@@ -110,8 +110,8 @@ public static enum OpenMode {
110110
/** Default value for whether calls to {@link IndexWriter#close()} include a commit. */
111111
public final static boolean DEFAULT_COMMIT_ON_CLOSE = true;
112112

113-
/** Default value for time to wait for merges on commit (when using a {@link MergePolicy} that implements {@link MergePolicy#findFullFlushMerges}). */
114-
public static final long DEFAULT_MAX_COMMIT_MERGE_WAIT_MILLIS = 0;
113+
/** Default value for time to wait for merges on commit or getReader (when using a {@link MergePolicy} that implements {@link MergePolicy#findFullFlushMerges}). */
114+
public static final long DEFAULT_MAX_FULL_FLUSH_MERGE_WAIT_MILLIS = 0;
115115

116116
// indicates whether this config instance is already attached to a writer.
117117
// not final so that it can be cloned properly.
@@ -463,17 +463,18 @@ public IndexWriterConfig setCommitOnClose(boolean commitOnClose) {
463463
}
464464

465465
/**
466-
* Expert: sets the amount of time to wait for merges (during {@link IndexWriter#commit}) returned by
466+
* Expert: sets the amount of time to wait for merges (during {@link IndexWriter#commit}
467+
* or {@link IndexWriter#getReader(boolean, boolean)}) returned by
467468
* MergePolicy.findFullFlushMerges(...).
468469
* If this time is reached, we proceed with the commit based on segments merged up to that point.
469-
* The merges are not cancelled, and will still run to completion independent of the commit,
470-
* like natural segment merges. The default is <code>{@value IndexWriterConfig#DEFAULT_MAX_COMMIT_MERGE_WAIT_MILLIS}</code>.
470+
* The merges are not aborted, and will still run to completion independent of the commit or getReader call,
471+
* like natural segment merges. The default is <code>{@value IndexWriterConfig#DEFAULT_MAX_FULL_FLUSH_MERGE_WAIT_MILLIS}</code>.
471472
*
472473
* Note: This settings has no effect unless {@link MergePolicy#findFullFlushMerges(MergeTrigger, SegmentInfos, MergePolicy.MergeContext)}
473474
* has an implementation that actually returns merges which by default doesn't return any merges.
474475
*/
475-
public IndexWriterConfig setMaxCommitMergeWaitMillis(long maxCommitMergeWaitMillis) {
476-
this.maxCommitMergeWaitMillis = maxCommitMergeWaitMillis;
476+
public IndexWriterConfig setMaxFullFlushMergeWaitMillis(long maxFullFlushMergeWaitMillis) {
477+
this.maxFullFlushMergeWaitMillis = maxFullFlushMergeWaitMillis;
477478
return this;
478479
}
479480

lucene/core/src/java/org/apache/lucene/index/LiveIndexWriterConfig.java

Lines changed: 5 additions & 5 deletions
Original file line numberDiff line numberDiff line change
@@ -110,7 +110,7 @@ public class LiveIndexWriterConfig {
110110
protected String softDeletesField = null;
111111

112112
/** Amount of time to wait for merges returned by MergePolicy.findFullFlushMerges(...) */
113-
protected volatile long maxCommitMergeWaitMillis;
113+
protected volatile long maxFullFlushMergeWaitMillis;
114114

115115
// used by IndexWriterConfig
116116
LiveIndexWriterConfig(Analyzer analyzer) {
@@ -134,7 +134,7 @@ public class LiveIndexWriterConfig {
134134
flushPolicy = new FlushByRamOrCountsPolicy();
135135
readerPooling = IndexWriterConfig.DEFAULT_READER_POOLING;
136136
perThreadHardLimitMB = IndexWriterConfig.DEFAULT_RAM_PER_THREAD_HARD_LIMIT_MB;
137-
maxCommitMergeWaitMillis = IndexWriterConfig.DEFAULT_MAX_COMMIT_MERGE_WAIT_MILLIS;
137+
maxFullFlushMergeWaitMillis = IndexWriterConfig.DEFAULT_MAX_FULL_FLUSH_MERGE_WAIT_MILLIS;
138138
}
139139

140140
/** Returns the default analyzer to use for indexing documents. */
@@ -469,8 +469,8 @@ public String getSoftDeletesField() {
469469
* If this time is reached, we proceed with the commit based on segments merged up to that point.
470470
* The merges are not cancelled, and may still run to completion independent of the commit.
471471
*/
472-
public long getMaxCommitMergeWaitMillis() {
473-
return maxCommitMergeWaitMillis;
472+
public long getMaxFullFlushMergeWaitMillis() {
473+
return maxFullFlushMergeWaitMillis;
474474
}
475475

476476
@Override
@@ -496,7 +496,7 @@ public String toString() {
496496
sb.append("indexSort=").append(getIndexSort()).append("\n");
497497
sb.append("checkPendingFlushOnUpdate=").append(isCheckPendingFlushOnUpdate()).append("\n");
498498
sb.append("softDeletesField=").append(getSoftDeletesField()).append("\n");
499-
sb.append("maxCommitMergeWaitMillis=").append(getMaxCommitMergeWaitMillis()).append("\n");
499+
sb.append("maxFullFlushMergeWaitMillis=").append(getMaxFullFlushMergeWaitMillis()).append("\n");
500500
return sb.toString();
501501
}
502502
}

lucene/core/src/java/org/apache/lucene/index/MergePolicy.java

Lines changed: 9 additions & 8 deletions
Original file line numberDiff line numberDiff line change
@@ -421,7 +421,7 @@ Optional<Boolean> hasCompletedSuccessfully() {
421421
/**
422422
* Called just before the merge is applied to IndexWriter's SegmentInfos
423423
*/
424-
void onMergeComplete() {
424+
void onMergeComplete() throws IOException {
425425
}
426426

427427
/**
@@ -623,19 +623,20 @@ public abstract MergeSpecification findForcedDeletesMerges(
623623
/**
624624
* Identifies merges that we want to execute (synchronously) on commit. By default, this will do no merging on commit.
625625
* If you implement this method in your {@code MergePolicy} you must also set a non-zero timeout using
626-
* {@link IndexWriterConfig#setMaxCommitMergeWaitMillis}.
626+
* {@link IndexWriterConfig#setMaxFullFlushMergeWaitMillis}.
627627
*
628-
* Any merges returned here will make {@link IndexWriter#commit()} or {@link IndexWriter#prepareCommit()} block until
629-
* the merges complete or until {@link IndexWriterConfig#getMaxCommitMergeWaitMillis()} has elapsed. This may be
630-
* used to merge small segments that have just been flushed as part of the commit, reducing the number of segments in
631-
* the commit. If a merge does not complete in the allotted time, it will continue to execute, and eventually finish and
632-
* apply to future commits, but will not be reflected in the current commit.
628+
* Any merges returned here will make {@link IndexWriter#commit()}, {@link IndexWriter#prepareCommit()}
629+
* or {@link IndexWriter#getReader(boolean, boolean)} block until
630+
* the merges complete or until {@link IndexWriterConfig#getMaxFullFlushMergeWaitMillis()} has elapsed. This may be
631+
* used to merge small segments that have just been flushed, reducing the number of segments in
632+
* the point in time snapshot. If a merge does not complete in the allotted time, it will continue to execute, and eventually finish and
633+
* apply to future point in time snapshot, but will not be reflected in the current one.
633634
*
634635
* If a {@link OneMerge} in the returned {@link MergeSpecification} includes a segment already included in a registered
635636
* merge, then {@link IndexWriter#commit()} or {@link IndexWriter#prepareCommit()} will throw a {@link IllegalStateException}.
636637
* Use {@link MergeContext#getMergingSegments()} to determine which segments are currently registered to merge.
637638
*
638-
* @param mergeTrigger the event that triggered the merge (COMMIT or FULL_FLUSH).
639+
* @param mergeTrigger the event that triggered the merge (COMMIT or GET_READER).
639640
* @param segmentInfos the total set of segments in the index (while preparing the commit)
640641
* @param mergeContext the MergeContext to find the merges on, which should be used to determine which segments are
641642
* already in a registered merge (see {@link MergeContext#getMergingSegments()}).

lucene/core/src/java/org/apache/lucene/index/MergeTrigger.java

Lines changed: 4 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -53,4 +53,8 @@ public enum MergeTrigger {
5353
* Merge was triggered on commit.
5454
*/
5555
COMMIT,
56+
/**
57+
* Merge was triggered on opening NRT readers.
58+
*/
59+
GET_READER,
5660
}

lucene/core/src/java/org/apache/lucene/index/ReaderPool.java

Lines changed: 1 addition & 1 deletion
Original file line numberDiff line numberDiff line change
@@ -404,7 +404,7 @@ private PendingDeletes newPendingDeletes(SegmentReader reader, SegmentCommitInfo
404404
private boolean noDups() {
405405
Set<String> seen = new HashSet<>();
406406
for(SegmentCommitInfo info : readerMap.keySet()) {
407-
assert !seen.contains(info.info.name);
407+
assert !seen.contains(info.info.name) : "seen twice: " + info.info.name ;
408408
seen.add(info.info.name);
409409
}
410410
return true;

lucene/core/src/java/org/apache/lucene/index/StandardDirectoryReader.java

Lines changed: 10 additions & 14 deletions
Original file line numberDiff line numberDiff line change
@@ -82,7 +82,8 @@ protected DirectoryReader doBody(String segmentFileName) throws IOException {
8282
}
8383

8484
/** Used by near real-time search */
85-
static DirectoryReader open(IndexWriter writer, SegmentInfos infos, boolean applyAllDeletes, boolean writeAllDeletes) throws IOException {
85+
static StandardDirectoryReader open(IndexWriter writer, IOUtils.IOFunction<SegmentCommitInfo, SegmentReader> readerFunction,
86+
SegmentInfos infos, boolean applyAllDeletes, boolean writeAllDeletes) throws IOException {
8687
// IndexWriter synchronizes externally before calling
8788
// us, which ensures infos will not change; so there's
8889
// no need to process segments in reverse order
@@ -101,19 +102,14 @@ static DirectoryReader open(IndexWriter writer, SegmentInfos infos, boolean appl
101102
// IndexWriter's segmentInfos:
102103
final SegmentCommitInfo info = infos.info(i);
103104
assert info.info.dir == dir;
104-
final ReadersAndUpdates rld = writer.getPooledInstance(info, true);
105-
try {
106-
final SegmentReader reader = rld.getReadOnlyClone(IOContext.READ);
107-
if (reader.numDocs() > 0 || writer.getConfig().mergePolicy.keepFullyDeletedSegment(() -> reader)) {
108-
// Steal the ref:
109-
readers.add(reader);
110-
infosUpto++;
111-
} else {
112-
reader.decRef();
113-
segmentInfos.remove(infosUpto);
114-
}
115-
} finally {
116-
writer.release(rld);
105+
final SegmentReader reader = readerFunction.apply(info);
106+
if (reader.numDocs() > 0 || writer.getConfig().mergePolicy.keepFullyDeletedSegment(() -> reader)) {
107+
// Steal the ref:
108+
readers.add(reader);
109+
infosUpto++;
110+
} else {
111+
reader.decRef();
112+
segmentInfos.remove(infosUpto);
117113
}
118114
}
119115

lucene/core/src/test/org/apache/lucene/index/TestIndexWriter.java

Lines changed: 1 addition & 1 deletion
Original file line numberDiff line numberDiff line change
@@ -4206,7 +4206,7 @@ public void mergeFinished(boolean success, boolean segmentDropped) throws IOExce
42064206
public void testMergeOnCommitKeepFullyDeletedSegments() throws Exception {
42074207
Directory dir = newDirectory();
42084208
IndexWriterConfig iwc = newIndexWriterConfig();
4209-
iwc.setMaxCommitMergeWaitMillis(30 * 1000);
4209+
iwc.setMaxFullFlushMergeWaitMillis(30 * 1000);
42104210
iwc.mergePolicy = new FilterMergePolicy(newMergePolicy()) {
42114211
@Override
42124212
public boolean keepFullyDeletedSegment(IOSupplier<CodecReader> readerIOSupplier) {

0 commit comments

Comments
 (0)