You are viewing a plain text version of this content. The canonical link for it is here.
Posted to java-commits@lucene.apache.org by mi...@apache.org on 2007/10/03 17:24:13 UTC

svn commit: r581625 - in /lucene/java/trunk: ./ src/java/org/apache/lucene/index/ src/java/org/apache/lucene/store/ src/test/org/apache/lucene/index/ src/test/org/apache/lucene/store/

Author: mikemccand
Date: Wed Oct  3 08:24:12 2007
New Revision: 581625

URL: http://svn.apache.org/viewvc?rev=581625&view=rev
Log:
LUCENE-1011: 'get multiple writers over NFS' to work even when NFS client dir listing cache can be stale

Added:
    lucene/java/trunk/src/java/org/apache/lucene/store/LockStressTest.java   (with props)
    lucene/java/trunk/src/java/org/apache/lucene/store/LockVerifyServer.java   (with props)
    lucene/java/trunk/src/java/org/apache/lucene/store/VerifyingLockFactory.java   (with props)
Modified:
    lucene/java/trunk/CHANGES.txt
    lucene/java/trunk/src/java/org/apache/lucene/index/IndexFileDeleter.java
    lucene/java/trunk/src/java/org/apache/lucene/index/IndexWriter.java
    lucene/java/trunk/src/java/org/apache/lucene/index/SegmentInfos.java
    lucene/java/trunk/src/java/org/apache/lucene/store/LockFactory.java
    lucene/java/trunk/src/java/org/apache/lucene/store/NativeFSLockFactory.java
    lucene/java/trunk/src/java/org/apache/lucene/store/RAMDirectory.java
    lucene/java/trunk/src/java/org/apache/lucene/store/SimpleFSLockFactory.java
    lucene/java/trunk/src/test/org/apache/lucene/index/TestDeletionPolicy.java
    lucene/java/trunk/src/test/org/apache/lucene/index/TestFieldInfos.java
    lucene/java/trunk/src/test/org/apache/lucene/index/TestMultiSegmentReader.java
    lucene/java/trunk/src/test/org/apache/lucene/store/MockRAMDirectory.java

Modified: lucene/java/trunk/CHANGES.txt
URL: http://svn.apache.org/viewvc/lucene/java/trunk/CHANGES.txt?rev=581625&r1=581624&r2=581625&view=diff
==============================================================================
--- lucene/java/trunk/CHANGES.txt (original)
+++ lucene/java/trunk/CHANGES.txt Wed Oct  3 08:24:12 2007
@@ -123,6 +123,10 @@
     autoCommit=false and documents are using stored fields and/or term
     vectors.  (Mark Miller via Mike McCandless)
 
+20. LUCENE-1011: Fixed corruption case when two or more machines,
+    sharing an index over NFS, can be writers in quick succession.
+    (Patrick Kimber vis Mike McCandless)
+
 New features
 
  1. LUCENE-906: Elision filter for French.
@@ -135,6 +139,15 @@
      actual storage of Term Vectors has taken place.
 
  4. LUCENE-975: Added PositionBasedTermVectorMapper that allows for position based lookup of term vector information.  See item #3 above (LUCENE-868).
+
+ 5. LUCENE-1011: Added simple tools (all in org.apache.lucene.store)
+    to verify that locking is working properly.  LockVerifyServer runs
+    a separate server to verify locks.  LockStressTest runs a simple
+    tool that rapidly obtains and releases locks.
+    VerifyingLockFactory is a LockFactory that wraps any other
+    LockFactory and consults the LockVerifyServer whenever a lock is
+    obtained or released, throwing an exception if an illegal lock
+    obtain occurred.  (Patrick Kimber vis Mike McCandless)
 
 Optimizations
 

Modified: lucene/java/trunk/src/java/org/apache/lucene/index/IndexFileDeleter.java
URL: http://svn.apache.org/viewvc/lucene/java/trunk/src/java/org/apache/lucene/index/IndexFileDeleter.java?rev=581625&r1=581624&r2=581625&view=diff
==============================================================================
--- lucene/java/trunk/src/java/org/apache/lucene/index/IndexFileDeleter.java (original)
+++ lucene/java/trunk/src/java/org/apache/lucene/index/IndexFileDeleter.java Wed Oct  3 08:24:12 2007
@@ -102,10 +102,12 @@
 
   void setInfoStream(PrintStream infoStream) {
     this.infoStream = infoStream;
+    if (infoStream != null)
+      message("setInfoStream deletionPolicy=" + policy);
   }
   
   private void message(String message) {
-    infoStream.println("Deleter [" + Thread.currentThread().getName() + "]: " + message);
+    infoStream.println("IFD [" + Thread.currentThread().getName() + "]: " + message);
   }
 
   /**
@@ -125,7 +127,7 @@
     this.infoStream = infoStream;
 
     if (infoStream != null)
-      message("init: current segments file is \"" + segmentInfos.getCurrentSegmentFileName() + "\"");
+      message("init: current segments file is \"" + segmentInfos.getCurrentSegmentFileName() + "\"; deletionPolicy=" + policy);
 
     this.policy = policy;
     this.directory = directory;
@@ -189,7 +191,24 @@
     }
 
     if (currentCommitPoint == null) {
-      throw new CorruptIndexException("failed to locate current segments_N file");
+      // We did not in fact see the segments_N file
+      // corresponding to the segmentInfos that was passed
+      // in.  Yet, it must exist, because our caller holds
+      // the write lock.  This can happen when the directory
+      // listing was stale (eg when index accessed via NFS
+      // client with stale directory listing cache).  So we
+      // try now to explicitly open this commit point:
+      SegmentInfos sis = new SegmentInfos();
+      try {
+        sis.read(directory, segmentInfos.getCurrentSegmentFileName());
+      } catch (IOException e) {
+        throw new CorruptIndexException("failed to locate current segments_N file");
+      }
+      if (infoStream != null)
+        message("forced open of current segments file " + segmentInfos.getCurrentSegmentFileName());
+      currentCommitPoint = new CommitPoint(sis);
+      commits.add(currentCommitPoint);
+      incRef(sis, true);
     }
 
     // We keep commits list in sorted order (oldest to newest):

Modified: lucene/java/trunk/src/java/org/apache/lucene/index/IndexWriter.java
URL: http://svn.apache.org/viewvc/lucene/java/trunk/src/java/org/apache/lucene/index/IndexWriter.java?rev=581625&r1=581624&r2=581625&view=diff
==============================================================================
--- lucene/java/trunk/src/java/org/apache/lucene/index/IndexWriter.java (original)
+++ lucene/java/trunk/src/java/org/apache/lucene/index/IndexWriter.java Wed Oct  3 08:24:12 2007
@@ -250,6 +250,11 @@
   // LUCENE-888 for details.
   private final static int MERGE_READ_BUFFER_SIZE = 4096;
 
+  // Used for printing messages
+  private static Object MESSAGE_ID_LOCK = new Object();
+  private static int MESSAGE_ID = 0;
+  private int messageID = -1;
+
   private Directory directory;  // where this index resides
   private Analyzer analyzer;    // how to analyze text
 
@@ -300,7 +305,15 @@
   }
 
   private void message(String message) {
-    infoStream.println("IW [" + Thread.currentThread().getName() + "]: " + message);
+    infoStream.println("IW " + messageID + " [" + Thread.currentThread().getName() + "]: " + message);
+  }
+
+  private synchronized void setMessageID() {
+    if (infoStream != null && messageID == -1) {
+      synchronized(MESSAGE_ID_LOCK) {
+        messageID = MESSAGE_ID++;
+      }
+    }
   }
 
   /**
@@ -658,6 +671,7 @@
     directory = d;
     analyzer = a;
     this.infoStream = defaultInfoStream;
+    setMessageID();
 
     if (create) {
       // Clear the write lock in case it's leftover:
@@ -702,6 +716,11 @@
 
       pushMaxBufferedDocs();
 
+      if (infoStream != null) {
+        message("init: create=" + create);
+        messageState();
+      }
+
     } catch (IOException e) {
       this.writeLock.release();
       this.writeLock = null;
@@ -721,6 +740,8 @@
       mergePolicy.close();
     mergePolicy = mp;
     pushMaxBufferedDocs();
+    if (infoStream != null)
+      message("setMergePolicy " + mp);
   }
 
   /**
@@ -745,6 +766,8 @@
       this.mergeScheduler.close();
     }
     this.mergeScheduler = mergeScheduler;
+    if (infoStream != null)
+      message("setMergeScheduler " + mergeScheduler);
   }
 
   /**
@@ -804,6 +827,8 @@
   public void setMaxFieldLength(int maxFieldLength) {
     ensureOpen();
     this.maxFieldLength = maxFieldLength;
+    if (infoStream != null)
+      message("setMaxFieldLength " + maxFieldLength);
   }
 
   /**
@@ -846,6 +871,8 @@
           "at least one of ramBufferSize and maxBufferedDocs must be enabled");
     docWriter.setMaxBufferedDocs(maxBufferedDocs);
     pushMaxBufferedDocs();
+    if (infoStream != null)
+      message("setMaxBufferedDocs " + maxBufferedDocs);
   }
 
   /**
@@ -905,6 +932,8 @@
       throw new IllegalArgumentException(
           "at least one of ramBufferSize and maxBufferedDocs must be enabled");
     docWriter.setRAMBufferSizeMB(mb);
+    if (infoStream != null)
+      message("setRAMBufferSizeMB " + mb);
   }
 
   /**
@@ -933,6 +962,8 @@
       throw new IllegalArgumentException(
           "maxBufferedDeleteTerms must at least be 1 when enabled");
     docWriter.setMaxBufferedDeleteTerms(maxBufferedDeleteTerms);
+    if (infoStream != null)
+      message("setMaxBufferedDeleteTerms " + maxBufferedDeleteTerms);
   }
 
   /**
@@ -1004,8 +1035,23 @@
   public void setInfoStream(PrintStream infoStream) {
     ensureOpen();
     this.infoStream = infoStream;
+    setMessageID();
     docWriter.setInfoStream(infoStream);
     deleter.setInfoStream(infoStream);
+    if (infoStream != null)
+      messageState();
+  }
+
+  private void messageState() {
+    message("setInfoStream: dir=" + directory +
+            " autoCommit=" + autoCommit +
+            " mergePolicy=" + mergePolicy +
+            " mergeScheduler=" + mergeScheduler +
+            " ramBufferSizeMB=" + docWriter.getRAMBufferSizeMB() +
+            " maxBuffereDocs=" + docWriter.getMaxBufferedDocs() +
+            " maxBuffereDeleteTerms=" + docWriter.getMaxBufferedDeleteTerms() +
+            " maxFieldLength=" + maxFieldLength +
+            " index=" + segString());
   }
 
   /**
@@ -1129,6 +1175,8 @@
 
   private void closeInternal(boolean waitForMerges) throws CorruptIndexException, IOException {
     try {
+      if (infoStream != null)
+        message("now flush at close");
 
       flush(true, true);
 
@@ -1601,11 +1649,12 @@
    *  background threads. */
   public void optimize(boolean doWait) throws CorruptIndexException, IOException {
     ensureOpen();
-    flush();
 
     if (infoStream != null)
       message("optimize: index now " + segString());
 
+    flush();
+
     synchronized(this) {
       resetMergeExceptions();
       segmentsToOptimize = new HashSet();
@@ -1758,6 +1807,10 @@
     localRollbackSegmentInfos = (SegmentInfos) segmentInfos.clone();
     localAutoCommit = autoCommit;
     if (localAutoCommit) {
+
+      if (infoStream != null)
+        message("flush at startTransaction");
+
       flush();
       // Turn off auto-commit during our local transaction:
       autoCommit = false;
@@ -1990,6 +2043,8 @@
     throws CorruptIndexException, IOException {
 
     ensureOpen();
+    if (infoStream != null)
+      message("flush at addIndexes");
     flush();
 
     int start = segmentInfos.size();
@@ -2045,6 +2100,8 @@
       throws CorruptIndexException, IOException {
 
     ensureOpen();
+    if (infoStream != null)
+      message("flush at addIndexesNoOptimize");
     flush();
 
     /* new merge policy
@@ -2836,6 +2893,8 @@
       // TODO: if we know we are about to merge away these
       // newly flushed doc store files then we should not
       // make compound file out of them...
+      if (infoStream != null)
+        message("flush at merge");
       flush(false, true);
     }
 

Modified: lucene/java/trunk/src/java/org/apache/lucene/index/SegmentInfos.java
URL: http://svn.apache.org/viewvc/lucene/java/trunk/src/java/org/apache/lucene/index/SegmentInfos.java?rev=581625&r1=581624&r2=581625&view=diff
==============================================================================
--- lucene/java/trunk/src/java/org/apache/lucene/index/SegmentInfos.java (original)
+++ lucene/java/trunk/src/java/org/apache/lucene/index/SegmentInfos.java Wed Oct  3 08:24:12 2007
@@ -194,6 +194,9 @@
   public final void read(Directory directory, String segmentFileName) throws CorruptIndexException, IOException {
     boolean success = false;
 
+    // Clear any previous segments:
+    clear();
+
     IndexInput input = directory.openInput(segmentFileName);
 
     generation = generationFromSegmentsFileName(segmentFileName);
@@ -234,6 +237,7 @@
       }
     }
   }
+
   /**
    * This version of read uses the retry logic (for lock-less
    * commits) to find the right segments file to load.
@@ -444,7 +448,7 @@
 
   private static void message(String message) {
     if (infoStream != null) {
-      infoStream.println(Thread.currentThread().getName() + ": " + message);
+      infoStream.println("SIS [" + Thread.currentThread().getName() + "]: " + message);
     }
   }
 
@@ -492,54 +496,50 @@
       // it.
       
       // We have three methods for determining the current
-      // generation.  We try each in sequence.
+      // generation.  We try the first two in parallel, and
+      // fall back to the third when necessary.
 
       while(true) {
 
-        // Method 1: list the directory and use the highest
-        // segments_N file.  This method works well as long
-        // as there is no stale caching on the directory
-        // contents:
-        String[] files = null;
-
         if (0 == method) {
-          if (directory != null) {
-            files = directory.list();
-            if (files == null)
-              throw new FileNotFoundException("cannot read directory " + directory + ": list() returned null");
-          } else {
-            files = fileDirectory.list();
-            if (files == null)
-              throw new FileNotFoundException("cannot read directory " + fileDirectory + ": list() returned null");
-          }
 
-          gen = getCurrentSegmentGeneration(files);
+          // Method 1: list the directory and use the highest
+          // segments_N file.  This method works well as long
+          // as there is no stale caching on the directory
+          // contents (NOTE: NFS clients often have such stale
+          // caching):
+          String[] files = null;
 
-          if (gen == -1) {
-            String s = "";
-            for(int i=0;i<files.length;i++) {
-              s += " " + files[i];
-            }
-            throw new FileNotFoundException("no segments* file found in " + directory + ": files:" + s);
-          }
-        }
-
-        // Method 2 (fallback if Method 1 isn't reliable):
-        // if the directory listing seems to be stale, then
-        // try loading the "segments.gen" file.
-        if (1 == method || (0 == method && lastGen == gen && retry)) {
+          long genA = -1;
 
-          method = 1;
-            
+          if (directory != null)
+            files = directory.list();
+          else
+            files = fileDirectory.list();
+          
+          if (files != null)
+            genA = getCurrentSegmentGeneration(files);
+
+          message("directory listing genA=" + genA);
+
+          // Method 2: open segments.gen and read its
+          // contents.  Then we take the larger of the two
+          // gen's.  This way, if either approach is hitting
+          // a stale cache (NFS) we have a better chance of
+          // getting the right generation.
+          long genB = -1;
           for(int i=0;i<defaultGenFileRetryCount;i++) {
             IndexInput genInput = null;
             try {
               genInput = directory.openInput(IndexFileNames.SEGMENTS_GEN);
+            } catch (FileNotFoundException e) {
+              message("segments.gen open: FileNotFoundException " + e);
+              break;
             } catch (IOException e) {
               message("segments.gen open: IOException " + e);
             }
-            if (genInput != null) {
 
+            if (genInput != null) {
               try {
                 int version = genInput.readInt();
                 if (version == FORMAT_LOCKLESS) {
@@ -548,10 +548,7 @@
                   message("fallback check: " + gen0 + "; " + gen1);
                   if (gen0 == gen1) {
                     // The file is consistent.
-                    if (gen0 > gen) {
-                      message("fallback to '" + IndexFileNames.SEGMENTS_GEN + "' check: now try generation " + gen0 + " > " + gen);
-                      gen = gen0;
-                    }
+                    genB = gen0;
                     break;
                   }
                 }
@@ -567,15 +564,35 @@
               // will retry
             }
           }
+
+          message(IndexFileNames.SEGMENTS_GEN + " check: genB=" + genB);
+
+          // Pick the larger of the two gen's:
+          if (genA > genB)
+            gen = genA;
+          else
+            gen = genB;
+          
+          if (gen == -1) {
+            // Neither approach found a generation
+            String s;
+            if (files != null) {
+              s = "";
+              for(int i=0;i<files.length;i++)
+                s += " " + files[i];
+            } else
+              s = " null";
+            throw new FileNotFoundException("no segments* file found in " + directory + ": files:" + s);
+          }
         }
 
-        // Method 3 (fallback if Methods 2 & 3 are not
-        // reliable): since both directory cache and file
-        // contents cache seem to be stale, just advance the
-        // generation.
-        if (2 == method || (1 == method && lastGen == gen && retry)) {
+        // Third method (fallback if first & second methods
+        // are not reliable): since both directory cache and
+        // file contents cache seem to be stale, just
+        // advance the generation.
+        if (1 == method || (0 == method && lastGen == gen && retry)) {
 
-          method = 2;
+          method = 1;
 
           if (genLookaheadCount < defaultGenLookaheadCount) {
             gen++;

Modified: lucene/java/trunk/src/java/org/apache/lucene/store/LockFactory.java
URL: http://svn.apache.org/viewvc/lucene/java/trunk/src/java/org/apache/lucene/store/LockFactory.java?rev=581625&r1=581624&r2=581625&view=diff
==============================================================================
--- lucene/java/trunk/src/java/org/apache/lucene/store/LockFactory.java (original)
+++ lucene/java/trunk/src/java/org/apache/lucene/store/LockFactory.java Wed Oct  3 08:24:12 2007
@@ -20,8 +20,17 @@
 import java.io.IOException;
 
 /**
- * Base class for Locking implementation.  {@link Directory} uses
- * instances of this class to implement locking.
+ * <p>Base class for Locking implementation.  {@link Directory} uses
+ * instances of this class to implement locking.</p>
+ *
+ * <p>Note that there are some useful tools to verify that
+ * your LockFactory is working correctly: {@link
+ * VerifyingLockFactory}, {@link LockStressTest}, {@link
+ * LockVerifyServer}.</p>
+ *
+ * @see LockVerifyServer
+ * @see LockStressTest
+ * @see VerifyingLockFactory
  */
 
 public abstract class LockFactory {

Added: lucene/java/trunk/src/java/org/apache/lucene/store/LockStressTest.java
URL: http://svn.apache.org/viewvc/lucene/java/trunk/src/java/org/apache/lucene/store/LockStressTest.java?rev=581625&view=auto
==============================================================================
--- lucene/java/trunk/src/java/org/apache/lucene/store/LockStressTest.java (added)
+++ lucene/java/trunk/src/java/org/apache/lucene/store/LockStressTest.java Wed Oct  3 08:24:12 2007
@@ -0,0 +1,117 @@
+package org.apache.lucene.store;
+
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+import java.io.IOException;
+import java.io.File;
+
+/**
+ * Simple standalone tool that forever acquires & releases a
+ * lock using a specific LockFactory.  Run without any args
+ * to see usage.
+ *
+ * @see VerifyingLockFactory
+ * @see LockVerifyServer
+ */ 
+
+public class LockStressTest {
+
+  public static void main(String[] args) throws Exception {
+
+    if (args.length != 6) {
+      System.out.println("\nUsage: java org.apache.lucene.store.LockStressTest myID verifierHostOrIP verifierPort lockFactoryClassName lockDirName sleepTime\n" +
+                         "\n" +
+                         "  myID = int from 0 .. 255 (should be unique for test process)\n" +
+                         "  verifierHostOrIP = host name or IP address where LockVerifyServer is running\n" +
+                         "  verifierPort = port that LockVerifyServer is listening on\n" +
+                         "  lockFactoryClassName = primary LockFactory class that we will use\n" +
+                         "  lockDirName = path to the lock directory (only set for Simple/NativeFSLockFactory\n" +
+                         "  sleepTimeMS = milliseconds to pause betweeen each lock obtain/release\n" +
+                         "\n" +
+                         "You should run multiple instances of this process, each with its own\n" +
+                         "unique ID, and each pointing to the same lock directory, to verify\n" +
+                         "that locking is working correctly.\n" +
+                         "\n" +
+                         "Make sure you are first running LockVerifyServer.\n" + 
+                         "\n");
+      System.exit(1);
+    }
+
+    final int myID = Integer.parseInt(args[0]);
+
+    if (myID < 0 || myID > 255) {
+      System.out.println("myID must be a unique int 0..255");
+      System.exit(1);
+    }
+
+    final String verifierHost = args[1];
+    final int verifierPort = Integer.parseInt(args[2]);
+    final String lockFactoryClassName = args[3];
+    final String lockDirName = args[4];
+    final int sleepTimeMS = Integer.parseInt(args[5]);
+
+    Class c;
+    try {
+      c = Class.forName(lockFactoryClassName);
+    } catch (ClassNotFoundException e) {
+      throw new IOException("unable to find LockClass " + lockFactoryClassName);
+    }
+
+    LockFactory lockFactory;
+    try {
+      lockFactory = (LockFactory) c.newInstance();          
+    } catch (IllegalAccessException e) {
+      throw new IOException("IllegalAccessException when instantiating LockClass " + lockFactoryClassName);
+    } catch (InstantiationException e) {
+      throw new IOException("InstantiationException when instantiating LockClass " + lockFactoryClassName);
+    } catch (ClassCastException e) {
+      throw new IOException("unable to cast LockClass " + lockFactoryClassName + " instance to a LockFactory");
+    }
+
+    File lockDir = new File(lockDirName);
+
+    if (lockFactory instanceof NativeFSLockFactory) {
+      ((NativeFSLockFactory) lockFactory).setLockDir(lockDir);
+    } else if (lockFactory instanceof SimpleFSLockFactory) {
+      ((SimpleFSLockFactory) lockFactory).setLockDir(lockDir);
+    }
+
+    lockFactory.setLockPrefix("test");
+    
+    LockFactory verifyLF = new VerifyingLockFactory((byte) myID, lockFactory, verifierHost, verifierPort);
+
+    Lock l = verifyLF.makeLock("test.lock");
+
+    while(true) {
+
+      boolean obtained = false;
+
+      try {
+        obtained = l.obtain(10);
+      } catch (LockObtainFailedException e) {
+        System.out.print("x");
+      }
+
+      if (obtained) {
+        System.out.print("l");
+        l.release();
+      }
+      Thread.sleep(sleepTimeMS);
+    }
+  }
+}

Propchange: lucene/java/trunk/src/java/org/apache/lucene/store/LockStressTest.java
------------------------------------------------------------------------------
    svn:eol-style = native

Added: lucene/java/trunk/src/java/org/apache/lucene/store/LockVerifyServer.java
URL: http://svn.apache.org/viewvc/lucene/java/trunk/src/java/org/apache/lucene/store/LockVerifyServer.java?rev=581625&view=auto
==============================================================================
--- lucene/java/trunk/src/java/org/apache/lucene/store/LockVerifyServer.java (added)
+++ lucene/java/trunk/src/java/org/apache/lucene/store/LockVerifyServer.java Wed Oct  3 08:24:12 2007
@@ -0,0 +1,98 @@
+package org.apache.lucene.store;
+
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+import java.net.ServerSocket;
+import java.net.Socket;
+import java.io.OutputStream;
+import java.io.InputStream;
+import java.io.IOException;
+
+/**
+ * Simple standalone server that must be running when you
+ * use {@link VerifyingLockFactory}.  This server simply
+ * verifies at most one process holds the lock at a time.
+ * Run without any args to see usage.
+ *
+ * @see VerifyingLockFactory
+ * @see LockStressTest
+ */
+
+public class LockVerifyServer {
+
+  private static String getTime(long startTime) {
+    return "[" + ((System.currentTimeMillis()-startTime)/1000) + "s] ";
+  }
+
+  public static void main(String[] args) throws IOException {
+
+    if (args.length != 1) {
+      System.out.println("\nUsage: java org.apache.lucene.store.LockVerifyServer port\n");
+      System.exit(1);
+    }
+
+    final int port = Integer.parseInt(args[0]);
+
+    ServerSocket s = new ServerSocket(port);
+    s.setReuseAddress(true);
+    System.out.println("\nReady on port " + port + "...");
+
+    int lockedID = 0;
+    long startTime = System.currentTimeMillis();
+
+    char[] message = new char[3];
+
+    while(true) {
+      Socket cs = s.accept();
+      OutputStream out = cs.getOutputStream();
+      InputStream in = cs.getInputStream();
+
+      int id = in.read();
+      int command = in.read();
+
+      boolean err = false;
+
+      if (command == 1) {
+        // Locked
+        if (lockedID != 0) {
+          err = true;
+          System.out.println(getTime(startTime) + " ERROR: id " + id + " got lock, but " + lockedID + " already holds the lock");
+        }
+        lockedID = id;
+      } else if (command == 0) {
+        if (lockedID != id) {
+          err = true;
+          System.out.println(getTime(startTime) + " ERROR: id " + id + " released the lock, but " + lockedID + " is the one holding the lock");
+        }
+        lockedID = 0;
+      } else
+        throw new RuntimeException("unrecognized command " + command);
+
+      System.out.print(".");
+
+      if (err)
+        out.write(1);
+      else
+        out.write(0);
+
+      out.close();
+      in.close();
+      cs.close();
+    }
+  }
+}

Propchange: lucene/java/trunk/src/java/org/apache/lucene/store/LockVerifyServer.java
------------------------------------------------------------------------------
    svn:eol-style = native

Modified: lucene/java/trunk/src/java/org/apache/lucene/store/NativeFSLockFactory.java
URL: http://svn.apache.org/viewvc/lucene/java/trunk/src/java/org/apache/lucene/store/NativeFSLockFactory.java?rev=581625&r1=581624&r2=581625&view=diff
==============================================================================
--- lucene/java/trunk/src/java/org/apache/lucene/store/NativeFSLockFactory.java (original)
+++ lucene/java/trunk/src/java/org/apache/lucene/store/NativeFSLockFactory.java Wed Oct  3 08:24:12 2007
@@ -26,32 +26,32 @@
 import java.util.Random;
 
 /**
- * Implements {@link LockFactory} using native OS file locks
- * (available through java.nio.*).  Note that for certain
- * filesystems native locks are possible but must be
- * explicity configured and enabled (and may be disabled by
- * default).  For example, for NFS servers there sometimes
- * must be a separate lockd process running, and other
- * configuration may be required such as running the server
- * in kernel mode.  Other filesystems may not even support
- * native OS locks in which case you must use a different
- * {@link LockFactory} implementation.
- *
- * <p>The advantage of this lock factory over
- * {@link SimpleFSLockFactory} is that the locks should be
- * "correct", whereas {@link SimpleFSLockFactory} uses
- * java.io.File.createNewFile which
- * <a target="_top" href="http://java.sun.com/j2se/1.4.2/docs/api/java/io/File.html#createNewFile()">has warnings</a> about not
- * using it for locking.  Furthermore, if the JVM crashes,
- * the OS will free any held locks, whereas
- * {@link SimpleFSLockFactory} will keep the locks held, requiring
- * manual removal before re-running Lucene.</p>
+ * <p>Implements {@link LockFactory} using native OS file
+ * locks.  Note that because this LockFactory relies on
+ * java.nio.* APIs for locking, any problems with those APIs
+ * will cause locking to fail.  Specifically, on certain NFS
+ * environments the java.nio.* locks will fail (the lock can
+ * incorrectly be double acquired) whereas {@link
+ * SimpleFSLockFactory} worked perfectly in those same
+ * environments.  For NFS based access to an index, it's
+ * recommended that you try {@link SimpleFSLockFactory}
+ * first and work around the one limitation that a lock file
+ * could be left when the JVM exits abnormally.</p>
  *
+ * <p>The primary benefit of {@link NativeFSLockFactory} is
+ * that lock files will be properly removed (by the OS) if
+ * the JVM has an abnormal exit.</p>
+ * 
  * <p>Note that, unlike {@link SimpleFSLockFactory}, the existence of
  * leftover lock files in the filesystem on exiting the JVM
  * is fine because the OS will free the locks held against
  * these files even though the files still remain.</p>
  *
+ * <p>If you suspect that this or any other LockFactory is
+ * not working properly in your environment, you can easily
+ * test it by using {@link VerifyingLockFactory}, {@link
+ * LockVerifyServer} and {@link LockStressTest}.</p>
+ *
  * @see LockFactory
  */
 
@@ -317,7 +317,7 @@
     }
   }
 
-  public boolean isLocked() {
+  public synchronized boolean isLocked() {
     return lock != null;
   }
 

Modified: lucene/java/trunk/src/java/org/apache/lucene/store/RAMDirectory.java
URL: http://svn.apache.org/viewvc/lucene/java/trunk/src/java/org/apache/lucene/store/RAMDirectory.java?rev=581625&r1=581624&r2=581625&view=diff
==============================================================================
--- lucene/java/trunk/src/java/org/apache/lucene/store/RAMDirectory.java (original)
+++ lucene/java/trunk/src/java/org/apache/lucene/store/RAMDirectory.java Wed Oct  3 08:24:12 2007
@@ -209,7 +209,7 @@
   }
 
   /** Creates a new, empty file in the directory with the given name. Returns a stream writing this file. */
-  public IndexOutput createOutput(String name) {
+  public IndexOutput createOutput(String name) throws IOException {
     ensureOpen();
     RAMFile file = new RAMFile(this);
     synchronized (this) {

Modified: lucene/java/trunk/src/java/org/apache/lucene/store/SimpleFSLockFactory.java
URL: http://svn.apache.org/viewvc/lucene/java/trunk/src/java/org/apache/lucene/store/SimpleFSLockFactory.java?rev=581625&r1=581624&r2=581625&view=diff
==============================================================================
--- lucene/java/trunk/src/java/org/apache/lucene/store/SimpleFSLockFactory.java (original)
+++ lucene/java/trunk/src/java/org/apache/lucene/store/SimpleFSLockFactory.java Wed Oct  3 08:24:12 2007
@@ -21,11 +21,33 @@
 import java.io.IOException;
 
 /**
- * Implements {@link LockFactory} using {@link File#createNewFile()}.  This is
- * currently the default LockFactory used for {@link FSDirectory} if no
- * LockFactory instance is otherwise provided.
+ * <p>Implements {@link LockFactory} using {@link
+ * File#createNewFile()}.  This is the default LockFactory
+ * for {@link FSDirectory}.</p>
  *
- * Note that there are known problems with this locking implementation on NFS.
+ * <p><b>NOTE:</b> the <a target="_top"
+ * href="http://java.sun.com/j2se/1.4.2/docs/api/java/io/File.html#createNewFile()">javadocs
+ * for <code>File.createNewFile</code></a> contain a vague
+ * yet spooky warning about not using the API for file
+ * locking.  This warning was added due to <a target="_top"
+ * href="http://bugs.sun.com/bugdatabase/view_bug.do?bug_id=4676183">this
+ * bug</a>, and in fact the only known problem with using
+ * this API for locking is that the Lucene write lock may
+ * not be released when the JVM exits abnormally.</p>
+
+ * <p>When this happens, a {@link LockObtainFailedException}
+ * is hit when trying to create a writer, in which case you
+ * need to explicitly clear the lock file first.  You can
+ * either manually remove the file, or use the {@link
+ * org.apache.lucene.index.IndexReader#unlock(Directory)}
+ * API.  But, first be certain that no writer is in fact
+ * writing to the index otherwise you can easily corrupt
+ * your index.</p>
+ *
+ * <p>If you suspect that this or any other LockFactory is
+ * not working properly in your environment, you can easily
+ * test it by using {@link VerifyingLockFactory}, {@link
+ * LockVerifyServer} and {@link LockStressTest}.</p>
  *
  * @see LockFactory
  */

Added: lucene/java/trunk/src/java/org/apache/lucene/store/VerifyingLockFactory.java
URL: http://svn.apache.org/viewvc/lucene/java/trunk/src/java/org/apache/lucene/store/VerifyingLockFactory.java?rev=581625&view=auto
==============================================================================
--- lucene/java/trunk/src/java/org/apache/lucene/store/VerifyingLockFactory.java (added)
+++ lucene/java/trunk/src/java/org/apache/lucene/store/VerifyingLockFactory.java Wed Oct  3 08:24:12 2007
@@ -0,0 +1,119 @@
+package org.apache.lucene.store;
+
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+import java.net.Socket;
+import java.io.IOException;
+import java.io.InputStream;
+import java.io.OutputStream;
+
+/**
+ * A {@link LockFactory} that wraps another {@link
+ * LockFactory} and verifies that each lock obtain/release
+ * is "correct" (never results in two processes holding the
+ * lock at the same time).  It does this by contacting an
+ * external server ({@link LockVerifyServer}) to assert that
+ * at most one process holds the lock at a time.  To use
+ * this, you should also run {@link LockVerifyServer} on the
+ * host & port matching what you pass to the constructor.
+ *
+ * @see LockVerifyServer
+ * @see LockStressTest
+ */
+
+public class VerifyingLockFactory extends LockFactory {
+
+  LockFactory lf;
+  byte id;
+  String host;
+  int port;
+
+  private class CheckedLock extends Lock {
+    private Lock lock;
+
+    public CheckedLock(Lock lock) {
+      this.lock = lock;
+    }
+
+    private void verify(byte message) {
+      try {
+        Socket s = new Socket(host, port);
+        OutputStream out = s.getOutputStream();
+        out.write(id);
+        out.write(message);
+        InputStream in = s.getInputStream();
+        int result = in.read();
+        in.close();
+        out.close();
+        s.close();
+        if (result != 0)
+          throw new RuntimeException("lock was double acquired");
+      } catch (Exception e) {
+        throw new RuntimeException(e);
+      }
+    }
+
+    public synchronized boolean obtain(long lockWaitTimeout)
+      throws LockObtainFailedException, IOException {
+      boolean obtained = lock.obtain(lockWaitTimeout);
+      if (obtained)
+        verify((byte) 1);
+      return obtained;
+    }
+
+    public synchronized boolean obtain()
+      throws LockObtainFailedException, IOException {
+      return lock.obtain();
+    }
+
+    public synchronized boolean isLocked() {
+      return lock.isLocked();
+    }
+
+    public synchronized void release() {
+      if (isLocked()) {
+        verify((byte) 0);
+        lock.release();
+      }
+    }
+  }
+
+  /**
+   * @param id should be a unique id across all clients
+   * @param lf the LockFactory that we are testing
+   * @param host host or IP where {@link LockVerifyServer}
+            is running
+   * @param port the port {@link LockVerifyServer} is
+            listening on
+  */
+  public VerifyingLockFactory(byte id, LockFactory lf, String host, int port) throws IOException {
+    this.id = id;
+    this.lf = lf;
+    this.host = host;
+    this.port = port;
+  }
+
+  public synchronized Lock makeLock(String lockName) {
+    return new CheckedLock(lf.makeLock(lockName));
+  }
+
+  public synchronized void clearLock(String lockName)
+    throws IOException {
+    lf.clearLock(lockName);
+  }
+}

Propchange: lucene/java/trunk/src/java/org/apache/lucene/store/VerifyingLockFactory.java
------------------------------------------------------------------------------
    svn:eol-style = native

Modified: lucene/java/trunk/src/test/org/apache/lucene/index/TestDeletionPolicy.java
URL: http://svn.apache.org/viewvc/lucene/java/trunk/src/test/org/apache/lucene/index/TestDeletionPolicy.java?rev=581625&r1=581624&r2=581625&view=diff
==============================================================================
--- lucene/java/trunk/src/test/org/apache/lucene/index/TestDeletionPolicy.java (original)
+++ lucene/java/trunk/src/test/org/apache/lucene/index/TestDeletionPolicy.java Wed Oct  3 08:24:12 2007
@@ -220,6 +220,7 @@
     String fileName = IndexFileNames.fileNameFromGeneration(IndexFileNames.SEGMENTS,
                                                             "",
                                                             gen);
+    dir.deleteFile(IndexFileNames.SEGMENTS_GEN);
     while(gen > 0) {
       try {
         IndexReader reader = IndexReader.open(dir);
@@ -279,6 +280,7 @@
 
       // Simplistic check: just verify all segments_N's still
       // exist, and, I can open a reader on each:
+      dir.deleteFile(IndexFileNames.SEGMENTS_GEN);
       long gen = SegmentInfos.getCurrentSegmentGeneration(dir);
       while(gen > 0) {
         IndexReader reader = IndexReader.open(dir);
@@ -386,6 +388,7 @@
 
       // Simplistic check: just verify only the past N segments_N's still
       // exist, and, I can open a reader on each:
+      dir.deleteFile(IndexFileNames.SEGMENTS_GEN);
       long gen = SegmentInfos.getCurrentSegmentGeneration(dir);
       for(int i=0;i<N+1;i++) {
         try {
@@ -470,6 +473,7 @@
       // exist, and, I can open a reader on each:
       long gen = SegmentInfos.getCurrentSegmentGeneration(dir);
 
+      dir.deleteFile(IndexFileNames.SEGMENTS_GEN);
       int expectedCount = 176;
 
       for(int i=0;i<N+1;i++) {
@@ -575,6 +579,7 @@
       // exist, and, I can open a reader on each:
       long gen = SegmentInfos.getCurrentSegmentGeneration(dir);
 
+      dir.deleteFile(IndexFileNames.SEGMENTS_GEN);
       int expectedCount = 0;
 
       for(int i=0;i<N+1;i++) {

Modified: lucene/java/trunk/src/test/org/apache/lucene/index/TestFieldInfos.java
URL: http://svn.apache.org/viewvc/lucene/java/trunk/src/test/org/apache/lucene/index/TestFieldInfos.java?rev=581625&r1=581624&r2=581625&view=diff
==============================================================================
--- lucene/java/trunk/src/test/org/apache/lucene/index/TestFieldInfos.java (original)
+++ lucene/java/trunk/src/test/org/apache/lucene/index/TestFieldInfos.java Wed Oct  3 08:24:12 2007
@@ -41,7 +41,7 @@
   protected void tearDown() {
   }
 
-  public void test() {
+  public void test() throws IOException {
     //Positive test of FieldInfos
     assertTrue(testDoc != null);
     FieldInfos fieldInfos = new FieldInfos();

Modified: lucene/java/trunk/src/test/org/apache/lucene/index/TestMultiSegmentReader.java
URL: http://svn.apache.org/viewvc/lucene/java/trunk/src/test/org/apache/lucene/index/TestMultiSegmentReader.java?rev=581625&r1=581624&r2=581625&view=diff
==============================================================================
--- lucene/java/trunk/src/test/org/apache/lucene/index/TestMultiSegmentReader.java (original)
+++ lucene/java/trunk/src/test/org/apache/lucene/index/TestMultiSegmentReader.java Wed Oct  3 08:24:12 2007
@@ -41,14 +41,14 @@
 
   protected void setUp() throws IOException {
     dir = new RAMDirectory();
-    sis = new SegmentInfos();
     doc1 = new Document();
     doc2 = new Document();
     DocHelper.setupDoc(doc1);
     DocHelper.setupDoc(doc2);
     SegmentInfo info1 = DocHelper.writeDoc(dir, doc1);
     SegmentInfo info2 = DocHelper.writeDoc(dir, doc2);
-    sis.write(dir);
+    sis = new SegmentInfos();
+    sis.read(dir);
   }
 
   protected IndexReader openReader() throws IOException {
@@ -97,6 +97,12 @@
     // Ensure undeleteAll survives commit/close/reopen:
     reader.commit();
     reader.close();
+
+    if (reader instanceof MultiReader)
+      // MultiReader does not "own" the directory so it does
+      // not write the changes to sis on commit:
+      sis.write(dir);
+
     sis.read(dir);
     reader = openReader();
     assertEquals( 2, reader.numDocs() );
@@ -105,6 +111,10 @@
     assertEquals( 1, reader.numDocs() );
     reader.commit();
     reader.close();
+    if (reader instanceof MultiReader)
+      // MultiReader does not "own" the directory so it does
+      // not write the changes to sis on commit:
+      sis.write(dir);
     sis.read(dir);
     reader = openReader();
     assertEquals( 1, reader.numDocs() );

Modified: lucene/java/trunk/src/test/org/apache/lucene/store/MockRAMDirectory.java
URL: http://svn.apache.org/viewvc/lucene/java/trunk/src/test/org/apache/lucene/store/MockRAMDirectory.java?rev=581625&r1=581624&r2=581625&view=diff
==============================================================================
--- lucene/java/trunk/src/test/org/apache/lucene/store/MockRAMDirectory.java (original)
+++ lucene/java/trunk/src/test/org/apache/lucene/store/MockRAMDirectory.java Wed Oct  3 08:24:12 2007
@@ -135,16 +135,13 @@
     super.deleteFile(name);
   }
 
-  public IndexOutput createOutput(String name) {
+  public IndexOutput createOutput(String name) throws IOException {
     if (openFiles == null) {
       openFiles = new HashMap();
     }
     synchronized(openFiles) {
-      if (noDeleteOpenFile && openFiles.containsKey(name)) {
-        // RuntimeException instead of IOException because
-        // super() does not throw IOException currently:
-        throw new RuntimeException("MockRAMDirectory: file \"" + name + "\" is still open: cannot overwrite");
-      }
+      if (noDeleteOpenFile && openFiles.containsKey(name))
+       throw new IOException("MockRAMDirectory: file \"" + name + "\" is still open: cannot overwrite");
     }
     RAMFile file = new RAMFile(this);
     synchronized (this) {