You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@lucene.apache.org by si...@apache.org on 2011/05/31 13:25:50 UTC

svn commit: r1129631 [4/7] - in /lucene/dev/branches/docvalues: ./ dev-tools/eclipse/ dev-tools/idea/.idea/ dev-tools/idea/lucene/contrib/spellchecker/ dev-tools/idea/modules/suggest/ dev-tools/maven/lucene/contrib/ dev-tools/maven/lucene/contrib/spell...

Modified: lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/index/codecs/sep/SepPostingsWriterImpl.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/index/codecs/sep/SepPostingsWriterImpl.java?rev=1129631&r1=1129630&r2=1129631&view=diff
==============================================================================
--- lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/index/codecs/sep/SepPostingsWriterImpl.java (original)
+++ lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/index/codecs/sep/SepPostingsWriterImpl.java Tue May 31 11:25:37 2011
@@ -31,6 +31,7 @@ import org.apache.lucene.store.IndexOutp
 import org.apache.lucene.store.RAMOutputStream;
 import org.apache.lucene.util.BytesRef;
 import org.apache.lucene.util.CodecUtil;
+import org.apache.lucene.util.IOUtils;
 
 /** Writes frq to .frq, docs to .doc, pos to .pos, payloads
  *  to .pyl, skip data to .skp
@@ -49,18 +50,18 @@ public final class SepPostingsWriterImpl
   final static int VERSION_START = 0;
   final static int VERSION_CURRENT = VERSION_START;
 
-  final IntIndexOutput freqOut;
-  final IntIndexOutput.Index freqIndex;
+  IntIndexOutput freqOut;
+  IntIndexOutput.Index freqIndex;
 
-  final IntIndexOutput posOut;
-  final IntIndexOutput.Index posIndex;
+  IntIndexOutput posOut;
+  IntIndexOutput.Index posIndex;
 
-  final IntIndexOutput docOut;
-  final IntIndexOutput.Index docIndex;
+  IntIndexOutput docOut;
+  IntIndexOutput.Index docIndex;
 
-  final IndexOutput payloadOut;
+  IndexOutput payloadOut;
 
-  final IndexOutput skipOut;
+  IndexOutput skipOut;
   IndexOutput termsOut;
 
   final SepSkipListWriter skipListWriter;
@@ -107,45 +108,51 @@ public final class SepPostingsWriterImpl
   }
 
   public SepPostingsWriterImpl(SegmentWriteState state, IntStreamFactory factory, int skipInterval) throws IOException {
-    super();
-    final String codecIdAsString = state.codecIdAsString();
-    this.skipInterval = skipInterval;
-    this.skipMinimum = skipInterval; /* set to the same for now */
-    final String docFileName = IndexFileNames.segmentFileName(state.segmentName, codecIdAsString, DOC_EXTENSION);
-    docOut = factory.createOutput(state.directory, docFileName);
-    docIndex = docOut.index();
-
-    if (state.fieldInfos.hasProx()) {
-      final String frqFileName = IndexFileNames.segmentFileName(state.segmentName, codecIdAsString, FREQ_EXTENSION);
-      freqOut = factory.createOutput(state.directory, frqFileName);
-      freqIndex = freqOut.index();
-
-      final String posFileName = IndexFileNames.segmentFileName(state.segmentName, codecIdAsString, POS_EXTENSION);
-      posOut = factory.createOutput(state.directory, posFileName);
-      posIndex = posOut.index();
-
-      // TODO: -- only if at least one field stores payloads?
-      final String payloadFileName = IndexFileNames.segmentFileName(state.segmentName, codecIdAsString, PAYLOAD_EXTENSION);
-      payloadOut = state.directory.createOutput(payloadFileName);
+    freqOut = null;
+    freqIndex = null;
+    posOut = null;
+    posIndex = null;
+    payloadOut = null;
+    boolean success = false;
+    try {
+      this.skipInterval = skipInterval;
+      this.skipMinimum = skipInterval; /* set to the same for now */
+      final String docFileName = IndexFileNames.segmentFileName(state.segmentName, state.codecIdAsString(), DOC_EXTENSION);
+      docOut = factory.createOutput(state.directory, docFileName);
+      docIndex = docOut.index();
+      
+      if (state.fieldInfos.hasProx()) {
+        final String frqFileName = IndexFileNames.segmentFileName(state.segmentName, state.codecIdAsString(), FREQ_EXTENSION);
+        freqOut = factory.createOutput(state.directory, frqFileName);
+        freqIndex = freqOut.index();
+        
+        final String posFileName = IndexFileNames.segmentFileName(state.segmentName, state.codecIdAsString(), POS_EXTENSION);
+        posOut = factory.createOutput(state.directory, posFileName);
+        posIndex = posOut.index();
+        
+        // TODO: -- only if at least one field stores payloads?
+        final String payloadFileName = IndexFileNames.segmentFileName(state.segmentName, state.codecIdAsString(), PAYLOAD_EXTENSION);
+        payloadOut = state.directory.createOutput(payloadFileName);
+      }
+      
+      final String skipFileName = IndexFileNames.segmentFileName(state.segmentName, state.codecIdAsString(), SKIP_EXTENSION);
+      skipOut = state.directory.createOutput(skipFileName);
+      
+      totalNumDocs = state.numDocs;
+      
+      skipListWriter = new SepSkipListWriter(skipInterval,
+          maxSkipLevels,
+          state.numDocs,
+          freqOut, docOut,
+          posOut, payloadOut);
+      
+      success = true;
+    } finally {
+      if (!success) {
+        IOUtils.closeSafely(true, docOut, skipOut, freqOut, posOut, payloadOut);
+      }
 
-    } else {
-      freqOut = null;
-      freqIndex = null;
-      posOut = null;
-      posIndex = null;
-      payloadOut = null;
     }
-
-    final String skipFileName = IndexFileNames.segmentFileName(state.segmentName, codecIdAsString, SKIP_EXTENSION);
-    skipOut = state.directory.createOutput(skipFileName);
-
-    totalNumDocs = state.numDocs;
-
-    skipListWriter = new SepSkipListWriter(skipInterval,
-                                           maxSkipLevels,
-                                           state.numDocs,
-                                           freqOut, docOut,
-                                           posOut, payloadOut);
   }
 
   @Override
@@ -307,25 +314,7 @@ public final class SepPostingsWriterImpl
 
   @Override
   public void close() throws IOException {
-    try {
-      docOut.close();
-    } finally {
-      try {
-        skipOut.close();
-      } finally {
-        if (freqOut != null) {
-          try {
-            freqOut.close();
-          } finally {
-            try {
-              posOut.close();
-            } finally {
-              payloadOut.close();
-            }
-          }
-        }
-      }
-    }
+    IOUtils.closeSafely(false, docOut, skipOut, freqOut, posOut, payloadOut);
   }
 
   public static void getExtensions(Set<String> extensions) {

Modified: lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/index/codecs/simpletext/SimpleTextFieldsReader.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/index/codecs/simpletext/SimpleTextFieldsReader.java?rev=1129631&r1=1129630&r2=1129631&view=diff
==============================================================================
--- lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/index/codecs/simpletext/SimpleTextFieldsReader.java (original)
+++ lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/index/codecs/simpletext/SimpleTextFieldsReader.java Tue May 31 11:25:37 2011
@@ -29,13 +29,14 @@ import org.apache.lucene.index.FieldInfo
 import org.apache.lucene.store.IndexInput;
 import org.apache.lucene.util.ArrayUtil;
 import org.apache.lucene.util.Bits;
+import org.apache.lucene.util.CharsRef;
 import org.apache.lucene.util.StringHelper;
 import org.apache.lucene.util.UnicodeUtil;
-import org.apache.lucene.util.automaton.fst.Builder;
-import org.apache.lucene.util.automaton.fst.BytesRefFSTEnum;
-import org.apache.lucene.util.automaton.fst.FST;
-import org.apache.lucene.util.automaton.fst.PositiveIntOutputs;
-import org.apache.lucene.util.automaton.fst.PairOutputs;
+import org.apache.lucene.util.fst.Builder;
+import org.apache.lucene.util.fst.BytesRefFSTEnum;
+import org.apache.lucene.util.fst.FST;
+import org.apache.lucene.util.fst.PositiveIntOutputs;
+import org.apache.lucene.util.fst.PairOutputs;
 
 import java.io.IOException;
 import java.util.Comparator;
@@ -236,7 +237,7 @@ class SimpleTextFieldsReader extends Fie
     private int tf;
     private Bits skipDocs;
     private final BytesRef scratch = new BytesRef(10);
-    private final UnicodeUtil.UTF16Result scratchUTF16 = new UnicodeUtil.UTF16Result();
+    private final CharsRef scratchUTF16 = new CharsRef(10);
     
     public SimpleTextDocsEnum() {
       this.inStart = SimpleTextFieldsReader.this.in;
@@ -286,7 +287,7 @@ class SimpleTextFieldsReader extends Fie
             return docID;
           }
           UnicodeUtil.UTF8toUTF16(scratch.bytes, scratch.offset+DOC.length, scratch.length-DOC.length, scratchUTF16);
-          docID = ArrayUtil.parseInt(scratchUTF16.result, 0, scratchUTF16.length);
+          docID = ArrayUtil.parseInt(scratchUTF16.chars, 0, scratchUTF16.length);
           termFreq = 0;
           first = false;
         } else if (scratch.startsWith(POS)) {
@@ -323,8 +324,8 @@ class SimpleTextFieldsReader extends Fie
     private Bits skipDocs;
     private final BytesRef scratch = new BytesRef(10);
     private final BytesRef scratch2 = new BytesRef(10);
-    private final UnicodeUtil.UTF16Result scratchUTF16 = new UnicodeUtil.UTF16Result();
-    private final UnicodeUtil.UTF16Result scratchUTF16_2 = new UnicodeUtil.UTF16Result();
+    private final CharsRef scratchUTF16 = new CharsRef(10);
+    private final CharsRef scratchUTF16_2 = new CharsRef(10);
     private BytesRef payload;
     private long nextDocStart;
 
@@ -368,7 +369,7 @@ class SimpleTextFieldsReader extends Fie
             return docID;
           }
           UnicodeUtil.UTF8toUTF16(scratch.bytes, scratch.offset+DOC.length, scratch.length-DOC.length, scratchUTF16);
-          docID = ArrayUtil.parseInt(scratchUTF16.result, 0, scratchUTF16.length);
+          docID = ArrayUtil.parseInt(scratchUTF16.chars, 0, scratchUTF16.length);
           tf = 0;
           posStart = in.getFilePointer();
           first = false;
@@ -400,7 +401,7 @@ class SimpleTextFieldsReader extends Fie
       readLine(in, scratch);
       assert scratch.startsWith(POS): "got line=" + scratch.utf8ToString();
       UnicodeUtil.UTF8toUTF16(scratch.bytes, scratch.offset+POS.length, scratch.length-POS.length, scratchUTF16_2);
-      final int pos = ArrayUtil.parseInt(scratchUTF16_2.result, 0, scratchUTF16_2.length);
+      final int pos = ArrayUtil.parseInt(scratchUTF16_2.chars, 0, scratchUTF16_2.length);
       final long fp = in.getFilePointer();
       readLine(in, scratch);
       if (scratch.startsWith(PAYLOAD)) {

Modified: lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/index/codecs/simpletext/SimpleTextFieldsWriter.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/index/codecs/simpletext/SimpleTextFieldsWriter.java?rev=1129631&r1=1129630&r2=1129631&view=diff
==============================================================================
--- lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/index/codecs/simpletext/SimpleTextFieldsWriter.java (original)
+++ lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/index/codecs/simpletext/SimpleTextFieldsWriter.java Tue May 31 11:25:37 2011
@@ -143,8 +143,11 @@ class SimpleTextFieldsWriter extends Fie
 
   @Override
   public void close() throws IOException {
-    write(END);
-    newline();
-    out.close();
+    try {
+      write(END);
+      newline();
+    } finally {
+      out.close();
+    }
   }
 }

Modified: lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/index/codecs/standard/StandardPostingsWriter.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/index/codecs/standard/StandardPostingsWriter.java?rev=1129631&r1=1129630&r2=1129631&view=diff
==============================================================================
--- lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/index/codecs/standard/StandardPostingsWriter.java (original)
+++ lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/index/codecs/standard/StandardPostingsWriter.java Tue May 31 11:25:37 2011
@@ -33,6 +33,7 @@ import org.apache.lucene.store.IndexOutp
 import org.apache.lucene.store.RAMOutputStream;
 import org.apache.lucene.util.BytesRef;
 import org.apache.lucene.util.CodecUtil;
+import org.apache.lucene.util.IOUtils;
 
 /** @lucene.experimental */
 public final class StandardPostingsWriter extends PostingsWriterBase {
@@ -42,8 +43,8 @@ public final class StandardPostingsWrite
   final static int VERSION_START = 0;
   final static int VERSION_CURRENT = VERSION_START;
 
-  final IndexOutput freqOut;
-  final IndexOutput proxOut;
+  IndexOutput freqOut;
+  IndexOutput proxOut;
   final DefaultSkipListWriter skipListWriter;
   /** Expert: The fraction of TermDocs entries stored in skip tables,
    * used to accelerate {@link DocsEnum#advance(int)}.  Larger values result in
@@ -85,31 +86,35 @@ public final class StandardPostingsWrite
   public StandardPostingsWriter(SegmentWriteState state) throws IOException {
     this(state, DEFAULT_SKIP_INTERVAL);
   }
+  
   public StandardPostingsWriter(SegmentWriteState state, int skipInterval) throws IOException {
-    super();
     this.skipInterval = skipInterval;
     this.skipMinimum = skipInterval; /* set to the same for now */
     //this.segment = state.segmentName;
     String fileName = IndexFileNames.segmentFileName(state.segmentName, state.codecIdAsString(), StandardCodec.FREQ_EXTENSION);
     freqOut = state.directory.createOutput(fileName);
-
-    if (state.fieldInfos.hasProx()) {
-      // At least one field does not omit TF, so create the
-      // prox file
-      fileName = IndexFileNames.segmentFileName(state.segmentName, state.codecIdAsString(), StandardCodec.PROX_EXTENSION);
-      proxOut = state.directory.createOutput(fileName);
-    } else {
-      // Every field omits TF so we will write no prox file
-      proxOut = null;
+    boolean success = false;
+    try {
+      if (state.fieldInfos.hasProx()) {
+        // At least one field does not omit TF, so create the
+        // prox file
+        fileName = IndexFileNames.segmentFileName(state.segmentName, state.codecIdAsString(), StandardCodec.PROX_EXTENSION);
+        proxOut = state.directory.createOutput(fileName);
+      } else {
+        // Every field omits TF so we will write no prox file
+        proxOut = null;
+      }
+      
+      totalNumDocs = state.numDocs;
+      
+      skipListWriter = new DefaultSkipListWriter(skipInterval, maxSkipLevels,
+          state.numDocs, freqOut, proxOut);
+      success = true;
+    } finally {
+      if (!success) {
+        IOUtils.closeSafely(true, freqOut, proxOut);
+      }
     }
-
-    totalNumDocs = state.numDocs;
-
-    skipListWriter = new DefaultSkipListWriter(skipInterval,
-                                               maxSkipLevels,
-                                               state.numDocs,
-                                               freqOut,
-                                               proxOut);
   }
 
   @Override
@@ -267,12 +272,6 @@ public final class StandardPostingsWrite
 
   @Override
   public void close() throws IOException {
-    try {
-      freqOut.close();
-    } finally {
-      if (proxOut != null) {
-        proxOut.close();
-      }
-    }
+    IOUtils.closeSafely(false, freqOut, proxOut);
   }
 }

Modified: lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/search/CachingCollector.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/search/CachingCollector.java?rev=1129631&r1=1129630&r2=1129631&view=diff
==============================================================================
--- lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/search/CachingCollector.java (original)
+++ lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/search/CachingCollector.java Tue May 31 11:25:37 2011
@@ -181,6 +181,7 @@ public abstract class CachingCollector e
             curUpto = 0;
           }
           cachedScorer.score = curScores[curUpto];
+          cachedScorer.doc = curDocs[curUpto];
           other.collect(curDocs[curUpto++]);
         }
       }

Modified: lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/store/FSDirectory.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/store/FSDirectory.java?rev=1129631&r1=1129630&r2=1129631&view=diff
==============================================================================
--- lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/store/FSDirectory.java (original)
+++ lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/store/FSDirectory.java Tue May 31 11:25:37 2011
@@ -22,8 +22,6 @@ import java.io.FileNotFoundException;
 import java.io.FilenameFilter;
 import java.io.IOException;
 import java.io.RandomAccessFile;
-import java.security.MessageDigest;
-import java.security.NoSuchAlgorithmException;
 
 import java.util.Collection;
 import static java.util.Collections.synchronizedSet;
@@ -111,15 +109,6 @@ import org.apache.lucene.util.Constants;
  * @see Directory
  */
 public abstract class FSDirectory extends Directory {
-  private final static MessageDigest DIGESTER;
-
-  static {
-    try {
-      DIGESTER = MessageDigest.getInstance("MD5");
-    } catch (NoSuchAlgorithmException e) {
-        throw new RuntimeException(e.toString(), e);
-    }
-  }
 
   /**
    * Default read chunk size.  This is a conditional default: on 32bit JVMs, it defaults to 100 MB.  On 64bit JVMs, it's
@@ -337,12 +326,6 @@ public abstract class FSDirectory extend
     return openInput(name, BufferedIndexInput.BUFFER_SIZE);
   }
 
-  /**
-   * So we can do some byte-to-hexchar conversion below
-   */
-  private static final char[] HEX_DIGITS =
-  {'0','1','2','3','4','5','6','7','8','9','a','b','c','d','e','f'};
-  
   @Override
   public String getLockID() {
     ensureOpen();
@@ -353,19 +336,12 @@ public abstract class FSDirectory extend
       throw new RuntimeException(e.toString(), e);
     }
 
-    byte digest[];
-    synchronized (DIGESTER) {
-      digest = DIGESTER.digest(dirName.getBytes());
-    }
-    StringBuilder buf = new StringBuilder();
-    buf.append("lucene-");
-    for (int i = 0; i < digest.length; i++) {
-      int b = digest[i];
-      buf.append(HEX_DIGITS[(b >> 4) & 0xf]);
-      buf.append(HEX_DIGITS[b & 0xf]);
+    int digest = 0;
+    for(int charIDX=0;charIDX<dirName.length();charIDX++) {
+      final char ch = dirName.charAt(charIDX);
+      digest = 31 * digest + ch;
     }
-
-    return buf.toString();
+    return "lucene-" + Integer.toHexString(digest);
   }
 
   /** Closes the store to future operations. */

Modified: lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/util/BytesRef.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/util/BytesRef.java?rev=1129631&r1=1129630&r2=1129631&view=diff
==============================================================================
--- lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/util/BytesRef.java (original)
+++ lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/util/BytesRef.java Tue May 31 11:25:37 2011
@@ -18,8 +18,6 @@ package org.apache.lucene.util;
  */
 
 import java.util.Comparator;
-import java.io.Serializable;
-import java.io.UnsupportedEncodingException;
 
 /** Represents byte[], as a slice (offset + length) into an
  *  existing byte[].
@@ -123,6 +121,7 @@ public final class BytesRef implements C
   public void copy(char text[], int offset, int length) {
     UnicodeUtil.UTF16toUTF8(text, offset, length, this);
   }
+  
   public boolean bytesEquals(BytesRef other) {
     if (length == other.length) {
       int otherUpto = other.offset;
@@ -199,13 +198,15 @@ public final class BytesRef implements C
   /** Interprets stored bytes as UTF8 bytes, returning the
    *  resulting string */
   public String utf8ToString() {
-    try {
-      return new String(bytes, offset, length, "UTF-8");
-    } catch (UnsupportedEncodingException uee) {
-      // should not happen -- UTF8 is presumably supported
-      // by all JREs
-      throw new RuntimeException(uee);
-    }
+    final CharsRef ref = new CharsRef(length);
+    UnicodeUtil.UTF8toUTF16(bytes, offset, length, ref);
+    return ref.toString(); 
+  }
+  
+  /** Interprets stored bytes as UTF8 bytes into the given {@link CharsRef} */
+  public CharsRef utf8ToChars(CharsRef ref) {
+    UnicodeUtil.UTF8toUTF16(bytes, offset, length, ref);
+    return ref;
   }
 
   /** Returns hex encoded bytes, eg [0x6c 0x75 0x63 0x65 0x6e 0x65] */
@@ -272,13 +273,12 @@ public final class BytesRef implements C
   }
 
   private final static Comparator<BytesRef> utf8SortedAsUnicodeSortOrder = new UTF8SortedAsUnicodeComparator();
-  
+
   public static Comparator<BytesRef> getUTF8SortedAsUnicodeComparator() {
     return utf8SortedAsUnicodeSortOrder;
   }
 
-  @SuppressWarnings("serial") // serializable to work with contrib/remote
-  private static final class UTF8SortedAsUnicodeComparator implements Serializable, Comparator<BytesRef> {
+  private static class UTF8SortedAsUnicodeComparator implements Comparator<BytesRef> {
     // Only singleton
     private UTF8SortedAsUnicodeComparator() {};
 

Modified: lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/util/Constants.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/util/Constants.java?rev=1129631&r1=1129630&r2=1129631&view=diff
==============================================================================
--- lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/util/Constants.java (original)
+++ lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/util/Constants.java Tue May 31 11:25:37 2011
@@ -43,6 +43,8 @@ public final class Constants {
   public static final boolean WINDOWS = OS_NAME.startsWith("Windows");
   /** True iff running on SunOS. */
   public static final boolean SUN_OS = OS_NAME.startsWith("SunOS");
+  /** True iff running on Mac OS X */
+  public static final boolean MAC_OS_X = OS_NAME.startsWith("Mac OS X");
 
   public static final String OS_ARCH = System.getProperty("os.arch");
   public static final String OS_VERSION = System.getProperty("os.version");

Modified: lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/util/IOUtils.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/util/IOUtils.java?rev=1129631&r1=1129630&r2=1129631&view=diff
==============================================================================
--- lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/util/IOUtils.java (original)
+++ lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/util/IOUtils.java Tue May 31 11:25:37 2011
@@ -47,45 +47,113 @@ public final class IOUtils {
    * @param objects         objects to call <tt>close()</tt> on
    */
   public static <E extends Exception> void closeSafely(E priorException, Closeable... objects) throws E, IOException {
-    IOException firstIOE = null;
+    Throwable th = null;
 
     for (Closeable object : objects) {
       try {
-        if (object != null)
+        if (object != null) {
           object.close();
-      } catch (IOException ioe) {
-        if (firstIOE == null)
-          firstIOE = ioe;
+        }
+      } catch (Throwable t) {
+        if (th == null) {
+          th = t;
+        }
       }
     }
 
     if (priorException != null) {
       throw priorException;
-    } else if (firstIOE != null) {
-      throw firstIOE;
+    } else if (th != null) {
+      if (th instanceof IOException) throw (IOException) th;
+      if (th instanceof RuntimeException) throw (RuntimeException) th;
+      if (th instanceof Error) throw (Error) th;
+      throw new RuntimeException(th);
+    }
+  }
+
+  /** @see #closeSafely(Exception, Closeable...) */
+  public static <E extends Exception> void closeSafely(E priorException, Iterable<Closeable> objects) throws E, IOException {
+    Throwable th = null;
+
+    for (Closeable object : objects) {
+      try {
+        if (object != null) {
+          object.close();
+        }
+      } catch (Throwable t) {
+        if (th == null) {
+          th = t;
+        }
+      }
+    }
+
+    if (priorException != null) {
+      throw priorException;
+    } else if (th != null) {
+      if (th instanceof IOException) throw (IOException) th;
+      if (th instanceof RuntimeException) throw (RuntimeException) th;
+      if (th instanceof Error) throw (Error) th;
+      throw new RuntimeException(th);
     }
   }
 
   /**
-   * <p>Closes all given <tt>Closeable</tt>s, suppressing all thrown exceptions. Some of the <tt>Closeable</tt>s
-   * may be null, they are ignored. After everything is closed, method either throws the first of suppressed exceptions,
-   * or completes normally.</p>
-   * @param objects         objects to call <tt>close()</tt> on
+   * Closes all given <tt>Closeable</tt>s, suppressing all thrown exceptions.
+   * Some of the <tt>Closeable</tt>s may be null, they are ignored. After
+   * everything is closed, and if {@code suppressExceptions} is {@code false},
+   * method either throws the first of suppressed exceptions, or completes
+   * normally.
+   * 
+   * @param suppressExceptions
+   *          if true then exceptions that occur during close() are suppressed
+   * @param objects
+   *          objects to call <tt>close()</tt> on
    */
-  public static void closeSafely(Closeable... objects) throws IOException {
-    IOException firstIOE = null;
+  public static void closeSafely(boolean suppressExceptions, Closeable... objects) throws IOException {
+    Throwable th = null;
 
     for (Closeable object : objects) {
       try {
-        if (object != null)
+        if (object != null) {
           object.close();
-      } catch (IOException ioe) {
-        if (firstIOE == null)
-          firstIOE = ioe;
+        }
+      } catch (Throwable t) {
+        if (th == null)
+          th = t;
       }
     }
 
-    if (firstIOE != null)
-      throw firstIOE;
+    if (th != null && !suppressExceptions) {
+      if (th instanceof IOException) throw (IOException) th;
+      if (th instanceof RuntimeException) throw (RuntimeException) th;
+      if (th instanceof Error) throw (Error) th;
+      throw new RuntimeException(th);
+    }
   }
+  
+  /**
+   * @see #closeSafely(boolean, Closeable...)
+   */
+  public static void closeSafely(boolean suppressExceptions, Iterable<? extends Closeable> objects) throws IOException {
+    Throwable th = null;
+
+    for (Closeable object : objects) {
+      try {
+        if (object != null) {
+          object.close();
+        }
+      } catch (Throwable t) {
+        if (th == null)
+          th = t;
+      }
+    }
+
+    if (th != null && !suppressExceptions) {
+      if (th instanceof IOException) throw (IOException) th;
+      if (th instanceof RuntimeException) throw (RuntimeException) th;
+      if (th instanceof Error) throw (Error) th;
+      throw new RuntimeException(th);
+    }
+  }
+
 }

Modified: lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/util/OpenBitSet.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/util/OpenBitSet.java?rev=1129631&r1=1129630&r2=1129631&view=diff
==============================================================================
--- lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/util/OpenBitSet.java (original)
+++ lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/util/OpenBitSet.java Tue May 31 11:25:37 2011
@@ -78,11 +78,15 @@ public class OpenBitSet extends DocIdSet
   protected long[] bits;
   protected int wlen;   // number of words (elements) used in the array
 
+  // Used only for assert:
+  private long numBits;
+
   /** Constructs an OpenBitSet large enough to hold numBits.
    *
    * @param numBits
    */
   public OpenBitSet(long numBits) {
+    this.numBits = numBits;
     bits = new long[bits2words(numBits)];
     wlen = bits.length;
   }
@@ -107,6 +111,7 @@ public class OpenBitSet extends DocIdSet
   public OpenBitSet(long[] bits, int numWords) {
     this.bits = bits;
     this.wlen = numWords;
+    this.numBits = wlen * 64;
   }
   
   @Override
@@ -170,6 +175,7 @@ public class OpenBitSet extends DocIdSet
    * The index should be less than the OpenBitSet size
    */
   public boolean fastGet(int index) {
+    assert index >= 0 && index < numBits;
     int i = index >> 6;               // div 64
     // signed shift will keep a negative index and force an
     // array-index-out-of-bounds-exception, removing the need for an explicit check.
@@ -194,6 +200,7 @@ public class OpenBitSet extends DocIdSet
    * The index should be less than the OpenBitSet size.
    */
   public boolean fastGet(long index) {
+    assert index >= 0 && index < numBits;
     int i = (int)(index >> 6);               // div 64
     int bit = (int)index & 0x3f;           // mod 64
     long bitmask = 1L << bit;
@@ -217,6 +224,7 @@ public class OpenBitSet extends DocIdSet
    * The index should be less than the OpenBitSet size
    */
   public int getBit(int index) {
+    assert index >= 0 && index < numBits;
     int i = index >> 6;                // div 64
     int bit = index & 0x3f;            // mod 64
     return ((int)(bits[i]>>>bit)) & 0x01;
@@ -245,6 +253,7 @@ public class OpenBitSet extends DocIdSet
   * The index should be less than the OpenBitSet size.
   */
   public void fastSet(int index) {
+    assert index >= 0 && index < numBits;
     int wordNum = index >> 6;      // div 64
     int bit = index & 0x3f;     // mod 64
     long bitmask = 1L << bit;
@@ -255,6 +264,7 @@ public class OpenBitSet extends DocIdSet
   * The index should be less than the OpenBitSet size.
   */
   public void fastSet(long index) {
+    assert index >= 0 && index < numBits;
     int wordNum = (int)(index >> 6);
     int bit = (int)index & 0x3f;
     long bitmask = 1L << bit;
@@ -296,6 +306,7 @@ public class OpenBitSet extends DocIdSet
       ensureCapacity(index+1);
       wlen = wordNum+1;
     }
+    assert (numBits = Math.max(numBits, index+1)) >= 0;
     return wordNum;
   }
 
@@ -304,6 +315,7 @@ public class OpenBitSet extends DocIdSet
    * The index should be less than the OpenBitSet size.
    */
   public void fastClear(int index) {
+    assert index >= 0 && index < numBits;
     int wordNum = index >> 6;
     int bit = index & 0x03f;
     long bitmask = 1L << bit;
@@ -321,6 +333,7 @@ public class OpenBitSet extends DocIdSet
    * The index should be less than the OpenBitSet size.
    */
   public void fastClear(long index) {
+    assert index >= 0 && index < numBits;
     int wordNum = (int)(index >> 6); // div 64
     int bit = (int)index & 0x3f;     // mod 64
     long bitmask = 1L << bit;
@@ -415,6 +428,7 @@ public class OpenBitSet extends DocIdSet
    * The index should be less than the OpenBitSet size.
    */
   public boolean getAndSet(int index) {
+    assert index >= 0 && index < numBits;
     int wordNum = index >> 6;      // div 64
     int bit = index & 0x3f;     // mod 64
     long bitmask = 1L << bit;
@@ -427,6 +441,7 @@ public class OpenBitSet extends DocIdSet
    * The index should be less than the OpenBitSet size.
    */
   public boolean getAndSet(long index) {
+    assert index >= 0 && index < numBits;
     int wordNum = (int)(index >> 6);      // div 64
     int bit = (int)index & 0x3f;     // mod 64
     long bitmask = 1L << bit;
@@ -439,6 +454,7 @@ public class OpenBitSet extends DocIdSet
    * The index should be less than the OpenBitSet size.
    */
   public void fastFlip(int index) {
+    assert index >= 0 && index < numBits;
     int wordNum = index >> 6;      // div 64
     int bit = index & 0x3f;     // mod 64
     long bitmask = 1L << bit;
@@ -449,6 +465,7 @@ public class OpenBitSet extends DocIdSet
    * The index should be less than the OpenBitSet size.
    */
   public void fastFlip(long index) {
+    assert index >= 0 && index < numBits;
     int wordNum = (int)(index >> 6);   // div 64
     int bit = (int)index & 0x3f;       // mod 64
     long bitmask = 1L << bit;
@@ -467,6 +484,7 @@ public class OpenBitSet extends DocIdSet
    * The index should be less than the OpenBitSet size.
    */
   public boolean flipAndGet(int index) {
+    assert index >= 0 && index < numBits;
     int wordNum = index >> 6;      // div 64
     int bit = index & 0x3f;     // mod 64
     long bitmask = 1L << bit;
@@ -478,6 +496,7 @@ public class OpenBitSet extends DocIdSet
    * The index should be less than the OpenBitSet size.
    */
   public boolean flipAndGet(long index) {
+    assert index >= 0 && index < numBits;
     int wordNum = (int)(index >> 6);   // div 64
     int bit = (int)index & 0x3f;       // mod 64
     long bitmask = 1L << bit;
@@ -674,6 +693,7 @@ public class OpenBitSet extends DocIdSet
   public void union(OpenBitSet other) {
     int newLen = Math.max(wlen,other.wlen);
     ensureCapacityWords(newLen);
+    assert (numBits = Math.max(other.numBits, numBits)) >= 0;
 
     long[] thisArr = this.bits;
     long[] otherArr = other.bits;
@@ -702,6 +722,7 @@ public class OpenBitSet extends DocIdSet
   public void xor(OpenBitSet other) {
     int newLen = Math.max(wlen,other.wlen);
     ensureCapacityWords(newLen);
+    assert (numBits = Math.max(other.numBits, numBits)) >= 0;
 
     long[] thisArr = this.bits;
     long[] otherArr = other.bits;

Modified: lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/util/UnicodeUtil.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/util/UnicodeUtil.java?rev=1129631&r1=1129630&r2=1129631&view=diff
==============================================================================
--- lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/util/UnicodeUtil.java (original)
+++ lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/util/UnicodeUtil.java Tue May 31 11:25:37 2011
@@ -94,6 +94,19 @@ package org.apache.lucene.util;
  */
 
 public final class UnicodeUtil {
+  
+  /** A binary term consisting of a number of 0xff bytes, likely to be bigger than other terms
+   *  one would normally encounter, and definitely bigger than any UTF-8 terms.
+   *  <p>
+   *  WARNING: This is not a valid UTF8 Term  
+   **/
+  public static final BytesRef BIG_TERM = new BytesRef(
+      new byte[] {-1,-1,-1,-1,-1,-1,-1,-1,-1,-1}
+  ); // TODO this is unrelated here find a better place for it
+  
+  public static void main(String[] args) {
+    System.out.println(Character.toChars(0x10FFFF + 1));
+  }
 
   private UnicodeUtil() {} // no instance
 
@@ -112,33 +125,6 @@ public final class UnicodeUtil {
     Character.MIN_SUPPLEMENTARY_CODE_POINT - 
     (UNI_SUR_HIGH_START << HALF_SHIFT) - UNI_SUR_LOW_START;
 
-  /**
-   * @lucene.internal
-   */
-  public static final class UTF16Result {
-    public char[] result = new char[10];
-    public int[] offsets = new int[10];
-    public int length;
-
-    public void setLength(int newLength) {
-      if (result.length < newLength)
-        result = ArrayUtil.grow(result, newLength);
-      length = newLength;
-    }
-
-    public void copyText(UTF16Result other) {
-      setLength(other.length);
-      System.arraycopy(other.result, 0, result, 0, length);
-    }
-
-    public void copyText(String other) {
-      final int otherLength = other.length();
-      setLength(otherLength);
-      other.getChars(0, otherLength, result, 0);
-      length = otherLength;
-    }
-  }
-
   /** Encode characters from a char[] source, starting at
    *  offset for length chars.  Returns a hash of the resulting bytes.  After encoding, result.offset will always be 0. */
   public static int UTF16toUTF8WithHash(final char[] source, final int offset, final int length, BytesRef result) {
@@ -302,135 +288,6 @@ public final class UnicodeUtil {
     result.length = upto;
   }
 
-  /** Convert UTF8 bytes into UTF16 characters.  If offset
-   *  is non-zero, conversion starts at that starting point
-   *  in utf8, re-using the results from the previous call
-   *  up until offset. */
-  public static void UTF8toUTF16(final byte[] utf8, final int offset, final int length, final UTF16Result result) {
-
-    final int end = offset + length;
-    char[] out = result.result;
-    if (result.offsets.length <= end) {
-      result.offsets = ArrayUtil.grow(result.offsets, end+1);
-    }
-    final int[] offsets = result.offsets;
-
-    // If incremental decoding fell in the middle of a
-    // single unicode character, rollback to its start:
-    int upto = offset;
-    while(offsets[upto] == -1)
-      upto--;
-
-    int outUpto = offsets[upto];
-
-    // Pre-allocate for worst case 1-for-1
-    if (outUpto+length >= out.length) {
-      out = result.result = ArrayUtil.grow(out, outUpto+length+1);
-    }
-
-    while (upto < end) {
-
-      final int b = utf8[upto]&0xff;
-      final int ch;
-
-      offsets[upto++] = outUpto;
-
-      if (b < 0xc0) {
-        assert b < 0x80;
-        ch = b;
-      } else if (b < 0xe0) {
-        ch = ((b&0x1f)<<6) + (utf8[upto]&0x3f);
-        offsets[upto++] = -1;
-      } else if (b < 0xf0) {
-        ch = ((b&0xf)<<12) + ((utf8[upto]&0x3f)<<6) + (utf8[upto+1]&0x3f);
-        offsets[upto++] = -1;
-        offsets[upto++] = -1;
-      } else {
-        assert b < 0xf8;
-        ch = ((b&0x7)<<18) + ((utf8[upto]&0x3f)<<12) + ((utf8[upto+1]&0x3f)<<6) + (utf8[upto+2]&0x3f);
-        offsets[upto++] = -1;
-        offsets[upto++] = -1;
-        offsets[upto++] = -1;
-      }
-
-      if (ch <= UNI_MAX_BMP) {
-        // target is a character <= 0xFFFF
-        out[outUpto++] = (char) ch;
-      } else {
-        // target is a character in range 0xFFFF - 0x10FFFF
-        out[outUpto++] = (char) ((ch >> HALF_SHIFT) + 0xD7C0 /* UNI_SUR_HIGH_START - 64 */);
-        out[outUpto++] = (char) ((ch & HALF_MASK) + UNI_SUR_LOW_START);
-      }
-    }
-    offsets[upto] = outUpto;
-    result.length = outUpto;
-  }
-
-  /**
-   * Get the next valid UTF-16 String in UTF-16 order.
-   * <p>
-   * If the input String is already valid, it is returned.
-   * Otherwise the next String in code unit order is returned.
-   * </p>
-   * @param s input String (possibly with unpaired surrogates)
-   * @return next valid UTF-16 String in UTF-16 order
-   */
-  public static String nextValidUTF16String(String s) {
-    if (validUTF16String(s))
-        return s;
-    else {
-      UTF16Result chars = new UTF16Result();
-      chars.copyText(s);
-      nextValidUTF16String(chars);
-      return new String(chars.result, 0, chars.length);
-    }
-  }
-  
-  public static void nextValidUTF16String(UTF16Result s) {
-    final int size = s.length;
-    for (int i = 0; i < size; i++) {
-      char ch = s.result[i];
-      if (ch >= UnicodeUtil.UNI_SUR_HIGH_START
-          && ch <= UnicodeUtil.UNI_SUR_HIGH_END) {
-        if (i < size - 1) {
-          i++;
-          char nextCH = s.result[i];
-          if (nextCH >= UnicodeUtil.UNI_SUR_LOW_START
-              && nextCH <= UnicodeUtil.UNI_SUR_LOW_END) {
-            // Valid surrogate pair
-          } else
-          // Unmatched high surrogate
-            if (nextCH < UnicodeUtil.UNI_SUR_LOW_START) { // SMP not enumerated
-              s.setLength(i + 1);
-              s.result[i] = (char) UnicodeUtil.UNI_SUR_LOW_START;             
-              return;
-            } else { // SMP already enumerated
-              if (s.result[i - 1] == UnicodeUtil.UNI_SUR_HIGH_END) {
-                s.result[i - 1] = (char) (UnicodeUtil.UNI_SUR_LOW_END + 1);
-                s.setLength(i);               
-              } else {
-                s.result[i - 1]++;
-                s.result[i] = (char) UnicodeUtil.UNI_SUR_LOW_START;
-                s.setLength(i + 1);
-              }            
-              return;
-            }
-        } else {
-        // Unmatched high surrogate in final position, SMP not yet enumerated
-          s.setLength(i + 2);
-          s.result[i + 1] = (char) UnicodeUtil.UNI_SUR_LOW_START;
-          return;
-        }
-      } else if (ch >= UnicodeUtil.UNI_SUR_LOW_START
-          && ch <= UnicodeUtil.UNI_SUR_LOW_END) {
-      // Unmatched low surrogate, SMP already enumerated
-        s.setLength(i + 1);
-        s.result[i] = (char) (UnicodeUtil.UNI_SUR_LOW_END + 1);
-        return;
-      }
-    }
-  }
-  
   // Only called from assert
   /*
   private static boolean matches(char[] source, int offset, int length, byte[] result, int upto) {
@@ -705,4 +562,51 @@ public final class UnicodeUtil {
     }
     return sb.toString();
   }
+  
+  /**
+   * Interprets the given byte array as UTF-8 and converts to UTF-16. The {@link CharsRef} will be extended if 
+   * it doesn't provide enough space to hold the worst case of each byte becoming a UTF-16 codepoint.
+   * <p>
+   * NOTE: Full characters are read, even if this reads past the length passed (and
+   * can result in an ArrayOutOfBoundsException if invalid UTF-8 is passed).
+   * Explicit checks for valid UTF-8 are not performed. 
+   */
+  public static void UTF8toUTF16(byte[] utf8, int offset, int length, CharsRef chars) {
+    int out_offset = chars.offset = 0;
+    final char[] out = chars.chars =  ArrayUtil.grow(chars.chars, length);
+    final int limit = offset + length;
+    while (offset < limit) {
+      int b = utf8[offset++]&0xff;
+      if (b < 0xc0) {
+        assert b < 0x80;
+        out[out_offset++] = (char)b;
+      } else if (b < 0xe0) {
+        out[out_offset++] = (char)(((b&0x1f)<<6) + (utf8[offset++]&0x3f));
+      } else if (b < 0xf0) {
+        out[out_offset++] = (char)(((b&0xf)<<12) + ((utf8[offset]&0x3f)<<6) + (utf8[offset+1]&0x3f));
+        offset += 2;
+      } else {
+        assert b < 0xf8;
+        int ch = ((b&0x7)<<18) + ((utf8[offset]&0x3f)<<12) + ((utf8[offset+1]&0x3f)<<6) + (utf8[offset+2]&0x3f);
+        offset += 3;
+        if (ch < UNI_MAX_BMP) {
+          out[out_offset++] = (char)ch;
+        } else {
+          int chHalf = ch - 0x0010000;
+          out[out_offset++] = (char) ((chHalf >> 10) + 0xD800);
+          out[out_offset++] = (char) ((chHalf & HALF_MASK) + 0xDC00);          
+        }
+      }
+    }
+    chars.length = out_offset - chars.offset;
+  }
+  
+  /**
+   * Utility method for {@link #UTF8toUTF16(byte[], int, int, CharsRef)}
+   * @see #UTF8toUTF16(byte[], int, int, CharsRef)
+   */
+  public static void UTF8toUTF16(BytesRef bytesRef, CharsRef chars) {
+    UTF8toUTF16(bytesRef.bytes, bytesRef.offset, bytesRef.length, chars);
+  }
+
 }

Modified: lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/util/Version.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/util/Version.java?rev=1129631&r1=1129630&r2=1129631&view=diff
==============================================================================
--- lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/util/Version.java (original)
+++ lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/util/Version.java Tue May 31 11:25:37 2011
@@ -27,6 +27,9 @@ package org.apache.lucene.util;
  * change the version at search-time, but instead also adjust
  * your indexing code to match, and re-index.
  */
+// remove me when java 5 is no longer supported
+// this is a workaround for a JDK bug that wrongly emits a warning.
+@SuppressWarnings("dep-ann")
 public enum Version {
   /**
    * Match settings and bugs in Lucene's 3.0 release.
@@ -48,6 +51,13 @@ public enum Version {
    */
   @Deprecated
   LUCENE_32,
+  
+  /**
+   * Match settings and bugs in Lucene's 3.3 release.
+   * @deprecated (4.0) Use latest
+   */
+  @Deprecated
+  LUCENE_33,
 
   /** Match settings and bugs in Lucene's 4.0 release. 
    *  <p>

Modified: lucene/dev/branches/docvalues/lucene/src/site/src/documentation/content/xdocs/fileformats.xml
URL: http://svn.apache.org/viewvc/lucene/dev/branches/docvalues/lucene/src/site/src/documentation/content/xdocs/fileformats.xml?rev=1129631&r1=1129630&r2=1129631&view=diff
==============================================================================
--- lucene/dev/branches/docvalues/lucene/src/site/src/documentation/content/xdocs/fileformats.xml (original)
+++ lucene/dev/branches/docvalues/lucene/src/site/src/documentation/content/xdocs/fileformats.xml Tue May 31 11:25:37 2011
@@ -12,7 +12,7 @@
 
             <p>
                 This document defines the index file formats used
-                in Lucene version 4.0. If you are using a different
+                in this version of Lucene. If you are using a different
                 version of Lucene, please consult the copy of
                 <code>docs/fileformats.html</code>
                 that was distributed

Modified: lucene/dev/branches/docvalues/lucene/src/site/src/documentation/content/xdocs/index.xml
URL: http://svn.apache.org/viewvc/lucene/dev/branches/docvalues/lucene/src/site/src/documentation/content/xdocs/index.xml?rev=1129631&r1=1129630&r2=1129631&view=diff
==============================================================================
--- lucene/dev/branches/docvalues/lucene/src/site/src/documentation/content/xdocs/index.xml (original)
+++ lucene/dev/branches/docvalues/lucene/src/site/src/documentation/content/xdocs/index.xml Tue May 31 11:25:37 2011
@@ -6,7 +6,7 @@
   
   <body>
     <p>
-    This is the official documentation for Lucene Java 4.0<BR/>
+    This is the official documentation for Lucene Java<BR/>
 	Please use the menu on the left to access the Javadocs and different documents.
     </p>
 	<p>

Modified: lucene/dev/branches/docvalues/lucene/src/site/src/documentation/content/xdocs/systemrequirements.xml
URL: http://svn.apache.org/viewvc/lucene/dev/branches/docvalues/lucene/src/site/src/documentation/content/xdocs/systemrequirements.xml?rev=1129631&r1=1129630&r2=1129631&view=diff
==============================================================================
--- lucene/dev/branches/docvalues/lucene/src/site/src/documentation/content/xdocs/systemrequirements.xml (original)
+++ lucene/dev/branches/docvalues/lucene/src/site/src/documentation/content/xdocs/systemrequirements.xml Tue May 31 11:25:37 2011
@@ -11,7 +11,7 @@
    </section> -->
 <section id="System Requirements"><title>System Requirements</title>
 <p>
-    Lucene Java 4.0 has the following minimum requirements:
+    Lucene Java has the following minimum requirements:
     <ul>
         <li>Java 1.5.x or greater.</li>
         <li>ANT 1.7.0 or greater.</li>

Modified: lucene/dev/branches/docvalues/lucene/src/test-framework/org/apache/lucene/index/RandomIndexWriter.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/docvalues/lucene/src/test-framework/org/apache/lucene/index/RandomIndexWriter.java?rev=1129631&r1=1129630&r2=1129631&view=diff
==============================================================================
--- lucene/dev/branches/docvalues/lucene/src/test-framework/org/apache/lucene/index/RandomIndexWriter.java (original)
+++ lucene/dev/branches/docvalues/lucene/src/test-framework/org/apache/lucene/index/RandomIndexWriter.java Tue May 31 11:25:37 2011
@@ -19,6 +19,7 @@ package org.apache.lucene.index;
 
 import java.io.Closeable;
 import java.io.IOException;
+import java.util.Iterator;
 import java.util.Random;
 
 import org.apache.lucene.analysis.Analyzer;
@@ -120,11 +121,47 @@ public class RandomIndexWriter implement
    * Adds a Document.
    * @see IndexWriter#addDocument(Document)
    */
-  public void addDocument(Document doc) throws IOException {
+  public void addDocument(final Document doc) throws IOException {
     if (doDocValues) {
       randomPerDocFieldValues(r, doc);
     }
-    w.addDocument(doc);
+
+    if (r.nextInt(5) == 3) {
+      // TODO: maybe, we should simply buffer up added docs
+      // (but we need to clone them), and only when
+      // getReader, commit, etc. are called, we do an
+      // addDocuments?  Would be better testing.
+      w.addDocuments(new Iterable<Document>() {
+
+        // @Override -- not until Java 1.6
+        public Iterator<Document> iterator() {
+          return new Iterator<Document>() {
+            boolean done;
+            
+            // @Override -- not until Java 1.6
+            public boolean hasNext() {
+              return !done;
+            }
+
+            // @Override -- not until Java 1.6
+            public void remove() {
+              throw new UnsupportedOperationException();
+            }
+
+            // @Override -- not until Java 1.6
+            public Document next() {
+              if (done) {
+                throw new IllegalStateException();
+              }
+              done = true;
+              return doc;
+            }
+          };
+        }
+        });
+    } else {
+      w.addDocument(doc);
+    }
     
     maybeCommit();
   }
@@ -188,15 +225,57 @@ public class RandomIndexWriter implement
     }
   }
   
+  public void addDocuments(Iterable<Document> docs) throws IOException {
+    w.addDocuments(docs);
+    maybeCommit();
+  }
+
+  public void updateDocuments(Term delTerm, Iterable<Document> docs) throws IOException {
+    w.updateDocuments(delTerm, docs);
+    maybeCommit();
+  }
+
   /**
    * Updates a document.
    * @see IndexWriter#updateDocument(Term, Document)
    */
-  public void updateDocument(Term t, Document doc) throws IOException {
+  public void updateDocument(final Term t, final Document doc) throws IOException {
     if (doDocValues) {
       randomPerDocFieldValues(r, doc);
     }
-    w.updateDocument(t, doc);
+    
+    if (r.nextInt(5) == 3) {
+      w.updateDocuments(t, new Iterable<Document>() {
+
+        // @Override -- not until Java 1.6
+        public Iterator<Document> iterator() {
+          return new Iterator<Document>() {
+            boolean done;
+            
+            // @Override -- not until Java 1.6
+            public boolean hasNext() {
+              return !done;
+            }
+
+            // @Override -- not until Java 1.6
+            public void remove() {
+              throw new UnsupportedOperationException();
+            }
+
+            // @Override -- not until Java 1.6
+            public Document next() {
+              if (done) {
+                throw new IllegalStateException();
+              }
+              done = true;
+              return doc;
+            }
+          };
+        }
+        });
+    } else {
+      w.updateDocument(t, doc);
+    }
     maybeCommit();
   }
   

Modified: lucene/dev/branches/docvalues/lucene/src/test-framework/org/apache/lucene/index/codecs/mockintblock/MockFixedIntBlockCodec.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/docvalues/lucene/src/test-framework/org/apache/lucene/index/codecs/mockintblock/MockFixedIntBlockCodec.java?rev=1129631&r1=1129630&r2=1129631&view=diff
==============================================================================
--- lucene/dev/branches/docvalues/lucene/src/test-framework/org/apache/lucene/index/codecs/mockintblock/MockFixedIntBlockCodec.java (original)
+++ lucene/dev/branches/docvalues/lucene/src/test-framework/org/apache/lucene/index/codecs/mockintblock/MockFixedIntBlockCodec.java Tue May 31 11:25:37 2011
@@ -50,6 +50,7 @@ import org.apache.lucene.index.codecs.Te
 import org.apache.lucene.index.codecs.standard.StandardCodec;
 import org.apache.lucene.store.*;
 import org.apache.lucene.util.BytesRef;
+import org.apache.lucene.util.IOUtils;
 
 /**
  * A silly test codec to verify core support for fixed
@@ -103,15 +104,25 @@ public class MockFixedIntBlockCodec exte
 
     @Override
     public IntIndexOutput createOutput(Directory dir, String fileName) throws IOException {
-      return new FixedIntBlockIndexOutput(dir.createOutput(fileName), blockSize) {
-        @Override
-        protected void flushBlock() throws IOException {
-          for(int i=0;i<buffer.length;i++) {
-            assert buffer[i] >= 0;
-            out.writeVInt(buffer[i]);
+      IndexOutput out = dir.createOutput(fileName);
+      boolean success = false;
+      try {
+        FixedIntBlockIndexOutput ret = new FixedIntBlockIndexOutput(out, blockSize) {
+          @Override
+          protected void flushBlock() throws IOException {
+            for(int i=0;i<buffer.length;i++) {
+              assert buffer[i] >= 0;
+              out.writeVInt(buffer[i]);
+            }
           }
+        };
+        success = true;
+        return ret;
+      } finally {
+        if (!success) {
+          IOUtils.closeSafely(true, out);
         }
-      };
+      }
     }
   }
 

Modified: lucene/dev/branches/docvalues/lucene/src/test-framework/org/apache/lucene/index/codecs/mockintblock/MockVariableIntBlockCodec.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/docvalues/lucene/src/test-framework/org/apache/lucene/index/codecs/mockintblock/MockVariableIntBlockCodec.java?rev=1129631&r1=1129630&r2=1129631&view=diff
==============================================================================
--- lucene/dev/branches/docvalues/lucene/src/test-framework/org/apache/lucene/index/codecs/mockintblock/MockVariableIntBlockCodec.java (original)
+++ lucene/dev/branches/docvalues/lucene/src/test-framework/org/apache/lucene/index/codecs/mockintblock/MockVariableIntBlockCodec.java Tue May 31 11:25:37 2011
@@ -52,6 +52,7 @@ import org.apache.lucene.store.Directory
 import org.apache.lucene.store.IndexInput;
 import org.apache.lucene.store.IndexOutput;
 import org.apache.lucene.util.BytesRef;
+import org.apache.lucene.util.IOUtils;
 
 /**
  * A silly test codec to verify core support for variable
@@ -108,34 +109,42 @@ public class MockVariableIntBlockCodec e
     @Override
     public IntIndexOutput createOutput(Directory dir, String fileName) throws IOException {
       final IndexOutput out = dir.createOutput(fileName);
-      out.writeInt(baseBlockSize);
-      return new VariableIntBlockIndexOutput(out, 2*baseBlockSize) {
-
-        int pendingCount;
-        final int[] buffer = new int[2+2*baseBlockSize];
-
-        @Override
-        protected int add(int value) throws IOException {
-          assert value >= 0;
-          buffer[pendingCount++] = value;
-          // silly variable block length int encoder: if
-          // first value <= 3, we write N vints at once;
-          // else, 2*N
-          final int flushAt = buffer[0] <= 3 ? baseBlockSize : 2*baseBlockSize;
-
-          // intentionally be non-causal here:
-          if (pendingCount == flushAt+1) {
-            for(int i=0;i<flushAt;i++) {
-              out.writeVInt(buffer[i]);
+      boolean success = false;
+      try {
+        out.writeInt(baseBlockSize);
+        VariableIntBlockIndexOutput ret = new VariableIntBlockIndexOutput(out, 2*baseBlockSize) {
+          int pendingCount;
+          final int[] buffer = new int[2+2*baseBlockSize];
+          
+          @Override
+          protected int add(int value) throws IOException {
+            assert value >= 0;
+            buffer[pendingCount++] = value;
+            // silly variable block length int encoder: if
+            // first value <= 3, we write N vints at once;
+            // else, 2*N
+            final int flushAt = buffer[0] <= 3 ? baseBlockSize : 2*baseBlockSize;
+            
+            // intentionally be non-causal here:
+            if (pendingCount == flushAt+1) {
+              for(int i=0;i<flushAt;i++) {
+                out.writeVInt(buffer[i]);
+              }
+              buffer[0] = buffer[flushAt];
+              pendingCount = 1;
+              return flushAt;
+            } else {
+              return 0;
             }
-            buffer[0] = buffer[flushAt];
-            pendingCount = 1;
-            return flushAt;
-          } else {
-            return 0;
           }
+        };
+        success = true;
+        return ret;
+      } finally {
+        if (!success) {
+          IOUtils.closeSafely(true, out);
         }
-      };
+      }
     }
   }
 

Modified: lucene/dev/branches/docvalues/lucene/src/test-framework/org/apache/lucene/index/codecs/mockrandom/MockRandomCodec.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/docvalues/lucene/src/test-framework/org/apache/lucene/index/codecs/mockrandom/MockRandomCodec.java?rev=1129631&r1=1129630&r2=1129631&view=diff
==============================================================================
--- lucene/dev/branches/docvalues/lucene/src/test-framework/org/apache/lucene/index/codecs/mockrandom/MockRandomCodec.java (original)
+++ lucene/dev/branches/docvalues/lucene/src/test-framework/org/apache/lucene/index/codecs/mockrandom/MockRandomCodec.java Tue May 31 11:25:37 2011
@@ -142,8 +142,11 @@ public class MockRandomCodec extends Cod
 
     final String seedFileName = IndexFileNames.segmentFileName(state.segmentName, state.codecIdAsString(), SEED_EXT);
     final IndexOutput out = state.directory.createOutput(seedFileName);
-    out.writeLong(seed);
-    out.close();
+    try {
+      out.writeLong(seed);
+    } finally {
+      out.close();
+    }
 
     final Random random = new Random(seed);
     

Modified: lucene/dev/branches/docvalues/lucene/src/test-framework/org/apache/lucene/index/codecs/mocksep/MockSingleIntIndexOutput.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/docvalues/lucene/src/test-framework/org/apache/lucene/index/codecs/mocksep/MockSingleIntIndexOutput.java?rev=1129631&r1=1129630&r2=1129631&view=diff
==============================================================================
--- lucene/dev/branches/docvalues/lucene/src/test-framework/org/apache/lucene/index/codecs/mocksep/MockSingleIntIndexOutput.java (original)
+++ lucene/dev/branches/docvalues/lucene/src/test-framework/org/apache/lucene/index/codecs/mocksep/MockSingleIntIndexOutput.java Tue May 31 11:25:37 2011
@@ -20,6 +20,7 @@ package org.apache.lucene.index.codecs.m
 import org.apache.lucene.store.IndexOutput;
 import org.apache.lucene.store.Directory;
 import org.apache.lucene.util.CodecUtil;
+import org.apache.lucene.util.IOUtils;
 import org.apache.lucene.index.codecs.sep.IntIndexOutput;
 import java.io.IOException;
 
@@ -36,7 +37,15 @@ public class MockSingleIntIndexOutput ex
 
   public MockSingleIntIndexOutput(Directory dir, String fileName) throws IOException {
     out = dir.createOutput(fileName);
-    CodecUtil.writeHeader(out, CODEC, VERSION_CURRENT);
+    boolean success = false;
+    try {
+      CodecUtil.writeHeader(out, CODEC, VERSION_CURRENT);
+      success = true;
+    } finally {
+      if (!success) {
+        IOUtils.closeSafely(true, out);
+      }
+    }
   }
 
   /** Write an int to the primary file */

Modified: lucene/dev/branches/docvalues/lucene/src/test-framework/org/apache/lucene/index/codecs/preflexrw/PreFlexFieldsWriter.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/docvalues/lucene/src/test-framework/org/apache/lucene/index/codecs/preflexrw/PreFlexFieldsWriter.java?rev=1129631&r1=1129630&r2=1129631&view=diff
==============================================================================
--- lucene/dev/branches/docvalues/lucene/src/test-framework/org/apache/lucene/index/codecs/preflexrw/PreFlexFieldsWriter.java (original)
+++ lucene/dev/branches/docvalues/lucene/src/test-framework/org/apache/lucene/index/codecs/preflexrw/PreFlexFieldsWriter.java Tue May 31 11:25:37 2011
@@ -17,22 +17,23 @@ package org.apache.lucene.index.codecs.p
  * limitations under the License.
  */
 
-import org.apache.lucene.util.BytesRef;
+import java.io.IOException;
+import java.util.Comparator;
+
+import org.apache.lucene.index.CorruptIndexException;
+import org.apache.lucene.index.FieldInfo;
+import org.apache.lucene.index.IndexFileNames;
+import org.apache.lucene.index.SegmentWriteState;
 import org.apache.lucene.index.codecs.FieldsConsumer;
-import org.apache.lucene.index.codecs.TermsConsumer;
 import org.apache.lucene.index.codecs.PostingsConsumer;
 import org.apache.lucene.index.codecs.TermStats;
-import org.apache.lucene.index.codecs.standard.DefaultSkipListWriter;
+import org.apache.lucene.index.codecs.TermsConsumer;
 import org.apache.lucene.index.codecs.preflex.PreFlexCodec;
-import org.apache.lucene.index.CorruptIndexException;
-import org.apache.lucene.index.IndexFileNames;
-import org.apache.lucene.index.SegmentWriteState;
-import org.apache.lucene.index.FieldInfo;
 import org.apache.lucene.index.codecs.preflex.TermInfo;
+import org.apache.lucene.index.codecs.standard.DefaultSkipListWriter;
 import org.apache.lucene.store.IndexOutput;
-
-import java.io.IOException;
-import java.util.Comparator;
+import org.apache.lucene.util.BytesRef;
+import org.apache.lucene.util.IOUtils;
 
 class PreFlexFieldsWriter extends FieldsConsumer {
 
@@ -76,11 +77,7 @@ class PreFlexFieldsWriter extends Fields
 
   @Override
   public void close() throws IOException {
-    termsOut.close();
-    freqOut.close();
-    if (proxOut != null) {
-      proxOut.close();
-    }
+    IOUtils.closeSafely(false, termsOut, freqOut, proxOut);
   }
 
   private class PreFlexTermsWriter extends TermsConsumer {

Modified: lucene/dev/branches/docvalues/lucene/src/test-framework/org/apache/lucene/index/codecs/preflexrw/TermInfosWriter.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/docvalues/lucene/src/test-framework/org/apache/lucene/index/codecs/preflexrw/TermInfosWriter.java?rev=1129631&r1=1129630&r2=1129631&view=diff
==============================================================================
--- lucene/dev/branches/docvalues/lucene/src/test-framework/org/apache/lucene/index/codecs/preflexrw/TermInfosWriter.java (original)
+++ lucene/dev/branches/docvalues/lucene/src/test-framework/org/apache/lucene/index/codecs/preflexrw/TermInfosWriter.java Tue May 31 11:25:37 2011
@@ -18,19 +18,23 @@ package org.apache.lucene.index.codecs.p
  */
 
 
+import java.io.Closeable;
 import java.io.IOException;
-import org.apache.lucene.store.IndexOutput;
-import org.apache.lucene.store.Directory;
-import org.apache.lucene.util.UnicodeUtil;
+
 import org.apache.lucene.index.FieldInfos;
-import org.apache.lucene.util.BytesRef;
 import org.apache.lucene.index.codecs.preflex.TermInfo;
+import org.apache.lucene.store.Directory;
+import org.apache.lucene.store.IndexOutput;
+import org.apache.lucene.util.BytesRef;
+import org.apache.lucene.util.CharsRef;
+import org.apache.lucene.util.IOUtils;
+import org.apache.lucene.util.UnicodeUtil;
 
 
 /** This stores a monotonically increasing set of <Term, TermInfo> pairs in a
   Directory.  A TermInfos can be written once, in order.  */
 
-final class TermInfosWriter {
+final class TermInfosWriter implements Closeable {
   /** The file format version, a negative number. */
   public static final int FORMAT = -3;
 
@@ -83,8 +87,26 @@ final class TermInfosWriter {
                   int interval)
        throws IOException {
     initialize(directory, segment, fis, interval, false);
+    boolean success = false;
+    try {
     other = new TermInfosWriter(directory, segment, fis, interval, true);
     other.other = this;
+      success = true;
+    } finally {
+      if (!success) {
+        try {
+          IOUtils.closeSafely(true, output);
+        } catch (IOException e) {
+          // cannot happen since we suppress exceptions
+          throw new RuntimeException(e);
+        }
+
+        try {
+          directory.deleteFile(segment + (isIndex ? ".tii" : ".tis"));
+        } catch (IOException ignored) {
+        }
+      }
+    }
   }
 
   private TermInfosWriter(Directory directory, String segment, FieldInfos fis,
@@ -98,23 +120,41 @@ final class TermInfosWriter {
     fieldInfos = fis;
     isIndex = isi;
     output = directory.createOutput(segment + (isIndex ? ".tii" : ".tis"));
+    boolean success = false;
+    try {
     output.writeInt(FORMAT_CURRENT);              // write format
     output.writeLong(0);                          // leave space for size
     output.writeInt(indexInterval);               // write indexInterval
     output.writeInt(skipInterval);                // write skipInterval
     output.writeInt(maxSkipLevels);               // write maxSkipLevels
     assert initUTF16Results();
+      success = true;
+    } finally {
+      if (!success) {
+        try {
+          IOUtils.closeSafely(true, output);
+        } catch (IOException e) {
+          // cannot happen since we suppress exceptions
+          throw new RuntimeException(e);
+        }
+
+        try {
+          directory.deleteFile(segment + (isIndex ? ".tii" : ".tis"));
+        } catch (IOException ignored) {
+        }
+      }
+    }
   }
 
   // Currently used only by assert statements
-  UnicodeUtil.UTF16Result utf16Result1;
-  UnicodeUtil.UTF16Result utf16Result2;
+  CharsRef utf16Result1;
+  CharsRef utf16Result2;
   private final BytesRef scratchBytes = new BytesRef();
 
   // Currently used only by assert statements
   private boolean initUTF16Results() {
-    utf16Result1 = new UnicodeUtil.UTF16Result();
-    utf16Result2 = new UnicodeUtil.UTF16Result();
+    utf16Result1 = new CharsRef(10);
+    utf16Result2 = new CharsRef(10);
     return true;
   }
 
@@ -145,8 +185,8 @@ final class TermInfosWriter {
       len = utf16Result2.length;
 
     for(int i=0;i<len;i++) {
-      final char ch1 = utf16Result1.result[i];
-      final char ch2 = utf16Result2.result[i];
+      final char ch1 = utf16Result1.chars[i];
+      final char ch2 = utf16Result2.chars[i];
       if (ch1 != ch2)
         return ch1-ch2;
     }
@@ -215,13 +255,18 @@ final class TermInfosWriter {
   }
 
   /** Called to complete TermInfos creation. */
-  void close() throws IOException {
-    output.seek(4);          // write size after format
-    output.writeLong(size);
-    output.close();
-
-    if (!isIndex)
-      other.close();
+  public void close() throws IOException {
+    try {
+      output.seek(4);          // write size after format
+      output.writeLong(size);
+    } finally {
+      try {
+        output.close();
+      } finally {
+        if (!isIndex) {
+          other.close();
+        }
+      }
+    }
   }
-
 }

Modified: lucene/dev/branches/docvalues/lucene/src/test-framework/org/apache/lucene/search/QueryUtils.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/docvalues/lucene/src/test-framework/org/apache/lucene/search/QueryUtils.java?rev=1129631&r1=1129630&r2=1129631&view=diff
==============================================================================
--- lucene/dev/branches/docvalues/lucene/src/test-framework/org/apache/lucene/search/QueryUtils.java (original)
+++ lucene/dev/branches/docvalues/lucene/src/test-framework/org/apache/lucene/search/QueryUtils.java Tue May 31 11:25:37 2011
@@ -1,8 +1,24 @@
 package org.apache.lucene.search;
 
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
 import java.io.IOException;
 import java.util.Random;
-import java.lang.reflect.Method;
 
 import junit.framework.Assert;
 
@@ -23,21 +39,6 @@ import org.apache.lucene.util._TestUtil;
 
 import static org.apache.lucene.util.LuceneTestCase.TEST_VERSION_CURRENT;
 
-/**
- * Copyright 2005 Apache Software Foundation
- *
- * Licensed 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.
- */
 
 
 

Modified: lucene/dev/branches/docvalues/lucene/src/test-framework/org/apache/lucene/store/MockDirectoryWrapper.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/docvalues/lucene/src/test-framework/org/apache/lucene/store/MockDirectoryWrapper.java?rev=1129631&r1=1129630&r2=1129631&view=diff
==============================================================================
--- lucene/dev/branches/docvalues/lucene/src/test-framework/org/apache/lucene/store/MockDirectoryWrapper.java (original)
+++ lucene/dev/branches/docvalues/lucene/src/test-framework/org/apache/lucene/store/MockDirectoryWrapper.java Tue May 31 11:25:37 2011
@@ -68,24 +68,25 @@ public class MockDirectoryWrapper extend
   boolean trackDiskUsage = false;
   private Set<String> unSyncedFiles;
   private Set<String> createdFiles;
-  Set<String> openFilesForWrite = new HashSet<String>();
+  private Set<String> openFilesForWrite = new HashSet<String>();
+  Set<String> openLocks = Collections.synchronizedSet(new HashSet<String>());
   volatile boolean crashed;
   private ThrottledIndexOutput throttledOutput;
   private Throttling throttling = Throttling.SOMETIMES;
 
   // use this for tracking files for crash.
   // additionally: provides debugging information in case you leave one open
-  Map<Closeable,Exception> openFileHandles = Collections.synchronizedMap(new IdentityHashMap<Closeable,Exception>());
+  private Map<Closeable,Exception> openFileHandles = Collections.synchronizedMap(new IdentityHashMap<Closeable,Exception>());
 
   // NOTE: we cannot initialize the Map here due to the
   // order in which our constructor actually does this
   // member initialization vs when it calls super.  It seems
   // like super is called, then our members are initialized:
-  Map<String,Integer> openFiles;
+  private Map<String,Integer> openFiles;
 
   // Only tracked if noDeleteOpenFile is true: if an attempt
   // is made to delete an open file, we enroll it here.
-  Set<String> openFilesDeleted;
+  private Set<String> openFilesDeleted;
 
   private synchronized void init() {
     if (openFiles == null) {
@@ -107,6 +108,12 @@ public class MockDirectoryWrapper extend
     this.randomState = new Random(random.nextInt());
     this.throttledOutput = new ThrottledIndexOutput(ThrottledIndexOutput
         .mBitsToBytes(40 + randomState.nextInt(10)), 5 + randomState.nextInt(5), null);
+    // force wrapping of lockfactory
+    try {
+      setLockFactory(new MockLockFactoryWrapper(this, delegate.getLockFactory()));
+    } catch (IOException e) {
+      throw new RuntimeException(e);
+    }
     init();
   }
 
@@ -127,7 +134,7 @@ public class MockDirectoryWrapper extend
     SOMETIMES,
     /** never throttle output */
     NEVER
-  };
+  }
   
   public void setThrottling(Throttling throttling) {
     this.throttling = throttling;
@@ -208,6 +215,7 @@ public class MockDirectoryWrapper extend
 
   public synchronized void clearCrash() throws IOException {
     crashed = false;
+    openLocks.clear();
   }
 
   public void setMaxSizeInBytes(long maxSize) {
@@ -362,9 +370,10 @@ public class MockDirectoryWrapper extend
         ramdir.fileMap.put(name, file);
       }
     }
+    
     //System.out.println(Thread.currentThread().getName() + ": MDW: create " + name);
     IndexOutput io = new MockIndexOutputWrapper(this, delegate.createOutput(name), name);
-    openFileHandles.put(io, new RuntimeException("unclosed IndexOutput"));
+    addFileHandle(io, name, false);
     openFilesForWrite.add(name);
     
     // throttling REALLY slows down tests, so don't do it very often for SOMETIMES.
@@ -379,6 +388,18 @@ public class MockDirectoryWrapper extend
     }
   }
 
+  private void addFileHandle(Closeable c, String name, boolean input) {
+    Integer v = openFiles.get(name);
+    if (v != null) {
+      v = Integer.valueOf(v.intValue()+1);
+      openFiles.put(name, v);
+    } else {
+      openFiles.put(name, Integer.valueOf(1));
+    }
+    
+    openFileHandles.put(c, new RuntimeException("unclosed Index" + (input ? "Input" : "Output") + ": " + name));
+  }
+  
   @Override
   public synchronized IndexInput openInput(String name) throws IOException {
     maybeYield();
@@ -391,16 +412,8 @@ public class MockDirectoryWrapper extend
       throw fillOpenTrace(new IOException("MockDirectoryWrapper: file \"" + name + "\" is still open for writing"), name, false);
     }
 
-    if (openFiles.containsKey(name)) {
-      Integer v =  openFiles.get(name);
-      v = Integer.valueOf(v.intValue()+1);
-      openFiles.put(name, v);
-    } else {
-      openFiles.put(name, Integer.valueOf(1));
-    }
-
     IndexInput ii = new MockIndexInputWrapper(this, name, delegate.openInput(name));
-    openFileHandles.put(ii, new RuntimeException("unclosed IndexInput"));
+    addFileHandle(ii, name, true);
     return ii;
   }
 
@@ -447,6 +460,9 @@ public class MockDirectoryWrapper extend
       // super() does not throw IOException currently:
       throw new RuntimeException("MockDirectoryWrapper: cannot close: there are still open files: " + openFiles, cause);
     }
+    if (noDeleteOpenFile && openLocks.size() > 0) {
+      throw new RuntimeException("MockDirectoryWrapper: cannot close: there are still open locks: " + openLocks);
+    }
     open = false;
     if (checkIndexOnClose) {
       if (LuceneTestCase.VERBOSE) {
@@ -465,6 +481,31 @@ public class MockDirectoryWrapper extend
     delegate.close();
   }
 
+  private synchronized void removeOpenFile(Closeable c, String name) {
+    Integer v = openFiles.get(name);
+    // Could be null when crash() was called
+    if (v != null) {
+      if (v.intValue() == 1) {
+        openFiles.remove(name);
+        openFilesDeleted.remove(name);
+      } else {
+        v = Integer.valueOf(v.intValue()-1);
+        openFiles.put(name, v);
+      }
+    }
+
+    openFileHandles.remove(c);
+  }
+  
+  public synchronized void removeIndexOutput(IndexOutput out, String name) {
+    openFilesForWrite.remove(name);
+    removeOpenFile(out, name);
+  }
+  
+  public synchronized void removeIndexInput(IndexInput in, String name) {
+    removeOpenFile(in, name);
+  }
+  
   private CodecProvider codecProvider;
 
   // We pass this CodecProvider to checkIndex when dir is closed...

Modified: lucene/dev/branches/docvalues/lucene/src/test-framework/org/apache/lucene/store/MockIndexInputWrapper.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/docvalues/lucene/src/test-framework/org/apache/lucene/store/MockIndexInputWrapper.java?rev=1129631&r1=1129630&r2=1129631&view=diff
==============================================================================
--- lucene/dev/branches/docvalues/lucene/src/test-framework/org/apache/lucene/store/MockIndexInputWrapper.java (original)
+++ lucene/dev/branches/docvalues/lucene/src/test-framework/org/apache/lucene/store/MockIndexInputWrapper.java Tue May 31 11:25:37 2011
@@ -31,8 +31,7 @@ public class MockIndexInputWrapper exten
   private IndexInput delegate;
   private boolean isClone;
 
-  /** Construct an empty output buffer. 
-   * @throws IOException */
+  /** Construct an empty output buffer. */
   public MockIndexInputWrapper(MockDirectoryWrapper dir, String name, IndexInput delegate) {
     this.name = name;
     this.dir = dir;
@@ -41,24 +40,17 @@ public class MockIndexInputWrapper exten
 
   @Override
   public void close() throws IOException {
-    delegate.close();
-    // Pending resolution on LUCENE-686 we may want to
-    // remove the conditional check so we also track that
-    // all clones get closed:
-    if (!isClone) {
-      synchronized(dir) {
-        Integer v = dir.openFiles.get(name);
-        // Could be null when MockRAMDirectory.crash() was called
-        if (v != null) {
-          if (v.intValue() == 1) {
-            dir.openFiles.remove(name);
-            dir.openFilesDeleted.remove(name);
-          } else {
-            v = Integer.valueOf(v.intValue()-1);
-            dir.openFiles.put(name, v);
-          }
-        }
-        dir.openFileHandles.remove(this);
+    try {
+      // turn on the following to look for leaks closing inputs,
+      // after fixing TestTransactions
+      // dir.maybeThrowDeterministicException();
+    } finally {
+      delegate.close();
+      // Pending resolution on LUCENE-686 we may want to
+      // remove the conditional check so we also track that
+      // all clones get closed:
+      if (!isClone) {
+        dir.removeIndexInput(this, name);
       }
     }
   }

Modified: lucene/dev/branches/docvalues/lucene/src/test-framework/org/apache/lucene/store/MockIndexOutputWrapper.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/docvalues/lucene/src/test-framework/org/apache/lucene/store/MockIndexOutputWrapper.java?rev=1129631&r1=1129630&r2=1129631&view=diff
==============================================================================
--- lucene/dev/branches/docvalues/lucene/src/test-framework/org/apache/lucene/store/MockIndexOutputWrapper.java (original)
+++ lucene/dev/branches/docvalues/lucene/src/test-framework/org/apache/lucene/store/MockIndexOutputWrapper.java Tue May 31 11:25:37 2011
@@ -45,19 +45,19 @@ public class MockIndexOutputWrapper exte
 
   @Override
   public void close() throws IOException {
-    dir.maybeThrowDeterministicException();
-    delegate.close();
-    if (dir.trackDiskUsage) {
-      // Now compute actual disk usage & track the maxUsedSize
-      // in the MockDirectoryWrapper:
-      long size = dir.getRecomputedActualSizeInBytes();
-      if (size > dir.maxUsedSize) {
-        dir.maxUsedSize = size;
+    try {
+      dir.maybeThrowDeterministicException();
+    } finally {
+      delegate.close();
+      if (dir.trackDiskUsage) {
+        // Now compute actual disk usage & track the maxUsedSize
+        // in the MockDirectoryWrapper:
+        long size = dir.getRecomputedActualSizeInBytes();
+        if (size > dir.maxUsedSize) {
+          dir.maxUsedSize = size;
+        }
       }
-    }
-    synchronized(dir) {
-      dir.openFileHandles.remove(this);
-      dir.openFilesForWrite.remove(name);
+      dir.removeIndexOutput(this, name);
     }
   }
 

Modified: lucene/dev/branches/docvalues/lucene/src/test-framework/org/apache/lucene/util/LuceneTestCase.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/docvalues/lucene/src/test-framework/org/apache/lucene/util/LuceneTestCase.java?rev=1129631&r1=1129630&r2=1129631&view=diff
==============================================================================
--- lucene/dev/branches/docvalues/lucene/src/test-framework/org/apache/lucene/util/LuceneTestCase.java (original)
+++ lucene/dev/branches/docvalues/lucene/src/test-framework/org/apache/lucene/util/LuceneTestCase.java Tue May 31 11:25:37 2011
@@ -28,6 +28,7 @@ import java.lang.reflect.Constructor;
 import java.lang.reflect.Method;
 import java.lang.reflect.Modifier;
 import java.util.*;
+import java.util.Map.Entry;
 import java.util.concurrent.ExecutorService;
 import java.util.concurrent.Executors;
 import java.util.concurrent.TimeUnit;
@@ -132,7 +133,7 @@ public abstract class LuceneTestCase ext
   }
   
   /** set of directories we created, in afterclass we try to clean these up */
-  static final Set<String> tempDirs = Collections.synchronizedSet(new HashSet<String>());
+  private static final Map<File, StackTraceElement[]> tempDirs = Collections.synchronizedMap(new HashMap<File, StackTraceElement[]>());
 
   // by default we randomly pick a different codec for
   // each test case (non-J4 tests) and each test class (J4
@@ -180,7 +181,7 @@ public abstract class LuceneTestCase ext
     SETUP,   // test has called setUp()
     RANTEST, // test is running
     TEARDOWN // test has called tearDown()
-  };
+  }
   
   /**
    * Some tests expect the directory to contain a single segment, and want to do tests on that segment's reader.
@@ -454,11 +455,21 @@ public abstract class LuceneTestCase ext
     }
     // clear out any temp directories if we can
     if (!testsFailed) {
-      for (String path : tempDirs) {
+      for (Entry<File, StackTraceElement[]> entry : tempDirs.entrySet()) {
         try {
-          _TestUtil.rmDir(new File(path));
+          _TestUtil.rmDir(entry.getKey());
         } catch (IOException e) {
           e.printStackTrace();
+          System.err.println("path " + entry.getKey() + " allocated from");
+          // first two STE's are Java's
+          StackTraceElement[] elements = entry.getValue();
+          for (int i = 2; i < elements.length; i++) {
+            StackTraceElement ste = elements[i];            
+            // print only our code's stack information
+            if (ste.getClassName().indexOf("org.apache.lucene") == -1) break; 
+            System.err.println("\t" + ste);
+          }
+          fail("could not remove temp dir: " + entry.getKey());
         }
       }
     }
@@ -961,7 +972,10 @@ public abstract class LuceneTestCase ext
 
         clazz = Class.forName(fsdirClass).asSubclass(FSDirectory.class);
       }
-      MockDirectoryWrapper dir = new MockDirectoryWrapper(random, newFSDirectoryImpl(clazz, f, lf));
+      MockDirectoryWrapper dir = new MockDirectoryWrapper(random, newFSDirectoryImpl(clazz, f));
+      if (lf != null) {
+        dir.setLockFactory(lf);
+      }
       stores.put(dir, Thread.currentThread().getStackTrace());
       return dir;
     } catch (Exception e) {
@@ -1095,7 +1109,7 @@ public abstract class LuceneTestCase ext
   }
 
   private static Directory newFSDirectoryImpl(
-      Class<? extends FSDirectory> clazz, File file, LockFactory lockFactory)
+      Class<? extends FSDirectory> clazz, File file)
       throws IOException {
     FSDirectory d = null;
     try {
@@ -1106,12 +1120,14 @@ public abstract class LuceneTestCase ext
     } catch (Exception e) {
       d = FSDirectory.open(file);
     }
-    if (lockFactory != null) {
-      d.setLockFactory(lockFactory);
-    }
     return d;
   }
 
+  /** Registers a temp file that will be deleted when tests are done. */
+  public static void registerTempFile(File tmpFile) {
+    tempDirs.put(tmpFile.getAbsoluteFile(), Thread.currentThread().getStackTrace());
+  }
+  
   static Directory newDirectoryImpl(Random random, String clazzName) {
     if (clazzName.equals("random"))
       clazzName = randomDirectory(random);
@@ -1121,11 +1137,11 @@ public abstract class LuceneTestCase ext
       final Class<? extends Directory> clazz = Class.forName(clazzName).asSubclass(Directory.class);
       // If it is a FSDirectory type, try its ctor(File)
       if (FSDirectory.class.isAssignableFrom(clazz)) {
-        final File tmpFile = File.createTempFile("test", "tmp", TEMP_DIR);
+        final File tmpFile = _TestUtil.createTempFile("test", "tmp", TEMP_DIR);
         tmpFile.delete();
         tmpFile.mkdir();
-        tempDirs.add(tmpFile.getAbsolutePath());
-        return newFSDirectoryImpl(clazz.asSubclass(FSDirectory.class), tmpFile, null);
+        registerTempFile(tmpFile);
+        return newFSDirectoryImpl(clazz.asSubclass(FSDirectory.class), tmpFile);
       }
 
       // try empty ctor
@@ -1383,6 +1399,11 @@ public abstract class LuceneTestCase ext
     }
 
     @Override
+    public synchronized boolean hasFieldCodec(String name) {
+      return true; // we have a codec for every field
+    }
+
+    @Override
     public synchronized String toString() {
       return "RandomCodecProvider: " + previousMappings.toString();
     }

Modified: lucene/dev/branches/docvalues/lucene/src/test-framework/org/apache/lucene/util/ThrottledIndexOutput.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/docvalues/lucene/src/test-framework/org/apache/lucene/util/ThrottledIndexOutput.java?rev=1129631&r1=1129630&r2=1129631&view=diff
==============================================================================
--- lucene/dev/branches/docvalues/lucene/src/test-framework/org/apache/lucene/util/ThrottledIndexOutput.java (original)
+++ lucene/dev/branches/docvalues/lucene/src/test-framework/org/apache/lucene/util/ThrottledIndexOutput.java Tue May 31 11:25:37 2011
@@ -73,9 +73,11 @@ public class ThrottledIndexOutput extend
 
   @Override
   public void close() throws IOException {
-    sleep(closeDelayMillis + getDelay(true));
-    delegate.close();
-
+    try {
+      sleep(closeDelayMillis + getDelay(true));
+    } finally {
+      delegate.close();
+    }
   }
 
   @Override

Modified: lucene/dev/branches/docvalues/lucene/src/test-framework/org/apache/lucene/util/_TestUtil.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/docvalues/lucene/src/test-framework/org/apache/lucene/util/_TestUtil.java?rev=1129631&r1=1129630&r2=1129631&view=diff
==============================================================================
--- lucene/dev/branches/docvalues/lucene/src/test-framework/org/apache/lucene/util/_TestUtil.java (original)
+++ lucene/dev/branches/docvalues/lucene/src/test-framework/org/apache/lucene/util/_TestUtil.java Tue May 31 11:25:37 2011
@@ -54,8 +54,8 @@ public class _TestUtil {
   /** Returns temp dir, containing String arg in its name;
    *  does not create the directory. */
   public static File getTempDir(String desc) {
-    File f = new File(LuceneTestCase.TEMP_DIR, desc + "." + new Random().nextLong());
-    LuceneTestCase.tempDirs.add(f.getAbsolutePath());
+    File f = new File(LuceneTestCase.TEMP_DIR, desc + "." + LuceneTestCase.random.nextLong());
+    LuceneTestCase.registerTempFile(f);
     return f;
   }
 
@@ -91,7 +91,7 @@ public class _TestUtil {
     rmDir(destDir);
     
     destDir.mkdir();
-    LuceneTestCase.tempDirs.add(destDir.getAbsolutePath());
+    LuceneTestCase.registerTempFile(destDir);
     
     while (entries.hasMoreElements()) {
       ZipEntry entry = entries.nextElement();
@@ -404,4 +404,51 @@ public class _TestUtil {
               field.isStoreOffsetWithTermVector(), field.getOmitNorms(), false, field.getOmitTermFreqAndPositions(), field.docValuesType());
     }
   }
+  
+  /** 
+   * insecure, fast version of File.createTempFile
+   * uses Random instead of SecureRandom.
+   */
+  public static File createTempFile(String prefix, String suffix, File directory)
+      throws IOException {
+    // Force a prefix null check first
+    if (prefix.length() < 3) {
+      throw new IllegalArgumentException("prefix must be 3");
+    }
+    String newSuffix = suffix == null ? ".tmp" : suffix;
+    File result;
+    do {
+      result = genTempFile(prefix, newSuffix, directory);
+    } while (!result.createNewFile());
+    return result;
+  }
+
+  /* Temp file counter */
+  private static int counter = 0;
+
+  /* identify for differnt VM processes */
+  private static int counterBase = 0;
+
+  private static class TempFileLocker {};
+  private static TempFileLocker tempFileLocker = new TempFileLocker();
+
+  private static File genTempFile(String prefix, String suffix, File directory) {
+    int identify = 0;
+
+    synchronized (tempFileLocker) {
+      if (counter == 0) {
+        int newInt = new Random().nextInt();
+        counter = ((newInt / 65535) & 0xFFFF) + 0x2710;
+        counterBase = counter;
+      }
+      identify = counter++;
+    }
+
+    StringBuilder newName = new StringBuilder();
+    newName.append(prefix);
+    newName.append(counterBase);
+    newName.append(identify);
+    newName.append(suffix);
+    return new File(directory, newName.toString());
+  }
 }