mirror of https://github.com/apache/lucene.git
LUCENE-3084: cutover to List<SegmentInfo> not SegmentInfos for MergePolicy.OneMerge.segments
git-svn-id: https://svn.apache.org/repos/asf/lucene/dev/trunk@1101574 13f79535-47bb-0310-9956-ffa450edef68
This commit is contained in:
parent
673cef5639
commit
4d48083664
|
@ -132,9 +132,9 @@ class BufferedDeletesStream {
|
|||
public final long gen;
|
||||
|
||||
// If non-null, contains segments that are 100% deleted
|
||||
public final SegmentInfos allDeleted;
|
||||
public final List<SegmentInfo> allDeleted;
|
||||
|
||||
ApplyDeletesResult(boolean anyDeletes, long gen, SegmentInfos allDeleted) {
|
||||
ApplyDeletesResult(boolean anyDeletes, long gen, List<SegmentInfo> allDeleted) {
|
||||
this.anyDeletes = anyDeletes;
|
||||
this.gen = gen;
|
||||
this.allDeleted = allDeleted;
|
||||
|
@ -164,7 +164,7 @@ class BufferedDeletesStream {
|
|||
/** Resolves the buffered deleted Term/Query/docIDs, into
|
||||
* actual deleted docIDs in the deletedDocs BitVector for
|
||||
* each SegmentReader. */
|
||||
public synchronized ApplyDeletesResult applyDeletes(IndexWriter.ReaderPool readerPool, SegmentInfos infos) throws IOException {
|
||||
public synchronized ApplyDeletesResult applyDeletes(IndexWriter.ReaderPool readerPool, List<SegmentInfo> infos) throws IOException {
|
||||
final long t0 = System.currentTimeMillis();
|
||||
|
||||
if (infos.size() == 0) {
|
||||
|
@ -182,7 +182,7 @@ class BufferedDeletesStream {
|
|||
message("applyDeletes: infos=" + infos + " packetCount=" + deletes.size());
|
||||
}
|
||||
|
||||
SegmentInfos infos2 = new SegmentInfos();
|
||||
List<SegmentInfo> infos2 = new ArrayList<SegmentInfo>();
|
||||
infos2.addAll(infos);
|
||||
Collections.sort(infos2, sortSegInfoByDelGen);
|
||||
|
||||
|
@ -192,7 +192,7 @@ class BufferedDeletesStream {
|
|||
int infosIDX = infos2.size()-1;
|
||||
int delIDX = deletes.size()-1;
|
||||
|
||||
SegmentInfos allDeleted = null;
|
||||
List<SegmentInfo> allDeleted = null;
|
||||
|
||||
while (infosIDX >= 0) {
|
||||
//System.out.println("BD: cycle delIDX=" + delIDX + " infoIDX=" + infosIDX);
|
||||
|
@ -245,7 +245,7 @@ class BufferedDeletesStream {
|
|||
|
||||
if (segAllDeletes) {
|
||||
if (allDeleted == null) {
|
||||
allDeleted = new SegmentInfos();
|
||||
allDeleted = new ArrayList<SegmentInfo>();
|
||||
}
|
||||
allDeleted.add(info);
|
||||
}
|
||||
|
@ -287,7 +287,7 @@ class BufferedDeletesStream {
|
|||
|
||||
if (segAllDeletes) {
|
||||
if (allDeleted == null) {
|
||||
allDeleted = new SegmentInfos();
|
||||
allDeleted = new ArrayList<SegmentInfo>();
|
||||
}
|
||||
allDeleted.add(info);
|
||||
}
|
||||
|
|
|
@ -135,8 +135,8 @@ public class ConcurrentMergeScheduler extends MergeScheduler {
|
|||
final MergePolicy.OneMerge m1 = t1.getCurrentMerge();
|
||||
final MergePolicy.OneMerge m2 = t2.getCurrentMerge();
|
||||
|
||||
final int c1 = m1 == null ? Integer.MAX_VALUE : m1.segments.totalDocCount();
|
||||
final int c2 = m2 == null ? Integer.MAX_VALUE : m2.segments.totalDocCount();
|
||||
final int c1 = m1 == null ? Integer.MAX_VALUE : m1.totalDocCount;
|
||||
final int c2 = m2 == null ? Integer.MAX_VALUE : m2.totalDocCount;
|
||||
|
||||
return c2 - c1;
|
||||
}
|
||||
|
|
|
@ -421,7 +421,7 @@ public class IndexWriter implements Closeable {
|
|||
private final Map<SegmentInfo,SegmentReader> readerMap = new HashMap<SegmentInfo,SegmentReader>();
|
||||
|
||||
/** Forcefully clear changes for the specified segments. This is called on successful merge. */
|
||||
synchronized void clear(SegmentInfos infos) throws IOException {
|
||||
synchronized void clear(List<SegmentInfo> infos) throws IOException {
|
||||
if (infos == null) {
|
||||
for (Map.Entry<SegmentInfo,SegmentReader> ent: readerMap.entrySet()) {
|
||||
ent.getValue().hasChanges = false;
|
||||
|
@ -511,7 +511,7 @@ public class IndexWriter implements Closeable {
|
|||
return false;
|
||||
}
|
||||
|
||||
public synchronized void drop(SegmentInfos infos) throws IOException {
|
||||
public synchronized void drop(List<SegmentInfo> infos) throws IOException {
|
||||
for(SegmentInfo info : infos) {
|
||||
drop(info);
|
||||
}
|
||||
|
@ -2729,7 +2729,7 @@ public class IndexWriter implements Closeable {
|
|||
|
||||
assert testPoint("startCommitMergeDeletes");
|
||||
|
||||
final SegmentInfos sourceSegments = merge.segments;
|
||||
final List<SegmentInfo> sourceSegments = merge.segments;
|
||||
|
||||
if (infoStream != null)
|
||||
message("commitMergeDeletes " + merge.segString(directory));
|
||||
|
@ -2741,7 +2741,7 @@ public class IndexWriter implements Closeable {
|
|||
long minGen = Long.MAX_VALUE;
|
||||
|
||||
for(int i=0; i < sourceSegments.size(); i++) {
|
||||
SegmentInfo info = sourceSegments.info(i);
|
||||
SegmentInfo info = sourceSegments.get(i);
|
||||
minGen = Math.min(info.getBufferedDeletesGen(), minGen);
|
||||
int docCount = info.docCount;
|
||||
final SegmentReader previousReader = merge.readerClones.get(i);
|
||||
|
@ -3184,7 +3184,7 @@ public class IndexWriter implements Closeable {
|
|||
// It's possible we are called twice, eg if there was an
|
||||
// exception inside mergeInit
|
||||
if (merge.registerDone) {
|
||||
final SegmentInfos sourceSegments = merge.segments;
|
||||
final List<SegmentInfo> sourceSegments = merge.segments;
|
||||
for(SegmentInfo info : sourceSegments) {
|
||||
mergingSegments.remove(info);
|
||||
}
|
||||
|
@ -3255,7 +3255,7 @@ public class IndexWriter implements Closeable {
|
|||
|
||||
int mergedDocCount = 0;
|
||||
|
||||
SegmentInfos sourceSegments = merge.segments;
|
||||
List<SegmentInfo> sourceSegments = merge.segments;
|
||||
|
||||
SegmentMerger merger = new SegmentMerger(directory, config.getTermIndexInterval(), mergedName, merge,
|
||||
codecs, payloadProcessorProvider,
|
||||
|
@ -3276,7 +3276,7 @@ public class IndexWriter implements Closeable {
|
|||
int segUpto = 0;
|
||||
while(segUpto < sourceSegments.size()) {
|
||||
|
||||
final SegmentInfo info = sourceSegments.info(segUpto);
|
||||
final SegmentInfo info = sourceSegments.get(segUpto);
|
||||
|
||||
// Hold onto the "live" reader; we will use this to
|
||||
// commit merged deletes
|
||||
|
@ -3470,14 +3470,14 @@ public class IndexWriter implements Closeable {
|
|||
}
|
||||
|
||||
/** @lucene.internal */
|
||||
public synchronized String segString(SegmentInfos infos) throws IOException {
|
||||
public synchronized String segString(List<SegmentInfo> infos) throws IOException {
|
||||
StringBuilder buffer = new StringBuilder();
|
||||
final int count = infos.size();
|
||||
for(int i = 0; i < count; i++) {
|
||||
if (i > 0) {
|
||||
buffer.append(' ');
|
||||
}
|
||||
buffer.append(segString(infos.info(i)));
|
||||
buffer.append(segString(infos.get(i)));
|
||||
}
|
||||
|
||||
return buffer.toString();
|
||||
|
|
|
@ -20,7 +20,6 @@ package org.apache.lucene.index;
|
|||
import java.io.IOException;
|
||||
import java.util.ArrayList;
|
||||
import java.util.Collection;
|
||||
import java.util.Comparator;
|
||||
import java.util.List;
|
||||
import java.util.Set;
|
||||
|
||||
|
@ -595,7 +594,7 @@ public abstract class LogMergePolicy extends MergePolicy {
|
|||
} else if (!anyTooLarge) {
|
||||
if (spec == null)
|
||||
spec = new MergeSpecification();
|
||||
final SegmentInfos mergeInfos = new SegmentInfos();
|
||||
final List<SegmentInfo> mergeInfos = new ArrayList<SegmentInfo>();
|
||||
for(int i=start;i<end;i++) {
|
||||
mergeInfos.add(levels.get(i).info);
|
||||
assert infos.contains(levels.get(i).info);
|
||||
|
|
|
@ -32,7 +32,7 @@ final class MergeDocIDRemapper {
|
|||
|
||||
public MergeDocIDRemapper(SegmentInfos infos, int[][] docMaps, int[] delCounts, MergePolicy.OneMerge merge, int mergedDocCount) {
|
||||
this.docMaps = docMaps;
|
||||
SegmentInfo firstSegment = merge.segments.info(0);
|
||||
SegmentInfo firstSegment = merge.segments.get(0);
|
||||
int i = 0;
|
||||
while(true) {
|
||||
SegmentInfo info = infos.info(i);
|
||||
|
@ -45,7 +45,7 @@ final class MergeDocIDRemapper {
|
|||
int numDocs = 0;
|
||||
for(int j=0;j<docMaps.length;i++,j++) {
|
||||
numDocs += infos.info(i).docCount;
|
||||
assert infos.info(i).equals(merge.segments.info(j));
|
||||
assert infos.info(i).equals(merge.segments.get(j));
|
||||
}
|
||||
maxDocID = minDocID + numDocs;
|
||||
|
||||
|
@ -55,7 +55,7 @@ final class MergeDocIDRemapper {
|
|||
starts[0] = minDocID;
|
||||
newStarts[0] = minDocID;
|
||||
for(i=1;i<docMaps.length;i++) {
|
||||
final int lastDocCount = merge.segments.info(i-1).docCount;
|
||||
final int lastDocCount = merge.segments.get(i-1).docCount;
|
||||
starts[i] = starts[i-1] + lastDocCount;
|
||||
newStarts[i] = newStarts[i-1] + lastDocCount - delCounts[i-1];
|
||||
}
|
||||
|
@ -69,7 +69,7 @@ final class MergeDocIDRemapper {
|
|||
// assert docShift > 0;
|
||||
|
||||
// Make sure it all adds up:
|
||||
assert docShift == maxDocID - (newStarts[docMaps.length-1] + merge.segments.info(docMaps.length-1).docCount - delCounts[docMaps.length-1]);
|
||||
assert docShift == maxDocID - (newStarts[docMaps.length-1] + merge.segments.get(docMaps.length-1).docCount - delCounts[docMaps.length-1]);
|
||||
}
|
||||
|
||||
public int remap(int oldDocID) {
|
||||
|
|
|
@ -75,15 +75,21 @@ public abstract class MergePolicy implements java.io.Closeable {
|
|||
long estimatedMergeBytes; // used by IndexWriter
|
||||
List<SegmentReader> readers; // used by IndexWriter
|
||||
List<SegmentReader> readerClones; // used by IndexWriter
|
||||
public final SegmentInfos segments;
|
||||
public final List<SegmentInfo> segments;
|
||||
public final int totalDocCount;
|
||||
boolean aborted;
|
||||
Throwable error;
|
||||
boolean paused;
|
||||
|
||||
public OneMerge(SegmentInfos segments) {
|
||||
public OneMerge(List<SegmentInfo> segments) {
|
||||
if (0 == segments.size())
|
||||
throw new RuntimeException("segments must include at least one segment");
|
||||
this.segments = segments;
|
||||
int count = 0;
|
||||
for(SegmentInfo info : segments) {
|
||||
count += info.docCount;
|
||||
}
|
||||
totalDocCount = count;
|
||||
}
|
||||
|
||||
/** Record that an exception occurred while executing
|
||||
|
@ -147,7 +153,7 @@ public abstract class MergePolicy implements java.io.Closeable {
|
|||
final int numSegments = segments.size();
|
||||
for(int i=0;i<numSegments;i++) {
|
||||
if (i > 0) b.append(' ');
|
||||
b.append(segments.info(i).toString(dir, 0));
|
||||
b.append(segments.get(i).toString(dir, 0));
|
||||
}
|
||||
if (info != null)
|
||||
b.append(" into ").append(info.name);
|
||||
|
|
|
@ -23,6 +23,8 @@ import java.util.Collection;
|
|||
import java.util.Collections;
|
||||
import java.util.HashSet;
|
||||
import java.util.Comparator;
|
||||
import java.util.List;
|
||||
import java.util.ArrayList;
|
||||
|
||||
/**
|
||||
* Merges segments of approximately equal size, subject to
|
||||
|
@ -249,7 +251,7 @@ public class TieredMergePolicy extends MergePolicy {
|
|||
final Collection<SegmentInfo> merging = writer.get().getMergingSegments();
|
||||
final Collection<SegmentInfo> toBeMerged = new HashSet<SegmentInfo>();
|
||||
|
||||
final SegmentInfos infosSorted = new SegmentInfos();
|
||||
final List<SegmentInfo> infosSorted = new ArrayList<SegmentInfo>();
|
||||
infosSorted.addAll(infos);
|
||||
|
||||
Collections.sort(infosSorted, segmentByteSizeDescending);
|
||||
|
@ -277,7 +279,7 @@ public class TieredMergePolicy extends MergePolicy {
|
|||
// If we have too-large segments, grace them out
|
||||
// of the maxSegmentCount:
|
||||
int tooBigCount = 0;
|
||||
while (tooBigCount < infosSorted.size() && size(infosSorted.info(tooBigCount)) >= maxMergedSegmentBytes/2.0) {
|
||||
while (tooBigCount < infosSorted.size() && size(infosSorted.get(tooBigCount)) >= maxMergedSegmentBytes/2.0) {
|
||||
totIndexBytes -= size(infosSorted.get(tooBigCount));
|
||||
tooBigCount++;
|
||||
}
|
||||
|
@ -310,7 +312,7 @@ public class TieredMergePolicy extends MergePolicy {
|
|||
// Gather eligible segments for merging, ie segments
|
||||
// not already being merged and not already picked (by
|
||||
// prior iteration of this loop) for merging:
|
||||
final SegmentInfos eligible = new SegmentInfos();
|
||||
final List<SegmentInfo> eligible = new ArrayList<SegmentInfo>();
|
||||
for(int idx = tooBigCount; idx<infosSorted.size(); idx++) {
|
||||
final SegmentInfo info = infosSorted.get(idx);
|
||||
if (merging.contains(info)) {
|
||||
|
@ -332,7 +334,7 @@ public class TieredMergePolicy extends MergePolicy {
|
|||
|
||||
// OK we are over budget -- find best merge!
|
||||
MergeScore bestScore = null;
|
||||
SegmentInfos best = null;
|
||||
List<SegmentInfo> best = null;
|
||||
boolean bestTooLarge = false;
|
||||
long bestMergeBytes = 0;
|
||||
|
||||
|
@ -341,10 +343,10 @@ public class TieredMergePolicy extends MergePolicy {
|
|||
|
||||
long totAfterMergeBytes = 0;
|
||||
|
||||
final SegmentInfos candidate = new SegmentInfos();
|
||||
final List<SegmentInfo> candidate = new ArrayList<SegmentInfo>();
|
||||
boolean hitTooLarge = false;
|
||||
for(int idx = startIdx;idx<eligible.size() && candidate.size() < maxMergeAtOnce;idx++) {
|
||||
final SegmentInfo info = eligible.info(idx);
|
||||
final SegmentInfo info = eligible.get(idx);
|
||||
final long segBytes = size(info);
|
||||
|
||||
if (totAfterMergeBytes + segBytes > maxMergedSegmentBytes) {
|
||||
|
@ -398,7 +400,7 @@ public class TieredMergePolicy extends MergePolicy {
|
|||
}
|
||||
|
||||
/** Expert: scores one merge; subclasses can override. */
|
||||
protected MergeScore score(SegmentInfos candidate, boolean hitTooLarge, long mergingBytes) throws IOException {
|
||||
protected MergeScore score(List<SegmentInfo> candidate, boolean hitTooLarge, long mergingBytes) throws IOException {
|
||||
long totBeforeMergeBytes = 0;
|
||||
long totAfterMergeBytes = 0;
|
||||
long totAfterMergeBytesFloored = 0;
|
||||
|
@ -420,7 +422,7 @@ public class TieredMergePolicy extends MergePolicy {
|
|||
// over time:
|
||||
skew = 1.0/maxMergeAtOnce;
|
||||
} else {
|
||||
skew = ((double) floorSize(size(candidate.info(0))))/totAfterMergeBytesFloored;
|
||||
skew = ((double) floorSize(size(candidate.get(0))))/totAfterMergeBytesFloored;
|
||||
}
|
||||
|
||||
// Strongly favor merges with less skew (smaller
|
||||
|
@ -458,7 +460,8 @@ public class TieredMergePolicy extends MergePolicy {
|
|||
if (verbose()) {
|
||||
message("findMergesForOptimize maxSegmentCount=" + maxSegmentCount + " infos=" + writer.get().segString(infos) + " segmentsToOptimize=" + segmentsToOptimize);
|
||||
}
|
||||
SegmentInfos eligible = new SegmentInfos();
|
||||
|
||||
List<SegmentInfo> eligible = new ArrayList<SegmentInfo>();
|
||||
boolean optimizeMergeRunning = false;
|
||||
final Collection<SegmentInfo> merging = writer.get().getMergingSegments();
|
||||
for(SegmentInfo info : infos) {
|
||||
|
@ -499,7 +502,7 @@ public class TieredMergePolicy extends MergePolicy {
|
|||
if (spec == null) {
|
||||
spec = new MergeSpecification();
|
||||
}
|
||||
final OneMerge merge = new OneMerge(eligible.range(end-maxMergeAtOnceExplicit, end));
|
||||
final OneMerge merge = new OneMerge(eligible.subList(end-maxMergeAtOnceExplicit, end));
|
||||
if (verbose()) {
|
||||
message("add merge=" + writer.get().segString(merge.segments));
|
||||
}
|
||||
|
@ -510,7 +513,7 @@ public class TieredMergePolicy extends MergePolicy {
|
|||
if (spec == null && !optimizeMergeRunning) {
|
||||
// Do final merge
|
||||
final int numToMerge = end - maxSegmentCount + 1;
|
||||
final OneMerge merge = new OneMerge(eligible.range(end-numToMerge, end));
|
||||
final OneMerge merge = new OneMerge(eligible.subList(end-numToMerge, end));
|
||||
if (verbose()) {
|
||||
message("add final merge=" + merge.segString(writer.get().getDirectory()));
|
||||
}
|
||||
|
@ -527,7 +530,7 @@ public class TieredMergePolicy extends MergePolicy {
|
|||
if (verbose()) {
|
||||
message("findMergesToExpungeDeletes infos=" + writer.get().segString(infos) + " expungeDeletesPctAllowed=" + expungeDeletesPctAllowed);
|
||||
}
|
||||
final SegmentInfos eligible = new SegmentInfos();
|
||||
final List<SegmentInfo> eligible = new ArrayList<SegmentInfo>();
|
||||
final Collection<SegmentInfo> merging = writer.get().getMergingSegments();
|
||||
for(SegmentInfo info : infos) {
|
||||
double pctDeletes = 100.*((double) writer.get().numDeletedDocs(info))/info.docCount;
|
||||
|
@ -580,7 +583,7 @@ public class TieredMergePolicy extends MergePolicy {
|
|||
spec = new MergeSpecification();
|
||||
}
|
||||
|
||||
final OneMerge merge = new OneMerge(eligible.range(start, upto));
|
||||
final OneMerge merge = new OneMerge(eligible.subList(start, upto));
|
||||
if (verbose()) {
|
||||
message("add merge=" + writer.get().segString(merge.segments));
|
||||
}
|
||||
|
|
|
@ -20,7 +20,9 @@ package org.apache.lucene.index;
|
|||
import org.apache.lucene.util.Constants;
|
||||
|
||||
import java.io.IOException;
|
||||
import java.util.ArrayList;
|
||||
import java.util.HashSet;
|
||||
import java.util.List;
|
||||
import java.util.Set;
|
||||
|
||||
/** This {@link MergePolicy} is used for upgrading all existing segments of
|
||||
|
@ -101,7 +103,7 @@ public class UpgradeIndexMergePolicy extends MergePolicy {
|
|||
if (verbose())
|
||||
message("findMergesForOptimize: " + base.getClass().getSimpleName() +
|
||||
" does not want to merge all old segments, merge remaining ones into new segment: " + oldSegments);
|
||||
final SegmentInfos newInfos = new SegmentInfos();
|
||||
final List<SegmentInfo> newInfos = new ArrayList<SegmentInfo>();
|
||||
for (final SegmentInfo si : segmentInfos) {
|
||||
if (oldSegments.contains(si)) {
|
||||
newInfos.add(si);
|
||||
|
|
|
@ -18,7 +18,9 @@ package org.apache.lucene.index;
|
|||
*/
|
||||
|
||||
import java.io.IOException;
|
||||
import java.util.ArrayList;
|
||||
import java.util.Collections;
|
||||
import java.util.List;
|
||||
import java.util.Random;
|
||||
import java.util.Set;
|
||||
|
||||
|
@ -58,7 +60,7 @@ public class MockRandomMergePolicy extends MergePolicy {
|
|||
SegmentInfos segmentInfos, int maxSegmentCount, Set<SegmentInfo> segmentsToOptimize)
|
||||
throws CorruptIndexException, IOException {
|
||||
|
||||
final SegmentInfos eligibleSegments = new SegmentInfos();
|
||||
final List<SegmentInfo> eligibleSegments = new ArrayList<SegmentInfo>();
|
||||
for(SegmentInfo info : segmentInfos) {
|
||||
if (segmentsToOptimize.contains(info)) {
|
||||
eligibleSegments.add(info);
|
||||
|
@ -76,7 +78,7 @@ public class MockRandomMergePolicy extends MergePolicy {
|
|||
while(upto < eligibleSegments.size()) {
|
||||
int max = Math.min(10, eligibleSegments.size()-upto);
|
||||
int inc = max <= 2 ? max : _TestUtil.nextInt(random, 2, max);
|
||||
mergeSpec.add(new OneMerge(eligibleSegments.range(upto, upto+inc)));
|
||||
mergeSpec.add(new OneMerge(eligibleSegments.subList(upto, upto+inc)));
|
||||
upto += inc;
|
||||
}
|
||||
}
|
||||
|
|
Loading…
Reference in New Issue