mirror of https://github.com/apache/lucene.git
LUCENE-2680: buffer pending deletes by segment
git-svn-id: https://svn.apache.org/repos/asf/lucene/dev/trunk@1044635 13f79535-47bb-0310-9956-ffa450edef68
This commit is contained in:
parent
3474026904
commit
bbf683cdbe
|
@ -17,153 +17,415 @@ package org.apache.lucene.index;
|
|||
* limitations under the License.
|
||||
*/
|
||||
|
||||
import java.io.IOException;
|
||||
import java.io.PrintStream;
|
||||
import java.util.HashMap;
|
||||
import java.util.Map;
|
||||
import java.util.TreeMap;
|
||||
import java.util.ArrayList;
|
||||
import java.util.List;
|
||||
import java.util.Date;
|
||||
import java.util.Map.Entry;
|
||||
import java.util.Map;
|
||||
import java.util.concurrent.atomic.AtomicInteger;
|
||||
import java.util.concurrent.atomic.AtomicLong;
|
||||
|
||||
import org.apache.lucene.search.IndexSearcher;
|
||||
import org.apache.lucene.search.Query;
|
||||
import org.apache.lucene.search.Scorer;
|
||||
import org.apache.lucene.search.Weight;
|
||||
|
||||
/** Holds a {@link SegmentDeletes} for each segment in the
|
||||
* index. */
|
||||
|
||||
/** Holds buffered deletes, by docID, term or query. We
|
||||
* hold two instances of this class: one for the deletes
|
||||
* prior to the last flush, the other for deletes after
|
||||
* the last flush. This is so if we need to abort
|
||||
* (discard all buffered docs) we can also discard the
|
||||
* buffered deletes yet keep the deletes done during
|
||||
* previously flushed segments. */
|
||||
class BufferedDeletes {
|
||||
int numTerms;
|
||||
Map<Term,Num> terms;
|
||||
Map<Query,Integer> queries = new HashMap<Query,Integer>();
|
||||
List<Integer> docIDs = new ArrayList<Integer>();
|
||||
long bytesUsed;
|
||||
private final boolean doTermSort;
|
||||
|
||||
public BufferedDeletes(boolean doTermSort) {
|
||||
this.doTermSort = doTermSort;
|
||||
if (doTermSort) {
|
||||
terms = new TreeMap<Term,Num>();
|
||||
// Deletes for all flushed/merged segments:
|
||||
private final Map<SegmentInfo,SegmentDeletes> deletesMap = new HashMap<SegmentInfo,SegmentDeletes>();
|
||||
|
||||
// used only by assert
|
||||
private Term lastDeleteTerm;
|
||||
|
||||
private PrintStream infoStream;
|
||||
private final AtomicLong bytesUsed = new AtomicLong();
|
||||
private final AtomicInteger numTerms = new AtomicInteger();
|
||||
private final int messageID;
|
||||
|
||||
public BufferedDeletes(int messageID) {
|
||||
this.messageID = messageID;
|
||||
}
|
||||
|
||||
private synchronized void message(String message) {
|
||||
if (infoStream != null) {
|
||||
infoStream.println("BD " + messageID + " [" + new Date() + "; " + Thread.currentThread().getName() + "]: BD " + message);
|
||||
}
|
||||
}
|
||||
|
||||
public synchronized void setInfoStream(PrintStream infoStream) {
|
||||
this.infoStream = infoStream;
|
||||
}
|
||||
|
||||
public synchronized void pushDeletes(SegmentDeletes newDeletes, SegmentInfo info) {
|
||||
pushDeletes(newDeletes, info, false);
|
||||
}
|
||||
|
||||
// Moves all pending deletes onto the provided segment,
|
||||
// then clears the pending deletes
|
||||
public synchronized void pushDeletes(SegmentDeletes newDeletes, SegmentInfo info, boolean noLimit) {
|
||||
assert newDeletes.any();
|
||||
numTerms.addAndGet(newDeletes.numTermDeletes.get());
|
||||
|
||||
if (!noLimit) {
|
||||
assert !deletesMap.containsKey(info);
|
||||
assert info != null;
|
||||
deletesMap.put(info, newDeletes);
|
||||
bytesUsed.addAndGet(newDeletes.bytesUsed.get());
|
||||
} else {
|
||||
terms = new HashMap<Term,Num>();
|
||||
final SegmentDeletes deletes = getDeletes(info);
|
||||
bytesUsed.addAndGet(-deletes.bytesUsed.get());
|
||||
deletes.update(newDeletes, noLimit);
|
||||
bytesUsed.addAndGet(deletes.bytesUsed.get());
|
||||
}
|
||||
if (infoStream != null) {
|
||||
message("push deletes seg=" + info + " dels=" + getDeletes(info));
|
||||
}
|
||||
assert checkDeleteStats();
|
||||
}
|
||||
|
||||
public synchronized void clear() {
|
||||
deletesMap.clear();
|
||||
numTerms.set(0);
|
||||
bytesUsed.set(0);
|
||||
}
|
||||
|
||||
synchronized boolean any() {
|
||||
return bytesUsed.get() != 0;
|
||||
}
|
||||
|
||||
public int numTerms() {
|
||||
return numTerms.get();
|
||||
}
|
||||
|
||||
public long bytesUsed() {
|
||||
return bytesUsed.get();
|
||||
}
|
||||
|
||||
// IW calls this on finishing a merge. While the merge
|
||||
// was running, it's possible new deletes were pushed onto
|
||||
// our last (and only our last) segment. In this case we
|
||||
// must carry forward those deletes onto the merged
|
||||
// segment.
|
||||
synchronized void commitMerge(MergePolicy.OneMerge merge) {
|
||||
assert checkDeleteStats();
|
||||
if (infoStream != null) {
|
||||
message("commitMerge merge.info=" + merge.info + " merge.segments=" + merge.segments);
|
||||
}
|
||||
final SegmentInfo lastInfo = merge.segments.lastElement();
|
||||
final SegmentDeletes lastDeletes = deletesMap.get(lastInfo);
|
||||
if (lastDeletes != null) {
|
||||
deletesMap.remove(lastInfo);
|
||||
assert !deletesMap.containsKey(merge.info);
|
||||
deletesMap.put(merge.info, lastDeletes);
|
||||
// don't need to update numTerms/bytesUsed since we
|
||||
// are just moving the deletes from one info to
|
||||
// another
|
||||
if (infoStream != null) {
|
||||
message("commitMerge done: new deletions=" + lastDeletes);
|
||||
}
|
||||
} else if (infoStream != null) {
|
||||
message("commitMerge done: no new deletions");
|
||||
}
|
||||
assert !anyDeletes(merge.segments.range(0, merge.segments.size()-1));
|
||||
assert checkDeleteStats();
|
||||
}
|
||||
|
||||
synchronized void clear(SegmentDeletes deletes) {
|
||||
deletes.clear();
|
||||
}
|
||||
|
||||
public synchronized boolean applyDeletes(IndexWriter.ReaderPool readerPool, SegmentInfos segmentInfos, SegmentInfos applyInfos) throws IOException {
|
||||
if (!any()) {
|
||||
return false;
|
||||
}
|
||||
final long t0 = System.currentTimeMillis();
|
||||
|
||||
if (infoStream != null) {
|
||||
message("applyDeletes: applyInfos=" + applyInfos + "; index=" + segmentInfos);
|
||||
}
|
||||
|
||||
assert checkDeleteStats();
|
||||
|
||||
assert applyInfos.size() > 0;
|
||||
|
||||
boolean any = false;
|
||||
|
||||
final SegmentInfo lastApplyInfo = applyInfos.lastElement();
|
||||
final int lastIdx = segmentInfos.indexOf(lastApplyInfo);
|
||||
|
||||
final SegmentInfo firstInfo = applyInfos.firstElement();
|
||||
final int firstIdx = segmentInfos.indexOf(firstInfo);
|
||||
|
||||
// applyInfos must be a slice of segmentInfos
|
||||
assert lastIdx - firstIdx + 1 == applyInfos.size();
|
||||
|
||||
// iterate over all segment infos backwards
|
||||
// coalesceing deletes along the way
|
||||
// when we're at or below the last of the
|
||||
// segments to apply to, start applying the deletes
|
||||
// we traverse up to the first apply infos
|
||||
SegmentDeletes coalescedDeletes = null;
|
||||
boolean hasDeletes = false;
|
||||
for (int segIdx=segmentInfos.size()-1; segIdx >= firstIdx; segIdx--) {
|
||||
final SegmentInfo info = segmentInfos.info(segIdx);
|
||||
final SegmentDeletes deletes = deletesMap.get(info);
|
||||
assert deletes == null || deletes.any();
|
||||
|
||||
if (deletes == null && coalescedDeletes == null) {
|
||||
continue;
|
||||
}
|
||||
|
||||
if (infoStream != null) {
|
||||
message("applyDeletes: seg=" + info + " segment's deletes=[" + (deletes == null ? "null" : deletes) + "]; coalesced deletes=[" + (coalescedDeletes == null ? "null" : coalescedDeletes) + "]");
|
||||
}
|
||||
|
||||
hasDeletes |= deletes != null;
|
||||
|
||||
if (segIdx <= lastIdx && hasDeletes) {
|
||||
|
||||
any |= applyDeletes(readerPool, info, coalescedDeletes, deletes);
|
||||
|
||||
if (deletes != null) {
|
||||
// we've applied doc ids, and they're only applied
|
||||
// on the current segment
|
||||
bytesUsed.addAndGet(-deletes.docIDs.size() * SegmentDeletes.BYTES_PER_DEL_DOCID);
|
||||
deletes.clearDocIDs();
|
||||
}
|
||||
}
|
||||
|
||||
// Number of documents a delete term applies to.
|
||||
final static class Num {
|
||||
private int num;
|
||||
|
||||
Num(int num) {
|
||||
this.num = num;
|
||||
// now coalesce at the max limit
|
||||
if (deletes != null) {
|
||||
if (coalescedDeletes == null) {
|
||||
coalescedDeletes = new SegmentDeletes();
|
||||
}
|
||||
|
||||
int getNum() {
|
||||
return num;
|
||||
}
|
||||
|
||||
void setNum(int num) {
|
||||
// Only record the new number if it's greater than the
|
||||
// current one. This is important because if multiple
|
||||
// threads are replacing the same doc at nearly the
|
||||
// same time, it's possible that one thread that got a
|
||||
// higher docID is scheduled before the other
|
||||
// threads.
|
||||
if (num > this.num)
|
||||
this.num = num;
|
||||
// TODO: we could make this single pass (coalesce as
|
||||
// we apply the deletes
|
||||
coalescedDeletes.update(deletes, true);
|
||||
}
|
||||
}
|
||||
|
||||
int size() {
|
||||
// We use numTerms not terms.size() intentionally, so
|
||||
// that deletes by the same term multiple times "count",
|
||||
// ie if you ask to flush every 1000 deletes then even
|
||||
// dup'd terms are counted towards that 1000
|
||||
return numTerms + queries.size() + docIDs.size();
|
||||
// move all deletes to segment just before our merge.
|
||||
if (firstIdx > 0) {
|
||||
|
||||
SegmentDeletes mergedDeletes = null;
|
||||
// TODO: we could also make this single pass
|
||||
for (SegmentInfo info : applyInfos) {
|
||||
final SegmentDeletes deletes = deletesMap.get(info);
|
||||
if (deletes != null) {
|
||||
assert deletes.any();
|
||||
if (mergedDeletes == null) {
|
||||
mergedDeletes = getDeletes(segmentInfos.info(firstIdx-1));
|
||||
numTerms.addAndGet(-mergedDeletes.numTermDeletes.get());
|
||||
bytesUsed.addAndGet(-mergedDeletes.bytesUsed.get());
|
||||
}
|
||||
|
||||
void update(BufferedDeletes in) {
|
||||
numTerms += in.numTerms;
|
||||
bytesUsed += in.bytesUsed;
|
||||
terms.putAll(in.terms);
|
||||
queries.putAll(in.queries);
|
||||
docIDs.addAll(in.docIDs);
|
||||
in.clear();
|
||||
mergedDeletes.update(deletes, true);
|
||||
}
|
||||
}
|
||||
|
||||
void clear() {
|
||||
terms.clear();
|
||||
queries.clear();
|
||||
docIDs.clear();
|
||||
numTerms = 0;
|
||||
bytesUsed = 0;
|
||||
if (mergedDeletes != null) {
|
||||
numTerms.addAndGet(mergedDeletes.numTermDeletes.get());
|
||||
bytesUsed.addAndGet(mergedDeletes.bytesUsed.get());
|
||||
}
|
||||
|
||||
void addBytesUsed(long b) {
|
||||
bytesUsed += b;
|
||||
}
|
||||
|
||||
boolean any() {
|
||||
return terms.size() > 0 || docIDs.size() > 0 || queries.size() > 0;
|
||||
}
|
||||
|
||||
// Remaps all buffered deletes based on a completed
|
||||
// merge
|
||||
synchronized void remap(MergeDocIDRemapper mapper,
|
||||
SegmentInfos infos,
|
||||
int[][] docMaps,
|
||||
int[] delCounts,
|
||||
MergePolicy.OneMerge merge,
|
||||
int mergeDocCount) {
|
||||
|
||||
final Map<Term,Num> newDeleteTerms;
|
||||
|
||||
// Remap delete-by-term
|
||||
if (terms.size() > 0) {
|
||||
if (doTermSort) {
|
||||
newDeleteTerms = new TreeMap<Term,Num>();
|
||||
if (infoStream != null) {
|
||||
if (mergedDeletes != null) {
|
||||
message("applyDeletes: merge all deletes into seg=" + segmentInfos.info(firstIdx-1) + ": " + mergedDeletes);
|
||||
} else {
|
||||
newDeleteTerms = new HashMap<Term,Num>();
|
||||
message("applyDeletes: no deletes to merge");
|
||||
}
|
||||
for(Entry<Term,Num> entry : terms.entrySet()) {
|
||||
Num num = entry.getValue();
|
||||
newDeleteTerms.put(entry.getKey(),
|
||||
new Num(mapper.remap(num.getNum())));
|
||||
}
|
||||
} else
|
||||
newDeleteTerms = null;
|
||||
|
||||
|
||||
// Remap delete-by-docID
|
||||
final List<Integer> newDeleteDocIDs;
|
||||
|
||||
if (docIDs.size() > 0) {
|
||||
newDeleteDocIDs = new ArrayList<Integer>(docIDs.size());
|
||||
for (Integer num : docIDs) {
|
||||
newDeleteDocIDs.add(Integer.valueOf(mapper.remap(num.intValue())));
|
||||
} else {
|
||||
// We drop the deletes in this case, because we've
|
||||
// applied them to segment infos starting w/ the first
|
||||
// segment. There are no prior segments so there's no
|
||||
// reason to keep them around. When the applyInfos ==
|
||||
// segmentInfos this means all deletes have been
|
||||
// removed:
|
||||
}
|
||||
} else
|
||||
newDeleteDocIDs = null;
|
||||
remove(applyInfos);
|
||||
|
||||
assert checkDeleteStats();
|
||||
assert applyInfos != segmentInfos || !any();
|
||||
|
||||
// Remap delete-by-query
|
||||
final HashMap<Query,Integer> newDeleteQueries;
|
||||
|
||||
if (queries.size() > 0) {
|
||||
newDeleteQueries = new HashMap<Query, Integer>(queries.size());
|
||||
for(Entry<Query,Integer> entry: queries.entrySet()) {
|
||||
Integer num = entry.getValue();
|
||||
newDeleteQueries.put(entry.getKey(),
|
||||
Integer.valueOf(mapper.remap(num.intValue())));
|
||||
if (infoStream != null) {
|
||||
message("applyDeletes took " + (System.currentTimeMillis()-t0) + " msec");
|
||||
}
|
||||
return any;
|
||||
}
|
||||
} else
|
||||
newDeleteQueries = null;
|
||||
|
||||
if (newDeleteTerms != null)
|
||||
terms = newDeleteTerms;
|
||||
if (newDeleteDocIDs != null)
|
||||
docIDs = newDeleteDocIDs;
|
||||
if (newDeleteQueries != null)
|
||||
queries = newDeleteQueries;
|
||||
private synchronized boolean applyDeletes(IndexWriter.ReaderPool readerPool,
|
||||
SegmentInfo info,
|
||||
SegmentDeletes coalescedDeletes,
|
||||
SegmentDeletes segmentDeletes) throws IOException {
|
||||
assert readerPool.infoIsLive(info);
|
||||
|
||||
assert coalescedDeletes == null || coalescedDeletes.docIDs.size() == 0;
|
||||
|
||||
boolean any = false;
|
||||
|
||||
// Lock order: IW -> BD -> RP
|
||||
SegmentReader reader = readerPool.get(info, false);
|
||||
try {
|
||||
if (coalescedDeletes != null) {
|
||||
any |= applyDeletes(coalescedDeletes, reader);
|
||||
}
|
||||
if (segmentDeletes != null) {
|
||||
any |= applyDeletes(segmentDeletes, reader);
|
||||
}
|
||||
} finally {
|
||||
readerPool.release(reader);
|
||||
}
|
||||
return any;
|
||||
}
|
||||
|
||||
private synchronized boolean applyDeletes(SegmentDeletes deletes, SegmentReader reader) throws IOException {
|
||||
boolean any = false;
|
||||
|
||||
assert checkDeleteTerm(null);
|
||||
|
||||
if (deletes.terms.size() > 0) {
|
||||
Fields fields = reader.fields();
|
||||
if (fields == null) {
|
||||
// This reader has no postings
|
||||
return false;
|
||||
}
|
||||
|
||||
TermsEnum termsEnum = null;
|
||||
|
||||
String currentField = null;
|
||||
DocsEnum docs = null;
|
||||
|
||||
for (Entry<Term,Integer> entry: deletes.terms.entrySet()) {
|
||||
Term term = entry.getKey();
|
||||
// Since we visit terms sorted, we gain performance
|
||||
// by re-using the same TermsEnum and seeking only
|
||||
// forwards
|
||||
if (term.field() != currentField) {
|
||||
assert currentField == null || currentField.compareTo(term.field()) < 0;
|
||||
currentField = term.field();
|
||||
Terms terms = fields.terms(currentField);
|
||||
if (terms != null) {
|
||||
termsEnum = terms.iterator();
|
||||
} else {
|
||||
termsEnum = null;
|
||||
}
|
||||
}
|
||||
|
||||
if (termsEnum == null) {
|
||||
continue;
|
||||
}
|
||||
assert checkDeleteTerm(term);
|
||||
|
||||
if (termsEnum.seek(term.bytes(), false) == TermsEnum.SeekStatus.FOUND) {
|
||||
DocsEnum docsEnum = termsEnum.docs(reader.getDeletedDocs(), docs);
|
||||
|
||||
if (docsEnum != null) {
|
||||
docs = docsEnum;
|
||||
final int limit = entry.getValue();
|
||||
while (true) {
|
||||
final int docID = docs.nextDoc();
|
||||
if (docID == DocsEnum.NO_MORE_DOCS || docID >= limit) {
|
||||
break;
|
||||
}
|
||||
reader.deleteDocument(docID);
|
||||
any = true;
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
// Delete by docID
|
||||
for (Integer docIdInt : deletes.docIDs) {
|
||||
int docID = docIdInt.intValue();
|
||||
reader.deleteDocument(docID);
|
||||
any = true;
|
||||
}
|
||||
|
||||
// Delete by query
|
||||
if (deletes.queries.size() > 0) {
|
||||
IndexSearcher searcher = new IndexSearcher(reader);
|
||||
try {
|
||||
for (Entry<Query, Integer> entry : deletes.queries.entrySet()) {
|
||||
Query query = entry.getKey();
|
||||
int limit = entry.getValue().intValue();
|
||||
Weight weight = query.weight(searcher);
|
||||
Scorer scorer = weight.scorer(reader, true, false);
|
||||
if (scorer != null) {
|
||||
while(true) {
|
||||
int doc = scorer.nextDoc();
|
||||
if (doc >= limit)
|
||||
break;
|
||||
reader.deleteDocument(doc);
|
||||
any = true;
|
||||
}
|
||||
}
|
||||
}
|
||||
} finally {
|
||||
searcher.close();
|
||||
}
|
||||
}
|
||||
return any;
|
||||
}
|
||||
|
||||
public synchronized SegmentDeletes getDeletes(SegmentInfo info) {
|
||||
SegmentDeletes deletes = deletesMap.get(info);
|
||||
if (deletes == null) {
|
||||
deletes = new SegmentDeletes();
|
||||
deletesMap.put(info, deletes);
|
||||
}
|
||||
return deletes;
|
||||
}
|
||||
|
||||
public synchronized void remove(SegmentInfos infos) {
|
||||
assert infos.size() > 0;
|
||||
for (SegmentInfo info : infos) {
|
||||
SegmentDeletes deletes = deletesMap.get(info);
|
||||
if (deletes != null) {
|
||||
bytesUsed.addAndGet(-deletes.bytesUsed.get());
|
||||
assert bytesUsed.get() >= 0: "bytesUsed=" + bytesUsed;
|
||||
numTerms.addAndGet(-deletes.numTermDeletes.get());
|
||||
assert numTerms.get() >= 0: "numTerms=" + numTerms;
|
||||
deletesMap.remove(info);
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
// used only by assert
|
||||
private boolean anyDeletes(SegmentInfos infos) {
|
||||
for(SegmentInfo info : infos) {
|
||||
if (deletesMap.containsKey(info)) {
|
||||
return true;
|
||||
}
|
||||
}
|
||||
return false;
|
||||
}
|
||||
|
||||
// used only by assert
|
||||
private boolean checkDeleteTerm(Term term) {
|
||||
if (term != null) {
|
||||
assert lastDeleteTerm == null || term.compareTo(lastDeleteTerm) > 0: "lastTerm=" + lastDeleteTerm + " vs term=" + term;
|
||||
}
|
||||
lastDeleteTerm = term;
|
||||
return true;
|
||||
}
|
||||
|
||||
// only for assert
|
||||
private boolean checkDeleteStats() {
|
||||
int numTerms2 = 0;
|
||||
long bytesUsed2 = 0;
|
||||
for(SegmentDeletes deletes : deletesMap.values()) {
|
||||
numTerms2 += deletes.numTermDeletes.get();
|
||||
bytesUsed2 += deletes.bytesUsed.get();
|
||||
}
|
||||
assert numTerms2 == numTerms.get(): "numTerms2=" + numTerms2 + " vs " + numTerms.get();
|
||||
assert bytesUsed2 == bytesUsed.get(): "bytesUsed2=" + bytesUsed2 + " vs " + bytesUsed;
|
||||
return true;
|
||||
}
|
||||
}
|
|
@ -160,7 +160,7 @@ public class CompoundFileReader extends Directory {
|
|||
id = IndexFileNames.stripSegmentName(id);
|
||||
FileEntry entry = entries.get(id);
|
||||
if (entry == null)
|
||||
throw new IOException("No sub-file with id " + id + " found");
|
||||
throw new IOException("No sub-file with id " + id + " found (files: " + entries.keySet() + ")");
|
||||
|
||||
return new CSIndexInput(stream, entry.offset, entry.length, readBufferSize);
|
||||
}
|
||||
|
|
File diff suppressed because it is too large
Load Diff
|
@ -27,7 +27,6 @@ final class DocumentsWriterThreadState {
|
|||
|
||||
boolean isIdle = true; // false if this is currently in use by a thread
|
||||
int numThreads = 1; // Number of threads that share this instance
|
||||
boolean doFlushAfter; // true if we should flush after processing current doc
|
||||
final DocConsumerPerThread consumer;
|
||||
final DocumentsWriter.DocState docState;
|
||||
|
||||
|
@ -45,6 +44,5 @@ final class DocumentsWriterThreadState {
|
|||
|
||||
void doAfterFlush() {
|
||||
numThreads = 0;
|
||||
doFlushAfter = false;
|
||||
}
|
||||
}
|
||||
|
|
|
@ -21,6 +21,7 @@ import java.io.IOException;
|
|||
|
||||
import org.apache.lucene.analysis.tokenattributes.PayloadAttribute;
|
||||
import org.apache.lucene.document.Fieldable;
|
||||
import org.apache.lucene.util.RamUsageEstimator;
|
||||
|
||||
// TODO: break into separate freq and prox writers as
|
||||
// codecs; make separate container (tii/tis/skip/*) that can
|
||||
|
@ -88,7 +89,7 @@ final class FreqProxTermsWriterPerField extends TermsHashConsumerPerField implem
|
|||
}
|
||||
}
|
||||
|
||||
final void writeProx(final int termID, int proxCode) {
|
||||
void writeProx(final int termID, int proxCode) {
|
||||
final Payload payload;
|
||||
if (payloadAttribute == null) {
|
||||
payload = null;
|
||||
|
@ -110,7 +111,7 @@ final class FreqProxTermsWriterPerField extends TermsHashConsumerPerField implem
|
|||
}
|
||||
|
||||
@Override
|
||||
final void newTerm(final int termID) {
|
||||
void newTerm(final int termID) {
|
||||
// First time we're seeing this term since the last
|
||||
// flush
|
||||
assert docState.testPoint("FreqProxTermsWriterPerField.newTerm start");
|
||||
|
@ -127,7 +128,7 @@ final class FreqProxTermsWriterPerField extends TermsHashConsumerPerField implem
|
|||
}
|
||||
|
||||
@Override
|
||||
final void addTerm(final int termID) {
|
||||
void addTerm(final int termID) {
|
||||
|
||||
assert docState.testPoint("FreqProxTermsWriterPerField.addTerm start");
|
||||
|
||||
|
@ -205,7 +206,7 @@ final class FreqProxTermsWriterPerField extends TermsHashConsumerPerField implem
|
|||
|
||||
@Override
|
||||
int bytesPerPosting() {
|
||||
return ParallelPostingsArray.BYTES_PER_POSTING + 4 * DocumentsWriter.INT_NUM_BYTE;
|
||||
return ParallelPostingsArray.BYTES_PER_POSTING + 4 * RamUsageEstimator.NUM_BYTES_INT;
|
||||
}
|
||||
}
|
||||
|
||||
|
|
File diff suppressed because it is too large
Load Diff
|
@ -1,7 +1,5 @@
|
|||
package org.apache.lucene.index;
|
||||
|
||||
import org.apache.lucene.util.ArrayUtil;
|
||||
|
||||
/**
|
||||
* Licensed to the Apache Software Foundation (ASF) under one or more
|
||||
* contributor license agreements. See the NOTICE file distributed with
|
||||
|
@ -19,9 +17,11 @@ import org.apache.lucene.util.ArrayUtil;
|
|||
* limitations under the License.
|
||||
*/
|
||||
|
||||
import org.apache.lucene.util.ArrayUtil;
|
||||
import org.apache.lucene.util.RamUsageEstimator;
|
||||
|
||||
class ParallelPostingsArray {
|
||||
final static int BYTES_PER_POSTING = 3 * DocumentsWriter.INT_NUM_BYTE;
|
||||
final static int BYTES_PER_POSTING = 3 * RamUsageEstimator.NUM_BYTES_INT;
|
||||
|
||||
final int size;
|
||||
final int[] textStarts;
|
||||
|
|
|
@ -0,0 +1,188 @@
|
|||
package org.apache.lucene.index;
|
||||
|
||||
/**
|
||||
* Licensed to the Apache Software Foundation (ASF) under one or more
|
||||
* contributor license agreements. See the NOTICE file distributed with
|
||||
* this work for additional information regarding copyright ownership.
|
||||
* The ASF licenses this file to You under the Apache License, Version 2.0
|
||||
* (the "License"); you may not use this file except in compliance with
|
||||
* the License. You may obtain a copy of the License at
|
||||
*
|
||||
* http://www.apache.org/licenses/LICENSE-2.0
|
||||
*
|
||||
* Unless required by applicable law or agreed to in writing, software
|
||||
* distributed under the License is distributed on an "AS IS" BASIS,
|
||||
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
|
||||
* See the License for the specific language governing permissions and
|
||||
* limitations under the License.
|
||||
*/
|
||||
|
||||
import java.util.ArrayList;
|
||||
import java.util.HashMap;
|
||||
import java.util.List;
|
||||
import java.util.Map;
|
||||
import java.util.SortedMap;
|
||||
import java.util.TreeMap;
|
||||
import java.util.concurrent.atomic.AtomicLong;
|
||||
import java.util.concurrent.atomic.AtomicInteger;
|
||||
|
||||
import org.apache.lucene.search.Query;
|
||||
import org.apache.lucene.util.RamUsageEstimator;
|
||||
|
||||
/** Holds buffered deletes, by docID, term or query for a
|
||||
* single segment. This is used to hold buffered pending
|
||||
* deletes against the to-be-flushed segment as well as
|
||||
* per-segment deletes for each segment in the index. */
|
||||
|
||||
// NOTE: we are sync'd by BufferedDeletes, ie, all access to
|
||||
// instances of this class is via sync'd methods on
|
||||
// BufferedDeletes
|
||||
class SegmentDeletes {
|
||||
|
||||
/* Rough logic: HashMap has an array[Entry] w/ varying
|
||||
load factor (say 2 * POINTER). Entry is object w/ Term
|
||||
key, Integer val, int hash, Entry next
|
||||
(OBJ_HEADER + 3*POINTER + INT). Term is object w/
|
||||
String field and String text (OBJ_HEADER + 2*POINTER).
|
||||
We don't count Term's field since it's interned.
|
||||
Term's text is String (OBJ_HEADER + 4*INT + POINTER +
|
||||
OBJ_HEADER + string.length*CHAR). Integer is
|
||||
OBJ_HEADER + INT. */
|
||||
final static int BYTES_PER_DEL_TERM = 8*RamUsageEstimator.NUM_BYTES_OBJ_REF + 5*RamUsageEstimator.NUM_BYTES_OBJ_HEADER + 6*RamUsageEstimator.NUM_BYTES_INT;
|
||||
|
||||
/* Rough logic: del docIDs are List<Integer>. Say list
|
||||
allocates ~2X size (2*POINTER). Integer is OBJ_HEADER
|
||||
+ int */
|
||||
final static int BYTES_PER_DEL_DOCID = 2*RamUsageEstimator.NUM_BYTES_OBJ_REF + RamUsageEstimator.NUM_BYTES_OBJ_HEADER + RamUsageEstimator.NUM_BYTES_INT;
|
||||
|
||||
/* Rough logic: HashMap has an array[Entry] w/ varying
|
||||
load factor (say 2 * POINTER). Entry is object w/
|
||||
Query key, Integer val, int hash, Entry next
|
||||
(OBJ_HEADER + 3*POINTER + INT). Query we often
|
||||
undercount (say 24 bytes). Integer is OBJ_HEADER + INT. */
|
||||
final static int BYTES_PER_DEL_QUERY = 5*RamUsageEstimator.NUM_BYTES_OBJ_REF + 2*RamUsageEstimator.NUM_BYTES_OBJ_HEADER + 2*RamUsageEstimator.NUM_BYTES_INT + 24;
|
||||
|
||||
// TODO: many of the deletes stored here will map to
|
||||
// Integer.MAX_VALUE; we could be more efficient for this
|
||||
// case ie use a SortedSet not a SortedMap. But: Java's
|
||||
// SortedSet impls are simply backed by a Map so we won't
|
||||
// save anything unless we do something custom...
|
||||
final AtomicInteger numTermDeletes = new AtomicInteger();
|
||||
final SortedMap<Term,Integer> terms = new TreeMap<Term,Integer>();
|
||||
final Map<Query,Integer> queries = new HashMap<Query,Integer>();
|
||||
final List<Integer> docIDs = new ArrayList<Integer>();
|
||||
|
||||
public static final Integer MAX_INT = Integer.valueOf(Integer.MAX_VALUE);
|
||||
|
||||
final AtomicLong bytesUsed = new AtomicLong();
|
||||
|
||||
private final static boolean VERBOSE_DELETES = false;
|
||||
|
||||
@Override
|
||||
public String toString() {
|
||||
if (VERBOSE_DELETES) {
|
||||
return "SegmentDeletes [numTerms=" + numTermDeletes + ", terms=" + terms
|
||||
+ ", queries=" + queries + ", docIDs=" + docIDs + ", bytesUsed="
|
||||
+ bytesUsed + "]";
|
||||
} else {
|
||||
String s = "";
|
||||
if (numTermDeletes.get() != 0) {
|
||||
s += " " + numTermDeletes.get() + " deleted terms (unique count=" + terms.size() + ")";
|
||||
}
|
||||
if (queries.size() != 0) {
|
||||
s += " " + queries.size() + " deleted queries";
|
||||
}
|
||||
if (docIDs.size() != 0) {
|
||||
s += " " + docIDs.size() + " deleted docIDs";
|
||||
}
|
||||
if (bytesUsed.get() != 0) {
|
||||
s += " bytesUsed=" + bytesUsed.get();
|
||||
}
|
||||
|
||||
return s;
|
||||
}
|
||||
}
|
||||
|
||||
void update(SegmentDeletes in, boolean noLimit) {
|
||||
numTermDeletes.addAndGet(in.numTermDeletes.get());
|
||||
for (Map.Entry<Term,Integer> ent : in.terms.entrySet()) {
|
||||
final Term term = ent.getKey();
|
||||
if (!terms.containsKey(term)) {
|
||||
// only incr bytesUsed if this term wasn't already buffered:
|
||||
bytesUsed.addAndGet(BYTES_PER_DEL_TERM);
|
||||
}
|
||||
final Integer limit;
|
||||
if (noLimit) {
|
||||
limit = MAX_INT;
|
||||
} else {
|
||||
limit = ent.getValue();
|
||||
}
|
||||
terms.put(term, limit);
|
||||
}
|
||||
|
||||
for (Map.Entry<Query,Integer> ent : in.queries.entrySet()) {
|
||||
final Query query = ent.getKey();
|
||||
if (!queries.containsKey(query)) {
|
||||
// only incr bytesUsed if this query wasn't already buffered:
|
||||
bytesUsed.addAndGet(BYTES_PER_DEL_QUERY);
|
||||
}
|
||||
final Integer limit;
|
||||
if (noLimit) {
|
||||
limit = MAX_INT;
|
||||
} else {
|
||||
limit = ent.getValue();
|
||||
}
|
||||
queries.put(query, limit);
|
||||
}
|
||||
|
||||
// docIDs never move across segments and the docIDs
|
||||
// should already be cleared
|
||||
}
|
||||
|
||||
public void addQuery(Query query, int docIDUpto) {
|
||||
queries.put(query, docIDUpto);
|
||||
bytesUsed.addAndGet(BYTES_PER_DEL_QUERY);
|
||||
}
|
||||
|
||||
public void addDocID(int docID) {
|
||||
docIDs.add(Integer.valueOf(docID));
|
||||
bytesUsed.addAndGet(BYTES_PER_DEL_DOCID);
|
||||
}
|
||||
|
||||
public void addTerm(Term term, int docIDUpto) {
|
||||
Integer current = terms.get(term);
|
||||
if (current != null && docIDUpto < current) {
|
||||
// Only record the new number if it's greater than the
|
||||
// current one. This is important because if multiple
|
||||
// threads are replacing the same doc at nearly the
|
||||
// same time, it's possible that one thread that got a
|
||||
// higher docID is scheduled before the other
|
||||
// threads. If we blindly replace than we can get
|
||||
// double-doc in the segment.
|
||||
return;
|
||||
}
|
||||
|
||||
terms.put(term, Integer.valueOf(docIDUpto));
|
||||
numTermDeletes.incrementAndGet();
|
||||
if (current == null) {
|
||||
bytesUsed.addAndGet(BYTES_PER_DEL_TERM + term.bytes.length);
|
||||
}
|
||||
}
|
||||
|
||||
void clear() {
|
||||
terms.clear();
|
||||
queries.clear();
|
||||
docIDs.clear();
|
||||
numTermDeletes.set(0);
|
||||
bytesUsed.set(0);
|
||||
}
|
||||
|
||||
void clearDocIDs() {
|
||||
bytesUsed.addAndGet(-docIDs.size()*BYTES_PER_DEL_DOCID);
|
||||
docIDs.clear();
|
||||
}
|
||||
|
||||
boolean any() {
|
||||
return terms.size() > 0 || docIDs.size() > 0 || queries.size() > 0;
|
||||
}
|
||||
}
|
|
@ -361,6 +361,10 @@ public final class SegmentInfo {
|
|||
return docStoreSegment;
|
||||
}
|
||||
|
||||
public void setDocStoreSegment(String segment) {
|
||||
docStoreSegment = segment;
|
||||
}
|
||||
|
||||
void setDocStoreOffset(int offset) {
|
||||
docStoreOffset = offset;
|
||||
clearFiles();
|
||||
|
@ -534,6 +538,12 @@ public final class SegmentInfo {
|
|||
|
||||
if (docStoreOffset != -1) {
|
||||
s.append("->").append(docStoreSegment);
|
||||
if (docStoreIsCompoundFile) {
|
||||
s.append('c');
|
||||
} else {
|
||||
s.append('C');
|
||||
}
|
||||
s.append('+').append(docStoreOffset);
|
||||
}
|
||||
|
||||
return s.toString();
|
||||
|
|
|
@ -24,6 +24,7 @@ import org.apache.lucene.document.Fieldable;
|
|||
import org.apache.lucene.store.IndexOutput;
|
||||
import org.apache.lucene.util.ByteBlockPool;
|
||||
import org.apache.lucene.util.BytesRef;
|
||||
import org.apache.lucene.util.RamUsageEstimator;
|
||||
|
||||
final class TermVectorsTermsWriterPerField extends TermsHashConsumerPerField {
|
||||
|
||||
|
@ -298,7 +299,7 @@ final class TermVectorsTermsWriterPerField extends TermsHashConsumerPerField {
|
|||
|
||||
@Override
|
||||
int bytesPerPosting() {
|
||||
return super.bytesPerPosting() + 3 * DocumentsWriter.INT_NUM_BYTE;
|
||||
return super.bytesPerPosting() + 3 * RamUsageEstimator.NUM_BYTES_INT;
|
||||
}
|
||||
}
|
||||
}
|
||||
|
|
|
@ -50,6 +50,7 @@ public class TestDemo extends LuceneTestCase {
|
|||
// To store an index on disk, use this instead:
|
||||
//Directory directory = FSDirectory.open("/tmp/testindex");
|
||||
RandomIndexWriter iwriter = new RandomIndexWriter(random, directory);
|
||||
iwriter.w.setInfoStream(VERBOSE ? System.out : null);
|
||||
Document doc = new Document();
|
||||
String longTerm = "longtermlongtermlongtermlongtermlongtermlongtermlongtermlongtermlongtermlongtermlongtermlongtermlongtermlongtermlongtermlongtermlongtermlongterm";
|
||||
String text = "This is the text to be indexed. " + longTerm;
|
||||
|
|
|
@ -84,6 +84,10 @@ public class TestSearchForDuplicates extends LuceneTestCase {
|
|||
lmp.setUseCompoundFile(useCompoundFiles);
|
||||
lmp.setUseCompoundDocStore(useCompoundFiles);
|
||||
IndexWriter writer = new IndexWriter(directory, conf);
|
||||
if (VERBOSE) {
|
||||
System.out.println("TEST: now build index");
|
||||
writer.setInfoStream(System.out);
|
||||
}
|
||||
|
||||
final int MAX_DOCS = 225;
|
||||
|
||||
|
|
|
@ -57,9 +57,9 @@ public class TestConcurrentMergeScheduler extends LuceneTestCase {
|
|||
isClose = true;
|
||||
}
|
||||
}
|
||||
if (isDoFlush && !isClose) {
|
||||
if (isDoFlush && !isClose && random.nextBoolean()) {
|
||||
hitExc = true;
|
||||
throw new IOException("now failing during flush");
|
||||
throw new IOException(Thread.currentThread().getName() + ": now failing during flush");
|
||||
}
|
||||
}
|
||||
}
|
||||
|
@ -73,12 +73,17 @@ public class TestConcurrentMergeScheduler extends LuceneTestCase {
|
|||
directory.failOn(failure);
|
||||
|
||||
IndexWriter writer = new IndexWriter(directory, newIndexWriterConfig(TEST_VERSION_CURRENT, new MockAnalyzer()).setMaxBufferedDocs(2));
|
||||
writer.setInfoStream(VERBOSE ? System.out : null);
|
||||
Document doc = new Document();
|
||||
Field idField = newField("id", "", Field.Store.YES, Field.Index.NOT_ANALYZED);
|
||||
doc.add(idField);
|
||||
int extraCount = 0;
|
||||
|
||||
for(int i=0;i<10;i++) {
|
||||
if (VERBOSE) {
|
||||
System.out.println("TEST: iter=" + i);
|
||||
}
|
||||
|
||||
for(int j=0;j<20;j++) {
|
||||
idField.setValue(Integer.toString(i*20+j));
|
||||
writer.addDocument(doc);
|
||||
|
@ -97,10 +102,14 @@ public class TestConcurrentMergeScheduler extends LuceneTestCase {
|
|||
}
|
||||
extraCount++;
|
||||
} catch (IOException ioe) {
|
||||
if (VERBOSE) {
|
||||
ioe.printStackTrace(System.out);
|
||||
}
|
||||
failure.clearDoFail();
|
||||
break;
|
||||
}
|
||||
}
|
||||
assertEquals(20*(i+1)+extraCount, writer.numDocs());
|
||||
}
|
||||
|
||||
writer.close();
|
||||
|
@ -155,8 +164,12 @@ public class TestConcurrentMergeScheduler extends LuceneTestCase {
|
|||
IndexWriter writer = new IndexWriter(directory, newIndexWriterConfig(
|
||||
TEST_VERSION_CURRENT, new MockAnalyzer())
|
||||
.setMaxBufferedDocs(2));
|
||||
writer.setInfoStream(VERBOSE ? System.out : null);
|
||||
|
||||
for(int iter=0;iter<7;iter++) {
|
||||
if (VERBOSE) {
|
||||
System.out.println("TEST: iter=" + iter);
|
||||
}
|
||||
|
||||
for(int j=0;j<21;j++) {
|
||||
Document doc = new Document();
|
||||
|
|
|
@ -1083,6 +1083,9 @@ public class TestIndexWriter extends LuceneTestCase {
|
|||
doc.add(idField);
|
||||
|
||||
for(int pass=0;pass<2;pass++) {
|
||||
if (VERBOSE) {
|
||||
System.out.println("TEST: pass=" + pass);
|
||||
}
|
||||
|
||||
IndexWriter writer = new IndexWriter(
|
||||
directory,
|
||||
|
@ -1094,10 +1097,12 @@ public class TestIndexWriter extends LuceneTestCase {
|
|||
// backed directory:
|
||||
setMergePolicy(newLogMergePolicy(false, 10))
|
||||
);
|
||||
writer.setInfoStream(VERBOSE ? System.out : null);
|
||||
|
||||
//System.out.println("TEST: pass=" + pass + " cms=" + (pass >= 2));
|
||||
for(int iter=0;iter<10;iter++) {
|
||||
//System.out.println("TEST: iter=" + iter);
|
||||
if (VERBOSE) {
|
||||
System.out.println("TEST: iter=" + iter);
|
||||
}
|
||||
for(int j=0;j<199;j++) {
|
||||
idField.setValue(Integer.toString(iter*201+j));
|
||||
writer.addDocument(doc);
|
||||
|
@ -1142,8 +1147,9 @@ public class TestIndexWriter extends LuceneTestCase {
|
|||
}
|
||||
};
|
||||
|
||||
if (failure.size() > 0)
|
||||
if (failure.size() > 0) {
|
||||
throw failure.get(0);
|
||||
}
|
||||
|
||||
t1.start();
|
||||
|
||||
|
@ -1156,6 +1162,7 @@ public class TestIndexWriter extends LuceneTestCase {
|
|||
|
||||
// Reopen
|
||||
writer = new IndexWriter(directory, newIndexWriterConfig( TEST_VERSION_CURRENT, new MockAnalyzer()).setOpenMode(OpenMode.APPEND));
|
||||
writer.setInfoStream(VERBOSE ? System.out : null);
|
||||
}
|
||||
writer.close();
|
||||
}
|
||||
|
@ -2575,7 +2582,7 @@ public class TestIndexWriter extends LuceneTestCase {
|
|||
|
||||
Directory dir = newDirectory();
|
||||
FlushCountingIndexWriter w = new FlushCountingIndexWriter(dir, newIndexWriterConfig( TEST_VERSION_CURRENT, new MockAnalyzer(MockTokenizer.WHITESPACE, true, false)).setRAMBufferSizeMB(0.5).setMaxBufferedDocs(-1).setMaxBufferedDeleteTerms(-1));
|
||||
//w.setInfoStream(System.out);
|
||||
w.setInfoStream(VERBOSE ? System.out : null);
|
||||
Document doc = new Document();
|
||||
doc.add(newField("field", "go 1 2 3 4 5 6 7 8 9 10 11 12 13 14 15 16 17 18 19 20", Field.Store.NO, Field.Index.ANALYZED));
|
||||
int num = 6 * RANDOM_MULTIPLIER;
|
||||
|
@ -2583,6 +2590,9 @@ public class TestIndexWriter extends LuceneTestCase {
|
|||
int count = 0;
|
||||
|
||||
final boolean doIndexing = r.nextBoolean();
|
||||
if (VERBOSE) {
|
||||
System.out.println("TEST: iter doIndexing=" + doIndexing);
|
||||
}
|
||||
if (doIndexing) {
|
||||
// Add docs until a flush is triggered
|
||||
final int startFlushCount = w.flushCount;
|
||||
|
|
|
@ -114,6 +114,9 @@ public class TestIndexWriterDelete extends LuceneTestCase {
|
|||
Directory dir = newDirectory();
|
||||
IndexWriter writer = new IndexWriter(dir, newIndexWriterConfig(
|
||||
TEST_VERSION_CURRENT, new MockAnalyzer(MockTokenizer.WHITESPACE, false)).setMaxBufferedDeleteTerms(1));
|
||||
|
||||
writer.setInfoStream(VERBOSE ? System.out : null);
|
||||
writer.addDocument(new Document());
|
||||
writer.deleteDocuments(new Term("foobar", "1"));
|
||||
writer.deleteDocuments(new Term("foobar", "1"));
|
||||
writer.deleteDocuments(new Term("foobar", "1"));
|
||||
|
@ -125,11 +128,14 @@ public class TestIndexWriterDelete extends LuceneTestCase {
|
|||
// test when delete terms only apply to ram segments
|
||||
public void testRAMDeletes() throws IOException {
|
||||
for(int t=0;t<2;t++) {
|
||||
if (VERBOSE) {
|
||||
System.out.println("TEST: t=" + t);
|
||||
}
|
||||
Directory dir = newDirectory();
|
||||
IndexWriter modifier = new IndexWriter(dir, newIndexWriterConfig(
|
||||
TEST_VERSION_CURRENT, new MockAnalyzer(MockTokenizer.WHITESPACE, false)).setMaxBufferedDocs(4)
|
||||
.setMaxBufferedDeleteTerms(4));
|
||||
|
||||
modifier.setInfoStream(VERBOSE ? System.out : null);
|
||||
int id = 0;
|
||||
int value = 100;
|
||||
|
||||
|
@ -439,6 +445,9 @@ public class TestIndexWriterDelete extends LuceneTestCase {
|
|||
|
||||
// Iterate w/ ever increasing free disk space:
|
||||
while (!done) {
|
||||
if (VERBOSE) {
|
||||
System.out.println("TEST: cycle");
|
||||
}
|
||||
MockDirectoryWrapper dir = new MockDirectoryWrapper(random, new RAMDirectory(startDir));
|
||||
dir.setPreventDoubleWrite(false);
|
||||
IndexWriter modifier = new IndexWriter(dir,
|
||||
|
@ -448,6 +457,7 @@ public class TestIndexWriterDelete extends LuceneTestCase {
|
|||
.setMaxBufferedDeleteTerms(1000)
|
||||
.setMergeScheduler(new ConcurrentMergeScheduler()));
|
||||
((ConcurrentMergeScheduler) modifier.getConfig().getMergeScheduler()).setSuppressExceptions();
|
||||
modifier.setInfoStream(VERBOSE ? System.out : null);
|
||||
|
||||
// For each disk size, first try to commit against
|
||||
// dir that will hit random IOExceptions & disk
|
||||
|
@ -456,6 +466,9 @@ public class TestIndexWriterDelete extends LuceneTestCase {
|
|||
boolean success = false;
|
||||
|
||||
for (int x = 0; x < 2; x++) {
|
||||
if (VERBOSE) {
|
||||
System.out.println("TEST: x=" + x);
|
||||
}
|
||||
|
||||
double rate = 0.1;
|
||||
double diskRatio = ((double)diskFree) / diskUsage;
|
||||
|
|
|
@ -51,7 +51,7 @@ public class TestIndexWriterExceptions extends LuceneTestCase {
|
|||
IndexWriter writer;
|
||||
|
||||
final Random r = new java.util.Random(47);
|
||||
Throwable failure;
|
||||
volatile Throwable failure;
|
||||
|
||||
public IndexerThread(int i, IndexWriter writer) {
|
||||
setName("Indexer " + i);
|
||||
|
@ -79,6 +79,9 @@ public class TestIndexWriterExceptions extends LuceneTestCase {
|
|||
final long stopTime = System.currentTimeMillis() + 500;
|
||||
|
||||
do {
|
||||
if (VERBOSE) {
|
||||
System.out.println(Thread.currentThread().getName() + ": TEST: IndexerThread: cycle");
|
||||
}
|
||||
doFail.set(this);
|
||||
final String id = ""+r.nextInt(50);
|
||||
idField.setValue(id);
|
||||
|
@ -136,7 +139,7 @@ public class TestIndexWriterExceptions extends LuceneTestCase {
|
|||
if (doFail.get() != null && !name.equals("startDoFlush") && r.nextInt(20) == 17) {
|
||||
if (VERBOSE) {
|
||||
System.out.println(Thread.currentThread().getName() + ": NOW FAIL: " + name);
|
||||
//new Throwable().printStackTrace(System.out);
|
||||
new Throwable().printStackTrace(System.out);
|
||||
}
|
||||
throw new RuntimeException(Thread.currentThread().getName() + ": intentionally failing at " + name);
|
||||
}
|
||||
|
@ -145,16 +148,23 @@ public class TestIndexWriterExceptions extends LuceneTestCase {
|
|||
}
|
||||
|
||||
public void testRandomExceptions() throws Throwable {
|
||||
if (VERBOSE) {
|
||||
System.out.println("\nTEST: start testRandomExceptions");
|
||||
}
|
||||
MockDirectoryWrapper dir = newDirectory();
|
||||
|
||||
MockIndexWriter writer = new MockIndexWriter(dir, newIndexWriterConfig( TEST_VERSION_CURRENT, new MockAnalyzer())
|
||||
.setRAMBufferSizeMB(0.1).setMergeScheduler(new ConcurrentMergeScheduler()));
|
||||
((ConcurrentMergeScheduler) writer.getConfig().getMergeScheduler()).setSuppressExceptions();
|
||||
//writer.setMaxBufferedDocs(10);
|
||||
if (VERBOSE) {
|
||||
System.out.println("TEST: initial commit");
|
||||
}
|
||||
writer.commit();
|
||||
|
||||
if (VERBOSE)
|
||||
if (VERBOSE) {
|
||||
writer.setInfoStream(System.out);
|
||||
}
|
||||
|
||||
IndexerThread thread = new IndexerThread(0, writer);
|
||||
thread.run();
|
||||
|
@ -163,6 +173,9 @@ public class TestIndexWriterExceptions extends LuceneTestCase {
|
|||
fail("thread " + thread.getName() + ": hit unexpected failure");
|
||||
}
|
||||
|
||||
if (VERBOSE) {
|
||||
System.out.println("TEST: commit after thread start");
|
||||
}
|
||||
writer.commit();
|
||||
|
||||
try {
|
||||
|
@ -192,8 +205,9 @@ public class TestIndexWriterExceptions extends LuceneTestCase {
|
|||
//writer.setMaxBufferedDocs(10);
|
||||
writer.commit();
|
||||
|
||||
if (VERBOSE)
|
||||
if (VERBOSE) {
|
||||
writer.setInfoStream(System.out);
|
||||
}
|
||||
|
||||
final int NUM_THREADS = 4;
|
||||
|
||||
|
@ -294,6 +308,7 @@ public class TestIndexWriterExceptions extends LuceneTestCase {
|
|||
public void testExceptionJustBeforeFlush() throws IOException {
|
||||
Directory dir = newDirectory();
|
||||
MockIndexWriter w = new MockIndexWriter(dir, newIndexWriterConfig( TEST_VERSION_CURRENT, new MockAnalyzer()).setMaxBufferedDocs(2));
|
||||
w.setInfoStream(VERBOSE ? System.out : null);
|
||||
Document doc = new Document();
|
||||
doc.add(newField("field", "a field", Field.Store.YES,
|
||||
Field.Index.ANALYZED));
|
||||
|
|
|
@ -47,29 +47,35 @@ public class TestIndexWriterOnDiskFull extends LuceneTestCase {
|
|||
public void testAddDocumentOnDiskFull() throws IOException {
|
||||
|
||||
for(int pass=0;pass<2;pass++) {
|
||||
if (VERBOSE)
|
||||
if (VERBOSE) {
|
||||
System.out.println("TEST: pass=" + pass);
|
||||
}
|
||||
boolean doAbort = pass == 1;
|
||||
long diskFree = 200;
|
||||
while(true) {
|
||||
if (VERBOSE)
|
||||
if (VERBOSE) {
|
||||
System.out.println("TEST: cycle: diskFree=" + diskFree);
|
||||
}
|
||||
MockDirectoryWrapper dir = new MockDirectoryWrapper(random, new RAMDirectory());
|
||||
dir.setMaxSizeInBytes(diskFree);
|
||||
IndexWriter writer = new IndexWriter(dir, newIndexWriterConfig( TEST_VERSION_CURRENT, new MockAnalyzer()));
|
||||
writer.setInfoStream(VERBOSE ? System.out : null);
|
||||
MergeScheduler ms = writer.getConfig().getMergeScheduler();
|
||||
if (ms instanceof ConcurrentMergeScheduler)
|
||||
if (ms instanceof ConcurrentMergeScheduler) {
|
||||
// This test intentionally produces exceptions
|
||||
// in the threads that CMS launches; we don't
|
||||
// want to pollute test output with these.
|
||||
((ConcurrentMergeScheduler) ms).setSuppressExceptions();
|
||||
}
|
||||
|
||||
boolean hitError = false;
|
||||
try {
|
||||
for(int i=0;i<200;i++) {
|
||||
addDoc(writer);
|
||||
}
|
||||
if (VERBOSE) {
|
||||
System.out.println("TEST: done adding docs; now commit");
|
||||
}
|
||||
writer.commit();
|
||||
} catch (IOException e) {
|
||||
if (VERBOSE) {
|
||||
|
@ -81,13 +87,19 @@ public class TestIndexWriterOnDiskFull extends LuceneTestCase {
|
|||
|
||||
if (hitError) {
|
||||
if (doAbort) {
|
||||
if (VERBOSE) {
|
||||
System.out.println("TEST: now rollback");
|
||||
}
|
||||
writer.rollback();
|
||||
} else {
|
||||
try {
|
||||
if (VERBOSE) {
|
||||
System.out.println("TEST: now close");
|
||||
}
|
||||
writer.close();
|
||||
} catch (IOException e) {
|
||||
if (VERBOSE) {
|
||||
System.out.println("TEST: exception on close");
|
||||
System.out.println("TEST: exception on close; retry w/ no disk space limit");
|
||||
e.printStackTrace(System.out);
|
||||
}
|
||||
dir.setMaxSizeInBytes(0);
|
||||
|
|
|
@ -106,6 +106,9 @@ public class TestIndexWriterWithThreads extends LuceneTestCase {
|
|||
int NUM_THREADS = 3;
|
||||
|
||||
for(int iter=0;iter<10;iter++) {
|
||||
if (VERBOSE) {
|
||||
System.out.println("\nTEST: iter=" + iter);
|
||||
}
|
||||
MockDirectoryWrapper dir = newDirectory();
|
||||
IndexWriter writer = new IndexWriter(
|
||||
dir,
|
||||
|
@ -116,6 +119,7 @@ public class TestIndexWriterWithThreads extends LuceneTestCase {
|
|||
);
|
||||
((ConcurrentMergeScheduler) writer.getConfig().getMergeScheduler()).setSuppressExceptions();
|
||||
dir.setMaxSizeInBytes(4*1024+20*iter);
|
||||
writer.setInfoStream(VERBOSE ? System.out : null);
|
||||
|
||||
IndexerThread[] threads = new IndexerThread[NUM_THREADS];
|
||||
|
||||
|
|
|
@ -0,0 +1,299 @@
|
|||
package org.apache.lucene.index;
|
||||
|
||||
/**
|
||||
* Licensed to the Apache Software Foundation (ASF) under one or more
|
||||
* contributor license agreements. See the NOTICE file distributed with
|
||||
* this work for additional information regarding copyright ownership.
|
||||
* The ASF licenses this file to You under the Apache License, Version 2.0
|
||||
* (the "License"); you may not use this file except in compliance with
|
||||
* the License. You may obtain a copy of the License at
|
||||
*
|
||||
* http://www.apache.org/licenses/LICENSE-2.0
|
||||
*
|
||||
* Unless required by applicable law or agreed to in writing, software
|
||||
* distributed under the License is distributed on an "AS IS" BASIS,
|
||||
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
|
||||
* See the License for the specific language governing permissions and
|
||||
* limitations under the License.
|
||||
*/
|
||||
|
||||
import java.io.IOException;
|
||||
import java.util.ArrayList;
|
||||
import java.util.List;
|
||||
import java.util.Random;
|
||||
import java.util.Set;
|
||||
|
||||
import org.apache.lucene.analysis.MockAnalyzer;
|
||||
import org.apache.lucene.document.Document;
|
||||
import org.apache.lucene.index.TermsEnum.SeekStatus;
|
||||
import org.apache.lucene.store.Directory;
|
||||
import org.apache.lucene.store.MockDirectoryWrapper;
|
||||
import org.apache.lucene.store.RAMDirectory;
|
||||
import org.apache.lucene.util.ArrayUtil;
|
||||
import org.apache.lucene.util.Bits;
|
||||
import org.apache.lucene.util.BytesRef;
|
||||
import org.apache.lucene.util.LuceneTestCase;
|
||||
import org.apache.lucene.util.Version;
|
||||
|
||||
public class TestPerSegmentDeletes extends LuceneTestCase {
|
||||
public void testDeletes1() throws Exception {
|
||||
//IndexWriter.debug2 = System.out;
|
||||
Directory dir = new MockDirectoryWrapper(new Random(), new RAMDirectory());
|
||||
IndexWriterConfig iwc = new IndexWriterConfig(Version.LUCENE_CURRENT,
|
||||
new MockAnalyzer());
|
||||
iwc.setMergeScheduler(new SerialMergeScheduler());
|
||||
iwc.setMaxBufferedDocs(5000);
|
||||
iwc.setRAMBufferSizeMB(100);
|
||||
RangeMergePolicy fsmp = new RangeMergePolicy(false);
|
||||
iwc.setMergePolicy(fsmp);
|
||||
IndexWriter writer = new IndexWriter(dir, iwc);
|
||||
for (int x = 0; x < 5; x++) {
|
||||
writer.addDocument(TestIndexWriterReader.createDocument(x, "1", 2));
|
||||
//System.out.println("numRamDocs(" + x + ")" + writer.numRamDocs());
|
||||
}
|
||||
//System.out.println("commit1");
|
||||
writer.commit();
|
||||
assertEquals(1, writer.segmentInfos.size());
|
||||
for (int x = 5; x < 10; x++) {
|
||||
writer.addDocument(TestIndexWriterReader.createDocument(x, "2", 2));
|
||||
//System.out.println("numRamDocs(" + x + ")" + writer.numRamDocs());
|
||||
}
|
||||
//System.out.println("commit2");
|
||||
writer.commit();
|
||||
assertEquals(2, writer.segmentInfos.size());
|
||||
|
||||
for (int x = 10; x < 15; x++) {
|
||||
writer.addDocument(TestIndexWriterReader.createDocument(x, "3", 2));
|
||||
//System.out.println("numRamDocs(" + x + ")" + writer.numRamDocs());
|
||||
}
|
||||
|
||||
writer.deleteDocuments(new Term("id", "1"));
|
||||
|
||||
writer.deleteDocuments(new Term("id", "11"));
|
||||
|
||||
// flushing without applying deletes means
|
||||
// there will still be deletes in the segment infos
|
||||
writer.flush(false, false, false);
|
||||
assertTrue(writer.bufferedDeletes.any());
|
||||
|
||||
// get reader flushes pending deletes
|
||||
// so there should not be anymore
|
||||
IndexReader r1 = writer.getReader();
|
||||
assertFalse(writer.bufferedDeletes.any());
|
||||
r1.close();
|
||||
|
||||
// delete id:2 from the first segment
|
||||
// merge segments 0 and 1
|
||||
// which should apply the delete id:2
|
||||
writer.deleteDocuments(new Term("id", "2"));
|
||||
writer.flush(false, false, false);
|
||||
fsmp.doMerge = true;
|
||||
fsmp.start = 0;
|
||||
fsmp.length = 2;
|
||||
writer.maybeMerge();
|
||||
|
||||
assertEquals(2, writer.segmentInfos.size());
|
||||
|
||||
// id:2 shouldn't exist anymore because
|
||||
// it's been applied in the merge and now it's gone
|
||||
IndexReader r2 = writer.getReader();
|
||||
int[] id2docs = toDocsArray(new Term("id", "2"), null, r2);
|
||||
assertTrue(id2docs == null);
|
||||
r2.close();
|
||||
|
||||
/**
|
||||
// added docs are in the ram buffer
|
||||
for (int x = 15; x < 20; x++) {
|
||||
writer.addDocument(TestIndexWriterReader.createDocument(x, "4", 2));
|
||||
System.out.println("numRamDocs(" + x + ")" + writer.numRamDocs());
|
||||
}
|
||||
assertTrue(writer.numRamDocs() > 0);
|
||||
// delete from the ram buffer
|
||||
writer.deleteDocuments(new Term("id", Integer.toString(13)));
|
||||
|
||||
Term id3 = new Term("id", Integer.toString(3));
|
||||
|
||||
// delete from the 1st segment
|
||||
writer.deleteDocuments(id3);
|
||||
|
||||
assertTrue(writer.numRamDocs() > 0);
|
||||
|
||||
//System.out
|
||||
// .println("segdels1:" + writer.docWriter.deletesToString());
|
||||
|
||||
//assertTrue(writer.docWriter.segmentDeletes.size() > 0);
|
||||
|
||||
// we cause a merge to happen
|
||||
fsmp.doMerge = true;
|
||||
fsmp.start = 0;
|
||||
fsmp.length = 2;
|
||||
System.out.println("maybeMerge "+writer.segmentInfos);
|
||||
|
||||
SegmentInfo info0 = writer.segmentInfos.get(0);
|
||||
SegmentInfo info1 = writer.segmentInfos.get(1);
|
||||
|
||||
writer.maybeMerge();
|
||||
System.out.println("maybeMerge after "+writer.segmentInfos);
|
||||
// there should be docs in RAM
|
||||
assertTrue(writer.numRamDocs() > 0);
|
||||
|
||||
// assert we've merged the 1 and 2 segments
|
||||
// and still have a segment leftover == 2
|
||||
assertEquals(2, writer.segmentInfos.size());
|
||||
assertFalse(segThere(info0, writer.segmentInfos));
|
||||
assertFalse(segThere(info1, writer.segmentInfos));
|
||||
|
||||
//System.out.println("segdels2:" + writer.docWriter.deletesToString());
|
||||
|
||||
//assertTrue(writer.docWriter.segmentDeletes.size() > 0);
|
||||
|
||||
IndexReader r = writer.getReader();
|
||||
IndexReader r1 = r.getSequentialSubReaders()[0];
|
||||
printDelDocs(r1.getDeletedDocs());
|
||||
int[] docs = toDocsArray(id3, null, r);
|
||||
System.out.println("id3 docs:"+Arrays.toString(docs));
|
||||
// there shouldn't be any docs for id:3
|
||||
assertTrue(docs == null);
|
||||
r.close();
|
||||
|
||||
part2(writer, fsmp);
|
||||
**/
|
||||
// System.out.println("segdels2:"+writer.docWriter.segmentDeletes.toString());
|
||||
//System.out.println("close");
|
||||
writer.close();
|
||||
dir.close();
|
||||
}
|
||||
|
||||
/**
|
||||
static boolean hasPendingDeletes(SegmentInfos infos) {
|
||||
for (SegmentInfo info : infos) {
|
||||
if (info.deletes.any()) {
|
||||
return true;
|
||||
}
|
||||
}
|
||||
return false;
|
||||
}
|
||||
**/
|
||||
void part2(IndexWriter writer, RangeMergePolicy fsmp) throws Exception {
|
||||
for (int x = 20; x < 25; x++) {
|
||||
writer.addDocument(TestIndexWriterReader.createDocument(x, "5", 2));
|
||||
//System.out.println("numRamDocs(" + x + ")" + writer.numRamDocs());
|
||||
}
|
||||
writer.flush(false, true, false);
|
||||
for (int x = 25; x < 30; x++) {
|
||||
writer.addDocument(TestIndexWriterReader.createDocument(x, "5", 2));
|
||||
//System.out.println("numRamDocs(" + x + ")" + writer.numRamDocs());
|
||||
}
|
||||
writer.flush(false, true, false);
|
||||
|
||||
//System.out.println("infos3:"+writer.segmentInfos);
|
||||
|
||||
Term delterm = new Term("id", "8");
|
||||
writer.deleteDocuments(delterm);
|
||||
//System.out.println("segdels3:" + writer.docWriter.deletesToString());
|
||||
|
||||
fsmp.doMerge = true;
|
||||
fsmp.start = 1;
|
||||
fsmp.length = 2;
|
||||
writer.maybeMerge();
|
||||
|
||||
// deletes for info1, the newly created segment from the
|
||||
// merge should have no deletes because they were applied in
|
||||
// the merge
|
||||
//SegmentInfo info1 = writer.segmentInfos.get(1);
|
||||
//assertFalse(exists(info1, writer.docWriter.segmentDeletes));
|
||||
|
||||
//System.out.println("infos4:"+writer.segmentInfos);
|
||||
//System.out.println("segdels4:" + writer.docWriter.deletesToString());
|
||||
}
|
||||
|
||||
boolean segThere(SegmentInfo info, SegmentInfos infos) {
|
||||
for (SegmentInfo si : infos) {
|
||||
if (si.name.equals(info.name)) return true;
|
||||
}
|
||||
return false;
|
||||
}
|
||||
|
||||
public static void printDelDocs(Bits bits) {
|
||||
if (bits == null) return;
|
||||
for (int x = 0; x < bits.length(); x++) {
|
||||
System.out.println(x + ":" + bits.get(x));
|
||||
}
|
||||
}
|
||||
|
||||
public static int[] toDocsArray(Term term, Bits bits, IndexReader reader)
|
||||
throws IOException {
|
||||
Fields fields = MultiFields.getFields(reader);
|
||||
Terms cterms = fields.terms(term.field);
|
||||
TermsEnum ctermsEnum = cterms.iterator();
|
||||
SeekStatus ss = ctermsEnum.seek(new BytesRef(term.text()), false);
|
||||
if (ss.equals(SeekStatus.FOUND)) {
|
||||
DocsEnum docsEnum = ctermsEnum.docs(bits, null);
|
||||
return toArray(docsEnum);
|
||||
}
|
||||
return null;
|
||||
}
|
||||
|
||||
public static int[] toArray(DocsEnum docsEnum) throws IOException {
|
||||
List<Integer> docs = new ArrayList<Integer>();
|
||||
while (docsEnum.nextDoc() != DocsEnum.NO_MORE_DOCS) {
|
||||
int docID = docsEnum.docID();
|
||||
docs.add(docID);
|
||||
}
|
||||
return ArrayUtil.toIntArray(docs);
|
||||
}
|
||||
|
||||
public class RangeMergePolicy extends MergePolicy {
|
||||
boolean doMerge = false;
|
||||
int start;
|
||||
int length;
|
||||
|
||||
private final boolean useCompoundFile;
|
||||
|
||||
private RangeMergePolicy(boolean useCompoundFile) {
|
||||
this.useCompoundFile = useCompoundFile;
|
||||
}
|
||||
|
||||
@Override
|
||||
public void close() {}
|
||||
|
||||
public MergeSpecification findMerges(SegmentInfos segmentInfos)
|
||||
throws CorruptIndexException, IOException {
|
||||
MergeSpecification ms = new MergeSpecification();
|
||||
if (doMerge) {
|
||||
SegmentInfos mergeInfos = new SegmentInfos();
|
||||
for (int x=start; x < (start+length); x++) {
|
||||
mergeInfos.add(segmentInfos.get(x));
|
||||
}
|
||||
OneMerge om = new OneMerge(mergeInfos);
|
||||
ms.add(om);
|
||||
doMerge = false;
|
||||
return ms;
|
||||
}
|
||||
return null;
|
||||
}
|
||||
|
||||
@Override
|
||||
public MergeSpecification findMergesForOptimize(SegmentInfos segmentInfos,
|
||||
int maxSegmentCount, Set<SegmentInfo> segmentsToOptimize)
|
||||
throws CorruptIndexException, IOException {
|
||||
return null;
|
||||
}
|
||||
|
||||
@Override
|
||||
public MergeSpecification findMergesToExpungeDeletes(
|
||||
SegmentInfos segmentInfos) throws CorruptIndexException, IOException {
|
||||
return null;
|
||||
}
|
||||
|
||||
@Override
|
||||
public boolean useCompoundDocStore(SegmentInfos segments) {
|
||||
return useCompoundFile;
|
||||
}
|
||||
|
||||
@Override
|
||||
public boolean useCompoundFile(SegmentInfos segments, SegmentInfo newSegment) {
|
||||
return useCompoundFile;
|
||||
}
|
||||
}
|
||||
}
|
|
@ -95,6 +95,9 @@ public class TestStressIndexing2 extends LuceneTestCase {
|
|||
|
||||
int num = 3 * RANDOM_MULTIPLIER;
|
||||
for (int i = 0; i < num; i++) { // increase iterations for better testing
|
||||
if (VERBOSE) {
|
||||
System.out.println("\n\nTEST: top iter=" + i);
|
||||
}
|
||||
sameFieldOrder=random.nextBoolean();
|
||||
mergeFactor=random.nextInt(3)+2;
|
||||
maxBufferedDocs=random.nextInt(3)+2;
|
||||
|
@ -107,10 +110,17 @@ public class TestStressIndexing2 extends LuceneTestCase {
|
|||
int range=random.nextInt(20)+1;
|
||||
Directory dir1 = newDirectory();
|
||||
Directory dir2 = newDirectory();
|
||||
if (VERBOSE) {
|
||||
System.out.println(" nThreads=" + nThreads + " iter=" + iter + " range=" + range + " doPooling=" + doReaderPooling + " maxThreadStates=" + maxThreadStates + " sameFieldOrder=" + sameFieldOrder + " mergeFactor=" + mergeFactor);
|
||||
}
|
||||
Map<String,Document> docs = indexRandom(nThreads, iter, range, dir1, maxThreadStates, doReaderPooling);
|
||||
//System.out.println("TEST: index serial");
|
||||
if (VERBOSE) {
|
||||
System.out.println("TEST: index serial");
|
||||
}
|
||||
indexSerial(random, docs, dir2);
|
||||
//System.out.println("TEST: verify");
|
||||
if (VERBOSE) {
|
||||
System.out.println("TEST: verify");
|
||||
}
|
||||
verifyEquals(dir1, dir2, "id");
|
||||
dir1.close();
|
||||
dir2.close();
|
||||
|
@ -140,6 +150,7 @@ public class TestStressIndexing2 extends LuceneTestCase {
|
|||
IndexWriter w = new MockIndexWriter(dir, newIndexWriterConfig(
|
||||
TEST_VERSION_CURRENT, new MockAnalyzer()).setOpenMode(OpenMode.CREATE).setRAMBufferSizeMB(
|
||||
0.1).setMaxBufferedDocs(maxBufferedDocs));
|
||||
w.setInfoStream(VERBOSE ? System.out : null);
|
||||
w.commit();
|
||||
LogMergePolicy lmp = (LogMergePolicy) w.getConfig().getMergePolicy();
|
||||
lmp.setUseCompoundFile(false);
|
||||
|
@ -190,10 +201,14 @@ public class TestStressIndexing2 extends LuceneTestCase {
|
|||
boolean doReaderPooling) throws IOException, InterruptedException {
|
||||
Map<String,Document> docs = new HashMap<String,Document>();
|
||||
for(int iter=0;iter<3;iter++) {
|
||||
if (VERBOSE) {
|
||||
System.out.println("TEST: iter=" + iter);
|
||||
}
|
||||
IndexWriter w = new MockIndexWriter(dir, newIndexWriterConfig(
|
||||
TEST_VERSION_CURRENT, new MockAnalyzer()).setOpenMode(OpenMode.CREATE)
|
||||
.setRAMBufferSizeMB(0.1).setMaxBufferedDocs(maxBufferedDocs).setMaxThreadStates(maxThreadStates)
|
||||
.setReaderPooling(doReaderPooling));
|
||||
w.setInfoStream(VERBOSE ? System.out : null);
|
||||
LogMergePolicy lmp = (LogMergePolicy) w.getConfig().getMergePolicy();
|
||||
lmp.setUseCompoundFile(false);
|
||||
lmp.setUseCompoundDocStore(false);
|
||||
|
@ -272,9 +287,33 @@ public class TestStressIndexing2 extends LuceneTestCase {
|
|||
r2.close();
|
||||
}
|
||||
|
||||
private static void printDocs(IndexReader r) throws Throwable {
|
||||
IndexReader[] subs = r.getSequentialSubReaders();
|
||||
for(IndexReader sub : subs) {
|
||||
Bits delDocs = sub.getDeletedDocs();
|
||||
System.out.println(" " + ((SegmentReader) sub).getSegmentInfo());
|
||||
for(int docID=0;docID<sub.maxDoc();docID++) {
|
||||
Document doc = sub.document(docID);
|
||||
if (delDocs == null || !delDocs.get(docID)) {
|
||||
System.out.println(" docID=" + docID + " id:" + doc.get("id"));
|
||||
} else {
|
||||
System.out.println(" DEL docID=" + docID + " id:" + doc.get("id"));
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
|
||||
public static void verifyEquals(IndexReader r1, IndexReader r2, String idField) throws Throwable {
|
||||
assertEquals(r1.numDocs(), r2.numDocs());
|
||||
if (VERBOSE) {
|
||||
System.out.println("\nr1 docs:");
|
||||
printDocs(r1);
|
||||
System.out.println("\nr2 docs:");
|
||||
printDocs(r2);
|
||||
}
|
||||
if (r1.numDocs() != r2.numDocs()) {
|
||||
assert false: "r1.numDocs()=" + r1.numDocs() + " vs r2.numDocs()=" + r2.numDocs();
|
||||
}
|
||||
boolean hasDeletes = !(r1.maxDoc()==r2.maxDoc() && r1.numDocs()==r1.maxDoc());
|
||||
|
||||
int[] r2r1 = new int[r2.maxDoc()]; // r2 id to r1 id mapping
|
||||
|
@ -692,19 +731,28 @@ public class TestStressIndexing2 extends LuceneTestCase {
|
|||
for (int i=0; i<fields.size(); i++) {
|
||||
d.add(fields.get(i));
|
||||
}
|
||||
if (VERBOSE) {
|
||||
System.out.println(Thread.currentThread().getName() + ": indexing id:" + idString);
|
||||
}
|
||||
w.updateDocument(idTerm.createTerm(idString), d);
|
||||
// System.out.println("indexing "+d);
|
||||
//System.out.println(Thread.currentThread().getName() + ": indexing "+d);
|
||||
docs.put(idString, d);
|
||||
}
|
||||
|
||||
public void deleteDoc() throws IOException {
|
||||
String idString = getIdString();
|
||||
if (VERBOSE) {
|
||||
System.out.println(Thread.currentThread().getName() + ": del id:" + idString);
|
||||
}
|
||||
w.deleteDocuments(idTerm.createTerm(idString));
|
||||
docs.remove(idString);
|
||||
}
|
||||
|
||||
public void deleteByQuery() throws IOException {
|
||||
String idString = getIdString();
|
||||
if (VERBOSE) {
|
||||
System.out.println(Thread.currentThread().getName() + ": del query id:" + idString);
|
||||
}
|
||||
w.deleteDocuments(new TermQuery(idTerm.createTerm(idString)));
|
||||
docs.remove(idString);
|
||||
}
|
||||
|
|
|
@ -353,12 +353,19 @@ public class TestTermVectors extends LuceneTestCase {
|
|||
RandomIndexWriter writer = new RandomIndexWriter(random, directory,
|
||||
newIndexWriterConfig(TEST_VERSION_CURRENT, new MockAnalyzer(MockTokenizer.SIMPLE, true))
|
||||
.setOpenMode(OpenMode.CREATE));
|
||||
writer.w.setInfoStream(VERBOSE ? System.out : null);
|
||||
if (VERBOSE) {
|
||||
System.out.println("TEST: now add non-vectors");
|
||||
}
|
||||
for (int i = 0; i < 100; i++) {
|
||||
Document doc = new Document();
|
||||
doc.add(new Field("field", English.intToEnglish(i),
|
||||
Field.Store.YES, Field.Index.ANALYZED, Field.TermVector.NO));
|
||||
writer.addDocument(doc);
|
||||
}
|
||||
if (VERBOSE) {
|
||||
System.out.println("TEST: now add vectors");
|
||||
}
|
||||
for(int i=0;i<10;i++) {
|
||||
Document doc = new Document();
|
||||
doc.add(new Field("field", English.intToEnglish(100+i),
|
||||
|
@ -366,6 +373,9 @@ public class TestTermVectors extends LuceneTestCase {
|
|||
writer.addDocument(doc);
|
||||
}
|
||||
|
||||
if (VERBOSE) {
|
||||
System.out.println("TEST: now getReader");
|
||||
}
|
||||
IndexReader reader = writer.getReader();
|
||||
writer.close();
|
||||
searcher = new IndexSearcher(reader);
|
||||
|
@ -374,6 +384,7 @@ public class TestTermVectors extends LuceneTestCase {
|
|||
ScoreDoc[] hits = searcher.search(query, null, 1000).scoreDocs;
|
||||
assertEquals(10, hits.length);
|
||||
for (int i = 0; i < hits.length; i++) {
|
||||
|
||||
TermFreqVector [] vector = searcher.reader.getTermFreqVectors(hits[i].doc);
|
||||
assertTrue(vector != null);
|
||||
assertTrue(vector.length == 1);
|
||||
|
|
|
@ -19,6 +19,8 @@ package org.apache.lucene.store;
|
|||
|
||||
import java.io.IOException;
|
||||
|
||||
import org.apache.lucene.util.LuceneTestCase;
|
||||
|
||||
/**
|
||||
* Used by MockRAMDirectory to create an output stream that
|
||||
* will throw an IOException on fake disk full, track max
|
||||
|
@ -102,6 +104,9 @@ public class MockIndexOutputWrapper extends IndexOutput {
|
|||
message += "; wrote " + freeSpace + " of " + len + " bytes";
|
||||
}
|
||||
message += ")";
|
||||
if (LuceneTestCase.VERBOSE) {
|
||||
System.out.println(Thread.currentThread().getName() + ": MDW: now throw fake disk full");
|
||||
}
|
||||
throw new IOException(message);
|
||||
} else {
|
||||
if (dir.randomState.nextBoolean()) {
|
||||
|
|
Loading…
Reference in New Issue