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 2011/11/04 18:48:12 UTC

svn commit: r1197690 - in /lucene/dev/trunk: lucene/contrib/misc/src/java/org/apache/lucene/store/ lucene/src/java/org/apache/lucene/index/ lucene/src/java/org/apache/lucene/index/codecs/ lucene/src/java/org/apache/lucene/index/codecs/lucene3x/ lucene/...

Author: mikemccand
Date: Fri Nov  4 17:48:11 2011
New Revision: 1197690

URL: http://svn.apache.org/viewvc?rev=1197690&view=rev
Log:
LUCENE-3539: try to include resource desc (file path for Dir impls based on file system) when we throw IOExc, for better transparency/debugging

Modified:
    lucene/dev/trunk/lucene/contrib/misc/src/java/org/apache/lucene/store/DirectIOLinuxDirectory.java
    lucene/dev/trunk/lucene/contrib/misc/src/java/org/apache/lucene/store/WindowsDirectory.java
    lucene/dev/trunk/lucene/src/java/org/apache/lucene/index/FieldInfos.java
    lucene/dev/trunk/lucene/src/java/org/apache/lucene/index/IndexFormatTooNewException.java
    lucene/dev/trunk/lucene/src/java/org/apache/lucene/index/IndexFormatTooOldException.java
    lucene/dev/trunk/lucene/src/java/org/apache/lucene/index/PrefixCodedTerms.java
    lucene/dev/trunk/lucene/src/java/org/apache/lucene/index/SegmentInfos.java
    lucene/dev/trunk/lucene/src/java/org/apache/lucene/index/TermVectorsReader.java
    lucene/dev/trunk/lucene/src/java/org/apache/lucene/index/codecs/DefaultFieldsReader.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/MultiLevelSkipListReader.java
    lucene/dev/trunk/lucene/src/java/org/apache/lucene/index/codecs/lucene3x/SegmentTermEnum.java
    lucene/dev/trunk/lucene/src/java/org/apache/lucene/index/codecs/lucene40/Lucene40PostingsWriter.java
    lucene/dev/trunk/lucene/src/java/org/apache/lucene/index/codecs/sep/SepPostingsWriter.java
    lucene/dev/trunk/lucene/src/java/org/apache/lucene/store/BufferedIndexInput.java
    lucene/dev/trunk/lucene/src/java/org/apache/lucene/store/ChecksumIndexInput.java
    lucene/dev/trunk/lucene/src/java/org/apache/lucene/store/CompoundFileDirectory.java
    lucene/dev/trunk/lucene/src/java/org/apache/lucene/store/Directory.java
    lucene/dev/trunk/lucene/src/java/org/apache/lucene/store/IndexInput.java
    lucene/dev/trunk/lucene/src/java/org/apache/lucene/store/MMapDirectory.java
    lucene/dev/trunk/lucene/src/java/org/apache/lucene/store/NIOFSDirectory.java
    lucene/dev/trunk/lucene/src/java/org/apache/lucene/store/RAMDirectory.java
    lucene/dev/trunk/lucene/src/java/org/apache/lucene/store/RAMInputStream.java
    lucene/dev/trunk/lucene/src/java/org/apache/lucene/store/SimpleFSDirectory.java
    lucene/dev/trunk/lucene/src/java/org/apache/lucene/util/CodecUtil.java
    lucene/dev/trunk/lucene/src/test-framework/java/org/apache/lucene/index/MockIndexInput.java
    lucene/dev/trunk/lucene/src/test-framework/java/org/apache/lucene/store/MockDirectoryWrapper.java
    lucene/dev/trunk/lucene/src/test-framework/java/org/apache/lucene/store/MockIndexInputWrapper.java
    lucene/dev/trunk/lucene/src/test/org/apache/lucene/index/TestBackwardsCompatibility.java
    lucene/dev/trunk/lucene/src/test/org/apache/lucene/index/TestFieldsReader.java
    lucene/dev/trunk/lucene/src/test/org/apache/lucene/index/TestLazyProxSkipping.java
    lucene/dev/trunk/lucene/src/test/org/apache/lucene/index/TestMultiLevelSkipList.java
    lucene/dev/trunk/lucene/src/test/org/apache/lucene/store/TestBufferedIndexInput.java
    lucene/dev/trunk/lucene/src/test/org/apache/lucene/store/TestHugeRamFile.java
    lucene/dev/trunk/modules/facet/src/test/org/apache/lucene/util/SlowRAMDirectory.java

Modified: lucene/dev/trunk/lucene/contrib/misc/src/java/org/apache/lucene/store/DirectIOLinuxDirectory.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/contrib/misc/src/java/org/apache/lucene/store/DirectIOLinuxDirectory.java?rev=1197690&r1=1197689&r2=1197690&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/contrib/misc/src/java/org/apache/lucene/store/DirectIOLinuxDirectory.java (original)
+++ lucene/dev/trunk/lucene/contrib/misc/src/java/org/apache/lucene/store/DirectIOLinuxDirectory.java Fri Nov  4 17:48:11 2011
@@ -245,6 +245,7 @@ public class DirectIOLinuxDirectory exte
 
     public DirectIOLinuxIndexInput(File path, int bufferSize) throws IOException {
       // TODO make use of IOContext
+      super("DirectIOLinuxIndexInput(path=\"" + path.getPath() + "\")");
       FileDescriptor fd = NativePosixUtil.open_direct(path.toString(), true);
       fis = new FileInputStream(fd);
       channel = fis.getChannel();
@@ -259,6 +260,7 @@ public class DirectIOLinuxDirectory exte
 
     // for clone
     public DirectIOLinuxIndexInput(DirectIOLinuxIndexInput other) throws IOException {
+      super(other.toString());
       this.fis = null;
       channel = other.channel;
       this.bufferSize = other.bufferSize;
@@ -308,7 +310,7 @@ public class DirectIOLinuxDirectory exte
       try {
         return channel.size();
       } catch (IOException ioe) {
-        throw new RuntimeException(ioe);
+        throw new RuntimeException("IOException during length(): " + this, ioe);
       }
     }
 
@@ -331,9 +333,14 @@ public class DirectIOLinuxDirectory exte
       bufferPos = 0;
       assert (filePos & ALIGN_NOT_MASK) == filePos : "filePos=" + filePos + " anded=" + (filePos & ALIGN_NOT_MASK);
       //System.out.println("X refill filePos=" + filePos);
-      int n = channel.read(buffer, filePos);
+      int n;
+      try {
+        n = channel.read(buffer, filePos);
+      } catch (IOException ioe) {
+        throw new IOException(ioe.getMessage() + ": " + this, ioe);
+      }
       if (n < 0) {
-        throw new IOException("eof");
+        throw new IOException("eof: " + this);
       }
       buffer.rewind();
     }
@@ -365,7 +372,7 @@ public class DirectIOLinuxDirectory exte
       try {
         return new DirectIOLinuxIndexInput(this);
       } catch (IOException ioe) {
-        throw new RuntimeException(ioe);
+        throw new RuntimeException("IOException during clone: " + this, ioe);
       }
     }
   }

Modified: lucene/dev/trunk/lucene/contrib/misc/src/java/org/apache/lucene/store/WindowsDirectory.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/contrib/misc/src/java/org/apache/lucene/store/WindowsDirectory.java?rev=1197690&r1=1197689&r2=1197690&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/contrib/misc/src/java/org/apache/lucene/store/WindowsDirectory.java (original)
+++ lucene/dev/trunk/lucene/contrib/misc/src/java/org/apache/lucene/store/WindowsDirectory.java Fri Nov  4 17:48:11 2011
@@ -19,6 +19,7 @@ package org.apache.lucene.store;
 
 import java.io.File;
 import java.io.IOException;
+import java.io.EOFException;
 
 import org.apache.lucene.store.Directory; // javadoc
 import org.apache.lucene.store.NativeFSLockFactory; // javadoc
@@ -80,7 +81,7 @@ public class WindowsDirectory extends FS
     boolean isOpen;
     
     public WindowsIndexInput(File file, int bufferSize) throws IOException {
-      super(bufferSize);
+      super("WindowsIndexInput(path=\"" + file.getPath() + "\")", bufferSize);
       fd = WindowsDirectory.open(file.getPath());
       length = WindowsDirectory.length(fd);
       isOpen = true;
@@ -88,8 +89,16 @@ public class WindowsDirectory extends FS
     
     @Override
     protected void readInternal(byte[] b, int offset, int length) throws IOException {
-      if (WindowsDirectory.read(fd, b, offset, length, getFilePointer()) != length)
-        throw new IOException("Read past EOF");
+      int bytesRead;
+      try {
+        bytesRead = WindowsDirectory.read(fd, b, offset, length, getFilePointer());
+      } catch (IOException ioe) {
+        throw new IOException(ioe.getMessage() + ": " + this, ioe);
+      }
+
+      if (bytesRead != length) {
+        throw new EOFException("Read past EOF: " + this);
+      }
     }
 
     @Override

Modified: lucene/dev/trunk/lucene/src/java/org/apache/lucene/index/FieldInfos.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/src/java/org/apache/lucene/index/FieldInfos.java?rev=1197690&r1=1197689&r2=1197690&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/src/java/org/apache/lucene/index/FieldInfos.java (original)
+++ lucene/dev/trunk/lucene/src/java/org/apache/lucene/index/FieldInfos.java Fri Nov  4 17:48:11 2011
@@ -650,10 +650,10 @@ public final class FieldInfos implements
     format = input.readVInt();
 
     if (format > FORMAT_MINIMUM) {
-      throw new IndexFormatTooOldException(fileName, format, FORMAT_MINIMUM, FORMAT_CURRENT);
+      throw new IndexFormatTooOldException(input, format, FORMAT_MINIMUM, FORMAT_CURRENT);
     }
     if (format < FORMAT_CURRENT) {
-      throw new IndexFormatTooNewException(fileName, format, FORMAT_MINIMUM, FORMAT_CURRENT);
+      throw new IndexFormatTooNewException(input, format, FORMAT_MINIMUM, FORMAT_CURRENT);
     }
 
     final int size = input.readVInt(); //read in the size
@@ -675,7 +675,7 @@ public final class FieldInfos implements
         if (format <= FORMAT_OMIT_POSITIONS) {
           indexOptions = IndexOptions.DOCS_AND_FREQS;
         } else {
-          throw new CorruptIndexException("Corrupt fieldinfos, OMIT_POSITIONS set but format=" + format);
+          throw new CorruptIndexException("Corrupt fieldinfos, OMIT_POSITIONS set but format=" + format + " (resource: " + input + ")");
         }
       } else {
         indexOptions = IndexOptions.DOCS_AND_FREQS_AND_POSITIONS;
@@ -745,7 +745,7 @@ public final class FieldInfos implements
     }
 
     if (input.getFilePointer() != input.length()) {
-      throw new CorruptIndexException("did not read all bytes from file \"" + fileName + "\": read " + input.getFilePointer() + " vs size " + input.length());
+      throw new CorruptIndexException("did not read all bytes from file \"" + fileName + "\": read " + input.getFilePointer() + " vs size " + input.length() + " (resource: " + input + ")");
     }    
   }
   

Modified: lucene/dev/trunk/lucene/src/java/org/apache/lucene/index/IndexFormatTooNewException.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/src/java/org/apache/lucene/index/IndexFormatTooNewException.java?rev=1197690&r1=1197689&r2=1197690&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/src/java/org/apache/lucene/index/IndexFormatTooNewException.java (original)
+++ lucene/dev/trunk/lucene/src/java/org/apache/lucene/index/IndexFormatTooNewException.java Fri Nov  4 17:48:11 2011
@@ -17,15 +17,24 @@
 
 package org.apache.lucene.index;
 
+import org.apache.lucene.store.DataInput;
+
 /**
  * This exception is thrown when Lucene detects
  * an index that is newer than this Lucene version.
  */
 public class IndexFormatTooNewException extends CorruptIndexException {
 
-  public IndexFormatTooNewException(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 + ")");
+  /** @lucene.internal */
+  public IndexFormatTooNewException(String resourceDesc, int version, int minVersion, int maxVersion) {
+    super("Format version is not supported (resource: " + resourceDesc + "): "
+      + version + " (needs to be between " + minVersion + " and " + maxVersion + ")");
+    assert resourceDesc != null;
+  }
+
+  /** @lucene.internal */
+  public IndexFormatTooNewException(DataInput in, int version, int minVersion, int maxVersion) {
+    this(in.toString(), version, minVersion, maxVersion);
   }
 
 }

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=1197690&r1=1197689&r2=1197690&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 Fri Nov  4 17:48:11 2011
@@ -17,21 +17,36 @@
 
 package org.apache.lucene.index;
 
+import org.apache.lucene.store.DataInput;
+
 /**
  * This exception is thrown when Lucene detects
  * an index that is too old for this Lucene version
  */
 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.");
+  /** @lucene.internal */
+  public IndexFormatTooOldException(String resourceDesc, String version) {
+    super("Format version is not supported (resource: " + resourceDesc + "): " +
+        version + ". This version of Lucene only supports indexes created with release 3.0 and later.");
+    assert resourceDesc != null;
+  }
+
+  /** @lucene.internal */
+  public IndexFormatTooOldException(DataInput in, String version) {
+    this(in.toString(), version);
   }
   
-  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 +
+  /** @lucene.internal */
+  public IndexFormatTooOldException(String resourceDesc, int version, int minVersion, int maxVersion) {
+    super("Format version is not supported (resource: " + resourceDesc + "): " +
+        version + " (needs to be between " + minVersion + " and " + maxVersion +
     "). This version of Lucene only supports indexes created with release 3.0 and later.");
+    assert resourceDesc != null;
   }
 
+  /** @lucene.internal */
+  public IndexFormatTooOldException(DataInput in, int version, int minVersion, int maxVersion) {
+    this(in.toString(), version, minVersion, maxVersion);
+  }
 }

Modified: lucene/dev/trunk/lucene/src/java/org/apache/lucene/index/PrefixCodedTerms.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/src/java/org/apache/lucene/index/PrefixCodedTerms.java?rev=1197690&r1=1197689&r2=1197690&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/src/java/org/apache/lucene/index/PrefixCodedTerms.java (original)
+++ lucene/dev/trunk/lucene/src/java/org/apache/lucene/index/PrefixCodedTerms.java Fri Nov  4 17:48:11 2011
@@ -20,7 +20,6 @@ package org.apache.lucene.index;
 import java.io.IOException;
 import java.util.Iterator;
 
-import org.apache.lucene.index.Term;
 import org.apache.lucene.store.IndexInput;
 import org.apache.lucene.store.RAMFile;
 import org.apache.lucene.store.RAMInputStream;
@@ -56,7 +55,7 @@ class PrefixCodedTerms implements Iterab
 
     PrefixCodedTermsIterator() {
       try {
-        input = new RAMInputStream(buffer);
+        input = new RAMInputStream("PrefixCodedTermsIterator", buffer);
       } catch (IOException e) {
         throw new RuntimeException(e);
       }

Modified: lucene/dev/trunk/lucene/src/java/org/apache/lucene/index/SegmentInfos.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/src/java/org/apache/lucene/index/SegmentInfos.java?rev=1197690&r1=1197689&r2=1197690&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/src/java/org/apache/lucene/index/SegmentInfos.java (original)
+++ lucene/dev/trunk/lucene/src/java/org/apache/lucene/index/SegmentInfos.java Fri Nov  4 17:48:11 2011
@@ -255,10 +255,10 @@ public final class SegmentInfos implemen
     
       // check that it is a format we can understand
       if (format > DefaultSegmentInfosWriter.FORMAT_MINIMUM)
-        throw new IndexFormatTooOldException(segmentFileName, format,
+        throw new IndexFormatTooOldException(input, format,
           DefaultSegmentInfosWriter.FORMAT_MINIMUM, DefaultSegmentInfosWriter.FORMAT_CURRENT);
       if (format < DefaultSegmentInfosWriter.FORMAT_CURRENT)
-        throw new IndexFormatTooNewException(segmentFileName, format,
+        throw new IndexFormatTooNewException(input, format,
           DefaultSegmentInfosWriter.FORMAT_MINIMUM, DefaultSegmentInfosWriter.FORMAT_CURRENT);
 
       if (format <= DefaultSegmentInfosWriter.FORMAT_4_0) {
@@ -271,7 +271,7 @@ public final class SegmentInfos implemen
       final long checksumNow = input.getChecksum();
       final long checksumThen = input.readLong();
       if (checksumNow != checksumThen)
-        throw new CorruptIndexException("checksum mismatch in segments file");
+        throw new CorruptIndexException("checksum mismatch in segments file (resource: " + input + ")");
       success = true;
     }
     finally {
@@ -634,8 +634,7 @@ public final class SegmentInfos implemen
                   }
                 } else {
                   /* TODO: Investigate this! 
-                  throw new IndexFormatTooNewException("segments.gen version number invalid: " + version +
-                    " (must be " + FORMAT_SEGMENTS_GEN_CURRENT + ")");
+                  throw new IndexFormatTooNewException(genInput, version, FORMAT_SEGMENTS_GEN_CURRENT, FORMAT_SEGMENTS_GEN_CURRENT);
                   */
                 }
               } catch (IOException err2) {

Modified: lucene/dev/trunk/lucene/src/java/org/apache/lucene/index/TermVectorsReader.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/src/java/org/apache/lucene/index/TermVectorsReader.java?rev=1197690&r1=1197689&r2=1197690&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/src/java/org/apache/lucene/index/TermVectorsReader.java (original)
+++ lucene/dev/trunk/lucene/src/java/org/apache/lucene/index/TermVectorsReader.java Fri Nov  4 17:48:11 2011
@@ -17,12 +17,9 @@ package org.apache.lucene.index;
  * limitations under the License.
  */
 
-import org.apache.lucene.index.MergePolicy.OneMerge;
-import org.apache.lucene.store.BufferedIndexInput;
 import org.apache.lucene.store.Directory;
 import org.apache.lucene.store.IOContext;
 import org.apache.lucene.store.IndexInput;
-import org.apache.lucene.store.IOContext.Context;
 import org.apache.lucene.util.ArrayUtil;
 import org.apache.lucene.util.BytesRef;
 import org.apache.lucene.util.IOUtils;
@@ -186,9 +183,9 @@ class TermVectorsReader implements Clone
   {
     int format = in.readInt();
     if (format < FORMAT_MINIMUM)
-      throw new IndexFormatTooOldException(fn, format, FORMAT_MINIMUM, FORMAT_CURRENT);
+      throw new IndexFormatTooOldException(in, format, FORMAT_MINIMUM, FORMAT_CURRENT);
     if (format > FORMAT_CURRENT)
-      throw new IndexFormatTooNewException(fn, format, FORMAT_MINIMUM, FORMAT_CURRENT);
+      throw new IndexFormatTooNewException(in, format, FORMAT_MINIMUM, FORMAT_CURRENT);
     return format;
   }
 

Modified: lucene/dev/trunk/lucene/src/java/org/apache/lucene/index/codecs/DefaultFieldsReader.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/src/java/org/apache/lucene/index/codecs/DefaultFieldsReader.java?rev=1197690&r1=1197689&r2=1197690&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/src/java/org/apache/lucene/index/codecs/DefaultFieldsReader.java (original)
+++ lucene/dev/trunk/lucene/src/java/org/apache/lucene/index/codecs/DefaultFieldsReader.java Fri Nov  4 17:48:11 2011
@@ -88,9 +88,9 @@ public final class DefaultFieldsReader e
     try {
       int format = idxStream.readInt();
       if (format < DefaultFieldsWriter.FORMAT_MINIMUM)
-        throw new IndexFormatTooOldException(indexStreamFN, format, DefaultFieldsWriter.FORMAT_MINIMUM, DefaultFieldsWriter.FORMAT_CURRENT);
+        throw new IndexFormatTooOldException(idxStream, format, DefaultFieldsWriter.FORMAT_MINIMUM, DefaultFieldsWriter.FORMAT_CURRENT);
       if (format > DefaultFieldsWriter.FORMAT_CURRENT)
-        throw new IndexFormatTooNewException(indexStreamFN, format, DefaultFieldsWriter.FORMAT_MINIMUM, DefaultFieldsWriter.FORMAT_CURRENT);
+        throw new IndexFormatTooNewException(idxStream, format, DefaultFieldsWriter.FORMAT_MINIMUM, DefaultFieldsWriter.FORMAT_CURRENT);
     } finally {
       idxStream.close();
     }
@@ -128,9 +128,9 @@ public final class DefaultFieldsReader e
       format = cloneableIndexStream.readInt();
 
       if (format < DefaultFieldsWriter.FORMAT_MINIMUM)
-        throw new IndexFormatTooOldException(indexStreamFN, format, DefaultFieldsWriter.FORMAT_MINIMUM, DefaultFieldsWriter.FORMAT_CURRENT);
+        throw new IndexFormatTooOldException(cloneableIndexStream, format, DefaultFieldsWriter.FORMAT_MINIMUM, DefaultFieldsWriter.FORMAT_CURRENT);
       if (format > DefaultFieldsWriter.FORMAT_CURRENT)
-        throw new IndexFormatTooNewException(indexStreamFN, format, DefaultFieldsWriter.FORMAT_MINIMUM, DefaultFieldsWriter.FORMAT_CURRENT);
+        throw new IndexFormatTooNewException(cloneableIndexStream, format, DefaultFieldsWriter.FORMAT_MINIMUM, DefaultFieldsWriter.FORMAT_CURRENT);
 
       fieldsStream = (IndexInput) cloneableFieldsStream.clone();
 
@@ -271,33 +271,4 @@ public final class DefaultFieldsReader e
 
     return fieldsStream;
   }
-
-  /**
-   * Skip the field.  We still have to read some of the information about the field, but can skip past the actual content.
-   * This will have the most payoff on large fields.
-   */
-  private void skipField(int numeric) throws IOException {
-    final int numBytes;
-    switch(numeric) {
-      case 0:
-        numBytes = fieldsStream.readVInt();
-        break;
-      case DefaultFieldsWriter.FIELD_IS_NUMERIC_INT:
-      case DefaultFieldsWriter.FIELD_IS_NUMERIC_FLOAT:
-        numBytes = 4;
-        break;
-      case DefaultFieldsWriter.FIELD_IS_NUMERIC_LONG:
-      case DefaultFieldsWriter.FIELD_IS_NUMERIC_DOUBLE:
-        numBytes = 8;
-        break;
-      default:
-        throw new FieldReaderException("Invalid numeric type: " + Integer.toHexString(numeric));
-    }
-    
-    skipFieldBytes(numBytes);
-  }
-  
-  private void skipFieldBytes(int toRead) throws IOException {
-    fieldsStream.seek(fieldsStream.getFilePointer() + toRead);
-  }
 }

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=1197690&r1=1197689&r2=1197690&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 Fri Nov  4 17:48:11 2011
@@ -64,7 +64,7 @@ public class DefaultSegmentInfosReader e
                 IndexFileNames.COMPOUND_FILE_STORE_EXTENSION), context, false);
           }
         } else if (si.getUseCompoundFile()) {
-          dir = new CompoundFileDirectory(dir,IndexFileNames.segmentFileName(
+          dir = new CompoundFileDirectory(dir, IndexFileNames.segmentFileName(
               si.name, "", IndexFileNames.COMPOUND_FILE_EXTENSION), context, false);
         }
 
@@ -83,7 +83,7 @@ public class DefaultSegmentInfosReader e
         // 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());
+        throw new IndexFormatTooOldException("segment " + si.name + " in resource " + input, si.getVersion());
       }
       infos.add(si);
     }

Modified: lucene/dev/trunk/lucene/src/java/org/apache/lucene/index/codecs/MultiLevelSkipListReader.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/src/java/org/apache/lucene/index/codecs/MultiLevelSkipListReader.java?rev=1197690&r1=1197689&r2=1197690&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/src/java/org/apache/lucene/index/codecs/MultiLevelSkipListReader.java (original)
+++ lucene/dev/trunk/lucene/src/java/org/apache/lucene/index/codecs/MultiLevelSkipListReader.java Fri Nov  4 17:48:11 2011
@@ -255,6 +255,7 @@ public abstract class MultiLevelSkipList
     private int pos;
     
     SkipBuffer(IndexInput input, int length) throws IOException {
+      super("SkipBuffer on " + input);
       data = new byte[length];
       pointer = input.getFilePointer();
       input.readBytes(data, 0, length);

Modified: lucene/dev/trunk/lucene/src/java/org/apache/lucene/index/codecs/lucene3x/SegmentTermEnum.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/src/java/org/apache/lucene/index/codecs/lucene3x/SegmentTermEnum.java?rev=1197690&r1=1197689&r2=1197690&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/src/java/org/apache/lucene/index/codecs/lucene3x/SegmentTermEnum.java (original)
+++ lucene/dev/trunk/lucene/src/java/org/apache/lucene/index/codecs/lucene3x/SegmentTermEnum.java Fri Nov  4 17:48:11 2011
@@ -85,9 +85,9 @@ public final class SegmentTermEnum imple
 
       // check that it is a format we can understand
       if (format > FORMAT_MINIMUM)
-        throw new IndexFormatTooOldException(null, format, FORMAT_MINIMUM, FORMAT_CURRENT);
+        throw new IndexFormatTooOldException(input, format, FORMAT_MINIMUM, FORMAT_CURRENT);
       if (format < FORMAT_CURRENT)
-        throw new IndexFormatTooNewException(null, format, FORMAT_MINIMUM, FORMAT_CURRENT);
+        throw new IndexFormatTooNewException(input, format, FORMAT_MINIMUM, FORMAT_CURRENT);
 
       size = input.readLong();                    // read the size
       

Modified: lucene/dev/trunk/lucene/src/java/org/apache/lucene/index/codecs/lucene40/Lucene40PostingsWriter.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/src/java/org/apache/lucene/index/codecs/lucene40/Lucene40PostingsWriter.java?rev=1197690&r1=1197689&r2=1197690&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/src/java/org/apache/lucene/index/codecs/lucene40/Lucene40PostingsWriter.java (original)
+++ lucene/dev/trunk/lucene/src/java/org/apache/lucene/index/codecs/lucene40/Lucene40PostingsWriter.java Fri Nov  4 17:48:11 2011
@@ -163,7 +163,7 @@ public final class Lucene40PostingsWrite
     final int delta = docID - lastDocID;
     
     if (docID < 0 || (df > 0 && delta <= 0)) {
-      throw new CorruptIndexException("docs out of order (" + docID + " <= " + lastDocID + " )");
+      throw new CorruptIndexException("docs out of order (" + docID + " <= " + lastDocID + " ) (freqOut: " + freqOut + ")");
     }
 
     if ((++df % skipInterval) == 0) {

Modified: lucene/dev/trunk/lucene/src/java/org/apache/lucene/index/codecs/sep/SepPostingsWriter.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/src/java/org/apache/lucene/index/codecs/sep/SepPostingsWriter.java?rev=1197690&r1=1197689&r2=1197690&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/src/java/org/apache/lucene/index/codecs/sep/SepPostingsWriter.java (original)
+++ lucene/dev/trunk/lucene/src/java/org/apache/lucene/index/codecs/sep/SepPostingsWriter.java Fri Nov  4 17:48:11 2011
@@ -20,7 +20,6 @@ package org.apache.lucene.index.codecs.s
 import java.io.IOException;
 import java.util.ArrayList;
 import java.util.List;
-import java.util.Set;
 
 import org.apache.lucene.index.CorruptIndexException;
 import org.apache.lucene.index.DocsEnum;
@@ -203,7 +202,7 @@ public final class SepPostingsWriter ext
     //System.out.println("SEPW: startDoc: write doc=" + docID + " delta=" + delta + " out.fp=" + docOut);
 
     if (docID < 0 || (df > 0 && delta <= 0)) {
-      throw new CorruptIndexException("docs out of order (" + docID + " <= " + lastDocID + " )");
+      throw new CorruptIndexException("docs out of order (" + docID + " <= " + lastDocID + " ) (docOut: " + docOut + ")");
     }
 
     if ((++df % skipInterval) == 0) {

Modified: lucene/dev/trunk/lucene/src/java/org/apache/lucene/store/BufferedIndexInput.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/src/java/org/apache/lucene/store/BufferedIndexInput.java?rev=1197690&r1=1197689&r2=1197690&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/src/java/org/apache/lucene/store/BufferedIndexInput.java (original)
+++ lucene/dev/trunk/lucene/src/java/org/apache/lucene/store/BufferedIndexInput.java Fri Nov  4 17:48:11 2011
@@ -51,14 +51,17 @@ public abstract class BufferedIndexInput
     return buffer[bufferPosition++];
   }
 
-  public BufferedIndexInput() {}
-  
-  public BufferedIndexInput(IOContext context) {
-    this(bufferSize(context));
+  public BufferedIndexInput(String resourceDesc) {
+    this(resourceDesc, BUFFER_SIZE);
+  }
+
+  public BufferedIndexInput(String resourceDesc, IOContext context) {
+    this(resourceDesc, bufferSize(context));
   }
 
   /** Inits BufferedIndexInput with a specific bufferSize */
-  public BufferedIndexInput(int bufferSize) {
+  public BufferedIndexInput(String resourceDesc, int bufferSize) {
+    super(resourceDesc);
     checkBufferSize(bufferSize);
     this.bufferSize = bufferSize;
   }

Modified: lucene/dev/trunk/lucene/src/java/org/apache/lucene/store/ChecksumIndexInput.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/src/java/org/apache/lucene/store/ChecksumIndexInput.java?rev=1197690&r1=1197689&r2=1197690&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/src/java/org/apache/lucene/store/ChecksumIndexInput.java (original)
+++ lucene/dev/trunk/lucene/src/java/org/apache/lucene/store/ChecksumIndexInput.java Fri Nov  4 17:48:11 2011
@@ -31,6 +31,7 @@ public class ChecksumIndexInput extends 
   Checksum digest;
 
   public ChecksumIndexInput(IndexInput main) {
+    super("ChecksumIndexInput(" + main + ")");
     this.main = main;
     digest = new CRC32();
   }

Modified: lucene/dev/trunk/lucene/src/java/org/apache/lucene/store/CompoundFileDirectory.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/src/java/org/apache/lucene/store/CompoundFileDirectory.java?rev=1197690&r1=1197689&r2=1197690&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/src/java/org/apache/lucene/store/CompoundFileDirectory.java (original)
+++ lucene/dev/trunk/lucene/src/java/org/apache/lucene/store/CompoundFileDirectory.java Fri Nov  4 17:48:11 2011
@@ -139,7 +139,7 @@ public final class CompoundFileDirectory
     if (firstInt < CompoundFileWriter.FORMAT_PRE_VERSION) {
       if (firstInt < CompoundFileWriter.FORMAT_CURRENT) {
         throw new CorruptIndexException("Incompatible format version: "
-            + firstInt + " expected " + CompoundFileWriter.FORMAT_CURRENT);
+            + firstInt + " expected " + CompoundFileWriter.FORMAT_CURRENT + " (resource: " + stream + ")");
       }
       // It's a post-3.1 index, read the count.
       count = stream.readVInt();
@@ -155,7 +155,7 @@ public final class CompoundFileDirectory
     for (int i=0; i<count; i++) {
       long offset = stream.readLong();
       if (offset < 0 || offset > streamLength) {
-        throw new CorruptIndexException("Invalid CFS entry offset: " + offset);
+        throw new CorruptIndexException("Invalid CFS entry offset: " + offset + " (resource: " + stream + ")");
       }
       String id = stream.readString();
       
@@ -218,7 +218,7 @@ public final class CompoundFileDirectory
     if (entry == null) {
       throw new IOException("No sub-file with id " + id + " found (fileName=" + name + " files: " + entries.keySet() + ")");
     }
-    return handle.openSlice(entry.offset, entry.length);
+    return handle.openSlice(name, entry.offset, entry.length);
   }
   
   /** Returns an array of strings, one for each file in the directory. */
@@ -318,13 +318,13 @@ public final class CompoundFileDirectory
       }
       
       @Override
-      public IndexInput openSlice(long offset, long length) throws IOException {
-        return handle.openSlice(entry.offset + offset, length);
+      public IndexInput openSlice(String sliceDescription, long offset, long length) throws IOException {
+        return handle.openSlice(sliceDescription, entry.offset + offset, length);
       }
 
       @Override
       public IndexInput openFullSlice() throws IOException {
-        return openSlice(0, entry.length);
+        return openSlice("full-slice", 0, entry.length);
       }
     };
   }

Modified: lucene/dev/trunk/lucene/src/java/org/apache/lucene/store/Directory.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/src/java/org/apache/lucene/store/Directory.java?rev=1197690&r1=1197689&r2=1197690&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/src/java/org/apache/lucene/store/Directory.java (original)
+++ lucene/dev/trunk/lucene/src/java/org/apache/lucene/store/Directory.java Fri Nov  4 17:48:11 2011
@@ -225,8 +225,8 @@ public abstract class Directory implemen
     return new IndexInputSlicer() {
       private final IndexInput base = Directory.this.openInput(name, context);
       @Override
-      public IndexInput openSlice(long offset, long length) {
-        return new SlicedIndexInput(base, offset, length);
+      public IndexInput openSlice(String sliceDescription, long offset, long length) {
+        return new SlicedIndexInput("SlicedIndexInput(" + sliceDescription + " in " + base + ")", base, offset, length);
       }
       @Override
       public void close() throws IOException {
@@ -258,7 +258,7 @@ public abstract class Directory implemen
     /**
      * Returns an {@link IndexInput} slice starting at the given offset with the given length.
      */
-    public abstract IndexInput openSlice(long offset, long length) throws IOException;
+    public abstract IndexInput openSlice(String sliceDescription, long offset, long length) throws IOException;
 
     /**
      * Returns an {@link IndexInput} slice starting at offset <i>0</i> with a
@@ -275,12 +275,12 @@ public abstract class Directory implemen
     long fileOffset;
     long length;
     
-    SlicedIndexInput(final IndexInput base, final long fileOffset, final long length) {
-      this(base, fileOffset, length, BufferedIndexInput.BUFFER_SIZE);
+    SlicedIndexInput(final String sliceDescription, final IndexInput base, final long fileOffset, final long length) {
+      this(sliceDescription, base, fileOffset, length, BufferedIndexInput.BUFFER_SIZE);
     }
     
-    SlicedIndexInput(final IndexInput base, final long fileOffset, final long length, int readBufferSize) {
-      super(readBufferSize);
+    SlicedIndexInput(final String sliceDescription, final IndexInput base, final long fileOffset, final long length, int readBufferSize) {
+      super("SlicedIndexInput(" + sliceDescription + " in " + base + " slice=" + fileOffset + ":" + (fileOffset+length) + ")", readBufferSize);
       this.base = (IndexInput) base.clone();
       this.fileOffset = fileOffset;
       this.length = length;

Modified: lucene/dev/trunk/lucene/src/java/org/apache/lucene/store/IndexInput.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/src/java/org/apache/lucene/store/IndexInput.java?rev=1197690&r1=1197689&r2=1197690&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/src/java/org/apache/lucene/store/IndexInput.java (original)
+++ lucene/dev/trunk/lucene/src/java/org/apache/lucene/store/IndexInput.java Fri Nov  4 17:48:11 2011
@@ -26,6 +26,18 @@ import java.io.IOException;
  */
 public abstract class IndexInput extends DataInput implements Cloneable,Closeable {
 
+  private final String resourceDescription;
+
+  /** resourceDescription should be a non-null, opaque string
+   *  describing this resource; it's returned from
+   *  {@link #toString}. */
+  protected IndexInput(String resourceDescription) {
+    if (resourceDescription == null) {
+      throw new IllegalArgumentException("resourceDescription must not be null");
+    }
+    this.resourceDescription = resourceDescription;
+  }
+
   /** Closes the stream to further operations. */
   public abstract void close() throws IOException;
 
@@ -66,5 +78,9 @@ public abstract class IndexInput extends
       numBytes -= toCopy;
     }
   }
-  
+
+  @Override
+  public String toString() {
+    return resourceDescription;
+  }
 }

Modified: lucene/dev/trunk/lucene/src/java/org/apache/lucene/store/MMapDirectory.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/src/java/org/apache/lucene/store/MMapDirectory.java?rev=1197690&r1=1197689&r2=1197690&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/src/java/org/apache/lucene/store/MMapDirectory.java (original)
+++ lucene/dev/trunk/lucene/src/java/org/apache/lucene/store/MMapDirectory.java Fri Nov  4 17:48:11 2011
@@ -213,7 +213,7 @@ public class MMapDirectory extends FSDir
     File f = new File(getDirectory(), name);
     RandomAccessFile raf = new RandomAccessFile(f, "r");
     try {
-      return new MMapIndexInput(raf, 0, raf.length(), chunkSizePower);
+      return new MMapIndexInput("MMapIndexInput(path=\"" + f + "\")", raf, 0, raf.length(), chunkSizePower);
     } finally {
       raf.close();
     }
@@ -221,7 +221,7 @@ public class MMapDirectory extends FSDir
   
   public IndexInputSlicer createSlicer(final String name, final IOContext context) throws IOException {
     ensureOpen();
-    File f = new File(getDirectory(), name);
+    final File f = new File(getDirectory(), name);
     final RandomAccessFile raf = new RandomAccessFile(f, "r");
     return new IndexInputSlicer() {
       @Override
@@ -230,13 +230,13 @@ public class MMapDirectory extends FSDir
       }
 
       @Override
-      public IndexInput openSlice(long offset, long length) throws IOException {
-        return new MMapIndexInput(raf, offset, length, chunkSizePower);
+      public IndexInput openSlice(String sliceDescription, long offset, long length) throws IOException {
+        return new MMapIndexInput("MMapIndexInput(" + sliceDescription + " in path=\"" + f + "\" slice=" + offset + ":" + (offset+length) + ")", raf, offset, length, chunkSizePower);
       }
 
       @Override
       public IndexInput openFullSlice() throws IOException {
-        return openSlice(0, raf.length());
+        return openSlice("full-slice", 0, raf.length());
       }
     };
   }
@@ -256,8 +256,9 @@ public class MMapDirectory extends FSDir
     private ByteBuffer curBuf; // redundant for speed: buffers[curBufIndex]
   
     private boolean isClone = false;
-    
-    MMapIndexInput(RandomAccessFile raf, long offset, long length, int chunkSizePower) throws IOException {
+
+    MMapIndexInput(String resourceDescription, RandomAccessFile raf, long offset, long length, int chunkSizePower) throws IOException {
+      super(resourceDescription);
       this.length = length;
       this.chunkSizePower = chunkSizePower;
       this.chunkSize = 1L << chunkSizePower;
@@ -296,8 +297,9 @@ public class MMapDirectory extends FSDir
       } catch (BufferUnderflowException e) {
         do {
           curBufIndex++;
-          if (curBufIndex >= buffers.length)
-            throw new IOException("read past EOF");
+          if (curBufIndex >= buffers.length) {
+            throw new IOException("read past EOF: " + this);
+          }
           curBuf = buffers[curBufIndex];
           curBuf.position(0);
         } while (!curBuf.hasRemaining());
@@ -316,8 +318,9 @@ public class MMapDirectory extends FSDir
           len -= curAvail;
           offset += curAvail;
           curBufIndex++;
-          if (curBufIndex >= buffers.length)
-            throw new IOException("read past EOF");
+          if (curBufIndex >= buffers.length) {
+            throw new IOException("read past EOF: " + this);
+          }
           curBuf = buffers[curBufIndex];
           curBuf.position(0);
           curAvail = curBuf.remaining();
@@ -369,13 +372,15 @@ public class MMapDirectory extends FSDir
         this.curBufIndex = bi;
         this.curBuf = b;
       } catch (ArrayIndexOutOfBoundsException aioobe) {
-        if (pos < 0L)
-          throw new IllegalArgumentException("Seeking to negative position");
+        if (pos < 0L) {
+          throw new IllegalArgumentException("Seeking to negative position: " + this);
+        }
         throw new IOException("seek past EOF");
       } catch (IllegalArgumentException iae) {
-        if (pos < 0L)
-          throw new IllegalArgumentException("Seeking to negative position");
-        throw new IOException("seek past EOF");
+        if (pos < 0L) {
+          throw new IllegalArgumentException("Seeking to negative position: " + this);
+        }
+        throw new IOException("seek past EOF: " + this);
       }
     }
   
@@ -386,8 +391,9 @@ public class MMapDirectory extends FSDir
   
     @Override
     public Object clone() {
-      if (buffers == null)
-        throw new AlreadyClosedException("MMapIndexInput already closed");
+      if (buffers == null) {
+        throw new AlreadyClosedException("MMapIndexInput already closed: " + this);
+      }
       final MMapIndexInput clone = (MMapIndexInput)super.clone();
       clone.isClone = true;
       clone.buffers = new ByteBuffer[buffers.length];
@@ -399,7 +405,7 @@ public class MMapDirectory extends FSDir
       try {
         clone.seek(getFilePointer());
       } catch(IOException ioe) {
-        throw new RuntimeException("Should never happen", ioe);
+        throw new RuntimeException("Should never happen: " + this, ioe);
       }
       return clone;
     }

Modified: lucene/dev/trunk/lucene/src/java/org/apache/lucene/store/NIOFSDirectory.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/src/java/org/apache/lucene/store/NIOFSDirectory.java?rev=1197690&r1=1197689&r2=1197690&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/src/java/org/apache/lucene/store/NIOFSDirectory.java (original)
+++ lucene/dev/trunk/lucene/src/java/org/apache/lucene/store/NIOFSDirectory.java Fri Nov  4 17:48:11 2011
@@ -83,8 +83,8 @@ public class NIOFSDirectory extends FSDi
   public IndexInputSlicer createSlicer(final String name,
       final IOContext context) throws IOException {
     ensureOpen();
-    final File file = new File(getDirectory(), name);
-    final Descriptor descriptor = new Descriptor(file, "r");
+    final File path = new File(getDirectory(), name);
+    final Descriptor descriptor = new Descriptor(path, "r");
     return new Directory.IndexInputSlicer() {
 
       @Override
@@ -93,14 +93,14 @@ public class NIOFSDirectory extends FSDi
       }
 
       @Override
-      public IndexInput openSlice(long offset, long length) throws IOException {
-        return new NIOFSIndexInput(descriptor, descriptor.getChannel(), offset,
+      public IndexInput openSlice(String sliceDescription, long offset, long length) throws IOException {
+        return new NIOFSIndexInput(sliceDescription, path, descriptor, descriptor.getChannel(), offset,
             length, BufferedIndexInput.bufferSize(context), getReadChunkSize());
       }
 
       @Override
       public IndexInput openFullSlice() throws IOException {
-        return openSlice(0, descriptor.length);
+        return openSlice("full-slice", 0, descriptor.length);
       }
     };
   }
@@ -115,12 +115,12 @@ public class NIOFSDirectory extends FSDi
     final FileChannel channel;
 
     public NIOFSIndexInput(File path, IOContext context, int chunkSize) throws IOException {
-      super(path, context, chunkSize);
+      super("NIOFSIndexInput(path=\"" + path + "\")", path, context, chunkSize);
       channel = file.getChannel();
     }
     
-    public NIOFSIndexInput(Descriptor file, FileChannel fc, long off, long length, int bufferSize, int chunkSize) throws IOException {
-      super(file, off, length, bufferSize, chunkSize);
+    public NIOFSIndexInput(String sliceDescription, File path, Descriptor file, FileChannel fc, long off, long length, int bufferSize, int chunkSize) throws IOException {
+      super("NIOFSIndexInput(" + sliceDescription + " in path=\"" + path + "\" slice=" + off + ":" + (off+length) + ")", file, off, length, bufferSize, chunkSize);
       channel = fc;
       isClone = true;
     }
@@ -181,7 +181,7 @@ public class NIOFSDirectory extends FSDi
       long pos = getFilePointer() + off;
       
       if (pos + len > end) {
-        throw new IOException("read past EOF");
+        throw new IOException("read past EOF: " + this);
       }
 
       try {
@@ -209,6 +209,8 @@ public class NIOFSDirectory extends FSDi
               + "with a value smaller than the current chunk size (" + chunkSize + ")");
         outOfMemoryError.initCause(e);
         throw outOfMemoryError;
+      } catch (IOException ioe) {
+        throw new IOException(ioe.getMessage() + ": " + this, ioe);
       }
     }
   }

Modified: lucene/dev/trunk/lucene/src/java/org/apache/lucene/store/RAMDirectory.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/src/java/org/apache/lucene/store/RAMDirectory.java?rev=1197690&r1=1197689&r2=1197690&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/src/java/org/apache/lucene/store/RAMDirectory.java (original)
+++ lucene/dev/trunk/lucene/src/java/org/apache/lucene/store/RAMDirectory.java Fri Nov  4 17:48:11 2011
@@ -183,7 +183,7 @@ public class RAMDirectory extends Direct
     if (file == null) {
       throw new FileNotFoundException(name);
     }
-    return new RAMInputStream(file);
+    return new RAMInputStream(name, file);
   }
 
   /** Closes the store to future operations, releasing associated memory. */

Modified: lucene/dev/trunk/lucene/src/java/org/apache/lucene/store/RAMInputStream.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/src/java/org/apache/lucene/store/RAMInputStream.java?rev=1197690&r1=1197689&r2=1197690&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/src/java/org/apache/lucene/store/RAMInputStream.java (original)
+++ lucene/dev/trunk/lucene/src/java/org/apache/lucene/store/RAMInputStream.java Fri Nov  4 17:48:11 2011
@@ -18,6 +18,7 @@ package org.apache.lucene.store;
  */
 
 import java.io.IOException;
+import java.io.EOFException;
 
 /** A memory-resident {@link IndexInput} implementation. 
  *  
@@ -35,11 +36,12 @@ public class RAMInputStream extends Inde
   private long bufferStart;
   private int bufferLength;
 
-  public RAMInputStream(RAMFile f) throws IOException {
+  public RAMInputStream(String name, RAMFile f) throws IOException {
+    super("RAMInputStream(name=" + name + ")");
     file = f;
     length = file.length;
     if (length/BUFFER_SIZE >= Integer.MAX_VALUE) {
-      throw new IOException("Too large RAMFile! "+length); 
+      throw new IOException("RAMInputStream too large length=" + length + ": " + name); 
     }
 
     // make sure that we switch to the
@@ -88,9 +90,9 @@ public class RAMInputStream extends Inde
     bufferStart = (long) BUFFER_SIZE * (long) currentBufferIndex;
     if (currentBufferIndex >= file.numBuffers()) {
       // end of file reached, no more buffers left
-      if (enforceEOF)
-        throw new IOException("Read past EOF");
-      else {
+      if (enforceEOF) {
+        throw new EOFException("Read past EOF: " + this);
+      } else {
         // Force EOF if a read takes place at this position
         currentBufferIndex--;
         bufferPosition = BUFFER_SIZE;

Modified: lucene/dev/trunk/lucene/src/java/org/apache/lucene/store/SimpleFSDirectory.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/src/java/org/apache/lucene/store/SimpleFSDirectory.java?rev=1197690&r1=1197689&r2=1197690&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/src/java/org/apache/lucene/store/SimpleFSDirectory.java (original)
+++ lucene/dev/trunk/lucene/src/java/org/apache/lucene/store/SimpleFSDirectory.java Fri Nov  4 17:48:11 2011
@@ -55,10 +55,9 @@ public class SimpleFSDirectory extends F
   @Override
   public IndexInput openInput(String name, IOContext context) throws IOException {
     ensureOpen();
-    return new SimpleFSIndexInput(new File(directory, name), context, getReadChunkSize());
+    final File path = new File(directory, name);
+    return new SimpleFSIndexInput("SimpleFSIndexInput(path=\"" + path.getPath() + "\")", path, context, getReadChunkSize());
   }
-  
-  
 
   public IndexInputSlicer createSlicer(final String name,
       final IOContext context) throws IOException {
@@ -73,19 +72,18 @@ public class SimpleFSDirectory extends F
       }
 
       @Override
-      public IndexInput openSlice(long offset, long length) throws IOException {
-        return new SimpleFSIndexInput(descriptor, offset,
+      public IndexInput openSlice(String sliceDescription, long offset, long length) throws IOException {
+        return new SimpleFSIndexInput("SimpleFSIndexInput(" + sliceDescription + " in path=\"" + file.getPath() + "\" slice=" + offset + ":" + (offset+length) + ")", descriptor, offset,
             length, BufferedIndexInput.bufferSize(context), getReadChunkSize());
       }
 
       @Override
       public IndexInput openFullSlice() throws IOException {
-        return openSlice(0, descriptor.length);
+        return openSlice("full-slice", 0, descriptor.length);
       }
     };
   }
 
-
   protected static class SimpleFSIndexInput extends BufferedIndexInput {
   
     protected static class Descriptor extends RandomAccessFile {
@@ -117,16 +115,16 @@ public class SimpleFSDirectory extends F
     protected final long off;
     protected final long end;
     
-    public SimpleFSIndexInput(File path, IOContext context, int chunkSize) throws IOException {
-      super(context);
+    public SimpleFSIndexInput(String resourceDesc, File path, IOContext context, int chunkSize) throws IOException {
+      super(resourceDesc, context);
       this.file = new Descriptor(path, "r"); 
       this.chunkSize = chunkSize;
       this.off = 0L;
       this.end = file.length;
     }
     
-    public SimpleFSIndexInput(Descriptor file, long off, long length, int bufferSize, int chunkSize) throws IOException {
-      super(bufferSize);
+    public SimpleFSIndexInput(String resourceDesc, Descriptor file, long off, long length, int bufferSize, int chunkSize) throws IOException {
+      super(resourceDesc, bufferSize);
       this.file = file;
       this.chunkSize = chunkSize;
       this.off = off;
@@ -147,7 +145,7 @@ public class SimpleFSDirectory extends F
         int total = 0;
 
         if (position + len > end) {
-          throw new IOException("read past EOF");
+          throw new IOException("read past EOF: " + this);
         }
 
         try {
@@ -172,6 +170,8 @@ public class SimpleFSDirectory extends F
               + "with a value smaller than the current chunk size (" + chunkSize + ")");
           outOfMemoryError.initCause(e);
           throw outOfMemoryError;
+        } catch (IOException ioe) {
+          throw new IOException(ioe.getMessage() + ": " + this, ioe);
         }
       }
     }

Modified: lucene/dev/trunk/lucene/src/java/org/apache/lucene/util/CodecUtil.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/src/java/org/apache/lucene/util/CodecUtil.java?rev=1197690&r1=1197689&r2=1197690&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/src/java/org/apache/lucene/util/CodecUtil.java (original)
+++ lucene/dev/trunk/lucene/src/java/org/apache/lucene/util/CodecUtil.java Fri Nov  4 17:48:11 2011
@@ -18,13 +18,13 @@ package org.apache.lucene.util;
  */
 
 
-import org.apache.lucene.store.DataInput;
-import org.apache.lucene.store.DataOutput;
+import java.io.IOException;
+
 import org.apache.lucene.index.CorruptIndexException;
 import org.apache.lucene.index.IndexFormatTooNewException;
 import org.apache.lucene.index.IndexFormatTooOldException;
-
-import java.io.IOException;
+import org.apache.lucene.store.DataInput;
+import org.apache.lucene.store.DataOutput;
 
 /**
  * @lucene.experimental
@@ -56,20 +56,20 @@ public final class CodecUtil {
     // Safety to guard against reading a bogus string:
     final int actualHeader = in.readInt();
     if (actualHeader != CODEC_MAGIC) {
-      throw new CorruptIndexException("codec header mismatch: actual header=" + actualHeader + " vs expected header=" + CODEC_MAGIC);
+      throw new CorruptIndexException("codec header mismatch: actual header=" + actualHeader + " vs expected header=" + CODEC_MAGIC + " (resource: " + in + ")");
     }
 
     final String actualCodec = in.readString();
     if (!actualCodec.equals(codec)) {
-      throw new CorruptIndexException("codec mismatch: actual codec=" + actualCodec + " vs expected codec=" + codec);
+      throw new CorruptIndexException("codec mismatch: actual codec=" + actualCodec + " vs expected codec=" + codec + " (resource: " + in + ")");
     }
 
     final int actualVersion = in.readInt();
     if (actualVersion < minVersion) {
-      throw new IndexFormatTooOldException(null, actualVersion, minVersion, maxVersion);
+      throw new IndexFormatTooOldException(in, actualVersion, minVersion, maxVersion);
     }
     if (actualVersion > maxVersion) {
-      throw new IndexFormatTooNewException(null, actualVersion, minVersion, maxVersion);
+      throw new IndexFormatTooNewException(in, actualVersion, minVersion, maxVersion);
     }
 
     return actualVersion;

Modified: lucene/dev/trunk/lucene/src/test-framework/java/org/apache/lucene/index/MockIndexInput.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/src/test-framework/java/org/apache/lucene/index/MockIndexInput.java?rev=1197690&r1=1197689&r2=1197690&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/src/test-framework/java/org/apache/lucene/index/MockIndexInput.java (original)
+++ lucene/dev/trunk/lucene/src/test-framework/java/org/apache/lucene/index/MockIndexInput.java Fri Nov  4 17:48:11 2011
@@ -25,6 +25,7 @@ public class MockIndexInput extends Buff
     private long length;
 
     public MockIndexInput(byte[] bytes) {
+        super("MockIndexInput", BufferedIndexInput.BUFFER_SIZE);
         buffer = bytes;
         length = bytes.length;
     }

Modified: lucene/dev/trunk/lucene/src/test-framework/java/org/apache/lucene/store/MockDirectoryWrapper.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/src/test-framework/java/org/apache/lucene/store/MockDirectoryWrapper.java?rev=1197690&r1=1197689&r2=1197690&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/src/test-framework/java/org/apache/lucene/store/MockDirectoryWrapper.java (original)
+++ lucene/dev/trunk/lucene/src/test-framework/java/org/apache/lucene/store/MockDirectoryWrapper.java Fri Nov  4 17:48:11 2011
@@ -673,9 +673,9 @@ public class MockDirectoryWrapper extend
       }
 
       @Override
-      public IndexInput openSlice(long offset, long length) throws IOException {
+      public IndexInput openSlice(String sliceDescription, long offset, long length) throws IOException {
         maybeYield();
-        IndexInput ii = new MockIndexInputWrapper(MockDirectoryWrapper.this, name, delegateHandle.openSlice(offset, length));
+        IndexInput ii = new MockIndexInputWrapper(MockDirectoryWrapper.this, name, delegateHandle.openSlice(sliceDescription, offset, length));
         addFileHandle(ii, name, Handle.Input);
         return ii;
       }

Modified: lucene/dev/trunk/lucene/src/test-framework/java/org/apache/lucene/store/MockIndexInputWrapper.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/src/test-framework/java/org/apache/lucene/store/MockIndexInputWrapper.java?rev=1197690&r1=1197689&r2=1197690&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/src/test-framework/java/org/apache/lucene/store/MockIndexInputWrapper.java (original)
+++ lucene/dev/trunk/lucene/src/test-framework/java/org/apache/lucene/store/MockIndexInputWrapper.java Fri Nov  4 17:48:11 2011
@@ -34,6 +34,7 @@ public class MockIndexInputWrapper exten
 
   /** Construct an empty output buffer. */
   public MockIndexInputWrapper(MockDirectoryWrapper dir, String name, IndexInput delegate) {
+    super("MockIndexInputWrapper(name=" + name + " delegate=" + delegate + ")");
     this.name = name;
     this.dir = dir;
     this.delegate = delegate;

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=1197690&r1=1197689&r2=1197690&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 Fri Nov  4 17:48:11 2011
@@ -122,7 +122,7 @@ public class TestBackwardsCompatibility 
                                       "31.optimized.nocfs",
   };
   
-  /** This test checks that *only* IndexFormatTooOldExceptions are throws when you open and operate on too old indexes! */
+  /** This test checks that *only* IndexFormatTooOldExceptions are thrown when you open and operate on too old indexes! */
   public void testUnsupportedOldIndexes() throws Exception {
     for(int i=0;i<unsupportedNames.length;i++) {
       if (VERBOSE) {
@@ -154,6 +154,8 @@ public class TestBackwardsCompatibility 
           System.out.println("TEST: got expected exc:");
           e.printStackTrace(System.out);
         }
+        // Make sure exc message includes a path=
+        assertTrue("got exc message: " + e.getMessage(), e.getMessage().indexOf("path=\"") != -1);
       } finally {
         // 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

Modified: lucene/dev/trunk/lucene/src/test/org/apache/lucene/index/TestFieldsReader.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/src/test/org/apache/lucene/index/TestFieldsReader.java?rev=1197690&r1=1197689&r2=1197690&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/src/test/org/apache/lucene/index/TestFieldsReader.java (original)
+++ lucene/dev/trunk/lucene/src/test/org/apache/lucene/index/TestFieldsReader.java Fri Nov  4 17:48:11 2011
@@ -161,6 +161,7 @@ public class TestFieldsReader extends Lu
     static boolean doFail;
     int count;
     private FaultyIndexInput(IndexInput delegate) {
+      super("FaultyIndexInput(" + delegate + ")", BufferedIndexInput.BUFFER_SIZE);
       this.delegate = delegate;
     }
     private void simOutage() throws IOException {

Modified: lucene/dev/trunk/lucene/src/test/org/apache/lucene/index/TestLazyProxSkipping.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/src/test/org/apache/lucene/index/TestLazyProxSkipping.java?rev=1197690&r1=1197689&r2=1197690&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/src/test/org/apache/lucene/index/TestLazyProxSkipping.java (original)
+++ lucene/dev/trunk/lucene/src/test/org/apache/lucene/index/TestLazyProxSkipping.java Fri Nov  4 17:48:11 2011
@@ -189,6 +189,7 @@ public class TestLazyProxSkipping extend
           
           
           SeeksCountingStream(IndexInput input) {
+              super("SeekCountingStream(" + input + ")");
               this.input = input;
           }      
                 

Modified: lucene/dev/trunk/lucene/src/test/org/apache/lucene/index/TestMultiLevelSkipList.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/src/test/org/apache/lucene/index/TestMultiLevelSkipList.java?rev=1197690&r1=1197689&r2=1197690&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/src/test/org/apache/lucene/index/TestMultiLevelSkipList.java (original)
+++ lucene/dev/trunk/lucene/src/test/org/apache/lucene/index/TestMultiLevelSkipList.java Fri Nov  4 17:48:11 2011
@@ -152,6 +152,7 @@ public class TestMultiLevelSkipList exte
     private IndexInput input;
 
     CountingStream(IndexInput input) {
+      super("CountingStream(" + input + ")");
       this.input = input;
     }
 

Modified: lucene/dev/trunk/lucene/src/test/org/apache/lucene/store/TestBufferedIndexInput.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/src/test/org/apache/lucene/store/TestBufferedIndexInput.java?rev=1197690&r1=1197689&r2=1197690&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/src/test/org/apache/lucene/store/TestBufferedIndexInput.java (original)
+++ lucene/dev/trunk/lucene/src/test/org/apache/lucene/store/TestBufferedIndexInput.java Fri Nov  4 17:48:11 2011
@@ -92,7 +92,7 @@ public class TestBufferedIndexInput exte
     writeBytes(tmpInputFile, TEST_FILE_LENGTH);
 
     // run test with chunk size of 10 bytes
-    runReadBytesAndClose(new SimpleFSIndexInput(tmpInputFile,
+    runReadBytesAndClose(new SimpleFSIndexInput("SimpleFSIndexInput(path=\"" + tmpInputFile + "\")", tmpInputFile,
         newIOContext(random), 10), inputBufferSize, random);
 
     // run test with chunk size of 10 bytes
@@ -211,6 +211,7 @@ public class TestBufferedIndexInput exte
       private long pos;
       private long len;
       public MyBufferedIndexInput(long len){
+        super("MyBufferedIndexInput(len=" + len + ")", BufferedIndexInput.BUFFER_SIZE);
         this.len = len;
         this.pos = 0;
       }

Modified: lucene/dev/trunk/lucene/src/test/org/apache/lucene/store/TestHugeRamFile.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/src/test/org/apache/lucene/store/TestHugeRamFile.java?rev=1197690&r1=1197689&r2=1197690&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/src/test/org/apache/lucene/store/TestHugeRamFile.java (original)
+++ lucene/dev/trunk/lucene/src/test/org/apache/lucene/store/TestHugeRamFile.java Fri Nov  4 17:48:11 2011
@@ -85,7 +85,7 @@ public class TestHugeRamFile extends Luc
     }
     out.close();
     // input part
-    RAMInputStream in = new RAMInputStream(f);
+    RAMInputStream in = new RAMInputStream("testcase", f);
     assertEquals("input length must match",n,in.length());
     //System.out.println("input length = "+in.length()+" % 1024 = "+in.length()%1024);
     for (int j=0; j<L; j++) {

Modified: lucene/dev/trunk/modules/facet/src/test/org/apache/lucene/util/SlowRAMDirectory.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/modules/facet/src/test/org/apache/lucene/util/SlowRAMDirectory.java?rev=1197690&r1=1197689&r2=1197690&view=diff
==============================================================================
--- lucene/dev/trunk/modules/facet/src/test/org/apache/lucene/util/SlowRAMDirectory.java (original)
+++ lucene/dev/trunk/modules/facet/src/test/org/apache/lucene/util/SlowRAMDirectory.java Fri Nov  4 17:48:11 2011
@@ -1,14 +1,5 @@
 package org.apache.lucene.util;
 
-import java.io.IOException;
-import java.util.Random;
-
-import org.apache.lucene.store.IOContext;
-import org.apache.lucene.store.IndexInput;
-import org.apache.lucene.store.IndexOutput;
-import org.apache.lucene.store.RAMDirectory;
-import org.apache.lucene.util.ThreadInterruptedException;
-
 /**
  * Licensed to the Apache Software Foundation (ASF) under one or more
  * contributor license agreements.  See the NOTICE file distributed with
@@ -26,6 +17,14 @@ import org.apache.lucene.util.ThreadInte
  * limitations under the License.
  */
 
+import java.io.IOException;
+import java.util.Random;
+
+import org.apache.lucene.store.IOContext;
+import org.apache.lucene.store.IndexInput;
+import org.apache.lucene.store.IndexOutput;
+import org.apache.lucene.store.RAMDirectory;
+
 /**
  * Test utility - slow directory
  */
@@ -84,6 +83,7 @@ public class SlowRAMDirectory extends RA
     private int numRead = 0;
     
     public SlowIndexInput(IndexInput ii) {
+      super("SlowIndexInput(" + ii + ")");
       this.ii = ii;
     }