Index: lucene/src/java/org/apache/lucene/index/BufferedDeletesStream.java =================================================================== --- lucene/src/java/org/apache/lucene/index/BufferedDeletesStream.java (revision 1206721) +++ lucene/src/java/org/apache/lucene/index/BufferedDeletesStream.java (working copy) @@ -88,7 +88,7 @@ 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 @@ 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 @@ 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 @@ 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 @@ } 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 @@ 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 @@ 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 */ protected boolean verbose() { - return writer != null && writer.verbose(); + return writer != null && writer.infoStream.isEnabled("CMS"); } /** @@ -226,10 +226,7 @@ * 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() { Index: lucene/src/java/org/apache/lucene/index/DirectoryReader.java =================================================================== --- lucene/src/java/org/apache/lucene/index/DirectoryReader.java (revision 1206721) +++ lucene/src/java/org/apache/lucene/index/DirectoryReader.java (working copy) @@ -36,6 +36,7 @@ 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 @@ 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(); Index: lucene/src/java/org/apache/lucene/index/DocumentsWriter.java =================================================================== --- lucene/src/java/org/apache/lucene/index/DocumentsWriter.java (revision 1206721) +++ lucene/src/java/org/apache/lucene/index/DocumentsWriter.java (working copy) @@ -216,9 +216,7 @@ } try { - if (infoStream != null) { - infoStream.message("DW", "abort"); - } + infoStream.message("DW", "abort"); final Iterator threadsIterator = perThreadPool.getActivePerThreadsIterator(); while (threadsIterator.hasNext()) { @@ -243,14 +241,14 @@ } 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 @@ 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 @@ 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 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 @@ 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 @@ 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 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 @@ // 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 void finishFullFlush(boolean success) { try { - if (infoStream != null) { + if (infoStream.isEnabled("DW")) { infoStream.message("DW", Thread.currentThread().getName() + " finishFullFlush success=" + success); } assert setFlushingDeleteQueue(null); Index: lucene/src/java/org/apache/lucene/index/DocumentsWriterFlushControl.java =================================================================== --- lucene/src/java/org/apache/lucene/index/DocumentsWriterFlushControl.java (revision 1206721) +++ lucene/src/java/org/apache/lucene/index/DocumentsWriterFlushControl.java (working copy) @@ -453,7 +453,7 @@ private final List fullFlushBuffer = new ArrayList(); 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; Index: lucene/src/java/org/apache/lucene/index/DocumentsWriterPerThread.java =================================================================== --- lucene/src/java/org/apache/lucene/index/DocumentsWriterPerThread.java (revision 1206721) +++ lucene/src/java/org/apache/lucene/index/DocumentsWriterPerThread.java (working copy) @@ -131,9 +131,7 @@ 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 @@ } 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 @@ // 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 @@ // 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 @@ 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 @@ 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 @@ 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 + Index: lucene/src/java/org/apache/lucene/index/FlushByRamOrCountsPolicy.java =================================================================== --- lucene/src/java/org/apache/lucene/index/FlushByRamOrCountsPolicy.java (revision 1206721) +++ lucene/src/java/org/apache/lucene/index/FlushByRamOrCountsPolicy.java (working copy) @@ -63,7 +63,7 @@ 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 @@ 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); Index: lucene/src/java/org/apache/lucene/index/FlushPolicy.java =================================================================== --- lucene/src/java/org/apache/lucene/index/FlushPolicy.java (revision 1206721) +++ lucene/src/java/org/apache/lucene/index/FlushPolicy.java (working copy) @@ -129,10 +129,7 @@ } 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; } Index: lucene/src/java/org/apache/lucene/index/IndexFileDeleter.java =================================================================== --- lucene/src/java/org/apache/lucene/index/IndexFileDeleter.java (revision 1206721) +++ lucene/src/java/org/apache/lucene/index/IndexFileDeleter.java (working copy) @@ -102,7 +102,7 @@ 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 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 @@ // 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 @@ // 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 @@ } 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 @@ } 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 @@ 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 @@ // the now-deleted commits: for(int i=0;i= 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 @@ flush(true, true); - if (infoStream != null) + if (infoStream.isEnabled("IW")) infoStream.message("IW", "forceMergeDeletes: index now " + segString()); MergePolicy.MergeSpecification spec; @@ -1947,9 +1931,7 @@ boolean success = false; - if (infoStream != null ) { - infoStream.message("IW", "rollback"); - } + infoStream.message("IW", "rollback"); try { synchronized(this) { @@ -1957,9 +1939,7 @@ 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 @@ // 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 @@ 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 @@ } 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 @@ // 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 @@ 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 @@ // 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 @@ 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 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 @@ // sanity check assert 0 == mergingSegments.size(); - if (infoStream != null) { - infoStream.message("IW", "waitForMerges done"); - } + infoStream.message("IW", "waitForMerges done"); } /** @@ -2175,7 +2147,7 @@ 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 @@ 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 @@ 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 @@ 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 @@ // generation right away nextGen = bufferedDeletesStream.getNextGen(); } - if (infoStream != null) { + if (infoStream.isEnabled("IW")) { infoStream.message("IW", "publish sets newSegment delGen=" + nextGen); } newSegment.setBufferedDeletesGen(nextGen); @@ -2365,15 +2335,14 @@ noDupDirs(dirs); try { - if (infoStream != null) - infoStream.message("IW", "flush at addIndexes(Directory...)"); + infoStream.message("IW", "flush at addIndexes(Directory...)"); flush(false, true); int docCount = 0; List infos = new ArrayList(); Comparator versionComparator = StringHelper.getVersionComparator(); 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 @@ -2387,7 +2356,7 @@ 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); } @@ -2452,8 +2421,7 @@ 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(); @@ -2645,7 +2613,7 @@ public final void prepareCommit(Map 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()); } @@ -2704,7 +2672,7 @@ } success = true; } finally { - if (!success && infoStream != null) { + if (!success) { infoStream.message("IW", "hit exception during prepareCommit"); } // Done: finish the full flush! @@ -2788,21 +2756,15 @@ private final void commitInternal(Map 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"); } @@ -2814,10 +2776,9 @@ 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); @@ -2831,13 +2792,11 @@ 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 @@ -2877,7 +2836,7 @@ 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()); } @@ -2907,18 +2866,16 @@ // 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()); } } @@ -2931,7 +2888,7 @@ 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) { @@ -2996,7 +2953,7 @@ final List sourceSegments = merge.segments; - if (infoStream != null) + if (infoStream.isEnabled("IW")) infoStream.message("IW", "commitMergeDeletes " + merge.segString(directory)); // Carefully merge deletes that occurred after we @@ -3094,7 +3051,7 @@ 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; @@ -3106,7 +3063,7 @@ // 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; } @@ -3122,7 +3079,7 @@ 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")); } @@ -3133,7 +3090,7 @@ readerPool.drop(merge.info); } - if (infoStream != null) { + if (infoStream.isEnabled("IW")) { infoStream.message("IW", "after commit: " + segString()); } @@ -3160,7 +3117,7 @@ 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); } @@ -3209,7 +3166,7 @@ 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); @@ -3223,8 +3180,7 @@ 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); } @@ -3240,7 +3196,7 @@ } 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"); @@ -3286,7 +3242,7 @@ 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; @@ -3296,7 +3252,7 @@ // 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(", "); @@ -3307,7 +3263,7 @@ 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); @@ -3328,9 +3284,7 @@ success = true; } finally { if (!success) { - if (infoStream != null) { - infoStream.message("IW", "hit exception in mergeInit"); - } + infoStream.message("IW", "hit exception in mergeInit"); mergeFinish(merge); } } @@ -3370,7 +3324,7 @@ } 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) { @@ -3395,7 +3349,7 @@ 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); } @@ -3537,7 +3491,7 @@ 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()); } @@ -3571,7 +3525,7 @@ segUpto++; } - if (infoStream != null) { + if (infoStream.isEnabled("IW")) { infoStream.message("IW", "merge: total " + totDocCount + " docs"); } @@ -3584,7 +3538,7 @@ // 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); } @@ -3604,7 +3558,7 @@ 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())); @@ -3623,9 +3577,7 @@ 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); @@ -3644,9 +3596,7 @@ 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; } @@ -3655,7 +3605,7 @@ 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.)); } @@ -3829,23 +3779,19 @@ 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(toSync); return; } - if (infoStream != null) { + if (infoStream.isEnabled("IW")) { infoStream.message("IW", "startCommit index=" + segString(toSync) + " changeCount=" + changeCount); } @@ -3882,9 +3828,7 @@ pendingCommit = toSync; } - if (infoStream != null) { - infoStream.message("IW", "done all syncs"); - } + infoStream.message("IW", "done all syncs"); assert testPoint("midStartCommitSuccess"); @@ -3897,9 +3841,7 @@ 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(toSync); @@ -3950,7 +3892,7 @@ } private void handleOOM(OutOfMemoryError oom, String location) { - if (infoStream != null) { + if (infoStream.isEnabled("IW")) { infoStream.message("IW", "hit OutOfMemoryError inside " + location); } hitOOM = true; @@ -3975,7 +3917,7 @@ 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()); } Index: lucene/src/java/org/apache/lucene/index/IndexWriterConfig.java =================================================================== --- lucene/src/java/org/apache/lucene/index/IndexWriterConfig.java (revision 1206721) +++ lucene/src/java/org/apache/lucene/index/IndexWriterConfig.java (working copy) @@ -695,6 +695,10 @@ * 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 @@ * 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 @@ 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"); Index: lucene/src/java/org/apache/lucene/index/LogMergePolicy.java =================================================================== --- lucene/src/java/org/apache/lucene/index/LogMergePolicy.java (revision 1206721) +++ lucene/src/java/org/apache/lucene/index/LogMergePolicy.java (working copy) @@ -84,8 +84,8 @@ } 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 @@ 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); } } Index: lucene/src/java/org/apache/lucene/index/SegmentMerger.java =================================================================== --- lucene/src/java/org/apache/lucene/index/SegmentMerger.java (revision 1206721) +++ lucene/src/java/org/apache/lucene/index/SegmentMerger.java (working copy) @@ -175,7 +175,7 @@ } } - 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"); Index: lucene/src/java/org/apache/lucene/index/TieredMergePolicy.java =================================================================== --- lucene/src/java/org/apache/lucene/index/TieredMergePolicy.java (revision 1206721) +++ lucene/src/java/org/apache/lucene/index/TieredMergePolicy.java (working copy) @@ -644,16 +644,13 @@ } 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); } } Index: lucene/src/java/org/apache/lucene/index/UpgradeIndexMergePolicy.java =================================================================== --- lucene/src/java/org/apache/lucene/index/UpgradeIndexMergePolicy.java (revision 1206721) +++ lucene/src/java/org/apache/lucene/index/UpgradeIndexMergePolicy.java (working copy) @@ -147,16 +147,13 @@ } 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); } } Index: lucene/src/java/org/apache/lucene/util/InfoStream.java =================================================================== --- lucene/src/java/org/apache/lucene/util/InfoStream.java (revision 1206721) +++ lucene/src/java/org/apache/lucene/util/InfoStream.java (working copy) @@ -18,30 +18,47 @@ */ 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; } + } Index: lucene/src/java/org/apache/lucene/util/PrintStreamInfoStream.java =================================================================== --- lucene/src/java/org/apache/lucene/util/PrintStreamInfoStream.java (revision 1206721) +++ lucene/src/java/org/apache/lucene/util/PrintStreamInfoStream.java (working copy) @@ -20,15 +20,25 @@ 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 @@ -37,6 +47,11 @@ } @Override + public boolean isEnabled(String component) { + return true; + } + + @Override public void close() throws IOException { if (!isSystemStream()) { stream.close(); Index: lucene/src/test-framework/java/org/apache/lucene/util/FailOnNonBulkMergesInfoStream.java =================================================================== --- lucene/src/test-framework/java/org/apache/lucene/util/FailOnNonBulkMergesInfoStream.java (revision 1206721) +++ lucene/src/test-framework/java/org/apache/lucene/util/FailOnNonBulkMergesInfoStream.java (working copy) @@ -26,6 +26,11 @@ @Override public void close() throws IOException { } + + @Override + public boolean isEnabled(String component) { + return true; + } @Override public void message(String component, String message) { Index: lucene/src/test-framework/java/org/apache/lucene/util/NullInfoStream.java =================================================================== --- lucene/src/test-framework/java/org/apache/lucene/util/NullInfoStream.java (revision 1206721) +++ lucene/src/test-framework/java/org/apache/lucene/util/NullInfoStream.java (working copy) @@ -20,7 +20,7 @@ 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 */ @@ -32,6 +32,12 @@ } @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 { } } Index: lucene/src/test/org/apache/lucene/index/TestIndexWriterExceptions.java =================================================================== --- lucene/src/test/org/apache/lucene/index/TestIndexWriterExceptions.java (revision 1206721) +++ lucene/src/test/org/apache/lucene/index/TestIndexWriterExceptions.java (working copy) @@ -24,6 +24,7 @@ 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; @@ -960,19 +961,23 @@ // LUCENE-1429 public void testOutOfMemoryErrorCausesCloseToFail() throws Exception { - final List thrown = new ArrayList(); + 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 {} })); Index: solr/core/src/java/org/apache/solr/update/SolrIndexWriter.java =================================================================== --- solr/core/src/java/org/apache/solr/update/SolrIndexWriter.java (revision 1206721) +++ solr/core/src/java/org/apache/solr/update/SolrIndexWriter.java (working copy) @@ -74,9 +74,9 @@ 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 @@ } } - - // 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); - } - } }