You are viewing a plain text version of this content. The canonical link for it is here.
Posted to java-commits@lucene.apache.org by us...@apache.org on 2010/01/14 20:05:42 UTC

svn commit: r899359 [6/7] - in /lucene/java/branches/flex_1458: ./ contrib/ contrib/analyzers/common/src/java/org/apache/lucene/analysis/ar/ contrib/analyzers/common/src/java/org/apache/lucene/analysis/bg/ contrib/analyzers/common/src/java/org/apache/l...

Modified: lucene/java/branches/flex_1458/src/java/org/apache/lucene/store/NativeFSLockFactory.java
URL: http://svn.apache.org/viewvc/lucene/java/branches/flex_1458/src/java/org/apache/lucene/store/NativeFSLockFactory.java?rev=899359&r1=899358&r2=899359&view=diff
==============================================================================
--- lucene/java/branches/flex_1458/src/java/org/apache/lucene/store/NativeFSLockFactory.java (original)
+++ lucene/java/branches/flex_1458/src/java/org/apache/lucene/store/NativeFSLockFactory.java Thu Jan 14 19:05:12 2010
@@ -146,7 +146,7 @@
       }
     }
   }
-};
+}
 
 class NativeFSLock extends Lock {
 
@@ -300,6 +300,24 @@
       }
       if (!path.delete())
         throw new LockReleaseFailedException("failed to delete " + path);
+    } else {
+      // if we don't hold the lock, and somebody still called release(), for
+      // example as a result of calling IndexWriter.unlock(), we should attempt
+      // to obtain the lock and release it. If the obtain fails, it means the
+      // lock cannot be released, and we should throw a proper exception rather
+      // than silently failing/not doing anything.
+      boolean obtained = false;
+      try {
+        if (!(obtained = obtain())) {
+          throw new LockReleaseFailedException(
+              "Cannot forcefully unlock a NativeFSLock which is held by another indexer component: "
+                  + path);
+        }
+      } finally {
+        if (obtained) {
+          release();
+        }
+      }
     }
   }
 

Modified: lucene/java/branches/flex_1458/src/java/org/apache/lucene/store/NoLockFactory.java
URL: http://svn.apache.org/viewvc/lucene/java/branches/flex_1458/src/java/org/apache/lucene/store/NoLockFactory.java?rev=899359&r1=899358&r2=899359&view=diff
==============================================================================
--- lucene/java/branches/flex_1458/src/java/org/apache/lucene/store/NoLockFactory.java (original)
+++ lucene/java/branches/flex_1458/src/java/org/apache/lucene/store/NoLockFactory.java Thu Jan 14 19:05:12 2010
@@ -39,6 +39,7 @@
    * @see #getNoLockFactory()
    */
   // make private in 4.0!
+  @Deprecated
   public NoLockFactory() {}
 
   public static NoLockFactory getNoLockFactory() {

Modified: lucene/java/branches/flex_1458/src/java/org/apache/lucene/util/AttributeSource.java
URL: http://svn.apache.org/viewvc/lucene/java/branches/flex_1458/src/java/org/apache/lucene/util/AttributeSource.java?rev=899359&r1=899358&r2=899359&view=diff
==============================================================================
--- lucene/java/branches/flex_1458/src/java/org/apache/lucene/util/AttributeSource.java (original)
+++ lucene/java/branches/flex_1458/src/java/org/apache/lucene/util/AttributeSource.java Thu Jan 14 19:05:12 2010
@@ -64,9 +64,9 @@
         try {
           return getClassForInterface(attClass).newInstance();
         } catch (InstantiationException e) {
-          throw new IllegalArgumentException("Could not instantiate class " + attClass.getName());
+          throw new IllegalArgumentException("Could not instantiate implementing class for " + attClass.getName());
         } catch (IllegalAccessException e) {
-          throw new IllegalArgumentException("Could not instantiate class " + attClass.getName());
+          throw new IllegalArgumentException("Could not instantiate implementing class for " + attClass.getName());
         }
       }
       
@@ -75,7 +75,10 @@
           Class<? extends AttributeImpl> clazz = attClassImplMap.get(attClass);
           if (clazz == null) {
             try {
-              attClassImplMap.put(attClass, clazz = Class.forName(attClass.getName() + "Impl").asSubclass(AttributeImpl.class));
+              attClassImplMap.put(attClass,
+                clazz = Class.forName(attClass.getName() + "Impl", true, attClass.getClassLoader())
+                .asSubclass(AttributeImpl.class)
+              );
             } catch (ClassNotFoundException e) {
               throw new IllegalArgumentException("Could not find implementing class for " + attClass.getName());
             }

Modified: lucene/java/branches/flex_1458/src/java/org/apache/lucene/util/BitVector.java
URL: http://svn.apache.org/viewvc/lucene/java/branches/flex_1458/src/java/org/apache/lucene/util/BitVector.java?rev=899359&r1=899358&r2=899359&view=diff
==============================================================================
--- lucene/java/branches/flex_1458/src/java/org/apache/lucene/util/BitVector.java (original)
+++ lucene/java/branches/flex_1458/src/java/org/apache/lucene/util/BitVector.java Thu Jan 14 19:05:12 2010
@@ -36,24 +36,28 @@
 
   private byte[] bits;
   private int size;
-  private int count = -1;
+  private int count;
 
   /** Constructs a vector capable of holding <code>n</code> bits. */
   public BitVector(int n) {
     size = n;
     bits = new byte[(size >> 3) + 1];
+    count = 0;
   }
   
   BitVector(byte[] bits, int size) {
     this.bits = bits;
     this.size = size;
+    count = -1;
   }
   
   @Override
   public Object clone() {
     byte[] copyBits = new byte[bits.length];
     System.arraycopy(bits, 0, copyBits, 0, bits.length);
-    return new BitVector(copyBits, size);
+    BitVector clone = new BitVector(copyBits, size);
+    clone.count = count;
+    return clone;
   }
   
   /** Sets the value of <code>bit</code> to one. */
@@ -121,6 +125,15 @@
     return count;
   }
 
+  /** For testing */
+  public final int getRecomputedCount() {
+    int c = 0;
+    int end = bits.length;
+    for (int i = 0; i < end; i++)
+      c += BYTE_COUNTS[bits[i] & 0xFF];	  // sum bits per byte
+    return c;
+  }
+
   private static final byte[] BYTE_COUNTS = {	  // table of bits/byte
     0, 1, 1, 2, 1, 2, 2, 3, 1, 2, 2, 3, 2, 3, 3, 4,
     1, 2, 2, 3, 2, 3, 3, 4, 2, 3, 3, 4, 3, 4, 4, 5,

Modified: lucene/java/branches/flex_1458/src/java/org/apache/lucene/util/IndexableBinaryStringTools.java
URL: http://svn.apache.org/viewvc/lucene/java/branches/flex_1458/src/java/org/apache/lucene/util/IndexableBinaryStringTools.java?rev=899359&r1=899358&r2=899359&view=diff
==============================================================================
--- lucene/java/branches/flex_1458/src/java/org/apache/lucene/util/IndexableBinaryStringTools.java (original)
+++ lucene/java/branches/flex_1458/src/java/org/apache/lucene/util/IndexableBinaryStringTools.java Thu Jan 14 19:05:12 2010
@@ -23,29 +23,33 @@
 /**
  * Provides support for converting byte sequences to Strings and back again.
  * The resulting Strings preserve the original byte sequences' sort order.
- * 
+ * <p/>
  * The Strings are constructed using a Base 8000h encoding of the original
  * binary data - each char of an encoded String represents a 15-bit chunk
  * from the byte sequence.  Base 8000h was chosen because it allows for all
  * lower 15 bits of char to be used without restriction; the surrogate range 
  * [U+D8000-U+DFFF] does not represent valid chars, and would require
  * complicated handling to avoid them and allow use of char's high bit.
- * 
+ * <p/>
  * Although unset bits are used as padding in the final char, the original
  * byte sequence could contain trailing bytes with no set bits (null bytes):
  * padding is indistinguishable from valid information.  To overcome this
  * problem, a char is appended, indicating the number of encoded bytes in the
  * final content char.
- * 
- * This class's operations are defined over CharBuffers and ByteBuffers, to
- * allow for wrapped arrays to be reused, reducing memory allocation costs for
- * repeated operations.  Note that this class calls array() and arrayOffset()
+ * <p/>
+ * Some methods in this class are defined over CharBuffers and ByteBuffers, but
+ * these are deprecated in favor of methods that operate directly on byte[] and
+ * char[] arrays.  Note that this class calls array() and arrayOffset()
  * on the CharBuffers and ByteBuffers it uses, so only wrapped arrays may be
- * used.  This class interprets the arrayOffset() and limit() values returned by
- * its input buffers as beginning and end+1 positions on the wrapped array,
+ * used.  This class interprets the arrayOffset() and limit() values returned 
+ * by its input buffers as beginning and end+1 positions on the wrapped array,
  * respectively; similarly, on the output buffer, arrayOffset() is the first
  * position written to, and limit() is set to one past the final output array
  * position.
+ * <p/>
+ * WARNING: This means that the deprecated Buffer-based methods 
+ * only work correctly with buffers that have an offset of 0. For example, they
+ * will not correctly interpret buffers returned by {@link ByteBuffer#slice}.  
  */
 public class IndexableBinaryStringTools {
 
@@ -68,204 +72,276 @@
   /**
    * Returns the number of chars required to encode the given byte sequence.
    * 
-   * @param original The byte sequence to be encoded.  Must be backed by an array.
+   * @param original The byte sequence to be encoded. Must be backed by an
+   *        array.
    * @return The number of chars required to encode the given byte sequence
-   * @throws IllegalArgumentException If the given ByteBuffer is not backed by an array
+   * @throws IllegalArgumentException If the given ByteBuffer is not backed by
+   *         an array
+   * @deprecated Use {@link #getEncodedLength(byte[], int, int)} instead. This
+   *             method will be removed in Lucene 4.0
    */
-  public static int getEncodedLength(ByteBuffer original) 
+  @Deprecated
+  public static int getEncodedLength(ByteBuffer original)
     throws IllegalArgumentException {
     if (original.hasArray()) {
-      // Use long for intermediaries to protect against overflow
-      long length = (long)(original.limit() - original.arrayOffset());
-      return (int)((length * 8L + 14L) / 15L) + 1;
+      return getEncodedLength(original.array(), original.arrayOffset(),
+          original.limit() - original.arrayOffset());
     } else {
       throw new IllegalArgumentException("original argument must have a backing array");
     }
   }
+  
+  /**
+   * Returns the number of chars required to encode the given bytes.
+   * 
+   * @param inputArray byte sequence to be encoded
+   * @param inputOffset initial offset into inputArray
+   * @param inputLength number of bytes in inputArray
+   * @return The number of chars required to encode the number of bytes.
+   */
+  public static int getEncodedLength(byte[] inputArray, int inputOffset,
+      int inputLength) {
+    // Use long for intermediaries to protect against overflow
+    return (int)(((long)inputLength * 8L + 14L) / 15L) + 1;
+  }
+
 
   /**
    * Returns the number of bytes required to decode the given char sequence.
    * 
-   * @param encoded The char sequence to be encoded.  Must be backed by an array.
+   * @param encoded The char sequence to be decoded. Must be backed by an array.
    * @return The number of bytes required to decode the given char sequence
-   * @throws IllegalArgumentException If the given CharBuffer is not backed by an array
+   * @throws IllegalArgumentException If the given CharBuffer is not backed by
+   *         an array
+   * @deprecated Use {@link #getDecodedLength(char[], int, int)} instead. This
+   *             method will be removed in Lucene 4.0
    */
+  @Deprecated
   public static int getDecodedLength(CharBuffer encoded) 
     throws IllegalArgumentException {
     if (encoded.hasArray()) {
-      int numChars = encoded.limit() - encoded.arrayOffset() - 1;
-      if (numChars <= 0) {
-        return 0;
-      } else {
-        int numFullBytesInFinalChar = encoded.charAt(encoded.limit() - 1);
-        int numEncodedChars = numChars - 1;
-        return (numEncodedChars * 15 + 7) / 8 + numFullBytesInFinalChar;
-      }
+      return getDecodedLength(encoded.array(), encoded.arrayOffset(), 
+          encoded.limit() - encoded.arrayOffset());
     } else {
       throw new IllegalArgumentException("encoded argument must have a backing array");
     }
   }
+  
+  /**
+   * Returns the number of bytes required to decode the given char sequence.
+   * 
+   * @param encoded char sequence to be decoded
+   * @param offset initial offset
+   * @param length number of characters
+   * @return The number of bytes required to decode the given char sequence
+   */
+  public static int getDecodedLength(char[] encoded, int offset, int length) {
+    final int numChars = length - 1;
+    if (numChars <= 0) {
+      return 0;
+    } else {
+      // Use long for intermediaries to protect against overflow
+      final long numFullBytesInFinalChar = encoded[offset + length - 1];
+      final long numEncodedChars = numChars - 1;
+      return (int)((numEncodedChars * 15L + 7L) / 8L + numFullBytesInFinalChar);
+    }
+  }
 
   /**
-   * Encodes the input byte sequence into the output char sequence.  Before
+   * Encodes the input byte sequence into the output char sequence. Before
    * calling this method, ensure that the output CharBuffer has sufficient
    * capacity by calling {@link #getEncodedLength(java.nio.ByteBuffer)}.
    * 
    * @param input The byte sequence to encode
-   * @param output Where the char sequence encoding result will go.  The limit
-   *  is set to one past the position of the final char.
+   * @param output Where the char sequence encoding result will go. The limit is
+   *        set to one past the position of the final char.
    * @throws IllegalArgumentException If either the input or the output buffer
-   *  is not backed by an array
+   *         is not backed by an array
+   * @deprecated Use {@link #encode(byte[], int, int, char[], int, int)}
+   *             instead. This method will be removed in Lucene 4.0
    */
+  @Deprecated
   public static void encode(ByteBuffer input, CharBuffer output) {
     if (input.hasArray() && output.hasArray()) {
-      byte[] inputArray = input.array();
-      int inputOffset = input.arrayOffset();
-      int inputLength = input.limit() - inputOffset; 
-      char[] outputArray = output.array();
-      int outputOffset = output.arrayOffset();
-      int outputLength = getEncodedLength(input);
-      output.limit(outputOffset + outputLength); // Set output final pos + 1
+      final int inputOffset = input.arrayOffset();
+      final int inputLength = input.limit() - inputOffset;
+      final int outputOffset = output.arrayOffset();
+      final int outputLength = getEncodedLength(input.array(), inputOffset,
+          inputLength);
+      output.limit(outputLength + outputOffset);
       output.position(0);
-      if (inputLength > 0) {
-        int inputByteNum = inputOffset;
-        int caseNum = 0;
-        int outputCharNum = outputOffset;
-        CodingCase codingCase;
-        for ( ; inputByteNum + CODING_CASES[caseNum].numBytes <= inputLength ;
-              ++outputCharNum                                                 ) {
-          codingCase = CODING_CASES[caseNum];
-          if (2 == codingCase.numBytes) {
-            outputArray[outputCharNum]
-              = (char)(((inputArray[inputByteNum] & 0xFF) << codingCase.initialShift)
-                       + (((inputArray[inputByteNum + 1] & 0xFF) >>> codingCase.finalShift)
-                          & codingCase.finalMask)
-                       & (short)0x7FFF);
-          } else { // numBytes is 3
-            outputArray[outputCharNum] 
-              = (char)(((inputArray[inputByteNum] & 0xFF) << codingCase.initialShift)
-                       + ((inputArray[inputByteNum + 1] & 0xFF) << codingCase.middleShift)
-                       + (((inputArray[inputByteNum + 2] & 0xFF) >>> codingCase.finalShift) 
-                          & codingCase.finalMask)
-                       & (short)0x7FFF);          
-          }
-          inputByteNum += codingCase.advanceBytes;          
-          if (++caseNum == CODING_CASES.length) {
-            caseNum = 0;
-          }
-        }
-        // Produce final char (if any) and trailing count chars.
+      encode(input.array(), inputOffset, inputLength, output.array(),
+          outputOffset, outputLength);
+    } else {
+      throw new IllegalArgumentException("Arguments must have backing arrays");
+    }
+  }
+  
+  /**
+   * Encodes the input byte sequence into the output char sequence.  Before
+   * calling this method, ensure that the output array has sufficient
+   * capacity by calling {@link #getEncodedLength(byte[], int, int)}.
+   * 
+   * @param inputArray byte sequence to be encoded
+   * @param inputOffset initial offset into inputArray
+   * @param inputLength number of bytes in inputArray
+   * @param outputArray char sequence to store encoded result
+   * @param outputOffset initial offset into outputArray
+   * @param outputLength length of output, must be getEncodedLength
+   */
+  public static void encode(byte[] inputArray, int inputOffset,
+      int inputLength, char[] outputArray, int outputOffset, int outputLength) {
+    assert (outputLength == getEncodedLength(inputArray, inputOffset,
+        inputLength));
+    if (inputLength > 0) {
+      int inputByteNum = inputOffset;
+      int caseNum = 0;
+      int outputCharNum = outputOffset;
+      CodingCase codingCase;
+      for (; inputByteNum + CODING_CASES[caseNum].numBytes <= inputLength; ++outputCharNum) {
         codingCase = CODING_CASES[caseNum];
-        
-        if (inputByteNum + 1 < inputLength) { // codingCase.numBytes must be 3
-          outputArray[outputCharNum++] 
-            = (char)((((inputArray[inputByteNum] & 0xFF) << codingCase.initialShift)
-                      + ((inputArray[inputByteNum + 1] & 0xFF) << codingCase.middleShift))
-                     & (short)0x7FFF);
-          // Add trailing char containing the number of full bytes in final char
-          outputArray[outputCharNum++] = (char)1;
-        } else if (inputByteNum < inputLength) {
-          outputArray[outputCharNum++] 
-            = (char)(((inputArray[inputByteNum] & 0xFF) << codingCase.initialShift)
-                     & (short)0x7FFF);
-          // Add trailing char containing the number of full bytes in final char
-          outputArray[outputCharNum++] = caseNum == 0 ? (char)1 : (char)0;
-        } else { // No left over bits - last char is completely filled.
-          // Add trailing char containing the number of full bytes in final char
-          outputArray[outputCharNum++] = (char)1;
+        if (2 == codingCase.numBytes) {
+          outputArray[outputCharNum] = (char) (((inputArray[inputByteNum] & 0xFF) << codingCase.initialShift)
+              + (((inputArray[inputByteNum + 1] & 0xFF) >>> codingCase.finalShift) & codingCase.finalMask) & (short) 0x7FFF);
+        } else { // numBytes is 3
+          outputArray[outputCharNum] = (char) (((inputArray[inputByteNum] & 0xFF) << codingCase.initialShift)
+              + ((inputArray[inputByteNum + 1] & 0xFF) << codingCase.middleShift)
+              + (((inputArray[inputByteNum + 2] & 0xFF) >>> codingCase.finalShift) & codingCase.finalMask) & (short) 0x7FFF);
+        }
+        inputByteNum += codingCase.advanceBytes;
+        if (++caseNum == CODING_CASES.length) {
+          caseNum = 0;
         }
       }
-    } else {
-      throw new IllegalArgumentException("Arguments must have backing arrays");
+      // Produce final char (if any) and trailing count chars.
+      codingCase = CODING_CASES[caseNum];
+
+      if (inputByteNum + 1 < inputLength) { // codingCase.numBytes must be 3
+        outputArray[outputCharNum++] = (char) ((((inputArray[inputByteNum] & 0xFF) << codingCase.initialShift) + ((inputArray[inputByteNum + 1] & 0xFF) << codingCase.middleShift)) & (short) 0x7FFF);
+        // Add trailing char containing the number of full bytes in final char
+        outputArray[outputCharNum++] = (char) 1;
+      } else if (inputByteNum < inputLength) {
+        outputArray[outputCharNum++] = (char) (((inputArray[inputByteNum] & 0xFF) << codingCase.initialShift) & (short) 0x7FFF);
+        // Add trailing char containing the number of full bytes in final char
+        outputArray[outputCharNum++] = caseNum == 0 ? (char) 1 : (char) 0;
+      } else { // No left over bits - last char is completely filled.
+        // Add trailing char containing the number of full bytes in final char
+        outputArray[outputCharNum++] = (char) 1;
+      }
     }
   }
 
   /**
-   * Decodes the input char sequence into the output byte sequence.  Before
+   * Decodes the input char sequence into the output byte sequence. Before
    * calling this method, ensure that the output ByteBuffer has sufficient
    * capacity by calling {@link #getDecodedLength(java.nio.CharBuffer)}.
    * 
    * @param input The char sequence to decode
-   * @param output Where the byte sequence decoding result will go.  The limit
-   *  is set to one past the position of the final char.
+   * @param output Where the byte sequence decoding result will go. The limit is
+   *        set to one past the position of the final char.
    * @throws IllegalArgumentException If either the input or the output buffer
-   *  is not backed by an array
+   *         is not backed by an array
+   * @deprecated Use {@link #decode(char[], int, int, byte[], int, int)}
+   *             instead. This method will be removed in Lucene 4.0
    */
+  @Deprecated
   public static void decode(CharBuffer input, ByteBuffer output) {
     if (input.hasArray() && output.hasArray()) {
-      int numInputChars = input.limit() - input.arrayOffset() - 1;
-      int numOutputBytes = getDecodedLength(input);
-      output.limit(numOutputBytes + output.arrayOffset()); // Set output final pos + 1
+      final int inputOffset = input.arrayOffset();
+      final int inputLength = input.limit() - inputOffset;
+      final int outputOffset = output.arrayOffset();
+      final int outputLength = getDecodedLength(input.array(), inputOffset,
+          inputLength);
+      output.limit(outputLength + outputOffset);
       output.position(0);
-      byte[] outputArray = output.array();
-      char[] inputArray = input.array();
-      if (numOutputBytes > 0) {
-        int caseNum = 0;
-        int outputByteNum = output.arrayOffset();
-        int inputCharNum = input.arrayOffset();
-        short inputChar;
-        CodingCase codingCase;
-        for ( ; inputCharNum < numInputChars - 1 ; ++inputCharNum) {
-          codingCase = CODING_CASES[caseNum];
-          inputChar = (short)inputArray[inputCharNum];
-          if (2 == codingCase.numBytes) {
-            if (0 == caseNum) {
-              outputArray[outputByteNum] = (byte)(inputChar >>> codingCase.initialShift);
-            } else {
-              outputArray[outputByteNum] += (byte)(inputChar >>> codingCase.initialShift);
-            }
-            outputArray[outputByteNum + 1] = (byte)((inputChar & codingCase.finalMask) 
-                                                    << codingCase.finalShift);
-          } else { // numBytes is 3
-            outputArray[outputByteNum] += (byte)(inputChar >>> codingCase.initialShift);
-            outputArray[outputByteNum + 1] = (byte)((inputChar & codingCase.middleMask)
-                                                    >>> codingCase.middleShift);
-            outputArray[outputByteNum + 2] = (byte)((inputChar & codingCase.finalMask) 
-                                                    << codingCase.finalShift);
-          }
-          outputByteNum += codingCase.advanceBytes;
-          if (++caseNum == CODING_CASES.length) {
-            caseNum = 0;
+      decode(input.array(), inputOffset, inputLength, output.array(),
+          outputOffset, outputLength);
+    } else {
+      throw new IllegalArgumentException("Arguments must have backing arrays");
+    }
+  }
+
+  /**
+   * Decodes the input char sequence into the output byte sequence. Before
+   * calling this method, ensure that the output array has sufficient capacity
+   * by calling {@link #getDecodedLength(char[], int, int)}.
+   * 
+   * @param inputArray char sequence to be decoded
+   * @param inputOffset initial offset into inputArray
+   * @param inputLength number of chars in inputArray
+   * @param outputArray byte sequence to store encoded result
+   * @param outputOffset initial offset into outputArray
+   * @param outputLength length of output, must be
+   *        getDecodedLength(inputArray, inputOffset, inputLength)
+   */
+  public static void decode(char[] inputArray, int inputOffset,
+      int inputLength, byte[] outputArray, int outputOffset, int outputLength) {
+    assert (outputLength == getDecodedLength(inputArray, inputOffset,
+        inputLength));
+    final int numInputChars = inputLength - 1;
+    final int numOutputBytes = outputLength;
+
+    if (numOutputBytes > 0) {
+      int caseNum = 0;
+      int outputByteNum = outputOffset;
+      int inputCharNum = inputOffset;
+      short inputChar;
+      CodingCase codingCase;
+      for (; inputCharNum < numInputChars - 1; ++inputCharNum) {
+        codingCase = CODING_CASES[caseNum];
+        inputChar = (short) inputArray[inputCharNum];
+        if (2 == codingCase.numBytes) {
+          if (0 == caseNum) {
+            outputArray[outputByteNum] = (byte) (inputChar >>> codingCase.initialShift);
+          } else {
+            outputArray[outputByteNum] += (byte) (inputChar >>> codingCase.initialShift);
           }
+          outputArray[outputByteNum + 1] = (byte) ((inputChar & codingCase.finalMask) << codingCase.finalShift);
+        } else { // numBytes is 3
+          outputArray[outputByteNum] += (byte) (inputChar >>> codingCase.initialShift);
+          outputArray[outputByteNum + 1] = (byte) ((inputChar & codingCase.middleMask) >>> codingCase.middleShift);
+          outputArray[outputByteNum + 2] = (byte) ((inputChar & codingCase.finalMask) << codingCase.finalShift);
         }
-        // Handle final char
-        inputChar = (short)inputArray[inputCharNum];
-        codingCase = CODING_CASES[caseNum];
-        if (0 == caseNum) {
-          outputArray[outputByteNum] = 0;
+        outputByteNum += codingCase.advanceBytes;
+        if (++caseNum == CODING_CASES.length) {
+          caseNum = 0;
         }
-        outputArray[outputByteNum] += (byte)(inputChar >>> codingCase.initialShift);
-        int bytesLeft = numOutputBytes - outputByteNum;
-        if (bytesLeft > 1) {
-          if (2 == codingCase.numBytes) {
-            outputArray[outputByteNum + 1] = (byte)((inputChar & codingCase.finalMask) 
-                                                    >>> codingCase.finalShift);
-          } else { // numBytes is 3
-            outputArray[outputByteNum + 1] = (byte)((inputChar & codingCase.middleMask)
-                                                    >>> codingCase.middleShift);
-            if (bytesLeft > 2) {
-              outputArray[outputByteNum + 2] = (byte)((inputChar & codingCase.finalMask) 
-                                                      << codingCase.finalShift);
-            }
+      }
+      // Handle final char
+      inputChar = (short) inputArray[inputCharNum];
+      codingCase = CODING_CASES[caseNum];
+      if (0 == caseNum) {
+        outputArray[outputByteNum] = 0;
+      }
+      outputArray[outputByteNum] += (byte) (inputChar >>> codingCase.initialShift);
+      final int bytesLeft = numOutputBytes - outputByteNum;
+      if (bytesLeft > 1) {
+        if (2 == codingCase.numBytes) {
+          outputArray[outputByteNum + 1] = (byte) ((inputChar & codingCase.finalMask) >>> codingCase.finalShift);
+        } else { // numBytes is 3
+          outputArray[outputByteNum + 1] = (byte) ((inputChar & codingCase.middleMask) >>> codingCase.middleShift);
+          if (bytesLeft > 2) {
+            outputArray[outputByteNum + 2] = (byte) ((inputChar & codingCase.finalMask) << codingCase.finalShift);
           }
         }
       }
-    } else {
-      throw new IllegalArgumentException("Arguments must have backing arrays");
     }
   }
 
   /**
    * Decodes the given char sequence, which must have been encoded by
-   * {@link #encode(java.nio.ByteBuffer)} or 
+   * {@link #encode(java.nio.ByteBuffer)} or
    * {@link #encode(java.nio.ByteBuffer, java.nio.CharBuffer)}.
    * 
    * @param input The char sequence to decode
-   * @return A byte sequence containing the decoding result.  The limit
-   *  is set to one past the position of the final char.
+   * @return A byte sequence containing the decoding result. The limit is set to
+   *         one past the position of the final char.
    * @throws IllegalArgumentException If the input buffer is not backed by an
-   *  array
+   *         array
+   * @deprecated Use {@link #decode(char[], int, int, byte[], int, int)}
+   *             instead. This method will be removed in Lucene 4.0
    */
+  @Deprecated
   public static ByteBuffer decode(CharBuffer input) {
     byte[] outputArray = new byte[getDecodedLength(input)];
     ByteBuffer output = ByteBuffer.wrap(outputArray);
@@ -277,11 +353,14 @@
    * Encodes the input byte sequence.
    * 
    * @param input The byte sequence to encode
-   * @return A char sequence containing the encoding result.  The limit is set
-   *  to one past the position of the final char.
+   * @return A char sequence containing the encoding result. The limit is set to
+   *         one past the position of the final char.
    * @throws IllegalArgumentException If the input buffer is not backed by an
-   *  array
+   *         array
+   * @deprecated Use {@link #encode(byte[], int, int, char[], int, int)}
+   *             instead. This method will be removed in Lucene 4.0
    */
+  @Deprecated
   public static CharBuffer encode(ByteBuffer input) {
     char[] outputArray = new char[getEncodedLength(input)];
     CharBuffer output = CharBuffer.wrap(outputArray);

Modified: lucene/java/branches/flex_1458/src/java/org/apache/lucene/util/Parameter.java
URL: http://svn.apache.org/viewvc/lucene/java/branches/flex_1458/src/java/org/apache/lucene/util/Parameter.java?rev=899359&r1=899358&r2=899359&view=diff
==============================================================================
--- lucene/java/branches/flex_1458/src/java/org/apache/lucene/util/Parameter.java (original)
+++ lucene/java/branches/flex_1458/src/java/org/apache/lucene/util/Parameter.java Thu Jan 14 19:05:12 2010
@@ -27,6 +27,7 @@
  * A serializable Enum class.
  * @deprecated Use Java 5 enum, will be removed in a later Lucene 3.x release.
  */
+@Deprecated
 @SuppressWarnings("serial")
 public abstract class Parameter implements Serializable
 {

Propchange: lucene/java/branches/flex_1458/src/java/org/apache/lucene/util/ReaderUtil.java
------------------------------------------------------------------------------
    svn:eol-style = native

Modified: lucene/java/branches/flex_1458/src/java/org/apache/lucene/util/cache/SimpleLRUCache.java
URL: http://svn.apache.org/viewvc/lucene/java/branches/flex_1458/src/java/org/apache/lucene/util/cache/SimpleLRUCache.java?rev=899359&r1=899358&r2=899359&view=diff
==============================================================================
--- lucene/java/branches/flex_1458/src/java/org/apache/lucene/util/cache/SimpleLRUCache.java (original)
+++ lucene/java/branches/flex_1458/src/java/org/apache/lucene/util/cache/SimpleLRUCache.java Thu Jan 14 19:05:12 2010
@@ -28,6 +28,7 @@
  * @deprecated Lucene's internal use of this class has now
  * switched to {@link DoubleBarrelLRUCache}.
  */
+@Deprecated
 public class SimpleLRUCache<K,V> extends SimpleMapCache<K,V> {
   private final static float LOADFACTOR = 0.75f;
 

Modified: lucene/java/branches/flex_1458/src/java/org/apache/lucene/util/cache/SimpleMapCache.java
URL: http://svn.apache.org/viewvc/lucene/java/branches/flex_1458/src/java/org/apache/lucene/util/cache/SimpleMapCache.java?rev=899359&r1=899358&r2=899359&view=diff
==============================================================================
--- lucene/java/branches/flex_1458/src/java/org/apache/lucene/util/cache/SimpleMapCache.java (original)
+++ lucene/java/branches/flex_1458/src/java/org/apache/lucene/util/cache/SimpleMapCache.java Thu Jan 14 19:05:12 2010
@@ -29,6 +29,7 @@
  * @deprecated Lucene's internal use of this class has now
  * switched to {@link DoubleBarrelLRUCache}.
  */
+@Deprecated
 public class SimpleMapCache<K,V> extends Cache<K,V> {
   protected Map<K,V> map;
   

Propchange: lucene/java/branches/flex_1458/src/site/src/documentation/skins/common/images/README.txt
------------------------------------------------------------------------------
    svn:eol-style = native

Propchange: lucene/java/branches/flex_1458/src/site/src/documentation/skins/common/translations/CommonMessages_de.xml
------------------------------------------------------------------------------
    svn:eol-style = native

Propchange: lucene/java/branches/flex_1458/src/site/src/documentation/skins/common/translations/CommonMessages_en_US.xml
------------------------------------------------------------------------------
    svn:eol-style = native

Propchange: lucene/java/branches/flex_1458/src/site/src/documentation/skins/common/translations/CommonMessages_es.xml
------------------------------------------------------------------------------
    svn:eol-style = native

Propchange: lucene/java/branches/flex_1458/src/site/src/documentation/skins/common/translations/CommonMessages_fr.xml
------------------------------------------------------------------------------
    svn:eol-style = native

Propchange: lucene/java/branches/flex_1458/src/site/src/documentation/skins/lucene/css/basic.css
------------------------------------------------------------------------------
    svn:eol-style = native

Propchange: lucene/java/branches/flex_1458/src/site/src/documentation/skins/lucene/css/print.css
------------------------------------------------------------------------------
    svn:eol-style = native

Propchange: lucene/java/branches/flex_1458/src/site/src/documentation/skins/lucene/css/screen.css
------------------------------------------------------------------------------
    svn:eol-style = native

Propchange: lucene/java/branches/flex_1458/src/site/src/documentation/skins/lucene/note.txt
------------------------------------------------------------------------------
    svn:eol-style = native

Modified: lucene/java/branches/flex_1458/src/test/org/apache/lucene/TestSearch.java
URL: http://svn.apache.org/viewvc/lucene/java/branches/flex_1458/src/test/org/apache/lucene/TestSearch.java?rev=899359&r1=899358&r2=899359&view=diff
==============================================================================
--- lucene/java/branches/flex_1458/src/test/org/apache/lucene/TestSearch.java (original)
+++ lucene/java/branches/flex_1458/src/test/org/apache/lucene/TestSearch.java Thu Jan 14 19:05:12 2010
@@ -134,6 +134,7 @@
 
   static long Time(int year, int month, int day) {
     GregorianCalendar calendar = new GregorianCalendar();
+    calendar.clear();
     calendar.set(year, month, day);
     return calendar.getTime().getTime();
   }

Modified: lucene/java/branches/flex_1458/src/test/org/apache/lucene/TestSnapshotDeletionPolicy.java
URL: http://svn.apache.org/viewvc/lucene/java/branches/flex_1458/src/test/org/apache/lucene/TestSnapshotDeletionPolicy.java?rev=899359&r1=899358&r2=899359&view=diff
==============================================================================
--- lucene/java/branches/flex_1458/src/test/org/apache/lucene/TestSnapshotDeletionPolicy.java (original)
+++ lucene/java/branches/flex_1458/src/test/org/apache/lucene/TestSnapshotDeletionPolicy.java Thu Jan 14 19:05:12 2010
@@ -121,7 +121,7 @@
         public void run() {
           Document doc = new Document();
           doc.add(new Field("content", "aaa", Field.Store.YES, Field.Index.ANALYZED, Field.TermVector.WITH_POSITIONS_OFFSETS));
-          while(System.currentTimeMillis() < stopTime) {
+          do {
             for(int i=0;i<27;i++) {
               try {
                 writer.addDocument(doc);
@@ -142,7 +142,7 @@
             } catch (InterruptedException ie) {
               throw new ThreadInterruptedException(ie);
             }
-          }
+          } while(System.currentTimeMillis() < stopTime);
         }
       };
 
@@ -150,12 +150,10 @@
 
     // While the above indexing thread is running, take many
     // backups:
-    while(System.currentTimeMillis() < stopTime) {
+    do {
       backupIndex(dir, dp);
       Thread.sleep(20);
-      if (!t.isAlive())
-        break;
-    }
+    } while(t.isAlive());
 
     t.join();
 

Modified: lucene/java/branches/flex_1458/src/test/org/apache/lucene/analysis/TestAnalyzers.java
URL: http://svn.apache.org/viewvc/lucene/java/branches/flex_1458/src/test/org/apache/lucene/analysis/TestAnalyzers.java?rev=899359&r1=899358&r2=899359&view=diff
==============================================================================
--- lucene/java/branches/flex_1458/src/test/org/apache/lucene/analysis/TestAnalyzers.java (original)
+++ lucene/java/branches/flex_1458/src/test/org/apache/lucene/analysis/TestAnalyzers.java Thu Jan 14 19:05:12 2010
@@ -154,6 +154,7 @@
    * @deprecated remove this when lucene 3.0 "broken unicode 4" support
    * is no longer needed.
    */
+  @Deprecated
   private static class LowerCaseWhitespaceAnalyzerBWComp extends Analyzer {
 
     @Override
@@ -215,6 +216,7 @@
    * @deprecated remove this test when lucene 3.0 "broken unicode 4" support
    * is no longer needed.
    */
+  @Deprecated
   public void testLowerCaseFilterBWComp() throws IOException {
     Analyzer a = new LowerCaseWhitespaceAnalyzerBWComp();
     // BMP

Modified: lucene/java/branches/flex_1458/src/test/org/apache/lucene/analysis/TestCharArraySet.java
URL: http://svn.apache.org/viewvc/lucene/java/branches/flex_1458/src/test/org/apache/lucene/analysis/TestCharArraySet.java?rev=899359&r1=899358&r2=899359&view=diff
==============================================================================
--- lucene/java/branches/flex_1458/src/test/org/apache/lucene/analysis/TestCharArraySet.java (original)
+++ lucene/java/branches/flex_1458/src/test/org/apache/lucene/analysis/TestCharArraySet.java Thu Jan 14 19:05:12 2010
@@ -17,11 +17,16 @@
  * limitations under the License.
  */
 
+import java.util.ArrayList;
 import java.util.Arrays;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Set;
 
 import org.apache.lucene.util.LuceneTestCase;
 import org.apache.lucene.util.Version;
 
+
 public class TestCharArraySet extends LuceneTestCase {
   
   static final String[] TEST_STOP_WORDS = {
@@ -61,24 +66,29 @@
     Integer val = Integer.valueOf(1);
     set.add(val);
     assertTrue(set.contains(val));
-    assertTrue(set.contains(Integer.valueOf(1)));
+    assertTrue(set.contains(new Integer(1))); // another integer
+    assertTrue(set.contains("1"));
+    assertTrue(set.contains(new char[]{'1'}));
     // test unmodifiable
     set = CharArraySet.unmodifiableSet(set);
     assertTrue(set.contains(val));
-    assertTrue(set.contains(Integer.valueOf(1)));
+    assertTrue(set.contains(new Integer(1))); // another integer
+    assertTrue(set.contains("1"));
+    assertTrue(set.contains(new char[]{'1'}));
   }
   
   public void testClear(){
     CharArraySet set=new CharArraySet(Version.LUCENE_CURRENT, 10,true);
     set.addAll(Arrays.asList(TEST_STOP_WORDS));
     assertEquals("Not all words added", TEST_STOP_WORDS.length, set.size());
-    try{
-      set.clear();
-      fail("remove is not supported");
-    }catch (UnsupportedOperationException e) {
-      // expected
-      assertEquals("Not all words added", TEST_STOP_WORDS.length, set.size());
-    }
+    set.clear();
+    assertEquals("not empty", 0, set.size());
+    for(int i=0;i<TEST_STOP_WORDS.length;i++)
+      assertFalse(set.contains(TEST_STOP_WORDS[i]));
+    set.addAll(Arrays.asList(TEST_STOP_WORDS));
+    assertEquals("Not all words added", TEST_STOP_WORDS.length, set.size());
+    for(int i=0;i<TEST_STOP_WORDS.length;i++)
+      assertTrue(set.contains(TEST_STOP_WORDS[i]));
   }
   
   public void testModifyOnUnmodifiable(){
@@ -165,9 +175,16 @@
   public void testUnmodifiableSet(){
     CharArraySet set = new CharArraySet(Version.LUCENE_CURRENT, 10,true);
     set.addAll(Arrays.asList(TEST_STOP_WORDS));
+    set.add(Integer.valueOf(1));
     final int size = set.size();
     set = CharArraySet.unmodifiableSet(set);
     assertEquals("Set size changed due to unmodifiableSet call" , size, set.size());
+    for (String stopword : TEST_STOP_WORDS) {
+      assertTrue(set.contains(stopword));
+    }
+    assertTrue(set.contains(Integer.valueOf(1)));
+    assertTrue(set.contains("1"));
+    assertTrue(set.contains(new char[]{'1'}));
     
     try{
       CharArraySet.unmodifiableSet(null);
@@ -237,6 +254,7 @@
    * @deprecated remove this test when lucene 3.0 "broken unicode 4" support is
    *             no longer needed.
    */
+  @Deprecated
   public void testSupplementaryCharsBWCompat() {
     String missing = "Term %s is missing in the set";
     String falsePos = "Term %s is in the set but shouldn't";
@@ -268,6 +286,7 @@
    * @deprecated remove this test when lucene 3.0 "broken unicode 4" support is
    *             no longer needed.
    */
+  @Deprecated
   public void testSingleHighSurrogateBWComapt() {
     String missing = "Term %s is missing in the set";
     String falsePos = "Term %s is in the set but shouldn't";
@@ -301,4 +320,88 @@
           .contains(lowerArr[i]));
     }
   }
+  
+  /**
+   * Test the static #copy() function with a CharArraySet as a source
+   */
+  public void testCopyCharArraySet() {
+    CharArraySet setIngoreCase = new CharArraySet(Version.LUCENE_CURRENT, 10, true);
+    CharArraySet setCaseSensitive = new CharArraySet(Version.LUCENE_CURRENT, 10, false);
+
+    List<String> stopwords = Arrays.asList(TEST_STOP_WORDS);
+    List<String> stopwordsUpper = new ArrayList<String>();
+    for (String string : stopwords) {
+      stopwordsUpper.add(string.toUpperCase());
+    }
+    setIngoreCase.addAll(Arrays.asList(TEST_STOP_WORDS));
+    setIngoreCase.add(Integer.valueOf(1));
+    setCaseSensitive.addAll(Arrays.asList(TEST_STOP_WORDS));
+    setCaseSensitive.add(Integer.valueOf(1));
+
+    CharArraySet copy = CharArraySet.copy(Version.LUCENE_CURRENT, setIngoreCase);
+    CharArraySet copyCaseSens = CharArraySet.copy(Version.LUCENE_CURRENT, setCaseSensitive);
+
+    assertEquals(setIngoreCase.size(), copy.size());
+    assertEquals(setCaseSensitive.size(), copy.size());
+
+    assertTrue(copy.containsAll(stopwords));
+    assertTrue(copy.containsAll(stopwordsUpper));
+    assertTrue(copyCaseSens.containsAll(stopwords));
+    for (String string : stopwordsUpper) {
+      assertFalse(copyCaseSens.contains(string));
+    }
+    // test adding terms to the copy
+    List<String> newWords = new ArrayList<String>();
+    for (String string : stopwords) {
+      newWords.add(string+"_1");
+    }
+    copy.addAll(newWords);
+    
+    assertTrue(copy.containsAll(stopwords));
+    assertTrue(copy.containsAll(stopwordsUpper));
+    assertTrue(copy.containsAll(newWords));
+    // new added terms are not in the source set
+    for (String string : newWords) {
+      assertFalse(setIngoreCase.contains(string));  
+      assertFalse(setCaseSensitive.contains(string));  
+
+    }
+  }
+  
+  /**
+   * Test the static #copy() function with a JDK {@link Set} as a source
+   */
+  public void testCopyJDKSet() {
+    Set<String> set = new HashSet<String>();
+
+    List<String> stopwords = Arrays.asList(TEST_STOP_WORDS);
+    List<String> stopwordsUpper = new ArrayList<String>();
+    for (String string : stopwords) {
+      stopwordsUpper.add(string.toUpperCase());
+    }
+    set.addAll(Arrays.asList(TEST_STOP_WORDS));
+
+    CharArraySet copy = CharArraySet.copy(Version.LUCENE_CURRENT, set);
+
+    assertEquals(set.size(), copy.size());
+    assertEquals(set.size(), copy.size());
+
+    assertTrue(copy.containsAll(stopwords));
+    for (String string : stopwordsUpper) {
+      assertFalse(copy.contains(string));
+    }
+    
+    List<String> newWords = new ArrayList<String>();
+    for (String string : stopwords) {
+      newWords.add(string+"_1");
+    }
+    copy.addAll(newWords);
+    
+    assertTrue(copy.containsAll(stopwords));
+    assertTrue(copy.containsAll(newWords));
+    // new added terms are not in the source set
+    for (String string : newWords) {
+      assertFalse(set.contains(string));  
+    }
+  }
 }

Propchange: lucene/java/branches/flex_1458/src/test/org/apache/lucene/analysis/TestISOLatin1AccentFilter.java
------------------------------------------------------------------------------
--- svn:mergeinfo (original)
+++ svn:mergeinfo Thu Jan 14 19:05:12 2010
@@ -1,4 +1,4 @@
 /lucene/java/branches/lucene_2_4/src/test/org/apache/lucene/analysis/TestISOLatin1AccentFilter.java:748824
-/lucene/java/branches/lucene_2_9/src/test/org/apache/lucene/analysis/TestISOLatin1AccentFilter.java:829134,829881,831036
-/lucene/java/branches/lucene_3_0/src/test/org/apache/lucene/analysis/TestISOLatin1AccentFilter.java:880793
-/lucene/java/trunk/src/test/org/apache/lucene/analysis/TestISOLatin1AccentFilter.java:824912-825292,826213-833960,880727-886190,889185,889622,889667
+/lucene/java/branches/lucene_2_9/src/test/org/apache/lucene/analysis/TestISOLatin1AccentFilter.java:829134,829881,831036,896850
+/lucene/java/branches/lucene_3_0/src/test/org/apache/lucene/analysis/TestISOLatin1AccentFilter.java:880793,896906
+/lucene/java/trunk/src/test/org/apache/lucene/analysis/TestISOLatin1AccentFilter.java:824912-825292,826213-833960,880727-886190,889185,889622,889667,889866-899001

Modified: lucene/java/branches/flex_1458/src/test/org/apache/lucene/document/TestDateTools.java
URL: http://svn.apache.org/viewvc/lucene/java/branches/flex_1458/src/test/org/apache/lucene/document/TestDateTools.java?rev=899359&r1=899358&r2=899359&view=diff
==============================================================================
--- lucene/java/branches/flex_1458/src/test/org/apache/lucene/document/TestDateTools.java (original)
+++ lucene/java/branches/flex_1458/src/test/org/apache/lucene/document/TestDateTools.java Thu Jan 14 19:05:12 2010
@@ -59,6 +59,7 @@
   public void testStringtoTime() throws ParseException {
     long time = DateTools.stringToTime("197001010000");
     Calendar cal = new GregorianCalendar();
+    cal.clear();
     cal.set(1970, 0, 1,    // year=1970, month=january, day=1
         0, 0, 0);          // hour, minute, second
     cal.set(Calendar.MILLISECOND, 0);
@@ -73,6 +74,7 @@
   
   public void testDateAndTimetoString() throws ParseException {
     Calendar cal = new GregorianCalendar();
+    cal.clear();
     cal.setTimeZone(TimeZone.getTimeZone("GMT"));
     cal.set(2004, 1, 3,   // year=2004, month=february(!), day=3
         22, 8, 56);       // hour, minute, second
@@ -137,6 +139,7 @@
   
   public void testRound() {
     Calendar cal = new GregorianCalendar();
+    cal.clear();
     cal.setTimeZone(TimeZone.getTimeZone("GMT"));
     cal.set(2004, 1, 3,   // year=2004, month=february(!), day=3
         22, 8, 56);       // hour, minute, second
@@ -194,4 +197,4 @@
     }
   }
 
-}
+}
\ No newline at end of file

Propchange: lucene/java/branches/flex_1458/src/test/org/apache/lucene/document/TestDateTools.java
------------------------------------------------------------------------------
--- svn:mergeinfo (original)
+++ svn:mergeinfo Thu Jan 14 19:05:12 2010
@@ -1,4 +1,4 @@
 /lucene/java/branches/lucene_2_4/src/test/org/apache/lucene/document/TestDateTools.java:748824
-/lucene/java/branches/lucene_2_9/src/test/org/apache/lucene/document/TestDateTools.java:829134,829881,831036
-/lucene/java/branches/lucene_3_0/src/test/org/apache/lucene/document/TestDateTools.java:880793
-/lucene/java/trunk/src/test/org/apache/lucene/document/TestDateTools.java:824912-825292,826213-833960,880727-886190,889185,889622,889667
+/lucene/java/branches/lucene_2_9/src/test/org/apache/lucene/document/TestDateTools.java:829134,829881,831036,896850
+/lucene/java/branches/lucene_3_0/src/test/org/apache/lucene/document/TestDateTools.java:880793,896906
+/lucene/java/trunk/src/test/org/apache/lucene/document/TestDateTools.java:824912-825292,826213-833960,880727-886190,889185,889622,889667,889866-899001

Propchange: lucene/java/branches/flex_1458/src/test/org/apache/lucene/document/TestNumberTools.java
------------------------------------------------------------------------------
--- svn:mergeinfo (original)
+++ svn:mergeinfo Thu Jan 14 19:05:12 2010
@@ -1,4 +1,4 @@
 /lucene/java/branches/lucene_2_4/src/test/org/apache/lucene/document/TestNumberTools.java:748824
-/lucene/java/branches/lucene_2_9/src/test/org/apache/lucene/document/TestNumberTools.java:829134,829881,831036
-/lucene/java/branches/lucene_3_0/src/test/org/apache/lucene/document/TestNumberTools.java:880793
-/lucene/java/trunk/src/test/org/apache/lucene/document/TestNumberTools.java:824912-825292,826213-833960,880727-886190,889185,889622,889667
+/lucene/java/branches/lucene_2_9/src/test/org/apache/lucene/document/TestNumberTools.java:829134,829881,831036,896850
+/lucene/java/branches/lucene_3_0/src/test/org/apache/lucene/document/TestNumberTools.java:880793,896906
+/lucene/java/trunk/src/test/org/apache/lucene/document/TestNumberTools.java:824912-825292,826213-833960,880727-886190,889185,889622,889667,889866-899001

Modified: lucene/java/branches/flex_1458/src/test/org/apache/lucene/index/TestAtomicUpdate.java
URL: http://svn.apache.org/viewvc/lucene/java/branches/flex_1458/src/test/org/apache/lucene/index/TestAtomicUpdate.java?rev=899359&r1=899358&r2=899359&view=diff
==============================================================================
--- lucene/java/branches/flex_1458/src/test/org/apache/lucene/index/TestAtomicUpdate.java (original)
+++ lucene/java/branches/flex_1458/src/test/org/apache/lucene/index/TestAtomicUpdate.java Thu Jan 14 19:05:12 2010
@@ -45,7 +45,7 @@
   }
 
   private static abstract class TimedThread extends Thread {
-    boolean failed;
+    volatile boolean failed;
     int count;
     private static float RUN_TIME_SEC = 0.5f;
     private TimedThread[] allThreads;
@@ -63,10 +63,11 @@
       count = 0;
 
       try {
-        while(System.currentTimeMillis() < stopTime && !anyErrors()) {
+        do {
+          if (anyErrors()) break;
           doWork();
           count++;
-        }
+        } while(System.currentTimeMillis() < stopTime);
       } catch (Throwable e) {
         System.out.println(Thread.currentThread().getName() + ": exc");
         e.printStackTrace(System.out);

Propchange: lucene/java/branches/flex_1458/src/test/org/apache/lucene/index/TestBackwardsCompatibility.java
------------------------------------------------------------------------------
--- svn:mergeinfo (original)
+++ svn:mergeinfo Thu Jan 14 19:05:12 2010
@@ -1,4 +1,4 @@
 /lucene/java/branches/lucene_2_4/src/test/org/apache/lucene/index/TestBackwardsCompatibility.java:748824
-/lucene/java/branches/lucene_2_9/src/test/org/apache/lucene/index/TestBackwardsCompatibility.java:829134,829881,831036
-/lucene/java/branches/lucene_3_0/src/test/org/apache/lucene/index/TestBackwardsCompatibility.java:880793
-/lucene/java/trunk/src/test/org/apache/lucene/index/TestBackwardsCompatibility.java:824912-825292,826213-833960,880727-886190,889185,889622,889667
+/lucene/java/branches/lucene_2_9/src/test/org/apache/lucene/index/TestBackwardsCompatibility.java:829134,829881,831036,896850
+/lucene/java/branches/lucene_3_0/src/test/org/apache/lucene/index/TestBackwardsCompatibility.java:880793,896906
+/lucene/java/trunk/src/test/org/apache/lucene/index/TestBackwardsCompatibility.java:824912-825292,826213-833960,880727-886190,889185,889622,889667,889866-899001

Modified: lucene/java/branches/flex_1458/src/test/org/apache/lucene/index/TestIndexWriter.java
URL: http://svn.apache.org/viewvc/lucene/java/branches/flex_1458/src/test/org/apache/lucene/index/TestIndexWriter.java?rev=899359&r1=899358&r2=899359&view=diff
==============================================================================
--- lucene/java/branches/flex_1458/src/test/org/apache/lucene/index/TestIndexWriter.java (original)
+++ lucene/java/branches/flex_1458/src/test/org/apache/lucene/index/TestIndexWriter.java Thu Jan 14 19:05:12 2010
@@ -2208,7 +2208,7 @@
       int fullCount = 0;
       final long stopTime = System.currentTimeMillis() + 200;
 
-      while(System.currentTimeMillis() < stopTime) {
+      do {
         try {
           writer.updateDocument(new Term("id", ""+(idUpto++)), doc);
           addCount++;
@@ -2242,7 +2242,7 @@
           }
           break;
         }
-      }
+      } while(System.currentTimeMillis() < stopTime);
     }
   }
 
@@ -2335,6 +2335,12 @@
       fail("did not hit disk full");
     } catch (IOException ioe) {
     }
+
+    // Make sure once disk space is avail again, we can
+    // cleanly close:
+    dir.setMaxSizeInBytes(0);
+    writer.close(false);
+    dir.close();
   }
 
   // LUCENE-1130: make sure immediate disk full on creating
@@ -2370,11 +2376,10 @@
         assertTrue("hit unexpected Throwable", threads[i].error == null);
       }
 
-      try {
-        writer.close(false);
-      } catch (IOException ioe) {
-      }
-
+      // Make sure once disk space is avail again, we can
+      // cleanly close:
+      dir.setMaxSizeInBytes(0);
+      writer.close(false);
       dir.close();
     }
   }
@@ -4632,6 +4637,7 @@
     for(int i=0;i<NUM_THREADS;i++) {
       final int finalI = i;
       threads[i] = new Thread() {
+          @Override
           public void run() {
             try {
               final Document doc = new Document();
@@ -4639,7 +4645,8 @@
               Field f = new Field("f", "", Field.Store.NO, Field.Index.NOT_ANALYZED);
               doc.add(f);
               int count = 0;
-              while(System.currentTimeMillis() < endTime && !failed.get()) {
+              do {
+                if (failed.get()) break;
                 for(int j=0;j<10;j++) {
                   final String s = finalI + "_" + String.valueOf(count++);
                   f.setValue(s);
@@ -4651,7 +4658,7 @@
                   r = r2;
                   assertEquals("term=f:" + s, 1, r.docFreq(new Term("f", s)));
                 }
-              }
+              } while(System.currentTimeMillis() < endTime);
               r.close();
             } catch (Throwable t) {
               failed.set(true);

Modified: lucene/java/branches/flex_1458/src/test/org/apache/lucene/index/TestIndexWriterExceptions.java
URL: http://svn.apache.org/viewvc/lucene/java/branches/flex_1458/src/test/org/apache/lucene/index/TestIndexWriterExceptions.java?rev=899359&r1=899358&r2=899359&view=diff
==============================================================================
--- lucene/java/branches/flex_1458/src/test/org/apache/lucene/index/TestIndexWriterExceptions.java (original)
+++ lucene/java/branches/flex_1458/src/test/org/apache/lucene/index/TestIndexWriterExceptions.java Thu Jan 14 19:05:12 2010
@@ -65,7 +65,7 @@
 
       final long stopTime = System.currentTimeMillis() + 500;
 
-      while(System.currentTimeMillis() < stopTime) {
+      do {
         doFail.set(this);
         final String id = ""+r.nextInt(50);
         idField.setValue(id);
@@ -105,7 +105,7 @@
           failure = t;
           break;
         }
-      }
+      } while(System.currentTimeMillis() < stopTime);
     }
   }
 

Modified: lucene/java/branches/flex_1458/src/test/org/apache/lucene/index/TestIndexWriterMergePolicy.java
URL: http://svn.apache.org/viewvc/lucene/java/branches/flex_1458/src/test/org/apache/lucene/index/TestIndexWriterMergePolicy.java?rev=899359&r1=899358&r2=899359&view=diff
==============================================================================
--- lucene/java/branches/flex_1458/src/test/org/apache/lucene/index/TestIndexWriterMergePolicy.java (original)
+++ lucene/java/branches/flex_1458/src/test/org/apache/lucene/index/TestIndexWriterMergePolicy.java Thu Jan 14 19:05:12 2010
@@ -231,7 +231,7 @@
         numSegments++;
       } else {
         if (upperBound * mergeFactor <= maxMergeDocs) {
-          assertTrue(numSegments < mergeFactor);
+          assertTrue("maxMergeDocs=" + maxMergeDocs + "; numSegments=" + numSegments + "; upperBound=" + upperBound + "; mergeFactor=" + mergeFactor + "; segs=" + writer.segString(), numSegments < mergeFactor);
         }
 
         do {
@@ -265,4 +265,4 @@
     }
   }
   */
-}
+}
\ No newline at end of file

Modified: lucene/java/branches/flex_1458/src/test/org/apache/lucene/index/TestIndexWriterReader.java
URL: http://svn.apache.org/viewvc/lucene/java/branches/flex_1458/src/test/org/apache/lucene/index/TestIndexWriterReader.java?rev=899359&r1=899358&r2=899359&view=diff
==============================================================================
--- lucene/java/branches/flex_1458/src/test/org/apache/lucene/index/TestIndexWriterReader.java (original)
+++ lucene/java/branches/flex_1458/src/test/org/apache/lucene/index/TestIndexWriterReader.java Thu Jan 14 19:05:12 2010
@@ -701,14 +701,14 @@
       threads[i] = new Thread() {
           @Override
           public void run() {
-            while(System.currentTimeMillis() < endTime) {
+            do {
               try {
                 writer.addIndexesNoOptimize(dirs);
               } catch (Throwable t) {
                 excs.add(t);
                 throw new RuntimeException(t);
               }
-            }
+            } while(System.currentTimeMillis() < endTime);
           }
         };
       threads[i].setDaemon(true);
@@ -731,6 +731,15 @@
     for(int i=0;i<NUM_THREAD;i++) {
       threads[i].join();
     }
+    // final check
+    IndexReader r2 = r.reopen();
+    if (r2 != r) {
+      r.close();
+      r = r2;
+    }
+    Query q = new TermQuery(new Term("indexname", "test"));
+    final int count = new IndexSearcher(r).search(q, 10).totalHits;
+    assertTrue(count >= lastCount);
 
     assertEquals(0, excs.size());
     writer.close();
@@ -767,7 +776,7 @@
           public void run() {
             int count = 0;
             final Random r = new Random();
-            while(System.currentTimeMillis() < endTime) {
+            do {
               try {
                 for(int i=0;i<10;i++) {
                   writer.addDocument(createDocument(10*count+i, "test", 4));
@@ -782,7 +791,7 @@
                 excs.add(t);
                 throw new RuntimeException(t);
               }
-            }
+            } while(System.currentTimeMillis() < endTime);
           }
         };
       threads[i].setDaemon(true);
@@ -803,7 +812,16 @@
     for(int i=0;i<NUM_THREAD;i++) {
       threads[i].join();
     }
-    assertTrue(sum > 0);
+    // at least search once
+    IndexReader r2 = r.reopen();
+    if (r2 != r) {
+      r.close();
+      r = r2;
+    }
+    Query q = new TermQuery(new Term("indexname", "test"));
+    sum += new IndexSearcher(r).search(q, 10).totalHits;
+
+    assertTrue("no documents found at all", sum > 0);
 
     assertEquals(0, excs.size());
     writer.close();
@@ -837,4 +855,34 @@
     r.close();
     dir.close();
   }
+
+  public void testDeletesNumDocs() throws Throwable {
+    Directory dir = new MockRAMDirectory();
+    final IndexWriter w = new IndexWriter(dir, new WhitespaceAnalyzer(),
+                                               IndexWriter.MaxFieldLength.LIMITED);
+    Document doc = new Document();
+    doc.add(new Field("field", "a b c", Field.Store.NO, Field.Index.ANALYZED));
+    Field id = new Field("id", "", Field.Store.NO, Field.Index.NOT_ANALYZED);
+    doc.add(id);
+    id.setValue("0");
+    w.addDocument(doc);
+    id.setValue("1");
+    w.addDocument(doc);
+    IndexReader r = w.getReader();
+    assertEquals(2, r.numDocs());
+    r.close();
+
+    w.deleteDocuments(new Term("id", "0"));
+    r = w.getReader();
+    assertEquals(1, r.numDocs());
+    r.close();
+
+    w.deleteDocuments(new Term("id", "1"));
+    r = w.getReader();
+    assertEquals(0, r.numDocs());
+    r.close();
+
+    w.close();
+    dir.close();
+  }
 }

Modified: lucene/java/branches/flex_1458/src/test/org/apache/lucene/index/TestStressIndexing.java
URL: http://svn.apache.org/viewvc/lucene/java/branches/flex_1458/src/test/org/apache/lucene/index/TestStressIndexing.java?rev=899359&r1=899358&r2=899359&view=diff
==============================================================================
--- lucene/java/branches/flex_1458/src/test/org/apache/lucene/index/TestStressIndexing.java (original)
+++ lucene/java/branches/flex_1458/src/test/org/apache/lucene/index/TestStressIndexing.java Thu Jan 14 19:05:12 2010
@@ -30,7 +30,7 @@
   private Random RANDOM;
 
   private static abstract class TimedThread extends Thread {
-    boolean failed;
+    volatile boolean failed;
     int count;
     private static int RUN_TIME_SEC = 1;
     private TimedThread[] allThreads;
@@ -48,10 +48,11 @@
       count = 0;
 
       try {
-        while(System.currentTimeMillis() < stopTime && !anyErrors()) {
+        do {
+          if (anyErrors()) break;
           doWork();
           count++;
-        }
+        } while(System.currentTimeMillis() < stopTime);
       } catch (Throwable e) {
         System.out.println(Thread.currentThread() + ": exc");
         e.printStackTrace(System.out);

Modified: lucene/java/branches/flex_1458/src/test/org/apache/lucene/index/TestTransactions.java
URL: http://svn.apache.org/viewvc/lucene/java/branches/flex_1458/src/test/org/apache/lucene/index/TestTransactions.java?rev=899359&r1=899358&r2=899359&view=diff
==============================================================================
--- lucene/java/branches/flex_1458/src/test/org/apache/lucene/index/TestTransactions.java (original)
+++ lucene/java/branches/flex_1458/src/test/org/apache/lucene/index/TestTransactions.java Thu Jan 14 19:05:12 2010
@@ -38,7 +38,7 @@
   }
 
   private static abstract class TimedThread extends Thread {
-    boolean failed;
+    volatile boolean failed;
     private static float RUN_TIME_SEC = 0.5f;
     private TimedThread[] allThreads;
 
@@ -53,8 +53,10 @@
       final long stopTime = System.currentTimeMillis() + (long) (1000*RUN_TIME_SEC);
 
       try {
-        while(System.currentTimeMillis() < stopTime && !anyErrors())
+        do {
+          if (anyErrors()) break;
           doWork();
+        } while (System.currentTimeMillis() < stopTime);
       } catch (Throwable e) {
         System.out.println(Thread.currentThread() + ": exc");
         e.printStackTrace(System.out);

Modified: lucene/java/branches/flex_1458/src/test/org/apache/lucene/queryParser/TestQueryParser.java
URL: http://svn.apache.org/viewvc/lucene/java/branches/flex_1458/src/test/org/apache/lucene/queryParser/TestQueryParser.java?rev=899359&r1=899358&r2=899359&view=diff
==============================================================================
--- lucene/java/branches/flex_1458/src/test/org/apache/lucene/queryParser/TestQueryParser.java (original)
+++ lucene/java/branches/flex_1458/src/test/org/apache/lucene/queryParser/TestQueryParser.java Thu Jan 14 19:05:12 2010
@@ -543,24 +543,24 @@
       }
     }
   
-  private String getLocalizedDate(int year, int month, int day, boolean extendLastDate) {
+  private String getLocalizedDate(int year, int month, int day) {
     DateFormat df = DateFormat.getDateInstance(DateFormat.SHORT);
     Calendar calendar = new GregorianCalendar();
+    calendar.clear();
     calendar.set(year, month, day);
-    if (extendLastDate) {
-      calendar.set(Calendar.HOUR_OF_DAY, 23);
-      calendar.set(Calendar.MINUTE, 59);
-      calendar.set(Calendar.SECOND, 59);
-      calendar.set(Calendar.MILLISECOND, 999);
-    }
+    calendar.set(Calendar.HOUR_OF_DAY, 23);
+    calendar.set(Calendar.MINUTE, 59);
+    calendar.set(Calendar.SECOND, 59);
+    calendar.set(Calendar.MILLISECOND, 999);
     return df.format(calendar.getTime());
   }
 
   /** for testing legacy DateField support */
   public void testLegacyDateRange() throws Exception {
-    String startDate = getLocalizedDate(2002, 1, 1, false);
-    String endDate = getLocalizedDate(2002, 1, 4, false);
+    String startDate = getLocalizedDate(2002, 1, 1);
+    String endDate = getLocalizedDate(2002, 1, 4);
     Calendar endDateExpected = new GregorianCalendar();
+    endDateExpected.clear();
     endDateExpected.set(2002, 1, 4, 23, 59, 59);
     endDateExpected.set(Calendar.MILLISECOND, 999);
     assertQueryEquals("[ " + escapeDateString(startDate) + " TO " + escapeDateString(endDate) + "]", null,
@@ -570,9 +570,10 @@
   }
   
   public void testDateRange() throws Exception {
-    String startDate = getLocalizedDate(2002, 1, 1, false);
-    String endDate = getLocalizedDate(2002, 1, 4, false);
+    String startDate = getLocalizedDate(2002, 1, 1);
+    String endDate = getLocalizedDate(2002, 1, 4);
     Calendar endDateExpected = new GregorianCalendar();
+    endDateExpected.clear();
     endDateExpected.set(2002, 1, 4, 23, 59, 59);
     endDateExpected.set(Calendar.MILLISECOND, 999);
     final String defaultField = "default";

Modified: lucene/java/branches/flex_1458/src/test/org/apache/lucene/search/CheckHits.java
URL: http://svn.apache.org/viewvc/lucene/java/branches/flex_1458/src/test/org/apache/lucene/search/CheckHits.java?rev=899359&r1=899358&r2=899359&view=diff
==============================================================================
--- lucene/java/branches/flex_1458/src/test/org/apache/lucene/search/CheckHits.java (original)
+++ lucene/java/branches/flex_1458/src/test/org/apache/lucene/search/CheckHits.java Thu Jan 14 19:05:12 2010
@@ -456,6 +456,7 @@
      * @deprecated
      * @see CheckHits#EXPLAIN_SCORE_TOLERANCE_DELTA
      */
+    @Deprecated
     public static float SCORE_TOLERANCE_DELTA = 0.00005f;
 
     Query q;

Modified: lucene/java/branches/flex_1458/src/test/org/apache/lucene/search/QueryUtils.java
URL: http://svn.apache.org/viewvc/lucene/java/branches/flex_1458/src/test/org/apache/lucene/search/QueryUtils.java?rev=899359&r1=899358&r2=899359&view=diff
==============================================================================
--- lucene/java/branches/flex_1458/src/test/org/apache/lucene/search/QueryUtils.java (original)
+++ lucene/java/branches/flex_1458/src/test/org/apache/lucene/search/QueryUtils.java Thu Jan 14 19:05:12 2010
@@ -5,8 +5,6 @@
 import java.io.IOException;
 import java.io.ObjectInputStream;
 import java.io.ObjectOutputStream;
-import java.util.ArrayList;
-import java.util.List;
 
 import junit.framework.Assert;
 
@@ -17,7 +15,6 @@
 import org.apache.lucene.index.MultiReader;
 import org.apache.lucene.index.IndexWriter.MaxFieldLength;
 import org.apache.lucene.store.RAMDirectory;
-import org.apache.lucene.util.ReaderUtil;
 
 /**
  * Copyright 2005 Apache Software Foundation
@@ -283,6 +280,8 @@
         // FUTURE: ensure scorer.doc()==-1
 
         final float maxDiff = 1e-5f;
+        final IndexReader lastReader[] = {null};
+
         s.search(q, new Collector() {
           private Scorer sc;
           private IndexReader reader;
@@ -338,7 +337,18 @@
           }
 
           @Override
-          public void setNextReader(IndexReader reader, int docBase) {
+          public void setNextReader(IndexReader reader, int docBase) throws IOException {
+            // confirm that skipping beyond the last doc, on the
+            // previous reader, hits NO_MORE_DOCS
+            if (lastReader[0] != null) {
+              final IndexReader previousReader = lastReader[0];
+              Weight w = q.weight(new IndexSearcher(previousReader));
+              Scorer scorer = w.scorer(previousReader, true, false);
+              if (scorer != null) {
+                boolean more = scorer.advance(lastDoc[0] + 1) != DocIdSetIterator.NO_MORE_DOCS;
+                Assert.assertFalse("query's last doc was "+ lastDoc[0] +" but skipTo("+(lastDoc[0]+1)+") got to "+scorer.docID(),more);
+              }
+            }
             this.reader = reader;
             this.scorer = null;
             lastDoc[0] = -1;
@@ -350,19 +360,15 @@
           }
         });
 
-        List<IndexReader> readerList = new ArrayList<IndexReader>();
-        ReaderUtil.gatherSubReaders(readerList, s.getIndexReader());
-        IndexReader[] readers =  readerList.toArray(new IndexReader[0]);
-        for(int i = 0; i < readers.length; i++) {
-          IndexReader reader = readers[i];
-          Weight w = q.weight(s);
-          Scorer scorer = w.scorer(reader, true, false);
-          
+        if (lastReader[0] != null) {
+          // confirm that skipping beyond the last doc, on the
+          // previous reader, hits NO_MORE_DOCS
+          final IndexReader previousReader = lastReader[0];
+          Weight w = q.weight(new IndexSearcher(previousReader));
+          Scorer scorer = w.scorer(previousReader, true, false);
           if (scorer != null) {
             boolean more = scorer.advance(lastDoc[0] + 1) != DocIdSetIterator.NO_MORE_DOCS;
-      
-            if (more && lastDoc[0] != -1) 
-              Assert.assertFalse("query's last doc was "+ lastDoc[0] +" but skipTo("+(lastDoc[0]+1)+") got to "+scorer.docID(),more);
+            Assert.assertFalse("query's last doc was "+ lastDoc[0] +" but skipTo("+(lastDoc[0]+1)+") got to "+scorer.docID(),more);
           }
         }
       }
@@ -373,6 +379,8 @@
     //System.out.println("checkFirstSkipTo: "+q);
     final float maxDiff = 1e-5f;
     final int lastDoc[] = {-1};
+    final IndexReader lastReader[] = {null};
+
     s.search(q,new Collector() {
       private Scorer scorer;
       private IndexReader reader;
@@ -399,9 +407,22 @@
           throw new RuntimeException(e);
         }
       }
+
       @Override
-      public void setNextReader(IndexReader reader, int docBase) {
-        this.reader = reader;
+      public void setNextReader(IndexReader reader, int docBase) throws IOException {
+        // confirm that skipping beyond the last doc, on the
+        // previous reader, hits NO_MORE_DOCS
+        if (lastReader[0] != null) {
+          final IndexReader previousReader = lastReader[0];
+          Weight w = q.weight(new IndexSearcher(previousReader));
+          Scorer scorer = w.scorer(previousReader, true, false);
+          if (scorer != null) {
+            boolean more = scorer.advance(lastDoc[0] + 1) != DocIdSetIterator.NO_MORE_DOCS;
+            Assert.assertFalse("query's last doc was "+ lastDoc[0] +" but skipTo("+(lastDoc[0]+1)+") got to "+scorer.docID(),more);
+          }
+        }
+
+        this.reader = lastReader[0] = reader;
         lastDoc[0] = -1;
       }
       @Override
@@ -409,22 +430,17 @@
         return false;
       }
     });
-    
-    List<IndexReader> readerList = new ArrayList<IndexReader>();
-    ReaderUtil.gatherSubReaders(readerList, s.getIndexReader());
-    IndexReader[] readers = readerList.toArray(new IndexReader[0]);
-    for(int i = 0; i < readers.length; i++) {
-      IndexReader reader = readers[i];
-      Weight w = q.weight(s);
-      Scorer scorer = w.scorer(reader, true, false);
-      
+
+    if (lastReader[0] != null) {
+      // confirm that skipping beyond the last doc, on the
+      // previous reader, hits NO_MORE_DOCS
+      final IndexReader previousReader = lastReader[0];
+      Weight w = q.weight(new IndexSearcher(previousReader));
+      Scorer scorer = w.scorer(previousReader, true, false);
       if (scorer != null) {
         boolean more = scorer.advance(lastDoc[0] + 1) != DocIdSetIterator.NO_MORE_DOCS;
-  
-        if (more && lastDoc[0] != -1) 
-          Assert.assertFalse("query's last doc was "+ lastDoc[0] +" but skipTo("+(lastDoc[0]+1)+") got to "+scorer.docID(),more);
+        Assert.assertFalse("query's last doc was "+ lastDoc[0] +" but skipTo("+(lastDoc[0]+1)+") got to "+scorer.docID(),more);
       }
     }
-
   }
 }

Propchange: lucene/java/branches/flex_1458/src/test/org/apache/lucene/search/TestQueryWrapperFilter.java
------------------------------------------------------------------------------
    svn:eol-style = native

Modified: lucene/java/branches/flex_1458/src/test/org/apache/lucene/search/function/FunctionTestSetup.java
URL: http://svn.apache.org/viewvc/lucene/java/branches/flex_1458/src/test/org/apache/lucene/search/function/FunctionTestSetup.java?rev=899359&r1=899358&r2=899359&view=diff
==============================================================================
--- lucene/java/branches/flex_1458/src/test/org/apache/lucene/search/function/FunctionTestSetup.java (original)
+++ lucene/java/branches/flex_1458/src/test/org/apache/lucene/search/function/FunctionTestSetup.java Thu Jan 14 19:05:12 2010
@@ -67,9 +67,19 @@
           "So here we are, with a perhaps much less interesting ",
           "text for the test, but oh much much safer. ",
   };
+  
+  protected Directory dir;
+  protected Analyzer anlzr;
+  
+  private final boolean doMultiSegment;
 
-  protected Directory dir = null;
-  protected Analyzer anlzr = null;
+  public FunctionTestSetup(boolean doMultiSegment) {
+    this.doMultiSegment = doMultiSegment;
+  }
+
+  public FunctionTestSetup() {
+    this(false);
+  }
 
   @Override
   @After
@@ -100,7 +110,10 @@
       addDoc(iw, i);
       done[i] = true;
       i = (i + 4) % N_DOCS;
-      remaining--;
+      if (doMultiSegment && remaining % 3 == 0) {
+        iw.commit();
+      }
+      remaining --;
     }
     iw.close();
   }

Modified: lucene/java/branches/flex_1458/src/test/org/apache/lucene/search/function/TestCustomScoreQuery.java
URL: http://svn.apache.org/viewvc/lucene/java/branches/flex_1458/src/test/org/apache/lucene/search/function/TestCustomScoreQuery.java?rev=899359&r1=899358&r2=899359&view=diff
==============================================================================
--- lucene/java/branches/flex_1458/src/test/org/apache/lucene/search/function/TestCustomScoreQuery.java (original)
+++ lucene/java/branches/flex_1458/src/test/org/apache/lucene/search/function/TestCustomScoreQuery.java Thu Jan 14 19:05:12 2010
@@ -29,12 +29,18 @@
 import java.util.HashMap;
 import java.util.Map;
 
+import org.apache.lucene.index.IndexReader;
+
 /**
  * Test CustomScoreQuery search.
  */
 @SuppressWarnings({"MagicNumber"})
 public class TestCustomScoreQuery extends FunctionTestSetup {
 
+  /* @override constructor */
+  public TestCustomScoreQuery() {
+    super(true);
+  }
 
   /**
    * Test that CustomScoreQuery of Type.BYTE returns the expected scores.
@@ -73,7 +79,7 @@
     // INT field can be parsed as float
     doTestCustomScore(INT_FIELD, FieldScoreQuery.Type.FLOAT, 1.0);
     doTestCustomScore(INT_FIELD, FieldScoreQuery.Type.FLOAT, 5.0);
-    // same values, but in flot format
+    // same values, but in float format
     doTestCustomScore(FLOAT_FIELD, FieldScoreQuery.Type.FLOAT, 1.0);
     doTestCustomScore(FLOAT_FIELD, FieldScoreQuery.Type.FLOAT, 6.0);
   }
@@ -133,6 +139,8 @@
       }
       if (valSrcScores.length == 1) {
         return subQueryScore + valSrcScores[0];
+        // confirm that skipping beyond the last doc, on the
+        // previous reader, hits NO_MORE_DOCS
       }
       return (subQueryScore + valSrcScores[0]) * valSrcScores[1]; // we know there are two
     }
@@ -157,6 +165,44 @@
     }
   }
 
+  private final class CustomExternalQuery extends CustomScoreQuery {
+    private IndexReader reader;
+    private int[] values;
+
+    public float customScore(int doc, float subScore, float valSrcScore) {
+      assertTrue(doc <= reader.maxDoc());
+      return (float) values[doc];
+    }
+
+    public void setNextReader(IndexReader r) throws IOException {
+      reader = r;
+      values = FieldCache.DEFAULT.getInts(r, INT_FIELD);
+    }
+
+    public CustomExternalQuery(Query q) {
+      super(q);
+    }
+  }
+
+  public void testCustomExternalQuery() throws Exception {
+    QueryParser qp = new QueryParser(Version.LUCENE_CURRENT, TEXT_FIELD,anlzr); 
+    String qtxt = "first aid text"; // from the doc texts in FunctionQuerySetup.
+    Query q1 = qp.parse(qtxt); 
+    
+    final Query q = new CustomExternalQuery(q1);
+    log(q);
+
+    IndexSearcher s = new IndexSearcher(dir);
+    TopDocs hits = s.search(q, 1000);
+    assertEquals(N_DOCS, hits.totalHits);
+    for(int i=0;i<N_DOCS;i++) {
+      final int doc = hits.scoreDocs[i].doc;
+      final float score = hits.scoreDocs[i].score;
+      assertEquals("doc=" + doc, (float) 1+(4*doc) % N_DOCS, score, 0.0001);
+    }
+    s.close();
+  }
+  
   // Test that FieldScoreQuery returns docs with expected score.
   private void doTestCustomScore(String field, FieldScoreQuery.Type tp, double dboost) throws Exception, ParseException {
     float boost = (float) dboost;

Modified: lucene/java/branches/flex_1458/src/test/org/apache/lucene/search/function/TestFieldScoreQuery.java
URL: http://svn.apache.org/viewvc/lucene/java/branches/flex_1458/src/test/org/apache/lucene/search/function/TestFieldScoreQuery.java?rev=899359&r1=899358&r2=899359&view=diff
==============================================================================
--- lucene/java/branches/flex_1458/src/test/org/apache/lucene/search/function/TestFieldScoreQuery.java (original)
+++ lucene/java/branches/flex_1458/src/test/org/apache/lucene/search/function/TestFieldScoreQuery.java Thu Jan 14 19:05:12 2010
@@ -18,7 +18,6 @@
  */
 
 import java.util.HashMap;
-import java.util.Map;
 
 import org.apache.lucene.index.IndexReader;
 import org.apache.lucene.search.IndexSearcher;
@@ -43,6 +42,11 @@
 @SuppressWarnings({"UseOfSystemOutOrSystemErr"})
 public class TestFieldScoreQuery extends FunctionTestSetup {
 
+  /* @override constructor */
+  public TestFieldScoreQuery() {
+    super(true);
+  }
+
   /** Test that FieldScoreQuery of Type.BYTE returns docs in expected order. */
   @Test
   public void testRankByte () throws Exception {
@@ -174,7 +178,7 @@
     expectedArrayTypes.put(FieldScoreQuery.Type.FLOAT, new float[0]);
     
     IndexSearcher s = new IndexSearcher(dir, true);
-    Object innerArray = null;
+    Object[] innerArray = new Object[s.getIndexReader().getSequentialSubReaders().length];
 
     boolean warned = false; // print warning once.
     for (int i=0; i<10; i++) {
@@ -186,16 +190,16 @@
         IndexReader reader = readers[j];
         try {
           if (i == 0) {
-            innerArray = q.valSrc.getValues(reader).getInnerArray();
-            log(i + ".  compare: " + innerArray.getClass() + " to "
+            innerArray[j] = q.valSrc.getValues(reader).getInnerArray();
+            log(i + ".  compare: " + innerArray[j].getClass() + " to "
                 + expectedArrayTypes.get(tp).getClass());
             assertEquals(
                 "field values should be cached in the correct array type!",
-                innerArray.getClass(), expectedArrayTypes.get(tp).getClass());
+                innerArray[j].getClass(), expectedArrayTypes.get(tp).getClass());
           } else {
-            log(i + ".  compare: " + innerArray + " to "
+            log(i + ".  compare: " + innerArray[j] + " to "
                 + q.valSrc.getValues(reader).getInnerArray());
-            assertSame("field values should be cached and reused!", innerArray,
+            assertSame("field values should be cached and reused!", innerArray[j],
                 q.valSrc.getValues(reader).getInnerArray());
           }
         } catch (UnsupportedOperationException e) {

Modified: lucene/java/branches/flex_1458/src/test/org/apache/lucene/search/function/TestOrdValues.java
URL: http://svn.apache.org/viewvc/lucene/java/branches/flex_1458/src/test/org/apache/lucene/search/function/TestOrdValues.java?rev=899359&r1=899358&r2=899359&view=diff
==============================================================================
--- lucene/java/branches/flex_1458/src/test/org/apache/lucene/search/function/TestOrdValues.java (original)
+++ lucene/java/branches/flex_1458/src/test/org/apache/lucene/search/function/TestOrdValues.java Thu Jan 14 19:05:12 2010
@@ -37,6 +37,11 @@
 @SuppressWarnings({"UseOfSystemOutOrSystemErr"})
 public class TestOrdValues extends FunctionTestSetup {
 
+  /* @override constructor */
+  public TestOrdValues() {
+    super(false);
+  }
+
   /**
    * Test OrdFieldSource
    */

Modified: lucene/java/branches/flex_1458/src/test/org/apache/lucene/store/TestLockFactory.java
URL: http://svn.apache.org/viewvc/lucene/java/branches/flex_1458/src/test/org/apache/lucene/store/TestLockFactory.java?rev=899359&r1=899358&r2=899359&view=diff
==============================================================================
--- lucene/java/branches/flex_1458/src/test/org/apache/lucene/store/TestLockFactory.java (original)
+++ lucene/java/branches/flex_1458/src/test/org/apache/lucene/store/TestLockFactory.java Thu Jan 14 19:05:12 2010
@@ -198,6 +198,26 @@
       assertFalse(l2.isLocked());
     }
 
+    public void testNativeFSLockReleaseByOtherLock() throws IOException {
+
+      NativeFSLockFactory f = new NativeFSLockFactory(System.getProperty("tempDir"));
+
+      f.setLockPrefix("test");
+      Lock l = f.makeLock("commit");
+      Lock l2 = f.makeLock("commit");
+
+      assertTrue("failed to obtain lock", l.obtain());
+      try {
+        assertTrue(l2.isLocked());
+        l2.release();
+        fail("should not have reached here. LockReleaseFailedException should have been thrown");
+      } catch (IOException e) {
+        assertTrue("Unexpected exception", e instanceof LockReleaseFailedException);
+      } finally {
+        l.release();
+      }
+    }
+
     // Verify: NativeFSLockFactory assigns null as lockPrefix if the lockDir is inside directory
     public void testNativeFSLockFactoryPrefix() throws IOException {
 

Modified: lucene/java/branches/flex_1458/src/test/org/apache/lucene/util/English.java
URL: http://svn.apache.org/viewvc/lucene/java/branches/flex_1458/src/test/org/apache/lucene/util/English.java?rev=899359&r1=899358&r2=899359&view=diff
==============================================================================
--- lucene/java/branches/flex_1458/src/test/org/apache/lucene/util/English.java (original)
+++ lucene/java/branches/flex_1458/src/test/org/apache/lucene/util/English.java Thu Jan 14 19:05:12 2010
@@ -7,9 +7,9 @@
  * 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
- *
+ * <p/>
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * <p/>
  * 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.
@@ -20,13 +20,13 @@
 
 public class English {
 
-  public static String intToEnglish(int i) {
+  public static String longToEnglish(long i) {
     StringBuilder result = new StringBuilder();
-    intToEnglish(i, result);
+    longToEnglish(i, result);
     return result.toString();
   }
 
-  public static void intToEnglish(int i, StringBuilder result) {
+  public static void longToEnglish(long i, StringBuilder result) {
     if (i == 0) {
       result.append("zero");
       return;
@@ -35,69 +35,152 @@
       result.append("minus ");
       i = -i;
     }
-    if (i >= 1000000000) {			  // billions
-      intToEnglish(i/1000000000, result);
+    if (i >= 1000000000000000000l) {        // quadrillion
+      longToEnglish(i / 1000000000000000000l, result);
+      result.append("quintillion, ");
+      i = i % 1000000000000000000l;
+    }
+    if (i >= 1000000000000000l) {        // quadrillion
+      longToEnglish(i / 1000000000000000l, result);
+      result.append("quadrillion, ");
+      i = i % 1000000000000000l;
+    }
+    if (i >= 1000000000000l) {        // trillions
+      longToEnglish(i / 1000000000000l, result);
+      result.append("trillion, ");
+      i = i % 1000000000000l;
+    }
+    if (i >= 1000000000) {        // billions
+      longToEnglish(i / 1000000000, result);
       result.append("billion, ");
-      i = i%1000000000;
+      i = i % 1000000000;
     }
-    if (i >= 1000000) {				  // millions
-      intToEnglish(i/1000000, result);
+    if (i >= 1000000) {          // millions
+      longToEnglish(i / 1000000, result);
       result.append("million, ");
-      i = i%1000000;
+      i = i % 1000000;
     }
-    if (i >= 1000) {				  // thousands
-      intToEnglish(i/1000, result);
+    if (i >= 1000) {          // thousands
+      longToEnglish(i / 1000, result);
       result.append("thousand, ");
-      i = i%1000;
+      i = i % 1000;
     }
-    if (i >= 100) {				  // hundreds
-      intToEnglish(i/100, result);
+    if (i >= 100) {          // hundreds
+      longToEnglish(i / 100, result);
       result.append("hundred ");
-      i = i%100;
+      i = i % 100;
     }
+    //we know we are smaller here so we can cast
     if (i >= 20) {
-      switch (i/10) {
-      case 9 : result.append("ninety"); break;
-      case 8 : result.append("eighty"); break;
-      case 7 : result.append("seventy"); break;
-      case 6 : result.append("sixty"); break;
-      case 5 : result.append("fifty"); break;
-      case 4 : result.append("forty"); break;
-      case 3 : result.append("thirty"); break;
-      case 2 : result.append("twenty"); break;
+      switch (((int) i) / 10) {
+        case 9:
+          result.append("ninety");
+          break;
+        case 8:
+          result.append("eighty");
+          break;
+        case 7:
+          result.append("seventy");
+          break;
+        case 6:
+          result.append("sixty");
+          break;
+        case 5:
+          result.append("fifty");
+          break;
+        case 4:
+          result.append("forty");
+          break;
+        case 3:
+          result.append("thirty");
+          break;
+        case 2:
+          result.append("twenty");
+          break;
       }
-      i = i%10;
+      i = i % 10;
       if (i == 0)
         result.append(" ");
-      else 
+      else
         result.append("-");
     }
-    switch (i) {
-    case 19 : result.append("nineteen "); break;
-    case 18 : result.append("eighteen "); break;
-    case 17 : result.append("seventeen "); break;
-    case 16 : result.append("sixteen "); break;
-    case 15 : result.append("fifteen "); break;
-    case 14 : result.append("fourteen "); break;
-    case 13 : result.append("thirteen "); break;
-    case 12 : result.append("twelve "); break;
-    case 11 : result.append("eleven "); break;
-    case 10 : result.append("ten "); break;
-    case 9 : result.append("nine "); break;
-    case 8 : result.append("eight "); break;
-    case 7 : result.append("seven "); break;
-    case 6 : result.append("six "); break;
-    case 5 : result.append("five "); break;
-    case 4 : result.append("four "); break;
-    case 3 : result.append("three "); break;
-    case 2 : result.append("two "); break;
-    case 1 : result.append("one "); break;
-    case 0 : result.append(""); break;
+    switch ((int) i) {
+      case 19:
+        result.append("nineteen ");
+        break;
+      case 18:
+        result.append("eighteen ");
+        break;
+      case 17:
+        result.append("seventeen ");
+        break;
+      case 16:
+        result.append("sixteen ");
+        break;
+      case 15:
+        result.append("fifteen ");
+        break;
+      case 14:
+        result.append("fourteen ");
+        break;
+      case 13:
+        result.append("thirteen ");
+        break;
+      case 12:
+        result.append("twelve ");
+        break;
+      case 11:
+        result.append("eleven ");
+        break;
+      case 10:
+        result.append("ten ");
+        break;
+      case 9:
+        result.append("nine ");
+        break;
+      case 8:
+        result.append("eight ");
+        break;
+      case 7:
+        result.append("seven ");
+        break;
+      case 6:
+        result.append("six ");
+        break;
+      case 5:
+        result.append("five ");
+        break;
+      case 4:
+        result.append("four ");
+        break;
+      case 3:
+        result.append("three ");
+        break;
+      case 2:
+        result.append("two ");
+        break;
+      case 1:
+        result.append("one ");
+        break;
+      case 0:
+        result.append("");
+        break;
     }
   }
 
+
+  public static String intToEnglish(int i) {
+    StringBuilder result = new StringBuilder();
+    longToEnglish(i, result);
+    return result.toString();
+  }
+
+  public static void intToEnglish(int i, StringBuilder result) {
+    longToEnglish(i, result);
+  }
+
   public static void main(String[] args) {
-    System.out.println(intToEnglish(Integer.parseInt(args[0])));
+    System.out.println(longToEnglish(Long.parseLong(args[0])));
   }
 
 }

Modified: lucene/java/branches/flex_1458/src/test/org/apache/lucene/util/LocalizedTestCase.java
URL: http://svn.apache.org/viewvc/lucene/java/branches/flex_1458/src/test/org/apache/lucene/util/LocalizedTestCase.java?rev=899359&r1=899358&r2=899359&view=diff
==============================================================================
--- lucene/java/branches/flex_1458/src/test/org/apache/lucene/util/LocalizedTestCase.java (original)
+++ lucene/java/branches/flex_1458/src/test/org/apache/lucene/util/LocalizedTestCase.java Thu Jan 14 19:05:12 2010
@@ -73,6 +73,8 @@
 
   @Override
   protected void tearDown() throws Exception {
+    assertEquals("default locale unexpectedly changed:", locale, Locale
+        .getDefault());
     Locale.setDefault(defaultLocale);
     super.tearDown();
   }