mirror of https://github.com/apache/lucene.git
LUCENE-7020: Remove TieredMergePolicy#setMaxMergeAtOnceExplicit. (#230)
TieredMergePolicy no longer bounds the number of segments that can be merged via a forced merge.
This commit is contained in:
parent
257d256def
commit
83ba5d859c
|
@ -133,6 +133,10 @@ API Changes
|
|||
* LUCENE-9959: Add non thread local based API for term vector reader usage. (Zach Chen, Adrien Grand,
|
||||
David Smiley, Robert Muir, Mike Drob)
|
||||
|
||||
* LUCENE-7020: TieredMergePolicy#setMaxMergeAtOnceExplicit has been removed.
|
||||
TieredMergePolicy no longer sets a limit on the maximum number of segments
|
||||
that can be merged at once via a forced merge. (Adrien Grand, Shawn Heisey)
|
||||
|
||||
* LUCENE-10027: Directory reader open API from indexCommit and leafSorter has been modified
|
||||
to add an extra parameter - minSupportedMajorVersion. (Mayya Sharipova)
|
||||
|
||||
|
@ -372,6 +376,10 @@ API Changes
|
|||
* LUCENE-10027: Add a new Directory reader open API from indexCommit and
|
||||
a custom comparator for sorting leaf readers. (Mayya Sharipova)
|
||||
|
||||
* LUCENE-7020: TieredMergePolicy#setMaxMergeAtOnceExplicit is deprecated
|
||||
and the number of segments that get merged via explicit merges is unlimited
|
||||
by default. (Adrien Grand, Shawn Heisey)
|
||||
|
||||
New Features
|
||||
---------------------
|
||||
(No changes)
|
||||
|
|
|
@ -84,7 +84,6 @@ public class TieredMergePolicy extends MergePolicy {
|
|||
// value and segsPerTier to avoid suboptimal merging.
|
||||
private int maxMergeAtOnce = 10;
|
||||
private long maxMergedSegmentBytes = 5 * 1024 * 1024 * 1024L;
|
||||
private int maxMergeAtOnceExplicit = 30;
|
||||
|
||||
private long floorSegmentBytes = 2 * 1024 * 1024L;
|
||||
private double segsPerTier = 10.0;
|
||||
|
@ -96,11 +95,7 @@ public class TieredMergePolicy extends MergePolicy {
|
|||
super(DEFAULT_NO_CFS_RATIO, MergePolicy.DEFAULT_MAX_CFS_SEGMENT_SIZE);
|
||||
}
|
||||
|
||||
/**
|
||||
* Maximum number of segments to be merged at a time during "normal" merging. For explicit merging
|
||||
* (eg, forceMerge or forceMergeDeletes was called), see {@link #setMaxMergeAtOnceExplicit}.
|
||||
* Default is 10.
|
||||
*/
|
||||
/** Maximum number of segments to be merged at a time during "normal" merging. Default is 10. */
|
||||
public TieredMergePolicy setMaxMergeAtOnce(int v) {
|
||||
if (v < 2) {
|
||||
throw new IllegalArgumentException("maxMergeAtOnce must be > 1 (got " + v + ")");
|
||||
|
@ -126,27 +121,6 @@ public class TieredMergePolicy extends MergePolicy {
|
|||
// TODO: should addIndexes do explicit merging, too? And,
|
||||
// if user calls IW.maybeMerge "explicitly"
|
||||
|
||||
/**
|
||||
* Maximum number of segments to be merged at a time, during forceMerge or forceMergeDeletes.
|
||||
* Default is 30.
|
||||
*/
|
||||
public TieredMergePolicy setMaxMergeAtOnceExplicit(int v) {
|
||||
if (v < 2) {
|
||||
throw new IllegalArgumentException("maxMergeAtOnceExplicit must be > 1 (got " + v + ")");
|
||||
}
|
||||
maxMergeAtOnceExplicit = v;
|
||||
return this;
|
||||
}
|
||||
|
||||
/**
|
||||
* Returns the current maxMergeAtOnceExplicit setting.
|
||||
*
|
||||
* @see #setMaxMergeAtOnceExplicit
|
||||
*/
|
||||
public int getMaxMergeAtOnceExplicit() {
|
||||
return maxMergeAtOnceExplicit;
|
||||
}
|
||||
|
||||
/**
|
||||
* Maximum sized segment to produce during normal merging. This setting is approximate: the
|
||||
* estimate of the merged segment size is made by summing sizes of to-be-merged segments
|
||||
|
@ -819,15 +793,12 @@ public class TieredMergePolicy extends MergePolicy {
|
|||
}
|
||||
|
||||
final int startingSegmentCount = sortedSizeAndDocs.size();
|
||||
final boolean finalMerge = startingSegmentCount < maxSegmentCount + maxMergeAtOnceExplicit - 1;
|
||||
if (finalMerge && forceMergeRunning) {
|
||||
if (forceMergeRunning) {
|
||||
return null;
|
||||
}
|
||||
|
||||
// This is the special case of merging down to one segment
|
||||
if (sortedSizeAndDocs.size() < maxMergeAtOnceExplicit
|
||||
&& maxSegmentCount == 1
|
||||
&& totalMergeBytes < maxMergeBytes) {
|
||||
if (maxSegmentCount == 1 && totalMergeBytes < maxMergeBytes) {
|
||||
MergeSpecification spec = new MergeSpecification();
|
||||
List<SegmentCommitInfo> allOfThem = new ArrayList<>();
|
||||
for (SegmentSizeAndDocs segSizeDocs : sortedSizeAndDocs) {
|
||||
|
@ -844,8 +815,7 @@ public class TieredMergePolicy extends MergePolicy {
|
|||
while (true) {
|
||||
List<SegmentCommitInfo> candidate = new ArrayList<>();
|
||||
long currentCandidateBytes = 0L;
|
||||
int mergesAllowed = maxMergeAtOnceExplicit;
|
||||
while (index >= 0 && resultingSegments > maxSegmentCount && mergesAllowed > 0) {
|
||||
while (index >= 0 && resultingSegments > maxSegmentCount) {
|
||||
final SegmentCommitInfo current = sortedSizeAndDocs.get(index).segInfo;
|
||||
final int initialCandidateSize = candidate.size();
|
||||
final long currentSegmentSize = current.sizeInBytes();
|
||||
|
@ -857,7 +827,6 @@ public class TieredMergePolicy extends MergePolicy {
|
|||
candidate.add(current);
|
||||
--index;
|
||||
currentCandidateBytes += currentSegmentSize;
|
||||
--mergesAllowed;
|
||||
if (initialCandidateSize > 0) {
|
||||
// Any merge that handles two or more segments reduces the resulting number of segments
|
||||
// by the number of segments handled - 1
|
||||
|
@ -872,9 +841,7 @@ public class TieredMergePolicy extends MergePolicy {
|
|||
// segments or that create a segment close to the
|
||||
// maximum allowed segment sized are permitted
|
||||
if (candidateSize > 1
|
||||
&& (forceMergeRunning == false
|
||||
|| candidateSize == maxMergeAtOnceExplicit
|
||||
|| candidateSize > 0.7 * maxMergeBytes)) {
|
||||
&& (forceMergeRunning == false || candidateSize > 0.7 * maxMergeBytes)) {
|
||||
final OneMerge merge = new OneMerge(candidate);
|
||||
if (verbose(mergeContext)) {
|
||||
message("add merge=" + segString(mergeContext, merge.segments), mergeContext);
|
||||
|
@ -937,7 +904,7 @@ public class TieredMergePolicy extends MergePolicy {
|
|||
return doFindMerges(
|
||||
sortedInfos,
|
||||
maxMergedSegmentBytes,
|
||||
maxMergeAtOnceExplicit,
|
||||
Integer.MAX_VALUE,
|
||||
Integer.MAX_VALUE,
|
||||
0,
|
||||
MERGE_TYPE.FORCE_MERGE_DELETES,
|
||||
|
@ -953,7 +920,6 @@ public class TieredMergePolicy extends MergePolicy {
|
|||
public String toString() {
|
||||
StringBuilder sb = new StringBuilder("[" + getClass().getSimpleName() + ": ");
|
||||
sb.append("maxMergeAtOnce=").append(maxMergeAtOnce).append(", ");
|
||||
sb.append("maxMergeAtOnceExplicit=").append(maxMergeAtOnceExplicit).append(", ");
|
||||
sb.append("maxMergedSegmentMB=").append(maxMergedSegmentBytes / 1024 / 1024.).append(", ");
|
||||
sb.append("floorSegmentMB=").append(floorSegmentBytes / 1024 / 1024.).append(", ");
|
||||
sb.append("forceMergeDeletesPctAllowed=").append(forceMergeDeletesPctAllowed).append(", ");
|
||||
|
|
|
@ -18,6 +18,9 @@ package org.apache.lucene.index;
|
|||
|
||||
import com.carrotsearch.randomizedtesting.generators.RandomStrings;
|
||||
import java.io.IOException;
|
||||
import java.util.ArrayList;
|
||||
import java.util.List;
|
||||
import java.util.Map;
|
||||
import java.util.concurrent.CountDownLatch;
|
||||
import java.util.concurrent.TimeUnit;
|
||||
import java.util.concurrent.atomic.AtomicBoolean;
|
||||
|
@ -416,11 +419,45 @@ public class TestConcurrentMergeScheduler extends LuceneTestCase {
|
|||
public void testLiveMaxMergeCount() throws Exception {
|
||||
Directory d = newDirectory();
|
||||
IndexWriterConfig iwc = new IndexWriterConfig(new MockAnalyzer(random()));
|
||||
TieredMergePolicy tmp = new TieredMergePolicy();
|
||||
tmp.setSegmentsPerTier(1000);
|
||||
tmp.setMaxMergeAtOnce(1000);
|
||||
tmp.setMaxMergeAtOnceExplicit(10);
|
||||
iwc.setMergePolicy(tmp);
|
||||
iwc.setMergePolicy(
|
||||
new MergePolicy() {
|
||||
|
||||
@Override
|
||||
public MergeSpecification findMerges(
|
||||
MergeTrigger mergeTrigger, SegmentInfos segmentInfos, MergeContext mergeContext)
|
||||
throws IOException {
|
||||
// no natural merges
|
||||
return null;
|
||||
}
|
||||
|
||||
@Override
|
||||
public MergeSpecification findForcedDeletesMerges(
|
||||
SegmentInfos segmentInfos, MergeContext mergeContext) throws IOException {
|
||||
// not needed
|
||||
return null;
|
||||
}
|
||||
|
||||
@Override
|
||||
public MergeSpecification findForcedMerges(
|
||||
SegmentInfos segmentInfos,
|
||||
int maxSegmentCount,
|
||||
Map<SegmentCommitInfo, Boolean> segmentsToMerge,
|
||||
MergeContext mergeContext)
|
||||
throws IOException {
|
||||
// The test is about testing that CMS bounds the number of merging threads, so we just
|
||||
// return many merges.
|
||||
MergeSpecification spec = new MergeSpecification();
|
||||
List<SegmentCommitInfo> oneMerge = new ArrayList<>();
|
||||
for (SegmentCommitInfo sci : segmentsToMerge.keySet()) {
|
||||
oneMerge.add(sci);
|
||||
if (oneMerge.size() >= 10) {
|
||||
spec.add(new OneMerge(new ArrayList<>(oneMerge)));
|
||||
oneMerge.clear();
|
||||
}
|
||||
}
|
||||
return spec;
|
||||
}
|
||||
});
|
||||
iwc.setMaxBufferedDocs(2);
|
||||
iwc.setRAMBufferSizeMB(-1);
|
||||
|
||||
|
|
|
@ -482,36 +482,6 @@ public class TestTieredMergePolicy extends BaseMergePolicyTestCase {
|
|||
// would have been the final merge
|
||||
// so we check that it was prevented.
|
||||
assertNull(specification);
|
||||
|
||||
SegmentInfos manySegmentsInfos = new SegmentInfos(Version.LATEST.major);
|
||||
final int manySegmentsCount = atLeast(500);
|
||||
for (int j = 0; j < manySegmentsCount; ++j) {
|
||||
manySegmentsInfos.add(
|
||||
makeSegmentCommitInfo("_" + j, 1000, 0, 0.1D, IndexWriter.SOURCE_MERGE));
|
||||
}
|
||||
|
||||
// We set one merge to be ongoing. Since we have more than 30 (the max merge count) times the
|
||||
// number of segments
|
||||
// of that we want to merge to this is not the final merge and hence the returned specification
|
||||
// must not be null.
|
||||
mergeContext.setMergingSegments(Collections.singleton(manySegmentsInfos.asList().get(0)));
|
||||
final MergeSpecification specificationManySegments =
|
||||
tmp.findForcedMerges(
|
||||
manySegmentsInfos, expectedCount, segmentsToMerge(manySegmentsInfos), mergeContext);
|
||||
assertMaxSize(specificationManySegments, maxSegmentSize);
|
||||
for (OneMerge merge : specificationManySegments.merges) {
|
||||
assertEquals(
|
||||
"No merges of less than the max merge count are permitted while another merge is in progress",
|
||||
merge.segments.size(),
|
||||
tmp.getMaxMergeAtOnceExplicit());
|
||||
}
|
||||
final int resultingCountManySegments =
|
||||
manySegmentsInfos.size()
|
||||
+ specificationManySegments.merges.size()
|
||||
- specificationManySegments.merges.stream()
|
||||
.mapToInt(spec -> spec.segments.size())
|
||||
.sum();
|
||||
assertTrue(resultingCountManySegments >= expectedCount);
|
||||
}
|
||||
|
||||
private static Map<SegmentCommitInfo, Boolean> segmentsToMerge(SegmentInfos infos) {
|
||||
|
|
|
@ -1158,10 +1158,8 @@ public abstract class LuceneTestCase extends Assert {
|
|||
TieredMergePolicy tmp = new TieredMergePolicy();
|
||||
if (rarely(r)) {
|
||||
tmp.setMaxMergeAtOnce(TestUtil.nextInt(r, 2, 9));
|
||||
tmp.setMaxMergeAtOnceExplicit(TestUtil.nextInt(r, 2, 9));
|
||||
} else {
|
||||
tmp.setMaxMergeAtOnce(TestUtil.nextInt(r, 10, 50));
|
||||
tmp.setMaxMergeAtOnceExplicit(TestUtil.nextInt(r, 10, 50));
|
||||
}
|
||||
if (rarely(r)) {
|
||||
tmp.setMaxMergedSegmentMB(0.2 + r.nextDouble() * 2.0);
|
||||
|
@ -1293,10 +1291,8 @@ public abstract class LuceneTestCase extends Assert {
|
|||
TieredMergePolicy tmp = (TieredMergePolicy) mp;
|
||||
if (rarely(r)) {
|
||||
tmp.setMaxMergeAtOnce(TestUtil.nextInt(r, 2, 9));
|
||||
tmp.setMaxMergeAtOnceExplicit(TestUtil.nextInt(r, 2, 9));
|
||||
} else {
|
||||
tmp.setMaxMergeAtOnce(TestUtil.nextInt(r, 10, 50));
|
||||
tmp.setMaxMergeAtOnceExplicit(TestUtil.nextInt(r, 10, 50));
|
||||
}
|
||||
if (rarely(r)) {
|
||||
tmp.setMaxMergedSegmentMB(0.2 + r.nextDouble() * 2.0);
|
||||
|
|
Loading…
Reference in New Issue