Commit LUCENE-3598 to not corrumpt Robert's work on Default*: Improve InfoStream class in trunk to be more consistent with logging-frameworks like slf4j/log4j/commons-logging

git-svn-id: https://svn.apache.org/repos/asf/lucene/dev/trunk@1209059 13f79535-47bb-0310-9956-ffa450edef68
This commit is contained in:
Uwe Schindler 2011-12-01 12:52:20 +00:00
parent 814d35e437
commit 21989c90c2
22 changed files with 221 additions and 266 deletions

View File

@ -88,7 +88,7 @@ class BufferedDeletesStream {
deletes.add(packet);
numTerms.addAndGet(packet.numTermDeletes);
bytesUsed.addAndGet(packet.bytesUsed);
if (infoStream != null) {
if (infoStream.isEnabled("BD")) {
infoStream.message("BD", "push deletes " + packet + " delGen=" + packet.delGen() + " packetCount=" + deletes.size() + " totBytesUsed=" + bytesUsed.get());
}
assert checkDeleteStats();
@ -159,13 +159,13 @@ class BufferedDeletesStream {
assert checkDeleteStats();
if (!any()) {
if (infoStream != null) {
if (infoStream.isEnabled("BD")) {
infoStream.message("BD", "applyDeletes: no deletes; skipping");
}
return new ApplyDeletesResult(false, nextGen++, null);
}
if (infoStream != null) {
if (infoStream.isEnabled("BD")) {
infoStream.message("BD", "applyDeletes: infos=" + infos + " packetCount=" + deletes.size());
}
@ -237,7 +237,7 @@ class BufferedDeletesStream {
allDeleted.add(info);
}
if (infoStream != null) {
if (infoStream.isEnabled("BD")) {
infoStream.message("BD", "seg=" + info + " segGen=" + segGen + " segDeletes=[" + packet + "]; coalesced deletes=[" + (coalescedDeletes == null ? "null" : coalescedDeletes) + "] delCount=" + delCount + (segAllDeletes ? " 100% deleted" : ""));
}
@ -279,7 +279,7 @@ class BufferedDeletesStream {
allDeleted.add(info);
}
if (infoStream != null) {
if (infoStream.isEnabled("BD")) {
infoStream.message("BD", "seg=" + info + " segGen=" + segGen + " coalesced deletes=[" + (coalescedDeletes == null ? "null" : coalescedDeletes) + "] delCount=" + delCount + (segAllDeletes ? " 100% deleted" : ""));
}
}
@ -290,7 +290,7 @@ class BufferedDeletesStream {
}
assert checkDeleteStats();
if (infoStream != null) {
if (infoStream.isEnabled("BD")) {
infoStream.message("BD", "applyDeletes took " + (System.currentTimeMillis()-t0) + " msec");
}
// assert infos != segmentInfos || !any() : "infos=" + infos + " segmentInfos=" + segmentInfos + " any=" + any;
@ -313,7 +313,7 @@ class BufferedDeletesStream {
minGen = Math.min(info.getBufferedDeletesGen(), minGen);
}
if (infoStream != null) {
if (infoStream.isEnabled("BD")) {
infoStream.message("BD", "prune sis=" + segmentInfos + " minGen=" + minGen + " packetCount=" + deletes.size());
}
final int limit = deletes.size();
@ -333,7 +333,7 @@ class BufferedDeletesStream {
private synchronized void prune(int count) {
if (count > 0) {
if (infoStream != null) {
if (infoStream.isEnabled("BD")) {
infoStream.message("BD", "pruneDeletes: prune " + count + " packets; " + (deletes.size() - count) + " packets remain");
}
for(int delIDX=0;delIDX<count;delIDX++) {

View File

@ -218,7 +218,7 @@ public class ConcurrentMergeScheduler extends MergeScheduler {
* </pre>
*/
protected boolean verbose() {
return writer != null && writer.verbose();
return writer != null && writer.infoStream.isEnabled("CMS");
}
/**
@ -226,10 +226,7 @@ public class ConcurrentMergeScheduler extends MergeScheduler {
* called and returned true.
*/
protected void message(String message) {
final InfoStream infoStream = writer.infoStream;
if (infoStream != null) {
infoStream.message("CMS", message);
}
writer.infoStream.message("CMS", message);
}
private synchronized void initMergeThreadPriority() {

View File

@ -36,6 +36,7 @@ import org.apache.lucene.store.LockObtainFailedException;
import org.apache.lucene.index.codecs.PerDocValues;
import org.apache.lucene.util.Bits;
import org.apache.lucene.util.BytesRef;
import org.apache.lucene.util.InfoStream;
import org.apache.lucene.util.MapBackedSet;
/**
@ -685,9 +686,10 @@ class DirectoryReader extends IndexReader implements Cloneable {
segmentInfos.setUserData(commitUserData);
// Default deleter (for backwards compatibility) is
// KeepOnlyLastCommitDeleter:
// nocommit: Decide what to do with InfoStream here? Use default or keep NO_OUTPUT?
IndexFileDeleter deleter = new IndexFileDeleter(directory,
deletionPolicy == null ? new KeepOnlyLastCommitDeletionPolicy() : deletionPolicy,
segmentInfos, null, null);
segmentInfos, InfoStream.NO_OUTPUT, null);
segmentInfos.updateGeneration(deleter.getLastSegmentInfos());
segmentInfos.changed();

View File

@ -216,9 +216,7 @@ final class DocumentsWriter {
}
try {
if (infoStream != null) {
infoStream.message("DW", "abort");
}
infoStream.message("DW", "abort");
final Iterator<ThreadState> threadsIterator = perThreadPool.getActivePerThreadsIterator();
while (threadsIterator.hasNext()) {
@ -243,14 +241,14 @@ final class DocumentsWriter {
}
success = true;
} finally {
if (infoStream != null) {
if (infoStream.isEnabled("DW")) {
infoStream.message("DW", "done abort; abortedFiles=" + abortedFiles + " success=" + success);
}
}
}
boolean anyChanges() {
if (infoStream != null) {
if (infoStream.isEnabled("DW")) {
infoStream.message("DW", "anyChanges? numDocsInRam=" + numDocsInRAM.get()
+ " deletes=" + anyDeletions() + " hasTickets:"
+ ticketQueue.hasTickets() + " pendingChangesInFullFlush: "
@ -289,7 +287,7 @@ final class DocumentsWriter {
boolean maybeMerge = false;
if (flushControl.anyStalledThreads() || flushControl.numQueuedFlushes() > 0) {
// Help out flushing any queued DWPTs so we can un-stall:
if (infoStream != null) {
if (infoStream.isEnabled("DW")) {
infoStream.message("DW", "DocumentsWriter has queued dwpt; will hijack this thread to flush pending segment(s)");
}
do {
@ -300,14 +298,14 @@ final class DocumentsWriter {
maybeMerge |= doFlush(flushingDWPT);
}
if (infoStream != null && flushControl.anyStalledThreads()) {
if (infoStream.isEnabled("DW") && flushControl.anyStalledThreads()) {
infoStream.message("DW", "WARNING DocumentsWriter has stalled threads; waiting");
}
flushControl.waitIfStalled(); // block if stalled
} while (flushControl.numQueuedFlushes() != 0); // still queued DWPTs try help flushing
if (infoStream != null) {
if (infoStream.isEnabled("DW")) {
infoStream.message("DW", "continue indexing after helping out flushing DocumentsWriter is healthy");
}
}
@ -466,7 +464,7 @@ final class DocumentsWriter {
final double ramBufferSizeMB = indexWriter.getConfig().getRAMBufferSizeMB();
if (ramBufferSizeMB != IndexWriterConfig.DISABLE_AUTO_FLUSH &&
flushControl.getDeleteBytesUsed() > (1024*1024*ramBufferSizeMB/2)) {
if (infoStream != null) {
if (infoStream.isEnabled("DW")) {
infoStream.message("DW", "force apply deletes bytesUsed=" + flushControl.getDeleteBytesUsed() + " vs ramBuffer=" + (1024*1024*ramBufferSizeMB));
}
applyAllDeletes(deleteQueue);
@ -500,7 +498,7 @@ final class DocumentsWriter {
assert bufferedDeletes != null;
if (bufferedDeletes != null && bufferedDeletes.any()) {
indexWriter.publishFrozenDeletes(bufferedDeletes);
if (infoStream != null) {
if (infoStream.isEnabled("DW")) {
infoStream.message("DW", "flush: push buffered deletes: " + bufferedDeletes);
}
}
@ -527,14 +525,14 @@ final class DocumentsWriter {
assert newSegment != null;
final SegmentInfo segInfo = indexWriter.prepareFlushedSegment(newSegment);
final BufferedDeletes deletes = newSegment.segmentDeletes;
if (infoStream != null) {
if (infoStream.isEnabled("DW")) {
infoStream.message("DW", Thread.currentThread().getName() + ": publishFlushedSegment seg-private deletes=" + deletes);
}
FrozenBufferedDeletes packet = null;
if (deletes != null && deletes.any()) {
// Segment private delete
packet = new FrozenBufferedDeletes(deletes, true);
if (infoStream != null) {
if (infoStream.isEnabled("DW")) {
infoStream.message("DW", "flush: push buffered seg private deletes: " + packet);
}
}
@ -560,7 +558,7 @@ final class DocumentsWriter {
final boolean flushAllThreads()
throws IOException {
final DocumentsWriterDeleteQueue flushingDeleteQueue;
if (infoStream != null) {
if (infoStream.isEnabled("DW")) {
infoStream.message("DW", Thread.currentThread().getName() + " startFullFlush");
}
@ -586,7 +584,7 @@ final class DocumentsWriter {
// If a concurrent flush is still in flight wait for it
flushControl.waitForFlush();
if (!anythingFlushed && flushingDeleteQueue.anyChanges()) { // apply deletes if we did not flush any document
if (infoStream != null) {
if (infoStream.isEnabled("DW")) {
infoStream.message("DW", Thread.currentThread().getName() + ": flush naked frozen global deletes");
}
synchronized (ticketQueue) {
@ -604,7 +602,7 @@ final class DocumentsWriter {
final void finishFullFlush(boolean success) {
try {
if (infoStream != null) {
if (infoStream.isEnabled("DW")) {
infoStream.message("DW", Thread.currentThread().getName() + " finishFullFlush success=" + success);
}
assert setFlushingDeleteQueue(null);

View File

@ -453,7 +453,7 @@ public final class DocumentsWriterFlushControl {
private final List<DocumentsWriterPerThread> fullFlushBuffer = new ArrayList<DocumentsWriterPerThread>();
void addFlushableState(ThreadState perThread) {
if (documentsWriter.infoStream != null) {
if (documentsWriter.infoStream.isEnabled("DWFC")) {
documentsWriter.infoStream.message("DWFC", Thread.currentThread().getName() + ": addFlushableState " + perThread.perThread);
}
final DocumentsWriterPerThread dwpt = perThread.perThread;

View File

@ -131,9 +131,7 @@ public class DocumentsWriterPerThread {
void abort() throws IOException {
hasAborted = aborting = true;
try {
if (infoStream != null) {
infoStream.message("DWPT", "now abort");
}
infoStream.message("DWPT", "now abort");
try {
consumer.abort();
} catch (Throwable t) {
@ -146,9 +144,7 @@ public class DocumentsWriterPerThread {
} finally {
aborting = false;
if (infoStream != null) {
infoStream.message("DWPT", "done abort");
}
infoStream.message("DWPT", "done abort");
}
}
private final static boolean INFO_VERBOSE = false;
@ -226,12 +222,12 @@ public class DocumentsWriterPerThread {
// this call is synchronized on IndexWriter.segmentInfos
segment = writer.newSegmentName();
assert numDocsInRAM == 0;
if (INFO_VERBOSE && infoStream != null) {
if (INFO_VERBOSE && infoStream.isEnabled("DWPT")) {
infoStream.message("DWPT", Thread.currentThread().getName() + " init seg=" + segment + " delQueue=" + deleteQueue);
}
}
if (INFO_VERBOSE && infoStream != null) {
if (INFO_VERBOSE && infoStream.isEnabled("DWPT")) {
infoStream.message("DWPT", Thread.currentThread().getName() + " update delTerm=" + delTerm + " docID=" + docState.docID + " seg=" + segment);
}
boolean success = false;
@ -274,11 +270,11 @@ public class DocumentsWriterPerThread {
// this call is synchronized on IndexWriter.segmentInfos
segment = writer.newSegmentName();
assert numDocsInRAM == 0;
if (INFO_VERBOSE && infoStream != null) {
if (INFO_VERBOSE && infoStream.isEnabled("DWPT")) {
infoStream.message("DWPT", Thread.currentThread().getName() + " init seg=" + segment + " delQueue=" + deleteQueue);
}
}
if (INFO_VERBOSE && infoStream != null) {
if (INFO_VERBOSE && infoStream.isEnabled("DWPT")) {
infoStream.message("DWPT", Thread.currentThread().getName() + " update delTerm=" + delTerm + " docID=" + docState.docID + " seg=" + segment);
}
int docCount = 0;
@ -459,12 +455,12 @@ public class DocumentsWriterPerThread {
pendingDeletes.docIDs.clear();
}
if (infoStream != null) {
if (infoStream.isEnabled("DWPT")) {
infoStream.message("DWPT", "flush postings as segment " + flushState.segmentName + " numDocs=" + numDocsInRAM);
}
if (aborting) {
if (infoStream != null) {
if (infoStream.isEnabled("DWPT")) {
infoStream.message("DWPT", "flush: skip because aborting is set");
}
return null;
@ -476,7 +472,7 @@ public class DocumentsWriterPerThread {
consumer.flush(flushState);
pendingDeletes.terms.clear();
final SegmentInfo newSegment = new SegmentInfo(segment, flushState.numDocs, directory, false, flushState.codec, fieldInfos.asReadOnly());
if (infoStream != null) {
if (infoStream.isEnabled("DWPT")) {
infoStream.message("DWPT", "new segment has " + (flushState.liveDocs == null ? 0 : (flushState.numDocs - flushState.liveDocs.count())) + " deleted docs");
infoStream.message("DWPT", "new segment has " + (newSegment.getHasVectors() ? "vectors" : "no vectors"));
infoStream.message("DWPT", "flushedFiles=" + newSegment.files());
@ -493,7 +489,7 @@ public class DocumentsWriterPerThread {
pendingDeletes = new BufferedDeletes();
}
if (infoStream != null) {
if (infoStream.isEnabled("DWPT")) {
final double newSegmentSizeNoStore = newSegment.sizeInBytes(false)/1024./1024.;
final double newSegmentSize = newSegment.sizeInBytes(true)/1024./1024.;
infoStream.message("DWPT", "flushed: segment=" + newSegment +

View File

@ -63,7 +63,7 @@ public class FlushByRamOrCountsPolicy extends FlushPolicy {
if ((flushOnRAM() &&
control.getDeleteBytesUsed() > (1024*1024*indexWriterConfig.getRAMBufferSizeMB()))) {
control.setApplyAllDeletes();
if (writer.infoStream != null) {
if (writer.infoStream.isEnabled("FP")) {
writer.infoStream.message("FP", "force apply deletes bytesUsed=" + control.getDeleteBytesUsed() + " vs ramBuffer=" + (1024*1024*indexWriterConfig.getRAMBufferSizeMB()));
}
}
@ -81,7 +81,7 @@ public class FlushByRamOrCountsPolicy extends FlushPolicy {
final long totalRam = control.activeBytes() + control.getDeleteBytesUsed();
if (totalRam >= limit) {
final DocumentsWriter writer = this.writer.get();
if (writer.infoStream != null) {
if (writer.infoStream.isEnabled("FP")) {
writer.infoStream.message("FP", "flush: activeBytes=" + control.activeBytes() + " deleteBytes=" + control.getDeleteBytesUsed() + " vs limit=" + limit);
}
markLargestWriterPending(control, state, totalRam);

View File

@ -129,10 +129,7 @@ public abstract class FlushPolicy {
}
private boolean assertMessage(String s) {
InfoStream infoStream = writer.get().infoStream;
if (infoStream != null) {
infoStream.message("FP", s);
}
writer.get().infoStream.message("FP", s);
return true;
}

View File

@ -102,7 +102,7 @@ final class IndexFileDeleter {
private SegmentInfos lastSegmentInfos;
/** Change to true to see details of reference counts when
* infoStream != null */
* infoStream is enabled */
public static boolean VERBOSE_REF_COUNTS = false;
// Used only for assert
@ -128,7 +128,7 @@ final class IndexFileDeleter {
final String currentSegmentsFile = segmentInfos.getCurrentSegmentFileName();
if (infoStream != null) {
if (infoStream.isEnabled("IFD")) {
infoStream.message("IFD", "init: current segments file is \"" + currentSegmentsFile + "\"; deletionPolicy=" + policy);
}
@ -160,7 +160,7 @@ final class IndexFileDeleter {
// This is a commit (segments or segments_N), and
// it's valid (<= the max gen). Load it, then
// incref all files it refers to:
if (infoStream != null) {
if (infoStream.isEnabled("IFD")) {
infoStream.message("IFD", "init: load commit \"" + fileName + "\"");
}
SegmentInfos sis = new SegmentInfos();
@ -174,7 +174,7 @@ final class IndexFileDeleter {
// file segments_X exists when in fact it
// doesn't. So, we catch this and handle it
// as if the file does not exist
if (infoStream != null) {
if (infoStream.isEnabled("IFD")) {
infoStream.message("IFD", "init: hit FileNotFoundException when loading commit \"" + fileName + "\"; skipping this commit point");
}
sis = null;
@ -205,7 +205,7 @@ final class IndexFileDeleter {
} catch (FileNotFoundException e) {
refresh(segmentInfo.name);
sis = null;
if (infoStream != null) {
if (infoStream.isEnabled("IFD")) {
infoStream.message("IFD", "init: hit FileNotFoundException when loading commit \"" + fileName + "\"; skipping this commit point");
}
}
@ -242,7 +242,7 @@ final class IndexFileDeleter {
} catch (IOException e) {
throw new CorruptIndexException("failed to locate current segments_N file");
}
if (infoStream != null) {
if (infoStream.isEnabled("IFD")) {
infoStream.message("IFD", "forced open of current segments file " + segmentInfos.getCurrentSegmentFileName());
}
currentCommitPoint = new CommitPoint(commitsToDelete, directory, sis);
@ -260,7 +260,7 @@ final class IndexFileDeleter {
RefCount rc = entry.getValue();
final String fileName = entry.getKey();
if (0 == rc.count) {
if (infoStream != null) {
if (infoStream.isEnabled("IFD")) {
infoStream.message("IFD", "init: removing unreferenced file \"" + fileName + "\"");
}
deleteFile(fileName);
@ -300,7 +300,7 @@ final class IndexFileDeleter {
// the now-deleted commits:
for(int i=0;i<size;i++) {
CommitPoint commit = commitsToDelete.get(i);
if (infoStream != null) {
if (infoStream.isEnabled("IFD")) {
infoStream.message("IFD", "deleteCommits: now decRef commit \"" + commit.getSegmentsFileName() + "\"");
}
for (final String file : commit.files) {
@ -360,7 +360,7 @@ final class IndexFileDeleter {
!refCounts.containsKey(fileName) &&
!fileName.equals(IndexFileNames.SEGMENTS_GEN)) {
// Unreferenced file, so remove it
if (infoStream != null) {
if (infoStream.isEnabled("IFD")) {
infoStream.message("IFD", "refresh [prefix=" + segmentName + "]: removing newly created unreferenced file \"" + fileName + "\"");
}
deleteFile(fileName);
@ -402,7 +402,7 @@ final class IndexFileDeleter {
*/
void revisitPolicy() throws IOException {
assert locked();
if (infoStream != null) {
if (infoStream.isEnabled("IFD")) {
infoStream.message("IFD", "now revisitPolicy");
}
@ -419,7 +419,7 @@ final class IndexFileDeleter {
deletable = null;
int size = oldDeletable.size();
for(int i=0;i<size;i++) {
if (infoStream != null) {
if (infoStream.isEnabled("IFD")) {
infoStream.message("IFD", "delete pending file " + oldDeletable.get(i));
}
deleteFile(oldDeletable.get(i));
@ -450,7 +450,7 @@ final class IndexFileDeleter {
public void checkpoint(SegmentInfos segmentInfos, boolean isCommit) throws IOException {
assert locked();
if (infoStream != null) {
if (infoStream.isEnabled("IFD")) {
infoStream.message("IFD", "now checkpoint \"" + segmentInfos.toString(directory) + "\" [" + segmentInfos.size() + " segments " + "; isCommit = " + isCommit + "]");
}
@ -501,7 +501,7 @@ final class IndexFileDeleter {
void incRef(String fileName) throws IOException {
assert locked();
RefCount rc = getRefCount(fileName);
if (infoStream != null && VERBOSE_REF_COUNTS) {
if (infoStream.isEnabled("IFD") && VERBOSE_REF_COUNTS) {
infoStream.message("IFD", " IncRef \"" + fileName + "\": pre-incr count is " + rc.count);
}
rc.IncRef();
@ -517,7 +517,7 @@ final class IndexFileDeleter {
void decRef(String fileName) throws IOException {
assert locked();
RefCount rc = getRefCount(fileName);
if (infoStream != null && VERBOSE_REF_COUNTS) {
if (infoStream.isEnabled("IFD") && VERBOSE_REF_COUNTS) {
infoStream.message("IFD", " DecRef \"" + fileName + "\": pre-decr count is " + rc.count);
}
if (0 == rc.DecRef()) {
@ -569,7 +569,7 @@ final class IndexFileDeleter {
assert locked();
for (final String fileName: files) {
if (!refCounts.containsKey(fileName)) {
if (infoStream != null) {
if (infoStream.isEnabled("IFD")) {
infoStream.message("IFD", "delete new file \"" + fileName + "\"");
}
deleteFile(fileName);
@ -581,7 +581,7 @@ final class IndexFileDeleter {
throws IOException {
assert locked();
try {
if (infoStream != null) {
if (infoStream.isEnabled("IFD")) {
infoStream.message("IFD", "delete \"" + fileName + "\"");
}
directory.deleteFile(fileName);
@ -595,7 +595,7 @@ final class IndexFileDeleter {
// the file is open in another process, and queue
// the file for subsequent deletion.
if (infoStream != null) {
if (infoStream.isEnabled("IFD")) {
infoStream.message("IFD", "unable to remove file \"" + fileName + "\": " + e.toString() + "; Will re-try later.");
}
if (deletable == null) {

View File

@ -131,11 +131,11 @@ public final class IndexUpgrader {
final IndexWriter w = new IndexWriter(dir, c);
try {
InfoStream infoStream = c.getInfoStream();
if (infoStream != null) {
if (infoStream.isEnabled("IndexUpgrader")) {
infoStream.message("IndexUpgrader", "Upgrading all pre-" + Constants.LUCENE_MAIN_VERSION + " segments of index directory '" + dir + "' to version " + Constants.LUCENE_MAIN_VERSION + "...");
}
w.forceMerge(1);
if (infoStream != null) {
if (infoStream.isEnabled("IndexUpgrader")) {
infoStream.message("IndexUpgrader", "All segments upgraded to version " + Constants.LUCENE_MAIN_VERSION);
}
} finally {

View File

@ -331,9 +331,7 @@ public class IndexWriter implements Closeable, TwoPhaseCommit {
final long tStart = System.currentTimeMillis();
if (infoStream != null) {
infoStream.message("IW", "flush at getReader");
}
infoStream.message("IW", "flush at getReader");
// Do this up front before flushing so that the readers
// obtained during this flush are pooled, the first time
// this method is called:
@ -364,7 +362,7 @@ public class IndexWriter implements Closeable, TwoPhaseCommit {
synchronized(this) {
maybeApplyDeletes(applyAllDeletes);
r = new DirectoryReader(this, segmentInfos, applyAllDeletes);
if (infoStream != null) {
if (infoStream.isEnabled("IW")) {
infoStream.message("IW", "return reader version=" + r.getVersion() + " reader=" + r);
}
}
@ -373,9 +371,7 @@ public class IndexWriter implements Closeable, TwoPhaseCommit {
// never reached but javac disagrees:
return null;
} finally {
if (!success && infoStream != null) {
infoStream.message("IW", "hit exception during NRT reader");
}
infoStream.message("IW", "hit exception during NRT reader");
// Done: finish the full flush!
docWriter.finishFullFlush(success);
doAfterFlush();
@ -384,7 +380,7 @@ public class IndexWriter implements Closeable, TwoPhaseCommit {
if (anySegmentFlushed) {
maybeMerge();
}
if (infoStream != null) {
if (infoStream.isEnabled("IW")) {
infoStream.message("IW", "getReader took " + (System.currentTimeMillis() - tStart) + " msec");
}
return r;
@ -874,7 +870,7 @@ public class IndexWriter implements Closeable, TwoPhaseCommit {
segmentInfos.replace(oldInfos);
changeCount++;
segmentInfos.changed();
if (infoStream != null)
if (infoStream.isEnabled("IW"))
infoStream.message("IW", "init: loaded commit \"" + commit.getSegmentsFileName() + "\"");
}
}
@ -902,7 +898,7 @@ public class IndexWriter implements Closeable, TwoPhaseCommit {
segmentInfos.changed();
}
if (infoStream != null) {
if (infoStream.isEnabled("IW")) {
infoStream.message("IW", "init: create=" + create);
messageState();
}
@ -911,9 +907,7 @@ public class IndexWriter implements Closeable, TwoPhaseCommit {
} finally {
if (!success) {
if (infoStream != null) {
infoStream.message("IW", "init: hit exception on init; releasing write lock");
}
infoStream.message("IW", "init: hit exception on init; releasing write lock");
try {
writeLock.release();
} catch (Throwable t) {
@ -941,7 +935,7 @@ public class IndexWriter implements Closeable, TwoPhaseCommit {
}
private void messageState() throws IOException {
if (infoStream != null) {
if (infoStream.isEnabled("IW")) {
infoStream.message("IW", "\ndir=" + directory + "\n" +
"index=" + segString() + "\n" +
"version=" + Constants.LUCENE_VERSION + "\n" +
@ -949,11 +943,6 @@ public class IndexWriter implements Closeable, TwoPhaseCommit {
}
}
/** Returns true if verbosing is enabled (i.e., infoStream != null). */
public boolean verbose() {
return infoStream != null;
}
/**
* Commits all changes to an index and closes all
* associated files. Note that this may be a costly
@ -1060,7 +1049,7 @@ public class IndexWriter implements Closeable, TwoPhaseCommit {
private void closeInternal(boolean waitForMerges) throws CorruptIndexException, IOException {
try {
if (infoStream != null) {
if (infoStream.isEnabled("IW")) {
infoStream.message("IW", "now flush at close waitForMerges=" + waitForMerges);
}
@ -1086,14 +1075,13 @@ public class IndexWriter implements Closeable, TwoPhaseCommit {
mergeScheduler.close();
if (infoStream != null)
infoStream.message("IW", "now call final commit()");
infoStream.message("IW", "now call final commit()");
if (!hitOOM) {
commitInternal(null);
}
if (infoStream != null)
if (infoStream.isEnabled("IW"))
infoStream.message("IW", "at close: " + segString());
// used by assert below
final DocumentsWriter oldWriter = docWriter;
@ -1118,8 +1106,7 @@ public class IndexWriter implements Closeable, TwoPhaseCommit {
closing = false;
notifyAll();
if (!closed) {
if (infoStream != null)
infoStream.message("IW", "hit exception while closing");
infoStream.message("IW", "hit exception while closing");
}
}
}
@ -1348,9 +1335,7 @@ public class IndexWriter implements Closeable, TwoPhaseCommit {
anySegmentFlushed = docWriter.updateDocuments(docs, analyzer, delTerm);
success = true;
} finally {
if (!success && infoStream != null) {
infoStream.message("IW", "hit exception updating document");
}
infoStream.message("IW", "hit exception updating document");
}
if (anySegmentFlushed) {
maybeMerge();
@ -1495,8 +1480,7 @@ public class IndexWriter implements Closeable, TwoPhaseCommit {
anySegmentFlushed = docWriter.updateDocument(doc, analyzer, term);
success = true;
} finally {
if (!success && infoStream != null)
infoStream.message("IW", "hit exception updating document");
infoStream.message("IW", "hit exception updating document");
}
if (anySegmentFlushed) {
@ -1644,7 +1628,7 @@ public class IndexWriter implements Closeable, TwoPhaseCommit {
if (maxNumSegments < 1)
throw new IllegalArgumentException("maxNumSegments must be >= 1; got " + maxNumSegments);
if (infoStream != null) {
if (infoStream.isEnabled("IW")) {
infoStream.message("IW", "forceMerge: index now " + segString());
infoStream.message("IW", "now flush at forceMerge");
}
@ -1753,7 +1737,7 @@ public class IndexWriter implements Closeable, TwoPhaseCommit {
flush(true, true);
if (infoStream != null)
if (infoStream.isEnabled("IW"))
infoStream.message("IW", "forceMergeDeletes: index now " + segString());
MergePolicy.MergeSpecification spec;
@ -1947,9 +1931,7 @@ public class IndexWriter implements Closeable, TwoPhaseCommit {
boolean success = false;
if (infoStream != null ) {
infoStream.message("IW", "rollback");
}
infoStream.message("IW", "rollback");
try {
synchronized(this) {
@ -1957,9 +1939,7 @@ public class IndexWriter implements Closeable, TwoPhaseCommit {
stopMerges = true;
}
if (infoStream != null ) {
infoStream.message("IW", "rollback: done finish merges");
}
infoStream.message("IW", "rollback: done finish merges");
// Must pre-close these two, in case they increment
// changeCount so that we can then set it to false
@ -1984,7 +1964,7 @@ public class IndexWriter implements Closeable, TwoPhaseCommit {
// will always write to a new generation ("write
// once").
segmentInfos.rollbackSegmentInfos(rollbackSegments);
if (infoStream != null ) {
if (infoStream.isEnabled("IW") ) {
infoStream.message("IW", "rollback: infos=" + segString(segmentInfos));
}
@ -2011,8 +1991,7 @@ public class IndexWriter implements Closeable, TwoPhaseCommit {
if (!success) {
closing = false;
notifyAll();
if (infoStream != null)
infoStream.message("IW", "hit exception during rollback");
infoStream.message("IW", "hit exception during rollback");
}
}
}
@ -2064,9 +2043,7 @@ public class IndexWriter implements Closeable, TwoPhaseCommit {
} catch (OutOfMemoryError oom) {
handleOOM(oom, "deleteAll");
} finally {
if (!success && infoStream != null) {
infoStream.message("IW", "hit exception during deleteAll");
}
infoStream.message("IW", "hit exception during deleteAll");
}
}
@ -2077,7 +2054,7 @@ public class IndexWriter implements Closeable, TwoPhaseCommit {
// Abort all pending & running merges:
for (final MergePolicy.OneMerge merge : pendingMerges) {
if (infoStream != null)
if (infoStream.isEnabled("IW"))
infoStream.message("IW", "now abort pending merge " + merge.segString(directory));
merge.abort();
mergeFinish(merge);
@ -2085,7 +2062,7 @@ public class IndexWriter implements Closeable, TwoPhaseCommit {
pendingMerges.clear();
for (final MergePolicy.OneMerge merge : runningMerges) {
if (infoStream != null)
if (infoStream.isEnabled("IW"))
infoStream.message("IW", "now abort running merge " + merge.segString(directory));
merge.abort();
}
@ -2096,7 +2073,7 @@ public class IndexWriter implements Closeable, TwoPhaseCommit {
// because the merge threads periodically check if
// they are aborted.
while(runningMerges.size() > 0) {
if (infoStream != null)
if (infoStream.isEnabled("IW"))
infoStream.message("IW", "now wait for " + runningMerges.size() + " running merge to abort");
doWait();
}
@ -2106,8 +2083,7 @@ public class IndexWriter implements Closeable, TwoPhaseCommit {
assert 0 == mergingSegments.size();
if (infoStream != null)
infoStream.message("IW", "all running merges have aborted");
infoStream.message("IW", "all running merges have aborted");
} else {
// waitForMerges() will ensure any running addIndexes finishes.
@ -2127,9 +2103,7 @@ public class IndexWriter implements Closeable, TwoPhaseCommit {
*/
public synchronized void waitForMerges() {
ensureOpen(false);
if (infoStream != null) {
infoStream.message("IW", "waitForMerges");
}
infoStream.message("IW", "waitForMerges");
while(pendingMerges.size() > 0 || runningMerges.size() > 0) {
doWait();
}
@ -2137,9 +2111,7 @@ public class IndexWriter implements Closeable, TwoPhaseCommit {
// sanity check
assert 0 == mergingSegments.size();
if (infoStream != null) {
infoStream.message("IW", "waitForMerges done");
}
infoStream.message("IW", "waitForMerges done");
}
/**
@ -2175,7 +2147,7 @@ public class IndexWriter implements Closeable, TwoPhaseCommit {
try {
if (useCompoundFile(newSegment)) {
String compoundFileName = IndexFileNames.segmentFileName(newSegment.name, "", IndexFileNames.COMPOUND_FILE_EXTENSION);
if (infoStream != null) {
if (infoStream.isEnabled("IW")) {
infoStream.message("IW", "creating compound file " + compoundFileName);
}
// Now build compound file
@ -2207,7 +2179,7 @@ public class IndexWriter implements Closeable, TwoPhaseCommit {
newSegment.setDelCount(delCount);
newSegment.advanceDelGen();
final String delFileName = newSegment.getDelFileName();
if (infoStream != null) {
if (infoStream.isEnabled("IW")) {
infoStream.message("IW", "flush: write " + delCount + " deletes to " + delFileName);
}
boolean success2 = false;
@ -2234,7 +2206,7 @@ public class IndexWriter implements Closeable, TwoPhaseCommit {
success = true;
} finally {
if (!success) {
if (infoStream != null) {
if (infoStream.isEnabled("IW")) {
infoStream.message("IW", "hit exception " +
"reating compound file for newly flushed segment " + newSegment.name);
}
@ -2266,9 +2238,7 @@ public class IndexWriter implements Closeable, TwoPhaseCommit {
FrozenBufferedDeletes packet, FrozenBufferedDeletes globalPacket) throws IOException {
// Lock order IW -> BDS
synchronized (bufferedDeletesStream) {
if (infoStream != null) {
infoStream.message("IW", "publishFlushedSegment");
}
infoStream.message("IW", "publishFlushedSegment");
if (globalPacket != null && globalPacket.any()) {
bufferedDeletesStream.push(globalPacket);
@ -2283,7 +2253,7 @@ public class IndexWriter implements Closeable, TwoPhaseCommit {
// generation right away
nextGen = bufferedDeletesStream.getNextGen();
}
if (infoStream != null) {
if (infoStream.isEnabled("IW")) {
infoStream.message("IW", "publish sets newSegment delGen=" + nextGen);
}
newSegment.setBufferedDeletesGen(nextGen);
@ -2362,13 +2332,12 @@ public class IndexWriter implements Closeable, TwoPhaseCommit {
noDupDirs(dirs);
try {
if (infoStream != null)
infoStream.message("IW", "flush at addIndexes(Directory...)");
infoStream.message("IW", "flush at addIndexes(Directory...)");
flush(false, true);
List<SegmentInfo> infos = new ArrayList<SegmentInfo>();
for (Directory dir : dirs) {
if (infoStream != null) {
if (infoStream.isEnabled("IW")) {
infoStream.message("IW", "addIndexes: process directory " + dir);
}
SegmentInfos sis = new SegmentInfos(); // read infos from dir
@ -2381,7 +2350,7 @@ public class IndexWriter implements Closeable, TwoPhaseCommit {
String newSegName = newSegmentName();
String dsName = info.getDocStoreSegment();
if (infoStream != null) {
if (infoStream.isEnabled("IW")) {
infoStream.message("IW", "addIndexes: process segment origName=" + info.name + " newName=" + newSegName + " dsName=" + dsName + " info=" + info);
}
@ -2432,8 +2401,7 @@ public class IndexWriter implements Closeable, TwoPhaseCommit {
int numDocs = 0;
try {
if (infoStream != null)
infoStream.message("IW", "flush at addIndexes(IndexReader...)");
infoStream.message("IW", "flush at addIndexes(IndexReader...)");
flush(false, true);
String mergedName = newSegmentName();
@ -2600,7 +2568,7 @@ public class IndexWriter implements Closeable, TwoPhaseCommit {
public final void prepareCommit(Map<String,String> commitUserData) throws CorruptIndexException, IOException {
ensureOpen(false);
if (infoStream != null) {
if (infoStream.isEnabled("IW")) {
infoStream.message("IW", "prepareCommit: flush");
infoStream.message("IW", " index before flush " + segString());
}
@ -2660,7 +2628,7 @@ public class IndexWriter implements Closeable, TwoPhaseCommit {
}
success = true;
} finally {
if (!success && infoStream != null) {
if (!success) {
infoStream.message("IW", "hit exception during prepareCommit");
}
// Done: finish the full flush!
@ -2745,21 +2713,15 @@ public class IndexWriter implements Closeable, TwoPhaseCommit {
private final void commitInternal(Map<String,String> commitUserData) throws CorruptIndexException, IOException {
if (infoStream != null) {
infoStream.message("IW", "commit: start");
}
infoStream.message("IW", "commit: start");
synchronized(commitLock) {
if (infoStream != null) {
infoStream.message("IW", "commit: enter lock");
}
infoStream.message("IW", "commit: enter lock");
if (pendingCommit == null) {
if (infoStream != null) {
infoStream.message("IW", "commit: now prepare");
}
infoStream.message("IW", "commit: now prepare");
prepareCommit(commitUserData);
} else if (infoStream != null) {
} else {
infoStream.message("IW", "commit: already prepared");
}
@ -2771,10 +2733,9 @@ public class IndexWriter implements Closeable, TwoPhaseCommit {
if (pendingCommit != null) {
try {
if (infoStream != null)
infoStream.message("IW", "commit: pendingCommit != null");
infoStream.message("IW", "commit: pendingCommit != null");
pendingCommit.finishCommit(directory, codec);
if (infoStream != null)
if (infoStream.isEnabled("IW"))
infoStream.message("IW", "commit: wrote segments file \"" + pendingCommit.getCurrentSegmentFileName() + "\"");
lastCommitChangeCount = pendingCommitChangeCount;
segmentInfos.updateGeneration(pendingCommit);
@ -2789,13 +2750,11 @@ public class IndexWriter implements Closeable, TwoPhaseCommit {
notifyAll();
}
} else if (infoStream != null) {
} else {
infoStream.message("IW", "commit: pendingCommit == null; skip");
}
if (infoStream != null) {
infoStream.message("IW", "commit: done");
}
infoStream.message("IW", "commit: done");
}
// Ensures only one flush() is actually flushing segments
@ -2835,7 +2794,7 @@ public class IndexWriter implements Closeable, TwoPhaseCommit {
boolean success = false;
try {
if (infoStream != null) {
if (infoStream.isEnabled("IW")) {
infoStream.message("IW", " start flush: applyAllDeletes=" + applyAllDeletes);
infoStream.message("IW", " index before flush " + segString());
}
@ -2865,18 +2824,16 @@ public class IndexWriter implements Closeable, TwoPhaseCommit {
// never hit
return false;
} finally {
if (!success && infoStream != null)
if (!success)
infoStream.message("IW", "hit exception during flush");
}
}
final synchronized void maybeApplyDeletes(boolean applyAllDeletes) throws IOException {
if (applyAllDeletes) {
if (infoStream != null) {
infoStream.message("IW", "apply all deletes during flush");
}
infoStream.message("IW", "apply all deletes during flush");
applyAllDeletes();
} else if (infoStream != null) {
} else if (infoStream.isEnabled("IW")) {
infoStream.message("IW", "don't apply deletes now delTermCount=" + bufferedDeletesStream.numTerms() + " bytesUsed=" + bufferedDeletesStream.bytesUsed());
}
}
@ -2889,7 +2846,7 @@ public class IndexWriter implements Closeable, TwoPhaseCommit {
checkpoint();
}
if (!keepFullyDeletedSegments && result.allDeleted != null) {
if (infoStream != null) {
if (infoStream.isEnabled("IW")) {
infoStream.message("IW", "drop 100% deleted segments: " + segString(result.allDeleted));
}
for (SegmentInfo info : result.allDeleted) {
@ -2954,7 +2911,7 @@ public class IndexWriter implements Closeable, TwoPhaseCommit {
final List<SegmentInfo> sourceSegments = merge.segments;
if (infoStream != null)
if (infoStream.isEnabled("IW"))
infoStream.message("IW", "commitMergeDeletes " + merge.segString(directory));
// Carefully merge deletes that occurred after we
@ -3052,7 +3009,7 @@ public class IndexWriter implements Closeable, TwoPhaseCommit {
throw new IllegalStateException("this writer hit an OutOfMemoryError; cannot complete merge");
}
if (infoStream != null)
if (infoStream.isEnabled("IW"))
infoStream.message("IW", "commitMerge: " + merge.segString(directory) + " index=" + segString());
assert merge.registerDone;
@ -3064,7 +3021,7 @@ public class IndexWriter implements Closeable, TwoPhaseCommit {
// file that current segments does not reference), we
// abort this merge
if (merge.isAborted()) {
if (infoStream != null)
if (infoStream.isEnabled("IW"))
infoStream.message("IW", "commitMerge: skipping merge " + merge.segString(directory) + ": it was aborted");
return false;
}
@ -3080,7 +3037,7 @@ public class IndexWriter implements Closeable, TwoPhaseCommit {
final boolean allDeleted = mergedReader.numDocs() == 0;
if (infoStream != null && allDeleted) {
if (allDeleted && infoStream.isEnabled("IW")) {
infoStream.message("IW", "merged segment " + merge.info + " is 100% deleted" + (keepFullyDeletedSegments ? "" : "; skipping insert"));
}
@ -3091,7 +3048,7 @@ public class IndexWriter implements Closeable, TwoPhaseCommit {
readerPool.drop(merge.info);
}
if (infoStream != null) {
if (infoStream.isEnabled("IW")) {
infoStream.message("IW", "after commit: " + segString());
}
@ -3118,7 +3075,7 @@ public class IndexWriter implements Closeable, TwoPhaseCommit {
final private void handleMergeException(Throwable t, MergePolicy.OneMerge merge) throws IOException {
if (infoStream != null) {
if (infoStream.isEnabled("IW")) {
infoStream.message("IW", "handleMergeException: merge=" + merge.segString(directory) + " exc=" + t);
}
@ -3167,7 +3124,7 @@ public class IndexWriter implements Closeable, TwoPhaseCommit {
try {
mergeInit(merge);
if (infoStream != null)
if (infoStream.isEnabled("IW"))
infoStream.message("IW", "now merge\n merge=" + merge.segString(directory) + "\n index=" + segString());
mergeMiddle(merge);
@ -3181,8 +3138,7 @@ public class IndexWriter implements Closeable, TwoPhaseCommit {
mergeFinish(merge);
if (!success) {
if (infoStream != null)
infoStream.message("IW", "hit exception during merge");
infoStream.message("IW", "hit exception during merge");
if (merge.info != null && !segmentInfos.contains(merge.info))
deleter.refresh(merge.info.name);
}
@ -3198,7 +3154,7 @@ public class IndexWriter implements Closeable, TwoPhaseCommit {
} catch (OutOfMemoryError oom) {
handleOOM(oom, "merge");
}
if (infoStream != null && merge.info != null) {
if (merge.info != null && infoStream.isEnabled("IW")) {
infoStream.message("IW", "merge time " + (System.currentTimeMillis()-t0) + " msec for " + merge.info.docCount + " docs");
}
//System.out.println(Thread.currentThread().getName() + ": merge end");
@ -3244,7 +3200,7 @@ public class IndexWriter implements Closeable, TwoPhaseCommit {
pendingMerges.add(merge);
if (infoStream != null)
if (infoStream.isEnabled("IW"))
infoStream.message("IW", "add merge to pendingMerges: " + merge.segString(directory) + " [total " + pendingMerges.size() + " pending]");
merge.mergeGen = mergeGen;
@ -3254,7 +3210,7 @@ public class IndexWriter implements Closeable, TwoPhaseCommit {
// is running (while synchronized) to avoid race
// condition where two conflicting merges from different
// threads, start
if (infoStream != null) {
if (infoStream.isEnabled("IW")) {
StringBuilder builder = new StringBuilder("registerMerge merging= [");
for (SegmentInfo info : mergingSegments) {
builder.append(info.name).append(", ");
@ -3265,7 +3221,7 @@ public class IndexWriter implements Closeable, TwoPhaseCommit {
infoStream.message("IW", builder.toString());
}
for(SegmentInfo info : merge.segments) {
if (infoStream != null) {
if (infoStream.isEnabled("IW")) {
infoStream.message("IW", "registerMerge info=" + info);
}
mergingSegments.add(info);
@ -3286,9 +3242,7 @@ public class IndexWriter implements Closeable, TwoPhaseCommit {
success = true;
} finally {
if (!success) {
if (infoStream != null) {
infoStream.message("IW", "hit exception in mergeInit");
}
infoStream.message("IW", "hit exception in mergeInit");
mergeFinish(merge);
}
}
@ -3328,7 +3282,7 @@ public class IndexWriter implements Closeable, TwoPhaseCommit {
}
if (!keepFullyDeletedSegments && result.allDeleted != null) {
if (infoStream != null) {
if (infoStream.isEnabled("IW")) {
infoStream.message("IW", "drop 100% deleted segments: " + result.allDeleted);
}
for(SegmentInfo info : result.allDeleted) {
@ -3353,7 +3307,7 @@ public class IndexWriter implements Closeable, TwoPhaseCommit {
details.put("mergeFactor", Integer.toString(merge.segments.size()));
setDiagnostics(merge.info, "merge", details);
if (infoStream != null) {
if (infoStream.isEnabled("IW")) {
infoStream.message("IW", "merge seg=" + merge.info.name);
}
@ -3495,7 +3449,7 @@ public class IndexWriter implements Closeable, TwoPhaseCommit {
SegmentMerger merger = new SegmentMerger(infoStream, directory, config.getTermIndexInterval(), mergedName, checkAbort,
payloadProcessorProvider, merge.info.getFieldInfos(), codec, context);
if (infoStream != null) {
if (infoStream.isEnabled("IW")) {
infoStream.message("IW", "merging " + merge.segString(directory) + " mergeVectors=" + merge.info.getFieldInfos().hasVectors());
}
@ -3529,7 +3483,7 @@ public class IndexWriter implements Closeable, TwoPhaseCommit {
segUpto++;
}
if (infoStream != null) {
if (infoStream.isEnabled("IW")) {
infoStream.message("IW", "merge: total " + totDocCount + " docs");
}
@ -3542,7 +3496,7 @@ public class IndexWriter implements Closeable, TwoPhaseCommit {
// Record which codec was used to write the segment
merge.info.setCodec(codec);
if (infoStream != null) {
if (infoStream.isEnabled("IW")) {
infoStream.message("IW", "merge codec=" + codec);
}
@ -3562,7 +3516,7 @@ public class IndexWriter implements Closeable, TwoPhaseCommit {
final String compoundFileName = IndexFileNames.segmentFileName(mergedName, "", IndexFileNames.COMPOUND_FILE_EXTENSION);
try {
if (infoStream != null) {
if (infoStream.isEnabled("IW")) {
infoStream.message("IW", "create compound file " + compoundFileName);
}
createCompoundFile(directory, compoundFileName, checkAbort, merge.info, new IOContext(merge.getMergeInfo()));
@ -3581,9 +3535,7 @@ public class IndexWriter implements Closeable, TwoPhaseCommit {
handleMergeException(t, merge);
} finally {
if (!success) {
if (infoStream != null) {
infoStream.message("IW", "hit exception creating compound file during merge");
}
infoStream.message("IW", "hit exception creating compound file during merge");
synchronized(this) {
deleter.deleteFile(compoundFileName);
@ -3602,9 +3554,7 @@ public class IndexWriter implements Closeable, TwoPhaseCommit {
deleter.deleteNewFiles(merge.info.files());
if (merge.isAborted()) {
if (infoStream != null) {
infoStream.message("IW", "abort merge after building CFS");
}
infoStream.message("IW", "abort merge after building CFS");
deleter.deleteFile(compoundFileName);
return 0;
}
@ -3613,7 +3563,7 @@ public class IndexWriter implements Closeable, TwoPhaseCommit {
merge.info.setUseCompoundFile(true);
}
if (infoStream != null) {
if (infoStream.isEnabled("IW")) {
infoStream.message("IW", String.format("merged segment size=%.3f MB vs estimate=%.3f MB", merge.info.sizeInBytes()/1024./1024., merge.estimatedMergeBytes/1024/1024.));
}
@ -3787,24 +3737,20 @@ public class IndexWriter implements Closeable, TwoPhaseCommit {
try {
if (infoStream != null) {
infoStream.message("IW", "startCommit(): start");
}
infoStream.message("IW", "startCommit(): start");
synchronized(this) {
assert lastCommitChangeCount <= changeCount;
if (pendingCommitChangeCount == lastCommitChangeCount) {
if (infoStream != null) {
infoStream.message("IW", " skip startCommit(): no changes pending");
}
infoStream.message("IW", " skip startCommit(): no changes pending");
deleter.decRef(filesToCommit);
filesToCommit = null;
return;
}
if (infoStream != null) {
if (infoStream.isEnabled("IW")) {
infoStream.message("IW", "startCommit index=" + segString(toSync) + " changeCount=" + changeCount);
}
@ -3841,9 +3787,7 @@ public class IndexWriter implements Closeable, TwoPhaseCommit {
pendingCommit = toSync;
}
if (infoStream != null) {
infoStream.message("IW", "done all syncs");
}
infoStream.message("IW", "done all syncs");
assert testPoint("midStartCommitSuccess");
@ -3856,9 +3800,7 @@ public class IndexWriter implements Closeable, TwoPhaseCommit {
segmentInfos.updateGeneration(toSync);
if (!pendingCommitSet) {
if (infoStream != null) {
infoStream.message("IW", "hit exception committing segments file");
}
infoStream.message("IW", "hit exception committing segments file");
// Hit exception
deleter.decRef(filesToCommit);
@ -3910,7 +3852,7 @@ public class IndexWriter implements Closeable, TwoPhaseCommit {
}
private void handleOOM(OutOfMemoryError oom, String location) {
if (infoStream != null) {
if (infoStream.isEnabled("IW")) {
infoStream.message("IW", "hit OutOfMemoryError inside " + location);
}
hitOOM = true;
@ -3935,7 +3877,7 @@ public class IndexWriter implements Closeable, TwoPhaseCommit {
synchronized boolean nrtIsCurrent(SegmentInfos infos) {
//System.out.println("IW.nrtIsCurrent " + (infos.version == segmentInfos.version && !docWriter.anyChanges() && !bufferedDeletesStream.any()));
ensureOpen();
if (infoStream != null) {
if (infoStream.isEnabled("IW")) {
infoStream.message("IW", "nrtIsCurrent: infoVersion matches: " + (infos.version == segmentInfos.version) + " DW changes: " + docWriter.anyChanges() + " BD changes: "+bufferedDeletesStream.any());
}

View File

@ -695,6 +695,10 @@ public final class IndexWriterConfig implements Cloneable {
* to this.
*/
public IndexWriterConfig setInfoStream(InfoStream infoStream) {
if (infoStream == null) {
throw new IllegalArgumentException("Cannot set InfoStream implementation to null. "+
"To disable logging use InfoStream.NO_OUTPUT");
}
this.infoStream = infoStream;
return this;
}
@ -703,8 +707,7 @@ public final class IndexWriterConfig implements Cloneable {
* Convenience method that uses {@link PrintStreamInfoStream}
*/
public IndexWriterConfig setInfoStream(PrintStream printStream) {
this.infoStream = printStream == null ? null : new PrintStreamInfoStream(printStream);
return this;
return setInfoStream(printStream == null ? InfoStream.NO_OUTPUT : new PrintStreamInfoStream(printStream));
}
@Override
@ -725,7 +728,7 @@ public final class IndexWriterConfig implements Cloneable {
sb.append("maxBufferedDocs=").append(maxBufferedDocs).append("\n");
sb.append("mergedSegmentWarmer=").append(mergedSegmentWarmer).append("\n");
sb.append("codec=").append(codec).append("\n");
sb.append("infoStream=").append(infoStream == null ? "null" : infoStream.getClass().getName()).append("\n");
sb.append("infoStream=").append(infoStream.getClass().getName()).append("\n");
sb.append("mergePolicy=").append(mergePolicy).append("\n");
sb.append("indexerThreadPool=").append(indexerThreadPool).append("\n");
sb.append("readerPooling=").append(readerPooling).append("\n");

View File

@ -84,8 +84,8 @@ public abstract class LogMergePolicy extends MergePolicy {
}
protected boolean verbose() {
IndexWriter w = writer.get();
return w != null && w.verbose();
final IndexWriter w = writer.get();
return w != null && w.infoStream.isEnabled("LMP");
}
/** @see #setNoCFSRatio */
@ -107,10 +107,7 @@ public abstract class LogMergePolicy extends MergePolicy {
protected void message(String message) {
if (verbose()) {
final InfoStream infoStream = writer.get().infoStream;
if (infoStream != null) {
infoStream.message("LMP", message);
}
writer.get().infoStream.message("LMP", message);
}
}

View File

@ -175,7 +175,7 @@ final class SegmentMerger {
}
}
if (mergeState.infoStream != null) {
if (mergeState.infoStream.isEnabled("SM")) {
mergeState.infoStream.message("SM", "merge store matchedCount=" + mergeState.matchedCount + " vs " + mergeState.readers.size());
if (mergeState.matchedCount != mergeState.readers.size()) {
mergeState.infoStream.message("SM", "" + (mergeState.readers.size() - mergeState.matchedCount) + " non-bulk merges");

View File

@ -644,16 +644,13 @@ public class TieredMergePolicy extends MergePolicy {
}
private boolean verbose() {
IndexWriter w = writer.get();
return w != null && w.verbose();
final IndexWriter w = writer.get();
return w != null && w.infoStream.isEnabled("TMP");
}
private void message(String message) {
if (verbose()) {
final InfoStream infoStream = writer.get().infoStream;
if (infoStream != null) {
infoStream.message("TMP", message);
}
writer.get().infoStream.message("TMP", message);
}
}

View File

@ -147,16 +147,13 @@ public class UpgradeIndexMergePolicy extends MergePolicy {
}
private boolean verbose() {
IndexWriter w = writer.get();
return w != null && w.verbose();
final IndexWriter w = writer.get();
return w != null && w.infoStream.isEnabled("UPGMP");
}
private void message(String message) {
if (verbose()) {
final InfoStream infoStream = writer.get().infoStream;
if (infoStream != null) {
infoStream.message("UPGMP", message);
}
writer.get().infoStream.message("UPGMP", message);
}
}

View File

@ -18,30 +18,47 @@ package org.apache.lucene.util;
*/
import java.io.Closeable;
import java.util.concurrent.atomic.AtomicInteger;
/** @lucene.internal */
public abstract class InfoStream implements Closeable {
// Used for printing messages
private static final AtomicInteger MESSAGE_ID = new AtomicInteger();
protected final int messageID = MESSAGE_ID.getAndIncrement();
/** Instance of InfoStream that does no logging at all. */
public static final InfoStream NO_OUTPUT = new NoOutput();
private static final class NoOutput extends InfoStream {
@Override
public void message(String component, String message) {}
@Override
public boolean isEnabled(String component) { return false; }
@Override
public void close() {}
}
/** prints a message */
public abstract void message(String component, String message);
private static InfoStream defaultInfoStream;
/** returns true if messages are enabled and should be posted to {@link #message}. */
public abstract boolean isEnabled(String component);
/** The default infoStream (possibly null) used
* by a newly instantiated classes.
private static InfoStream defaultInfoStream = NO_OUTPUT;
/** The default {@code InfoStream} used by a newly instantiated classes.
* @see #setDefault */
public static InfoStream getDefault() {
public static synchronized InfoStream getDefault() {
return defaultInfoStream;
}
/** Sets the default infoStream (possibly null) used
* by a newly instantiated classes.
* @see #setDefault */
public static void setDefault(InfoStream infoStream) {
/** Sets the default {@code InfoStream} used
* by a newly instantiated classes. It cannot be {@code null},
* to disable logging use {@link #NO_OUTPUT}.
* @see #getDefault */
public static synchronized void setDefault(InfoStream infoStream) {
if (infoStream == null) {
throw new IllegalArgumentException("Cannot set InfoStream default implementation to null. "+
"To disable logging use InfoStream.NO_OUTPUT");
}
defaultInfoStream = infoStream;
}
}

View File

@ -20,15 +20,25 @@ package org.apache.lucene.util;
import java.io.IOException;
import java.io.PrintStream;
import java.util.Date;
import java.util.concurrent.atomic.AtomicInteger;
/**
* @lucene.internal
*/
public class PrintStreamInfoStream extends InfoStream {
// Used for printing messages
private static final AtomicInteger MESSAGE_ID = new AtomicInteger();
protected final int messageID;
private final PrintStream stream;
public PrintStreamInfoStream(PrintStream stream) {
this(stream, MESSAGE_ID.getAndIncrement());
}
public PrintStreamInfoStream(PrintStream stream, int messageID) {
this.stream = stream;
this.messageID = messageID;
}
@Override
@ -36,6 +46,11 @@ public class PrintStreamInfoStream extends InfoStream {
stream.println(component + " " + messageID + " [" + new Date() + "; " + Thread.currentThread().getName() + "]: " + message);
}
@Override
public boolean isEnabled(String component) {
return true;
}
@Override
public void close() throws IOException {
if (!isSystemStream()) {

View File

@ -26,6 +26,11 @@ public class FailOnNonBulkMergesInfoStream extends InfoStream {
@Override
public void close() throws IOException {
}
@Override
public boolean isEnabled(String component) {
return true;
}
@Override
public void message(String component, String message) {

View File

@ -20,7 +20,7 @@ package org.apache.lucene.util;
import java.io.IOException;
/**
* Prints nothing. Just to make sure tests pass w/ and without infostream
* Prints nothing. Just to make sure tests pass w/ and without enabled InfoStream
* without actually making noise.
* @lucene.experimental
*/
@ -31,6 +31,12 @@ public class NullInfoStream extends InfoStream {
assert message != null;
}
@Override
public boolean isEnabled(String component) {
assert component != null;
return true; // to actually enable logging, we just ignore on message()
}
@Override
public void close() throws IOException {
}

View File

@ -24,6 +24,7 @@ import java.util.ArrayList;
import java.util.Iterator;
import java.util.List;
import java.util.Random;
import java.util.concurrent.atomic.AtomicBoolean;
import org.apache.lucene.analysis.*;
import org.apache.lucene.document.Document;
@ -958,18 +959,22 @@ public class TestIndexWriterExceptions extends LuceneTestCase {
// LUCENE-1429
public void testOutOfMemoryErrorCausesCloseToFail() throws Exception {
final List<Throwable> thrown = new ArrayList<Throwable>();
final AtomicBoolean thrown = new AtomicBoolean(false);
final Directory dir = newDirectory();
final IndexWriter writer = new IndexWriter(dir,
newIndexWriterConfig( TEST_VERSION_CURRENT, new MockAnalyzer(random)).setInfoStream(new InfoStream() {
@Override
public void message(String component, final String message) {
if (message.startsWith("now flush at close") && 0 == thrown.size()) {
thrown.add(null);
if (message.startsWith("now flush at close") && thrown.compareAndSet(false, true)) {
throw new OutOfMemoryError("fake OOME at " + message);
}
}
@Override
public boolean isEnabled(String component) {
return true;
}
@Override
public void close() throws IOException {}
}));

View File

@ -74,9 +74,9 @@ public class SolrIndexWriter extends IndexWriter {
File parent = f.getParentFile();
if (parent != null) parent.mkdirs();
FileOutputStream fos = new FileOutputStream(f, true);
return new PrintStreamInfoStream(new TimeLoggingPrintStream(fos, true));
return new PrintStreamInfoStream(new PrintStream(fos, true));
} else {
return null;
return InfoStream.NO_OUTPUT;
}
}
@ -155,23 +155,4 @@ public class SolrIndexWriter extends IndexWriter {
}
}
// Helper class for adding timestamps to infoStream logging
static class TimeLoggingPrintStream extends PrintStream {
private DateFormat dateFormat;
public TimeLoggingPrintStream(OutputStream underlyingOutputStream,
boolean autoFlush) {
super(underlyingOutputStream, autoFlush);
this.dateFormat = DateFormat.getDateTimeInstance();
}
// We might ideally want to override print(String) as well, but
// looking through the code that writes to infoStream, it appears
// that all the classes except CheckIndex just use println.
@Override
public void println(String x) {
print(dateFormat.format(new Date()) + " ");
super.println(x);
}
}
}