LUCENE-4472: provide propper context information to MergePolicy which events triggered a findMerges

git-svn-id: https://svn.apache.org/repos/asf/lucene/dev/trunk@1399712 13f79535-47bb-0310-9956-ffa450edef68
This commit is contained in:
Simon Willnauer 2012-10-18 16:03:07 +00:00
parent 3088a66b17
commit 6b739db2ff
11 changed files with 80 additions and 28 deletions

View File

@ -49,6 +49,10 @@ API Changes
* LUCENE-4399: Deprecated AppendingCodec. Lucene's term dictionaries
no longer seek when writing. (Adrien Grand, Robert Muir)
* LUCENE-4472: MergePolicy now accepts a MergeTrigger that provides
information about the trigger of the merge ie. merge triggered due
to a segment merge or a full flush etc. (Simon Willnauer)
Bug Fixes
* LUCENE-1822: BaseFragListBuilder hard-coded 6 char margin is too naive.

View File

@ -36,6 +36,7 @@ import org.apache.lucene.analysis.Analyzer;
import org.apache.lucene.codecs.Codec;
import org.apache.lucene.index.FieldInfos.FieldNumbers;
import org.apache.lucene.index.IndexWriterConfig.OpenMode;
import org.apache.lucene.index.MergePolicy.MergeTrigger;
import org.apache.lucene.index.MergeState.CheckAbort;
import org.apache.lucene.search.Query;
import org.apache.lucene.store.AlreadyClosedException;
@ -181,6 +182,10 @@ import org.apache.lucene.util.ThreadInterruptedException;
* keeps track of the last non commit checkpoint.
*/
public class IndexWriter implements Closeable, TwoPhaseCommit {
private static final int UNBOUNDED_MAX_MERGE_SEGMENTS = -1;
/**
* Name of the write lock in the index.
*/
@ -377,7 +382,7 @@ public class IndexWriter implements Closeable, TwoPhaseCommit {
}
}
if (anySegmentFlushed) {
maybeMerge();
maybeMerge(MergeTrigger.FULL_FLUSH, UNBOUNDED_MAX_MERGE_SEGMENTS);
}
if (infoStream.isEnabled("IW")) {
infoStream.message("IW", "getReader took " + (System.currentTimeMillis() - tStart) + " msec");
@ -1226,7 +1231,7 @@ public class IndexWriter implements Closeable, TwoPhaseCommit {
}
}
if (anySegmentFlushed) {
maybeMerge();
maybeMerge(MergeTrigger.SEGMENT_FLUSH, UNBOUNDED_MAX_MERGE_SEGMENTS);
}
} catch (OutOfMemoryError oom) {
handleOOM(oom, "updateDocuments");
@ -1448,7 +1453,7 @@ public class IndexWriter implements Closeable, TwoPhaseCommit {
}
if (anySegmentFlushed) {
maybeMerge();
maybeMerge(MergeTrigger.SEGMENT_FLUSH, UNBOUNDED_MAX_MERGE_SEGMENTS);
}
} catch (OutOfMemoryError oom) {
handleOOM(oom, "updateDocument");
@ -1621,7 +1626,7 @@ public class IndexWriter implements Closeable, TwoPhaseCommit {
}
}
maybeMerge(maxNumSegments);
maybeMerge(MergeTrigger.EXPLICIT, maxNumSegments);
if (doWait) {
synchronized(this) {
@ -1796,25 +1801,28 @@ public class IndexWriter implements Closeable, TwoPhaseCommit {
* Explicit calls to maybeMerge() are usually not
* necessary. The most common case is when merge policy
* parameters have changed.
*
* This method will call the {@link MergePolicy} with
* {@link MergeTrigger#EXPLICIT}.
*
* <p><b>NOTE</b>: if this method hits an OutOfMemoryError
* you should immediately close the writer. See <a
* href="#OOME">above</a> for details.</p>
*/
public final void maybeMerge() throws IOException {
maybeMerge(-1);
maybeMerge(MergeTrigger.EXPLICIT, UNBOUNDED_MAX_MERGE_SEGMENTS);
}
private final void maybeMerge(int maxNumSegments) throws IOException {
private final void maybeMerge(MergeTrigger trigger, int maxNumSegments) throws IOException {
ensureOpen(false);
updatePendingMerges(maxNumSegments);
updatePendingMerges(trigger, maxNumSegments);
mergeScheduler.merge(this);
}
private synchronized void updatePendingMerges(int maxNumSegments)
private synchronized void updatePendingMerges(MergeTrigger trigger, int maxNumSegments)
throws IOException {
assert maxNumSegments == -1 || maxNumSegments > 0;
assert trigger != null;
if (stopMerges) {
return;
}
@ -1825,7 +1833,9 @@ public class IndexWriter implements Closeable, TwoPhaseCommit {
}
final MergePolicy.MergeSpecification spec;
if (maxNumSegments != -1) {
if (maxNumSegments != UNBOUNDED_MAX_MERGE_SEGMENTS) {
assert trigger == MergeTrigger.EXPLICIT || trigger == MergeTrigger.MERGE_FINISHED :
"Expected EXPLICT or MERGE_FINISHED as trigger even with maxNumSegments set but was: " + trigger.name();
spec = mergePolicy.findForcedMerges(segmentInfos, maxNumSegments, Collections.unmodifiableMap(segmentsToMerge));
if (spec != null) {
final int numMerges = spec.merges.size();
@ -1836,7 +1846,7 @@ public class IndexWriter implements Closeable, TwoPhaseCommit {
}
} else {
spec = mergePolicy.findMerges(segmentInfos);
spec = mergePolicy.findMerges(trigger, segmentInfos);
}
if (spec != null) {
@ -2653,7 +2663,7 @@ public class IndexWriter implements Closeable, TwoPhaseCommit {
boolean success = false;
try {
if (anySegmentsFlushed) {
maybeMerge();
maybeMerge(MergeTrigger.FULL_FLUSH, UNBOUNDED_MAX_MERGE_SEGMENTS);
}
success = true;
} finally {
@ -2809,7 +2819,7 @@ public class IndexWriter implements Closeable, TwoPhaseCommit {
// We can be called during close, when closing==true, so we must pass false to ensureOpen:
ensureOpen(false);
if (doFlush(applyAllDeletes) && triggerMerge) {
maybeMerge();
maybeMerge(MergeTrigger.FULL_FLUSH, UNBOUNDED_MAX_MERGE_SEGMENTS);
}
}
@ -3240,7 +3250,7 @@ public class IndexWriter implements Closeable, TwoPhaseCommit {
// segments) may now enable new merges, so we call
// merge policy & update pending merges.
if (success && !merge.isAborted() && (merge.maxNumSegments != -1 || (!closed && !closing))) {
updatePendingMerges(merge.maxNumSegments);
updatePendingMerges(MergeTrigger.MERGE_FINISHED, merge.maxNumSegments);
}
}
}

View File

@ -24,6 +24,8 @@ import java.util.List;
import java.util.Locale;
import java.util.Map;
import org.apache.lucene.index.MergePolicy.MergeTrigger;
/**
* <p>This class implements a {@link MergePolicy} that tries
@ -560,7 +562,7 @@ public abstract class LogMergePolicy extends MergePolicy {
* will return multiple merges, allowing the {@link
* MergeScheduler} to use concurrency. */
@Override
public MergeSpecification findMerges(SegmentInfos infos) throws IOException {
public MergeSpecification findMerges(MergeTrigger mergeTrigger, SegmentInfos infos) throws IOException {
final int numSegments = infos.size();
if (verbose()) {

View File

@ -19,6 +19,7 @@ package org.apache.lucene.index;
import java.io.IOException;
import java.util.ArrayList;
import java.util.Collections;
import java.util.List;
import java.util.Map;
@ -57,7 +58,7 @@ import org.apache.lucene.util.SetOnce;
*/
public abstract class MergePolicy implements java.io.Closeable, Cloneable {
/** OneMerge provides the information necessary to perform
* an individual primitive merge operation, resulting in
* a single new segment. The merge spec includes the
@ -333,11 +334,11 @@ public abstract class MergePolicy implements java.io.Closeable, Cloneable {
* {@link IndexWriter} calls this whenever there is a change to the segments.
* This call is always synchronized on the {@link IndexWriter} instance so
* only one thread at a time will call this method.
*
* @param mergeTrigger the event that triggered the merge
* @param segmentInfos
* the total set of segments in the index
*/
public abstract MergeSpecification findMerges(SegmentInfos segmentInfos)
public abstract MergeSpecification findMerges(MergeTrigger mergeTrigger, SegmentInfos segmentInfos)
throws IOException;
/**
@ -379,9 +380,36 @@ public abstract class MergePolicy implements java.io.Closeable, Cloneable {
* Release all resources for the policy.
*/
public abstract void close();
/**
* Returns true if a new segment (regardless of its origin) should use the compound file format.
*/
public abstract boolean useCompoundFile(SegmentInfos segments, SegmentInfoPerCommit newSegment) throws IOException;
/**
* MergeTrigger is passed to
* {@link MergePolicy#findMerges(MergeTrigger, SegmentInfos)} to indicate the
* event that triggered the merge.
*/
public static enum MergeTrigger {
/**
* Merge was triggered by a segment flush.
*/
SEGMENT_FLUSH,
/**
* Merge was triggered by a full flush. Full flushes
* can be caused by a commit, NRT reader reopen or a close call on the index writer.
*/
FULL_FLUSH,
/**
* Merge has been triggered explicitly by the user.
*/
EXPLICIT,
/**
* Merge was triggered by a successfully finished merge.
*/
MERGE_FINISHED,
}
}

View File

@ -17,8 +17,12 @@ package org.apache.lucene.index;
* limitations under the License.
*/
import java.io.IOException;
import java.util.Map;
import org.apache.lucene.index.MergePolicy.MergeTrigger;
import org.apache.lucene.index.MergePolicy.MergeSpecification;
/**
* A {@link MergePolicy} which never returns merges to execute (hence it's
* name). It is also a singleton and can be accessed through
@ -53,7 +57,7 @@ public final class NoMergePolicy extends MergePolicy {
public void close() {}
@Override
public MergeSpecification findMerges(SegmentInfos segmentInfos) { return null; }
public MergeSpecification findMerges(MergeTrigger mergeTrigger, SegmentInfos segmentInfos) { return null; }
@Override
public MergeSpecification findForcedMerges(SegmentInfos segmentInfos,

View File

@ -27,6 +27,8 @@ import java.util.Comparator;
import java.util.List;
import java.util.ArrayList;
import org.apache.lucene.index.MergePolicy.MergeTrigger;
/**
* Merges segments of approximately equal size, subject to
@ -298,7 +300,7 @@ public class TieredMergePolicy extends MergePolicy {
}
@Override
public MergeSpecification findMerges(SegmentInfos infos) throws IOException {
public MergeSpecification findMerges(MergeTrigger mergeTrigger, SegmentInfos infos) throws IOException {
if (verbose()) {
message("findMerges: " + infos.size() + " segments");
}

View File

@ -17,6 +17,7 @@ package org.apache.lucene.index;
* limitations under the License.
*/
import org.apache.lucene.index.MergePolicy.MergeTrigger;
import org.apache.lucene.util.Constants;
import java.io.IOException;
@ -75,8 +76,8 @@ public class UpgradeIndexMergePolicy extends MergePolicy {
}
@Override
public MergeSpecification findMerges(SegmentInfos segmentInfos) throws IOException {
return base.findMerges(segmentInfos);
public MergeSpecification findMerges(MergeTrigger mergeTrigger, SegmentInfos segmentInfos) throws IOException {
return base.findMerges(null, segmentInfos);
}
@Override

View File

@ -30,7 +30,7 @@ public class TestNoMergePolicy extends LuceneTestCase {
@Test
public void testNoMergePolicy() throws Exception {
MergePolicy mp = NoMergePolicy.NO_COMPOUND_FILES;
assertNull(mp.findMerges(null));
assertNull(mp.findMerges(null, (SegmentInfos)null));
assertNull(mp.findForcedMerges(null, 0, null));
assertNull(mp.findForcedDeletesMerges(null));
assertFalse(mp.useCompoundFile(null, null));

View File

@ -24,6 +24,7 @@ import java.util.Map;
import java.util.Random;
import org.apache.lucene.analysis.MockAnalyzer;
import org.apache.lucene.index.MergePolicy.MergeTrigger;
import org.apache.lucene.search.DocIdSetIterator;
import org.apache.lucene.store.Directory;
import org.apache.lucene.store.MockDirectoryWrapper;
@ -257,7 +258,7 @@ public class TestPerSegmentDeletes extends LuceneTestCase {
public void close() {}
@Override
public MergeSpecification findMerges(SegmentInfos segmentInfos)
public MergeSpecification findMerges(MergeTrigger mergeTrigger, SegmentInfos segmentInfos)
throws IOException {
MergeSpecification ms = new MergeSpecification();
if (doMerge) {

View File

@ -25,6 +25,7 @@ import java.util.List;
import java.util.Map;
import java.util.Random;
import org.apache.lucene.index.MergePolicy.MergeTrigger;
import org.apache.lucene.util._TestUtil;
/**
@ -40,7 +41,7 @@ public class MockRandomMergePolicy extends MergePolicy {
}
@Override
public MergeSpecification findMerges(SegmentInfos segmentInfos) {
public MergeSpecification findMerges(MergeTrigger mergeTrigger, SegmentInfos segmentInfos) {
MergeSpecification mergeSpec = null;
//System.out.println("MRMP: findMerges sis=" + segmentInfos);
@ -110,7 +111,7 @@ public class MockRandomMergePolicy extends MergePolicy {
@Override
public MergeSpecification findForcedDeletesMerges(SegmentInfos segmentInfos) throws IOException {
return findMerges(segmentInfos);
return findMerges(null, segmentInfos);
}
@Override

View File

@ -763,7 +763,6 @@ public abstract class LuceneTestCase extends Assert {
} else {
c.setMergePolicy(newLogMergePolicy());
}
c.setReaderPooling(r.nextBoolean());
c.setReaderTermsIndexDivisor(_TestUtil.nextInt(r, 1, 4));
return c;