mirror of https://github.com/apache/lucene.git
LUCENE-2773: don't build compound files for large merged segments (by default); default maxMergeMB to 2GB
git-svn-id: https://svn.apache.org/repos/asf/lucene/dev/trunk@1038562 13f79535-47bb-0310-9956-ffa450edef68
This commit is contained in:
parent
2867ab831e
commit
a1678b7d81
|
@ -17,42 +17,43 @@
|
|||
|
||||
package org.apache.lucene.benchmark.byTask;
|
||||
|
||||
import java.io.StringReader;
|
||||
import java.io.BufferedReader;
|
||||
import java.io.File;
|
||||
import java.io.FileReader;
|
||||
import java.io.BufferedReader;
|
||||
import java.io.StringReader;
|
||||
import java.text.Collator;
|
||||
import java.util.List;
|
||||
import java.util.Locale;
|
||||
|
||||
import org.apache.lucene.analysis.Analyzer;
|
||||
import org.apache.lucene.analysis.BaseTokenStreamTestCase;
|
||||
import org.apache.lucene.analysis.TokenStream;
|
||||
import org.apache.lucene.analysis.MockAnalyzer;
|
||||
import org.apache.lucene.analysis.TokenStream;
|
||||
import org.apache.lucene.analysis.tokenattributes.CharTermAttribute;
|
||||
import org.apache.lucene.benchmark.BenchmarkTestCase;
|
||||
import org.apache.lucene.benchmark.byTask.feeds.DocMaker;
|
||||
import org.apache.lucene.benchmark.byTask.feeds.ReutersQueryMaker;
|
||||
import org.apache.lucene.benchmark.byTask.tasks.CountingSearchTestTask;
|
||||
import org.apache.lucene.benchmark.byTask.tasks.CountingHighlighterTestTask;
|
||||
import org.apache.lucene.benchmark.byTask.stats.TaskStats;
|
||||
import org.apache.lucene.benchmark.byTask.tasks.CountingHighlighterTestTask;
|
||||
import org.apache.lucene.benchmark.byTask.tasks.CountingSearchTestTask;
|
||||
import org.apache.lucene.collation.CollationKeyAnalyzer;
|
||||
import org.apache.lucene.index.DocsEnum;
|
||||
import org.apache.lucene.index.FieldsEnum;
|
||||
import org.apache.lucene.index.IndexReader;
|
||||
import org.apache.lucene.index.IndexWriter;
|
||||
import org.apache.lucene.index.TermsEnum;
|
||||
import org.apache.lucene.index.MultiFields;
|
||||
import org.apache.lucene.index.FieldsEnum;
|
||||
import org.apache.lucene.index.DocsEnum;
|
||||
import org.apache.lucene.index.IndexWriterConfig;
|
||||
import org.apache.lucene.index.LogMergePolicy;
|
||||
import org.apache.lucene.index.SerialMergeScheduler;
|
||||
import org.apache.lucene.index.LogDocMergePolicy;
|
||||
import org.apache.lucene.index.TermFreqVector;
|
||||
import org.apache.lucene.util.BytesRef;
|
||||
import org.apache.lucene.index.IndexWriterConfig.OpenMode;
|
||||
import org.apache.lucene.store.Directory;
|
||||
import org.apache.lucene.index.IndexWriterConfig;
|
||||
import org.apache.lucene.index.LogDocMergePolicy;
|
||||
import org.apache.lucene.index.LogMergePolicy;
|
||||
import org.apache.lucene.index.MultiFields;
|
||||
import org.apache.lucene.index.SegmentInfos;
|
||||
import org.apache.lucene.index.SerialMergeScheduler;
|
||||
import org.apache.lucene.index.TermFreqVector;
|
||||
import org.apache.lucene.index.TermsEnum;
|
||||
import org.apache.lucene.search.FieldCache.DocTermsIndex;
|
||||
import org.apache.lucene.search.FieldCache;
|
||||
import org.apache.lucene.store.Directory;
|
||||
import org.apache.lucene.util.BytesRef;
|
||||
|
||||
/**
|
||||
* Test very simply that perf tasks - simple algorithms - are doing what they should.
|
||||
|
@ -809,12 +810,9 @@ public class TestPerfTasksLogic extends BenchmarkTestCase {
|
|||
ir.close();
|
||||
|
||||
// Make sure we have 3 segments:
|
||||
final String[] files = benchmark.getRunData().getDirectory().listAll();
|
||||
int cfsCount = 0;
|
||||
for(int i=0;i<files.length;i++)
|
||||
if (files[i].endsWith(".cfs"))
|
||||
cfsCount++;
|
||||
assertEquals(3, cfsCount);
|
||||
SegmentInfos infos = new SegmentInfos();
|
||||
infos.read(benchmark.getRunData().getDirectory());
|
||||
assertEquals(3, infos.size());
|
||||
}
|
||||
|
||||
/**
|
||||
|
|
|
@ -28,7 +28,7 @@ public class LogByteSizeMergePolicy extends LogMergePolicy {
|
|||
|
||||
/** Default maximum segment size. A segment of this size
|
||||
* or larger will never be merged. @see setMaxMergeMB */
|
||||
public static final double DEFAULT_MAX_MERGE_MB = Long.MAX_VALUE;
|
||||
public static final double DEFAULT_MAX_MERGE_MB = 2048;
|
||||
|
||||
public LogByteSizeMergePolicy() {
|
||||
minMergeSize = (long) (DEFAULT_MIN_MERGE_MB*1024*1024);
|
||||
|
|
|
@ -54,12 +54,19 @@ public abstract class LogMergePolicy extends MergePolicy {
|
|||
* or larger will never be merged. @see setMaxMergeDocs */
|
||||
public static final int DEFAULT_MAX_MERGE_DOCS = Integer.MAX_VALUE;
|
||||
|
||||
/** Default noCFSRatio. If a merge's size is >= 10% of
|
||||
* the index, then we disable compound file for it.
|
||||
* @see setNoCFSRatio */
|
||||
public static final double DEFAULT_NO_CFS_RATIO = 0.1;
|
||||
|
||||
protected int mergeFactor = DEFAULT_MERGE_FACTOR;
|
||||
|
||||
protected long minMergeSize;
|
||||
protected long maxMergeSize;
|
||||
protected int maxMergeDocs = DEFAULT_MAX_MERGE_DOCS;
|
||||
|
||||
protected double noCFSRatio = DEFAULT_NO_CFS_RATIO;
|
||||
|
||||
protected boolean calibrateSizeByDeletes = true;
|
||||
|
||||
protected boolean useCompoundFile = true;
|
||||
|
@ -74,6 +81,23 @@ public abstract class LogMergePolicy extends MergePolicy {
|
|||
return w != null && w.verbose();
|
||||
}
|
||||
|
||||
/** @see setNoCFSRatio */
|
||||
public double getNoCFSRatio() {
|
||||
return noCFSRatio;
|
||||
}
|
||||
|
||||
/** If a merged segment will be more than this percentage
|
||||
* of the total size of the index, leave the segment as
|
||||
* non-compound file even if compound file is enabled.
|
||||
* Set to 1.0 to always use CFS regardless of merge
|
||||
* size. */
|
||||
public void setNoCFSRatio(double noCFSRatio) {
|
||||
if (noCFSRatio < 0.0 || noCFSRatio > 1.0) {
|
||||
throw new IllegalArgumentException("noCFSRatio must be 0.0 to 1.0 inclusive; got " + noCFSRatio);
|
||||
}
|
||||
this.noCFSRatio = noCFSRatio;
|
||||
}
|
||||
|
||||
protected void message(String message) {
|
||||
if (verbose())
|
||||
writer.get().message("LMP: " + message);
|
||||
|
@ -207,7 +231,8 @@ public abstract class LogMergePolicy extends MergePolicy {
|
|||
return !hasDeletions &&
|
||||
!info.hasSeparateNorms() &&
|
||||
info.dir == w.getDirectory() &&
|
||||
info.getUseCompoundFile() == useCompoundFile;
|
||||
(info.getUseCompoundFile() == useCompoundFile ||
|
||||
(noCFSRatio != 1.0 && !info.getUseCompoundFile()));
|
||||
}
|
||||
|
||||
/**
|
||||
|
@ -230,12 +255,12 @@ public abstract class LogMergePolicy extends MergePolicy {
|
|||
// unless there is only 1 which is optimized.
|
||||
if (last - start - 1 > 1 || (start != last - 1 && !isOptimized(infos.info(start + 1)))) {
|
||||
// there is more than 1 segment to the right of this one, or an unoptimized single segment.
|
||||
spec.add(new OneMerge(infos.range(start + 1, last), useCompoundFile));
|
||||
spec.add(makeOneMerge(infos, infos.range(start + 1, last)));
|
||||
}
|
||||
last = start;
|
||||
} else if (last - start == mergeFactor) {
|
||||
// mergeFactor eligible segments were found, add them as a merge.
|
||||
spec.add(new OneMerge(infos.range(start, last), useCompoundFile));
|
||||
spec.add(makeOneMerge(infos, infos.range(start, last)));
|
||||
last = start;
|
||||
}
|
||||
--start;
|
||||
|
@ -243,7 +268,7 @@ public abstract class LogMergePolicy extends MergePolicy {
|
|||
|
||||
// Add any left-over segments, unless there is just 1 already optimized.
|
||||
if (last > 0 && (++start + 1 < last || !isOptimized(infos.info(start)))) {
|
||||
spec.add(new OneMerge(infos.range(start, last), useCompoundFile));
|
||||
spec.add(makeOneMerge(infos, infos.range(start, last)));
|
||||
}
|
||||
|
||||
return spec.merges.size() == 0 ? null : spec;
|
||||
|
@ -260,7 +285,7 @@ public abstract class LogMergePolicy extends MergePolicy {
|
|||
// First, enroll all "full" merges (size
|
||||
// mergeFactor) to potentially be run concurrently:
|
||||
while (last - maxNumSegments + 1 >= mergeFactor) {
|
||||
spec.add(new OneMerge(infos.range(last-mergeFactor, last), useCompoundFile));
|
||||
spec.add(makeOneMerge(infos, infos.range(last-mergeFactor, last)));
|
||||
last -= mergeFactor;
|
||||
}
|
||||
|
||||
|
@ -272,7 +297,7 @@ public abstract class LogMergePolicy extends MergePolicy {
|
|||
// Since we must optimize down to 1 segment, the
|
||||
// choice is simple:
|
||||
if (last > 1 || !isOptimized(infos.info(0))) {
|
||||
spec.add(new OneMerge(infos.range(0, last), useCompoundFile));
|
||||
spec.add(makeOneMerge(infos, infos.range(0, last)));
|
||||
}
|
||||
} else if (last > maxNumSegments) {
|
||||
|
||||
|
@ -301,7 +326,7 @@ public abstract class LogMergePolicy extends MergePolicy {
|
|||
}
|
||||
}
|
||||
|
||||
spec.add(new OneMerge(infos.range(bestStart, bestStart+finalMergeSize), useCompoundFile));
|
||||
spec.add(makeOneMerge(infos, infos.range(bestStart, bestStart+finalMergeSize)));
|
||||
}
|
||||
}
|
||||
return spec.merges.size() == 0 ? null : spec;
|
||||
|
@ -389,7 +414,7 @@ public abstract class LogMergePolicy extends MergePolicy {
|
|||
// deletions, so force a merge now:
|
||||
if (verbose())
|
||||
message(" add merge " + firstSegmentWithDeletions + " to " + (i-1) + " inclusive");
|
||||
spec.add(new OneMerge(segmentInfos.range(firstSegmentWithDeletions, i), useCompoundFile));
|
||||
spec.add(makeOneMerge(segmentInfos, segmentInfos.range(firstSegmentWithDeletions, i)));
|
||||
firstSegmentWithDeletions = i;
|
||||
}
|
||||
} else if (firstSegmentWithDeletions != -1) {
|
||||
|
@ -398,7 +423,7 @@ public abstract class LogMergePolicy extends MergePolicy {
|
|||
// mergeFactor segments
|
||||
if (verbose())
|
||||
message(" add merge " + firstSegmentWithDeletions + " to " + (i-1) + " inclusive");
|
||||
spec.add(new OneMerge(segmentInfos.range(firstSegmentWithDeletions, i), useCompoundFile));
|
||||
spec.add(makeOneMerge(segmentInfos, segmentInfos.range(firstSegmentWithDeletions, i)));
|
||||
firstSegmentWithDeletions = -1;
|
||||
}
|
||||
}
|
||||
|
@ -406,7 +431,7 @@ public abstract class LogMergePolicy extends MergePolicy {
|
|||
if (firstSegmentWithDeletions != -1) {
|
||||
if (verbose())
|
||||
message(" add merge " + firstSegmentWithDeletions + " to " + (numSegments-1) + " inclusive");
|
||||
spec.add(new OneMerge(segmentInfos.range(firstSegmentWithDeletions, numSegments), useCompoundFile));
|
||||
spec.add(makeOneMerge(segmentInfos, segmentInfos.range(firstSegmentWithDeletions, numSegments)));
|
||||
}
|
||||
|
||||
return spec;
|
||||
|
@ -506,7 +531,7 @@ public abstract class LogMergePolicy extends MergePolicy {
|
|||
spec = new MergeSpecification();
|
||||
if (verbose())
|
||||
message(" " + start + " to " + end + ": add this merge");
|
||||
spec.add(new OneMerge(infos.range(start, end), useCompoundFile));
|
||||
spec.add(makeOneMerge(infos, infos.range(start, end)));
|
||||
} else if (verbose())
|
||||
message(" " + start + " to " + end + ": contains segment over maxMergeSize or maxMergeDocs; skipping");
|
||||
|
||||
|
@ -520,6 +545,29 @@ public abstract class LogMergePolicy extends MergePolicy {
|
|||
return spec;
|
||||
}
|
||||
|
||||
protected OneMerge makeOneMerge(SegmentInfos infos, SegmentInfos infosToMerge) throws IOException {
|
||||
final boolean doCFS;
|
||||
if (!useCompoundFile) {
|
||||
doCFS = false;
|
||||
} else if (noCFSRatio == 1.0) {
|
||||
doCFS = true;
|
||||
} else {
|
||||
|
||||
long totSize = 0;
|
||||
for(SegmentInfo info : infos) {
|
||||
totSize += size(info);
|
||||
}
|
||||
long mergeSize = 0;
|
||||
for(SegmentInfo info : infosToMerge) {
|
||||
mergeSize += size(info);
|
||||
}
|
||||
|
||||
doCFS = mergeSize <= noCFSRatio * totSize;
|
||||
}
|
||||
|
||||
return new OneMerge(infosToMerge, doCFS);
|
||||
}
|
||||
|
||||
/** <p>Determines the largest segment (measured by
|
||||
* document count) that may be merged with other segments.
|
||||
* Small values (e.g., less than 10,000) are best for
|
||||
|
|
|
@ -2595,13 +2595,13 @@ public class TestIndexWriter extends LuceneTestCase {
|
|||
files = Arrays.asList(dir.listAll());
|
||||
assertTrue(files.contains("_0.cfs"));
|
||||
// optimize created this
|
||||
assertTrue(files.contains("_2.cfs"));
|
||||
//assertTrue(files.contains("_2.cfs"));
|
||||
w.deleteUnusedFiles();
|
||||
|
||||
files = Arrays.asList(dir.listAll());
|
||||
// r still holds this file open
|
||||
assertTrue(files.contains("_0.cfs"));
|
||||
assertTrue(files.contains("_2.cfs"));
|
||||
//assertTrue(files.contains("_2.cfs"));
|
||||
|
||||
r.close();
|
||||
if (iter == 0) {
|
||||
|
@ -2614,7 +2614,7 @@ public class TestIndexWriter extends LuceneTestCase {
|
|||
files = Arrays.asList(dir.listAll());
|
||||
assertFalse(files.contains("_0.cfs"));
|
||||
}
|
||||
assertTrue(files.contains("_2.cfs"));
|
||||
//assertTrue(files.contains("_2.cfs"));
|
||||
|
||||
w.close();
|
||||
r2.close();
|
||||
|
|
|
@ -1005,6 +1005,7 @@ public class TestIndexWriterExceptions extends LuceneTestCase {
|
|||
|
||||
writer = new IndexWriter(dir, newIndexWriterConfig( TEST_VERSION_CURRENT, new MockAnalyzer()));
|
||||
((LogMergePolicy) writer.getMergePolicy()).setUseCompoundFile(true);
|
||||
((LogMergePolicy) writer.getMergePolicy()).setNoCFSRatio(1.0);
|
||||
|
||||
// add 100 documents
|
||||
for (int i = 0; i < 100; i++) {
|
||||
|
|
|
@ -252,25 +252,5 @@ public class TestIndexWriterMergePolicy extends LuceneTestCase {
|
|||
if (upperBound * mergeFactor <= maxMergeDocs) {
|
||||
assertTrue(numSegments < mergeFactor);
|
||||
}
|
||||
|
||||
String[] files = writer.getDirectory().listAll();
|
||||
int segmentCfsCount = 0;
|
||||
for (int i = 0; i < files.length; i++) {
|
||||
if (files[i].endsWith(".cfs")) {
|
||||
segmentCfsCount++;
|
||||
}
|
||||
}
|
||||
assertEquals("index=" + writer.segString(), segmentCount, segmentCfsCount);
|
||||
}
|
||||
|
||||
/*
|
||||
private void printSegmentDocCounts(IndexWriter writer) {
|
||||
int segmentCount = writer.getSegmentCount();
|
||||
System.out.println("" + segmentCount + " segments total");
|
||||
for (int i = 0; i < segmentCount; i++) {
|
||||
System.out.println(" segment " + i + " has " + writer.getDocCount(i)
|
||||
+ " docs");
|
||||
}
|
||||
}
|
||||
*/
|
||||
}
|
||||
|
|
Loading…
Reference in New Issue