You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@lucene.apache.org by mi...@apache.org on 2010/05/10 09:47:13 UTC

svn commit: r942676 - in /lucene/dev/trunk/lucene: ./ src/java/org/apache/lucene/index/codecs/preflex/ src/java/org/apache/lucene/index/codecs/standard/ src/java/org/apache/lucene/search/ src/java/org/apache/lucene/util/ src/java/org/apache/lucene/util...

Author: mikemccand
Date: Mon May 10 07:47:12 2010
New Revision: 942676

URL: http://svn.apache.org/viewvc?rev=942676&view=rev
Log:
LUCENE-2449: add 'knob' (-Drandom.multiplier=N, defaults to 1) to tune up how much random testing is done; fix sneaky bugs (workaround java's bug in new String(int[], int, int); nasty key reuse bug in DBLRUCache) that this deeper testing uncovered

Modified:
    lucene/dev/trunk/lucene/LICENSE.txt
    lucene/dev/trunk/lucene/common-build.xml
    lucene/dev/trunk/lucene/src/java/org/apache/lucene/index/codecs/preflex/TermInfosReader.java
    lucene/dev/trunk/lucene/src/java/org/apache/lucene/index/codecs/standard/StandardTermsDictReader.java
    lucene/dev/trunk/lucene/src/java/org/apache/lucene/search/FuzzyTermsEnum.java
    lucene/dev/trunk/lucene/src/java/org/apache/lucene/search/MultiSearcher.java
    lucene/dev/trunk/lucene/src/java/org/apache/lucene/util/DoubleBarrelLRUCache.java
    lucene/dev/trunk/lucene/src/java/org/apache/lucene/util/UnicodeUtil.java
    lucene/dev/trunk/lucene/src/java/org/apache/lucene/util/automaton/BasicOperations.java
    lucene/dev/trunk/lucene/src/test/org/apache/lucene/analysis/TestCharArrayMap.java
    lucene/dev/trunk/lucene/src/test/org/apache/lucene/analysis/TestCharTokenizers.java
    lucene/dev/trunk/lucene/src/test/org/apache/lucene/index/FlexTestUtil.java
    lucene/dev/trunk/lucene/src/test/org/apache/lucene/index/TestAtomicUpdate.java
    lucene/dev/trunk/lucene/src/test/org/apache/lucene/index/TestByteSlices.java
    lucene/dev/trunk/lucene/src/test/org/apache/lucene/index/TestCodecs.java
    lucene/dev/trunk/lucene/src/test/org/apache/lucene/index/TestIndexReaderReopen.java
    lucene/dev/trunk/lucene/src/test/org/apache/lucene/index/TestIndexWriter.java
    lucene/dev/trunk/lucene/src/test/org/apache/lucene/index/TestPayloads.java
    lucene/dev/trunk/lucene/src/test/org/apache/lucene/index/TestStressIndexing.java
    lucene/dev/trunk/lucene/src/test/org/apache/lucene/index/TestStressIndexing2.java
    lucene/dev/trunk/lucene/src/test/org/apache/lucene/index/TestTransactions.java
    lucene/dev/trunk/lucene/src/test/org/apache/lucene/search/TestBoolean2.java
    lucene/dev/trunk/lucene/src/test/org/apache/lucene/search/TestBooleanMinShouldMatch.java
    lucene/dev/trunk/lucene/src/test/org/apache/lucene/search/TestCustomSearcherSort.java
    lucene/dev/trunk/lucene/src/test/org/apache/lucene/search/TestMultiValuedNumericRangeQuery.java
    lucene/dev/trunk/lucene/src/test/org/apache/lucene/search/TestNumericRangeQuery32.java
    lucene/dev/trunk/lucene/src/test/org/apache/lucene/search/TestNumericRangeQuery64.java
    lucene/dev/trunk/lucene/src/test/org/apache/lucene/search/TestRegexpRandom.java
    lucene/dev/trunk/lucene/src/test/org/apache/lucene/search/TestRegexpRandom2.java
    lucene/dev/trunk/lucene/src/test/org/apache/lucene/search/TestScorerPerf.java
    lucene/dev/trunk/lucene/src/test/org/apache/lucene/search/TestSort.java
    lucene/dev/trunk/lucene/src/test/org/apache/lucene/search/TestThreadSafe.java
    lucene/dev/trunk/lucene/src/test/org/apache/lucene/search/TestWildcardRandom.java
    lucene/dev/trunk/lucene/src/test/org/apache/lucene/store/TestWindowsMMap.java
    lucene/dev/trunk/lucene/src/test/org/apache/lucene/util/TestArrayUtil.java
    lucene/dev/trunk/lucene/src/test/org/apache/lucene/util/TestDoubleBarrelLRUCache.java
    lucene/dev/trunk/lucene/src/test/org/apache/lucene/util/TestIndexableBinaryStringTools.java
    lucene/dev/trunk/lucene/src/test/org/apache/lucene/util/TestOpenBitSet.java
    lucene/dev/trunk/lucene/src/test/org/apache/lucene/util/TestPriorityQueue.java
    lucene/dev/trunk/lucene/src/test/org/apache/lucene/util/TestSmallFloat.java
    lucene/dev/trunk/lucene/src/test/org/apache/lucene/util/TestStringIntern.java
    lucene/dev/trunk/lucene/src/test/org/apache/lucene/util/TestUnicodeUtil.java
    lucene/dev/trunk/lucene/src/test/org/apache/lucene/util/_TestUtil.java
    lucene/dev/trunk/lucene/src/test/org/apache/lucene/util/automaton/TestBasicOperations.java
    lucene/dev/trunk/lucene/src/test/org/apache/lucene/util/automaton/TestDeterminism.java
    lucene/dev/trunk/lucene/src/test/org/apache/lucene/util/automaton/TestDeterminizeLexicon.java
    lucene/dev/trunk/lucene/src/test/org/apache/lucene/util/automaton/TestUTF32ToUTF8.java
    lucene/dev/trunk/lucene/src/test/org/apache/lucene/util/packed/TestPackedInts.java

Modified: lucene/dev/trunk/lucene/LICENSE.txt
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/LICENSE.txt?rev=942676&r1=942675&r2=942676&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/LICENSE.txt (original)
+++ lucene/dev/trunk/lucene/LICENSE.txt Mon May 10 07:47:12 2010
@@ -300,3 +300,37 @@ Here is the copyright for those sources:
 # WHETHER IN AN ACTION OF CONTRACT, TORT OR OTHERWISE, ARISING
 # FROM, OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR
 # OTHER DEALINGS IN THE SOFTWARE.
+
+Some code in src/java/org/apache/lucene/util/UnicodeUtil.java was
+derived from ICU (http://www.icu-project.org)
+The full license is available here: 
+  http://source.icu-project.org/repos/icu/icu/trunk/license.html
+
+/*
+ * Copyright (C) 1999-2010, International Business Machines
+ * Corporation and others.  All Rights Reserved.
+ *
+ * Permission is hereby granted, free of charge, to any person obtaining a copy 
+ * of this software and associated documentation files (the "Software"), to deal
+ * in the Software without restriction, including without limitation the rights 
+ * to use, copy, modify, merge, publish, distribute, and/or sell copies of the 
+ * Software, and to permit persons to whom the Software is furnished to do so, 
+ * provided that the above copyright notice(s) and this permission notice appear 
+ * in all copies of the Software and that both the above copyright notice(s) and
+ * this permission notice appear in supporting documentation.
+ * 
+ * THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, EXPRESS OR 
+ * IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES OF MERCHANTABILITY, 
+ * FITNESS FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT OF THIRD PARTY RIGHTS. 
+ * IN NO EVENT SHALL THE COPYRIGHT HOLDER OR HOLDERS INCLUDED IN THIS NOTICE BE 
+ * LIABLE FOR ANY CLAIM, OR ANY SPECIAL INDIRECT OR CONSEQUENTIAL DAMAGES, OR 
+ * ANY DAMAGES WHATSOEVER RESULTING FROM LOSS OF USE, DATA OR PROFITS, WHETHER 
+ * IN AN ACTION OF CONTRACT, NEGLIGENCE OR OTHER TORTIOUS ACTION, ARISING OUT 
+ * OF OR IN CONNECTION WITH THE USE OR PERFORMANCE OF THIS SOFTWARE.
+ *
+ * Except as contained in this notice, the name of a copyright holder shall not 
+ * be used in advertising or otherwise to promote the sale, use or other 
+ * dealings in this Software without prior written authorization of the 
+ * copyright holder.
+ */
+ 
\ No newline at end of file

Modified: lucene/dev/trunk/lucene/common-build.xml
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/common-build.xml?rev=942676&r1=942675&r2=942676&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/common-build.xml (original)
+++ lucene/dev/trunk/lucene/common-build.xml Mon May 10 07:47:12 2010
@@ -55,6 +55,7 @@
   <!-- default arguments to pass to JVM executing tests -->
   <property name="args" value=""/>
   <property name="threadsPerProcessor" value="1" />
+  <property name="random.multiplier" value="1" />
     
   <property name="javac.deprecation" value="off"/>
   <property name="javac.debug" value="on"/>
@@ -439,6 +440,7 @@
 	      <sysproperty key="tests.verbose" value="${tests.verbose}"/>
 	
 	      <!-- TODO: create propertyset for test properties, so each project can have its own set -->
+              <sysproperty key="random.multiplier" value="${random.multiplier}"/>
 	      <sysproperty key="tempDir" file="@{tempDir}/@{threadNum}"/>
 
 	      <sysproperty key="lucene.version" value="${dev.version}"/>

Modified: lucene/dev/trunk/lucene/src/java/org/apache/lucene/index/codecs/preflex/TermInfosReader.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/src/java/org/apache/lucene/index/codecs/preflex/TermInfosReader.java?rev=942676&r1=942675&r2=942676&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/src/java/org/apache/lucene/index/codecs/preflex/TermInfosReader.java (original)
+++ lucene/dev/trunk/lucene/src/java/org/apache/lucene/index/codecs/preflex/TermInfosReader.java Mon May 10 07:47:12 2010
@@ -61,7 +61,26 @@ public final class TermInfosReader {
     }
   }
 
-  private final DoubleBarrelLRUCache<Term,TermInfoAndOrd> termsCache = new DoubleBarrelLRUCache<Term,TermInfoAndOrd>(DEFAULT_CACHE_SIZE);
+  private static class CloneableTerm extends DoubleBarrelLRUCache.CloneableKey {
+    private Term term;
+    public CloneableTerm(Term t) {
+      this.term = t;
+    }
+
+    public boolean equals(Object other) {
+      return this.term.equals(other);
+    }
+
+    public int hashCode() {
+      return term.hashCode();
+    }
+
+    public Object clone() {
+      return new CloneableTerm(term);
+    }
+  }
+
+  private final DoubleBarrelLRUCache<CloneableTerm,TermInfoAndOrd> termsCache = new DoubleBarrelLRUCache<CloneableTerm,TermInfoAndOrd>(DEFAULT_CACHE_SIZE);
 
   /**
    * Per-thread resources managed by ThreadLocal
@@ -197,8 +216,7 @@ public final class TermInfosReader {
     if (size == 0) return null;
 
     ensureIndexIsRead();
-
-    TermInfoAndOrd tiOrd = termsCache.get(term);
+    TermInfoAndOrd tiOrd = termsCache.get(new CloneableTerm(term));
     ThreadResources resources = getThreadResources();
 
     if (!mustSeekEnum && tiOrd != null) {
@@ -209,7 +227,7 @@ public final class TermInfosReader {
   }
 
   TermInfo seekEnum(SegmentTermEnum enumerator, Term term) throws IOException {
-    return seekEnum(enumerator, term, termsCache.get(term));
+    return seekEnum(enumerator, term, termsCache.get(new CloneableTerm(term)));
   }
 
   TermInfo seekEnum(SegmentTermEnum enumerator, Term term, TermInfoAndOrd tiOrd) throws IOException {
@@ -235,7 +253,7 @@ public final class TermInfosReader {
             // wipe out the cache when they iterate over a large numbers
             // of terms in order
             if (tiOrd == null) {
-              termsCache.put(term, new TermInfoAndOrd(ti, (int) enumerator.position));
+              termsCache.put(new CloneableTerm(term), new TermInfoAndOrd(ti, (int) enumerator.position));
             } else {
               assert sameTermInfo(ti, tiOrd, enumerator);
               assert (int) enumerator.position == tiOrd.termOrd;
@@ -264,7 +282,7 @@ public final class TermInfosReader {
     if (enumerator.term() != null && term.compareTo(enumerator.term()) == 0) {
       ti = enumerator.termInfo();
       if (tiOrd == null) {
-        termsCache.put(term, new TermInfoAndOrd(ti, (int) enumerator.position));
+        termsCache.put(new CloneableTerm(term), new TermInfoAndOrd(ti, (int) enumerator.position));
       } else {
         assert sameTermInfo(ti, tiOrd, enumerator);
         assert (int) enumerator.position == tiOrd.termOrd;

Modified: lucene/dev/trunk/lucene/src/java/org/apache/lucene/index/codecs/standard/StandardTermsDictReader.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/src/java/org/apache/lucene/index/codecs/standard/StandardTermsDictReader.java?rev=942676&r1=942675&r2=942676&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/src/java/org/apache/lucene/index/codecs/standard/StandardTermsDictReader.java (original)
+++ lucene/dev/trunk/lucene/src/java/org/apache/lucene/index/codecs/standard/StandardTermsDictReader.java Mon May 10 07:47:12 2010
@@ -73,7 +73,7 @@ public class StandardTermsDictReader ext
   private StandardTermsIndexReader indexReader;
 
   // Used as key for the terms cache
-  private static class FieldAndTerm {
+  private static class FieldAndTerm extends DoubleBarrelLRUCache.CloneableKey {
     String field;
     BytesRef term;
 
@@ -92,6 +92,11 @@ public class StandardTermsDictReader ext
     }
 
     @Override
+    public Object clone() {
+      return new FieldAndTerm(this);
+    }
+
+    @Override
     public int hashCode() {
       return field.hashCode() * 31 + term.hashCode();
     }
@@ -291,7 +296,7 @@ public class StandardTermsDictReader ext
         fieldTerm.term = term;
         TermState cachedState;
         if (useCache) {
-          cachedState = termsCache.get(fieldTerm);
+          cachedState = termsCache.get(new FieldAndTerm(fieldTerm));
           if (cachedState != null) {
             state.copy(cachedState);
             seekPending = true;
@@ -370,13 +375,12 @@ public class StandardTermsDictReader ext
           } else if (cmp > 0) {
             return SeekStatus.NOT_FOUND;
           }
-
           // The purpose of the terms dict index is to seek
           // the enum to the closest index term before the
           // term we are looking for.  So, we should never
           // cross another index term (besides the first
           // one) while we are scanning:
-          assert state.ord == startOrd || !fieldIndexReader.isIndexTerm(state.ord, state.docFreq, true);
+          assert state.ord == startOrd || !fieldIndexReader.isIndexTerm(state.ord, state.docFreq, true): "state.ord=" + state.ord + " startOrd=" + startOrd + " ir.isIndexTerm=" + fieldIndexReader.isIndexTerm(state.ord, state.docFreq, true) + " state.docFreq=" + state.docFreq;
         }
 
         return SeekStatus.END;

Modified: lucene/dev/trunk/lucene/src/java/org/apache/lucene/search/FuzzyTermsEnum.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/src/java/org/apache/lucene/search/FuzzyTermsEnum.java?rev=942676&r1=942675&r2=942676&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/src/java/org/apache/lucene/search/FuzzyTermsEnum.java (original)
+++ lucene/dev/trunk/lucene/src/java/org/apache/lucene/search/FuzzyTermsEnum.java Mon May 10 07:47:12 2010
@@ -133,7 +133,7 @@ public final class FuzzyTermsEnum extend
     if (runAutomata == null && 
         maxDistance <= LevenshteinAutomata.MAXIMUM_SUPPORTED_DISTANCE) {
       LevenshteinAutomata builder = 
-        new LevenshteinAutomata(new String(termText, realPrefixLength, termText.length - realPrefixLength));
+        new LevenshteinAutomata(UnicodeUtil.newString(termText, realPrefixLength, termText.length - realPrefixLength));
 
       runAutomata = new ArrayList<ByteRunAutomaton>(maxDistance);
       for (int i = 0; i <= maxDistance; i++) {
@@ -141,7 +141,7 @@ public final class FuzzyTermsEnum extend
         // constant prefix
         if (realPrefixLength > 0) {
           Automaton prefix = BasicAutomata.makeString(
-              new String(termText, 0, realPrefixLength));
+              UnicodeUtil.newString(termText, 0, realPrefixLength));
           a = BasicOperations.concatenate(prefix, a);
         }
         runAutomata.add(new ByteRunAutomaton(a));
@@ -364,7 +364,7 @@ public final class FuzzyTermsEnum extend
 
       this.text = new int[termLength - realPrefixLength];
       System.arraycopy(termText, realPrefixLength, text, 0, text.length);
-      final String prefix = new String(termText, 0, realPrefixLength);
+      final String prefix = UnicodeUtil.newString(termText, 0, realPrefixLength);
       prefixBytesRef = new BytesRef(prefix);
       initializeMaxDistances();
       this.d = initDistanceArray();

Modified: lucene/dev/trunk/lucene/src/java/org/apache/lucene/search/MultiSearcher.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/src/java/org/apache/lucene/search/MultiSearcher.java?rev=942676&r1=942675&r2=942676&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/src/java/org/apache/lucene/search/MultiSearcher.java (original)
+++ lucene/dev/trunk/lucene/src/java/org/apache/lucene/search/MultiSearcher.java Mon May 10 07:47:12 2010
@@ -201,6 +201,7 @@ public class MultiSearcher extends Searc
   public TopDocs search(Weight weight, Filter filter, int nDocs)
       throws IOException {
 
+    nDocs = Math.min(nDocs, maxDoc());
     final HitQueue hq = new HitQueue(nDocs, false);
     int totalHits = 0;
 
@@ -221,6 +222,7 @@ public class MultiSearcher extends Searc
 
   @Override
   public TopFieldDocs search (Weight weight, Filter filter, int n, Sort sort) throws IOException {
+    n = Math.min(n, maxDoc());
     FieldDocSortedHitQueue hq = new FieldDocSortedHitQueue(n);
     int totalHits = 0;
 

Modified: lucene/dev/trunk/lucene/src/java/org/apache/lucene/util/DoubleBarrelLRUCache.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/src/java/org/apache/lucene/util/DoubleBarrelLRUCache.java?rev=942676&r1=942675&r2=942676&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/src/java/org/apache/lucene/util/DoubleBarrelLRUCache.java (original)
+++ lucene/dev/trunk/lucene/src/java/org/apache/lucene/util/DoubleBarrelLRUCache.java Mon May 10 07:47:12 2010
@@ -42,6 +42,11 @@ import java.util.Map;
  */
 
 final public class DoubleBarrelLRUCache<K,V> {
+
+  public static abstract class CloneableKey {
+    abstract public Object clone();
+  }
+
   private final Map<K,V> cache1;
   private final Map<K,V> cache2;
   private final AtomicInteger countdown;

Modified: lucene/dev/trunk/lucene/src/java/org/apache/lucene/util/UnicodeUtil.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/src/java/org/apache/lucene/util/UnicodeUtil.java?rev=942676&r1=942675&r2=942676&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/src/java/org/apache/lucene/util/UnicodeUtil.java (original)
+++ lucene/dev/trunk/lucene/src/java/org/apache/lucene/util/UnicodeUtil.java Mon May 10 07:47:12 2010
@@ -49,6 +49,42 @@ package org.apache.lucene.util;
  * remains attached.
  */
 
+/*
+ * Additional code came from the IBM ICU library.
+ *
+ *  http://www.icu-project.org
+ *
+ * Full Copyright for that code follows.
+ */
+
+/*
+ * Copyright (C) 1999-2010, International Business Machines
+ * Corporation and others.  All Rights Reserved.
+ *
+ * Permission is hereby granted, free of charge, to any person obtaining a copy
+ * of this software and associated documentation files (the "Software"), to deal
+ * in the Software without restriction, including without limitation the rights
+ * to use, copy, modify, merge, publish, distribute, and/or sell copies of the
+ * Software, and to permit persons to whom the Software is furnished to do so,
+ * provided that the above copyright notice(s) and this permission notice appear
+ * in all copies of the Software and that both the above copyright notice(s) and
+ * this permission notice appear in supporting documentation.
+ *
+ * THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, EXPRESS OR
+ * IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES OF MERCHANTABILITY,
+ * FITNESS FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT OF THIRD PARTY RIGHTS.
+ * IN NO EVENT SHALL THE COPYRIGHT HOLDER OR HOLDERS INCLUDED IN THIS NOTICE BE
+ * LIABLE FOR ANY CLAIM, OR ANY SPECIAL INDIRECT OR CONSEQUENTIAL DAMAGES, OR
+ * ANY DAMAGES WHATSOEVER RESULTING FROM LOSS OF USE, DATA OR PROFITS, WHETHER
+ * IN AN ACTION OF CONTRACT, NEGLIGENCE OR OTHER TORTIOUS ACTION, ARISING OUT
+ * OF OR IN CONNECTION WITH THE USE OR PERFORMANCE OF THIS SOFTWARE.
+ *
+ * Except as contained in this notice, the name of a copyright holder shall not
+ * be used in advertising or otherwise to promote the sale, use or other
+ * dealings in this Software without prior written authorization of the
+ * copyright holder.
+ */
+
 /**
  * Class to encode java's UTF16 char[] into UTF8 byte[]
  * without always allocating a new byte[] as
@@ -579,4 +615,62 @@ final public class UnicodeUtil {
     utf32.offset = 0;
     utf32.length = utf32Count;
   }
+
+  /** Shift value for lead surrogate to form a supplementary character. */
+  private static final int LEAD_SURROGATE_SHIFT_ = 10;
+  /** Mask to retrieve the significant value from a trail surrogate.*/
+  private static final int TRAIL_SURROGATE_MASK_ = 0x3FF;
+  /** Trail surrogate minimum value */
+  private static final int TRAIL_SURROGATE_MIN_VALUE = 0xDC00;
+  /** Lead surrogate minimum value */
+  private static final int LEAD_SURROGATE_MIN_VALUE = 0xD800;
+  /** The minimum value for Supplementary code points */
+  private static final int SUPPLEMENTARY_MIN_VALUE = 0x10000;
+  /** Value that all lead surrogate starts with */
+  private static final int LEAD_SURROGATE_OFFSET_ = LEAD_SURROGATE_MIN_VALUE
+          - (SUPPLEMENTARY_MIN_VALUE >> LEAD_SURROGATE_SHIFT_);
+
+  /**
+   * Cover JDK 1.5 API. Create a String from an array of codePoints.
+   *
+   * @param codePoints The code array
+   * @param offset The start of the text in the code point array
+   * @param count The number of code points
+   * @return a String representing the code points between offset and count
+   * @throws IllegalArgumentException If an invalid code point is encountered
+   * @throws IndexOutOfBoundsException If the offset or count are out of bounds.
+   */
+  public static String newString(int[] codePoints, int offset, int count) {
+      if (count < 0) {
+          throw new IllegalArgumentException();
+      }
+      char[] chars = new char[count];
+      int w = 0;
+      for (int r = offset, e = offset + count; r < e; ++r) {
+          int cp = codePoints[r];
+          if (cp < 0 || cp > 0x10ffff) {
+              throw new IllegalArgumentException();
+          }
+          while (true) {
+              try {
+                  if (cp < 0x010000) {
+                      chars[w] = (char) cp;
+                      w++;
+                  } else {
+                      chars[w] = (char) (LEAD_SURROGATE_OFFSET_ + (cp >> LEAD_SURROGATE_SHIFT_));
+                      chars[w + 1] = (char) (TRAIL_SURROGATE_MIN_VALUE + (cp & TRAIL_SURROGATE_MASK_));
+                      w += 2;
+                  }
+                  break;
+              } catch (IndexOutOfBoundsException ex) {
+                  int newlen = (int) (Math.ceil((double) codePoints.length * (w + 2)
+                          / (r - offset + 1)));
+                  char[] temp = new char[newlen];
+                  System.arraycopy(chars, 0, temp, 0, w);
+                  chars = temp;
+              }
+          }
+      }
+      return new String(chars, 0, w);
+  }
 }

Modified: lucene/dev/trunk/lucene/src/java/org/apache/lucene/util/automaton/BasicOperations.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/src/java/org/apache/lucene/util/automaton/BasicOperations.java?rev=942676&r1=942675&r2=942676&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/src/java/org/apache/lucene/util/automaton/BasicOperations.java (original)
+++ lucene/dev/trunk/lucene/src/java/org/apache/lucene/util/automaton/BasicOperations.java Mon May 10 07:47:12 2010
@@ -30,7 +30,6 @@
 package org.apache.lucene.util.automaton;
 
 import org.apache.lucene.util.ArrayUtil;
-import org.apache.lucene.util.UnicodeUtil;
 import org.apache.lucene.util.RamUsageEstimator;
 
 import java.util.ArrayList;
@@ -855,13 +854,7 @@ final public class BasicOperations {
   // "defined" in UTF32.  Don't call this on a transition
   // that only accepts UTF16 surrogate values!!
   private static int getRandomCodePoint(final Random r, final Transition t) {
-    while(true) {
-      final int v = t.min+r.nextInt(t.max-t.min+1);
-      if (v < UnicodeUtil.UNI_SUR_HIGH_START ||
-          v > UnicodeUtil.UNI_SUR_LOW_END) {
-        return v;
-      }
-    }
+    return t.min+r.nextInt(t.max-t.min+1);
   }
 
   public static class RandomAcceptedStrings {

Modified: lucene/dev/trunk/lucene/src/test/org/apache/lucene/analysis/TestCharArrayMap.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/src/test/org/apache/lucene/analysis/TestCharArrayMap.java?rev=942676&r1=942675&r2=942676&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/src/test/org/apache/lucene/analysis/TestCharArrayMap.java (original)
+++ lucene/dev/trunk/lucene/src/test/org/apache/lucene/analysis/TestCharArrayMap.java Mon May 10 07:47:12 2010
@@ -19,6 +19,7 @@ package org.apache.lucene.analysis;
 
 import java.util.*;
 import org.apache.lucene.util.LuceneTestCase;
+import org.apache.lucene.util._TestUtil;
 
 public class TestCharArrayMap extends LuceneTestCase {
   Random r = newRandom();
@@ -55,7 +56,7 @@ public class TestCharArrayMap extends Lu
   }
 
   public void testCharArrayMap() {
-    for (int i=0; i<5; i++) {  // pump this up for more random testing
+    for (int i=0; i<5*_TestUtil.getRandomMultiplier(); i++) {  // pump this up for more random testing
       doRandom(1000,false);
       doRandom(1000,true);      
     }

Modified: lucene/dev/trunk/lucene/src/test/org/apache/lucene/analysis/TestCharTokenizers.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/src/test/org/apache/lucene/analysis/TestCharTokenizers.java?rev=942676&r1=942675&r2=942676&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/src/test/org/apache/lucene/analysis/TestCharTokenizers.java (original)
+++ lucene/dev/trunk/lucene/src/test/org/apache/lucene/analysis/TestCharTokenizers.java Mon May 10 07:47:12 2010
@@ -23,6 +23,7 @@ import java.io.StringReader;
 import java.util.Random;
 
 import org.apache.lucene.util.Version;
+import org.apache.lucene.util._TestUtil;
 
 /**
  * Testcase for {@link CharTokenizer} subclasses
@@ -38,7 +39,7 @@ public class TestCharTokenizers extends 
     Random newRandom = newRandom();
     // create random input
     int num = 1024 + newRandom.nextInt(1024);
-    for (int i = 1; i < num; i++) {
+    for (int i = 1; i < num*_TestUtil.getRandomMultiplier(); i++) {
       builder.append("\ud801\udc1cabc");
       if((i % 10) == 0)
         builder.append(" ");

Modified: lucene/dev/trunk/lucene/src/test/org/apache/lucene/index/FlexTestUtil.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/src/test/org/apache/lucene/index/FlexTestUtil.java?rev=942676&r1=942675&r2=942676&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/src/test/org/apache/lucene/index/FlexTestUtil.java (original)
+++ lucene/dev/trunk/lucene/src/test/org/apache/lucene/index/FlexTestUtil.java Mon May 10 07:47:12 2010
@@ -428,7 +428,7 @@ public class FlexTestUtil {
   }
 
   private static void testRandomSeeks(Random rand, IndexReader r) throws Exception {
-    final int ITER = 100;
+    final int ITER = 100 * _TestUtil.getRandomMultiplier();
     List<String> allFields = new ArrayList<String>();
     Fields fields = MultiFields.getFields(r);
     if (fields == null) {

Modified: lucene/dev/trunk/lucene/src/test/org/apache/lucene/index/TestAtomicUpdate.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/src/test/org/apache/lucene/index/TestAtomicUpdate.java?rev=942676&r1=942675&r2=942676&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/src/test/org/apache/lucene/index/TestAtomicUpdate.java (original)
+++ lucene/dev/trunk/lucene/src/test/org/apache/lucene/index/TestAtomicUpdate.java Mon May 10 07:47:12 2010
@@ -47,7 +47,7 @@ public class TestAtomicUpdate extends Lu
   private static abstract class TimedThread extends Thread {
     volatile boolean failed;
     int count;
-    private static float RUN_TIME_SEC = 0.5f;
+    private static float RUN_TIME_SEC = 0.5f * (float)_TestUtil.getRandomMultiplier();
     private TimedThread[] allThreads;
 
     abstract public void doWork() throws Throwable;

Modified: lucene/dev/trunk/lucene/src/test/org/apache/lucene/index/TestByteSlices.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/src/test/org/apache/lucene/index/TestByteSlices.java?rev=942676&r1=942675&r2=942676&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/src/test/org/apache/lucene/index/TestByteSlices.java (original)
+++ lucene/dev/trunk/lucene/src/test/org/apache/lucene/index/TestByteSlices.java Mon May 10 07:47:12 2010
@@ -18,6 +18,7 @@ import java.util.Random;
 import java.util.ArrayList;
 import java.util.List;
 import org.apache.lucene.util.LuceneTestCase;
+import org.apache.lucene.util._TestUtil;
 
 public class TestByteSlices extends LuceneTestCase {
 
@@ -73,7 +74,7 @@ public class TestByteSlices extends Luce
         counters[stream] = 0;
       }
       
-      for(int iter=0;iter<10000;iter++) {
+      for(int iter=0;iter<10000*_TestUtil.getRandomMultiplier();iter++) {
         int stream = r.nextInt(NUM_STREAM);
         if (VERBOSE)
           System.out.println("write stream=" + stream);

Modified: lucene/dev/trunk/lucene/src/test/org/apache/lucene/index/TestCodecs.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/src/test/org/apache/lucene/index/TestCodecs.java?rev=942676&r1=942675&r2=942676&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/src/test/org/apache/lucene/index/TestCodecs.java (original)
+++ lucene/dev/trunk/lucene/src/test/org/apache/lucene/index/TestCodecs.java Mon May 10 07:47:12 2010
@@ -44,6 +44,7 @@ import org.apache.lucene.store.RAMDirect
 import org.apache.lucene.util.BytesRef;
 import org.apache.lucene.util.MultiCodecTestCase;
 import org.apache.lucene.util.Version;
+import org.apache.lucene.util._TestUtil;
 
 // TODO: test multiple codecs here?
 
@@ -68,7 +69,7 @@ public class TestCodecs extends MultiCod
   private Random RANDOM;
   private static String[] fieldNames = new String[] {"one", "two", "three", "four"};
 
-  private final static int NUM_TEST_ITER = 20;
+  private final static int NUM_TEST_ITER = 20*_TestUtil.getRandomMultiplier();
   private final static int NUM_TEST_THREADS = 3;
   private final static int NUM_FIELDS = 4;
   private final static int NUM_TERMS_RAND = 50; // must be > 16 to test skipping

Modified: lucene/dev/trunk/lucene/src/test/org/apache/lucene/index/TestIndexReaderReopen.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/src/test/org/apache/lucene/index/TestIndexReaderReopen.java?rev=942676&r1=942675&r2=942676&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/src/test/org/apache/lucene/index/TestIndexReaderReopen.java (original)
+++ lucene/dev/trunk/lucene/src/test/org/apache/lucene/index/TestIndexReaderReopen.java Mon May 10 07:47:12 2010
@@ -47,6 +47,7 @@ import org.apache.lucene.store.MockRAMDi
 import org.apache.lucene.store.AlreadyClosedException;
 import org.apache.lucene.util.LuceneTestCase;
 import org.apache.lucene.util.BitVector;
+import org.apache.lucene.util._TestUtil;
 
 public class TestIndexReaderReopen extends LuceneTestCase {
     
@@ -702,7 +703,7 @@ public class TestIndexReaderReopen exten
   
   public void testThreadSafety() throws Exception {
     final Directory dir = new MockRAMDirectory();
-    final int n = 30;
+    final int n = 30*_TestUtil.getRandomMultiplier();
 
     IndexWriter writer = new IndexWriter(dir, new IndexWriterConfig(
         TEST_VERSION_CURRENT, new StandardAnalyzer(TEST_VERSION_CURRENT)));

Modified: lucene/dev/trunk/lucene/src/test/org/apache/lucene/index/TestIndexWriter.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/src/test/org/apache/lucene/index/TestIndexWriter.java?rev=942676&r1=942675&r2=942676&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/src/test/org/apache/lucene/index/TestIndexWriter.java (original)
+++ lucene/dev/trunk/lucene/src/test/org/apache/lucene/index/TestIndexWriter.java Mon May 10 07:47:12 2010
@@ -3407,7 +3407,7 @@ public class TestIndexWriter extends Luc
     BytesRef utf8 = new BytesRef(20);
     UnicodeUtil.UTF16Result utf16 = new UnicodeUtil.UTF16Result();
 
-    for(int iter=0;iter<100000;iter++) {
+    for(int iter=0;iter<100000*_TestUtil.getRandomMultiplier();iter++) {
       boolean hasIllegal = fillUnicode(buffer, expected, 0, 20);
 
       UnicodeUtil.UTF16toUTF8(buffer, 0, 20, utf8);
@@ -3438,7 +3438,7 @@ public class TestIndexWriter extends Luc
     boolean hasIllegal = false;
     byte[] last = new byte[60];
 
-    for(int iter=0;iter<100000;iter++) {
+    for(int iter=0;iter<100000*_TestUtil.getRandomMultiplier();iter++) {
 
       final int prefix;
 
@@ -4748,7 +4748,7 @@ public class TestIndexWriter extends Luc
     Random rnd = newRandom();
     final Set<String> allTerms = new HashSet<String>();
 
-    for(int i=0;i<200;i++) {
+    for(int i=0;i<200*_TestUtil.getRandomMultiplier();i++) {
 
       final String s;
       if (rnd.nextBoolean()) {
@@ -4783,7 +4783,6 @@ public class TestIndexWriter extends Luc
 
     // Test each sub-segment
     final IndexReader[] subs = r.getSequentialSubReaders();
-    assertEquals(5, subs.length);
     for(int i=0;i<subs.length;i++) {
       checkTermsOrder(subs[i], allTerms, false);
     }
@@ -4940,7 +4939,7 @@ public class TestIndexWriter extends Luc
     //w.setInfoStream(System.out);
     Document doc = new Document();
     doc.add(new Field("field", "go 1 2 3 4 5 6 7 8 9 10 11 12 13 14 15 16 17 18 19 20", Field.Store.NO, Field.Index.ANALYZED));
-    for(int iter=0;iter<6;iter++) {
+    for(int iter=0;iter<6*_TestUtil.getRandomMultiplier();iter++) {
       int count = 0;
 
       final boolean doIndexing = r.nextBoolean();

Modified: lucene/dev/trunk/lucene/src/test/org/apache/lucene/index/TestPayloads.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/src/test/org/apache/lucene/index/TestPayloads.java?rev=942676&r1=942675&r2=942676&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/src/test/org/apache/lucene/index/TestPayloads.java (original)
+++ lucene/dev/trunk/lucene/src/test/org/apache/lucene/index/TestPayloads.java Mon May 10 07:47:12 2010
@@ -481,7 +481,7 @@ public class TestPayloads extends Lucene
     public void testThreadSafety() throws Exception {
         rnd = newRandom();
         final int numThreads = 5;
-        final int numDocs = 50;
+        final int numDocs = 50* _TestUtil.getRandomMultiplier();
         final ByteArrayPool pool = new ByteArrayPool(numThreads, 5);
         
         Directory dir = new MockRAMDirectory();

Modified: lucene/dev/trunk/lucene/src/test/org/apache/lucene/index/TestStressIndexing.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/src/test/org/apache/lucene/index/TestStressIndexing.java?rev=942676&r1=942675&r2=942676&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/src/test/org/apache/lucene/index/TestStressIndexing.java (original)
+++ lucene/dev/trunk/lucene/src/test/org/apache/lucene/index/TestStressIndexing.java Mon May 10 07:47:12 2010
@@ -32,7 +32,7 @@ public class TestStressIndexing extends 
   private static abstract class TimedThread extends Thread {
     volatile boolean failed;
     int count;
-    private static int RUN_TIME_SEC = 1;
+    private static int RUN_TIME_SEC = 1*_TestUtil.getRandomMultiplier();
     private TimedThread[] allThreads;
 
     abstract public void doWork() throws Throwable;

Modified: lucene/dev/trunk/lucene/src/test/org/apache/lucene/index/TestStressIndexing2.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/src/test/org/apache/lucene/index/TestStressIndexing2.java?rev=942676&r1=942675&r2=942676&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/src/test/org/apache/lucene/index/TestStressIndexing2.java (original)
+++ lucene/dev/trunk/lucene/src/test/org/apache/lucene/index/TestStressIndexing2.java Mon May 10 07:47:12 2010
@@ -103,7 +103,7 @@ public class TestStressIndexing2 extends
 
     r = newRandom();
 
-    for (int i=0; i<3; i++) {  // increase iterations for better testing
+    for (int i=0; i<3*_TestUtil.getRandomMultiplier(); i++) {  // increase iterations for better testing
       sameFieldOrder=r.nextBoolean();
       mergeFactor=r.nextInt(3)+2;
       maxBufferedDocs=r.nextInt(3)+2;

Modified: lucene/dev/trunk/lucene/src/test/org/apache/lucene/index/TestTransactions.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/src/test/org/apache/lucene/index/TestTransactions.java?rev=942676&r1=942675&r2=942676&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/src/test/org/apache/lucene/index/TestTransactions.java (original)
+++ lucene/dev/trunk/lucene/src/test/org/apache/lucene/index/TestTransactions.java Mon May 10 07:47:12 2010
@@ -27,6 +27,7 @@ import org.apache.lucene.store.Directory
 import org.apache.lucene.store.MockRAMDirectory;
 import org.apache.lucene.util.English;
 import org.apache.lucene.util.LuceneTestCase;
+import org.apache.lucene.util._TestUtil;
 
 public class TestTransactions extends LuceneTestCase {
   
@@ -43,7 +44,7 @@ public class TestTransactions extends Lu
 
   private static abstract class TimedThread extends Thread {
     volatile boolean failed;
-    private static float RUN_TIME_SEC = 0.5f;
+    private static float RUN_TIME_SEC = 0.5f*_TestUtil.getRandomMultiplier();
     private TimedThread[] allThreads;
 
     abstract public void doWork() throws Throwable;

Modified: lucene/dev/trunk/lucene/src/test/org/apache/lucene/search/TestBoolean2.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/src/test/org/apache/lucene/search/TestBoolean2.java?rev=942676&r1=942675&r2=942676&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/src/test/org/apache/lucene/search/TestBoolean2.java (original)
+++ lucene/dev/trunk/lucene/src/test/org/apache/lucene/search/TestBoolean2.java Mon May 10 07:47:12 2010
@@ -33,6 +33,7 @@ import org.apache.lucene.store.RAMDirect
 import org.apache.lucene.store.MockRAMDirectory;
 import org.apache.lucene.store.Directory;
 import org.apache.lucene.util.LuceneTestCase;
+import org.apache.lucene.util._TestUtil;
 
 /** Test BooleanQuery2 against BooleanQuery by overriding the standard query parser.
  * This also tests the scoring order of BooleanQuery.
@@ -209,7 +210,7 @@ public class TestBoolean2 extends Lucene
     try {
 
       // increase number of iterations for more complete testing
-      for (int i=0; i<50; i++) {
+      for (int i=0; i<50*_TestUtil.getRandomMultiplier(); i++) {
         int level = rnd.nextInt(3);
         q1 = randBoolQuery(new Random(rnd.nextLong()), rnd.nextBoolean(), level, field, vals, null);
         

Modified: lucene/dev/trunk/lucene/src/test/org/apache/lucene/search/TestBooleanMinShouldMatch.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/src/test/org/apache/lucene/search/TestBooleanMinShouldMatch.java?rev=942676&r1=942675&r2=942676&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/src/test/org/apache/lucene/search/TestBooleanMinShouldMatch.java (original)
+++ lucene/dev/trunk/lucene/src/test/org/apache/lucene/search/TestBooleanMinShouldMatch.java Mon May 10 07:47:12 2010
@@ -19,6 +19,7 @@ package org.apache.lucene.search;
 
 
 import org.apache.lucene.util.LuceneTestCase;
+import org.apache.lucene.util._TestUtil;
 import org.apache.lucene.analysis.WhitespaceAnalyzer;
 import org.apache.lucene.document.Document;
 import org.apache.lucene.document.Field;
@@ -315,7 +316,7 @@ public class TestBooleanMinShouldMatch e
 
 
       // increase number of iterations for more complete testing      
-      for (int i=0; i<50; i++) {
+      for (int i=0; i<50*_TestUtil.getRandomMultiplier(); i++) {
         int lev = rnd.nextInt(maxLev);
         final long seed = rnd.nextLong();
         BooleanQuery q1 = TestBoolean2.randBoolQuery(new Random(seed), true, lev, field, vals, null);

Modified: lucene/dev/trunk/lucene/src/test/org/apache/lucene/search/TestCustomSearcherSort.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/src/test/org/apache/lucene/search/TestCustomSearcherSort.java?rev=942676&r1=942675&r2=942676&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/src/test/org/apache/lucene/search/TestCustomSearcherSort.java (original)
+++ lucene/dev/trunk/lucene/src/test/org/apache/lucene/search/TestCustomSearcherSort.java Mon May 10 07:47:12 2010
@@ -39,6 +39,7 @@ import org.apache.lucene.index.Term;
 import org.apache.lucene.store.Directory;
 import org.apache.lucene.store.RAMDirectory;
 import org.apache.lucene.util.LuceneTestCase;
+import org.apache.lucene.util._TestUtil;
 
 /**
  * Unit test for sorting code.
@@ -52,7 +53,7 @@ implements Serializable {
     private Directory index = null;
     private Query query = null;
     // reduced from 20000 to 2000 to speed up test...
-    private final static int INDEX_SIZE = 2000;
+    private final static int INDEX_SIZE = 2000*_TestUtil.getRandomMultiplier();
 
   public TestCustomSearcherSort (String name) {
     super (name);
@@ -152,7 +153,7 @@ implements Serializable {
   private void matchHits (Searcher searcher, Sort sort)
   throws IOException {
       // make a query without sorting first
-    ScoreDoc[] hitsByRank = searcher.search(query, null, 1000).scoreDocs;
+    ScoreDoc[] hitsByRank = searcher.search(query, null, Integer.MAX_VALUE).scoreDocs;
     checkHits(hitsByRank, "Sort by rank: "); // check for duplicates
         Map<Integer,Integer> resultMap = new TreeMap<Integer,Integer>();
         // store hits in TreeMap - TreeMap does not allow duplicates; existing entries are silently overwritten
@@ -163,7 +164,7 @@ implements Serializable {
         }
         
         // now make a query using the sort criteria
-    ScoreDoc[] resultSort = searcher.search (query, null, 1000, sort).scoreDocs;
+    ScoreDoc[] resultSort = searcher.search (query, null, Integer.MAX_VALUE, sort).scoreDocs;
     checkHits(resultSort, "Sort by custom criteria: "); // check for duplicates
     
         // besides the sorting both sets of hits must be identical

Modified: lucene/dev/trunk/lucene/src/test/org/apache/lucene/search/TestMultiValuedNumericRangeQuery.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/src/test/org/apache/lucene/search/TestMultiValuedNumericRangeQuery.java?rev=942676&r1=942675&r2=942676&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/src/test/org/apache/lucene/search/TestMultiValuedNumericRangeQuery.java (original)
+++ lucene/dev/trunk/lucene/src/test/org/apache/lucene/search/TestMultiValuedNumericRangeQuery.java Mon May 10 07:47:12 2010
@@ -30,6 +30,7 @@ import org.apache.lucene.index.IndexWrit
 import org.apache.lucene.index.IndexWriterConfig;
 import org.apache.lucene.store.RAMDirectory;
 import org.apache.lucene.util.LuceneTestCase;
+import org.apache.lucene.util._TestUtil;
 
 public class TestMultiValuedNumericRangeQuery extends LuceneTestCase {
 
@@ -47,7 +48,7 @@ public class TestMultiValuedNumericRange
     
     DecimalFormat format = new DecimalFormat("00000000000", new DecimalFormatSymbols(Locale.US));
     
-    for (int l=0; l<5000; l++) {
+    for (int l=0; l<5000*_TestUtil.getRandomMultiplier(); l++) {
       Document doc = new Document();
       for (int m=0, c=rnd.nextInt(10); m<=c; m++) {
         int value = rnd.nextInt(Integer.MAX_VALUE);
@@ -59,7 +60,7 @@ public class TestMultiValuedNumericRange
     writer.close();
     
     Searcher searcher=new IndexSearcher(directory, true);
-    for (int i=0; i<50; i++) {
+    for (int i=0; i<50*_TestUtil.getRandomMultiplier(); i++) {
       int lower=rnd.nextInt(Integer.MAX_VALUE);
       int upper=rnd.nextInt(Integer.MAX_VALUE);
       if (lower>upper) {

Modified: lucene/dev/trunk/lucene/src/test/org/apache/lucene/search/TestNumericRangeQuery32.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/src/test/org/apache/lucene/search/TestNumericRangeQuery32.java?rev=942676&r1=942675&r2=942676&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/src/test/org/apache/lucene/search/TestNumericRangeQuery32.java (original)
+++ lucene/dev/trunk/lucene/src/test/org/apache/lucene/search/TestNumericRangeQuery32.java Mon May 10 07:47:12 2010
@@ -24,13 +24,13 @@ import org.apache.lucene.document.Docume
 import org.apache.lucene.document.Field;
 import org.apache.lucene.document.NumericField;
 import org.apache.lucene.index.IndexWriter;
-import org.apache.lucene.index.IndexWriter.MaxFieldLength;
 import org.apache.lucene.index.IndexWriterConfig;
 import org.apache.lucene.index.TermsEnum;
 import org.apache.lucene.store.RAMDirectory;
 import org.apache.lucene.util.BytesRef;
 import org.apache.lucene.util.LuceneTestCaseJ4;
 import org.apache.lucene.util.NumericUtils;
+import org.apache.lucene.util._TestUtil;
 
 import org.junit.Test;
 import org.junit.AfterClass;
@@ -43,7 +43,7 @@ public class TestNumericRangeQuery32 ext
   // shift the starting of the values to the left, to also have negative values:
   private static final int startOffset = - 1 << 15;
   // number of docs to generate for testing
-  private static final int noDocs = 10000;
+  private static final int noDocs = 10000*_TestUtil.getRandomMultiplier();
   
   private static RAMDirectory directory = null;
   private static IndexSearcher searcher = null;
@@ -327,7 +327,7 @@ public class TestNumericRangeQuery32 ext
     final Random rnd=newRandom();
     String field="field"+precisionStep;
     int termCountT=0,termCountC=0;
-    for (int i=0; i<10; i++) {
+    for (int i=0; i<10*_TestUtil.getRandomMultiplier(); i++) {
       int lower=(int)(rnd.nextDouble()*noDocs*distance)+startOffset;
       int upper=(int)(rnd.nextDouble()*noDocs*distance)+startOffset;
       if (lower>upper) {
@@ -405,7 +405,7 @@ public class TestNumericRangeQuery32 ext
     final Random rnd=newRandom();
     String field="ascfield"+precisionStep;
     // 10 random tests
-    for (int i=0; i<10; i++) {
+    for (int i=0; i<10*_TestUtil.getRandomMultiplier(); i++) {
       int lower=(int)(rnd.nextDouble()*noDocs - noDocs/2);
       int upper=(int)(rnd.nextDouble()*noDocs - noDocs/2);
       if (lower>upper) {
@@ -481,7 +481,7 @@ public class TestNumericRangeQuery32 ext
     String field="field"+precisionStep;
     // 10 random tests, the index order is ascending,
     // so using a reverse sort field should retun descending documents
-    for (int i=0; i<10; i++) {
+    for (int i=0; i<10*_TestUtil.getRandomMultiplier(); i++) {
       int lower=(int)(rnd.nextDouble()*noDocs*distance)+startOffset;
       int upper=(int)(rnd.nextDouble()*noDocs*distance)+startOffset;
       if (lower>upper) {

Modified: lucene/dev/trunk/lucene/src/test/org/apache/lucene/search/TestNumericRangeQuery64.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/src/test/org/apache/lucene/search/TestNumericRangeQuery64.java?rev=942676&r1=942675&r2=942676&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/src/test/org/apache/lucene/search/TestNumericRangeQuery64.java (original)
+++ lucene/dev/trunk/lucene/src/test/org/apache/lucene/search/TestNumericRangeQuery64.java Mon May 10 07:47:12 2010
@@ -29,6 +29,7 @@ import org.apache.lucene.store.RAMDirect
 import org.apache.lucene.util.BytesRef;
 import org.apache.lucene.util.LuceneTestCaseJ4;
 import org.apache.lucene.util.NumericUtils;
+import org.apache.lucene.util._TestUtil;
 
 import org.junit.Test;
 import org.junit.AfterClass;
@@ -41,7 +42,7 @@ public class TestNumericRangeQuery64 ext
   // shift the starting of the values to the left, to also have negative values:
   private static final long startOffset = - 1L << 31;
   // number of docs to generate for testing
-  private static final int noDocs = 10000;
+  private static final int noDocs = 10000*_TestUtil.getRandomMultiplier();
   
   private static RAMDirectory directory = null;
   private static IndexSearcher searcher = null;
@@ -345,7 +346,7 @@ public class TestNumericRangeQuery64 ext
     final Random rnd=newRandom();
     String field="field"+precisionStep;
     int termCountT=0,termCountC=0;
-    for (int i=0; i<10; i++) {
+    for (int i=0; i<10*_TestUtil.getRandomMultiplier(); i++) {
       long lower=(long)(rnd.nextDouble()*noDocs*distance)+startOffset;
       long upper=(long)(rnd.nextDouble()*noDocs*distance)+startOffset;
       if (lower>upper) {
@@ -428,7 +429,7 @@ public class TestNumericRangeQuery64 ext
     final Random rnd=newRandom();
     String field="ascfield"+precisionStep;
     // 10 random tests
-    for (int i=0; i<10; i++) {
+    for (int i=0; i<10*_TestUtil.getRandomMultiplier(); i++) {
       long lower=(long)(rnd.nextDouble()*noDocs - noDocs/2);
       long upper=(long)(rnd.nextDouble()*noDocs - noDocs/2);
       if (lower>upper) {
@@ -514,7 +515,7 @@ public class TestNumericRangeQuery64 ext
     String field="field"+precisionStep;
     // 10 random tests, the index order is ascending,
     // so using a reverse sort field should retun descending documents
-    for (int i=0; i<10; i++) {
+    for (int i=0; i<10*_TestUtil.getRandomMultiplier(); i++) {
       long lower=(long)(rnd.nextDouble()*noDocs*distance)+startOffset;
       long upper=(long)(rnd.nextDouble()*noDocs*distance)+startOffset;
       if (lower>upper) {

Modified: lucene/dev/trunk/lucene/src/test/org/apache/lucene/search/TestRegexpRandom.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/src/test/org/apache/lucene/search/TestRegexpRandom.java?rev=942676&r1=942675&r2=942676&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/src/test/org/apache/lucene/search/TestRegexpRandom.java (original)
+++ lucene/dev/trunk/lucene/src/test/org/apache/lucene/search/TestRegexpRandom.java Mon May 10 07:47:12 2010
@@ -28,6 +28,7 @@ import org.apache.lucene.index.IndexWrit
 import org.apache.lucene.index.Term;
 import org.apache.lucene.store.RAMDirectory;
 import org.apache.lucene.util.LuceneTestCase;
+import org.apache.lucene.util._TestUtil;
 
 /**
  * Create an index with terms from 0000-9999.
@@ -91,7 +92,7 @@ public class TestRegexpRandom extends Lu
   
   public void testRegexps() throws Exception {
     random = newRandom(System.nanoTime());
-    for (int i = 0; i < 100; i++) {
+    for (int i = 0; i < 100*_TestUtil.getRandomMultiplier(); i++) {
       assertPatternHits("NNNN", 1);
       assertPatternHits(".NNN", 10);
       assertPatternHits("N.NN", 10);
@@ -99,7 +100,7 @@ public class TestRegexpRandom extends Lu
       assertPatternHits("NNN.", 10);
     }
     
-    for (int i = 0; i < 10; i++) {
+    for (int i = 0; i < 10*_TestUtil.getRandomMultiplier(); i++) {
       assertPatternHits(".{1,2}NN", 100);
       assertPatternHits("N.{1,2}N", 100);
       assertPatternHits("NN.{1,2}", 100);

Modified: lucene/dev/trunk/lucene/src/test/org/apache/lucene/search/TestRegexpRandom2.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/src/test/org/apache/lucene/search/TestRegexpRandom2.java?rev=942676&r1=942675&r2=942676&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/src/test/org/apache/lucene/search/TestRegexpRandom2.java (original)
+++ lucene/dev/trunk/lucene/src/test/org/apache/lucene/search/TestRegexpRandom2.java Mon May 10 07:47:12 2010
@@ -57,7 +57,7 @@ public class TestRegexpRandom2 extends L
     Field field = new Field("field", "", Field.Store.NO, Field.Index.ANALYZED);
     doc.add(field);
     
-    for (int i = 0; i < 2000; i++) {
+    for (int i = 0; i < 2000*_TestUtil.getRandomMultiplier(); i++) {
       field.setValue(_TestUtil.randomUnicodeString(random));
       writer.addDocument(doc);
     }
@@ -113,7 +113,7 @@ public class TestRegexpRandom2 extends L
   
   /** test a bunch of random regular expressions */
   public void testRegexps() throws Exception {
-      for (int i = 0; i < 1000; i++)
+      for (int i = 0; i < 1000*_TestUtil.getRandomMultiplier(); i++)
         assertSame(AutomatonTestUtil.randomRegexp(random).toString());
   }
   

Modified: lucene/dev/trunk/lucene/src/test/org/apache/lucene/search/TestScorerPerf.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/src/test/org/apache/lucene/search/TestScorerPerf.java?rev=942676&r1=942675&r2=942676&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/src/test/org/apache/lucene/search/TestScorerPerf.java (original)
+++ lucene/dev/trunk/lucene/src/test/org/apache/lucene/search/TestScorerPerf.java Mon May 10 07:47:12 2010
@@ -2,6 +2,8 @@ package org.apache.lucene.search;
 
 import org.apache.lucene.util.DocIdBitSet;
 import org.apache.lucene.util.LuceneTestCase;
+import org.apache.lucene.util._TestUtil;
+
 import java.util.Random;
 import java.util.BitSet;
 import java.io.IOException;
@@ -317,9 +319,9 @@ public class TestScorerPerf extends Luce
     r = newRandom();
     createDummySearcher();
     validate=true;
-    sets=randBitSets(1000,10);
-    doConjunctions(10000,5);
-    doNestedConjunctions(10000,3,3);
+    sets=randBitSets(1000*_TestUtil.getRandomMultiplier(),10*_TestUtil.getRandomMultiplier());
+    doConjunctions(10000*_TestUtil.getRandomMultiplier(),5*_TestUtil.getRandomMultiplier());
+    doNestedConjunctions(10000*_TestUtil.getRandomMultiplier(),3*_TestUtil.getRandomMultiplier(),3*_TestUtil.getRandomMultiplier());
     s.close();
   }
 

Modified: lucene/dev/trunk/lucene/src/test/org/apache/lucene/search/TestSort.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/src/test/org/apache/lucene/search/TestSort.java?rev=942676&r1=942675&r2=942676&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/src/test/org/apache/lucene/search/TestSort.java (original)
+++ lucene/dev/trunk/lucene/src/test/org/apache/lucene/search/TestSort.java Mon May 10 07:47:12 2010
@@ -45,6 +45,7 @@ import org.apache.lucene.store.LockObtai
 import org.apache.lucene.store.RAMDirectory;
 import org.apache.lucene.util.DocIdBitSet;
 import org.apache.lucene.util.LuceneTestCase;
+import org.apache.lucene.util._TestUtil;
 
 /**
  * Unit tests for sorting code.
@@ -56,7 +57,7 @@ import org.apache.lucene.util.LuceneTest
 
 public class TestSort extends LuceneTestCase implements Serializable {
 
-  private static final int NUM_STRINGS = 6000;
+  private static final int NUM_STRINGS = 6000*_TestUtil.getRandomMultiplier();
   private Searcher full;
   private Searcher searchX;
   private Searcher searchY;

Modified: lucene/dev/trunk/lucene/src/test/org/apache/lucene/search/TestThreadSafe.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/src/test/org/apache/lucene/search/TestThreadSafe.java?rev=942676&r1=942675&r2=942676&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/src/test/org/apache/lucene/search/TestThreadSafe.java (original)
+++ lucene/dev/trunk/lucene/src/test/org/apache/lucene/search/TestThreadSafe.java Mon May 10 07:47:12 2010
@@ -17,6 +17,7 @@ package org.apache.lucene.search;
  */
 
 import org.apache.lucene.util.LuceneTestCase;
+import org.apache.lucene.util._TestUtil;
 import org.apache.lucene.store.Directory;
 import org.apache.lucene.store.RAMDirectory;
 import org.apache.lucene.index.IndexReader;
@@ -147,7 +148,7 @@ public class TestThreadSafe extends Luce
     buildDir(dir1, 15, 5, 2000);
 
     // do many small tests so the thread locals go away inbetween
-    for (int i=0; i<100; i++) {
+    for (int i=0; i<100*_TestUtil.getRandomMultiplier(); i++) {
       ir1 = IndexReader.open(dir1, false);
       doTest(10,100);
     }

Modified: lucene/dev/trunk/lucene/src/test/org/apache/lucene/search/TestWildcardRandom.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/src/test/org/apache/lucene/search/TestWildcardRandom.java?rev=942676&r1=942675&r2=942676&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/src/test/org/apache/lucene/search/TestWildcardRandom.java (original)
+++ lucene/dev/trunk/lucene/src/test/org/apache/lucene/search/TestWildcardRandom.java Mon May 10 07:47:12 2010
@@ -28,6 +28,7 @@ import org.apache.lucene.index.IndexWrit
 import org.apache.lucene.index.Term;
 import org.apache.lucene.store.RAMDirectory;
 import org.apache.lucene.util.LuceneTestCase;
+import org.apache.lucene.util._TestUtil;
 
 /**
  * Create an index with terms from 0000-9999.
@@ -92,7 +93,7 @@ public class TestWildcardRandom extends 
   
   public void testWildcards() throws Exception {
     random = newRandom(System.nanoTime());
-    for (int i = 0; i < 100; i++) {
+    for (int i = 0; i < 100*_TestUtil.getRandomMultiplier(); i++) {
       assertPatternHits("NNNN", 1);
       assertPatternHits("?NNN", 10);
       assertPatternHits("N?NN", 10);
@@ -100,7 +101,7 @@ public class TestWildcardRandom extends 
       assertPatternHits("NNN?", 10);
     }
     
-    for (int i = 0; i < 10; i++) {
+    for (int i = 0; i < 10*_TestUtil.getRandomMultiplier(); i++) {
       assertPatternHits("??NN", 100);
       assertPatternHits("N??N", 100);
       assertPatternHits("NN??", 100);

Modified: lucene/dev/trunk/lucene/src/test/org/apache/lucene/store/TestWindowsMMap.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/src/test/org/apache/lucene/store/TestWindowsMMap.java?rev=942676&r1=942675&r2=942676&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/src/test/org/apache/lucene/store/TestWindowsMMap.java (original)
+++ lucene/dev/trunk/lucene/src/test/org/apache/lucene/store/TestWindowsMMap.java Mon May 10 07:47:12 2010
@@ -22,6 +22,7 @@ import java.util.Random;
 import java.io.File;
 
 import org.apache.lucene.util.LuceneTestCase;
+import org.apache.lucene.util._TestUtil;
 
 import org.apache.lucene.analysis.standard.StandardAnalyzer;
 import org.apache.lucene.document.Document;
@@ -83,7 +84,7 @@ public class TestWindowsMMap extends Luc
     writer.commit();
     IndexSearcher searcher = new IndexSearcher(dir, true);
     
-    for(int dx = 0; dx < 1000; dx ++) {
+    for(int dx = 0; dx < 1000*_TestUtil.getRandomMultiplier(); dx ++) {
       String f = randomField();
       Document doc = new Document();
       doc.add(new Field("data", f, Field.Store.YES, Field.Index.ANALYZED));	

Modified: lucene/dev/trunk/lucene/src/test/org/apache/lucene/util/TestArrayUtil.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/src/test/org/apache/lucene/util/TestArrayUtil.java?rev=942676&r1=942675&r2=942676&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/src/test/org/apache/lucene/util/TestArrayUtil.java (original)
+++ lucene/dev/trunk/lucene/src/test/org/apache/lucene/util/TestArrayUtil.java Mon May 10 07:47:12 2010
@@ -49,7 +49,7 @@ public class TestArrayUtil extends Lucen
 
   public void testInvalidElementSizes() {
     final Random r = newRandom();
-    for(int iter=0;iter<10000;iter++) {
+    for(int iter=0;iter<10000*_TestUtil.getRandomMultiplier();iter++) {
       final int minTargetSize = r.nextInt(Integer.MAX_VALUE);
       final int elemSize = r.nextInt(11);
       final int v = ArrayUtil.oversize(minTargetSize, elemSize);

Modified: lucene/dev/trunk/lucene/src/test/org/apache/lucene/util/TestDoubleBarrelLRUCache.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/src/test/org/apache/lucene/util/TestDoubleBarrelLRUCache.java?rev=942676&r1=942675&r2=942676&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/src/test/org/apache/lucene/util/TestDoubleBarrelLRUCache.java (original)
+++ lucene/dev/trunk/lucene/src/test/org/apache/lucene/util/TestDoubleBarrelLRUCache.java Mon May 10 07:47:12 2010
@@ -17,57 +17,58 @@ package org.apache.lucene.util;
 * limitations under the License.
 */
 
+import org.apache.lucene.util.LuceneTestCase;
 
 public class TestDoubleBarrelLRUCache extends LuceneTestCase {
 
-  private void testCache(DoubleBarrelLRUCache<Integer,Object> cache, int n) throws Exception {
+  private void testCache(DoubleBarrelLRUCache<CloneableInteger,Object> cache, int n) throws Exception {
     Object dummy = new Object();
     
     for (int i = 0; i < n; i++) {
-      cache.put(Integer.valueOf(i), dummy);
+      cache.put(new CloneableInteger(i), dummy);
     }
     
     // access every 2nd item in cache
     for (int i = 0; i < n; i+=2) {
-      assertNotNull(cache.get(Integer.valueOf(i)));
+      assertNotNull(cache.get(new CloneableInteger(i)));
     }
     
     // add n/2 elements to cache, the ones that weren't
     // touched in the previous loop should now be thrown away
     for (int i = n; i < n + (n / 2); i++) {
-      cache.put(Integer.valueOf(i), dummy);
+      cache.put(new CloneableInteger(i), dummy);
     }
     
     // access every 4th item in cache
     for (int i = 0; i < n; i+=4) {
-      assertNotNull(cache.get(Integer.valueOf(i)));
+      assertNotNull(cache.get(new CloneableInteger(i)));
     }
 
     // add 3/4n elements to cache, the ones that weren't
     // touched in the previous loops should now be thrown away
     for (int i = n; i < n + (n * 3 / 4); i++) {
-      cache.put(Integer.valueOf(i), dummy);
+      cache.put(new CloneableInteger(i), dummy);
     }
     
     // access every 4th item in cache
     for (int i = 0; i < n; i+=4) {
-      assertNotNull(cache.get(Integer.valueOf(i)));
+      assertNotNull(cache.get(new CloneableInteger(i)));
     }
   }
     
   public void testLRUCache() throws Exception {
     final int n = 100;
-    testCache(new DoubleBarrelLRUCache<Integer,Object>(n), n);
+    testCache(new DoubleBarrelLRUCache<CloneableInteger,Object>(n), n);
   }
 
   private class CacheThread extends Thread {
-    private final Object[] objs;
-    private final DoubleBarrelLRUCache<Object,Object> c;
+    private final CloneableObject[] objs;
+    private final DoubleBarrelLRUCache<CloneableObject,Object> c;
     private final long endTime;
     volatile boolean failed;
 
-    public CacheThread(DoubleBarrelLRUCache<Object,Object> c,
-                     Object[] objs, long endTime) {
+    public CacheThread(DoubleBarrelLRUCache<CloneableObject,Object> c,
+                       CloneableObject[] objs, long endTime) {
       this.c = c;
       this.objs = objs;
       this.endTime = endTime;
@@ -82,10 +83,10 @@ public class TestDoubleBarrelLRUCache ex
         final int limit = objs.length;
 
         while(true) {
-          final Object obj = objs[(int) ((count/2) % limit)];
+          final CloneableObject obj = objs[(int) ((count/2) % limit)];
           Object v = c.get(obj);
           if (v == null) {
-            c.put(obj, obj);
+            c.put(new CloneableObject(obj), obj);
             miss++;
           } else {
             assert obj == v;
@@ -117,11 +118,11 @@ public class TestDoubleBarrelLRUCache ex
     final int CACHE_SIZE = 512;
     final int OBJ_COUNT = 3*CACHE_SIZE;
 
-    DoubleBarrelLRUCache<Object,Object> c = new DoubleBarrelLRUCache<Object,Object>(1024);
+    DoubleBarrelLRUCache<CloneableObject,Object> c = new DoubleBarrelLRUCache<CloneableObject,Object>(1024);
 
-    Object[] objs = new Object[OBJ_COUNT];
+    CloneableObject[] objs = new CloneableObject[OBJ_COUNT];
     for(int i=0;i<OBJ_COUNT;i++) {
-      objs[i] = new Object();
+      objs[i] = new CloneableObject(new Object());
     }
     
     final CacheThread[] threads = new CacheThread[NUM_THREADS];
@@ -137,4 +138,45 @@ public class TestDoubleBarrelLRUCache ex
     //System.out.println("hits=" + totHit + " misses=" + totMiss);
   }
   
+  private static class CloneableObject extends DoubleBarrelLRUCache.CloneableKey {
+    private Object value;
+
+    public CloneableObject(Object value) {
+      this.value = value;
+    }
+
+    public boolean equals(Object other) {
+      return this.value.equals(((CloneableObject) other).value);
+    }
+
+    public int hashCode() {
+      return value.hashCode();
+    }
+
+    public Object clone() {
+      return new CloneableObject(value);
+    }
+  }
+
+  protected static class CloneableInteger extends DoubleBarrelLRUCache.CloneableKey {
+    private Integer value;
+
+    public CloneableInteger(Integer value) {
+      this.value = value;
+    }
+
+    public boolean equals(Object other) {
+      return this.value.equals(((CloneableInteger) other).value);
+    }
+
+    public int hashCode() {
+      return value.hashCode();
+    }
+
+    public Object clone() {
+      return new CloneableInteger(value);
+    }
+  }
+
+
 }

Modified: lucene/dev/trunk/lucene/src/test/org/apache/lucene/util/TestIndexableBinaryStringTools.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/src/test/org/apache/lucene/util/TestIndexableBinaryStringTools.java?rev=942676&r1=942675&r2=942676&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/src/test/org/apache/lucene/util/TestIndexableBinaryStringTools.java (original)
+++ lucene/dev/trunk/lucene/src/test/org/apache/lucene/util/TestIndexableBinaryStringTools.java Mon May 10 07:47:12 2010
@@ -22,8 +22,8 @@ import java.nio.CharBuffer;
 import java.nio.ByteBuffer;
 
 public class TestIndexableBinaryStringTools extends LuceneTestCase {
-  private static final int NUM_RANDOM_TESTS = 2000;
-  private static final int MAX_RANDOM_BINARY_LENGTH = 300;
+  private static final int NUM_RANDOM_TESTS = 2000*_TestUtil.getRandomMultiplier();
+  private static final int MAX_RANDOM_BINARY_LENGTH = 300*_TestUtil.getRandomMultiplier();
   
   /** @deprecated remove this test for Lucene 4.0 */
   @Deprecated

Modified: lucene/dev/trunk/lucene/src/test/org/apache/lucene/util/TestOpenBitSet.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/src/test/org/apache/lucene/util/TestOpenBitSet.java?rev=942676&r1=942675&r2=942676&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/src/test/org/apache/lucene/util/TestOpenBitSet.java (original)
+++ lucene/dev/trunk/lucene/src/test/org/apache/lucene/util/TestOpenBitSet.java Mon May 10 07:47:12 2010
@@ -175,8 +175,8 @@ public class TestOpenBitSet extends Luce
   // larger testsuite.
   public void testSmall() {
     rand = newRandom();
-    doRandomSets(1200,1000, 1);
-    doRandomSets(1200,1000, 2);
+    doRandomSets(1200*_TestUtil.getRandomMultiplier(),1000*_TestUtil.getRandomMultiplier(), 1);
+    doRandomSets(1200*_TestUtil.getRandomMultiplier(),1000*_TestUtil.getRandomMultiplier(), 2);
   }
 
   public void testBig() {

Modified: lucene/dev/trunk/lucene/src/test/org/apache/lucene/util/TestPriorityQueue.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/src/test/org/apache/lucene/util/TestPriorityQueue.java?rev=942676&r1=942675&r2=942676&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/src/test/org/apache/lucene/util/TestPriorityQueue.java (original)
+++ lucene/dev/trunk/lucene/src/test/org/apache/lucene/util/TestPriorityQueue.java Mon May 10 07:47:12 2010
@@ -37,7 +37,7 @@ public class TestPriorityQueue extends L
     }
 
     public void testPQ() throws Exception {
-        testPQ(10000, newRandom());
+        testPQ(10000*_TestUtil.getRandomMultiplier(), newRandom());
     }
 
     public static void testPQ(int count, Random gen) {

Modified: lucene/dev/trunk/lucene/src/test/org/apache/lucene/util/TestSmallFloat.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/src/test/org/apache/lucene/util/TestSmallFloat.java?rev=942676&r1=942675&r2=942676&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/src/test/org/apache/lucene/util/TestSmallFloat.java (original)
+++ lucene/dev/trunk/lucene/src/test/org/apache/lucene/util/TestSmallFloat.java Mon May 10 07:47:12 2010
@@ -72,7 +72,7 @@ public class TestSmallFloat extends Luce
   public void testFloatToByte() {
     Random rand = newRandom();
     // up iterations for more exhaustive test after changing something
-    for (int i=0; i<100000; i++) {
+    for (int i=0; i<100000*_TestUtil.getRandomMultiplier(); i++) {
       float f = Float.intBitsToFloat(rand.nextInt());
       if (f!=f) continue;    // skip NaN
       byte b1 = orig_floatToByte(f);

Modified: lucene/dev/trunk/lucene/src/test/org/apache/lucene/util/TestStringIntern.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/src/test/org/apache/lucene/util/TestStringIntern.java?rev=942676&r1=942675&r2=942676&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/src/test/org/apache/lucene/util/TestStringIntern.java (original)
+++ lucene/dev/trunk/lucene/src/test/org/apache/lucene/util/TestStringIntern.java Mon May 10 07:47:12 2010
@@ -44,7 +44,7 @@ public class TestStringIntern extends Lu
     // makeStrings(100);  // realistic for perf testing
     int nThreads = 20;
     // final int iter=100000;
-    final int iter=1000000;
+    final int iter=1000000*_TestUtil.getRandomMultiplier();
     
     // try native intern
     // StringHelper.interner = new StringInterner();

Modified: lucene/dev/trunk/lucene/src/test/org/apache/lucene/util/TestUnicodeUtil.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/src/test/org/apache/lucene/util/TestUnicodeUtil.java?rev=942676&r1=942675&r2=942676&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/src/test/org/apache/lucene/util/TestUnicodeUtil.java (original)
+++ lucene/dev/trunk/lucene/src/test/org/apache/lucene/util/TestUnicodeUtil.java Mon May 10 07:47:12 2010
@@ -19,7 +19,6 @@ package org.apache.lucene.util;
 
 import java.util.Random;
 
-
 /*
  * Some of this code came from the excellent Unicode
  * conversion examples from:
@@ -51,6 +50,42 @@ import java.util.Random;
  * remains attached.
  */
 
+/*
+ * Additional code came from the IBM ICU library.
+ *
+ *  http://www.icu-project.org
+ *
+ * Full Copyright for that code follows.
+ */
+
+/*
+ * Copyright (C) 1999-2010, International Business Machines
+ * Corporation and others.  All Rights Reserved.
+ *
+ * Permission is hereby granted, free of charge, to any person obtaining a copy
+ * of this software and associated documentation files (the "Software"), to deal
+ * in the Software without restriction, including without limitation the rights
+ * to use, copy, modify, merge, publish, distribute, and/or sell copies of the
+ * Software, and to permit persons to whom the Software is furnished to do so,
+ * provided that the above copyright notice(s) and this permission notice appear
+ * in all copies of the Software and that both the above copyright notice(s) and
+ * this permission notice appear in supporting documentation.
+ *
+ * THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, EXPRESS OR
+ * IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES OF MERCHANTABILITY,
+ * FITNESS FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT OF THIRD PARTY RIGHTS.
+ * IN NO EVENT SHALL THE COPYRIGHT HOLDER OR HOLDERS INCLUDED IN THIS NOTICE BE
+ * LIABLE FOR ANY CLAIM, OR ANY SPECIAL INDIRECT OR CONSEQUENTIAL DAMAGES, OR
+ * ANY DAMAGES WHATSOEVER RESULTING FROM LOSS OF USE, DATA OR PROFITS, WHETHER
+ * IN AN ACTION OF CONTRACT, NEGLIGENCE OR OTHER TORTIOUS ACTION, ARISING OUT
+ * OF OR IN CONNECTION WITH THE USE OR PERFORMANCE OF THIS SOFTWARE.
+ *
+ * Except as contained in this notice, the name of a copyright holder shall not
+ * be used in advertising or otherwise to promote the sale, use or other
+ * dealings in this Software without prior written authorization of the
+ * copyright holder.
+ */
+
 public class TestUnicodeUtil extends LuceneTestCase {
   public void testNextValidUTF16String() {
     // valid UTF-16
@@ -87,7 +122,7 @@ public class TestUnicodeUtil extends Luc
   public void testCodePointCount() {
     final Random r = newRandom();
     BytesRef utf8 = new BytesRef(20);
-    for(int i=0;i<50000;i++) {
+    for(int i=0;i<50000*_TestUtil.getRandomMultiplier();i++) {
       final String s = _TestUtil.randomUnicodeString(r);
       UnicodeUtil.UTF16toUTF8(s, 0, s.length(), utf8);
       assertEquals(s.codePointCount(0, s.length()),
@@ -100,7 +135,7 @@ public class TestUnicodeUtil extends Luc
     BytesRef utf8 = new BytesRef(20);
     IntsRef utf32 = new IntsRef(20);
     int[] codePoints = new int[20];
-    for(int i=0;i<50000;i++) {
+    for(int i=0;i<50000*_TestUtil.getRandomMultiplier();i++) {
       final String s = _TestUtil.randomUnicodeString(r);
       UnicodeUtil.UTF16toUTF8(s, 0, s.length(), utf8);
       UnicodeUtil.UTF8toUTF32(utf8, utf32);
@@ -126,4 +161,42 @@ public class TestUnicodeUtil extends Luc
       }
     }
   }
+
+  public void testNewString() {
+    final int[] codePoints = {
+        Character.toCodePoint(Character.MIN_HIGH_SURROGATE,
+            Character.MAX_LOW_SURROGATE),
+        Character.toCodePoint(Character.MAX_HIGH_SURROGATE,
+            Character.MIN_LOW_SURROGATE), Character.MAX_HIGH_SURROGATE, 'A',
+        -1,};
+
+    final String cpString = "" + Character.MIN_HIGH_SURROGATE
+        + Character.MAX_LOW_SURROGATE + Character.MAX_HIGH_SURROGATE
+        + Character.MIN_LOW_SURROGATE + Character.MAX_HIGH_SURROGATE + 'A';
+
+    final int[][] tests = { {0, 1, 0, 2}, {0, 2, 0, 4}, {1, 1, 2, 2},
+        {1, 2, 2, 3}, {1, 3, 2, 4}, {2, 2, 4, 2}, {2, 3, 0, -1}, {4, 5, 0, -1},
+        {3, -1, 0, -1}};
+
+    for (int i = 0; i < tests.length; ++i) {
+      int[] t = tests[i];
+      int s = t[0];
+      int c = t[1];
+      int rs = t[2];
+      int rc = t[3];
+
+      Exception e = null;
+      try {
+        String str = UnicodeUtil.newString(codePoints, s, c);
+        assertFalse(rc == -1);
+        assertEquals(cpString.substring(rs, rs + rc), str);
+        continue;
+      } catch (IndexOutOfBoundsException e1) {
+        e = e1;
+      } catch (IllegalArgumentException e2) {
+        e = e2;
+      }
+      assertTrue(rc == -1);
+    }
+  }
 }

Modified: lucene/dev/trunk/lucene/src/test/org/apache/lucene/util/_TestUtil.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/src/test/org/apache/lucene/util/_TestUtil.java?rev=942676&r1=942675&r2=942676&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/src/test/org/apache/lucene/util/_TestUtil.java (original)
+++ lucene/dev/trunk/lucene/src/test/org/apache/lucene/util/_TestUtil.java Mon May 10 07:47:12 2010
@@ -140,4 +140,10 @@ public class _TestUtil {
     }
     return new String(buffer, 0, end);
   }
+
+  /** gets a random multiplier, which you should use when writing
+   *  random tests: multiply it by the number of iterations */
+  public static int getRandomMultiplier() {
+    return Integer.parseInt(System.getProperty("random.multiplier", "1"));
+  }
 }

Modified: lucene/dev/trunk/lucene/src/test/org/apache/lucene/util/automaton/TestBasicOperations.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/src/test/org/apache/lucene/util/automaton/TestBasicOperations.java?rev=942676&r1=942675&r2=942676&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/src/test/org/apache/lucene/util/automaton/TestBasicOperations.java (original)
+++ lucene/dev/trunk/lucene/src/test/org/apache/lucene/util/automaton/TestBasicOperations.java Mon May 10 07:47:12 2010
@@ -18,6 +18,8 @@ package org.apache.lucene.util.automaton
  */
 
 import org.apache.lucene.util.LuceneTestCase;
+import org.apache.lucene.util.UnicodeUtil;
+import org.apache.lucene.util._TestUtil;
 
 import java.util.Random;
 
@@ -85,8 +87,8 @@ public class TestBasicOperations extends
 
   public void testGetRandomAcceptedString() throws Throwable {
     final Random r = newRandom();
-    final int ITER1 = 100;
-    final int ITER2 = 100;
+    final int ITER1 = 100*_TestUtil.getRandomMultiplier();
+    final int ITER2 = 100*_TestUtil.getRandomMultiplier();
     for(int i=0;i<ITER1;i++) {
 
       final RegExp re = AutomatonTestUtil.randomRegexp(r);
@@ -98,7 +100,7 @@ public class TestBasicOperations extends
         int[] acc = null;
         try {
           acc = rx.getRandomAcceptedString(r);
-          final String s = new String(acc, 0, acc.length);
+          final String s = UnicodeUtil.newString(acc, 0, acc.length);
           assertTrue(BasicOperations.run(a, s));
         } catch (Throwable t) {
           System.out.println("regexp: " + re);

Modified: lucene/dev/trunk/lucene/src/test/org/apache/lucene/util/automaton/TestDeterminism.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/src/test/org/apache/lucene/util/automaton/TestDeterminism.java?rev=942676&r1=942675&r2=942676&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/src/test/org/apache/lucene/util/automaton/TestDeterminism.java (original)
+++ lucene/dev/trunk/lucene/src/test/org/apache/lucene/util/automaton/TestDeterminism.java Mon May 10 07:47:12 2010
@@ -20,6 +20,7 @@ package org.apache.lucene.util.automaton
 import java.util.Random;
 
 import org.apache.lucene.util.LuceneTestCase;
+import org.apache.lucene.util._TestUtil;
 
 /**
  * Not thorough, but tries to test determinism correctness
@@ -36,7 +37,7 @@ public class TestDeterminism extends Luc
   
   /** test a bunch of random regular expressions */
   public void testRegexps() throws Exception {
-      for (int i = 0; i < 500; i++)
+      for (int i = 0; i < 500*_TestUtil.getRandomMultiplier(); i++)
         assertAutomaton(AutomatonTestUtil.randomRegexp(random).toAutomaton());
   }
   

Modified: lucene/dev/trunk/lucene/src/test/org/apache/lucene/util/automaton/TestDeterminizeLexicon.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/src/test/org/apache/lucene/util/automaton/TestDeterminizeLexicon.java?rev=942676&r1=942675&r2=942676&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/src/test/org/apache/lucene/util/automaton/TestDeterminizeLexicon.java (original)
+++ lucene/dev/trunk/lucene/src/test/org/apache/lucene/util/automaton/TestDeterminizeLexicon.java Mon May 10 07:47:12 2010
@@ -35,19 +35,16 @@ public class TestDeterminizeLexicon exte
   private List<String> terms = new ArrayList<String>();
   private Random random;
   
-  @Override
-  protected void setUp() throws Exception {
-    super.setUp();
-    random = newRandom();
-    for (int i = 0; i < 5000; i++) {
-      String randomString = _TestUtil.randomUnicodeString(random);
-      terms.add(randomString);
-      automata.add(BasicAutomata.makeString(randomString));
-    }
-  }
-
   public void testLexicon() {
-    for (int i = 0; i < 3; i++) {
+    random = newRandom();
+    for (int i = 0; i < 3*_TestUtil.getRandomMultiplier(); i++) {
+      automata.clear();
+      terms.clear();
+      for (int j = 0; j < 5000; j++) {
+        String randomString = _TestUtil.randomUnicodeString(random);
+        terms.add(randomString);
+        automata.add(BasicAutomata.makeString(randomString));
+      }
       assertLexicon();
     }
   }

Modified: lucene/dev/trunk/lucene/src/test/org/apache/lucene/util/automaton/TestUTF32ToUTF8.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/src/test/org/apache/lucene/util/automaton/TestUTF32ToUTF8.java?rev=942676&r1=942675&r2=942676&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/src/test/org/apache/lucene/util/automaton/TestUTF32ToUTF8.java (original)
+++ lucene/dev/trunk/lucene/src/test/org/apache/lucene/util/automaton/TestUTF32ToUTF8.java Mon May 10 07:47:12 2010
@@ -99,8 +99,8 @@ public class TestUTF32ToUTF8 extends Luc
 
   public void testRandomRanges() throws Exception {
     final Random r = random;
-    int ITERS = 10;
-    int ITERS_PER_DFA = 100;
+    int ITERS = 10*_TestUtil.getRandomMultiplier();
+    int ITERS_PER_DFA = 100*_TestUtil.getRandomMultiplier();
     for(int iter=0;iter<ITERS;iter++) {
       int x1 = getCodeStart(r);
       int x2 = getCodeStart(r);
@@ -166,16 +166,16 @@ public class TestUTF32ToUTF8 extends Luc
   }
   
   public void testRandomRegexes() throws Exception {
-    for (int i = 0; i < 250; i++)
+    for (int i = 0; i < 250*_TestUtil.getRandomMultiplier(); i++)
       assertAutomaton(AutomatonTestUtil.randomRegexp(random).toAutomaton());
   }
   
-  private void assertAutomaton(Automaton automaton) {
+  private void assertAutomaton(Automaton automaton) throws Exception {
     CharacterRunAutomaton cra = new CharacterRunAutomaton(automaton);
     ByteRunAutomaton bra = new ByteRunAutomaton(automaton);
     final BasicOperations.RandomAcceptedStrings ras = new BasicOperations.RandomAcceptedStrings(automaton);
     
-    for (int i = 0; i < 1000; i++) {
+    for (int i = 0; i < 1000*_TestUtil.getRandomMultiplier(); i++) {
       final String string;
       if (random.nextBoolean()) {
         // likely not accepted
@@ -183,7 +183,15 @@ public class TestUTF32ToUTF8 extends Luc
       } else {
         // will be accepted
         int[] codepoints = ras.getRandomAcceptedString(random);
-        string = new String(codepoints, 0, codepoints.length);
+        try {
+          string = UnicodeUtil.newString(codepoints, 0, codepoints.length);
+        } catch (Exception e) {
+          System.out.println(codepoints.length + " codepoints:");
+          for(int j=0;j<codepoints.length;j++) {
+            System.out.println("  " + Integer.toHexString(codepoints[j]));
+          }
+          throw e;
+        }
       }
       BytesRef bytesRef = new BytesRef(string);
       assertEquals(cra.run(string), bra.run(bytesRef.bytes, 0, bytesRef.length));

Modified: lucene/dev/trunk/lucene/src/test/org/apache/lucene/util/packed/TestPackedInts.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/src/test/org/apache/lucene/util/packed/TestPackedInts.java?rev=942676&r1=942675&r2=942676&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/src/test/org/apache/lucene/util/packed/TestPackedInts.java (original)
+++ lucene/dev/trunk/lucene/src/test/org/apache/lucene/util/packed/TestPackedInts.java Mon May 10 07:47:12 2010
@@ -19,6 +19,7 @@ package org.apache.lucene.util.packed;
 
 import org.apache.lucene.store.*;
 import org.apache.lucene.util.LuceneTestCase;
+import org.apache.lucene.util._TestUtil;
 
 import java.util.ArrayList;
 import java.util.List;
@@ -51,7 +52,7 @@ public class TestPackedInts extends Luce
 
   public void testPackedInts() throws IOException {
     rnd = newRandom();
-    for(int iter=0;iter<5;iter++) {
+    for(int iter=0;iter<5*_TestUtil.getRandomMultiplier();iter++) {
       long ceil = 2;
       for(int nbits=1;nbits<63;nbits++) {
         final int valueCount = 100+rnd.nextInt(500);