You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@lucene.apache.org by sh...@apache.org on 2011/01/23 06:10:49 UTC

svn commit: r1062325 - in /lucene/dev/trunk/lucene: ./ src/java/org/apache/lucene/index/ src/java/org/apache/lucene/index/codecs/ src/java/org/apache/lucene/util/ src/test/org/apache/lucene/index/

Author: shaie
Date: Sun Jan 23 05:10:48 2011
New Revision: 1062325

URL: http://svn.apache.org/viewvc?rev=1062325&view=rev
Log:
LUCENE-2720: IndexWriter should throw IndexFormatTooOldExc on open, not later during optimize/getReader/close (trunk)

Modified:
    lucene/dev/trunk/lucene/CHANGES.txt
    lucene/dev/trunk/lucene/src/java/org/apache/lucene/index/FieldsReader.java
    lucene/dev/trunk/lucene/src/java/org/apache/lucene/index/IndexFormatTooOldException.java
    lucene/dev/trunk/lucene/src/java/org/apache/lucene/index/IndexWriter.java
    lucene/dev/trunk/lucene/src/java/org/apache/lucene/index/SegmentInfo.java
    lucene/dev/trunk/lucene/src/java/org/apache/lucene/index/SegmentReader.java
    lucene/dev/trunk/lucene/src/java/org/apache/lucene/index/codecs/DefaultSegmentInfosReader.java
    lucene/dev/trunk/lucene/src/java/org/apache/lucene/index/codecs/DefaultSegmentInfosWriter.java
    lucene/dev/trunk/lucene/src/java/org/apache/lucene/util/Constants.java
    lucene/dev/trunk/lucene/src/test/org/apache/lucene/index/TestBackwardsCompatibility.java
    lucene/dev/trunk/lucene/src/test/org/apache/lucene/index/index.31.cfs.zip
    lucene/dev/trunk/lucene/src/test/org/apache/lucene/index/index.31.nocfs.zip

Modified: lucene/dev/trunk/lucene/CHANGES.txt
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/CHANGES.txt?rev=1062325&r1=1062324&r2=1062325&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/CHANGES.txt (original)
+++ lucene/dev/trunk/lucene/CHANGES.txt Sun Jan 23 05:10:48 2011
@@ -140,6 +140,9 @@ Changes in Runtime Behavior
   documents that don't have the field get a norm byte value of 0. Previously, Lucene
   would populate "fake norms" with Similarity.getDefault() for these documents.
   (Robert Muir, Mike Mccandless)
+  
+* LUCENE-2720: IndexWriter throws IndexFormatTooOldException on open, rather 
+  than later when e.g. a merge starts. (Shai Erera, Mike McCandless, Uwe Schindler)
 
 API Changes
 

Modified: lucene/dev/trunk/lucene/src/java/org/apache/lucene/index/FieldsReader.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/src/java/org/apache/lucene/index/FieldsReader.java?rev=1062325&r1=1062324&r2=1062325&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/src/java/org/apache/lucene/index/FieldsReader.java (original)
+++ lucene/dev/trunk/lucene/src/java/org/apache/lucene/index/FieldsReader.java Sun Jan 23 05:10:48 2011
@@ -37,8 +37,10 @@ import java.io.Reader;
  * Class responsible for access to stored document fields.
  * <p/>
  * It uses &lt;segment&gt;.fdt and &lt;segment&gt;.fdx; files.
+ * 
+ * @lucene.internal
  */
-final class FieldsReader implements Cloneable {
+public final class FieldsReader implements Cloneable {
   private final static int FORMAT_SIZE = 4;
 
   private final FieldInfos fieldInfos;
@@ -74,6 +76,23 @@ final class FieldsReader implements Clon
     ensureOpen();
     return new FieldsReader(fieldInfos, numTotalDocs, size, format, docStoreOffset, cloneableFieldsStream, cloneableIndexStream);
   }
+
+  /** Verifies that the code version which wrote the segment is supported. */
+  public static void checkCodeVersion(Directory dir, String segment) throws IOException {
+    final String indexStreamFN = IndexFileNames.segmentFileName(segment, "", IndexFileNames.FIELDS_INDEX_EXTENSION);
+    IndexInput idxStream = dir.openInput(indexStreamFN, 1024);
+    
+    try {
+      int format = idxStream.readInt();
+      if (format < FieldsWriter.FORMAT_MINIMUM)
+        throw new IndexFormatTooOldException(indexStreamFN, format, FieldsWriter.FORMAT_MINIMUM, FieldsWriter.FORMAT_CURRENT);
+      if (format > FieldsWriter.FORMAT_CURRENT)
+        throw new IndexFormatTooNewException(indexStreamFN, format, FieldsWriter.FORMAT_MINIMUM, FieldsWriter.FORMAT_CURRENT);
+    } finally {
+      idxStream.close();
+    }
+  
+  }
   
   // Used only by clone
   private FieldsReader(FieldInfos fieldInfos, int numTotalDocs, int size, int format, int docStoreOffset,
@@ -89,11 +108,11 @@ final class FieldsReader implements Clon
     indexStream = (IndexInput) cloneableIndexStream.clone();
   }
   
-  FieldsReader(Directory d, String segment, FieldInfos fn) throws IOException {
+  public FieldsReader(Directory d, String segment, FieldInfos fn) throws IOException {
     this(d, segment, fn, BufferedIndexInput.BUFFER_SIZE, -1, 0);
   }
 
-  FieldsReader(Directory d, String segment, FieldInfos fn, int readBufferSize, int docStoreOffset, int size) throws IOException {
+  public FieldsReader(Directory d, String segment, FieldInfos fn, int readBufferSize, int docStoreOffset, int size) throws IOException {
     boolean success = false;
     isOriginal = true;
     try {
@@ -157,7 +176,7 @@ final class FieldsReader implements Clon
    *
    * @throws IOException
    */
-  final void close() throws IOException {
+  public final void close() throws IOException {
     if (!closed) {
       if (fieldsStream != null) {
         fieldsStream.close();
@@ -178,7 +197,7 @@ final class FieldsReader implements Clon
     }
   }
 
-  final int size() {
+  public final int size() {
     return size;
   }
 
@@ -186,7 +205,7 @@ final class FieldsReader implements Clon
     indexStream.seek(FORMAT_SIZE + (docID + docStoreOffset) * 8L);
   }
 
-  final Document doc(int n, FieldSelector fieldSelector) throws CorruptIndexException, IOException {
+  public final Document doc(int n, FieldSelector fieldSelector) throws CorruptIndexException, IOException {
     seekIndex(n);
     long position = indexStream.readLong();
     fieldsStream.seek(position);
@@ -237,7 +256,7 @@ final class FieldsReader implements Clon
    *  contiguous range of length numDocs starting with
    *  startDocID.  Returns the IndexInput (the fieldStream),
    *  already seeked to the starting point for startDocID.*/
-  final IndexInput rawDocs(int[] lengths, int startDocID, int numDocs) throws IOException {
+  public final IndexInput rawDocs(int[] lengths, int startDocID, int numDocs) throws IOException {
     seekIndex(startDocID);
     long startOffset = indexStream.readLong();
     long lastOffset = startOffset;

Modified: lucene/dev/trunk/lucene/src/java/org/apache/lucene/index/IndexFormatTooOldException.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/src/java/org/apache/lucene/index/IndexFormatTooOldException.java?rev=1062325&r1=1062324&r2=1062325&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/src/java/org/apache/lucene/index/IndexFormatTooOldException.java (original)
+++ lucene/dev/trunk/lucene/src/java/org/apache/lucene/index/IndexFormatTooOldException.java Sun Jan 23 05:10:48 2011
@@ -23,10 +23,15 @@ package org.apache.lucene.index;
  */
 public class IndexFormatTooOldException extends CorruptIndexException {
 
+  public IndexFormatTooOldException(String filename, String version) {
+    super("Format version is not supported" + (filename!=null ? (" in file '" + filename + "'") : "") +
+        ": " + version + ". This version of Lucene only supports indexes created with release 3.0 and later.");
+  }
+  
   public IndexFormatTooOldException(String filename, int version, int minVersion, int maxVersion) {
     super("Format version is not supported" + (filename!=null ? (" in file '" + filename + "'") : "") +
-      ": " + version + " (needs to be between " + minVersion + " and " + maxVersion +
-      "). This version of Lucene only supports indexes created with release 3.0 and later.");
+        ": " + version + " (needs to be between " + minVersion + " and " + maxVersion +
+    "). This version of Lucene only supports indexes created with release 3.0 and later.");
   }
 
 }

Modified: lucene/dev/trunk/lucene/src/java/org/apache/lucene/index/IndexWriter.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/src/java/org/apache/lucene/index/IndexWriter.java?rev=1062325&r1=1062324&r2=1062325&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/src/java/org/apache/lucene/index/IndexWriter.java (original)
+++ lucene/dev/trunk/lucene/src/java/org/apache/lucene/index/IndexWriter.java Sun Jan 23 05:10:48 2011
@@ -605,8 +605,6 @@ public class IndexWriter implements Clos
     }
   }
   
-  
-  
   /**
    * Obtain the number of deleted docs for a pooled reader.
    * If the reader isn't being pooled, the segmentInfo's 
@@ -715,11 +713,8 @@ public class IndexWriter implements Clos
 
     boolean success = false;
 
-    // TODO: we should check whether this index is too old,
-    // and throw an IndexFormatTooOldExc up front, here,
-    // instead of later when merge, applyDeletes, getReader
-    // is attempted.  I think to do this we should store the
-    // oldest segment's version in segments_N.
+    // If index is too old, reading the segments will throw
+    // IndexFormatTooOldException.
     segmentInfos = new SegmentInfos(codecs);
     try {
       if (create) {
@@ -982,6 +977,7 @@ public class IndexWriter implements Clos
    * @throws CorruptIndexException if the index is corrupt
    * @throws IOException if there is a low-level IO error
    */
+  @Override
   public void close() throws CorruptIndexException, IOException {
     close(true);
   }

Modified: lucene/dev/trunk/lucene/src/java/org/apache/lucene/index/SegmentInfo.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/src/java/org/apache/lucene/index/SegmentInfo.java?rev=1062325&r1=1062324&r2=1062325&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/src/java/org/apache/lucene/index/SegmentInfo.java (original)
+++ lucene/dev/trunk/lucene/src/java/org/apache/lucene/index/SegmentInfo.java Sun Jan 23 05:10:48 2011
@@ -20,6 +20,7 @@ package org.apache.lucene.index;
 import org.apache.lucene.store.Directory;
 import org.apache.lucene.store.IndexOutput;
 import org.apache.lucene.store.IndexInput;
+import org.apache.lucene.util.Constants;
 import org.apache.lucene.index.codecs.Codec;
 import org.apache.lucene.index.codecs.CodecProvider;
 import org.apache.lucene.index.codecs.DefaultSegmentInfosWriter;
@@ -87,6 +88,13 @@ public final class SegmentInfo {
 
   private Map<String,String> diagnostics;
 
+  // Tracks the Lucene version this segment was created with, since 3.1. Null 
+  // indicates an older than 3.0 index, and it's used to detect a too old index.
+  // The format expected is "x.y" - "2.x" for pre-3.0 indexes (or null), and 
+  // specific versions afterwards ("3.0", "3.1" etc.).
+  // see Constants.LUCENE_MAIN_VERSION.
+  private String version;
+  
   public SegmentInfo(String name, int docCount, Directory dir, boolean isCompoundFile,
                      boolean hasProx, SegmentCodecs segmentCodecs, boolean hasVectors) {
     this.name = name;
@@ -99,6 +107,7 @@ public final class SegmentInfo {
     this.segmentCodecs = segmentCodecs;
     this.hasVectors = hasVectors;
     delCount = 0;
+    version = Constants.LUCENE_MAIN_VERSION;
   }
 
   /**
@@ -106,6 +115,7 @@ public final class SegmentInfo {
    */
   void reset(SegmentInfo src) {
     clearFiles();
+    version = src.version;
     name = src.name;
     docCount = src.docCount;
     dir = src.dir;
@@ -145,6 +155,9 @@ public final class SegmentInfo {
    */
   public SegmentInfo(Directory dir, int format, IndexInput input, CodecProvider codecs) throws IOException {
     this.dir = dir;
+    if (format <= DefaultSegmentInfosWriter.FORMAT_3_1) {
+      version = input.readString();
+    }
     name = input.readString();
     docCount = input.readInt();
     delGen = input.readLong();
@@ -293,6 +306,7 @@ public final class SegmentInfo {
       si.normGen = normGen.clone();
     }
     si.hasVectors = hasVectors;
+    si.version = version;
     return si;
   }
 
@@ -433,6 +447,8 @@ public final class SegmentInfo {
   public void write(IndexOutput output)
     throws IOException {
     assert delCount <= docCount: "delCount=" + delCount + " docCount=" + docCount + " segment=" + name;
+    // Write the Lucene version that created this segment, since 3.1
+    output.writeString(version);
     output.writeString(name);
     output.writeInt(docCount);
     output.writeLong(delGen);
@@ -574,8 +590,9 @@ public final class SegmentInfo {
   /** Used for debugging.  Format may suddenly change.
    * 
    *  <p>Current format looks like
-   *  <code>_a:c45/4->_1</code>, which means the segment's
-   *  name is <code>_a</code>; it's using compound file
+   *  <code>_a(3.1):c45/4->_1</code>, which means the segment's
+   *  name is <code>_a</code>; it was created with Lucene 3.1 (or
+   *  '?' if it's unkown); it's using compound file
    *  format (would be <code>C</code> if not compound); it
    *  has 45 documents; it has 4 deletions (this part is
    *  left off when there are no deletions); it's using the
@@ -585,7 +602,7 @@ public final class SegmentInfo {
   public String toString(Directory dir, int pendingDelCount) {
 
     StringBuilder s = new StringBuilder();
-    s.append(name).append(':');
+    s.append(name).append('(').append(version == null ? "?" : version).append(')').append(':');
 
     char cfs = getUseCompoundFile() ? 'c' : 'C';
     s.append(cfs);
@@ -633,4 +650,25 @@ public final class SegmentInfo {
   public int hashCode() {
     return dir.hashCode() + name.hashCode();
   }
+
+  /**
+   * Used by DefaultSegmentInfosReader to upgrade a 3.0 segment to record its
+   * version is "3.0". This method can be removed when we're not required to
+   * support 3x indexes anymore, e.g. in 5.0.
+   * <p>
+   * <b>NOTE:</b> this method is used for internal purposes only - you should
+   * not modify the version of a SegmentInfo, or it may result in unexpected
+   * exceptions thrown when you attempt to open the index.
+   * 
+   * @lucene.internal
+   */
+  public void setVersion(String version) {
+    this.version = version;
+  }
+  
+  /** Returns the version of the code which wrote the segment. */
+  public String getVersion() {
+    return version;
+  }
+  
 }

Modified: lucene/dev/trunk/lucene/src/java/org/apache/lucene/index/SegmentReader.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/src/java/org/apache/lucene/index/SegmentReader.java?rev=1062325&r1=1062324&r2=1062325&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/src/java/org/apache/lucene/index/SegmentReader.java (original)
+++ lucene/dev/trunk/lucene/src/java/org/apache/lucene/index/SegmentReader.java Sun Jan 23 05:10:48 2011
@@ -226,6 +226,7 @@ public class SegmentReader extends Index
           assert storeDir != null;
         }
 
+        // nocommit: this can be simplified to always be si.getDocStoreSegment()
         final String storesSegment;
         if (si.getDocStoreOffset() != -1) {
           storesSegment = si.getDocStoreSegment();

Modified: lucene/dev/trunk/lucene/src/java/org/apache/lucene/index/codecs/DefaultSegmentInfosReader.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/src/java/org/apache/lucene/index/codecs/DefaultSegmentInfosReader.java?rev=1062325&r1=1062324&r2=1062325&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/src/java/org/apache/lucene/index/codecs/DefaultSegmentInfosReader.java (original)
+++ lucene/dev/trunk/lucene/src/java/org/apache/lucene/index/codecs/DefaultSegmentInfosReader.java Sun Jan 23 05:10:48 2011
@@ -19,7 +19,10 @@ package org.apache.lucene.index.codecs;
 
 import java.io.IOException;
 
+import org.apache.lucene.index.CompoundFileReader;
 import org.apache.lucene.index.CorruptIndexException;
+import org.apache.lucene.index.FieldsReader;
+import org.apache.lucene.index.IndexFileNames;
 import org.apache.lucene.index.IndexFormatTooOldException;
 import org.apache.lucene.index.IndexFormatTooNewException;
 import org.apache.lucene.index.SegmentInfo;
@@ -55,7 +58,41 @@ public class DefaultSegmentInfosReader e
       infos.counter = input.readInt(); // read counter
   
       for (int i = input.readInt(); i > 0; i--) { // read segmentInfos
-        infos.add(new SegmentInfo(directory, format, input, codecs));
+        SegmentInfo si = new SegmentInfo(directory, format, input, codecs);
+        if (si.getVersion() == null) {
+          // Could be a 3.0 - try to open the doc stores - if it fails, it's a
+          // 2.x segment, and an IndexFormatTooOldException will be thrown,
+          // which is what we want.
+          Directory dir = directory;
+          if (si.getDocStoreOffset() != -1) {
+            if (si.getDocStoreIsCompoundFile()) {
+              dir = new CompoundFileReader(dir, IndexFileNames.segmentFileName(
+                  si.getDocStoreSegment(), "",
+                  IndexFileNames.COMPOUND_FILE_STORE_EXTENSION), 1024);
+            }
+          } else if (si.getUseCompoundFile()) {
+            dir = new CompoundFileReader(dir, IndexFileNames.segmentFileName(
+                si.name, "", IndexFileNames.COMPOUND_FILE_EXTENSION), 1024);
+          }
+
+          try {
+            FieldsReader.checkCodeVersion(dir, si.getDocStoreSegment());
+          } finally {
+            // If we opened the directory, close it
+            if (dir != directory) dir.close();
+          }
+          
+          // Above call succeeded, so it's a 3.0 segment. Upgrade it so the next
+          // time the segment is read, its version won't be null and we won't
+          // need to open FieldsReader every time for each such segment.
+          si.setVersion("3.0");
+        } else if (si.getVersion().equals("2.x")) {
+          // If it's a 3x index touched by 3.1+ code, then segments record their
+          // version, whether they are 2.x ones or not. We detect that and throw
+          // appropriate exception.
+          throw new IndexFormatTooOldException(si.name, si.getVersion());
+        }
+        infos.add(si);
       }
       
       infos.userData = input.readStringStringMap();

Modified: lucene/dev/trunk/lucene/src/java/org/apache/lucene/index/codecs/DefaultSegmentInfosWriter.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/src/java/org/apache/lucene/index/codecs/DefaultSegmentInfosWriter.java?rev=1062325&r1=1062324&r2=1062325&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/src/java/org/apache/lucene/index/codecs/DefaultSegmentInfosWriter.java (original)
+++ lucene/dev/trunk/lucene/src/java/org/apache/lucene/index/codecs/DefaultSegmentInfosWriter.java Sun Jan 23 05:10:48 2011
@@ -38,9 +38,12 @@ public class DefaultSegmentInfosWriter e
   /** Each segment records whether it has term vectors */
   public static final int FORMAT_HAS_VECTORS = -10;
 
+  /** Each segment records the Lucene version that created it. */
+  public static final int FORMAT_3_1 = -11;
+
   /** Each segment records whether its postings are written
    *  in the new flex format */
-  public static final int FORMAT_4_0 = -11;
+  public static final int FORMAT_4_0 = -12;
 
   /** This must always point to the most recent file format.
    * whenever you add a new format, make it 1 smaller (negative version logic)! */

Modified: lucene/dev/trunk/lucene/src/java/org/apache/lucene/util/Constants.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/src/java/org/apache/lucene/util/Constants.java?rev=1062325&r1=1062324&r2=1062325&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/src/java/org/apache/lucene/util/Constants.java (original)
+++ lucene/dev/trunk/lucene/src/java/org/apache/lucene/util/Constants.java Sun Jan 23 05:10:48 2011
@@ -70,6 +70,9 @@ public final class Constants {
     return s.toString();
   }
   
+  // NOTE: we track per-segment version as a String with the "X.Y" format, e.g.
+  // "4.0", "3.1", "3.0". Therefore when we change this constant, we should keep
+  // the format.
   public static final String LUCENE_MAIN_VERSION = ident("4.0");
 
   public static final String LUCENE_VERSION;

Modified: lucene/dev/trunk/lucene/src/test/org/apache/lucene/index/TestBackwardsCompatibility.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/src/test/org/apache/lucene/index/TestBackwardsCompatibility.java?rev=1062325&r1=1062324&r2=1062325&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/src/test/org/apache/lucene/index/TestBackwardsCompatibility.java (original)
+++ lucene/dev/trunk/lucene/src/test/org/apache/lucene/index/TestBackwardsCompatibility.java Sun Jan 23 05:10:48 2011
@@ -171,15 +171,7 @@ public class TestBackwardsCompatibility 
 
       try {
         writer = new IndexWriter(dir, newIndexWriterConfig(
-          TEST_VERSION_CURRENT, new MockAnalyzer())
-          .setMergeScheduler(new SerialMergeScheduler()) // no threads!
-        );
-        // TODO: Make IndexWriter fail on open!
-        if (random.nextBoolean()) {
-          writer.optimize();
-        } else {
-          reader = writer.getReader();
-        }
+          TEST_VERSION_CURRENT, new MockAnalyzer()));
         fail("IndexWriter creation should not pass for "+unsupportedNames[i]);
       } catch (IndexFormatTooOldException e) {
         // pass
@@ -188,17 +180,13 @@ public class TestBackwardsCompatibility 
           e.printStackTrace(System.out);
         }
       } finally {
-        if (reader != null) reader.close();
-        reader = null;
+        // we should fail to open IW, and so it should be null when we get here.
+        // However, if the test fails (i.e., IW did not fail on open), we need
+        // to close IW. However, if merges are run, IW may throw
+        // IndexFormatTooOldException, and we don't want to mask the fail()
+        // above, so close without waiting for merges.
         if (writer != null) {
-          try {
-            writer.close();
-          } catch (IndexFormatTooOldException e) {
-            // OK -- since IW gives merge scheduler a chance
-            // to merge at close, it's possible and fine to
-            // hit this exc here
-            writer.close(false);
-          }
+          writer.close(false);
         }
         writer = null;
       }

Modified: lucene/dev/trunk/lucene/src/test/org/apache/lucene/index/index.31.cfs.zip
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/src/test/org/apache/lucene/index/index.31.cfs.zip?rev=1062325&r1=1062324&r2=1062325&view=diff
==============================================================================
Binary files - no diff available.

Modified: lucene/dev/trunk/lucene/src/test/org/apache/lucene/index/index.31.nocfs.zip
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/src/test/org/apache/lucene/index/index.31.nocfs.zip?rev=1062325&r1=1062324&r2=1062325&view=diff
==============================================================================
Binary files - no diff available.



Re: svn commit: r1062325 - in /lucene/dev/trunk/lucene: ./ src/java/org/apache/lucene/index/ src/java/org/apache/lucene/index/codecs/ src/java/org/apache/lucene/util/ src/test/org/apache/lucene/index/

Posted by Shai Erera <se...@gmail.com>.
Sorry, I intended to take care of it before commit, but forgot :).

I'll fix it now.

Shai

On Sun, Jan 23, 2011 at 11:27 AM, Uwe Schindler <uw...@thetaphi.de> wrote:

> Hi Shai,
>
> The Hudson build failures are caused  by "nocommits" -- this time not the
> Hudson bug (see recent build logs):
>
> > lucene/dev/trunk/lucene/src/java/org/apache/lucene/index/SegmentRead
> > er.java (original)
> > +++
> > lucene/dev/trunk/lucene/src/java/org/apache/lucene/index/SegmentRead
> > +++ er.java Sun Jan 23 05:10:48 2011
> > @@ -226,6 +226,7 @@ public class SegmentReader extends Index
> >            assert storeDir != null;
> >          }
> >
> > +        // nocommit: this can be simplified to always be
> > + si.getDocStoreSegment()
> >          final String storesSegment;
> >          if (si.getDocStoreOffset() != -1) {
> >            storesSegment = si.getDocStoreSegment();
>
> Uwe
>
>
> ---------------------------------------------------------------------
> To unsubscribe, e-mail: dev-unsubscribe@lucene.apache.org
> For additional commands, e-mail: dev-help@lucene.apache.org
>
>

RE: svn commit: r1062325 - in /lucene/dev/trunk/lucene: ./ src/java/org/apache/lucene/index/ src/java/org/apache/lucene/index/codecs/ src/java/org/apache/lucene/util/ src/test/org/apache/lucene/index/

Posted by Uwe Schindler <uw...@thetaphi.de>.
Hi Shai,

The Hudson build failures are caused  by "nocommits" -- this time not the Hudson bug (see recent build logs):

> lucene/dev/trunk/lucene/src/java/org/apache/lucene/index/SegmentRead
> er.java (original)
> +++
> lucene/dev/trunk/lucene/src/java/org/apache/lucene/index/SegmentRead
> +++ er.java Sun Jan 23 05:10:48 2011
> @@ -226,6 +226,7 @@ public class SegmentReader extends Index
>            assert storeDir != null;
>          }
> 
> +        // nocommit: this can be simplified to always be
> + si.getDocStoreSegment()
>          final String storesSegment;
>          if (si.getDocStoreOffset() != -1) {
>            storesSegment = si.getDocStoreSegment();

Uwe


---------------------------------------------------------------------
To unsubscribe, e-mail: dev-unsubscribe@lucene.apache.org
For additional commands, e-mail: dev-help@lucene.apache.org