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/08/27 07:30:47 UTC

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

Author: rmuir
Date: Wed Aug 27 05:30:47 2014
New Revision: 1620779

URL: http://svn.apache.org/r1620779
Log:
LUCENE-5904: add tests for segment/gen inflation, fix gen inflation (it always parsed gen of 0), fix off-by-one in gen-inflation (was starting at 2 instead of 1), add robustness to trash files we might be looking at, add minor restriction to segment suffix names so we can always parse generations correctly from them

Added:
    lucene/dev/branches/lucene5904/lucene/core/src/test/org/apache/lucene/index/TestIndexFileNames.java   (with props)
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/SegmentWriteState.java
    lucene/dev/branches/lucene5904/lucene/core/src/test/org/apache/lucene/index/TestIndexFileDeleter.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=1620779&r1=1620778&r2=1620779&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 Wed Aug 27 05:30:47 2014
@@ -284,20 +284,28 @@ final class IndexFileDeleter implements 
       if (fileName.equals(IndexFileNames.SEGMENTS_GEN)) {
         // do nothing
       } else if (fileName.startsWith(IndexFileNames.SEGMENTS)) {
-        maxSegmentGen = Math.max(SegmentInfos.generationFromSegmentsFileName(fileName), maxSegmentGen);
+        try {
+          maxSegmentGen = Math.max(SegmentInfos.generationFromSegmentsFileName(fileName), maxSegmentGen);
+        } catch (NumberFormatException ignore) {
+          // trash file: we have to handle this since we allow anything starting with 'segments' here
+        }
       } 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;
+          curGen = 0L;
+        }
+
+        try {
+          curGen = Math.max(curGen, IndexFileNames.parseGeneration(fileName));
+        } catch (NumberFormatException ignore) {
+          // trash file: we have to handle this since codec regex is only so good
         }
-        maxPerSegmentGen.put(segmentName, Math.max(curGen, gen));
+        maxPerSegmentGen.put(segmentName, curGen);
       }
     }
 

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=1620779&r1=1620778&r2=1620779&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 Wed Aug 27 05:30:47 2014
@@ -176,15 +176,17 @@ public final class IndexFileNames {
   /** 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;
+    assert filename.startsWith("_");
+    String parts[] = stripExtension(filename).substring(1).split("_");
+    // 4 cases: 
+    // segment.ext
+    // segment_gen.ext
+    // segment_codec_suffix.ext
+    // segment_gen_codec_suffix.ext
+    if (parts.length == 2 || parts.length == 4) {
+      return Long.parseLong(parts[1], Character.MAX_RADIX);
     } else {
-      assert gen.startsWith("_");
-      return Long.parseLong(gen.substring(1), Character.MAX_RADIX);
+      return 0;
     }
   }
   

Modified: lucene/dev/branches/lucene5904/lucene/core/src/java/org/apache/lucene/index/SegmentWriteState.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene5904/lucene/core/src/java/org/apache/lucene/index/SegmentWriteState.java?rev=1620779&r1=1620778&r2=1620779&view=diff
==============================================================================
--- lucene/dev/branches/lucene5904/lucene/core/src/java/org/apache/lucene/index/SegmentWriteState.java (original)
+++ lucene/dev/branches/lucene5904/lucene/core/src/java/org/apache/lucene/index/SegmentWriteState.java Wed Aug 27 05:30:47 2014
@@ -65,7 +65,9 @@ public class SegmentWriteState {
    *  each of the postings formats it wraps.  If you create
    *  a new {@link PostingsFormat} then any files you
    *  write/read must be derived using this suffix (use
-   *  {@link IndexFileNames#segmentFileName(String,String,String)}). */
+   *  {@link IndexFileNames#segmentFileName(String,String,String)}).
+   *  
+   *  Note: the suffix must be either empty, or be a textual suffix contain exactly two parts (separated by underscore), or be a base36 generation. */
   public final String segmentSuffix;
   
   /** {@link IOContext} for all writes; you should pass this
@@ -91,6 +93,7 @@ public class SegmentWriteState {
     this.directory = directory;
     this.segmentInfo = segmentInfo;
     this.fieldInfos = fieldInfos;
+    assert assertSegmentSuffix(segmentSuffix);
     this.segmentSuffix = segmentSuffix;
     this.context = context;
   }
@@ -107,4 +110,23 @@ public class SegmentWriteState {
     delCountOnFlush = state.delCountOnFlush;
     liveDocs = state.liveDocs;
   }
+  
+  // currently only used by assert? clean up and make real check?
+  // either its a segment suffix (_X_Y) or its a parseable generation
+  // TODO: this is very confusing how ReadersAndUpdates passes generations via
+  // this mechanism, maybe add 'generation' explicitly to ctor create the 'actual suffix' here?
+  private boolean assertSegmentSuffix(String segmentSuffix) {
+    assert segmentSuffix != null;
+    if (!segmentSuffix.isEmpty()) {
+      int numParts = segmentSuffix.split("_").length;
+      if (numParts == 2) {
+        return true;
+      } else if (numParts == 1) {
+        Long.parseLong(segmentSuffix, Character.MAX_RADIX);
+        return true;
+      }
+      return false; // invalid
+    }
+    return true;
+  }
 }

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=1620779&r1=1620778&r2=1620779&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 Wed Aug 27 05:30:47 2014
@@ -286,4 +286,116 @@ public class TestIndexFileDeleter extend
 
     dir.close();
   }
+  
+  public void testSegmentNameInflation() throws IOException {
+    MockDirectoryWrapper dir = newMockDirectory();
+    
+    // empty commit
+    new IndexWriter(dir, new IndexWriterConfig(null)).close();   
+    
+    SegmentInfos sis = new SegmentInfos();
+    sis.read(dir);
+    assertEquals(0, sis.counter);
+    
+    // no inflation
+    IndexFileDeleter.inflateGens(sis, Arrays.asList(dir.listAll()), InfoStream.getDefault());
+    assertEquals(0, sis.counter);
+    
+    // add trash per-segment file
+    dir.createOutput(IndexFileNames.segmentFileName("_0", "", "foo"), IOContext.DEFAULT).close();
+    
+    // ensure inflation
+    IndexFileDeleter.inflateGens(sis, Arrays.asList(dir.listAll()), InfoStream.getDefault());
+    assertEquals(1, sis.counter);
+    
+    // add trash per-segment file
+    dir.createOutput(IndexFileNames.segmentFileName("_3", "", "foo"), IOContext.DEFAULT).close();
+    IndexFileDeleter.inflateGens(sis, Arrays.asList(dir.listAll()), InfoStream.getDefault());
+    assertEquals(4, sis.counter);
+    
+    // ensure we write _4 segment next
+    IndexWriter iw = new IndexWriter(dir, new IndexWriterConfig(null));
+    iw.addDocument(new Document());
+    iw.commit();
+    iw.close();
+    sis = new SegmentInfos();
+    sis.read(dir);
+    assertEquals("_4", sis.info(0).info.name);
+    assertEquals(5, sis.counter);
+    
+    dir.close();
+  }
+  
+  public void testGenerationInflation() throws IOException {
+    MockDirectoryWrapper dir = newMockDirectory();
+    
+    // initial commit
+    IndexWriter iw = new IndexWriter(dir, new IndexWriterConfig(null));
+    iw.addDocument(new Document());
+    iw.commit();
+    iw.close();   
+    
+    // no deletes: start at 1
+    SegmentInfos sis = new SegmentInfos();
+    sis.read(dir);
+    assertEquals(1, sis.info(0).getNextDelGen());
+    
+    // no inflation
+    IndexFileDeleter.inflateGens(sis, Arrays.asList(dir.listAll()), InfoStream.getDefault());
+    assertEquals(1, sis.info(0).getNextDelGen());
+    
+    // add trash per-segment deletes file
+    dir.createOutput(IndexFileNames.fileNameFromGeneration("_0", "del", 2), IOContext.DEFAULT).close();
+    
+    // ensure inflation
+    IndexFileDeleter.inflateGens(sis, Arrays.asList(dir.listAll()), InfoStream.getDefault());
+    assertEquals(3, sis.info(0).getNextDelGen());
+    
+    dir.close();
+  }
+  
+  public void testTrashyFile() throws IOException {
+    MockDirectoryWrapper dir = newMockDirectory();
+    dir.setCheckIndexOnClose(false); // TODO: maybe handle such trash better elsewhere...
+    
+    // empty commit
+    new IndexWriter(dir, new IndexWriterConfig(null)).close();   
+    
+    SegmentInfos sis = new SegmentInfos();
+    sis.read(dir);
+    assertEquals(1, sis.getGeneration());
+    
+    // add trash file
+    dir.createOutput(IndexFileNames.SEGMENTS + "_", IOContext.DEFAULT).close();
+    
+    // no inflation
+    IndexFileDeleter.inflateGens(sis, Arrays.asList(dir.listAll()), InfoStream.getDefault());
+    assertEquals(1, sis.getGeneration());
+
+    dir.close();
+  }
+  
+  public void testTrashyGenFile() throws IOException {
+    MockDirectoryWrapper dir = newMockDirectory();
+    
+    // initial commit
+    IndexWriter iw = new IndexWriter(dir, new IndexWriterConfig(null));
+    iw.addDocument(new Document());
+    iw.commit();
+    iw.close();   
+    
+    // no deletes: start at 1
+    SegmentInfos sis = new SegmentInfos();
+    sis.read(dir);
+    assertEquals(1, sis.info(0).getNextDelGen());
+    
+    // add trash file
+    dir.createOutput("_1_A", IOContext.DEFAULT).close();
+    
+    // no inflation
+    IndexFileDeleter.inflateGens(sis, Arrays.asList(dir.listAll()), InfoStream.getDefault());
+    assertEquals(1, sis.info(0).getNextDelGen());
+
+    dir.close();
+  }
 }

Added: lucene/dev/branches/lucene5904/lucene/core/src/test/org/apache/lucene/index/TestIndexFileNames.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene5904/lucene/core/src/test/org/apache/lucene/index/TestIndexFileNames.java?rev=1620779&view=auto
==============================================================================
--- lucene/dev/branches/lucene5904/lucene/core/src/test/org/apache/lucene/index/TestIndexFileNames.java (added)
+++ lucene/dev/branches/lucene5904/lucene/core/src/test/org/apache/lucene/index/TestIndexFileNames.java Wed Aug 27 05:30:47 2014
@@ -0,0 +1,34 @@
+package org.apache.lucene.index;
+
+/*
+ * 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 org.apache.lucene.util.LuceneTestCase;
+
+public class TestIndexFileNames extends LuceneTestCase {
+
+  public void testParseGeneration() {
+    // 1 part: segment (_1.fnm)
+    assertEquals(0, IndexFileNames.parseGeneration("_1.fnm"));
+    // 2 parts: segment + gen (_1_2.fnm)
+    assertEquals(2, IndexFileNames.parseGeneration("_1_2.fnm"));
+    // 3 parts: segment + suffix (_1_Lucene410_2.dvd)
+    assertEquals(0, IndexFileNames.parseGeneration("_1_Lucene410_2.dvd"));
+    // 4 parts: segment + suffix + gen (_1_2_Lucene410_3.dvm)
+    assertEquals(2, IndexFileNames.parseGeneration("_1_2_Lucene410_3.dvm"));
+  }
+}