mirror of https://github.com/apache/lucene.git
LUCENE-8962: add ability to selectively merge on commit (#1552)
Co-authored-by: Michael Froh <msfroh@apache.org> Co-authored-by: Simon Willnauer <simonw@apache.org>
This commit is contained in:
parent
56febf05c3
commit
972c84022e
|
@ -405,6 +405,10 @@ Improvements
|
|||
|
||||
* LUCENE-9253: KoreanTokenizer now supports custom dictionaries(system, unknown). (Namgyu Kim)
|
||||
|
||||
* LUCENE-8962: Add IndexWriter merge-on-commit feature to selectively merge small segments on commit,
|
||||
subject to a configurable timeout, to improve search performance by reducing the number of small
|
||||
segments for searching (Michael Froh, Mike Mccandless, Simon Willnauer)
|
||||
|
||||
* LUCENE-9171: QueryBuilder can now use BoostAttributes on input token streams to selectively
|
||||
boost particular terms or synonyms in parsed queries. (Alessandro Benedetti, Alan Woodward)
|
||||
|
||||
|
|
|
@ -57,6 +57,11 @@ public class FilterMergePolicy extends MergePolicy {
|
|||
return in.findForcedDeletesMerges(segmentInfos, mergeContext);
|
||||
}
|
||||
|
||||
@Override
|
||||
public MergeSpecification findFullFlushMerges(MergeTrigger mergeTrigger, SegmentInfos segmentInfos, MergeContext mergeContext) throws IOException {
|
||||
return in.findFullFlushMerges(mergeTrigger, segmentInfos, mergeContext);
|
||||
}
|
||||
|
||||
@Override
|
||||
public boolean useCompoundFile(SegmentInfos infos, SegmentCommitInfo mergedInfo, MergeContext mergeContext)
|
||||
throws IOException {
|
||||
|
|
|
@ -33,6 +33,7 @@ import java.util.Queue;
|
|||
import java.util.Set;
|
||||
import java.util.concurrent.ConcurrentLinkedQueue;
|
||||
import java.util.concurrent.Semaphore;
|
||||
import java.util.concurrent.TimeUnit;
|
||||
import java.util.concurrent.atomic.AtomicBoolean;
|
||||
import java.util.concurrent.atomic.AtomicInteger;
|
||||
import java.util.concurrent.atomic.AtomicLong;
|
||||
|
@ -2166,7 +2167,13 @@ public class IndexWriter implements Closeable, TwoPhaseCommit, Accountable,
|
|||
}
|
||||
}
|
||||
} else {
|
||||
spec = mergePolicy.findMerges(trigger, segmentInfos, this);
|
||||
switch (trigger) {
|
||||
case COMMIT:
|
||||
spec = mergePolicy.findFullFlushMerges(trigger, segmentInfos, this);
|
||||
break;
|
||||
default:
|
||||
spec = mergePolicy.findMerges(trigger, segmentInfos, this);
|
||||
}
|
||||
}
|
||||
if (spec != null) {
|
||||
final int numMerges = spec.merges.size();
|
||||
|
@ -3172,7 +3179,9 @@ public class IndexWriter implements Closeable, TwoPhaseCommit, Accountable,
|
|||
SegmentInfos toCommit = null;
|
||||
boolean anyChanges = false;
|
||||
long seqNo;
|
||||
|
||||
MergePolicy.MergeSpecification onCommitMerges = null;
|
||||
AtomicBoolean includeInCommit = new AtomicBoolean(true);
|
||||
final long maxCommitMergeWaitSeconds = config.getMaxCommitMergeWaitSeconds();
|
||||
// This is copied from doFlush, except it's modified to
|
||||
// clone & incRef the flushed SegmentInfos inside the
|
||||
// sync block:
|
||||
|
@ -3226,6 +3235,45 @@ public class IndexWriter implements Closeable, TwoPhaseCommit, Accountable,
|
|||
// sneak into the commit point:
|
||||
toCommit = segmentInfos.clone();
|
||||
|
||||
if (anyChanges && maxCommitMergeWaitSeconds > 0) {
|
||||
SegmentInfos committingSegmentInfos = toCommit;
|
||||
onCommitMerges = updatePendingMerges(new OneMergeWrappingMergePolicy(config.getMergePolicy(), toWrap ->
|
||||
new MergePolicy.OneMerge(toWrap.segments) {
|
||||
@Override
|
||||
public void mergeFinished(boolean committed) throws IOException {
|
||||
assert Thread.holdsLock(IndexWriter.this);
|
||||
if (committed && includeInCommit.get()) {
|
||||
deleter.incRef(info.files());
|
||||
Set<String> mergedSegmentNames = new HashSet<>();
|
||||
for (SegmentCommitInfo sci : segments) {
|
||||
mergedSegmentNames.add(sci.info.name);
|
||||
}
|
||||
List<SegmentCommitInfo> toCommitMergedAwaySegments = new ArrayList<>();
|
||||
for (SegmentCommitInfo sci : committingSegmentInfos) {
|
||||
if (mergedSegmentNames.contains(sci.info.name)) {
|
||||
toCommitMergedAwaySegments.add(sci);
|
||||
deleter.decRef(sci.files());
|
||||
}
|
||||
}
|
||||
// Construct a OneMerge that applies to toCommit
|
||||
MergePolicy.OneMerge applicableMerge = new MergePolicy.OneMerge(toCommitMergedAwaySegments);
|
||||
applicableMerge.info = info.clone();
|
||||
long segmentCounter = Long.parseLong(info.info.name.substring(1), Character.MAX_RADIX);
|
||||
committingSegmentInfos.counter = Math.max(committingSegmentInfos.counter, segmentCounter + 1);
|
||||
committingSegmentInfos.applyMergeChanges(applicableMerge, false);
|
||||
}
|
||||
toWrap.mergeFinished(committed);
|
||||
super.mergeFinished(committed);
|
||||
}
|
||||
|
||||
@Override
|
||||
public CodecReader wrapForMerge(CodecReader reader) throws IOException {
|
||||
return toWrap.wrapForMerge(reader);
|
||||
}
|
||||
}
|
||||
), MergeTrigger.COMMIT, UNBOUNDED_MAX_MERGE_SEGMENTS);
|
||||
}
|
||||
|
||||
pendingCommitChangeCount = changeCount.get();
|
||||
|
||||
// This protects the segmentInfos we are now going
|
||||
|
@ -3233,8 +3281,7 @@ public class IndexWriter implements Closeable, TwoPhaseCommit, Accountable,
|
|||
// we are trying to sync all referenced files, a
|
||||
// merge completes which would otherwise have
|
||||
// removed the files we are now syncing.
|
||||
filesToCommit = toCommit.files(false);
|
||||
deleter.incRef(filesToCommit);
|
||||
deleter.incRef(toCommit.files(false));
|
||||
}
|
||||
success = true;
|
||||
} finally {
|
||||
|
@ -3255,7 +3302,16 @@ public class IndexWriter implements Closeable, TwoPhaseCommit, Accountable,
|
|||
} finally {
|
||||
maybeCloseOnTragicEvent();
|
||||
}
|
||||
|
||||
|
||||
if (onCommitMerges != null) {
|
||||
mergeScheduler.merge(mergeSource, MergeTrigger.COMMIT);
|
||||
onCommitMerges.await(maxCommitMergeWaitSeconds, TimeUnit.SECONDS);
|
||||
synchronized (this) {
|
||||
// we need to call this under lock since mergeFinished above is also called under the IW lock
|
||||
includeInCommit.set(false);
|
||||
}
|
||||
}
|
||||
filesToCommit = toCommit.files(false);
|
||||
try {
|
||||
if (anyChanges) {
|
||||
maybeMerge.set(true);
|
||||
|
|
|
@ -19,6 +19,7 @@ package org.apache.lucene.index;
|
|||
|
||||
import java.io.PrintStream;
|
||||
import java.util.Arrays;
|
||||
import java.util.EnumSet;
|
||||
import java.util.stream.Collectors;
|
||||
|
||||
import org.apache.lucene.analysis.Analyzer;
|
||||
|
@ -109,6 +110,9 @@ public final class IndexWriterConfig extends LiveIndexWriterConfig {
|
|||
|
||||
/** Default value for whether calls to {@link IndexWriter#close()} include a commit. */
|
||||
public final static boolean DEFAULT_COMMIT_ON_CLOSE = true;
|
||||
|
||||
/** Default value for time to wait for merges on commit (when using a {@link MergePolicy} that implements findFullFlushMerges). */
|
||||
public static final long DEFAULT_MAX_COMMIT_MERGE_WAIT_SECONDS = 0;
|
||||
|
||||
// indicates whether this config instance is already attached to a writer.
|
||||
// not final so that it can be cloned properly.
|
||||
|
@ -459,6 +463,27 @@ public final class IndexWriterConfig extends LiveIndexWriterConfig {
|
|||
return this;
|
||||
}
|
||||
|
||||
/**
|
||||
* Expert: sets the amount of time to wait for merges returned by MergePolicy.findFullFlushMerges(...).
|
||||
* If this time is reached, we proceed with the commit based on segments merged up to that point.
|
||||
* The merges are not cancelled, and will still run to completion independent of the commit
|
||||
* like normal segment merges. The default is <code>{@value IndexWriterConfig#DEFAULT_MAX_COMMIT_MERGE_WAIT_SECONDS}</code>.
|
||||
*
|
||||
* Note: This settings has no effect unless {@link MergePolicy#findFullFlushMerges(MergeTrigger, SegmentInfos, MergePolicy.MergeContext)}
|
||||
* has an implementation that actually returns merges which by default doesn't return any merges.
|
||||
*/
|
||||
public IndexWriterConfig setMaxCommitMergeWaitSeconds(long maxCommitMergeWaitSeconds) {
|
||||
this.maxCommitMergeWaitSeconds = maxCommitMergeWaitSeconds;
|
||||
return this;
|
||||
}
|
||||
|
||||
/** We only allow sorting on these types */
|
||||
private static final EnumSet<SortField.Type> ALLOWED_INDEX_SORT_TYPES = EnumSet.of(SortField.Type.STRING,
|
||||
SortField.Type.LONG,
|
||||
SortField.Type.INT,
|
||||
SortField.Type.DOUBLE,
|
||||
SortField.Type.FLOAT);
|
||||
|
||||
/**
|
||||
* Set the {@link Sort} order to use for all (flushed and merged) segments.
|
||||
*/
|
||||
|
|
|
@ -109,6 +109,8 @@ public class LiveIndexWriterConfig {
|
|||
/** soft deletes field */
|
||||
protected String softDeletesField = null;
|
||||
|
||||
/** Amount of time to wait for merges returned by MergePolicy.findFullFlushMerges(...) */
|
||||
protected volatile long maxCommitMergeWaitSeconds;
|
||||
|
||||
// used by IndexWriterConfig
|
||||
LiveIndexWriterConfig(Analyzer analyzer) {
|
||||
|
@ -132,6 +134,7 @@ public class LiveIndexWriterConfig {
|
|||
flushPolicy = new FlushByRamOrCountsPolicy();
|
||||
readerPooling = IndexWriterConfig.DEFAULT_READER_POOLING;
|
||||
perThreadHardLimitMB = IndexWriterConfig.DEFAULT_RAM_PER_THREAD_HARD_LIMIT_MB;
|
||||
maxCommitMergeWaitSeconds = IndexWriterConfig.DEFAULT_MAX_COMMIT_MERGE_WAIT_SECONDS;
|
||||
}
|
||||
|
||||
/** Returns the default analyzer to use for indexing documents. */
|
||||
|
@ -461,6 +464,15 @@ public class LiveIndexWriterConfig {
|
|||
return softDeletesField;
|
||||
}
|
||||
|
||||
/**
|
||||
* Expert: return the amount of time to wait for merges returned by by MergePolicy.findFullFlushMerges(...).
|
||||
* If this time is reached, we proceed with the commit based on segments merged up to that point.
|
||||
* The merges are not cancelled, and may still run to completion independent of the commit.
|
||||
*/
|
||||
public long getMaxCommitMergeWaitSeconds() {
|
||||
return maxCommitMergeWaitSeconds;
|
||||
}
|
||||
|
||||
@Override
|
||||
public String toString() {
|
||||
StringBuilder sb = new StringBuilder();
|
||||
|
@ -484,6 +496,7 @@ public class LiveIndexWriterConfig {
|
|||
sb.append("indexSort=").append(getIndexSort()).append("\n");
|
||||
sb.append("checkPendingFlushOnUpdate=").append(isCheckPendingFlushOnUpdate()).append("\n");
|
||||
sb.append("softDeletesField=").append(getSoftDeletesField()).append("\n");
|
||||
sb.append("maxCommitMergeWaitSeconds=").append(getMaxCommitMergeWaitSeconds()).append("\n");
|
||||
return sb.toString();
|
||||
}
|
||||
}
|
||||
|
|
|
@ -555,7 +555,7 @@ public abstract class MergePolicy {
|
|||
* an original segment present in the
|
||||
* to-be-merged index; else, it was a segment
|
||||
* produced by a cascaded merge.
|
||||
* @param mergeContext the IndexWriter to find the merges on
|
||||
* @param mergeContext the MergeContext to find the merges on
|
||||
*/
|
||||
public abstract MergeSpecification findForcedMerges(
|
||||
SegmentInfos segmentInfos, int maxSegmentCount, Map<SegmentCommitInfo,Boolean> segmentsToMerge, MergeContext mergeContext)
|
||||
|
@ -566,11 +566,33 @@ public abstract class MergePolicy {
|
|||
* deletes from the index.
|
||||
* @param segmentInfos
|
||||
* the total set of segments in the index
|
||||
* @param mergeContext the IndexWriter to find the merges on
|
||||
* @param mergeContext the MergeContext to find the merges on
|
||||
*/
|
||||
public abstract MergeSpecification findForcedDeletesMerges(
|
||||
SegmentInfos segmentInfos, MergeContext mergeContext) throws IOException;
|
||||
|
||||
/**
|
||||
* Identifies merges that we want to execute (synchronously) on commit. By default, do not synchronously merge on commit.
|
||||
*
|
||||
* Any merges returned here will make {@link IndexWriter#commit()} or {@link IndexWriter#prepareCommit()} block until
|
||||
* the merges complete or until {@link IndexWriterConfig#getMaxCommitMergeWaitSeconds()} have elapsed. This may be
|
||||
* used to merge small segments that have just been flushed as part of the commit, reducing the number of segments in
|
||||
* the commit. If a merge does not complete in the allotted time, it will continue to execute, but will not be reflected
|
||||
* in the commit.
|
||||
*
|
||||
* If a {@link OneMerge} in the returned {@link MergeSpecification} includes a segment already included in a registered
|
||||
* merge, then {@link IndexWriter#commit()} or {@link IndexWriter#prepareCommit()} will throw a {@link IllegalStateException}.
|
||||
* Use {@link MergeContext#getMergingSegments()} to determine which segments are currently registered to merge.
|
||||
*
|
||||
* @param mergeTrigger the event that triggered the merge (COMMIT or FULL_FLUSH).
|
||||
* @param segmentInfos the total set of segments in the index (while preparing the commit)
|
||||
* @param mergeContext the MergeContext to find the merges on, which should be used to determine which segments are
|
||||
* already in a registered merge (see {@link MergeContext#getMergingSegments()}).
|
||||
*/
|
||||
public MergeSpecification findFullFlushMerges(MergeTrigger mergeTrigger, SegmentInfos segmentInfos, MergeContext mergeContext) throws IOException {
|
||||
return null;
|
||||
}
|
||||
|
||||
/**
|
||||
* Returns true if a new segment (regardless of its origin) should use the
|
||||
* compound file format. The default implementation returns <code>true</code>
|
||||
|
|
|
@ -47,5 +47,10 @@ public enum MergeTrigger {
|
|||
/**
|
||||
* Merge was triggered by a closing IndexWriter.
|
||||
*/
|
||||
CLOSING
|
||||
CLOSING,
|
||||
|
||||
/**
|
||||
* Merge was triggered on commit.
|
||||
*/
|
||||
COMMIT,
|
||||
}
|
||||
|
|
|
@ -45,6 +45,9 @@ public final class NoMergePolicy extends MergePolicy {
|
|||
@Override
|
||||
public MergeSpecification findForcedDeletesMerges(SegmentInfos segmentInfos, MergeContext mergeContext) { return null; }
|
||||
|
||||
@Override
|
||||
public MergeSpecification findFullFlushMerges(MergeTrigger mergeTrigger, SegmentInfos segmentInfos, MergeContext mergeContext) { return null; }
|
||||
|
||||
@Override
|
||||
public boolean useCompoundFile(SegmentInfos segments, SegmentCommitInfo newSegment, MergeContext mergeContext) {
|
||||
return newSegment.info.getUseCompoundFile();
|
||||
|
|
|
@ -59,6 +59,11 @@ public class OneMergeWrappingMergePolicy extends FilterMergePolicy {
|
|||
return wrapSpec(in.findForcedDeletesMerges(segmentInfos, mergeContext));
|
||||
}
|
||||
|
||||
@Override
|
||||
public MergeSpecification findFullFlushMerges(MergeTrigger mergeTrigger, SegmentInfos segmentInfos, MergeContext mergeContext) throws IOException {
|
||||
return wrapSpec(in.findFullFlushMerges(mergeTrigger, segmentInfos, mergeContext));
|
||||
}
|
||||
|
||||
private MergeSpecification wrapSpec(MergeSpecification spec) {
|
||||
MergeSpecification wrapped = spec == null ? null : new MergeSpecification();
|
||||
if (wrapped != null) {
|
||||
|
|
|
@ -344,7 +344,7 @@ public class TestIndexWriter extends LuceneTestCase {
|
|||
// Make sure it's OK to change RAM buffer size and
|
||||
// maxBufferedDocs in a write session
|
||||
public void testChangingRAMBuffer() throws IOException {
|
||||
Directory dir = newDirectory();
|
||||
Directory dir = newDirectory();
|
||||
IndexWriter writer = new IndexWriter(dir, newIndexWriterConfig(new MockAnalyzer(random())));
|
||||
writer.getConfig().setMaxBufferedDocs(10);
|
||||
writer.getConfig().setRAMBufferSizeMB(IndexWriterConfig.DISABLE_AUTO_FLUSH);
|
||||
|
@ -607,7 +607,7 @@ public class TestIndexWriter extends LuceneTestCase {
|
|||
doc.add(newField("content4", contents, customType));
|
||||
type = customType;
|
||||
} else
|
||||
type = TextField.TYPE_NOT_STORED;
|
||||
type = TextField.TYPE_NOT_STORED;
|
||||
doc.add(newTextField("content1", contents, Field.Store.NO));
|
||||
doc.add(newField("content3", "", customType));
|
||||
doc.add(newField("content5", "", type));
|
||||
|
@ -663,13 +663,13 @@ public class TestIndexWriter extends LuceneTestCase {
|
|||
writer.close();
|
||||
dir.close();
|
||||
}
|
||||
|
||||
|
||||
public void testEmptyFieldNameTerms() throws IOException {
|
||||
Directory dir = newDirectory();
|
||||
IndexWriter writer = new IndexWriter(dir, newIndexWriterConfig(new MockAnalyzer(random())));
|
||||
Document doc = new Document();
|
||||
doc.add(newTextField("", "a b c", Field.Store.NO));
|
||||
writer.addDocument(doc);
|
||||
writer.addDocument(doc);
|
||||
writer.close();
|
||||
DirectoryReader reader = DirectoryReader.open(dir);
|
||||
LeafReader subreader = getOnlyLeafReader(reader);
|
||||
|
@ -681,7 +681,7 @@ public class TestIndexWriter extends LuceneTestCase {
|
|||
reader.close();
|
||||
dir.close();
|
||||
}
|
||||
|
||||
|
||||
public void testEmptyFieldNameWithEmptyTerm() throws IOException {
|
||||
Directory dir = newDirectory();
|
||||
IndexWriter writer = new IndexWriter(dir, newIndexWriterConfig(new MockAnalyzer(random())));
|
||||
|
@ -690,7 +690,7 @@ public class TestIndexWriter extends LuceneTestCase {
|
|||
doc.add(newStringField("", "a", Field.Store.NO));
|
||||
doc.add(newStringField("", "b", Field.Store.NO));
|
||||
doc.add(newStringField("", "c", Field.Store.NO));
|
||||
writer.addDocument(doc);
|
||||
writer.addDocument(doc);
|
||||
writer.close();
|
||||
DirectoryReader reader = DirectoryReader.open(dir);
|
||||
LeafReader subreader = getOnlyLeafReader(reader);
|
||||
|
@ -834,7 +834,7 @@ public class TestIndexWriter extends LuceneTestCase {
|
|||
customType.setStoreTermVectors(true);
|
||||
customType.setStoreTermVectorPositions(true);
|
||||
customType.setStoreTermVectorOffsets(true);
|
||||
|
||||
|
||||
doc.add(newField("content", "aaa bbb ccc ddd eee fff ggg hhh iii", customType));
|
||||
writer.addDocument(doc);
|
||||
writer.addDocument(doc);
|
||||
|
@ -922,7 +922,7 @@ public class TestIndexWriter extends LuceneTestCase {
|
|||
|
||||
// open/close slowly sometimes
|
||||
dir.setUseSlowOpenClosers(true);
|
||||
|
||||
|
||||
// throttle a little
|
||||
dir.setThrottling(MockDirectoryWrapper.Throttling.SOMETIMES);
|
||||
|
||||
|
@ -1148,7 +1148,7 @@ public class TestIndexWriter extends LuceneTestCase {
|
|||
|
||||
FieldType customType = new FieldType(StoredField.TYPE);
|
||||
customType.setTokenized(true);
|
||||
|
||||
|
||||
Field f = new Field("binary", b, 10, 17, customType);
|
||||
// TODO: this is evil, changing the type after creating the field:
|
||||
customType.setIndexOptions(IndexOptions.DOCS);
|
||||
|
@ -1157,7 +1157,7 @@ public class TestIndexWriter extends LuceneTestCase {
|
|||
f.setTokenStream(doc1field1);
|
||||
|
||||
FieldType customType2 = new FieldType(TextField.TYPE_STORED);
|
||||
|
||||
|
||||
Field f2 = newField("string", "value", customType2);
|
||||
final MockTokenizer doc1field2 = new MockTokenizer(MockTokenizer.WHITESPACE, false);
|
||||
doc1field2.setReader(new StringReader("doc1field2"));
|
||||
|
@ -1233,7 +1233,7 @@ public class TestIndexWriter extends LuceneTestCase {
|
|||
public void testDeleteUnusedFiles() throws Exception {
|
||||
assumeFalse("test relies on exact filenames", Codec.getDefault() instanceof SimpleTextCodec);
|
||||
assumeWorkingMMapOnWindows();
|
||||
|
||||
|
||||
for(int iter=0;iter<2;iter++) {
|
||||
// relies on windows semantics
|
||||
Path path = createTempDir();
|
||||
|
@ -1250,7 +1250,7 @@ public class TestIndexWriter extends LuceneTestCase {
|
|||
}
|
||||
|
||||
MergePolicy mergePolicy = newLogMergePolicy(true);
|
||||
|
||||
|
||||
// This test expects all of its segments to be in CFS
|
||||
mergePolicy.setNoCFSRatio(1.0);
|
||||
mergePolicy.setMaxCFSSegmentSizeMB(Double.POSITIVE_INFINITY);
|
||||
|
@ -1338,7 +1338,7 @@ public class TestIndexWriter extends LuceneTestCase {
|
|||
customType.setStoreTermVectors(true);
|
||||
customType.setStoreTermVectorPositions(true);
|
||||
customType.setStoreTermVectorOffsets(true);
|
||||
|
||||
|
||||
doc.add(newField("c", "val", customType));
|
||||
writer.addDocument(doc);
|
||||
writer.commit();
|
||||
|
@ -1379,7 +1379,7 @@ public class TestIndexWriter extends LuceneTestCase {
|
|||
// indexed, flushed (but not committed) and then IW rolls back, then no
|
||||
// files are left in the Directory.
|
||||
Directory dir = newDirectory();
|
||||
|
||||
|
||||
String[] origFiles = dir.listAll();
|
||||
IndexWriter writer = new IndexWriter(dir, newIndexWriterConfig(new MockAnalyzer(random()))
|
||||
.setMaxBufferedDocs(2)
|
||||
|
@ -1409,8 +1409,8 @@ public class TestIndexWriter extends LuceneTestCase {
|
|||
// Adding just one document does not call flush yet.
|
||||
int computedExtraFileCount = 0;
|
||||
for (String file : dir.listAll()) {
|
||||
if (IndexWriter.WRITE_LOCK_NAME.equals(file) ||
|
||||
file.startsWith(IndexFileNames.SEGMENTS) ||
|
||||
if (IndexWriter.WRITE_LOCK_NAME.equals(file) ||
|
||||
file.startsWith(IndexFileNames.SEGMENTS) ||
|
||||
IndexFileNames.CODEC_FILE_PATTERN.matcher(file).matches()) {
|
||||
if (file.lastIndexOf('.') < 0
|
||||
// don't count stored fields and term vectors in, or any temporary files they might
|
||||
|
@ -1458,7 +1458,7 @@ public class TestIndexWriter extends LuceneTestCase {
|
|||
FieldType customType3 = new FieldType(TextField.TYPE_STORED);
|
||||
customType3.setTokenized(false);
|
||||
customType3.setOmitNorms(true);
|
||||
|
||||
|
||||
for (int i=0; i<2; i++) {
|
||||
Document doc = new Document();
|
||||
doc.add(new Field("id", Integer.toString(i)+BIG, customType3));
|
||||
|
@ -1478,7 +1478,7 @@ public class TestIndexWriter extends LuceneTestCase {
|
|||
SegmentReader sr = (SegmentReader) ctx.reader();
|
||||
assertFalse(sr.getFieldInfos().hasVectors());
|
||||
}
|
||||
|
||||
|
||||
r0.close();
|
||||
dir.close();
|
||||
}
|
||||
|
@ -1501,7 +1501,7 @@ public class TestIndexWriter extends LuceneTestCase {
|
|||
|
||||
@Override
|
||||
public final boolean incrementToken() {
|
||||
clearAttributes();
|
||||
clearAttributes();
|
||||
if (upto < tokens.length) {
|
||||
termAtt.setEmpty();
|
||||
termAtt.append(tokens[upto]);
|
||||
|
@ -1724,7 +1724,7 @@ public class TestIndexWriter extends LuceneTestCase {
|
|||
r.close();
|
||||
dir.close();
|
||||
}
|
||||
|
||||
|
||||
public void testDontInvokeAnalyzerForUnAnalyzedFields() throws Exception {
|
||||
Analyzer analyzer = new Analyzer() {
|
||||
@Override
|
||||
|
@ -1759,13 +1759,13 @@ public class TestIndexWriter extends LuceneTestCase {
|
|||
w.close();
|
||||
dir.close();
|
||||
}
|
||||
|
||||
|
||||
//LUCENE-1468 -- make sure opening an IndexWriter with
|
||||
// create=true does not remove non-index files
|
||||
|
||||
|
||||
public void testOtherFiles() throws Throwable {
|
||||
Directory dir = newDirectory();
|
||||
IndexWriter iw = new IndexWriter(dir,
|
||||
IndexWriter iw = new IndexWriter(dir,
|
||||
newIndexWriterConfig(new MockAnalyzer(random())));
|
||||
iw.addDocument(new Document());
|
||||
iw.close();
|
||||
|
@ -1774,15 +1774,15 @@ public class TestIndexWriter extends LuceneTestCase {
|
|||
IndexOutput out = dir.createOutput("myrandomfile", newIOContext(random()));
|
||||
out.writeByte((byte) 42);
|
||||
out.close();
|
||||
|
||||
|
||||
new IndexWriter(dir, newIndexWriterConfig(new MockAnalyzer(random()))).close();
|
||||
|
||||
|
||||
assertTrue(slowFileExists(dir, "myrandomfile"));
|
||||
} finally {
|
||||
dir.close();
|
||||
}
|
||||
}
|
||||
|
||||
|
||||
// LUCENE-3849
|
||||
public void testStopwordsPosIncHole() throws Exception {
|
||||
Directory dir = newDirectory();
|
||||
|
@ -1811,7 +1811,7 @@ public class TestIndexWriter extends LuceneTestCase {
|
|||
ir.close();
|
||||
dir.close();
|
||||
}
|
||||
|
||||
|
||||
// LUCENE-3849
|
||||
public void testStopwordsPosIncHole2() throws Exception {
|
||||
// use two stopfilters for testing here
|
||||
|
@ -1843,23 +1843,23 @@ public class TestIndexWriter extends LuceneTestCase {
|
|||
ir.close();
|
||||
dir.close();
|
||||
}
|
||||
|
||||
|
||||
// LUCENE-4575
|
||||
public void testCommitWithUserDataOnly() throws Exception {
|
||||
Directory dir = newDirectory();
|
||||
IndexWriter writer = new IndexWriter(dir, newIndexWriterConfig(null));
|
||||
writer.commit(); // first commit to complete IW create transaction.
|
||||
|
||||
|
||||
// this should store the commit data, even though no other changes were made
|
||||
writer.setLiveCommitData(new HashMap<String,String>() {{
|
||||
put("key", "value");
|
||||
}}.entrySet());
|
||||
writer.commit();
|
||||
|
||||
|
||||
DirectoryReader r = DirectoryReader.open(dir);
|
||||
assertEquals("value", r.getIndexCommit().getUserData().get("key"));
|
||||
r.close();
|
||||
|
||||
|
||||
// now check setCommitData and prepareCommit/commit sequence
|
||||
writer.setLiveCommitData(new HashMap<String,String>() {{
|
||||
put("key", "value1");
|
||||
|
@ -1873,7 +1873,7 @@ public class TestIndexWriter extends LuceneTestCase {
|
|||
r = DirectoryReader.open(dir);
|
||||
assertEquals("value1", r.getIndexCommit().getUserData().get("key"));
|
||||
r.close();
|
||||
|
||||
|
||||
// now should commit the second commitData - there was a bug where
|
||||
// IndexWriter.finishCommit overrode the second commitData
|
||||
writer.commit();
|
||||
|
@ -1881,7 +1881,7 @@ public class TestIndexWriter extends LuceneTestCase {
|
|||
assertEquals("IndexWriter.finishCommit may have overridden the second commitData",
|
||||
"value2", r.getIndexCommit().getUserData().get("key"));
|
||||
r.close();
|
||||
|
||||
|
||||
writer.close();
|
||||
dir.close();
|
||||
}
|
||||
|
@ -1896,7 +1896,7 @@ public class TestIndexWriter extends LuceneTestCase {
|
|||
}
|
||||
return data;
|
||||
}
|
||||
|
||||
|
||||
@Test
|
||||
public void testGetCommitData() throws Exception {
|
||||
Directory dir = newDirectory();
|
||||
|
@ -1906,16 +1906,16 @@ public class TestIndexWriter extends LuceneTestCase {
|
|||
}}.entrySet());
|
||||
assertEquals("value", getLiveCommitData(writer).get("key"));
|
||||
writer.close();
|
||||
|
||||
|
||||
// validate that it's also visible when opening a new IndexWriter
|
||||
writer = new IndexWriter(dir, newIndexWriterConfig(null)
|
||||
.setOpenMode(OpenMode.APPEND));
|
||||
assertEquals("value", getLiveCommitData(writer).get("key"));
|
||||
writer.close();
|
||||
|
||||
|
||||
dir.close();
|
||||
}
|
||||
|
||||
|
||||
public void testNullAnalyzer() throws IOException {
|
||||
Directory dir = newDirectory();
|
||||
IndexWriterConfig iwConf = newIndexWriterConfig(null);
|
||||
|
@ -1942,7 +1942,7 @@ public class TestIndexWriter extends LuceneTestCase {
|
|||
iw.close();
|
||||
dir.close();
|
||||
}
|
||||
|
||||
|
||||
public void testNullDocument() throws IOException {
|
||||
Directory dir = newDirectory();
|
||||
RandomIndexWriter iw = new RandomIndexWriter(random(), dir);
|
||||
|
@ -1967,7 +1967,7 @@ public class TestIndexWriter extends LuceneTestCase {
|
|||
iw.close();
|
||||
dir.close();
|
||||
}
|
||||
|
||||
|
||||
public void testNullDocuments() throws IOException {
|
||||
Directory dir = newDirectory();
|
||||
RandomIndexWriter iw = new RandomIndexWriter(random(), dir);
|
||||
|
@ -1992,7 +1992,7 @@ public class TestIndexWriter extends LuceneTestCase {
|
|||
iw.close();
|
||||
dir.close();
|
||||
}
|
||||
|
||||
|
||||
public void testIterableFieldThrowsException() throws IOException {
|
||||
Directory dir = newDirectory();
|
||||
IndexWriter w = new IndexWriter(dir, newIndexWriterConfig(new MockAnalyzer(random())));
|
||||
|
@ -2000,7 +2000,7 @@ public class TestIndexWriter extends LuceneTestCase {
|
|||
int docCount = 0;
|
||||
int docId = 0;
|
||||
Set<String> liveIds = new HashSet<>();
|
||||
for (int i = 0; i < iters; i++) {
|
||||
for (int i = 0; i < iters; i++) {
|
||||
int numDocs = atLeast(4);
|
||||
for (int j = 0; j < numDocs; j++) {
|
||||
String id = Integer.toString(docId++);
|
||||
|
@ -2008,7 +2008,7 @@ public class TestIndexWriter extends LuceneTestCase {
|
|||
fields.add(new StringField("id", id, Field.Store.YES));
|
||||
fields.add(new StringField("foo", TestUtil.randomSimpleString(random()), Field.Store.NO));
|
||||
docId++;
|
||||
|
||||
|
||||
boolean success = false;
|
||||
try {
|
||||
w.addDocument(new RandomFailingIterable<IndexableField>(fields, random()));
|
||||
|
@ -2040,7 +2040,7 @@ public class TestIndexWriter extends LuceneTestCase {
|
|||
w.close();
|
||||
IOUtils.close(reader, dir);
|
||||
}
|
||||
|
||||
|
||||
public void testIterableThrowsException() throws IOException {
|
||||
Directory dir = newDirectory();
|
||||
IndexWriter w = new IndexWriter(dir, newIndexWriterConfig(new MockAnalyzer(random())));
|
||||
|
@ -2088,7 +2088,7 @@ public class TestIndexWriter extends LuceneTestCase {
|
|||
w.close();
|
||||
IOUtils.close(reader, dir);
|
||||
}
|
||||
|
||||
|
||||
public void testIterableThrowsException2() throws IOException {
|
||||
Directory dir = newDirectory();
|
||||
IndexWriter w = new IndexWriter(dir, newIndexWriterConfig(new MockAnalyzer(random())));
|
||||
|
@ -2128,7 +2128,7 @@ public class TestIndexWriter extends LuceneTestCase {
|
|||
this.list = list;
|
||||
this.failOn = random.nextInt(5);
|
||||
}
|
||||
|
||||
|
||||
@Override
|
||||
public Iterator<T> iterator() {
|
||||
final Iterator<? extends T> docIter = list.iterator();
|
||||
|
@ -2254,7 +2254,7 @@ public class TestIndexWriter extends LuceneTestCase {
|
|||
writer.close();
|
||||
dir.close();
|
||||
}
|
||||
|
||||
|
||||
public void testMergeAllDeleted() throws IOException {
|
||||
Directory dir = newDirectory();
|
||||
IndexWriterConfig iwc = newIndexWriterConfig(new MockAnalyzer(random()));
|
||||
|
@ -2477,12 +2477,12 @@ public class TestIndexWriter extends LuceneTestCase {
|
|||
IndexWriter w = new IndexWriter(d, newIndexWriterConfig(new MockAnalyzer(random())));
|
||||
w.addDocument(new Document());
|
||||
w.close();
|
||||
|
||||
|
||||
SegmentInfos sis = SegmentInfos.readLatestCommit(d);
|
||||
byte[] id1 = sis.getId();
|
||||
assertNotNull(id1);
|
||||
assertEquals(StringHelper.ID_LENGTH, id1.length);
|
||||
|
||||
|
||||
byte[] id2 = sis.info(0).info.getId();
|
||||
byte[] sciId2 = sis.info(0).getId();
|
||||
assertNotNull(id2);
|
||||
|
@ -2514,7 +2514,7 @@ public class TestIndexWriter extends LuceneTestCase {
|
|||
ids.add(id);
|
||||
}
|
||||
}
|
||||
|
||||
|
||||
public void testEmptyNorm() throws Exception {
|
||||
Directory d = newDirectory();
|
||||
IndexWriter w = new IndexWriter(d, newIndexWriterConfig(new MockAnalyzer(random())));
|
||||
|
@ -2579,7 +2579,7 @@ public class TestIndexWriter extends LuceneTestCase {
|
|||
assertEquals(1, r2.getIndexCommit().getGeneration());
|
||||
assertEquals("segments_1", r2.getIndexCommit().getSegmentsFileName());
|
||||
r2.close();
|
||||
|
||||
|
||||
// make a change and another commit
|
||||
w.addDocument(new Document());
|
||||
w.commit();
|
||||
|
@ -2866,7 +2866,7 @@ public class TestIndexWriter extends LuceneTestCase {
|
|||
IndexWriterConfig iwc = new IndexWriterConfig(new MockAnalyzer(random()));
|
||||
IndexWriter w = new IndexWriter(dir, iwc);
|
||||
w.close();
|
||||
|
||||
|
||||
IndexOutput out = dir.createTempOutput("_0", "bkd", IOContext.DEFAULT);
|
||||
String tempName = out.getName();
|
||||
out.close();
|
||||
|
@ -3151,7 +3151,7 @@ public class TestIndexWriter extends LuceneTestCase {
|
|||
expectThrows(IllegalArgumentException.class, () -> {
|
||||
writer.softUpdateDocument(null, new Document(), new NumericDocValuesField("soft_delete", 1));
|
||||
});
|
||||
|
||||
|
||||
expectThrows(IllegalArgumentException.class, () -> {
|
||||
writer.softUpdateDocument(new Term("id", "1"), new Document());
|
||||
});
|
||||
|
@ -4186,19 +4186,57 @@ public class TestIndexWriter extends LuceneTestCase {
|
|||
}
|
||||
};
|
||||
})))) {
|
||||
Document doc = new Document();
|
||||
doc.add(new StringField("id", "1", Field.Store.NO));
|
||||
writer.addDocument(doc);
|
||||
writer.flush();
|
||||
writer.addDocument(doc);
|
||||
writer.flush();
|
||||
writer.deleteDocuments(new Term("id", "1"));
|
||||
writer.flush();
|
||||
assertEquals(2, writer.getSegmentCount());
|
||||
assertEquals(0, writer.getDocStats().numDocs);
|
||||
assertEquals(2, writer.getDocStats().maxDoc);
|
||||
writer.forceMerge(1);
|
||||
Document doc = new Document();
|
||||
doc.add(new StringField("id", "1", Field.Store.NO));
|
||||
writer.addDocument(doc);
|
||||
writer.flush();
|
||||
writer.addDocument(doc);
|
||||
writer.flush();
|
||||
writer.deleteDocuments(new Term("id", "1"));
|
||||
writer.flush();
|
||||
assertEquals(2, writer.getSegmentCount());
|
||||
assertEquals(0, writer.getDocStats().numDocs);
|
||||
assertEquals(2, writer.getDocStats().maxDoc);
|
||||
writer.forceMerge(1);
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
public void testMergeOnCommitKeepFullyDeletedSegments() throws Exception {
|
||||
Directory dir = newDirectory();
|
||||
IndexWriterConfig iwc = newIndexWriterConfig();
|
||||
iwc.setMaxCommitMergeWaitSeconds(30);
|
||||
iwc.mergePolicy = new FilterMergePolicy(newMergePolicy()) {
|
||||
@Override
|
||||
public boolean keepFullyDeletedSegment(IOSupplier<CodecReader> readerIOSupplier) {
|
||||
return true;
|
||||
}
|
||||
|
||||
@Override
|
||||
public MergeSpecification findFullFlushMerges(MergeTrigger mergeTrigger,
|
||||
SegmentInfos segmentInfos,
|
||||
MergeContext mergeContext) {
|
||||
List<SegmentCommitInfo> fullyDeletedSegments = segmentInfos.asList().stream()
|
||||
.filter(s -> s.info.maxDoc() - s.getDelCount() == 0)
|
||||
.collect(Collectors.toList());
|
||||
if (fullyDeletedSegments.isEmpty()) {
|
||||
return null;
|
||||
}
|
||||
MergeSpecification spec = new MergeSpecification();
|
||||
spec.add(new OneMerge(fullyDeletedSegments));
|
||||
return spec;
|
||||
}
|
||||
};
|
||||
IndexWriter w = new IndexWriter(dir, iwc);
|
||||
Document d = new Document();
|
||||
d.add(new StringField("id", "1", Field.Store.YES));
|
||||
w.addDocument(d);
|
||||
w.commit();
|
||||
w.updateDocument(new Term("id", "1"), d);
|
||||
w.commit();
|
||||
try (DirectoryReader reader = w.getReader()) {
|
||||
assertEquals(1, reader.numDocs());
|
||||
}
|
||||
IOUtils.close(w, dir);
|
||||
}
|
||||
}
|
||||
|
|
|
@ -18,17 +18,42 @@ package org.apache.lucene.index;
|
|||
|
||||
|
||||
import java.io.IOException;
|
||||
import java.util.ArrayList;
|
||||
import java.util.List;
|
||||
|
||||
import org.apache.lucene.analysis.MockAnalyzer;
|
||||
import org.apache.lucene.document.Document;
|
||||
import org.apache.lucene.document.Field;
|
||||
import org.apache.lucene.index.IndexWriterConfig.OpenMode;
|
||||
import org.apache.lucene.search.IndexSearcher;
|
||||
import org.apache.lucene.search.MatchAllDocsQuery;
|
||||
import org.apache.lucene.store.Directory;
|
||||
|
||||
import org.apache.lucene.util.LuceneTestCase;
|
||||
|
||||
public class TestIndexWriterMergePolicy extends LuceneTestCase {
|
||||
|
||||
|
||||
private static final MergePolicy MERGE_ON_COMMIT_POLICY = new LogDocMergePolicy() {
|
||||
@Override
|
||||
public MergeSpecification findFullFlushMerges(MergeTrigger mergeTrigger, SegmentInfos segmentInfos, MergeContext mergeContext) {
|
||||
// Optimize down to a single segment on commit
|
||||
if (mergeTrigger == MergeTrigger.COMMIT && segmentInfos.size() > 1) {
|
||||
List<SegmentCommitInfo> nonMergingSegments = new ArrayList<>();
|
||||
for (SegmentCommitInfo sci : segmentInfos) {
|
||||
if (mergeContext.getMergingSegments().contains(sci) == false) {
|
||||
nonMergingSegments.add(sci);
|
||||
}
|
||||
}
|
||||
if (nonMergingSegments.size() > 1) {
|
||||
MergeSpecification mergeSpecification = new MergeSpecification();
|
||||
mergeSpecification.add(new OneMerge(nonMergingSegments));
|
||||
return mergeSpecification;
|
||||
}
|
||||
}
|
||||
return null;
|
||||
}
|
||||
};
|
||||
|
||||
// Test the normal case
|
||||
public void testNormalCase() throws IOException {
|
||||
Directory dir = newDirectory();
|
||||
|
@ -278,6 +303,50 @@ public class TestIndexWriterMergePolicy extends LuceneTestCase {
|
|||
assertSetters(new LogDocMergePolicy());
|
||||
}
|
||||
|
||||
// Test basic semantics of merge on commit
|
||||
public void testMergeOnCommit() throws IOException {
|
||||
Directory dir = newDirectory();
|
||||
|
||||
IndexWriter firstWriter = new IndexWriter(dir, newIndexWriterConfig(new MockAnalyzer(random()))
|
||||
.setMergePolicy(NoMergePolicy.INSTANCE));
|
||||
for (int i = 0; i < 5; i++) {
|
||||
TestIndexWriter.addDoc(firstWriter);
|
||||
firstWriter.flush();
|
||||
}
|
||||
DirectoryReader firstReader = DirectoryReader.open(firstWriter);
|
||||
assertEquals(5, firstReader.leaves().size());
|
||||
firstReader.close();
|
||||
firstWriter.close(); // When this writer closes, it does not merge on commit.
|
||||
|
||||
IndexWriterConfig iwc = newIndexWriterConfig(new MockAnalyzer(random()))
|
||||
.setMergePolicy(MERGE_ON_COMMIT_POLICY).setMaxCommitMergeWaitSeconds(30);
|
||||
|
||||
|
||||
IndexWriter writerWithMergePolicy = new IndexWriter(dir, iwc);
|
||||
writerWithMergePolicy.commit(); // No changes. Commit doesn't trigger a merge.
|
||||
|
||||
DirectoryReader unmergedReader = DirectoryReader.open(writerWithMergePolicy);
|
||||
assertEquals(5, unmergedReader.leaves().size());
|
||||
unmergedReader.close();
|
||||
|
||||
TestIndexWriter.addDoc(writerWithMergePolicy);
|
||||
writerWithMergePolicy.commit(); // Doc added, do merge on commit.
|
||||
assertEquals(1, writerWithMergePolicy.getSegmentCount()); //
|
||||
|
||||
DirectoryReader mergedReader = DirectoryReader.open(writerWithMergePolicy);
|
||||
assertEquals(1, mergedReader.leaves().size());
|
||||
mergedReader.close();
|
||||
|
||||
try (IndexReader reader = writerWithMergePolicy.getReader()) {
|
||||
IndexSearcher searcher = new IndexSearcher(reader);
|
||||
assertEquals(6, reader.numDocs());
|
||||
assertEquals(6, searcher.count(new MatchAllDocsQuery()));
|
||||
}
|
||||
|
||||
writerWithMergePolicy.close();
|
||||
dir.close();
|
||||
}
|
||||
|
||||
private void assertSetters(MergePolicy lmp) {
|
||||
lmp.setMaxCFSSegmentSizeMB(2.0);
|
||||
assertEquals(2.0, lmp.getMaxCFSSegmentSizeMB(), EPSILON);
|
||||
|
|
|
@ -128,6 +128,38 @@ public class MockRandomMergePolicy extends MergePolicy {
|
|||
return findMerges(null, segmentInfos, mergeContext);
|
||||
}
|
||||
|
||||
@Override
|
||||
public MergeSpecification findFullFlushMerges(MergeTrigger mergeTrigger, SegmentInfos segmentInfos, MergeContext mergeContext) throws IOException {
|
||||
MergeSpecification mergeSpecification = findMerges(null, segmentInfos, mergeContext);
|
||||
if (mergeSpecification == null) {
|
||||
return null;
|
||||
}
|
||||
// Do not return any merges involving already-merging segments.
|
||||
MergeSpecification filteredMergeSpecification = new MergeSpecification();
|
||||
for (OneMerge oneMerge : mergeSpecification.merges) {
|
||||
boolean filtered = false;
|
||||
List<SegmentCommitInfo> nonMergingSegments = new ArrayList<>();
|
||||
for (SegmentCommitInfo sci : oneMerge.segments) {
|
||||
if (mergeContext.getMergingSegments().contains(sci) == false) {
|
||||
nonMergingSegments.add(sci);
|
||||
} else {
|
||||
filtered = true;
|
||||
}
|
||||
}
|
||||
if (filtered == true) {
|
||||
if (nonMergingSegments.size() > 0) {
|
||||
filteredMergeSpecification.add(new OneMerge(nonMergingSegments));
|
||||
}
|
||||
} else {
|
||||
filteredMergeSpecification.add(oneMerge);
|
||||
}
|
||||
}
|
||||
if (filteredMergeSpecification.merges.size() > 0) {
|
||||
return filteredMergeSpecification;
|
||||
}
|
||||
return null;
|
||||
}
|
||||
|
||||
@Override
|
||||
public boolean useCompoundFile(SegmentInfos infos, SegmentCommitInfo mergedInfo, MergeContext mergeContext) throws IOException {
|
||||
// 80% of the time we create CFS:
|
||||
|
|
|
@ -1003,6 +1003,7 @@ public abstract class LuceneTestCase extends Assert {
|
|||
if (rarely(r)) {
|
||||
c.setCheckPendingFlushUpdate(false);
|
||||
}
|
||||
c.setMaxCommitMergeWaitSeconds(atLeast(r, 1));
|
||||
return c;
|
||||
}
|
||||
|
||||
|
|
Loading…
Reference in New Issue