mirror of https://github.com/apache/lucene.git
LUCENE-3661: cut back to Bits (from MutableBits) in some places in IW
git-svn-id: https://svn.apache.org/repos/asf/lucene/dev/branches/lucene3661@1233931 13f79535-47bb-0310-9956-ffa450edef68
This commit is contained in:
parent
6117558c44
commit
5c920cf73a
|
@ -47,6 +47,7 @@ import org.apache.lucene.store.IOContext;
|
||||||
import org.apache.lucene.store.Lock;
|
import org.apache.lucene.store.Lock;
|
||||||
import org.apache.lucene.store.LockObtainFailedException;
|
import org.apache.lucene.store.LockObtainFailedException;
|
||||||
import org.apache.lucene.store.MergeInfo;
|
import org.apache.lucene.store.MergeInfo;
|
||||||
|
import org.apache.lucene.util.Bits;
|
||||||
import org.apache.lucene.util.Constants;
|
import org.apache.lucene.util.Constants;
|
||||||
import org.apache.lucene.util.IOUtils;
|
import org.apache.lucene.util.IOUtils;
|
||||||
import org.apache.lucene.util.InfoStream;
|
import org.apache.lucene.util.InfoStream;
|
||||||
|
@ -588,7 +589,7 @@ public class IndexWriter implements Closeable, TwoPhaseCommit {
|
||||||
}
|
}
|
||||||
|
|
||||||
// nocommit: if this is read-only live docs, why doesn't it return Bits?!
|
// nocommit: if this is read-only live docs, why doesn't it return Bits?!
|
||||||
public synchronized MutableBits getReadOnlyLiveDocs() {
|
public synchronized Bits getReadOnlyLiveDocs() {
|
||||||
//System.out.println("getROLiveDocs seg=" + info);
|
//System.out.println("getROLiveDocs seg=" + info);
|
||||||
assert Thread.holdsLock(IndexWriter.this);
|
assert Thread.holdsLock(IndexWriter.this);
|
||||||
shared = true;
|
shared = true;
|
||||||
|
@ -2993,7 +2994,7 @@ public class IndexWriter implements Closeable, TwoPhaseCommit {
|
||||||
* saves the resulting deletes file (incrementing the
|
* saves the resulting deletes file (incrementing the
|
||||||
* delete generation for merge.info). If no deletes were
|
* delete generation for merge.info). If no deletes were
|
||||||
* flushed, no new deletes file is saved. */
|
* flushed, no new deletes file is saved. */
|
||||||
synchronized private ReadersAndLiveDocs commitMergedDeletes(MergePolicy.OneMerge merge) throws IOException {
|
synchronized private ReadersAndLiveDocs commitMergedDeletes(MergePolicy.OneMerge merge, MergeState mergeState) throws IOException {
|
||||||
|
|
||||||
assert testPoint("startCommitMergeDeletes");
|
assert testPoint("startCommitMergeDeletes");
|
||||||
|
|
||||||
|
@ -3015,8 +3016,8 @@ public class IndexWriter implements Closeable, TwoPhaseCommit {
|
||||||
SegmentInfo info = sourceSegments.get(i);
|
SegmentInfo info = sourceSegments.get(i);
|
||||||
minGen = Math.min(info.getBufferedDeletesGen(), minGen);
|
minGen = Math.min(info.getBufferedDeletesGen(), minGen);
|
||||||
final int docCount = info.docCount;
|
final int docCount = info.docCount;
|
||||||
final MutableBits prevLiveDocs = merge.readerLiveDocs.get(i);
|
final Bits prevLiveDocs = merge.readerLiveDocs.get(i);
|
||||||
final MutableBits currentLiveDocs;
|
final Bits currentLiveDocs;
|
||||||
ReadersAndLiveDocs rld = readerPool.get(info, false);
|
ReadersAndLiveDocs rld = readerPool.get(info, false);
|
||||||
// We enrolled in mergeInit:
|
// We enrolled in mergeInit:
|
||||||
assert rld != null;
|
assert rld != null;
|
||||||
|
@ -3035,7 +3036,7 @@ public class IndexWriter implements Closeable, TwoPhaseCommit {
|
||||||
// newly flushed deletes but mapping them to the new
|
// newly flushed deletes but mapping them to the new
|
||||||
// docIDs.
|
// docIDs.
|
||||||
|
|
||||||
if (currentLiveDocs.count() < prevLiveDocs.count()) {
|
if (currentLiveDocs != prevLiveDocs) {
|
||||||
// This means this segment received new deletes
|
// This means this segment received new deletes
|
||||||
// since we started the merge, so we
|
// since we started the merge, so we
|
||||||
// must merge them:
|
// must merge them:
|
||||||
|
@ -3054,8 +3055,13 @@ public class IndexWriter implements Closeable, TwoPhaseCommit {
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
} else {
|
} else {
|
||||||
assert currentLiveDocs.count() == prevLiveDocs.count(): "currentLiveDocs.count()==" + currentLiveDocs.count() + " vs prevLiveDocs.count()=" + prevLiveDocs.count() + " info=" + info;
|
final int readerDocCount;
|
||||||
docUpto += currentLiveDocs.count();
|
if (i == sourceSegments.size()-1) {
|
||||||
|
readerDocCount = mergeState.mergedDocCount - mergeState.docBase[i];
|
||||||
|
} else {
|
||||||
|
readerDocCount = mergeState.docBase[i+1] - mergeState.docBase[i];
|
||||||
|
}
|
||||||
|
docUpto += readerDocCount;
|
||||||
}
|
}
|
||||||
} else if (currentLiveDocs != null) {
|
} else if (currentLiveDocs != null) {
|
||||||
// This segment had no deletes before but now it
|
// This segment had no deletes before but now it
|
||||||
|
@ -3096,7 +3102,7 @@ public class IndexWriter implements Closeable, TwoPhaseCommit {
|
||||||
return mergedDeletes;
|
return mergedDeletes;
|
||||||
}
|
}
|
||||||
|
|
||||||
synchronized private boolean commitMerge(MergePolicy.OneMerge merge) throws IOException {
|
synchronized private boolean commitMerge(MergePolicy.OneMerge merge, MergeState mergeState) throws IOException {
|
||||||
|
|
||||||
assert testPoint("startCommitMerge");
|
assert testPoint("startCommitMerge");
|
||||||
|
|
||||||
|
@ -3123,7 +3129,7 @@ public class IndexWriter implements Closeable, TwoPhaseCommit {
|
||||||
return false;
|
return false;
|
||||||
}
|
}
|
||||||
|
|
||||||
final ReadersAndLiveDocs mergedDeletes = commitMergedDeletes(merge);
|
final ReadersAndLiveDocs mergedDeletes = commitMergedDeletes(merge, mergeState);
|
||||||
|
|
||||||
assert mergedDeletes == null || mergedDeletes.pendingDeleteCount != 0;
|
assert mergedDeletes == null || mergedDeletes.pendingDeleteCount != 0;
|
||||||
|
|
||||||
|
@ -3556,13 +3562,12 @@ public class IndexWriter implements Closeable, TwoPhaseCommit {
|
||||||
}
|
}
|
||||||
|
|
||||||
merge.readers = new ArrayList<SegmentReader>();
|
merge.readers = new ArrayList<SegmentReader>();
|
||||||
merge.readerLiveDocs = new ArrayList<MutableBits>();
|
merge.readerLiveDocs = new ArrayList<Bits>();
|
||||||
|
|
||||||
// This is try/finally to make sure merger's readers are
|
// This is try/finally to make sure merger's readers are
|
||||||
// closed:
|
// closed:
|
||||||
boolean success = false;
|
boolean success = false;
|
||||||
try {
|
try {
|
||||||
int totDocCount = 0;
|
|
||||||
int segUpto = 0;
|
int segUpto = 0;
|
||||||
while(segUpto < sourceSegments.size()) {
|
while(segUpto < sourceSegments.size()) {
|
||||||
|
|
||||||
|
@ -3575,7 +3580,7 @@ public class IndexWriter implements Closeable, TwoPhaseCommit {
|
||||||
assert reader != null;
|
assert reader != null;
|
||||||
|
|
||||||
// Carefully pull the most recent live docs:
|
// Carefully pull the most recent live docs:
|
||||||
final MutableBits liveDocs;
|
final Bits liveDocs;
|
||||||
synchronized(this) {
|
synchronized(this) {
|
||||||
// Must sync to ensure BufferedDeletesStream
|
// Must sync to ensure BufferedDeletesStream
|
||||||
// cannot change liveDocs/pendingDeleteCount while
|
// cannot change liveDocs/pendingDeleteCount while
|
||||||
|
@ -3596,19 +3601,10 @@ public class IndexWriter implements Closeable, TwoPhaseCommit {
|
||||||
merge.readerLiveDocs.add(liveDocs);
|
merge.readerLiveDocs.add(liveDocs);
|
||||||
merge.readers.add(reader);
|
merge.readers.add(reader);
|
||||||
|
|
||||||
if (liveDocs == null || liveDocs.count() > 0) {
|
merger.add(reader, liveDocs);
|
||||||
merger.add(reader, liveDocs);
|
|
||||||
totDocCount += liveDocs == null ? reader.maxDoc() : liveDocs.count();
|
|
||||||
} else {
|
|
||||||
//System.out.println(" skip seg: fully deleted");
|
|
||||||
}
|
|
||||||
segUpto++;
|
segUpto++;
|
||||||
}
|
}
|
||||||
|
|
||||||
if (infoStream.isEnabled("IW")) {
|
|
||||||
infoStream.message("IW", "merge: total " + totDocCount + " docs");
|
|
||||||
}
|
|
||||||
|
|
||||||
merge.checkAborted(directory);
|
merge.checkAborted(directory);
|
||||||
|
|
||||||
// This is where all the work happens:
|
// This is where all the work happens:
|
||||||
|
@ -3619,11 +3615,9 @@ public class IndexWriter implements Closeable, TwoPhaseCommit {
|
||||||
merge.info.setCodec(codec);
|
merge.info.setCodec(codec);
|
||||||
|
|
||||||
if (infoStream.isEnabled("IW")) {
|
if (infoStream.isEnabled("IW")) {
|
||||||
infoStream.message("IW", "merge codec=" + codec);
|
infoStream.message("IW", "merge codec=" + codec + " docCount=" + mergedDocCount);
|
||||||
}
|
}
|
||||||
|
|
||||||
assert mergedDocCount == totDocCount: "mergedDocCount=" + mergedDocCount + " vs " + totDocCount;
|
|
||||||
|
|
||||||
// Very important to do this before opening the reader
|
// Very important to do this before opening the reader
|
||||||
// because codec must know if prox was written for
|
// because codec must know if prox was written for
|
||||||
// this segment:
|
// this segment:
|
||||||
|
@ -3709,7 +3703,7 @@ public class IndexWriter implements Closeable, TwoPhaseCommit {
|
||||||
|
|
||||||
// Force READ context because we merge deletes onto
|
// Force READ context because we merge deletes onto
|
||||||
// this reader:
|
// this reader:
|
||||||
if (!commitMerge(merge)) {
|
if (!commitMerge(merge, mergeState)) {
|
||||||
// commitMerge will return false if this merge was aborted
|
// commitMerge will return false if this merge was aborted
|
||||||
return 0;
|
return 0;
|
||||||
}
|
}
|
||||||
|
@ -3767,7 +3761,6 @@ public class IndexWriter implements Closeable, TwoPhaseCommit {
|
||||||
|
|
||||||
/** @lucene.internal */
|
/** @lucene.internal */
|
||||||
public synchronized String segString(SegmentInfo info) throws IOException {
|
public synchronized String segString(SegmentInfo info) throws IOException {
|
||||||
StringBuilder buffer = new StringBuilder();
|
|
||||||
return info.toString(info.dir, numDeletedDocs(info) - info.getDelCount());
|
return info.toString(info.dir, numDeletedDocs(info) - info.getDelCount());
|
||||||
}
|
}
|
||||||
|
|
||||||
|
|
|
@ -24,7 +24,7 @@ import java.util.Map;
|
||||||
|
|
||||||
import org.apache.lucene.store.Directory;
|
import org.apache.lucene.store.Directory;
|
||||||
import org.apache.lucene.store.MergeInfo;
|
import org.apache.lucene.store.MergeInfo;
|
||||||
import org.apache.lucene.util.MutableBits;
|
import org.apache.lucene.util.Bits;
|
||||||
import org.apache.lucene.util.SetOnce.AlreadySetException;
|
import org.apache.lucene.util.SetOnce.AlreadySetException;
|
||||||
import org.apache.lucene.util.SetOnce;
|
import org.apache.lucene.util.SetOnce;
|
||||||
|
|
||||||
|
@ -74,7 +74,7 @@ public abstract class MergePolicy implements java.io.Closeable {
|
||||||
int maxNumSegments = -1; // used by IndexWriter
|
int maxNumSegments = -1; // used by IndexWriter
|
||||||
public long estimatedMergeBytes; // used by IndexWriter
|
public long estimatedMergeBytes; // used by IndexWriter
|
||||||
List<SegmentReader> readers; // used by IndexWriter
|
List<SegmentReader> readers; // used by IndexWriter
|
||||||
List<MutableBits> readerLiveDocs; // used by IndexWriter
|
List<Bits> readerLiveDocs; // used by IndexWriter
|
||||||
public final List<SegmentInfo> segments;
|
public final List<SegmentInfo> segments;
|
||||||
public final int totalDocCount;
|
public final int totalDocCount;
|
||||||
boolean aborted;
|
boolean aborted;
|
||||||
|
|
Loading…
Reference in New Issue