You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@lucene.apache.org by mi...@apache.org on 2014/08/27 00:16:32 UTC

svn commit: r1620753 - in /lucene/dev/branches/lucene5904/lucene/core/src: java/org/apache/lucene/index/ test/org/apache/lucene/index/

Author: mikemccand
Date: Tue Aug 26 22:16:32 2014
New Revision: 1620753

URL: http://svn.apache.org/r1620753
Log:
LUCENE-5904: first cut at gen inflation to prevent index corruption when files are re-used after first IW has unclean shutdown and 2nd IW encounters virus checker

Modified:
    lucene/dev/branches/lucene5904/lucene/core/src/java/org/apache/lucene/index/IndexFileDeleter.java
    lucene/dev/branches/lucene5904/lucene/core/src/java/org/apache/lucene/index/IndexFileNames.java
    lucene/dev/branches/lucene5904/lucene/core/src/java/org/apache/lucene/index/SegmentCommitInfo.java
    lucene/dev/branches/lucene5904/lucene/core/src/java/org/apache/lucene/index/SegmentInfos.java
    lucene/dev/branches/lucene5904/lucene/core/src/test/org/apache/lucene/index/TestIndexFileDeleter.java
    lucene/dev/branches/lucene5904/lucene/core/src/test/org/apache/lucene/index/TestIndexWriterCommit.java
    lucene/dev/branches/lucene5904/lucene/core/src/test/org/apache/lucene/index/TestIndexWriterDelete.java

Modified: lucene/dev/branches/lucene5904/lucene/core/src/java/org/apache/lucene/index/IndexFileDeleter.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene5904/lucene/core/src/java/org/apache/lucene/index/IndexFileDeleter.java?rev=1620753&r1=1620752&r2=1620753&view=diff
==============================================================================
--- lucene/dev/branches/lucene5904/lucene/core/src/java/org/apache/lucene/index/IndexFileDeleter.java (original)
+++ lucene/dev/branches/lucene5904/lucene/core/src/java/org/apache/lucene/index/IndexFileDeleter.java Tue Aug 26 22:16:32 2014
@@ -150,7 +150,7 @@ final class IndexFileDeleter implements 
       // it means the directory is empty, so ignore it.
       files = new String[0];
     }
-    
+
     if (currentSegmentsFile != null) {
       Matcher m = IndexFileNames.CODEC_FILE_PATTERN.matcher("");
       for (String fileName : files) {
@@ -236,6 +236,8 @@ final class IndexFileDeleter implements 
     // We keep commits list in sorted order (oldest to newest):
     CollectionUtil.timSort(commits);
 
+    inflateGens(segmentInfos);
+
     // Now delete anything with ref count at 0.  These are
     // presumably abandoned files eg due to crash of
     // IndexWriter.
@@ -263,6 +265,74 @@ final class IndexFileDeleter implements 
     deleteCommits();
   }
 
+  /** Set all gens beyond what we currently see in the directory, to avoid double-write in cases where the previous IndexWriter did not
+   *  gracefully close/rollback (e.g. os/machine crashed or lost power). */
+  private void inflateGens(SegmentInfos infos) {
+
+    long maxSegmentGen = Long.MIN_VALUE;
+    int maxSegmentName = Integer.MIN_VALUE;
+
+    // Confusingly, this is the union of liveDocs, field infos, doc values
+    // (and maybe others, in the future) gens.  This is somewhat messy,
+    // since it means DV updates will suddenly write to the next gen after
+    // live docs' gen, for example, but we don't have the APIs to ask the
+    // codec which file is which:
+    Map<String,Long> maxPerSegmentGen = new HashMap<>();
+
+    // refCounts only includes "normal" filenames (does not include segments.gen, write.lock)
+    for(String fileName : refCounts.keySet()) {
+      if (fileName.startsWith(IndexFileNames.SEGMENTS)) {
+        maxSegmentGen = Math.max(SegmentInfos.generationFromSegmentsFileName(fileName), maxSegmentGen);
+      } else {
+        String segmentName = IndexFileNames.parseSegmentName(fileName);
+        assert segmentName.startsWith("_");
+
+        maxSegmentName = Math.max(maxSegmentName, Integer.parseInt(segmentName.substring(1), Character.MAX_RADIX));
+
+        long gen = IndexFileNames.parseGeneration(fileName);
+        Long curGen = maxPerSegmentGen.get(segmentName);
+        if (curGen == null) {
+          // We can't detect a gen=0 situation, so we always assume gen=1 to start:
+          curGen = 1L;
+        }
+        maxPerSegmentGen.put(segmentName, Math.max(curGen, gen));
+      }
+    }
+
+    // Generation is advanced before write:
+    infos.setGeneration(Math.max(infos.getGeneration(), maxSegmentGen));
+    if (infos.counter < 1+maxSegmentName) {
+      if (infoStream.isEnabled("IFD")) {
+        infoStream.message("IFD", "init: inflate infos.counter to " + (1+maxSegmentName) + " vs current=" + infos.counter);
+      }
+      infos.counter = 1+maxSegmentName;
+    }
+
+    for(SegmentCommitInfo info : infos) {
+      Long gen = maxPerSegmentGen.get(info.info.name);
+      assert gen != null;
+      long genLong = gen;
+      if (info.getNextWriteDelGen() < genLong+1) {
+        if (infoStream.isEnabled("IFD")) {
+          infoStream.message("IFD", "init: seg=" + info.info.name + " set nextWriteDelGen=" + (genLong+1) + " vs current=" + info.getNextWriteDelGen());
+        }
+        info.setNextWriteDelGen(genLong+1);
+      }
+      if (info.getNextWriteFieldInfosGen() < genLong+1) {
+        if (infoStream.isEnabled("IFD")) {
+          infoStream.message("IFD", "init: seg=" + info.info.name + " set nextWriteFieldInfosGen=" + (genLong+1) + " vs current=" + info.getNextWriteFieldInfosGen());
+        }
+        info.setNextWriteFieldInfosGen(genLong+1);
+      }
+      if (info.getNextWriteDocValuesGen() < genLong+1) {
+        if (infoStream.isEnabled("IFD")) {
+          infoStream.message("IFD", "init: seg=" + info.info.name + " set nextWriteDocValuesGen=" + (genLong+1) + " vs current=" + info.getNextWriteDocValuesGen());
+        }
+        info.setNextWriteDocValuesGen(genLong+1);
+      }
+    }
+  }
+
   private void ensureOpen() throws AlreadyClosedException {
     if (writer == null) {
       throw new AlreadyClosedException("this IndexWriter is closed");

Modified: lucene/dev/branches/lucene5904/lucene/core/src/java/org/apache/lucene/index/IndexFileNames.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene5904/lucene/core/src/java/org/apache/lucene/index/IndexFileNames.java?rev=1620753&r1=1620752&r2=1620753&view=diff
==============================================================================
--- lucene/dev/branches/lucene5904/lucene/core/src/java/org/apache/lucene/index/IndexFileNames.java (original)
+++ lucene/dev/branches/lucene5904/lucene/core/src/java/org/apache/lucene/index/IndexFileNames.java Tue Aug 26 22:16:32 2014
@@ -172,6 +172,21 @@ public final class IndexFileNames {
     }
     return filename;
   }
+
+  /** Returns the generation from this file name, or 0 if there is no
+   *  generation. */
+  public static long parseGeneration(String filename) {
+    String gen = stripExtension(stripSegmentName(filename));
+    if (gen.length() == 0) {
+      return 0L;
+    } else if (gen.indexOf('_') != -1) {
+      // This is a non-generational segment file of form _seg_codec_perFieldId.ext:
+      return 0L;
+    } else {
+      assert gen.startsWith("_");
+      return Long.parseLong(gen.substring(1), Character.MAX_RADIX);
+    }
+  }
   
   /**
    * Parses the segment name out of the given file name.

Modified: lucene/dev/branches/lucene5904/lucene/core/src/java/org/apache/lucene/index/SegmentCommitInfo.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene5904/lucene/core/src/java/org/apache/lucene/index/SegmentCommitInfo.java?rev=1620753&r1=1620752&r2=1620753&view=diff
==============================================================================
--- lucene/dev/branches/lucene5904/lucene/core/src/java/org/apache/lucene/index/SegmentCommitInfo.java (original)
+++ lucene/dev/branches/lucene5904/lucene/core/src/java/org/apache/lucene/index/SegmentCommitInfo.java Tue Aug 26 22:16:32 2014
@@ -148,6 +148,16 @@ public class SegmentCommitInfo {
     nextWriteDelGen++;
   }
   
+  /** Gets the nextWriteDelGen. */
+  long getNextWriteDelGen() {
+    return nextWriteDelGen;
+  }
+  
+  /** Sets the nextWriteDelGen. */
+  void setNextWriteDelGen(long v) {
+    nextWriteDelGen = v;
+  }
+  
   /** Called when we succeed in writing a new FieldInfos generation. */
   void advanceFieldInfosGen() {
     fieldInfosGen = nextWriteFieldInfosGen;
@@ -163,6 +173,16 @@ public class SegmentCommitInfo {
     nextWriteFieldInfosGen++;
   }
   
+  /** Gets the nextWriteFieldInfosGen. */
+  long getNextWriteFieldInfosGen() {
+    return nextWriteFieldInfosGen;
+  }
+  
+  /** Sets the nextWriteFieldInfosGen. */
+  void setNextWriteFieldInfosGen(long v) {
+    nextWriteFieldInfosGen = v;
+  }
+
   /** Called when we succeed in writing a new DocValues generation. */
   void advanceDocValuesGen() {
     docValuesGen = nextWriteDocValuesGen;
@@ -178,6 +198,16 @@ public class SegmentCommitInfo {
     nextWriteDocValuesGen++;
   }
 
+  /** Gets the nextWriteDocValuesGen. */
+  long getNextWriteDocValuesGen() {
+    return nextWriteDocValuesGen;
+  }
+  
+  /** Sets the nextWriteDocValuesGen. */
+  void setNextWriteDocValuesGen(long v) {
+    nextWriteDocValuesGen = v;
+  }
+  
   /** Returns total size in bytes of all files for this
    *  segment. */
   public long sizeInBytes() throws IOException {

Modified: lucene/dev/branches/lucene5904/lucene/core/src/java/org/apache/lucene/index/SegmentInfos.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene5904/lucene/core/src/java/org/apache/lucene/index/SegmentInfos.java?rev=1620753&r1=1620752&r2=1620753&view=diff
==============================================================================
--- lucene/dev/branches/lucene5904/lucene/core/src/java/org/apache/lucene/index/SegmentInfos.java (original)
+++ lucene/dev/branches/lucene5904/lucene/core/src/java/org/apache/lucene/index/SegmentInfos.java Tue Aug 26 22:16:32 2014
@@ -150,6 +150,7 @@ public final class SegmentInfos implemen
   public static final int FORMAT_SEGMENTS_GEN_CURRENT = FORMAT_SEGMENTS_GEN_CHECKSUM;
 
   /** Used to name new segments. */
+  // TODO: should this be a long ...?
   public int counter;
   
   /** Counts how often the index has been changed.  */
@@ -337,6 +338,8 @@ public final class SegmentInfos implemen
   public final void read(Directory directory, String segmentFileName) throws IOException {
     boolean success = false;
 
+    System.out.println("SIS.read " + segmentFileName);
+
     // Clear any previous segments:
     this.clear();
 
@@ -863,6 +866,11 @@ public final class SegmentInfos implemen
     generation = other.generation;
   }
 
+  void setGeneration(long generation) {
+    this.generation = generation;
+    this.lastGeneration = generation;
+  }
+
   final void rollbackCommit(Directory dir) {
     if (pendingSegnOutput != null) {
       // Suppress so we keep throwing the original exception

Modified: lucene/dev/branches/lucene5904/lucene/core/src/test/org/apache/lucene/index/TestIndexFileDeleter.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene5904/lucene/core/src/test/org/apache/lucene/index/TestIndexFileDeleter.java?rev=1620753&r1=1620752&r2=1620753&view=diff
==============================================================================
--- lucene/dev/branches/lucene5904/lucene/core/src/test/org/apache/lucene/index/TestIndexFileDeleter.java (original)
+++ lucene/dev/branches/lucene5904/lucene/core/src/test/org/apache/lucene/index/TestIndexFileDeleter.java Tue Aug 26 22:16:32 2014
@@ -209,8 +209,6 @@ public class TestIndexFileDeleter extend
     writer.addDocument(doc);
   }
   
-  @Ignore("not yet")
-  // nocommit
   public void testVirusScannerDoesntCorruptIndex() throws IOException {
     MockDirectoryWrapper dir = newMockDirectory();
     dir.setPreventDoubleWrite(false); // we arent trying to test this

Modified: lucene/dev/branches/lucene5904/lucene/core/src/test/org/apache/lucene/index/TestIndexWriterCommit.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene5904/lucene/core/src/test/org/apache/lucene/index/TestIndexWriterCommit.java?rev=1620753&r1=1620752&r2=1620753&view=diff
==============================================================================
--- lucene/dev/branches/lucene5904/lucene/core/src/test/org/apache/lucene/index/TestIndexWriterCommit.java (original)
+++ lucene/dev/branches/lucene5904/lucene/core/src/test/org/apache/lucene/index/TestIndexWriterCommit.java Tue Aug 26 22:16:32 2014
@@ -18,6 +18,7 @@ package org.apache.lucene.index;
  */
 
 import java.io.IOException;
+import java.util.Arrays;
 import java.util.HashMap;
 import java.util.Map;
 import java.util.concurrent.atomic.AtomicBoolean;
@@ -569,6 +570,8 @@ public class TestIndexWriterCommit exten
   // LUCENE-1274: test writer.prepareCommit()
   public void testPrepareCommitRollback() throws IOException {
     Directory dir = newDirectory();
+
+    // nocommit remove:
     if (dir instanceof MockDirectoryWrapper) {
       ((MockDirectoryWrapper)dir).setPreventDoubleWrite(false);
     }
@@ -581,8 +584,9 @@ public class TestIndexWriterCommit exten
     );
     writer.commit();
 
-    for (int i = 0; i < 23; i++)
+    for (int i = 0; i < 23; i++) {
       TestIndexWriter.addDoc(writer);
+    }
 
     DirectoryReader reader = DirectoryReader.open(dir);
     assertEquals(0, reader.numDocs());
@@ -593,6 +597,7 @@ public class TestIndexWriterCommit exten
     assertEquals(0, reader2.numDocs());
 
     writer.rollback();
+    System.out.println("TEST: after rollback: " + Arrays.toString(dir.listAll()));
 
     IndexReader reader3 = DirectoryReader.openIfChanged(reader);
     assertNull(reader3);
@@ -602,8 +607,9 @@ public class TestIndexWriterCommit exten
     reader2.close();
 
     writer = new IndexWriter(dir, newIndexWriterConfig(new MockAnalyzer(random())));
-    for (int i = 0; i < 17; i++)
+    for (int i = 0; i < 17; i++) {
       TestIndexWriter.addDoc(writer);
+    }
 
     reader = DirectoryReader.open(dir);
     assertEquals(0, reader.numDocs());
@@ -611,6 +617,9 @@ public class TestIndexWriterCommit exten
 
     writer.prepareCommit();
 
+    // nocommit test fails here because at this point (thanks to gen inflation) we have a good segments_1 (empty first commit), a broken
+    // segments_2 (prepareCommit then rollback) and a broken segmetns_3 (prepareCommit, writer still open) ... this easily can happen in a
+    // real app ... I think we must fix SIS fallback logic to try more than just the last one?
     reader = DirectoryReader.open(dir);
     assertEquals(0, reader.numDocs());
     reader.close();

Modified: lucene/dev/branches/lucene5904/lucene/core/src/test/org/apache/lucene/index/TestIndexWriterDelete.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene5904/lucene/core/src/test/org/apache/lucene/index/TestIndexWriterDelete.java?rev=1620753&r1=1620752&r2=1620753&view=diff
==============================================================================
--- lucene/dev/branches/lucene5904/lucene/core/src/test/org/apache/lucene/index/TestIndexWriterDelete.java (original)
+++ lucene/dev/branches/lucene5904/lucene/core/src/test/org/apache/lucene/index/TestIndexWriterDelete.java Tue Aug 26 22:16:32 2014
@@ -741,7 +741,7 @@ public class TestIndexWriterDelete exten
                 System.out.println("TEST: mock failure: now fail");
                 new Throwable().printStackTrace(System.out);
               }
-              throw new IOException("fail after applyDeletes");
+              throw new RuntimeException("fail after applyDeletes");
             }
           }
           if (!failed) {
@@ -845,7 +845,7 @@ public class TestIndexWriterDelete exten
         System.out.println("TEST: now commit for failure");
       }
       modifier.commit();
-    } catch (IOException ioe) {
+    } catch (RuntimeException ioe) {
       // expected
       failed = true;
     }