fix verbosity when infoStream is on

git-svn-id: https://svn.apache.org/repos/asf/lucene/dev/trunk@1344931 13f79535-47bb-0310-9956-ffa450edef68
This commit is contained in:
Michael McCandless 2012-05-31 22:27:11 +00:00
parent 5f05354817
commit d65ba8d79e
4 changed files with 16 additions and 7 deletions

View File

@ -93,7 +93,7 @@ class BufferedDeletes {
} else {
String s = "gen=" + gen;
if (numTermDeletes.get() != 0) {
s += " " + numTermDeletes.get() + " deleted terms (unique count=" + terms.size() + ") terms=" + terms.keySet();
s += " " + numTermDeletes.get() + " deleted terms (unique count=" + terms.size() + ")";
}
if (queries.size() != 0) {
s += " " + queries.size() + " deleted queries";

View File

@ -482,7 +482,7 @@ final class DocumentsWriterFlushControl implements MemoryController {
void addFlushableState(ThreadState perThread) {
if (documentsWriter.infoStream.isEnabled("DWFC")) {
documentsWriter.infoStream.message("DWFC", Thread.currentThread().getName() + ": addFlushableState " + perThread.dwpt);
documentsWriter.infoStream.message("DWFC", "addFlushableState " + perThread.dwpt);
}
final DocumentsWriterPerThread dwpt = perThread.dwpt;
assert perThread.isHeldByCurrentThread();

View File

@ -2123,7 +2123,7 @@ public class IndexWriter implements Closeable, TwoPhaseCommit {
nextGen = bufferedDeletesStream.getNextGen();
}
if (infoStream.isEnabled("IW")) {
infoStream.message("IW", "publish sets newSegment delGen=" + nextGen + " seg=" + newSegment);
infoStream.message("IW", "publish sets newSegment delGen=" + nextGen + " seg=" + segString(newSegment));
}
newSegment.setBufferedDeletesGen(nextGen);
segmentInfos.add(newSegment);
@ -3213,7 +3213,7 @@ public class IndexWriter implements Closeable, TwoPhaseCommit {
}
for(SegmentInfoPerCommit info : merge.segments) {
if (infoStream.isEnabled("IW")) {
infoStream.message("IW", "registerMerge info=" + info);
infoStream.message("IW", "registerMerge info=" + segString(info));
}
mergingSegments.add(info);
}
@ -3453,11 +3453,11 @@ public class IndexWriter implements Closeable, TwoPhaseCommit {
if (infoStream.isEnabled("IW")) {
if (rld.getPendingDeleteCount() != 0) {
infoStream.message("IW", "seg=" + info + " delCount=" + info.getDelCount() + " pendingDelCount=" + rld.getPendingDeleteCount());
infoStream.message("IW", "seg=" + segString(info) + " delCount=" + info.getDelCount() + " pendingDelCount=" + rld.getPendingDeleteCount());
} else if (info.getDelCount() != 0) {
infoStream.message("IW", "seg=" + info + " delCount=" + info.getDelCount());
infoStream.message("IW", "seg=" + segString(info) + " delCount=" + info.getDelCount());
} else {
infoStream.message("IW", "seg=" + info + " no deletes");
infoStream.message("IW", "seg=" + segString(info) + " no deletes");
}
}
}

View File

@ -131,6 +131,15 @@ public class SegmentInfoPerCommit {
return info.toString(dir, delCount + pendingDelCount);
}
@Override
public String toString() {
String s = info.toString(info.dir, delCount);
if (delGen != -1) {
s += ":delGen=" + delGen;
}
return s;
}
@Override
public SegmentInfoPerCommit clone() {
return new SegmentInfoPerCommit(info, delCount, delGen);