You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@lucene.apache.org by rm...@apache.org on 2014/10/01 13:29:20 UTC

svn commit: r1628678 - in /lucene/dev/branches/branch_5x: ./ lucene/ lucene/backward-codecs/src/test/org/apache/lucene/index/ lucene/core/ lucene/core/src/java/org/apache/lucene/index/ lucene/core/src/test/org/apache/lucene/index/ lucene/replicator/ lu...

Author: rmuir
Date: Wed Oct  1 11:29:20 2014
New Revision: 1628678

URL: http://svn.apache.org/r1628678
Log:
LUCENE-5981: fix CheckIndex to obtain write.lock

Modified:
    lucene/dev/branches/branch_5x/   (props changed)
    lucene/dev/branches/branch_5x/lucene/   (props changed)
    lucene/dev/branches/branch_5x/lucene/CHANGES.txt   (contents, props changed)
    lucene/dev/branches/branch_5x/lucene/backward-codecs/src/test/org/apache/lucene/index/TestBackwardsCompatibility.java
    lucene/dev/branches/branch_5x/lucene/core/   (props changed)
    lucene/dev/branches/branch_5x/lucene/core/src/java/org/apache/lucene/index/CheckIndex.java
    lucene/dev/branches/branch_5x/lucene/core/src/test/org/apache/lucene/index/TestCheckIndex.java
    lucene/dev/branches/branch_5x/lucene/core/src/test/org/apache/lucene/index/TestIndexWriter.java
    lucene/dev/branches/branch_5x/lucene/core/src/test/org/apache/lucene/index/TestIndexWriterDelete.java
    lucene/dev/branches/branch_5x/lucene/replicator/   (props changed)
    lucene/dev/branches/branch_5x/lucene/replicator/src/test/org/apache/lucene/replicator/IndexAndTaxonomyReplicationClientTest.java
    lucene/dev/branches/branch_5x/lucene/test-framework/   (props changed)
    lucene/dev/branches/branch_5x/lucene/test-framework/src/java/org/apache/lucene/util/TestUtil.java

Modified: lucene/dev/branches/branch_5x/lucene/CHANGES.txt
URL: http://svn.apache.org/viewvc/lucene/dev/branches/branch_5x/lucene/CHANGES.txt?rev=1628678&r1=1628677&r2=1628678&view=diff
==============================================================================
--- lucene/dev/branches/branch_5x/lucene/CHANGES.txt (original)
+++ lucene/dev/branches/branch_5x/lucene/CHANGES.txt Wed Oct  1 11:29:20 2014
@@ -138,6 +138,11 @@ Bug Fixes
   work across multi-valued fields. Previously some cases across multi-valued
   fields would happily create a corrupt index. (Dawid Weiss, Robert Muir)
 
+* LUCENE-5981: CheckIndex obtains write.lock, since with some parameters it
+  may modify the index, and to prevent false corruption reports, as it does
+  not have the regular "spinlock" of DirectoryReader.open. It now implements
+  Closeable and you must close it to release the lock.  (Mike McCandless, Robert Muir)
+
 Documentation
 
 * LUCENE-5392: Add/improve analysis package documentation to reflect

Modified: lucene/dev/branches/branch_5x/lucene/backward-codecs/src/test/org/apache/lucene/index/TestBackwardsCompatibility.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/branch_5x/lucene/backward-codecs/src/test/org/apache/lucene/index/TestBackwardsCompatibility.java?rev=1628678&r1=1628677&r2=1628678&view=diff
==============================================================================
--- lucene/dev/branches/branch_5x/lucene/backward-codecs/src/test/org/apache/lucene/index/TestBackwardsCompatibility.java (original)
+++ lucene/dev/branches/branch_5x/lucene/backward-codecs/src/test/org/apache/lucene/index/TestBackwardsCompatibility.java Wed Oct  1 11:29:20 2014
@@ -498,6 +498,7 @@ public class TestBackwardsCompatibility 
       CheckIndex.Status indexStatus = checker.checkIndex();
       assertFalse(indexStatus.clean);
       assertTrue(bos.toString(IOUtils.UTF_8).contains(IndexFormatTooOldException.class.getName()));
+      checker.close();
 
       dir.close();
       IOUtils.rm(oldIndexDir);

Modified: lucene/dev/branches/branch_5x/lucene/core/src/java/org/apache/lucene/index/CheckIndex.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/branch_5x/lucene/core/src/java/org/apache/lucene/index/CheckIndex.java?rev=1628678&r1=1628677&r2=1628678&view=diff
==============================================================================
--- lucene/dev/branches/branch_5x/lucene/core/src/java/org/apache/lucene/index/CheckIndex.java (original)
+++ lucene/dev/branches/branch_5x/lucene/core/src/java/org/apache/lucene/index/CheckIndex.java Wed Oct  1 11:29:20 2014
@@ -17,6 +17,7 @@ package org.apache.lucene.index;
  * limitations under the License.
  */
 
+import java.io.Closeable;
 import java.io.IOException;
 import java.io.PrintStream;
 import java.nio.file.Path;
@@ -36,10 +37,13 @@ import org.apache.lucene.document.Docume
 import org.apache.lucene.index.CheckIndex.Status.DocValuesStatus;
 import org.apache.lucene.index.FieldInfo.IndexOptions;
 import org.apache.lucene.search.DocIdSetIterator;
+import org.apache.lucene.store.AlreadyClosedException;
 import org.apache.lucene.store.Directory;
 import org.apache.lucene.store.FSDirectory;
 import org.apache.lucene.store.IOContext;
 import org.apache.lucene.store.IndexInput;
+import org.apache.lucene.store.Lock;
+import org.apache.lucene.store.LockObtainFailedException;
 import org.apache.lucene.util.Accountables;
 import org.apache.lucene.util.Bits;
 import org.apache.lucene.util.BytesRef;
@@ -63,10 +67,12 @@ import org.apache.lucene.util.Version;
  * @lucene.experimental Please make a complete backup of your
  * index before using this to exorcise corrupted documents from your index!
  */
-public class CheckIndex {
+public class CheckIndex implements Closeable {
 
   private PrintStream infoStream;
   private Directory dir;
+  private Lock writeLock;
+  private volatile boolean closed;
 
   /**
    * Returned from {@link #checkIndex()} detailing the health and status of the index.
@@ -343,9 +349,36 @@ public class CheckIndex {
   }
 
   /** Create a new CheckIndex on the directory. */
-  public CheckIndex(Directory dir) {
+  public CheckIndex(Directory dir) throws IOException {
+    this(dir, dir.makeLock(IndexWriter.WRITE_LOCK_NAME));
+  }
+  
+  /** 
+   * Expert: create a directory with the specified lock.
+   * This should really not be used except for unit tests!!!!
+   * It exists only to support special tests (such as TestIndexWriterExceptions*),
+   * that would otherwise be more complicated to debug if they had to close the writer
+   * for each check.
+   */
+  public CheckIndex(Directory dir, Lock writeLock) throws IOException {
     this.dir = dir;
-    infoStream = null;
+    this.writeLock = writeLock;
+    this.infoStream = null;
+    if (!writeLock.obtain(IndexWriterConfig.WRITE_LOCK_TIMEOUT)) { // obtain write lock
+      throw new LockObtainFailedException("Index locked for write: " + writeLock);
+    }
+  }
+  
+  private void ensureOpen() {
+    if (closed) {
+      throw new AlreadyClosedException("this instance is closed");
+    }
+  }
+
+  @Override
+  public void close() throws IOException {
+    closed = true;
+    IOUtils.close(writeLock);
   }
 
   private boolean crossCheckTermVectors;
@@ -416,12 +449,9 @@ public class CheckIndex {
    *
    *  <p>As this method checks every byte in the specified
    *  segments, on a large index it can take quite a long
-   *  time to run.
-   *
-   *  <p><b>WARNING</b>: make sure
-   *  you only call this when the index is not opened by any
-   *  writer. */
+   *  time to run. */
   public Status checkIndex(List<String> onlySegments) throws IOException {
+    ensureOpen();
     NumberFormat nf = NumberFormat.getInstance(Locale.ROOT);
     SegmentInfos sis = new SegmentInfos();
     Status result = new Status();
@@ -2009,10 +2039,9 @@ public class CheckIndex {
    *  new segments file into the index, effectively removing
    *  all documents in broken segments from the index.
    *  BE CAREFUL.
-   *
-   * <p><b>WARNING</b>: Make sure you only call this when the
-   *  index is not opened  by any writer. */
+   */
   public void exorciseIndex(Status result) throws IOException {
+    ensureOpen();
     if (result.partial)
       throw new IllegalArgumentException("can only exorcise an index that was fully checked (this status checked a subset of segments)");
     result.newSegments.changed();
@@ -2064,6 +2093,12 @@ public class CheckIndex {
                        corruption, else 0.
    */
   public static void main(String[] args) throws IOException, InterruptedException {
+    int exitCode = doMain(args);
+    System.exit(exitCode);
+  }
+  
+  // actual main: returns exit code instead of terminating JVM (for easy testing)
+  private static int doMain(String args[]) throws IOException, InterruptedException {
 
     boolean doExorcise = false;
     boolean doCrossCheckTermVectors = false;
@@ -2083,21 +2118,21 @@ public class CheckIndex {
       } else if (arg.equals("-segment")) {
         if (i == args.length-1) {
           System.out.println("ERROR: missing name for -segment option");
-          System.exit(1);
+          return 1;
         }
         i++;
         onlySegments.add(args[i]);
       } else if ("-dir-impl".equals(arg)) {
         if (i == args.length - 1) {
           System.out.println("ERROR: missing value for -dir-impl option");
-          System.exit(1);
+          return 1;
         }
         i++;
         dirImpl = args[i];
       } else {
         if (indexPath != null) {
           System.out.println("ERROR: unexpected extra argument '" + args[i] + "'");
-          System.exit(1);
+          return 1;
         }
         indexPath = args[i];
       }
@@ -2131,7 +2166,7 @@ public class CheckIndex {
                          "\n" +
                          "This tool exits with exit code 1 if the index cannot be opened or has any\n" +
                          "corruption, else 0.\n");
-      System.exit(1);
+      return 1;
     }
 
     if (!assertsOn())
@@ -2141,56 +2176,57 @@ public class CheckIndex {
       onlySegments = null;
     else if (doExorcise) {
       System.out.println("ERROR: cannot specify both -exorcise and -segment");
-      System.exit(1);
+      return 1;
     }
 
     System.out.println("\nOpening index @ " + indexPath + "\n");
-    Directory dir = null;
+    Directory directory = null;
     Path path = Paths.get(indexPath);
     try {
       if (dirImpl == null) {
-        dir = FSDirectory.open(path);
+        directory = FSDirectory.open(path);
       } else {
-        dir = CommandLineUtil.newFSDirectory(dirImpl, path);
+        directory = CommandLineUtil.newFSDirectory(dirImpl, path);
       }
     } catch (Throwable t) {
       System.out.println("ERROR: could not open directory \"" + indexPath + "\"; exiting");
       t.printStackTrace(System.out);
-      System.exit(1);
-    }
-
-    CheckIndex checker = new CheckIndex(dir);
-    checker.setCrossCheckTermVectors(doCrossCheckTermVectors);
-    checker.setInfoStream(System.out, verbose);
-
-    Status result = checker.checkIndex(onlySegments);
-    if (result.missingSegments) {
-      System.exit(1);
+      return 1;
     }
 
-    if (!result.clean) {
-      if (!doExorcise) {
-        System.out.println("WARNING: would write new segments file, and " + result.totLoseDocCount + " documents would be lost, if -exorcise were specified\n");
+    try (Directory dir = directory;
+         CheckIndex checker = new CheckIndex(dir)) {
+      checker.setCrossCheckTermVectors(doCrossCheckTermVectors);
+      checker.setInfoStream(System.out, verbose);
+      
+      Status result = checker.checkIndex(onlySegments);
+      if (result.missingSegments) {
+        return 1;
+      }
+      
+      if (!result.clean) {
+        if (!doExorcise) {
+          System.out.println("WARNING: would write new segments file, and " + result.totLoseDocCount + " documents would be lost, if -exorcise were specified\n");
+        } else {
+          System.out.println("WARNING: " + result.totLoseDocCount + " documents will be lost\n");
+          System.out.println("NOTE: will write new segments file in 5 seconds; this will remove " + result.totLoseDocCount + " docs from the index. YOU WILL LOSE DATA. THIS IS YOUR LAST CHANCE TO CTRL+C!");
+          for(int s=0;s<5;s++) {
+            Thread.sleep(1000);
+            System.out.println("  " + (5-s) + "...");
+          }
+          System.out.println("Writing...");
+          checker.exorciseIndex(result);
+          System.out.println("OK");
+          System.out.println("Wrote new segments file \"" + result.newSegments.getSegmentsFileName() + "\"");
+        }
+      }
+      System.out.println("");
+      
+      if (result.clean == true) {
+        return 0;
       } else {
-        System.out.println("WARNING: " + result.totLoseDocCount + " documents will be lost\n");
-        System.out.println("NOTE: will write new segments file in 5 seconds; this will remove " + result.totLoseDocCount + " docs from the index. YOU WILL LOSE DATA. THIS IS YOUR LAST CHANCE TO CTRL+C!");
-        for(int s=0;s<5;s++) {
-          Thread.sleep(1000);
-          System.out.println("  " + (5-s) + "...");
-        }
-        System.out.println("Writing...");
-        checker.exorciseIndex(result);
-        System.out.println("OK");
-        System.out.println("Wrote new segments file \"" + result.newSegments.getSegmentsFileName() + "\"");
-      }
-    }
-    System.out.println("");
-
-    final int exitCode;
-    if (result.clean == true)
-      exitCode = 0;
-    else
-      exitCode = 1;
-    System.exit(exitCode);
+        return 1;
+      }
+    }
   }
 }

Modified: lucene/dev/branches/branch_5x/lucene/core/src/test/org/apache/lucene/index/TestCheckIndex.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/branch_5x/lucene/core/src/test/org/apache/lucene/index/TestCheckIndex.java?rev=1628678&r1=1628677&r2=1628678&view=diff
==============================================================================
--- lucene/dev/branches/branch_5x/lucene/core/src/test/org/apache/lucene/index/TestCheckIndex.java (original)
+++ lucene/dev/branches/branch_5x/lucene/core/src/test/org/apache/lucene/index/TestCheckIndex.java Wed Oct  1 11:29:20 2014
@@ -26,6 +26,7 @@ import java.util.ArrayList;
 import org.apache.lucene.util.IOUtils;
 import org.apache.lucene.util.LuceneTestCase;
 import org.apache.lucene.store.Directory;
+import org.apache.lucene.store.LockObtainFailedException;
 import org.apache.lucene.analysis.CannedTokenStream;
 import org.apache.lucene.analysis.MockAnalyzer;
 import org.apache.lucene.analysis.Token;
@@ -95,6 +96,7 @@ public class TestCheckIndex extends Luce
     onlySegments.add("_0");
     
     assertTrue(checker.checkIndex(onlySegments).clean == true);
+    checker.close();
     dir.close();
   }
   
@@ -115,4 +117,22 @@ public class TestCheckIndex extends Luce
     iw.close();
     dir.close(); // checkindex
   }
+  
+  public void testObtainsLock() throws IOException {
+    Directory dir = newDirectory();
+    IndexWriter iw = new IndexWriter(dir, newIndexWriterConfig(null));
+    iw.addDocument(new Document());
+    iw.commit();
+    
+    // keep IW open...
+    try {
+      new CheckIndex(dir);
+      fail("should not have obtained write lock");
+    } catch (LockObtainFailedException expected) {
+      // ok
+    }
+    
+    iw.close();
+    dir.close();
+  }
 }

Modified: lucene/dev/branches/branch_5x/lucene/core/src/test/org/apache/lucene/index/TestIndexWriter.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/branch_5x/lucene/core/src/test/org/apache/lucene/index/TestIndexWriter.java?rev=1628678&r1=1628677&r2=1628678&view=diff
==============================================================================
--- lucene/dev/branches/branch_5x/lucene/core/src/test/org/apache/lucene/index/TestIndexWriter.java (original)
+++ lucene/dev/branches/branch_5x/lucene/core/src/test/org/apache/lucene/index/TestIndexWriter.java Wed Oct  1 11:29:20 2014
@@ -2705,6 +2705,7 @@ public class TestIndexWriter extends Luc
     checker.setInfoStream(new PrintStream(bos, false, IOUtils.UTF_8), false);
     CheckIndex.Status indexStatus = checker.checkIndex(null);
     String s = bos.toString(IOUtils.UTF_8);
+    checker.close();
     // Make sure CheckIndex didn't fail
     assertTrue(s, indexStatus != null && indexStatus.clean);
 

Modified: lucene/dev/branches/branch_5x/lucene/core/src/test/org/apache/lucene/index/TestIndexWriterDelete.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/branch_5x/lucene/core/src/test/org/apache/lucene/index/TestIndexWriterDelete.java?rev=1628678&r1=1628677&r2=1628678&view=diff
==============================================================================
--- lucene/dev/branches/branch_5x/lucene/core/src/test/org/apache/lucene/index/TestIndexWriterDelete.java (original)
+++ lucene/dev/branches/branch_5x/lucene/core/src/test/org/apache/lucene/index/TestIndexWriterDelete.java Wed Oct  1 11:29:20 2014
@@ -1194,6 +1194,7 @@ public class TestIndexWriterDelete exten
     checker.setInfoStream(new PrintStream(bos, false, IOUtils.UTF_8), false);
     CheckIndex.Status indexStatus = checker.checkIndex(null);
     assertTrue(indexStatus.clean);
+    checker.close();
     String s = bos.toString(IOUtils.UTF_8);
 
     // Segment should have deletions:
@@ -1204,9 +1205,11 @@ public class TestIndexWriterDelete exten
     w.close();
 
     bos = new ByteArrayOutputStream(1024);
+    checker = new CheckIndex(dir);
     checker.setInfoStream(new PrintStream(bos, false, IOUtils.UTF_8), false);
     indexStatus = checker.checkIndex(null);
     assertTrue(indexStatus.clean);
+    checker.close();
     s = bos.toString(IOUtils.UTF_8);
     assertFalse(s.contains("has deletions"));
     dir.close();

Modified: lucene/dev/branches/branch_5x/lucene/replicator/src/test/org/apache/lucene/replicator/IndexAndTaxonomyReplicationClientTest.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/branch_5x/lucene/replicator/src/test/org/apache/lucene/replicator/IndexAndTaxonomyReplicationClientTest.java?rev=1628678&r1=1628677&r2=1628678&view=diff
==============================================================================
--- lucene/dev/branches/branch_5x/lucene/replicator/src/test/org/apache/lucene/replicator/IndexAndTaxonomyReplicationClientTest.java (original)
+++ lucene/dev/branches/branch_5x/lucene/replicator/src/test/org/apache/lucene/replicator/IndexAndTaxonomyReplicationClientTest.java Wed Oct  1 11:29:20 2014
@@ -412,16 +412,18 @@ public class IndexAndTaxonomyReplication
             // verify taxonomy index is fully consistent (since we only add one
             // category to all documents, there's nothing much more to validate.
             ByteArrayOutputStream bos = new ByteArrayOutputStream(1024);
-            CheckIndex checker = new CheckIndex(handlerTaxoDir.getDelegate());
-            checker.setFailFast(true);
-            checker.setInfoStream(new PrintStream(bos, false, IOUtils.UTF_8), false);
             CheckIndex.Status indexStatus = null;
-            try {
-              indexStatus = checker.checkIndex(null);
-            } catch (IOException ioe) {
-              // ok: we fallback below
-            } catch (RuntimeException re) {
-              // ok: we fallback below
+
+            try (CheckIndex checker = new CheckIndex(handlerTaxoDir.getDelegate())) {
+              checker.setFailFast(true);
+              checker.setInfoStream(new PrintStream(bos, false, IOUtils.UTF_8), false);
+              try {
+                indexStatus = checker.checkIndex(null);
+              } catch (IOException ioe) {
+                // ok: we fallback below
+              } catch (RuntimeException re) {
+                // ok: we fallback below
+              }
             }
 
             if (indexStatus == null || indexStatus.clean == false) {

Modified: lucene/dev/branches/branch_5x/lucene/test-framework/src/java/org/apache/lucene/util/TestUtil.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/branch_5x/lucene/test-framework/src/java/org/apache/lucene/util/TestUtil.java?rev=1628678&r1=1628677&r2=1628678&view=diff
==============================================================================
--- lucene/dev/branches/branch_5x/lucene/test-framework/src/java/org/apache/lucene/util/TestUtil.java (original)
+++ lucene/dev/branches/branch_5x/lucene/test-framework/src/java/org/apache/lucene/util/TestUtil.java Wed Oct  1 11:29:20 2014
@@ -85,6 +85,7 @@ import org.apache.lucene.search.Filtered
 import org.apache.lucene.search.ScoreDoc;
 import org.apache.lucene.search.TopDocs;
 import org.apache.lucene.store.Directory;
+import org.apache.lucene.store.NoLockFactory;
 import org.junit.Assert;
 
 import com.carrotsearch.randomizedtesting.generators.RandomInts;
@@ -213,20 +214,24 @@ public final class TestUtil {
    *  look for any other corruption.  */
   public static CheckIndex.Status checkIndex(Directory dir, boolean crossCheckTermVectors, boolean failFast) throws IOException {
     ByteArrayOutputStream bos = new ByteArrayOutputStream(1024);
-    CheckIndex checker = new CheckIndex(dir);
-    checker.setCrossCheckTermVectors(crossCheckTermVectors);
-    checker.setFailFast(failFast);
-    checker.setInfoStream(new PrintStream(bos, false, IOUtils.UTF_8), false);
-    CheckIndex.Status indexStatus = checker.checkIndex(null);
-    if (indexStatus == null || indexStatus.clean == false) {
-      System.out.println("CheckIndex failed");
-      System.out.println(bos.toString(IOUtils.UTF_8));
-      throw new RuntimeException("CheckIndex failed");
-    } else {
-      if (LuceneTestCase.INFOSTREAM) {
+    // TODO: actually use the dir's lock factory, unless test uses a special method?
+    // some tests e.g. exception tests become much more complicated if they have to close the writer
+    try (CheckIndex checker = new CheckIndex(dir, NoLockFactory.getNoLockFactory().makeLock("bogus"))) {
+      checker.setCrossCheckTermVectors(crossCheckTermVectors);
+      checker.setFailFast(failFast);
+      checker.setInfoStream(new PrintStream(bos, false, IOUtils.UTF_8), false);
+      CheckIndex.Status indexStatus = checker.checkIndex(null);
+      
+      if (indexStatus == null || indexStatus.clean == false) {
+        System.out.println("CheckIndex failed");
         System.out.println(bos.toString(IOUtils.UTF_8));
+        throw new RuntimeException("CheckIndex failed");
+      } else {
+        if (LuceneTestCase.INFOSTREAM) {
+          System.out.println(bos.toString(IOUtils.UTF_8));
+        }
+        return indexStatus;
       }
-      return indexStatus;
     }
   }