You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@lucene.apache.org by us...@apache.org on 2011/12/01 13:52:27 UTC

svn commit: r1209059 - in /lucene/dev/trunk: lucene/src/java/org/apache/lucene/index/ lucene/src/java/org/apache/lucene/util/ lucene/src/test-framework/java/org/apache/lucene/util/ lucene/src/test/org/apache/lucene/index/ solr/core/src/java/org/apache/...

Author: uschindler
Date: Thu Dec  1 12:52:20 2011
New Revision: 1209059

URL: http://svn.apache.org/viewvc?rev=1209059&view=rev
Log:
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

Modified:
    lucene/dev/trunk/lucene/src/java/org/apache/lucene/index/BufferedDeletesStream.java
    lucene/dev/trunk/lucene/src/java/org/apache/lucene/index/ConcurrentMergeScheduler.java
    lucene/dev/trunk/lucene/src/java/org/apache/lucene/index/DirectoryReader.java
    lucene/dev/trunk/lucene/src/java/org/apache/lucene/index/DocumentsWriter.java
    lucene/dev/trunk/lucene/src/java/org/apache/lucene/index/DocumentsWriterFlushControl.java
    lucene/dev/trunk/lucene/src/java/org/apache/lucene/index/DocumentsWriterPerThread.java
    lucene/dev/trunk/lucene/src/java/org/apache/lucene/index/FlushByRamOrCountsPolicy.java
    lucene/dev/trunk/lucene/src/java/org/apache/lucene/index/FlushPolicy.java
    lucene/dev/trunk/lucene/src/java/org/apache/lucene/index/IndexFileDeleter.java
    lucene/dev/trunk/lucene/src/java/org/apache/lucene/index/IndexUpgrader.java
    lucene/dev/trunk/lucene/src/java/org/apache/lucene/index/IndexWriter.java
    lucene/dev/trunk/lucene/src/java/org/apache/lucene/index/IndexWriterConfig.java
    lucene/dev/trunk/lucene/src/java/org/apache/lucene/index/LogMergePolicy.java
    lucene/dev/trunk/lucene/src/java/org/apache/lucene/index/SegmentMerger.java
    lucene/dev/trunk/lucene/src/java/org/apache/lucene/index/TieredMergePolicy.java
    lucene/dev/trunk/lucene/src/java/org/apache/lucene/index/UpgradeIndexMergePolicy.java
    lucene/dev/trunk/lucene/src/java/org/apache/lucene/util/InfoStream.java
    lucene/dev/trunk/lucene/src/java/org/apache/lucene/util/PrintStreamInfoStream.java
    lucene/dev/trunk/lucene/src/test-framework/java/org/apache/lucene/util/FailOnNonBulkMergesInfoStream.java
    lucene/dev/trunk/lucene/src/test-framework/java/org/apache/lucene/util/NullInfoStream.java
    lucene/dev/trunk/lucene/src/test/org/apache/lucene/index/TestIndexWriterExceptions.java
    lucene/dev/trunk/solr/core/src/java/org/apache/solr/update/SolrIndexWriter.java

Modified: lucene/dev/trunk/lucene/src/java/org/apache/lucene/index/BufferedDeletesStream.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/src/java/org/apache/lucene/index/BufferedDeletesStream.java?rev=1209059&r1=1209058&r2=1209059&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/src/java/org/apache/lucene/index/BufferedDeletesStream.java (original)
+++ lucene/dev/trunk/lucene/src/java/org/apache/lucene/index/BufferedDeletesStream.java Thu Dec  1 12:52:20 2011
@@ -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++) {

Modified: lucene/dev/trunk/lucene/src/java/org/apache/lucene/index/ConcurrentMergeScheduler.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/src/java/org/apache/lucene/index/ConcurrentMergeScheduler.java?rev=1209059&r1=1209058&r2=1209059&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/src/java/org/apache/lucene/index/ConcurrentMergeScheduler.java (original)
+++ lucene/dev/trunk/lucene/src/java/org/apache/lucene/index/ConcurrentMergeScheduler.java Thu Dec  1 12:52:20 2011
@@ -218,7 +218,7 @@ public class ConcurrentMergeScheduler ex
    * </pre>
    */
   protected boolean verbose() {
-    return writer != null && writer.verbose();
+    return writer != null && writer.infoStream.isEnabled("CMS");
   }
   
   /**
@@ -226,10 +226,7 @@ public class ConcurrentMergeScheduler ex
    * 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() {

Modified: lucene/dev/trunk/lucene/src/java/org/apache/lucene/index/DirectoryReader.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/src/java/org/apache/lucene/index/DirectoryReader.java?rev=1209059&r1=1209058&r2=1209059&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/src/java/org/apache/lucene/index/DirectoryReader.java (original)
+++ lucene/dev/trunk/lucene/src/java/org/apache/lucene/index/DirectoryReader.java Thu Dec  1 12:52:20 2011
@@ -36,6 +36,7 @@ import org.apache.lucene.store.LockObtai
 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 IndexReade
       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();
 

Modified: lucene/dev/trunk/lucene/src/java/org/apache/lucene/index/DocumentsWriter.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/src/java/org/apache/lucene/index/DocumentsWriter.java?rev=1209059&r1=1209058&r2=1209059&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/src/java/org/apache/lucene/index/DocumentsWriter.java (original)
+++ lucene/dev/trunk/lucene/src/java/org/apache/lucene/index/DocumentsWriter.java Thu Dec  1 12:52:20 2011
@@ -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);

Modified: lucene/dev/trunk/lucene/src/java/org/apache/lucene/index/DocumentsWriterFlushControl.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/src/java/org/apache/lucene/index/DocumentsWriterFlushControl.java?rev=1209059&r1=1209058&r2=1209059&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/src/java/org/apache/lucene/index/DocumentsWriterFlushControl.java (original)
+++ lucene/dev/trunk/lucene/src/java/org/apache/lucene/index/DocumentsWriterFlushControl.java Thu Dec  1 12:52:20 2011
@@ -453,7 +453,7 @@ public final class DocumentsWriterFlushC
   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;

Modified: lucene/dev/trunk/lucene/src/java/org/apache/lucene/index/DocumentsWriterPerThread.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/src/java/org/apache/lucene/index/DocumentsWriterPerThread.java?rev=1209059&r1=1209058&r2=1209059&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/src/java/org/apache/lucene/index/DocumentsWriterPerThread.java (original)
+++ lucene/dev/trunk/lucene/src/java/org/apache/lucene/index/DocumentsWriterPerThread.java Thu Dec  1 12:52:20 2011
@@ -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 + 

Modified: lucene/dev/trunk/lucene/src/java/org/apache/lucene/index/FlushByRamOrCountsPolicy.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/src/java/org/apache/lucene/index/FlushByRamOrCountsPolicy.java?rev=1209059&r1=1209058&r2=1209059&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/src/java/org/apache/lucene/index/FlushByRamOrCountsPolicy.java (original)
+++ lucene/dev/trunk/lucene/src/java/org/apache/lucene/index/FlushByRamOrCountsPolicy.java Thu Dec  1 12:52:20 2011
@@ -63,7 +63,7 @@ public class FlushByRamOrCountsPolicy ex
     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 ex
       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);

Modified: lucene/dev/trunk/lucene/src/java/org/apache/lucene/index/FlushPolicy.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/src/java/org/apache/lucene/index/FlushPolicy.java?rev=1209059&r1=1209058&r2=1209059&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/src/java/org/apache/lucene/index/FlushPolicy.java (original)
+++ lucene/dev/trunk/lucene/src/java/org/apache/lucene/index/FlushPolicy.java Thu Dec  1 12:52:20 2011
@@ -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;
   }
 

Modified: lucene/dev/trunk/lucene/src/java/org/apache/lucene/index/IndexFileDeleter.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/src/java/org/apache/lucene/index/IndexFileDeleter.java?rev=1209059&r1=1209058&r2=1209059&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/src/java/org/apache/lucene/index/IndexFileDeleter.java (original)
+++ lucene/dev/trunk/lucene/src/java/org/apache/lucene/index/IndexFileDeleter.java Thu Dec  1 12:52:20 2011
@@ -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) {

Modified: lucene/dev/trunk/lucene/src/java/org/apache/lucene/index/IndexUpgrader.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/src/java/org/apache/lucene/index/IndexUpgrader.java?rev=1209059&r1=1209058&r2=1209059&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/src/java/org/apache/lucene/index/IndexUpgrader.java (original)
+++ lucene/dev/trunk/lucene/src/java/org/apache/lucene/index/IndexUpgrader.java Thu Dec  1 12:52:20 2011
@@ -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 {

Modified: lucene/dev/trunk/lucene/src/java/org/apache/lucene/index/IndexWriter.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/src/java/org/apache/lucene/index/IndexWriter.java?rev=1209059&r1=1209058&r2=1209059&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/src/java/org/apache/lucene/index/IndexWriter.java (original)
+++ lucene/dev/trunk/lucene/src/java/org/apache/lucene/index/IndexWriter.java Thu Dec  1 12:52:20 2011
@@ -331,9 +331,7 @@ public class IndexWriter implements Clos
 
     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 Clos
         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 Clos
         // 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 Clos
     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 Clos
           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 Clos
         segmentInfos.changed();
       }
 
-      if (infoStream != null) {
+      if (infoStream.isEnabled("IW")) {
         infoStream.message("IW", "init: create=" + create);
         messageState();
       }
@@ -911,9 +907,7 @@ public class IndexWriter implements Clos
 
     } 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 Clos
   }
 
   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 Clos
     }
   }
 
-  /** 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 Clos
   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 Clos
 
       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 Clos
         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 Clos
         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 Clos
         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 Clos
     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 Clos
 
     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 Clos
 
     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 Clos
         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 Clos
         // 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 Clos
         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 Clos
     } 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 Clos
 
       // 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 Clos
       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 Clos
       // 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 Clos
 
       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 Clos
    */
   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 Clos
     // 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 Clos
     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 Clos
         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 Clos
       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 Clos
       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 Clos
         // 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 Clos
     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 Clos
           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 Clos
     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 Clos
   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 Clos
           }
           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 Clos
 
   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 Clos
 
     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 Clos
         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 Clos
     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 Clos
       // 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 Clos
       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 Clos
 
     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 Clos
       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 Clos
     // 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 Clos
 
     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 Clos
       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 Clos
 
   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 Clos
         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 Clos
           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 Clos
     } 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 Clos
 
     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 Clos
     // 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 Clos
       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 Clos
       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 Clos
     }
 
     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 Clos
     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 Clos
     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 Clos
         segUpto++;
       }
 
-      if (infoStream != null) {
+      if (infoStream.isEnabled("IW")) {
         infoStream.message("IW", "merge: total " + totDocCount + " docs");
       }
 
@@ -3542,7 +3496,7 @@ public class IndexWriter implements Clos
       // 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 Clos
         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 Clos
           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 Clos
           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 Clos
         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 Clos
 
     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 Clos
           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 Clos
           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 Clos
   }
 
   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 Clos
   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());
 
     }

Modified: lucene/dev/trunk/lucene/src/java/org/apache/lucene/index/IndexWriterConfig.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/src/java/org/apache/lucene/index/IndexWriterConfig.java?rev=1209059&r1=1209058&r2=1209059&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/src/java/org/apache/lucene/index/IndexWriterConfig.java (original)
+++ lucene/dev/trunk/lucene/src/java/org/apache/lucene/index/IndexWriterConfig.java Thu Dec  1 12:52:20 2011
@@ -695,6 +695,10 @@ public final class IndexWriterConfig imp
    * 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 imp
    * 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 imp
     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");

Modified: lucene/dev/trunk/lucene/src/java/org/apache/lucene/index/LogMergePolicy.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/src/java/org/apache/lucene/index/LogMergePolicy.java?rev=1209059&r1=1209058&r2=1209059&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/src/java/org/apache/lucene/index/LogMergePolicy.java (original)
+++ lucene/dev/trunk/lucene/src/java/org/apache/lucene/index/LogMergePolicy.java Thu Dec  1 12:52:20 2011
@@ -84,8 +84,8 @@ public abstract class LogMergePolicy ext
   }
 
   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 ext
   
   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);
     }
   }
 

Modified: lucene/dev/trunk/lucene/src/java/org/apache/lucene/index/SegmentMerger.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/src/java/org/apache/lucene/index/SegmentMerger.java?rev=1209059&r1=1209058&r2=1209059&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/src/java/org/apache/lucene/index/SegmentMerger.java (original)
+++ lucene/dev/trunk/lucene/src/java/org/apache/lucene/index/SegmentMerger.java Thu Dec  1 12:52:20 2011
@@ -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");

Modified: lucene/dev/trunk/lucene/src/java/org/apache/lucene/index/TieredMergePolicy.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/src/java/org/apache/lucene/index/TieredMergePolicy.java?rev=1209059&r1=1209058&r2=1209059&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/src/java/org/apache/lucene/index/TieredMergePolicy.java (original)
+++ lucene/dev/trunk/lucene/src/java/org/apache/lucene/index/TieredMergePolicy.java Thu Dec  1 12:52:20 2011
@@ -644,16 +644,13 @@ public class TieredMergePolicy extends M
   }
 
   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);
     }
   }
 

Modified: lucene/dev/trunk/lucene/src/java/org/apache/lucene/index/UpgradeIndexMergePolicy.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/src/java/org/apache/lucene/index/UpgradeIndexMergePolicy.java?rev=1209059&r1=1209058&r2=1209059&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/src/java/org/apache/lucene/index/UpgradeIndexMergePolicy.java (original)
+++ lucene/dev/trunk/lucene/src/java/org/apache/lucene/index/UpgradeIndexMergePolicy.java Thu Dec  1 12:52:20 2011
@@ -147,16 +147,13 @@ public class UpgradeIndexMergePolicy ext
   }
   
   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);
     }
   }
   

Modified: lucene/dev/trunk/lucene/src/java/org/apache/lucene/util/InfoStream.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/src/java/org/apache/lucene/util/InfoStream.java?rev=1209059&r1=1209058&r2=1209059&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/src/java/org/apache/lucene/util/InfoStream.java (original)
+++ lucene/dev/trunk/lucene/src/java/org/apache/lucene/util/InfoStream.java Thu Dec  1 12:52:20 2011
@@ -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;
   }
+  
 }

Modified: lucene/dev/trunk/lucene/src/java/org/apache/lucene/util/PrintStreamInfoStream.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/src/java/org/apache/lucene/util/PrintStreamInfoStream.java?rev=1209059&r1=1209058&r2=1209059&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/src/java/org/apache/lucene/util/PrintStreamInfoStream.java (original)
+++ lucene/dev/trunk/lucene/src/java/org/apache/lucene/util/PrintStreamInfoStream.java Thu Dec  1 12:52:20 2011
@@ -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
@@ -37,6 +47,11 @@ public class PrintStreamInfoStream exten
   }
 
   @Override
+  public boolean isEnabled(String component) {
+    return true;
+  }
+
+  @Override
   public void close() throws IOException {
     if (!isSystemStream()) {
       stream.close();

Modified: lucene/dev/trunk/lucene/src/test-framework/java/org/apache/lucene/util/FailOnNonBulkMergesInfoStream.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/src/test-framework/java/org/apache/lucene/util/FailOnNonBulkMergesInfoStream.java?rev=1209059&r1=1209058&r2=1209059&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/src/test-framework/java/org/apache/lucene/util/FailOnNonBulkMergesInfoStream.java (original)
+++ lucene/dev/trunk/lucene/src/test-framework/java/org/apache/lucene/util/FailOnNonBulkMergesInfoStream.java Thu Dec  1 12:52:20 2011
@@ -26,6 +26,11 @@ public class FailOnNonBulkMergesInfoStre
   @Override
   public void close() throws IOException {   
   }
+  
+  @Override
+  public boolean isEnabled(String component) {
+    return true;
+  }
 
   @Override
   public void message(String component, String message) {

Modified: lucene/dev/trunk/lucene/src/test-framework/java/org/apache/lucene/util/NullInfoStream.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/src/test-framework/java/org/apache/lucene/util/NullInfoStream.java?rev=1209059&r1=1209058&r2=1209059&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/src/test-framework/java/org/apache/lucene/util/NullInfoStream.java (original)
+++ lucene/dev/trunk/lucene/src/test-framework/java/org/apache/lucene/util/NullInfoStream.java Thu Dec  1 12:52:20 2011
@@ -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
  */
@@ -32,6 +32,12 @@ public class NullInfoStream extends Info
   }
   
   @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 {   
   }
 }

Modified: lucene/dev/trunk/lucene/src/test/org/apache/lucene/index/TestIndexWriterExceptions.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/src/test/org/apache/lucene/index/TestIndexWriterExceptions.java?rev=1209059&r1=1209058&r2=1209059&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/src/test/org/apache/lucene/index/TestIndexWriterExceptions.java (original)
+++ lucene/dev/trunk/lucene/src/test/org/apache/lucene/index/TestIndexWriterExceptions.java Thu Dec  1 12:52:20 2011
@@ -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,19 +959,23 @@ public class TestIndexWriterExceptions e
   // 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 {}
       }));
 

Modified: lucene/dev/trunk/solr/core/src/java/org/apache/solr/update/SolrIndexWriter.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/solr/core/src/java/org/apache/solr/update/SolrIndexWriter.java?rev=1209059&r1=1209058&r2=1209059&view=diff
==============================================================================
--- lucene/dev/trunk/solr/core/src/java/org/apache/solr/update/SolrIndexWriter.java (original)
+++ lucene/dev/trunk/solr/core/src/java/org/apache/solr/update/SolrIndexWriter.java Thu Dec  1 12:52:20 2011
@@ -74,9 +74,9 @@ public class SolrIndexWriter extends Ind
       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 Ind
     }
     
   }
-  
-  // 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);
-    }
-  }
 }