lucene-commits mailing list archives

Site index · List index
Message view « Date » · « Thread »
Top « Date » · « Thread »
From mikemcc...@apache.org
Subject svn commit: r1213085 - in /lucene/dev/trunk/lucene/src: java/org/apache/lucene/index/ java/org/apache/lucene/util/ test-framework/java/org/apache/lucene/util/
Date Sun, 11 Dec 2011 21:55:45 GMT
Author: mikemccand
Date: Sun Dec 11 21:55:45 2011
New Revision: 1213085

URL: http://svn.apache.org/viewvc?rev=1213085&view=rev
Log:
LUCENE-3598: always call InfoStream.isEnabled before InfoStream.message

Modified:
    lucene/dev/trunk/lucene/src/java/org/apache/lucene/index/ConcurrentMergeScheduler.java
    lucene/dev/trunk/lucene/src/java/org/apache/lucene/index/DocFieldProcessor.java
    lucene/dev/trunk/lucene/src/java/org/apache/lucene/index/DocumentsWriter.java
    lucene/dev/trunk/lucene/src/java/org/apache/lucene/index/DocumentsWriterPerThread.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/IndexWriter.java
    lucene/dev/trunk/lucene/src/java/org/apache/lucene/index/LogMergePolicy.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/test-framework/java/org/apache/lucene/util/LuceneTestCase.java
    lucene/dev/trunk/lucene/src/test-framework/java/org/apache/lucene/util/NullInfoStream.java

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=1213085&r1=1213084&r2=1213085&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
Sun Dec 11 21:55:45 2011
@@ -18,7 +18,6 @@ package org.apache.lucene.index;
  */
 
 import org.apache.lucene.store.Directory;
-import org.apache.lucene.util.InfoStream;
 import org.apache.lucene.util.ThreadInterruptedException;
 import org.apache.lucene.util.CollectionUtil;
 
@@ -336,8 +335,9 @@ public class ConcurrentMergeScheduler ex
       // merges to do first (the easiest ones?), etc.
       MergePolicy.OneMerge merge = writer.getNextMerge();
       if (merge == null) {
-        if (verbose())
+        if (verbose()) {
           message("  no more merges pending; now return");
+        }
         return;
       }
 
@@ -348,7 +348,9 @@ public class ConcurrentMergeScheduler ex
       boolean success = false;
       try {
         synchronized(this) {
-          message("  consider merge " + merge.segString(dir));
+          if (verbose()) {
+            message("  consider merge " + merge.segString(dir));
+          }
 
           // OK to spawn a new merge thread to handle this
           // merge:
@@ -440,8 +442,9 @@ public class ConcurrentMergeScheduler ex
       
       try {
 
-        if (verbose())
+        if (verbose()) {
           message("  merge thread: start");
+        }
 
         while(true) {
           setRunningMerge(merge);
@@ -453,15 +456,17 @@ public class ConcurrentMergeScheduler ex
           if (merge != null) {
             tWriter.mergeInit(merge);
             updateMergeThreads();
-            if (verbose())
+            if (verbose()) {
               message("  merge thread: do another merge " + merge.segString(dir));
+            }
           } else {
             break;
           }
         }
 
-        if (verbose())
+        if (verbose()) {
           message("  merge thread: done");
+        }
 
       } catch (Throwable exc) {
 

Modified: lucene/dev/trunk/lucene/src/java/org/apache/lucene/index/DocFieldProcessor.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/src/java/org/apache/lucene/index/DocFieldProcessor.java?rev=1213085&r1=1213084&r2=1213085&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/src/java/org/apache/lucene/index/DocFieldProcessor.java (original)
+++ lucene/dev/trunk/lucene/src/java/org/apache/lucene/index/DocFieldProcessor.java Sun Dec
11 21:55:45 2011
@@ -277,8 +277,8 @@ final class DocFieldProcessor extends Do
       perField.consumer.processFields(perField.fields, perField.fieldCount);
     }
 
-    if (docState.maxTermPrefix != null && docState.infoStream != null) {
-      docState.infoStream.println("WARNING: document contains at least one immense term (whose
UTF8 encoding is longer than the max length " + DocumentsWriterPerThread.MAX_TERM_LENGTH_UTF8
+ "), all of which were skipped.  Please correct the analyzer to not produce such terms. 
The prefix of the first immense term is: '" + docState.maxTermPrefix + "...'");
+    if (docState.maxTermPrefix != null && docState.infoStream.isEnabled("IW")) {
+      docState.infoStream.message("IW", "WARNING: document contains at least one immense
term (whose UTF8 encoding is longer than the max length " + DocumentsWriterPerThread.MAX_TERM_LENGTH_UTF8
+ "), all of which were skipped.  Please correct the analyzer to not produce such terms. 
The prefix of the first immense term is: '" + docState.maxTermPrefix + "...'");
       docState.maxTermPrefix = null;
     }
   }

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=1213085&r1=1213084&r2=1213085&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 Sun Dec
11 21:55:45 2011
@@ -18,7 +18,6 @@ package org.apache.lucene.index;
  */
 
 import java.io.IOException;
-import java.io.PrintStream;
 import java.util.Collection;
 import java.util.Iterator;
 import java.util.LinkedList;
@@ -216,7 +215,9 @@ final class DocumentsWriter {
     }
 
     try {
-      infoStream.message("DW", "abort");
+      if (infoStream.isEnabled("DW")) {
+        infoStream.message("DW", "abort");
+      }
 
       final Iterator<ThreadState> threadsIterator = perThreadPool.getActivePerThreadsIterator();
       while (threadsIterator.hasNext()) {
@@ -298,8 +299,10 @@ final class DocumentsWriter {
           maybeMerge |= doFlush(flushingDWPT);
         }
   
-        if (infoStream.isEnabled("DW") && flushControl.anyStalledThreads()) {
-          infoStream.message("DW", "WARNING DocumentsWriter has stalled threads; waiting");
+        if (infoStream.isEnabled("DW")) {
+          if (flushControl.anyStalledThreads()) {
+            infoStream.message("DW", "WARNING DocumentsWriter has stalled threads; waiting");
+          }
         }
         
         flushControl.waitIfStalled(); // block if stalled

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=1213085&r1=1213084&r2=1213085&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
Sun Dec 11 21:55:45 2011
@@ -21,7 +21,6 @@ import static org.apache.lucene.util.Byt
 import static org.apache.lucene.util.ByteBlockPool.BYTE_BLOCK_SIZE;
 
 import java.io.IOException;
-import java.io.PrintStream;
 import java.text.NumberFormat;
 
 import org.apache.lucene.analysis.Analyzer;
@@ -88,14 +87,15 @@ public class DocumentsWriterPerThread {
   static class DocState {
     final DocumentsWriterPerThread docWriter;
     Analyzer analyzer;
-    PrintStream infoStream;
+    InfoStream infoStream;
     SimilarityProvider similarityProvider;
     int docID;
     Iterable<? extends IndexableField> doc;
     String maxTermPrefix;
 
-    DocState(DocumentsWriterPerThread docWriter) {
+    DocState(DocumentsWriterPerThread docWriter, InfoStream infoStream) {
       this.docWriter = docWriter;
+      this.infoStream = infoStream;
     }
 
     // Only called by asserts
@@ -131,7 +131,9 @@ public class DocumentsWriterPerThread {
   void abort() throws IOException {
     hasAborted = aborting = true;
     try {
-      infoStream.message("DWPT", "now abort");
+      if (infoStream.isEnabled("DWPT")) {
+        infoStream.message("DWPT", "now abort");
+      }
       try {
         consumer.abort();
       } catch (Throwable t) {
@@ -144,7 +146,9 @@ public class DocumentsWriterPerThread {
 
     } finally {
       aborting = false;
-      infoStream.message("DWPT", "done abort");
+      if (infoStream.isEnabled("DWPT")) {
+        infoStream.message("DWPT", "done abort");
+      }
     }
   }
   private final static boolean INFO_VERBOSE = false;
@@ -181,7 +185,7 @@ public class DocumentsWriterPerThread {
     this.writer = parent.indexWriter;
     this.infoStream = parent.infoStream;
     this.codec = parent.codec;
-    this.docState = new DocState(this);
+    this.docState = new DocState(this, infoStream);
     this.docState.similarityProvider = parent.indexWriter.getConfig()
         .getSimilarityProvider();
     bytesUsed = Counter.newCounter();

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=1213085&r1=1213084&r2=1213085&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 Sun Dec 11 21:55:45
2011
@@ -20,7 +20,6 @@ import java.util.Iterator;
 
 import org.apache.lucene.index.DocumentsWriterPerThreadPool.ThreadState;
 import org.apache.lucene.store.Directory;
-import org.apache.lucene.util.InfoStream;
 import org.apache.lucene.util.SetOnce;
 
 /**
@@ -129,7 +128,9 @@ public abstract class FlushPolicy {
   }
   
   private boolean assertMessage(String s) {
-    writer.get().infoStream.message("FP", s);
+    if (writer.get().infoStream.isEnabled("FP")) {
+      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=1213085&r1=1213084&r2=1213085&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 Sun Dec
11 21:55:45 2011
@@ -501,8 +501,10 @@ final class IndexFileDeleter {
   void incRef(String fileName) throws IOException {
     assert locked();
     RefCount rc = getRefCount(fileName);
-    if (infoStream.isEnabled("IFD") && VERBOSE_REF_COUNTS) {
-      infoStream.message("IFD", "  IncRef \"" + fileName + "\": pre-incr count is " + rc.count);
+    if (infoStream.isEnabled("IFD")) {
+      if (VERBOSE_REF_COUNTS) {
+        infoStream.message("IFD", "  IncRef \"" + fileName + "\": pre-incr count is " + rc.count);
+      }
     }
     rc.IncRef();
   }
@@ -517,8 +519,10 @@ final class IndexFileDeleter {
   void decRef(String fileName) throws IOException {
     assert locked();
     RefCount rc = getRefCount(fileName);
-    if (infoStream.isEnabled("IFD") && VERBOSE_REF_COUNTS) {
-      infoStream.message("IFD", "  DecRef \"" + fileName + "\": pre-decr count is " + rc.count);
+    if (infoStream.isEnabled("IFD")) {
+      if (VERBOSE_REF_COUNTS) {
+        infoStream.message("IFD", "  DecRef \"" + fileName + "\": pre-decr count is " + rc.count);
+      }
     }
     if (0 == rc.DecRef()) {
       // This file is no longer referenced by any past

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=1213085&r1=1213084&r2=1213085&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 Sun Dec 11 21:55:45
2011
@@ -332,7 +332,9 @@ public class IndexWriter implements Clos
 
     final long tStart = System.currentTimeMillis();
 
-    infoStream.message("IW", "flush at getReader");
+    if (infoStream.isEnabled("IW")) {
+      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:
@@ -372,8 +374,10 @@ public class IndexWriter implements Clos
         // never reached but javac disagrees:
         return null;
       } finally {
-        if (!success && infoStream.isEnabled("IW")) {
-          infoStream.message("IW", "hit exception during NRT reader");
+        if (!success) {
+          if (infoStream.isEnabled("IW")) {
+            infoStream.message("IW", "hit exception during NRT reader");
+          }
         }
         // Done: finish the full flush!
         docWriter.finishFullFlush(success);
@@ -873,8 +877,9 @@ public class IndexWriter implements Clos
           segmentInfos.replace(oldInfos);
           changeCount++;
           segmentInfos.changed();
-          if (infoStream.isEnabled("IW"))
+          if (infoStream.isEnabled("IW")) {
             infoStream.message("IW", "init: loaded commit \"" + commit.getSegmentsFileName()
+ "\"");
+          }
         }
       }
 
@@ -910,7 +915,9 @@ public class IndexWriter implements Clos
 
     } finally {
       if (!success) {
-        infoStream.message("IW", "init: hit exception on init; releasing write lock");
+        if (infoStream.isEnabled("IW")) {
+          infoStream.message("IW", "init: hit exception on init; releasing write lock");
+        }
         try {
           writeLock.release();
         } catch (Throwable t) {
@@ -1078,14 +1085,17 @@ public class IndexWriter implements Clos
 
       mergeScheduler.close();
 
-      infoStream.message("IW", "now call final commit()");
+      if (infoStream.isEnabled("IW")) {
+        infoStream.message("IW", "now call final commit()");
+      }
 
       if (!hitOOM) {
         commitInternal(null);
       }
 
-      if (infoStream.isEnabled("IW"))
+      if (infoStream.isEnabled("IW")) {
         infoStream.message("IW", "at close: " + segString());
+      }
       // used by assert below
       final DocumentsWriter oldWriter = docWriter;
       synchronized(this) {
@@ -1109,7 +1119,9 @@ public class IndexWriter implements Clos
         closing = false;
         notifyAll();
         if (!closed) {
-          infoStream.message("IW", "hit exception while closing");
+          if (infoStream.isEnabled("IW")) {
+            infoStream.message("IW", "hit exception while closing");
+          }
         }
       }
     }
@@ -1338,8 +1350,10 @@ public class IndexWriter implements Clos
         anySegmentFlushed = docWriter.updateDocuments(docs, analyzer, delTerm);
         success = true;
       } finally {
-        if (!success && infoStream.isEnabled("IW")) {
-          infoStream.message("IW", "hit exception updating document");
+        if (!success) {
+          if (infoStream.isEnabled("IW")) {
+            infoStream.message("IW", "hit exception updating document");
+          }
         }
       }
       if (anySegmentFlushed) {
@@ -1485,8 +1499,10 @@ public class IndexWriter implements Clos
         anySegmentFlushed = docWriter.updateDocument(doc, analyzer, term);
         success = true;
       } finally {
-        if (!success && infoStream.isEnabled("IW")) {
-          infoStream.message("IW", "hit exception updating document");
+        if (!success) {
+          if (infoStream.isEnabled("IW")) {
+            infoStream.message("IW", "hit exception updating document");
+          }
         }
       }
 
@@ -1744,8 +1760,9 @@ public class IndexWriter implements Clos
 
     flush(true, true);
 
-    if (infoStream.isEnabled("IW"))
+    if (infoStream.isEnabled("IW")) {
       infoStream.message("IW", "forceMergeDeletes: index now " + segString());
+    }
 
     MergePolicy.MergeSpecification spec;
 
@@ -1938,7 +1955,9 @@ public class IndexWriter implements Clos
 
     boolean success = false;
 
-    infoStream.message("IW", "rollback");
+    if (infoStream.isEnabled("IW")) {
+      infoStream.message("IW", "rollback");
+    }
 
     try {
       synchronized(this) {
@@ -1946,7 +1965,9 @@ public class IndexWriter implements Clos
         stopMerges = true;
       }
 
-      infoStream.message("IW", "rollback: done finish merges");
+      if (infoStream.isEnabled("IW")) {
+        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
@@ -1998,7 +2019,9 @@ public class IndexWriter implements Clos
         if (!success) {
           closing = false;
           notifyAll();
-          infoStream.message("IW", "hit exception during rollback");
+          if (infoStream.isEnabled("IW")) {
+            infoStream.message("IW", "hit exception during rollback");
+          }
         }
       }
     }
@@ -2050,8 +2073,10 @@ public class IndexWriter implements Clos
     } catch (OutOfMemoryError oom) {
       handleOOM(oom, "deleteAll");
     } finally {
-      if (!success && infoStream.isEnabled("IW")) {
-        infoStream.message("IW", "hit exception during deleteAll");
+      if (!success) {
+        if (infoStream.isEnabled("IW")) {
+          infoStream.message("IW", "hit exception during deleteAll");
+        }
       }
     }
   }
@@ -2063,16 +2088,18 @@ public class IndexWriter implements Clos
 
       // Abort all pending & running merges:
       for (final MergePolicy.OneMerge merge : pendingMerges) {
-        if (infoStream.isEnabled("IW"))
+        if (infoStream.isEnabled("IW")) {
           infoStream.message("IW", "now abort pending merge " + merge.segString(directory));
+        }
         merge.abort();
         mergeFinish(merge);
       }
       pendingMerges.clear();
 
       for (final MergePolicy.OneMerge merge : runningMerges) {
-        if (infoStream.isEnabled("IW"))
+        if (infoStream.isEnabled("IW")) {
           infoStream.message("IW", "now abort running merge " + merge.segString(directory));
+        }
         merge.abort();
       }
 
@@ -2082,8 +2109,9 @@ public class IndexWriter implements Clos
       // because the merge threads periodically check if
       // they are aborted.
       while(runningMerges.size() > 0) {
-        if (infoStream.isEnabled("IW"))
+        if (infoStream.isEnabled("IW")) {
           infoStream.message("IW", "now wait for " + runningMerges.size() + " running merge
to abort");
+        }
         doWait();
       }
 
@@ -2092,7 +2120,9 @@ public class IndexWriter implements Clos
 
       assert 0 == mergingSegments.size();
 
-      infoStream.message("IW", "all running merges have aborted");
+      if (infoStream.isEnabled("IW")) {
+        infoStream.message("IW", "all running merges have aborted");
+      }
 
     } else {
       // waitForMerges() will ensure any running addIndexes finishes.
@@ -2112,7 +2142,9 @@ public class IndexWriter implements Clos
    */
   public synchronized void waitForMerges() {
     ensureOpen(false);
-    infoStream.message("IW", "waitForMerges");
+    if (infoStream.isEnabled("IW")) {
+      infoStream.message("IW", "waitForMerges");
+    }
     while(pendingMerges.size() > 0 || runningMerges.size() > 0) {
       doWait();
     }
@@ -2120,7 +2152,9 @@ public class IndexWriter implements Clos
     // sanity check
     assert 0 == mergingSegments.size();
 
-    infoStream.message("IW", "waitForMerges done");
+    if (infoStream.isEnabled("IW")) {
+      infoStream.message("IW", "waitForMerges done");
+    }
   }
 
   /**
@@ -2247,7 +2281,9 @@ public class IndexWriter implements Clos
       FrozenBufferedDeletes packet, FrozenBufferedDeletes globalPacket) throws IOException
{
     // Lock order IW -> BDS
     synchronized (bufferedDeletesStream) {
-      infoStream.message("IW", "publishFlushedSegment");  
+      if (infoStream.isEnabled("IW")) {
+        infoStream.message("IW", "publishFlushedSegment");
+      }
       
       if (globalPacket != null && globalPacket.any()) {
         bufferedDeletesStream.push(globalPacket);
@@ -2341,7 +2377,10 @@ public class IndexWriter implements Clos
     noDupDirs(dirs);
 
     try {
-      infoStream.message("IW", "flush at addIndexes(Directory...)");
+      if (infoStream.isEnabled("IW")) {
+        infoStream.message("IW", "flush at addIndexes(Directory...)");
+      }
+
       flush(false, true);
 
       List<SegmentInfo> infos = new ArrayList<SegmentInfo>();
@@ -2410,7 +2449,9 @@ public class IndexWriter implements Clos
     int numDocs = 0;
 
     try {
-      infoStream.message("IW", "flush at addIndexes(IndexReader...)");
+      if (infoStream.isEnabled("IW")) {
+        infoStream.message("IW", "flush at addIndexes(IndexReader...)");
+      }
       flush(false, true);
 
       String mergedName = newSegmentName();
@@ -2638,7 +2679,9 @@ public class IndexWriter implements Clos
           success = true;
         } finally {
           if (!success) {
-            infoStream.message("IW", "hit exception during prepareCommit");
+            if (infoStream.isEnabled("IW")) {
+              infoStream.message("IW", "hit exception during prepareCommit");
+            }
           }
           // Done: finish the full flush!
           docWriter.finishFullFlush(flushSuccess);
@@ -2722,16 +2765,24 @@ public class IndexWriter implements Clos
 
   private final void commitInternal(Map<String,String> commitUserData) throws CorruptIndexException,
IOException {
 
-    infoStream.message("IW", "commit: start");
+    if (infoStream.isEnabled("IW")) {
+      infoStream.message("IW", "commit: start");
+    }
 
     synchronized(commitLock) {
-      infoStream.message("IW", "commit: enter lock");
+      if (infoStream.isEnabled("IW")) {
+        infoStream.message("IW", "commit: enter lock");
+      }
 
       if (pendingCommit == null) {
-        infoStream.message("IW", "commit: now prepare");
+        if (infoStream.isEnabled("IW")) {
+          infoStream.message("IW", "commit: now prepare");
+        }
         prepareCommit(commitUserData);
       } else {
-        infoStream.message("IW", "commit: already prepared");
+        if (infoStream.isEnabled("IW")) {
+          infoStream.message("IW", "commit: already prepared");
+        }
       }
 
       finishCommit();
@@ -2742,10 +2793,13 @@ public class IndexWriter implements Clos
 
     if (pendingCommit != null) {
       try {
-        infoStream.message("IW", "commit: pendingCommit != null");
+        if (infoStream.isEnabled("IW")) {
+          infoStream.message("IW", "commit: pendingCommit != null");
+        }
         pendingCommit.finishCommit(directory, codec);
-        if (infoStream.isEnabled("IW"))
+        if (infoStream.isEnabled("IW")) {
           infoStream.message("IW", "commit: wrote segments file \"" + pendingCommit.getCurrentSegmentFileName()
+ "\"");
+        }
         lastCommitChangeCount = pendingCommitChangeCount;
         segmentInfos.updateGeneration(pendingCommit);
         segmentInfos.setUserData(pendingCommit.getUserData());
@@ -2760,10 +2814,14 @@ public class IndexWriter implements Clos
       }
 
     } else {
-      infoStream.message("IW", "commit: pendingCommit == null; skip");
+      if (infoStream.isEnabled("IW")) {
+        infoStream.message("IW", "commit: pendingCommit == null; skip");
+      }
     }
 
-    infoStream.message("IW", "commit: done");
+    if (infoStream.isEnabled("IW")) {
+      infoStream.message("IW", "commit: done");
+    }
   }
 
   // Ensures only one flush() is actually flushing segments
@@ -2833,14 +2891,19 @@ public class IndexWriter implements Clos
       // never hit
       return false;
     } finally {
-      if (!success)
-        infoStream.message("IW", "hit exception during flush");
+      if (!success) {
+        if (infoStream.isEnabled("IW")) {
+          infoStream.message("IW", "hit exception during flush");
+        }
+      }
     }
   }
   
   final synchronized void maybeApplyDeletes(boolean applyAllDeletes) throws IOException {
     if (applyAllDeletes) {
-      infoStream.message("IW", "apply all deletes during flush");
+      if (infoStream.isEnabled("IW")) {
+        infoStream.message("IW", "apply all deletes during flush");
+      }
       applyAllDeletes();
     } else if (infoStream.isEnabled("IW")) {
       infoStream.message("IW", "don't apply deletes now delTermCount=" + bufferedDeletesStream.numTerms()
+ " bytesUsed=" + bufferedDeletesStream.bytesUsed());
@@ -2920,8 +2983,9 @@ public class IndexWriter implements Clos
 
     final List<SegmentInfo> sourceSegments = merge.segments;
 
-    if (infoStream.isEnabled("IW"))
+    if (infoStream.isEnabled("IW")) {
       infoStream.message("IW", "commitMergeDeletes " + merge.segString(directory));
+    }
 
     // Carefully merge deletes that occurred after we
     // started merging:
@@ -3018,8 +3082,9 @@ public class IndexWriter implements Clos
       throw new IllegalStateException("this writer hit an OutOfMemoryError; cannot complete
merge");
     }
 
-    if (infoStream.isEnabled("IW"))
+    if (infoStream.isEnabled("IW")) {
       infoStream.message("IW", "commitMerge: " + merge.segString(directory) + " index=" +
segString());
+    }
 
     assert merge.registerDone;
 
@@ -3030,8 +3095,9 @@ public class IndexWriter implements Clos
     // file that current segments does not reference), we
     // abort this merge
     if (merge.isAborted()) {
-      if (infoStream.isEnabled("IW"))
+      if (infoStream.isEnabled("IW")) {
         infoStream.message("IW", "commitMerge: skipping merge " + merge.segString(directory)
+ ": it was aborted");
+      }
       return false;
     }
 
@@ -3046,8 +3112,10 @@ public class IndexWriter implements Clos
 
     final boolean allDeleted = mergedReader.numDocs() == 0;
 
-    if (allDeleted && infoStream.isEnabled("IW")) {
-      infoStream.message("IW", "merged segment " + merge.info + " is 100% deleted" +  (keepFullyDeletedSegments
? "" : "; skipping insert"));
+    if (allDeleted) {
+      if (infoStream.isEnabled("IW")) {
+        infoStream.message("IW", "merged segment " + merge.info + " is 100% deleted" +  (keepFullyDeletedSegments
? "" : "; skipping insert"));
+      }
     }
 
     final boolean dropSegment = allDeleted && !keepFullyDeletedSegments;
@@ -3133,8 +3201,9 @@ public class IndexWriter implements Clos
         try {
           mergeInit(merge);
 
-          if (infoStream.isEnabled("IW"))
+          if (infoStream.isEnabled("IW")) {
             infoStream.message("IW", "now merge\n  merge=" + merge.segString(directory) +
"\n  index=" + segString());
+          }
 
           mergeMiddle(merge);
           mergeSuccess(merge);
@@ -3147,9 +3216,12 @@ public class IndexWriter implements Clos
           mergeFinish(merge);
 
           if (!success) {
-            infoStream.message("IW", "hit exception during merge");
-            if (merge.info != null && !segmentInfos.contains(merge.info))
+            if (infoStream.isEnabled("IW")) {
+              infoStream.message("IW", "hit exception during merge");
+            }
+            if (merge.info != null && !segmentInfos.contains(merge.info)) {
               deleter.refresh(merge.info.name);
+            }
           }
 
           // This merge (and, generally, any change to the
@@ -3163,8 +3235,10 @@ public class IndexWriter implements Clos
     } catch (OutOfMemoryError oom) {
       handleOOM(oom, "merge");
     }
-    if (merge.info != null && infoStream.isEnabled("IW")) {
-      infoStream.message("IW", "merge time " + (System.currentTimeMillis()-t0) + " msec for
" + merge.info.docCount + " docs");
+    if (merge.info != null) {
+      if (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");
   }
@@ -3209,8 +3283,9 @@ public class IndexWriter implements Clos
 
     pendingMerges.add(merge);
 
-    if (infoStream.isEnabled("IW"))
+    if (infoStream.isEnabled("IW")) {
       infoStream.message("IW", "add merge to pendingMerges: " + merge.segString(directory)
+ " [total " + pendingMerges.size() + " pending]");
+    }
 
     merge.mergeGen = mergeGen;
     merge.isExternal = isExternal;
@@ -3227,7 +3302,9 @@ public class IndexWriter implements Clos
       builder.append("]");
       // don't call mergingSegments.toString() could lead to ConcurrentModException
       // since merge updates the segments FieldInfos
-      infoStream.message("IW", builder.toString());  
+      if (infoStream.isEnabled("IW")) {
+        infoStream.message("IW", builder.toString());  
+      }
     }
     for(SegmentInfo info : merge.segments) {
       if (infoStream.isEnabled("IW")) {
@@ -3251,7 +3328,9 @@ public class IndexWriter implements Clos
       success = true;
     } finally {
       if (!success) {
-        infoStream.message("IW", "hit exception in mergeInit");
+        if (infoStream.isEnabled("IW")) {
+          infoStream.message("IW", "hit exception in mergeInit");
+        }
         mergeFinish(merge);
       }
     }
@@ -3544,7 +3623,9 @@ public class IndexWriter implements Clos
           handleMergeException(t, merge);
         } finally {
           if (!success) {
-            infoStream.message("IW", "hit exception creating compound file during merge");
+            if (infoStream.isEnabled("IW")) {
+              infoStream.message("IW", "hit exception creating compound file during merge");
+            }
 
             synchronized(this) {
               deleter.deleteFile(compoundFileName);
@@ -3563,7 +3644,9 @@ public class IndexWriter implements Clos
           deleter.deleteNewFiles(merge.info.files());
 
           if (merge.isAborted()) {
-            infoStream.message("IW", "abort merge after building CFS");
+            if (infoStream.isEnabled("IW")) {
+              infoStream.message("IW", "abort merge after building CFS");
+            }
             deleter.deleteFile(compoundFileName);
             return 0;
           }
@@ -3746,14 +3829,18 @@ public class IndexWriter implements Clos
 
     try {
 
-      infoStream.message("IW", "startCommit(): start");
+      if (infoStream.isEnabled("IW")) {
+        infoStream.message("IW", "startCommit(): start");
+      }
 
       synchronized(this) {
 
         assert lastCommitChangeCount <= changeCount;
 
         if (pendingCommitChangeCount == lastCommitChangeCount) {
-          infoStream.message("IW", "  skip startCommit(): no changes pending");
+          if (infoStream.isEnabled("IW")) {
+            infoStream.message("IW", "  skip startCommit(): no changes pending");
+          }
           deleter.decRef(filesToCommit);
           filesToCommit = null;
           return;
@@ -3796,7 +3883,9 @@ public class IndexWriter implements Clos
           pendingCommit = toSync;
         }
 
-        infoStream.message("IW", "done all syncs");
+        if (infoStream.isEnabled("IW")) {
+          infoStream.message("IW", "done all syncs");
+        }
 
         assert testPoint("midStartCommitSuccess");
 
@@ -3809,7 +3898,9 @@ public class IndexWriter implements Clos
           segmentInfos.updateGeneration(toSync);
 
           if (!pendingCommitSet) {
-            infoStream.message("IW", "hit exception committing segments file");
+            if (infoStream.isEnabled("IW")) {
+              infoStream.message("IW", "hit exception committing segments file");
+            }
 
             // Hit exception
             deleter.decRef(filesToCommit);

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=1213085&r1=1213084&r2=1213085&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 Sun Dec 11
21:55:45 2011
@@ -23,7 +23,6 @@ import java.util.Collection;
 import java.util.List;
 import java.util.Map;
 
-import org.apache.lucene.util.InfoStream;
 
 /** <p>This class implements a {@link MergePolicy} that tries
  *  to merge segments into levels of exponentially
@@ -424,8 +423,9 @@ public abstract class LogMergePolicy ext
     final List<SegmentInfo> segments = segmentInfos.asList();
     final int numSegments = segments.size();
 
-    if (verbose())
+    if (verbose()) {
       message("findForcedDeleteMerges: " + numSegments + " segments");
+    }
 
     MergeSpecification spec = new MergeSpecification();
     int firstSegmentWithDeletions = -1;
@@ -435,15 +435,17 @@ public abstract class LogMergePolicy ext
       final SegmentInfo info = segmentInfos.info(i);
       int delCount = w.numDeletedDocs(info);
       if (delCount > 0) {
-        if (verbose())
+        if (verbose()) {
           message("  segment " + info.name + " has deletions");
+        }
         if (firstSegmentWithDeletions == -1)
           firstSegmentWithDeletions = i;
         else if (i - firstSegmentWithDeletions == mergeFactor) {
           // We've seen mergeFactor segments in a row with
           // deletions, so force a merge now:
-          if (verbose())
+          if (verbose()) {
             message("  add merge " + firstSegmentWithDeletions + " to " + (i-1) + " inclusive");
+          }
           spec.add(new OneMerge(segments.subList(firstSegmentWithDeletions, i)));
           firstSegmentWithDeletions = i;
         }
@@ -451,16 +453,18 @@ public abstract class LogMergePolicy ext
         // End of a sequence of segments with deletions, so,
         // merge those past segments even if it's fewer than
         // mergeFactor segments
-        if (verbose())
+        if (verbose()) {
           message("  add merge " + firstSegmentWithDeletions + " to " + (i-1) + " inclusive");
+        }
         spec.add(new OneMerge(segments.subList(firstSegmentWithDeletions, i)));
         firstSegmentWithDeletions = -1;
       }
     }
 
     if (firstSegmentWithDeletions != -1) {
-      if (verbose())
+      if (verbose()) {
         message("  add merge " + firstSegmentWithDeletions + " to " + (numSegments-1) + "
inclusive");
+      }
       spec.add(new OneMerge(segments.subList(firstSegmentWithDeletions, numSegments)));
     }
 
@@ -500,8 +504,9 @@ public abstract class LogMergePolicy ext
   public MergeSpecification findMerges(SegmentInfos infos) throws IOException {
 
     final int numSegments = infos.size();
-    if (verbose())
+    if (verbose()) {
       message("findMerges: " + numSegments + " segments");
+    }
 
     // Compute levels, which is just log (base mergeFactor)
     // of the size of each segment
@@ -582,8 +587,9 @@ public abstract class LogMergePolicy ext
         }
         upto--;
       }
-      if (verbose())
+      if (verbose()) {
         message("  level " + levelBottom + " to " + maxLevel + ": " + (1+upto-start) + "
segments");
+      }
 
       // Finally, record all merges that are viable at this level:
       int end = start + mergeFactor;

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=1213085&r1=1213084&r2=1213085&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 Sun Dec
11 21:55:45 2011
@@ -26,7 +26,6 @@ import java.util.Comparator;
 import java.util.List;
 import java.util.ArrayList;
 
-import org.apache.lucene.util.InfoStream;
 
 /**
  *  Merges segments of approximately equal size, subject to
@@ -346,7 +345,9 @@ public class TieredMergePolicy extends M
 
       final boolean maxMergeIsRunning = mergingBytes >= maxMergedSegmentBytes;
 
-      message("  allowedSegmentCount=" + allowedSegCountInt + " vs count=" + infosSorted.size()
+ " (eligible count=" + eligible.size() + ") tooBigCount=" + tooBigCount);
+      if (verbose()) {
+        message("  allowedSegmentCount=" + allowedSegCountInt + " vs count=" + infosSorted.size()
+ " (eligible count=" + eligible.size() + ") tooBigCount=" + tooBigCount);
+      }
 
       if (eligible.size() == 0) {
         return spec;
@@ -386,7 +387,9 @@ public class TieredMergePolicy extends M
           }
 
           final MergeScore score = score(candidate, hitTooLarge, mergingBytes);
-          message("  maybe=" + writer.get().segString(candidate) + " score=" + score.getScore()
+ " " + score.getExplanation() + " tooLarge=" + hitTooLarge + " size=" + String.format("%.3f
MB", totAfterMergeBytes/1024./1024.));
+          if (verbose()) {
+            message("  maybe=" + writer.get().segString(candidate) + " score=" + score.getScore()
+ " " + score.getExplanation() + " tooLarge=" + hitTooLarge + " size=" + String.format("%.3f
MB", totAfterMergeBytes/1024./1024.));
+          }
 
           // If we are already running a max sized merge
           // (maxMergeIsRunning), don't allow another max
@@ -649,9 +652,7 @@ public class TieredMergePolicy extends M
   }
 
   private void message(String message) {
-    if (verbose()) {
-      writer.get().infoStream.message("TMP", message);
-    }
+    writer.get().infoStream.message("TMP", message);
   }
 
   @Override

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=1213085&r1=1213084&r2=1213085&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
Sun Dec 11 21:55:45 2011
@@ -18,7 +18,6 @@ package org.apache.lucene.index;
  */
 
 import org.apache.lucene.util.Constants;
-import org.apache.lucene.util.InfoStream;
 
 import java.io.IOException;
 import java.util.ArrayList;
@@ -90,7 +89,9 @@ public class UpgradeIndexMergePolicy ext
       }
     }
     
-    if (verbose()) message("findForcedMerges: segmentsToUpgrade=" + oldSegments);
+    if (verbose()) {
+      message("findForcedMerges: segmentsToUpgrade=" + oldSegments);
+    }
       
     if (oldSegments.isEmpty())
       return null;
@@ -107,9 +108,10 @@ public class UpgradeIndexMergePolicy ext
     }
 
     if (!oldSegments.isEmpty()) {
-      if (verbose())
+      if (verbose()) {
         message("findForcedMerges: " +  base.getClass().getSimpleName() +
         " does not want to merge all old segments, merge remaining ones into new segment:
" + oldSegments);
+      }
       final List<SegmentInfo> newInfos = new ArrayList<SegmentInfo>();
       for (final SegmentInfo si : segmentInfos) {
         if (oldSegments.containsKey(si)) {
@@ -152,9 +154,6 @@ public class UpgradeIndexMergePolicy ext
   }
 
   private void message(String message) {
-    if (verbose()) {
-      writer.get().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=1213085&r1=1213084&r2=1213085&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 Sun Dec 11 21:55:45
2011
@@ -26,10 +26,14 @@ public abstract class InfoStream impleme
   public static final InfoStream NO_OUTPUT = new NoOutput();
   private static final class NoOutput extends InfoStream {
     @Override
-    public void message(String component, String message) {}
+    public void message(String component, String message) {
+      assert false: "message() should not be called when isEnabled returns false";
+    }
     
     @Override
-    public boolean isEnabled(String component) { return false; }
+    public boolean isEnabled(String component) {
+      return false;
+    }
 
     @Override
     public void close() {}

Modified: lucene/dev/trunk/lucene/src/test-framework/java/org/apache/lucene/util/LuceneTestCase.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/src/test-framework/java/org/apache/lucene/util/LuceneTestCase.java?rev=1213085&r1=1213084&r2=1213085&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/src/test-framework/java/org/apache/lucene/util/LuceneTestCase.java
(original)
+++ lucene/dev/trunk/lucene/src/test-framework/java/org/apache/lucene/util/LuceneTestCase.java
Sun Dec 11 21:55:45 2011
@@ -32,7 +32,6 @@ import java.util.Map.Entry;
 import java.util.concurrent.ExecutorService;
 import java.util.concurrent.Executors;
 import java.util.concurrent.TimeUnit;
-import java.util.regex.Pattern;
 
 import org.apache.lucene.analysis.Analyzer;
 import org.apache.lucene.document.Field;

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=1213085&r1=1213084&r2=1213085&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
Sun Dec 11 21:55:45 2011
@@ -25,6 +25,7 @@ import java.io.IOException;
  * @lucene.experimental
  */
 public class NullInfoStream extends InfoStream {
+
   @Override
   public void message(String component, String message) {
     assert component != null;



Mime
View raw message