You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@lucene.apache.org by rm...@apache.org on 2013/10/21 20:58:44 UTC

svn commit: r1534320 [17/39] - in /lucene/dev/branches/lucene4956: ./ dev-tools/ dev-tools/idea/.idea/ dev-tools/idea/lucene/expressions/ dev-tools/idea/solr/contrib/velocity/ dev-tools/maven/ dev-tools/maven/lucene/ dev-tools/maven/lucene/expressions/...

Modified: lucene/dev/branches/lucene4956/lucene/core/src/java/org/apache/lucene/util/packed/EliasFanoEncoder.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene4956/lucene/core/src/java/org/apache/lucene/util/packed/EliasFanoEncoder.java?rev=1534320&r1=1534319&r2=1534320&view=diff
==============================================================================
--- lucene/dev/branches/lucene4956/lucene/core/src/java/org/apache/lucene/util/packed/EliasFanoEncoder.java (original)
+++ lucene/dev/branches/lucene4956/lucene/core/src/java/org/apache/lucene/util/packed/EliasFanoEncoder.java Mon Oct 21 18:58:24 2013
@@ -19,6 +19,7 @@ package org.apache.lucene.util.packed;
 
 import java.util.Arrays;
 
+import org.apache.lucene.util.ToStringUtils;
 import org.apache.lucene.util.FixedBitSet; // for javadocs
 
 
@@ -64,10 +65,12 @@ import org.apache.lucene.util.FixedBitSe
  * In this implementation the values in the sequence can be given as <code>long</code>,
  * <code>numValues = 0</code> and <code>upperBound = 0</code> are allowed,
  * and each of the upper and lower bit arrays should fit in a <code>long[]</code>.
+ * <br>
+ * An index of positions of zero's in the upper bits is also built.
  * <p>
  * This implementation is based on this article:
  * <br>
- * Sebastiano Vigna, "Quasi Succinct Indices", June 19, 2012, sections 3 and 4.
+ * Sebastiano Vigna, "Quasi Succinct Indices", June 19, 2012, sections 3, 4 and 9.
  * Retrieved from http://arxiv.org/pdf/1206.4300 .
  *
  * <p>The articles originally describing the Elias-Fano representation are:
@@ -91,6 +94,19 @@ public class EliasFanoEncoder {
   long numEncoded = 0L;
   long lastEncoded = 0L;
 
+  /** The default index interval for zero upper bits. */
+  public static final long DEFAULT_INDEX_INTERVAL = 256;
+  final long numIndexEntries;
+  final long indexInterval;
+  final int nIndexEntryBits;
+  /** upperZeroBitPositionIndex[i] (filled using packValue) will contain the bit position
+   *  just after the zero bit ((i+1) * indexInterval) in the upper bits.
+   */
+  final long[] upperZeroBitPositionIndex;
+  long currentEntryIndex; // also indicates how many entries in the index are valid.
+
+
+
   /**
    * Construct an Elias-Fano encoder.
    * After construction, call {@link #encodeNext} <code>numValues</code> times to encode
@@ -101,6 +117,10 @@ public class EliasFanoEncoder {
    *                or is the first higher than the actual maximum.
    *                <br>When <code>numValues >= (upperBound/3)</code>
    *                a {@link FixedBitSet} will take less space.
+   * @param indexInterval The number of high zero bits for which a single index entry is built.
+   *                The index will have at most <code>2 * numValues / indexInterval</code> entries
+   *                and each index entry will use at most <code>ceil(log2(3 * numValues))</code> bits,
+   *                see {@link EliasFanoEncoder}.
    * @throws IllegalArgumentException when:
    *         <ul>
    *         <li><code>numValues</code> is negative, or
@@ -108,10 +128,13 @@ public class EliasFanoEncoder {
    *         <li>the low bits do not fit in a <code>long[]</code>:
    *             <code>(L * numValues / 64) > Integer.MAX_VALUE</code>, or
    *         <li>the high bits do not fit in a <code>long[]</code>:
-   *             <code>(2 * numValues / 64) > Integer.MAX_VALUE</code>.
+   *             <code>(2 * numValues / 64) > Integer.MAX_VALUE</code>, or
+   *         <li><code>indexInterval < 2</code>,
+   *         <li>the index bits do not fit in a <code>long[]</code>:
+   *             <code>(numValues / indexInterval * ceil(2log(3 * numValues)) / 64) > Integer.MAX_VALUE</code>.
    *         </ul>
    */
-  public EliasFanoEncoder(long numValues, long upperBound) {
+  public EliasFanoEncoder(long numValues, long upperBound, long indexInterval) {
     if (numValues < 0L) {
       throw new IllegalArgumentException("numValues should not be negative: " + numValues);
     }
@@ -145,18 +168,42 @@ public class EliasFanoEncoder {
       throw new IllegalArgumentException("numLongsForHighBits too large to index a long array: " + numLongsForHighBits);
     }
     this.upperLongs = new long[(int) numLongsForHighBits];
+    if (indexInterval < 2) {
+      throw new IllegalArgumentException("indexInterval should at least 2: " + indexInterval);
+    }
+    // For the index:
+    long maxHighValue = upperBound >>> this.numLowBits;
+    long nIndexEntries = maxHighValue / indexInterval; // no zero value index entry
+    this.numIndexEntries = (nIndexEntries >= 0) ? nIndexEntries : 0;
+    long maxIndexEntry = maxHighValue + numValues - 1; // clear upper bits, set upper bits, start at zero
+    this.nIndexEntryBits = (maxIndexEntry <= 0) ? 0
+                          : (64 - Long.numberOfLeadingZeros(maxIndexEntry - 1));
+    long numLongsForIndexBits = numLongsForBits(numIndexEntries * nIndexEntryBits);
+    if (numLongsForIndexBits > Integer.MAX_VALUE) {
+      throw new IllegalArgumentException("numLongsForIndexBits too large to index a long array: " + numLongsForIndexBits);
+    }
+    this.upperZeroBitPositionIndex = new long[(int) numLongsForIndexBits];
+    this.currentEntryIndex = 0;
+    this.indexInterval = indexInterval;
   }
 
-  private static long numLongsForBits(long numBits) {
+  /**
+  * Construct an Elias-Fano encoder using {@link #DEFAULT_INDEX_INTERVAL}.
+  */
+  public EliasFanoEncoder(long numValues, long upperBound) {
+    this(numValues, upperBound, DEFAULT_INDEX_INTERVAL);
+  }
+
+  private static long numLongsForBits(long numBits) { // Note: int version in FixedBitSet.bits2words()
     assert numBits >= 0 : numBits;
     return (numBits + (Long.SIZE-1)) >>> LOG2_LONG_SIZE;
   }
 
   /** Call at most <code>numValues</code> times to encode a non decreasing sequence of non negative numbers.
    * @param x The next number to be encoded.
+   * @throws IllegalStateException when called more than <code>numValues</code> times.
    * @throws IllegalArgumentException when:
    *         <ul>
-   *         <li>called more than <code>numValues</code> times, or
    *         <li><code>x</code> is smaller than an earlier encoded value, or
    *         <li><code>x</code> is larger than <code>upperBound</code>.
    *         </ul>
@@ -171,10 +218,19 @@ public class EliasFanoEncoder {
     if (x > upperBound) {
       throw new IllegalArgumentException(x + " larger than upperBound " + upperBound);
     }
-    encodeUpperBits(x >>> numLowBits);
+    long highValue = x >>> numLowBits;
+    encodeUpperBits(highValue);
     encodeLowerBits(x & lowerBitsMask);
-    numEncoded++;
     lastEncoded = x;
+    // Add index entries:
+    long indexValue = (currentEntryIndex + 1) * indexInterval;
+    while (indexValue <= highValue) { 
+      long afterZeroBitPosition = indexValue + numEncoded;
+      packValue(afterZeroBitPosition, upperZeroBitPositionIndex, nIndexEntryBits, currentEntryIndex);
+      currentEntryIndex += 1;
+      indexValue += indexInterval;
+    }
+    numEncoded++;
   }
 
   private void encodeUpperBits(long highValue) {
@@ -198,25 +254,28 @@ public class EliasFanoEncoder {
     }
   }
 
-  /** Provide an indication that is better to use an {@link EliasFanoEncoder} than a {@link FixedBitSet}
+  /** Provide an indication that it is better to use an {@link EliasFanoEncoder} than a {@link FixedBitSet}
    *  to encode document identifiers.
    *  This indication is not precise and may change in the future.
    *  <br>An EliasFanoEncoder is favoured when the size of the encoding by the EliasFanoEncoder
-   *  is at most 5/6 of the size of the FixedBitSet.
-   *  <br>This condition is the same as comparing estimates of the number of bits accessed by a pair of FixedBitSets and
+   *  (including some space for its index) is at most about 5/6 of the size of the FixedBitSet,
+   *  this is the same as comparing estimates of the number of bits accessed by a pair of FixedBitSets and
    *  by a pair of non indexed EliasFanoDocIdSets when determining the intersections of the pairs.
+   *  <br>A bit set is preferred when <code>upperbound <= 256</code>.
+   *  <br>It is assumed that {@link #DEFAULT_INDEX_INTERVAL} is used.
    *  @param numValues The number of document identifiers that is to be encoded. Should be non negative.
-   *  @param upperBound The maximum possible value for a document identifier. Should be at least numValues.
+   *  @param upperBound The maximum possible value for a document identifier. Should be at least <code>numValues</code>.
    */
   public static boolean sufficientlySmallerThanBitSet(long numValues, long upperBound) {
     /* When (upperBound / 6) == numValues,
      * the number of bits per entry for the EliasFanoEncoder is 2 + ceil(2log(upperBound/numValues)) == 5.
-     */
-    /* For intersecting two bit sets upperBound bits are accessed, roughly half of one, half of the other.
+     *
+     * For intersecting two bit sets upperBound bits are accessed, roughly half of one, half of the other.
      * For intersecting two EliasFano sequences without index on the upper bits,
      * all (2 * 3 * numValues) upper bits are accessed.
      */
-    return (upperBound / 6) > numValues;
+    return (upperBound > (4 * Long.SIZE)) // prefer a bit set when it takes no more than 4 longs.
+            && (upperBound / 7) > numValues; // 6 + 1 to allow some room for the index.
   }
 
   /**
@@ -237,6 +296,11 @@ public class EliasFanoEncoder {
   public long[] getUpperBits() {
     return upperLongs;
   }
+  
+  /** Expert. The index bits. */
+  public long[] getIndexBits() {
+    return upperZeroBitPositionIndex;
+  }
 
   @Override
   public String toString() {
@@ -248,11 +312,16 @@ public class EliasFanoEncoder {
     s.append(" numLowBits " + numLowBits);
     s.append("\nupperLongs[" + upperLongs.length + "]");
     for (int i = 0; i < upperLongs.length; i++) {
-      s.append(" " + longHex(upperLongs[i]));
+      s.append(" " + ToStringUtils.longHex(upperLongs[i]));
     }
     s.append("\nlowerLongs[" + lowerLongs.length + "]");
     for (int i = 0; i < lowerLongs.length; i++) {
-      s.append(" " + longHex(lowerLongs[i]));
+      s.append(" " + ToStringUtils.longHex(lowerLongs[i]));
+    }
+    s.append("\nindexInterval: " + indexInterval + ", nIndexEntryBits: " + nIndexEntryBits);
+    s.append("\nupperZeroBitPositionIndex[" + upperZeroBitPositionIndex.length + "]");
+    for (int i = 0; i < upperZeroBitPositionIndex.length; i++) { 
+      s.append(" " + ToStringUtils.longHex(upperZeroBitPositionIndex[i]));
     }
     return s.toString();
   }
@@ -267,29 +336,19 @@ public class EliasFanoEncoder {
     return (this.numValues == oefs.numValues)
         && (this.numEncoded == oefs.numEncoded)
         && (this.numLowBits == oefs.numLowBits)
+        && (this.numIndexEntries == oefs.numIndexEntries)
+        && (this.indexInterval == oefs.indexInterval) // no need to check index content
         && Arrays.equals(this.upperLongs, oefs.upperLongs)
         && Arrays.equals(this.lowerLongs, oefs.lowerLongs);
   }
 
   @Override
   public int hashCode() {
-    int h = ((int) (numValues + numEncoded))
-        ^ numLowBits
-        ^ Arrays.hashCode(upperLongs)
-        ^ Arrays.hashCode(lowerLongs);
+    int h = ((int) (31*(numValues + 7*(numEncoded + 5*(numLowBits + 3*(numIndexEntries + 11*indexInterval))))))
+            ^ Arrays.hashCode(upperLongs)
+            ^ Arrays.hashCode(lowerLongs);
     return h;
   }
 
-  public static String longHex(long x) {
-    String hx = Long.toHexString(x);
-    StringBuilder sb = new StringBuilder("0x");
-    int l = 16 - hx.length();
-    while (l > 0) {
-      sb.append('0');
-      l--;
-    }
-    sb.append(hx);
-    return sb.toString();
-  }
 }
 

Modified: lucene/dev/branches/lucene4956/lucene/core/src/java/org/apache/lucene/util/packed/MonotonicBlockPackedReader.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene4956/lucene/core/src/java/org/apache/lucene/util/packed/MonotonicBlockPackedReader.java?rev=1534320&r1=1534319&r2=1534320&view=diff
==============================================================================
--- lucene/dev/branches/lucene4956/lucene/core/src/java/org/apache/lucene/util/packed/MonotonicBlockPackedReader.java (original)
+++ lucene/dev/branches/lucene4956/lucene/core/src/java/org/apache/lucene/util/packed/MonotonicBlockPackedReader.java Mon Oct 21 18:58:24 2013
@@ -27,6 +27,7 @@ import java.io.IOException;
 
 import org.apache.lucene.store.DataInput;
 import org.apache.lucene.store.IndexInput;
+import org.apache.lucene.util.RamUsageEstimator;
 
 /**
  * Provides random access to a stream written with
@@ -88,5 +89,16 @@ public final class MonotonicBlockPackedR
   public long size() {
     return valueCount;
   }
+  
+  /** Returns the approximate RAM bytes used */
+  public long ramBytesUsed() {
+    long sizeInBytes = 0;
+    sizeInBytes += RamUsageEstimator.sizeOf(minValues);
+    sizeInBytes += RamUsageEstimator.sizeOf(averages);
+    for(PackedInts.Reader reader: subReaders) {
+      sizeInBytes += reader.ramBytesUsed();
+    }
+    return sizeInBytes;
+  }
 
 }

Modified: lucene/dev/branches/lucene4956/lucene/core/src/resources/META-INF/services/org.apache.lucene.codecs.Codec
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene4956/lucene/core/src/resources/META-INF/services/org.apache.lucene.codecs.Codec?rev=1534320&r1=1534319&r2=1534320&view=diff
==============================================================================
--- lucene/dev/branches/lucene4956/lucene/core/src/resources/META-INF/services/org.apache.lucene.codecs.Codec (original)
+++ lucene/dev/branches/lucene4956/lucene/core/src/resources/META-INF/services/org.apache.lucene.codecs.Codec Mon Oct 21 18:58:24 2013
@@ -16,3 +16,5 @@
 org.apache.lucene.codecs.lucene40.Lucene40Codec
 org.apache.lucene.codecs.lucene41.Lucene41Codec
 org.apache.lucene.codecs.lucene42.Lucene42Codec
+org.apache.lucene.codecs.lucene45.Lucene45Codec
+org.apache.lucene.codecs.lucene46.Lucene46Codec
\ No newline at end of file

Modified: lucene/dev/branches/lucene4956/lucene/core/src/resources/META-INF/services/org.apache.lucene.codecs.DocValuesFormat
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene4956/lucene/core/src/resources/META-INF/services/org.apache.lucene.codecs.DocValuesFormat?rev=1534320&r1=1534319&r2=1534320&view=diff
==============================================================================
--- lucene/dev/branches/lucene4956/lucene/core/src/resources/META-INF/services/org.apache.lucene.codecs.DocValuesFormat (original)
+++ lucene/dev/branches/lucene4956/lucene/core/src/resources/META-INF/services/org.apache.lucene.codecs.DocValuesFormat Mon Oct 21 18:58:24 2013
@@ -14,3 +14,4 @@
 #  limitations under the License.
 
 org.apache.lucene.codecs.lucene42.Lucene42DocValuesFormat
+org.apache.lucene.codecs.lucene45.Lucene45DocValuesFormat
\ No newline at end of file

Modified: lucene/dev/branches/lucene4956/lucene/core/src/test/org/apache/lucene/TestExternalCodecs.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene4956/lucene/core/src/test/org/apache/lucene/TestExternalCodecs.java?rev=1534320&r1=1534319&r2=1534320&view=diff
==============================================================================
--- lucene/dev/branches/lucene4956/lucene/core/src/test/org/apache/lucene/TestExternalCodecs.java (original)
+++ lucene/dev/branches/lucene4956/lucene/core/src/test/org/apache/lucene/TestExternalCodecs.java Mon Oct 21 18:58:24 2013
@@ -17,21 +17,27 @@ package org.apache.lucene;
  * limitations under the License.
  */
 
-import org.apache.lucene.analysis.*;
-import org.apache.lucene.codecs.*;
-import org.apache.lucene.codecs.lucene42.Lucene42Codec;
-import org.apache.lucene.document.*;
-import org.apache.lucene.index.*;
-import org.apache.lucene.search.*;
-import org.apache.lucene.store.*;
-import org.apache.lucene.util.*;
+import org.apache.lucene.analysis.MockAnalyzer;
+import org.apache.lucene.codecs.PostingsFormat;
+import org.apache.lucene.codecs.lucene46.Lucene46Codec;
+import org.apache.lucene.document.Document;
+import org.apache.lucene.document.Field;
+import org.apache.lucene.index.DirectoryReader;
+import org.apache.lucene.index.IndexReader;
+import org.apache.lucene.index.IndexWriter;
+import org.apache.lucene.index.Term;
+import org.apache.lucene.search.IndexSearcher;
+import org.apache.lucene.search.TermQuery;
+import org.apache.lucene.store.BaseDirectoryWrapper;
+import org.apache.lucene.util.LuceneTestCase;
+
 
 /* Intentionally outside of oal.index to verify fully
    external codecs work fine */
 
 public class TestExternalCodecs extends LuceneTestCase {
 
-  private static final class CustomPerFieldCodec extends Lucene42Codec {
+  private static final class CustomPerFieldCodec extends Lucene46Codec {
     
     private final PostingsFormat ramFormat = PostingsFormat.forName("RAMOnly");
     private final PostingsFormat defaultFormat = PostingsFormat.forName("Lucene41");

Modified: lucene/dev/branches/lucene4956/lucene/core/src/test/org/apache/lucene/analysis/TestGraphTokenizers.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene4956/lucene/core/src/test/org/apache/lucene/analysis/TestGraphTokenizers.java?rev=1534320&r1=1534319&r2=1534320&view=diff
==============================================================================
--- lucene/dev/branches/lucene4956/lucene/core/src/test/org/apache/lucene/analysis/TestGraphTokenizers.java (original)
+++ lucene/dev/branches/lucene4956/lucene/core/src/test/org/apache/lucene/analysis/TestGraphTokenizers.java Mon Oct 21 18:58:24 2013
@@ -68,7 +68,8 @@ public class TestGraphTokenizers extends
     }
 
     @Override
-    public void reset() {
+    public void reset() throws IOException {
+      super.reset();
       tokens = null;
       upto = 0;
     }

Modified: lucene/dev/branches/lucene4956/lucene/core/src/test/org/apache/lucene/analysis/TestLookaheadTokenFilter.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene4956/lucene/core/src/test/org/apache/lucene/analysis/TestLookaheadTokenFilter.java?rev=1534320&r1=1534319&r2=1534320&view=diff
==============================================================================
--- lucene/dev/branches/lucene4956/lucene/core/src/test/org/apache/lucene/analysis/TestLookaheadTokenFilter.java (original)
+++ lucene/dev/branches/lucene4956/lucene/core/src/test/org/apache/lucene/analysis/TestLookaheadTokenFilter.java Mon Oct 21 18:58:24 2013
@@ -21,8 +21,6 @@ import java.io.IOException;
 import java.io.Reader;
 import java.util.Random;
 
-import org.apache.lucene.analysis.tokenattributes.CharTermAttribute;
-
 public class TestLookaheadTokenFilter extends BaseTokenStreamTestCase {
 
   public void testRandomStrings() throws Exception {
@@ -65,4 +63,35 @@ public class TestLookaheadTokenFilter ex
       };
     checkRandomData(random(), a, 200*RANDOM_MULTIPLIER, 8192);
   }
+
+  public void testMissedFirstToken() throws Exception {
+    Analyzer analyzer = new Analyzer() {
+      @Override
+      protected TokenStreamComponents createComponents(String fieldName,
+                                                       Reader reader) {
+        Tokenizer source = new MockTokenizer(reader, MockTokenizer.WHITESPACE, false);
+        TrivialLookaheadFilter filter = new TrivialLookaheadFilter(source);
+        return new TokenStreamComponents(source, filter);
+     }
+    };
+
+    assertAnalyzesTo(analyzer,
+        "Only he who is running knows .",
+        new String[]{
+            "Only",
+            "Only-huh?",
+            "he",
+            "he-huh?",
+            "who",
+            "who-huh?",
+            "is",
+            "is-huh?",
+            "running",
+            "running-huh?",
+            "knows",
+            "knows-huh?",
+            ".",
+            ".-huh?"
+        });
+  }
 }

Modified: lucene/dev/branches/lucene4956/lucene/core/src/test/org/apache/lucene/analysis/TestMockAnalyzer.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene4956/lucene/core/src/test/org/apache/lucene/analysis/TestMockAnalyzer.java?rev=1534320&r1=1534319&r2=1534320&view=diff
==============================================================================
--- lucene/dev/branches/lucene4956/lucene/core/src/test/org/apache/lucene/analysis/TestMockAnalyzer.java (original)
+++ lucene/dev/branches/lucene4956/lucene/core/src/test/org/apache/lucene/analysis/TestMockAnalyzer.java Mon Oct 21 18:58:24 2013
@@ -7,6 +7,7 @@ import java.util.Random;
 
 import org.apache.lucene.util._TestUtil;
 import org.apache.lucene.util.automaton.Automaton;
+import org.apache.lucene.util.automaton.AutomatonTestUtil;
 import org.apache.lucene.util.automaton.BasicAutomata;
 import org.apache.lucene.util.automaton.BasicOperations;
 import org.apache.lucene.util.automaton.CharacterRunAutomaton;
@@ -36,9 +37,9 @@ public class TestMockAnalyzer extends Ba
     Analyzer a = new MockAnalyzer(random());
     assertAnalyzesTo(a, "A bc defg hiJklmn opqrstuv wxy z ",
         new String[] { "a", "bc", "defg", "hijklmn", "opqrstuv", "wxy", "z" });
-    assertAnalyzesToReuse(a, "aba cadaba shazam",
+    assertAnalyzesTo(a, "aba cadaba shazam",
         new String[] { "aba", "cadaba", "shazam" });
-    assertAnalyzesToReuse(a, "break on whitespace",
+    assertAnalyzesTo(a, "break on whitespace",
         new String[] { "break", "on", "whitespace" });
   }
   
@@ -47,9 +48,9 @@ public class TestMockAnalyzer extends Ba
     Analyzer a = new MockAnalyzer(random(), MockTokenizer.SIMPLE, true);
     assertAnalyzesTo(a, "a-bc123 defg+hijklmn567opqrstuv78wxy_z ",
         new String[] { "a", "bc", "defg", "hijklmn", "opqrstuv", "wxy", "z" });
-    assertAnalyzesToReuse(a, "aba4cadaba-Shazam",
+    assertAnalyzesTo(a, "aba4cadaba-Shazam",
         new String[] { "aba", "cadaba", "shazam" });
-    assertAnalyzesToReuse(a, "break+on/Letters",
+    assertAnalyzesTo(a, "break+on/Letters",
         new String[] { "break", "on", "letters" });
   }
   
@@ -58,10 +59,87 @@ public class TestMockAnalyzer extends Ba
     Analyzer a = new MockAnalyzer(random(), MockTokenizer.KEYWORD, false);
     assertAnalyzesTo(a, "a-bc123 defg+hijklmn567opqrstuv78wxy_z ",
         new String[] { "a-bc123 defg+hijklmn567opqrstuv78wxy_z " });
-    assertAnalyzesToReuse(a, "aba4cadaba-Shazam",
+    assertAnalyzesTo(a, "aba4cadaba-Shazam",
         new String[] { "aba4cadaba-Shazam" });
-    assertAnalyzesToReuse(a, "break+on/Nothing",
+    assertAnalyzesTo(a, "break+on/Nothing",
         new String[] { "break+on/Nothing" });
+    // currently though emits no tokens for empty string: maybe we can do it,
+    // but we don't want to emit tokens infinitely...
+    assertAnalyzesTo(a, "", new String[0]);
+  }
+  
+  // Test some regular expressions as tokenization patterns
+  /** Test a configuration where each character is a term */
+  public void testSingleChar() throws Exception {
+    CharacterRunAutomaton single =
+        new CharacterRunAutomaton(new RegExp(".").toAutomaton());
+    Analyzer a = new MockAnalyzer(random(), single, false);
+    assertAnalyzesTo(a, "foobar",
+        new String[] { "f", "o", "o", "b", "a", "r" },
+        new int[] { 0, 1, 2, 3, 4, 5 },
+        new int[] { 1, 2, 3, 4, 5, 6 }
+    );
+    checkRandomData(random(), a, 100);
+  }
+  
+  /** Test a configuration where two characters makes a term */
+  public void testTwoChars() throws Exception {
+    CharacterRunAutomaton single =
+        new CharacterRunAutomaton(new RegExp("..").toAutomaton());
+    Analyzer a = new MockAnalyzer(random(), single, false);
+    assertAnalyzesTo(a, "foobar",
+        new String[] { "fo", "ob", "ar"},
+        new int[] { 0, 2, 4 },
+        new int[] { 2, 4, 6 }
+    );
+    // make sure when last term is a "partial" match that end() is correct
+    assertTokenStreamContents(a.tokenStream("bogus", "fooba"),
+        new String[] { "fo", "ob" },
+        new int[] { 0, 2 },
+        new int[] { 2, 4 },
+        new int[] { 1, 1 },
+        new Integer(5)
+    );
+    checkRandomData(random(), a, 100);
+  }
+  
+  /** Test a configuration where three characters makes a term */
+  public void testThreeChars() throws Exception {
+    CharacterRunAutomaton single =
+        new CharacterRunAutomaton(new RegExp("...").toAutomaton());
+    Analyzer a = new MockAnalyzer(random(), single, false);
+    assertAnalyzesTo(a, "foobar",
+        new String[] { "foo", "bar"},
+        new int[] { 0, 3 },
+        new int[] { 3, 6 }
+    );
+    // make sure when last term is a "partial" match that end() is correct
+    assertTokenStreamContents(a.tokenStream("bogus", "fooba"),
+        new String[] { "foo" },
+        new int[] { 0 },
+        new int[] { 3 },
+        new int[] { 1 },
+        new Integer(5)
+    );
+    checkRandomData(random(), a, 100);
+  }
+  
+  /** Test a configuration where word starts with one uppercase */
+  public void testUppercase() throws Exception {
+    CharacterRunAutomaton single =
+        new CharacterRunAutomaton(new RegExp("[A-Z][a-z]*").toAutomaton());
+    Analyzer a = new MockAnalyzer(random(), single, false);
+    assertAnalyzesTo(a, "FooBarBAZ",
+        new String[] { "Foo", "Bar", "B", "A", "Z"},
+        new int[] { 0, 3, 6, 7, 8 },
+        new int[] { 3, 6, 7, 8, 9 }
+    );
+    assertAnalyzesTo(a, "aFooBar",
+        new String[] { "Foo", "Bar" },
+        new int[] { 1, 4 },
+        new int[] { 4, 7 }
+    );
+    checkRandomData(random(), a, 100);
   }
   
   /** Test a configuration that behaves a lot like StopAnalyzer */
@@ -94,19 +172,42 @@ public class TestMockAnalyzer extends Ba
         new int[] { 1, 2 });
   }
   
+  /** Test MockTokenizer encountering a too long token */
+  public void testTooLongToken() throws Exception {
+    Analyzer whitespace = new Analyzer() {
+      @Override
+      protected TokenStreamComponents createComponents(String fieldName, Reader reader) {
+        Tokenizer t = new MockTokenizer(reader, MockTokenizer.WHITESPACE, false, 5);
+        return new TokenStreamComponents(t, t);
+      }
+    };
+    
+    assertTokenStreamContents(whitespace.tokenStream("bogus", "test 123 toolong ok "),
+        new String[] { "test", "123", "toolo", "ng", "ok" },
+        new int[] { 0, 5, 9, 14, 17 },
+        new int[] { 4, 8, 14, 16, 19 },
+        new Integer(20));
+    
+    assertTokenStreamContents(whitespace.tokenStream("bogus", "test 123 toolo"),
+        new String[] { "test", "123", "toolo" },
+        new int[] { 0, 5, 9 },
+        new int[] { 4, 8, 14 },
+        new Integer(14));
+  }
+  
   public void testLUCENE_3042() throws Exception {
     String testString = "t";
     
     Analyzer analyzer = new MockAnalyzer(random());
-    TokenStream stream = analyzer.tokenStream("dummy", testString);
-    stream.reset();
-    while (stream.incrementToken()) {
-      // consume
+    try (TokenStream stream = analyzer.tokenStream("dummy", testString)) {
+      stream.reset();
+      while (stream.incrementToken()) {
+        // consume
+      }
+      stream.end();
     }
-    stream.end();
-    stream.close();
     
-    assertAnalyzesToReuse(analyzer, testString, new String[] { "t" });
+    assertAnalyzesTo(analyzer, testString, new String[] { "t" });
   }
 
   /** blast some random strings through the analyzer */
@@ -114,6 +215,25 @@ public class TestMockAnalyzer extends Ba
     checkRandomData(random(), new MockAnalyzer(random()), atLeast(1000));
   }
   
+  /** blast some random strings through differently configured tokenizers */
+  public void testRandomRegexps() throws Exception {
+    int iters = atLeast(30);
+    for (int i = 0; i < iters; i++) {
+      final CharacterRunAutomaton dfa = new CharacterRunAutomaton(AutomatonTestUtil.randomAutomaton(random()));
+      final boolean lowercase = random().nextBoolean();
+      final int limit = _TestUtil.nextInt(random(), 0, 500);
+      Analyzer a = new Analyzer() {
+        @Override
+        protected TokenStreamComponents createComponents(String fieldName, Reader reader) {
+          Tokenizer t = new MockTokenizer(reader, dfa, lowercase, limit);
+          return new TokenStreamComponents(t, t);
+        }
+      };
+      checkRandomData(random(), a, 100);
+      a.close();
+    }
+  }
+  
   public void testForwardOffsets() throws Exception {
     int num = atLeast(10000);
     for (int i = 0; i < num; i++) {
@@ -121,13 +241,13 @@ public class TestMockAnalyzer extends Ba
       StringReader reader = new StringReader(s);
       MockCharFilter charfilter = new MockCharFilter(reader, 2);
       MockAnalyzer analyzer = new MockAnalyzer(random());
-      TokenStream ts = analyzer.tokenStream("bogus", charfilter);
-      ts.reset();
-      while (ts.incrementToken()) {
-        ;
+      try (TokenStream ts = analyzer.tokenStream("bogus", charfilter)) {
+        ts.reset();
+        while (ts.incrementToken()) {
+          ;
+        }
+        ts.end();
       }
-      ts.end();
-      ts.close();
     }
   }
   
@@ -135,7 +255,8 @@ public class TestMockAnalyzer extends Ba
     // LUCENE-5153: test that wrapping an analyzer's reader is allowed
     final Random random = random();
     
-    Analyzer a = new AnalyzerWrapper() {
+    final Analyzer delegate = new MockAnalyzer(random);
+    Analyzer a = new AnalyzerWrapper(delegate.getReuseStrategy()) {
       
       @Override
       protected Reader wrapReader(String fieldName, Reader reader) {
@@ -149,7 +270,7 @@ public class TestMockAnalyzer extends Ba
       
       @Override
       protected Analyzer getWrappedAnalyzer(String fieldName) {
-        return new MockAnalyzer(random);
+        return delegate;
       }
     };
     

Modified: lucene/dev/branches/lucene4956/lucene/core/src/test/org/apache/lucene/codecs/compressing/AbstractTestCompressionMode.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene4956/lucene/core/src/test/org/apache/lucene/codecs/compressing/AbstractTestCompressionMode.java?rev=1534320&r1=1534319&r2=1534320&view=diff
==============================================================================
--- lucene/dev/branches/lucene4956/lucene/core/src/test/org/apache/lucene/codecs/compressing/AbstractTestCompressionMode.java (original)
+++ lucene/dev/branches/lucene4956/lucene/core/src/test/org/apache/lucene/codecs/compressing/AbstractTestCompressionMode.java Mon Oct 21 18:58:24 2013
@@ -24,6 +24,7 @@ import org.apache.lucene.store.ByteArray
 import org.apache.lucene.store.ByteArrayDataOutput;
 import org.apache.lucene.util.BytesRef;
 import org.apache.lucene.util.LuceneTestCase;
+import org.apache.lucene.util._TestUtil;
 
 import com.carrotsearch.randomizedtesting.generators.RandomInts;
 
@@ -49,15 +50,15 @@ public abstract class AbstractTestCompre
     return arr;
   }
 
-  byte[] compress(byte[] decompressed) throws IOException {
+  byte[] compress(byte[] decompressed, int off, int len) throws IOException {
     Compressor compressor = mode.newCompressor();
-    return compress(compressor, decompressed);
+    return compress(compressor, decompressed, off, len);
   }
 
-  static byte[] compress(Compressor compressor, byte[] decompressed) throws IOException {
-    byte[] compressed = new byte[decompressed.length * 2 + 16]; // should be enough
+  static byte[] compress(Compressor compressor, byte[] decompressed, int off, int len) throws IOException {
+    byte[] compressed = new byte[len * 2 + 16]; // should be enough
     ByteArrayDataOutput out = new ByteArrayDataOutput(compressed);
-    compressor.compress(decompressed, 0, decompressed.length, out);
+    compressor.compress(decompressed, off, len, out);
     final int compressedLen = out.getPosition();
     return Arrays.copyOf(compressed, compressedLen);
   }
@@ -84,9 +85,11 @@ public abstract class AbstractTestCompre
     final int iterations = atLeast(10);
     for (int i = 0; i < iterations; ++i) {
       final byte[] decompressed = randomArray();
-      final byte[] compressed = compress(decompressed);
-      final byte[] restored = decompress(compressed, decompressed.length);
-      assertArrayEquals(decompressed, restored);
+      final int off = random().nextBoolean() ? 0 : _TestUtil.nextInt(random(), 0, decompressed.length);
+      final int len = random().nextBoolean() ? decompressed.length - off : _TestUtil.nextInt(random(), 0, decompressed.length - off);
+      final byte[] compressed = compress(decompressed, off, len);
+      final byte[] restored = decompress(compressed, len);
+      assertArrayEquals(Arrays.copyOfRange(decompressed, off, off+len), restored);
     }
   }
 
@@ -94,7 +97,7 @@ public abstract class AbstractTestCompre
     final int iterations = atLeast(10);
     for (int i = 0; i < iterations; ++i) {
       final byte[] decompressed = randomArray();
-      final byte[] compressed = compress(decompressed);
+      final byte[] compressed = compress(decompressed, 0, decompressed.length);
       final int offset, length;
       if (decompressed.length == 0) {
         offset = length = 0;
@@ -108,9 +111,13 @@ public abstract class AbstractTestCompre
   }
 
   public byte[] test(byte[] decompressed) throws IOException {
-    final byte[] compressed = compress(decompressed);
-    final byte[] restored = decompress(compressed, decompressed.length);
-    assertEquals(decompressed.length, restored.length);
+    return test(decompressed, 0, decompressed.length);
+  }
+
+  public byte[] test(byte[] decompressed, int off, int len) throws IOException {
+    final byte[] compressed = compress(decompressed, off, len);
+    final byte[] restored = decompress(compressed, len);
+    assertEquals(len, restored.length);
     return compressed;
   }
 
@@ -130,4 +137,64 @@ public abstract class AbstractTestCompre
     test(decompressed);
   }
 
+  public void testConstant() throws IOException {
+    final byte[] decompressed = new byte[_TestUtil.nextInt(random(), 1, 10000)];
+    Arrays.fill(decompressed, (byte) random().nextInt());
+    test(decompressed);
+  }
+
+  public void testLUCENE5201() throws IOException {
+    byte[] data = new byte[]{
+        14, 72, 14, 85, 3, 72, 14, 85, 3, 72, 14, 72, 14, 72, 14, 85, 3, 72, 14, 72, 14, 72, 14, 72, 14, 72, 14, 72, 14, 85, 3, 72,
+        14, 85, 3, 72, 14, 85, 3, 72, 14, 85, 3, 72, 14, 85, 3, 72, 14, 85, 3, 72, 14, 50, 64, 0, 46, -1, 0, 0, 0, 29, 3, 85,
+        8, -113, 0, 68, -97, 3, 0, 2, 3, -97, 6, 0, 68, -113, 0, 2, 3, -97, 6, 0, 68, -113, 0, 2, 3, 85, 8, -113, 0, 68, -97, 3,
+        0, 2, 3, -97, 6, 0, 68, -113, 0, 2, 3, -97, 6, 0, 68, -113, 0, 2, 3, -97, 6, 0, 68, -113, 0, 2, 3, -97, 6, 0, 68, -113,
+        0, 2, 3, -97, 6, 0, 68, -113, 0, 2, 3, -97, 6, 0, 68, -113, 0, 2, 3, -97, 6, 0, 68, -113, 0, 2, 3, -97, 6, 0, 68, -113,
+        0, 2, 3, -97, 6, 0, 68, -113, 0, 2, 3, -97, 6, 0, 68, -113, 0, 50, 64, 0, 47, -105, 0, 0, 0, 30, 3, -97, 6, 0, 68, -113,
+        0, 2, 3, -97, 6, 0, 68, -113, 0, 2, 3, 85, 8, -113, 0, 68, -97, 3, 0, 2, 3, 85, 8, -113, 0, 68, -97, 3, 0, 2, 3, 85,
+        8, -113, 0, 68, -97, 3, 0, 2, -97, 6, 0, 2, 3, 85, 8, -113, 0, 68, -97, 3, 0, 2, 3, -97, 6, 0, 68, -113, 0, 2, 3, -97,
+        6, 0, 68, -113, 0, 120, 64, 0, 48, 4, 0, 0, 0, 31, 34, 72, 29, 72, 37, 72, 35, 72, 45, 72, 23, 72, 46, 72, 20, 72, 40, 72,
+        33, 72, 25, 72, 39, 72, 38, 72, 26, 72, 28, 72, 42, 72, 24, 72, 27, 72, 36, 72, 41, 72, 32, 72, 18, 72, 30, 72, 22, 72, 31, 72,
+        43, 72, 19, 72, 34, 72, 29, 72, 37, 72, 35, 72, 45, 72, 23, 72, 46, 72, 20, 72, 40, 72, 33, 72, 25, 72, 39, 72, 38, 72, 26, 72,
+        28, 72, 42, 72, 24, 72, 27, 72, 36, 72, 41, 72, 32, 72, 18, 72, 30, 72, 22, 72, 31, 72, 43, 72, 19, 72, 34, 72, 29, 72, 37, 72,
+        35, 72, 45, 72, 23, 72, 46, 72, 20, 72, 40, 72, 33, 72, 25, 72, 39, 72, 38, 72, 26, 72, 28, 72, 42, 72, 24, 72, 27, 72, 36, 72,
+        41, 72, 32, 72, 18, 16, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0,
+        0, 39, 24, 32, 34, 124, 0, 120, 64, 0, 48, 80, 0, 0, 0, 31, 30, 72, 22, 72, 31, 72, 43, 72, 19, 72, 34, 72, 29, 72, 37, 72,
+        35, 72, 45, 72, 23, 72, 46, 72, 20, 72, 40, 72, 33, 72, 25, 72, 39, 72, 38, 72, 26, 72, 28, 72, 42, 72, 24, 72, 27, 72, 36, 72,
+        41, 72, 32, 72, 18, 72, 30, 72, 22, 72, 31, 72, 43, 72, 19, 72, 34, 72, 29, 72, 37, 72, 35, 72, 45, 72, 23, 72, 46, 72, 20, 72,
+        40, 72, 33, 72, 25, 72, 39, 72, 38, 72, 26, 72, 28, 72, 42, 72, 24, 72, 27, 72, 36, 72, 41, 72, 32, 72, 18, 72, 30, 72, 22, 72,
+        31, 72, 43, 72, 19, 72, 34, 72, 29, 72, 37, 72, 35, 72, 45, 72, 23, 72, 46, 72, 20, 72, 40, 72, 33, 72, 25, 72, 39, 72, 38, 72,
+        26, 72, 28, 72, 42, 72, 24, 72, 27, 72, 36, 72, 41, 72, 32, 72, 18, 72, 30, 72, 22, 72, 31, 72, 43, 72, 19, 72, 34, 72, 29, 72,
+        37, 72, 35, 72, 45, 72, 23, 72, 46, 72, 20, 72, 40, 72, 33, 72, 25, 72, 39, 72, 38, 72, 26, 72, 28, 72, 42, 72, 24, 72, 27, 72,
+        36, 72, 41, 72, 32, 72, 18, 72, 30, 72, 22, 72, 31, 72, 43, 72, 19, 72, 34, 72, 29, 72, 37, 72, 35, 72, 45, 72, 23, 72, 46, 72,
+        20, 72, 40, 72, 33, 72, 25, 72, 39, 72, 38, 72, 26, 72, 28, 72, 42, 72, 24, 72, 27, 72, 36, 72, 41, 72, 32, 72, 18, 72, 30, 72,
+        22, 72, 31, 72, 43, 72, 19, 72, 34, 72, 29, 72, 37, 72, 35, 72, 45, 72, 23, 72, 46, 72, 20, 72, 40, 72, 33, 72, 25, 72, 39, 72,
+        38, 72, 26, 72, 28, 72, 42, 72, 24, 72, 27, 72, 36, 72, 41, 72, 32, 72, 18, 72, 30, 72, 22, 72, 31, 72, 43, 72, 19, 72, 34, 72,
+        29, 72, 37, 72, 35, 72, 45, 72, 23, 72, 46, 72, 20, 72, 40, 72, 33, 72, 25, 72, 39, 72, 38, 72, 26, 72, 28, 72, 42, 72, 24, 72,
+        27, 72, 36, 72, 41, 72, 32, 72, 18, 72, 30, 72, 22, 72, 31, 72, 43, 72, 19, 50, 64, 0, 49, 20, 0, 0, 0, 32, 3, -97, 6, 0,
+        68, -113, 0, 2, 3, 85, 8, -113, 0, 68, -97, 3, 0, 2, 3, -97, 6, 0, 68, -113, 0, 2, 3, -97, 6, 0, 68, -113, 0, 2, 3, -97,
+        6, 0, 68, -113, 0, 2, 3, 85, 8, -113, 0, 68, -97, 3, 0, 2, 3, -97, 6, 0, 68, -113, 0, 2, 3, -97, 6, 0, 68, -113, 0, 2,
+        3, -97, 6, 0, 68, -113, 0, 2, 3, -97, 6, 0, 68, -113, 0, 2, 3, -97, 6, 0, 68, -113, 0, 2, 3, -97, 6, 0, 68, -113, 0, 2,
+        3, -97, 6, 0, 50, 64, 0, 50, 53, 0, 0, 0, 34, 3, -97, 6, 0, 68, -113, 0, 2, 3, 85, 8, -113, 0, 68, -113, 0, 2, 3, -97,
+        6, 0, 68, -113, 0, 2, 3, 85, 8, -113, 0, 68, -113, 0, 2, 3, -97, 6, 0, 68, -113, 0, 2, 3, -97, 6, 0, 68, -113, 0, 2, 3,
+        -97, 6, 0, 68, -113, 0, 2, 3, 85, 8, -113, 0, 68, -97, 3, 0, 2, 3, -97, 6, 0, 68, -113, 0, 2, 3, 85, 8, -113, 0, 68, -97,
+        3, 0, 2, 3, 85, 8, -113, 0, 68, -97, 3, 0, 2, 3, -97, 6, 0, 68, -113, 0, 2, 3, 85, 8, -113, 0, 68, -97, 3, 0, 2, 3,
+        85, 8, -113, 0, 68, -97, 3, 0, 2, 3, -97, 6, 0, 68, -113, 0, 2, 3, -97, 6, 0, 68, -113, 0, 2, 3, -97, 6, 0, 68, -113, 0,
+        2, 3, 85, 8, -113, 0, 68, -97, 3, 0, 2, 3, 85, 8, -113, 0, 68, -97, 3, 0, 2, 3, 85, 8, -113, 0, 68, -97, 3, 0, 2, 3,
+        -97, 6, 0, 50, 64, 0, 51, 85, 0, 0, 0, 36, 3, 85, 8, -113, 0, 68, -97, 3, 0, 2, 3, -97, 6, 0, 68, -113, 0, 2, 3, -97,
+        6, 0, 68, -113, 0, 2, 3, -97, 6, 0, 68, -113, 0, 2, 3, -97, 6, 0, 68, -113, 0, 2, -97, 5, 0, 2, 3, 85, 8, -113, 0, 68,
+        -97, 3, 0, 2, 3, -97, 6, 0, 68, -113, 0, 2, 3, -97, 6, 0, 68, -113, 0, 2, 3, -97, 6, 0, 68, -113, 0, 2, 3, -97, 6, 0,
+        68, -113, 0, 2, 3, -97, 6, 0, 50, -64, 0, 51, -45, 0, 0, 0, 37, 68, -113, 0, 2, 3, -97, 6, 0, 68, -113, 0, 2, 3, -97, 6,
+        0, 68, -113, 0, 2, 3, -97, 6, 0, 68, -113, 0, 2, 3, -97, 6, 0, 68, -113, 0, 2, 3, 85, 8, -113, 0, 68, -113, 0, 2, 3, -97,
+        6, 0, 68, -113, 0, 2, 3, 85, 8, -113, 0, 68, -97, 3, 0, 2, 3, 85, 8, -113, 0, 68, -97, 3, 0, 120, 64, 0, 52, -88, 0, 0,
+        0, 39, 13, 85, 5, 72, 13, 85, 5, 72, 13, 85, 5, 72, 13, 72, 13, 85, 5, 72, 13, 85, 5, 72, 13, 85, 5, 72, 13, 85, 5, 72,
+        13, 72, 13, 85, 5, 72, 13, 85, 5, 72, 13, 72, 13, 72, 13, 85, 5, 72, 13, 85, 5, 72, 13, 85, 5, 72, 13, 85, 5, 72, 13, 85,
+        5, 72, 13, 85, 5, 72, 13, 72, 13, 72, 13, 72, 13, 85, 5, 72, 13, 85, 5, 72, 13, 72, 13, 85, 5, 72, 13, 85, 5, 72, 13, 85,
+        5, 72, 13, 85, 5, 72, 13, 85, 5, 72, 13, 85, 5, 72, 13, 85, 5, 72, 13, 85, 5, 72, 13, 85, 5, 72, 13, 85, 5, 72, 13, 85,
+        5, 72, 13, 85, 5, 72, 13, 72, 13, 72, 13, 72, 13, 85, 5, 72, 13, 85, 5, 72, 13, 85, 5, 72, 13, 72, 13, 85, 5, 72, 13, 72,
+        13, 85, 5, 72, 13, 72, 13, 85, 5, 72, 13, -19, -24, -101, -35
+      };
+    test(data, 9, data.length - 9);
+  }
+
 }
\ No newline at end of file

Modified: lucene/dev/branches/lucene4956/lucene/core/src/test/org/apache/lucene/codecs/compressing/AbstractTestLZ4CompressionMode.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene4956/lucene/core/src/test/org/apache/lucene/codecs/compressing/AbstractTestLZ4CompressionMode.java?rev=1534320&r1=1534319&r2=1534320&view=diff
==============================================================================
--- lucene/dev/branches/lucene4956/lucene/core/src/test/org/apache/lucene/codecs/compressing/AbstractTestLZ4CompressionMode.java (original)
+++ lucene/dev/branches/lucene4956/lucene/core/src/test/org/apache/lucene/codecs/compressing/AbstractTestLZ4CompressionMode.java Mon Oct 21 18:58:24 2013
@@ -104,4 +104,8 @@ public abstract class AbstractTestLZ4Com
     test(decompressed);
   }
 
+  public void testMatchRightBeforeLastLiterals() throws IOException {
+    test(new byte[] {1,2,3,4, 1,2,3,4, 1,2,3,4,5});
+  }
+
 }

Modified: lucene/dev/branches/lucene4956/lucene/core/src/test/org/apache/lucene/codecs/compressing/TestCompressingStoredFieldsFormat.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene4956/lucene/core/src/test/org/apache/lucene/codecs/compressing/TestCompressingStoredFieldsFormat.java?rev=1534320&r1=1534319&r2=1534320&view=diff
==============================================================================
--- lucene/dev/branches/lucene4956/lucene/core/src/test/org/apache/lucene/codecs/compressing/TestCompressingStoredFieldsFormat.java (original)
+++ lucene/dev/branches/lucene4956/lucene/core/src/test/org/apache/lucene/codecs/compressing/TestCompressingStoredFieldsFormat.java Mon Oct 21 18:58:24 2013
@@ -32,8 +32,10 @@ import org.apache.lucene.index.RandomInd
 import org.apache.lucene.store.Directory;
 import org.junit.Test;
 
+import com.carrotsearch.randomizedtesting.annotations.Repeat;
 import com.carrotsearch.randomizedtesting.generators.RandomInts;
 
+@Repeat(iterations=5) // give it a chance to test various compression modes with different chunk sizes
 public class TestCompressingStoredFieldsFormat extends BaseStoredFieldsFormatTestCase {
 
   @Override

Modified: lucene/dev/branches/lucene4956/lucene/core/src/test/org/apache/lucene/codecs/compressing/TestCompressingTermVectorsFormat.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene4956/lucene/core/src/test/org/apache/lucene/codecs/compressing/TestCompressingTermVectorsFormat.java?rev=1534320&r1=1534319&r2=1534320&view=diff
==============================================================================
--- lucene/dev/branches/lucene4956/lucene/core/src/test/org/apache/lucene/codecs/compressing/TestCompressingTermVectorsFormat.java (original)
+++ lucene/dev/branches/lucene4956/lucene/core/src/test/org/apache/lucene/codecs/compressing/TestCompressingTermVectorsFormat.java Mon Oct 21 18:58:24 2013
@@ -14,6 +14,8 @@ import org.apache.lucene.index.TermsEnum
 import org.apache.lucene.store.Directory;
 import org.apache.lucene.util.BytesRef;
 
+import com.carrotsearch.randomizedtesting.annotations.Repeat;
+
 /*
  * Licensed to the Apache Software Foundation (ASF) under one or more
  * contributor license agreements.  See the NOTICE file distributed with
@@ -31,6 +33,7 @@ import org.apache.lucene.util.BytesRef;
  * limitations under the License.
  */
 
+@Repeat(iterations=5) // give it a chance to test various compression modes with different chunk sizes
 public class TestCompressingTermVectorsFormat extends BaseTermVectorsFormatTestCase {
 
   @Override

Modified: lucene/dev/branches/lucene4956/lucene/core/src/test/org/apache/lucene/codecs/compressing/TestFastDecompressionMode.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene4956/lucene/core/src/test/org/apache/lucene/codecs/compressing/TestFastDecompressionMode.java?rev=1534320&r1=1534319&r2=1534320&view=diff
==============================================================================
--- lucene/dev/branches/lucene4956/lucene/core/src/test/org/apache/lucene/codecs/compressing/TestFastDecompressionMode.java (original)
+++ lucene/dev/branches/lucene4956/lucene/core/src/test/org/apache/lucene/codecs/compressing/TestFastDecompressionMode.java Mon Oct 21 18:58:24 2013
@@ -28,9 +28,9 @@ public class TestFastDecompressionMode e
   }
 
   @Override
-  public byte[] test(byte[] decompressed) throws IOException {
-    final byte[] compressed = super.test(decompressed);
-    final byte[] compressed2 = compress(CompressionMode.FAST.newCompressor(), decompressed);
+  public byte[] test(byte[] decompressed, int off, int len) throws IOException {
+    final byte[] compressed = super.test(decompressed, off, len);
+    final byte[] compressed2 = compress(CompressionMode.FAST.newCompressor(), decompressed, off, len);
     // because of the way this compression mode works, its output is necessarily
     // smaller than the output of CompressionMode.FAST
     assertTrue(compressed.length <= compressed2.length);

Modified: lucene/dev/branches/lucene4956/lucene/core/src/test/org/apache/lucene/codecs/lucene40/TestLucene40DocValuesFormat.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene4956/lucene/core/src/test/org/apache/lucene/codecs/lucene40/TestLucene40DocValuesFormat.java?rev=1534320&r1=1534319&r2=1534320&view=diff
==============================================================================
--- lucene/dev/branches/lucene4956/lucene/core/src/test/org/apache/lucene/codecs/lucene40/TestLucene40DocValuesFormat.java (original)
+++ lucene/dev/branches/lucene4956/lucene/core/src/test/org/apache/lucene/codecs/lucene40/TestLucene40DocValuesFormat.java Mon Oct 21 18:58:24 2013
@@ -30,5 +30,11 @@ public class TestLucene40DocValuesFormat
   protected Codec getCodec() {
     return codec;
   }
-  
+
+  // LUCENE-4583: This codec should throw IAE on huge binary values:
+  @Override
+  protected boolean codecAcceptsHugeBinaryValues(String field) {
+    return false;
+  }
+
 }

Modified: lucene/dev/branches/lucene4956/lucene/core/src/test/org/apache/lucene/codecs/lucene41/TestBlockPostingsFormat3.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene4956/lucene/core/src/test/org/apache/lucene/codecs/lucene41/TestBlockPostingsFormat3.java?rev=1534320&r1=1534319&r2=1534320&view=diff
==============================================================================
--- lucene/dev/branches/lucene4956/lucene/core/src/test/org/apache/lucene/codecs/lucene41/TestBlockPostingsFormat3.java (original)
+++ lucene/dev/branches/lucene4956/lucene/core/src/test/org/apache/lucene/codecs/lucene41/TestBlockPostingsFormat3.java Mon Oct 21 18:58:24 2013
@@ -67,7 +67,7 @@ public class TestBlockPostingsFormat3 ex
   // creates 8 fields with different options and does "duels" of fields against each other
   public void test() throws Exception {
     Directory dir = newDirectory();
-    Analyzer analyzer = new Analyzer(new Analyzer.PerFieldReuseStrategy()) {
+    Analyzer analyzer = new Analyzer(Analyzer.PER_FIELD_REUSE_STRATEGY) {
       @Override
       protected TokenStreamComponents createComponents(String fieldName, Reader reader) {
         Tokenizer tokenizer = new MockTokenizer(reader);
@@ -261,7 +261,6 @@ public class TestBlockPostingsFormat3 ex
    * checks collection-level statistics on Terms 
    */
   public void assertTermsStatistics(Terms leftTerms, Terms rightTerms) throws Exception {
-    assert leftTerms.getComparator() == rightTerms.getComparator();
     if (leftTerms.getDocCount() != -1 && rightTerms.getDocCount() != -1) {
       assertEquals(leftTerms.getDocCount(), rightTerms.getDocCount());
     }

Modified: lucene/dev/branches/lucene4956/lucene/core/src/test/org/apache/lucene/codecs/lucene42/TestLucene42DocValuesFormat.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene4956/lucene/core/src/test/org/apache/lucene/codecs/lucene42/TestLucene42DocValuesFormat.java?rev=1534320&r1=1534319&r2=1534320&view=diff
==============================================================================
--- lucene/dev/branches/lucene4956/lucene/core/src/test/org/apache/lucene/codecs/lucene42/TestLucene42DocValuesFormat.java (original)
+++ lucene/dev/branches/lucene4956/lucene/core/src/test/org/apache/lucene/codecs/lucene42/TestLucene42DocValuesFormat.java Mon Oct 21 18:58:24 2013
@@ -24,10 +24,15 @@ import org.apache.lucene.index.BaseCompr
  * Tests Lucene42DocValuesFormat
  */
 public class TestLucene42DocValuesFormat extends BaseCompressingDocValuesFormatTestCase {
-  private final Codec codec = new Lucene42Codec();
+  private final Codec codec = new Lucene42RWCodec();
 
   @Override
   protected Codec getCodec() {
     return codec;
   }
+
+  @Override
+  protected boolean codecAcceptsHugeBinaryValues(String field) {
+    return false;
+  }
 }

Modified: lucene/dev/branches/lucene4956/lucene/core/src/test/org/apache/lucene/codecs/perfield/TestPerFieldDocValuesFormat.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene4956/lucene/core/src/test/org/apache/lucene/codecs/perfield/TestPerFieldDocValuesFormat.java?rev=1534320&r1=1534319&r2=1534320&view=diff
==============================================================================
--- lucene/dev/branches/lucene4956/lucene/core/src/test/org/apache/lucene/codecs/perfield/TestPerFieldDocValuesFormat.java (original)
+++ lucene/dev/branches/lucene4956/lucene/core/src/test/org/apache/lucene/codecs/perfield/TestPerFieldDocValuesFormat.java Mon Oct 21 18:58:24 2013
@@ -25,7 +25,7 @@ import org.apache.lucene.analysis.Analyz
 import org.apache.lucene.analysis.MockAnalyzer;
 import org.apache.lucene.codecs.Codec;
 import org.apache.lucene.codecs.DocValuesFormat;
-import org.apache.lucene.codecs.lucene42.Lucene42Codec;
+import org.apache.lucene.codecs.lucene46.Lucene46Codec;
 import org.apache.lucene.document.BinaryDocValuesField;
 import org.apache.lucene.document.Document;
 import org.apache.lucene.document.Field;
@@ -46,6 +46,7 @@ import org.apache.lucene.search.TermQuer
 import org.apache.lucene.search.TopDocs;
 import org.apache.lucene.store.Directory;
 import org.apache.lucene.util.BytesRef;
+import org.apache.lucene.util._TestUtil;
 
 /**
  * Basic tests of PerFieldDocValuesFormat
@@ -63,6 +64,11 @@ public class TestPerFieldDocValuesFormat
   protected Codec getCodec() {
     return codec;
   }
+
+  @Override
+  protected boolean codecAcceptsHugeBinaryValues(String field) {
+    return _TestUtil.fieldSupportsHugeBinaryDocValues(field);
+  }
   
   // just a simple trivial test
   // TODO: we should come up with a test that somehow checks that segment suffix
@@ -73,9 +79,9 @@ public class TestPerFieldDocValuesFormat
     Directory directory = newDirectory();
     // we don't use RandomIndexWriter because it might add more docvalues than we expect !!!!1
     IndexWriterConfig iwc = newIndexWriterConfig(TEST_VERSION_CURRENT, analyzer);
-    final DocValuesFormat fast = DocValuesFormat.forName("Lucene42");
+    final DocValuesFormat fast = DocValuesFormat.forName("Lucene45");
     final DocValuesFormat slow = DocValuesFormat.forName("SimpleText");
-    iwc.setCodec(new Lucene42Codec() {
+    iwc.setCodec(new Lucene46Codec() {
       @Override
       public DocValuesFormat getDocValuesFormatForField(String field) {
         if ("dv1".equals(field)) {

Modified: lucene/dev/branches/lucene4956/lucene/core/src/test/org/apache/lucene/codecs/perfield/TestPerFieldPostingsFormat2.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene4956/lucene/core/src/test/org/apache/lucene/codecs/perfield/TestPerFieldPostingsFormat2.java?rev=1534320&r1=1534319&r2=1534320&view=diff
==============================================================================
--- lucene/dev/branches/lucene4956/lucene/core/src/test/org/apache/lucene/codecs/perfield/TestPerFieldPostingsFormat2.java (original)
+++ lucene/dev/branches/lucene4956/lucene/core/src/test/org/apache/lucene/codecs/perfield/TestPerFieldPostingsFormat2.java Mon Oct 21 18:58:24 2013
@@ -21,8 +21,8 @@ import java.io.IOException;
 import org.apache.lucene.analysis.MockAnalyzer;
 import org.apache.lucene.codecs.Codec;
 import org.apache.lucene.codecs.PostingsFormat;
-import org.apache.lucene.codecs.lucene42.Lucene42Codec;
 import org.apache.lucene.codecs.lucene41.Lucene41PostingsFormat;
+import org.apache.lucene.codecs.lucene46.Lucene46Codec;
 import org.apache.lucene.codecs.mocksep.MockSepPostingsFormat;
 import org.apache.lucene.codecs.pulsing.Pulsing41PostingsFormat;
 import org.apache.lucene.codecs.simpletext.SimpleTextPostingsFormat;
@@ -34,10 +34,10 @@ import org.apache.lucene.index.Directory
 import org.apache.lucene.index.IndexReader;
 import org.apache.lucene.index.IndexWriter;
 import org.apache.lucene.index.IndexWriterConfig;
+import org.apache.lucene.index.IndexWriterConfig.OpenMode;
 import org.apache.lucene.index.LogDocMergePolicy;
 import org.apache.lucene.index.RandomIndexWriter;
 import org.apache.lucene.index.Term;
-import org.apache.lucene.index.IndexWriterConfig.OpenMode;
 import org.apache.lucene.search.IndexSearcher;
 import org.apache.lucene.search.TermQuery;
 import org.apache.lucene.search.TopDocs;
@@ -200,7 +200,7 @@ public class TestPerFieldPostingsFormat2
 
   }
 
-  public static class MockCodec extends Lucene42Codec {
+  public static class MockCodec extends Lucene46Codec {
     final PostingsFormat lucene40 = new Lucene41PostingsFormat();
     final PostingsFormat simpleText = new SimpleTextPostingsFormat();
     final PostingsFormat mockSep = new MockSepPostingsFormat();
@@ -217,7 +217,7 @@ public class TestPerFieldPostingsFormat2
     }
   }
 
-  public static class MockCodec2 extends Lucene42Codec {
+  public static class MockCodec2 extends Lucene46Codec {
     final PostingsFormat lucene40 = new Lucene41PostingsFormat();
     final PostingsFormat simpleText = new SimpleTextPostingsFormat();
     
@@ -268,7 +268,7 @@ public class TestPerFieldPostingsFormat2
   }
   
   public void testSameCodecDifferentInstance() throws Exception {
-    Codec codec = new Lucene42Codec() {
+    Codec codec = new Lucene46Codec() {
       @Override
       public PostingsFormat getPostingsFormatForField(String field) {
         if ("id".equals(field)) {
@@ -284,7 +284,7 @@ public class TestPerFieldPostingsFormat2
   }
   
   public void testSameCodecDifferentParams() throws Exception {
-    Codec codec = new Lucene42Codec() {
+    Codec codec = new Lucene46Codec() {
       @Override
       public PostingsFormat getPostingsFormatForField(String field) {
         if ("id".equals(field)) {

Modified: lucene/dev/branches/lucene4956/lucene/core/src/test/org/apache/lucene/index/BinaryTokenStream.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene4956/lucene/core/src/test/org/apache/lucene/index/BinaryTokenStream.java?rev=1534320&r1=1534319&r2=1534320&view=diff
==============================================================================
--- lucene/dev/branches/lucene4956/lucene/core/src/test/org/apache/lucene/index/BinaryTokenStream.java (original)
+++ lucene/dev/branches/lucene4956/lucene/core/src/test/org/apache/lucene/index/BinaryTokenStream.java Mon Oct 21 18:58:24 2013
@@ -31,16 +31,19 @@ import org.apache.lucene.analysis.Canned
  */
 public final class BinaryTokenStream extends TokenStream {
   private final ByteTermAttribute bytesAtt = addAttribute(ByteTermAttribute.class);
+  private final BytesRef bytes;
   private boolean available = true;
   
   public BinaryTokenStream(BytesRef bytes) {
-    bytesAtt.setBytesRef(bytes);
+    this.bytes = bytes;
   }
   
   @Override
   public boolean incrementToken() {
     if (available) {
+      clearAttributes();
       available = false;
+      bytesAtt.setBytesRef(bytes);
       return true;
     }
     return false;

Modified: lucene/dev/branches/lucene4956/lucene/core/src/test/org/apache/lucene/index/Test2BPositions.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene4956/lucene/core/src/test/org/apache/lucene/index/Test2BPositions.java?rev=1534320&r1=1534319&r2=1534320&view=diff
==============================================================================
--- lucene/dev/branches/lucene4956/lucene/core/src/test/org/apache/lucene/index/Test2BPositions.java (original)
+++ lucene/dev/branches/lucene4956/lucene/core/src/test/org/apache/lucene/index/Test2BPositions.java Mon Oct 21 18:58:24 2013
@@ -88,14 +88,12 @@ public class Test2BPositions extends Luc
     private final PositionIncrementAttribute posIncAtt = addAttribute(PositionIncrementAttribute.class);
     int index;
 
-    public MyTokenStream() {
-      termAtt.setLength(1);
-      termAtt.buffer()[0] = 'a';
-    }
-    
     @Override
     public boolean incrementToken() {
       if (index < 52) {
+        clearAttributes();
+        termAtt.setLength(1);
+        termAtt.buffer()[0] = 'a';
         posIncAtt.setPositionIncrement(1+index);
         index++;
         return true;

Modified: lucene/dev/branches/lucene4956/lucene/core/src/test/org/apache/lucene/index/Test2BPostings.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene4956/lucene/core/src/test/org/apache/lucene/index/Test2BPostings.java?rev=1534320&r1=1534319&r2=1534320&view=diff
==============================================================================
--- lucene/dev/branches/lucene4956/lucene/core/src/test/org/apache/lucene/index/Test2BPostings.java (original)
+++ lucene/dev/branches/lucene4956/lucene/core/src/test/org/apache/lucene/index/Test2BPostings.java Mon Oct 21 18:58:24 2013
@@ -85,18 +85,14 @@ public class Test2BPostings extends Luce
   
   public static final class MyTokenStream extends TokenStream {
     private final CharTermAttribute termAtt = addAttribute(CharTermAttribute.class);
-    private final char buffer[];
     int index;
 
-    public MyTokenStream() {
-      termAtt.setLength(1);
-      buffer = termAtt.buffer();
-    }
-    
     @Override
     public boolean incrementToken() {
       if (index <= 'z') {
-        buffer[0] = (char) index++;
+        clearAttributes();
+        termAtt.setLength(1);
+        termAtt.buffer()[0] = (char) index++;
         return true;
       }
       return false;

Modified: lucene/dev/branches/lucene4956/lucene/core/src/test/org/apache/lucene/index/Test2BPostingsBytes.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene4956/lucene/core/src/test/org/apache/lucene/index/Test2BPostingsBytes.java?rev=1534320&r1=1534319&r2=1534320&view=diff
==============================================================================
--- lucene/dev/branches/lucene4956/lucene/core/src/test/org/apache/lucene/index/Test2BPostingsBytes.java (original)
+++ lucene/dev/branches/lucene4956/lucene/core/src/test/org/apache/lucene/index/Test2BPostingsBytes.java Mon Oct 21 18:58:24 2013
@@ -129,14 +129,12 @@ public class Test2BPostingsBytes extends
     int index;
     int n;
 
-    public MyTokenStream() {
-      termAtt.setLength(1);
-      termAtt.buffer()[0] = 'a';
-    }
-    
     @Override
     public boolean incrementToken() {
       if (index < n) {
+        clearAttributes();
+        termAtt.buffer()[0] = 'a';
+        termAtt.setLength(1);
         index++;
         return true;
       }

Modified: lucene/dev/branches/lucene4956/lucene/core/src/test/org/apache/lucene/index/TestAddIndexes.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene4956/lucene/core/src/test/org/apache/lucene/index/TestAddIndexes.java?rev=1534320&r1=1534319&r2=1534320&view=diff
==============================================================================
--- lucene/dev/branches/lucene4956/lucene/core/src/test/org/apache/lucene/index/TestAddIndexes.java (original)
+++ lucene/dev/branches/lucene4956/lucene/core/src/test/org/apache/lucene/index/TestAddIndexes.java Mon Oct 21 18:58:24 2013
@@ -28,7 +28,7 @@ import org.apache.lucene.analysis.MockAn
 import org.apache.lucene.codecs.Codec;
 import org.apache.lucene.codecs.FilterCodec;
 import org.apache.lucene.codecs.PostingsFormat;
-import org.apache.lucene.codecs.lucene42.Lucene42Codec;
+import org.apache.lucene.codecs.lucene46.Lucene46Codec;
 import org.apache.lucene.codecs.pulsing.Pulsing41PostingsFormat;
 import org.apache.lucene.document.Document;
 import org.apache.lucene.document.Field;
@@ -43,7 +43,6 @@ import org.apache.lucene.store.BaseDirec
 import org.apache.lucene.store.Directory;
 import org.apache.lucene.store.MockDirectoryWrapper;
 import org.apache.lucene.store.RAMDirectory;
-import org.apache.lucene.util.Bits;
 import org.apache.lucene.util.LuceneTestCase;
 import org.apache.lucene.util._TestUtil;
 
@@ -1060,7 +1059,7 @@ public class TestAddIndexes extends Luce
     aux2.close();
   }
 
-  private static final class CustomPerFieldCodec extends Lucene42Codec {
+  private static final class CustomPerFieldCodec extends Lucene46Codec {
     private final PostingsFormat simpleTextFormat = PostingsFormat.forName("SimpleText");
     private final PostingsFormat defaultFormat = PostingsFormat.forName("Lucene41");
     private final PostingsFormat mockSepFormat = PostingsFormat.forName("MockSep");
@@ -1111,7 +1110,7 @@ public class TestAddIndexes extends Luce
   
   private static final class UnRegisteredCodec extends FilterCodec {
     public UnRegisteredCodec() {
-      super("NotRegistered", new Lucene42Codec());
+      super("NotRegistered", new Lucene46Codec());
     }
   }
   
@@ -1229,17 +1228,7 @@ public class TestAddIndexes extends Luce
     Directory src = newDirectory(), dest = newDirectory();
     RandomIndexWriter w = new RandomIndexWriter(random(), src);
     w.addDocument(new Document());
-    IndexReader allDeletedReader = new FilterAtomicReader(w.getReader().leaves().get(0).reader()) {
-      @Override
-      public Bits getLiveDocs() {
-        return new Bits() {
-          @Override public int length() { return 1; }
-          @Override public boolean get(int index) { return false; }
-        };
-      }
-      @Override public boolean hasDeletions() { return true; }
-      @Override public int numDocs() { return 0; }
-    };
+    IndexReader allDeletedReader = new AllDeletedFilterReader(w.getReader().leaves().get(0).reader());
     w.close();
     
     w = new RandomIndexWriter(random(), dest);

Modified: lucene/dev/branches/lucene4956/lucene/core/src/test/org/apache/lucene/index/TestAllFilesHaveCodecHeader.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene4956/lucene/core/src/test/org/apache/lucene/index/TestAllFilesHaveCodecHeader.java?rev=1534320&r1=1534319&r2=1534320&view=diff
==============================================================================
--- lucene/dev/branches/lucene4956/lucene/core/src/test/org/apache/lucene/index/TestAllFilesHaveCodecHeader.java (original)
+++ lucene/dev/branches/lucene4956/lucene/core/src/test/org/apache/lucene/index/TestAllFilesHaveCodecHeader.java Mon Oct 21 18:58:24 2013
@@ -21,12 +21,9 @@ import java.io.IOException;
 
 import org.apache.lucene.analysis.MockAnalyzer;
 import org.apache.lucene.codecs.CodecUtil;
-import org.apache.lucene.codecs.lucene42.Lucene42Codec;
+import org.apache.lucene.codecs.lucene46.Lucene46Codec;
 import org.apache.lucene.document.Document;
 import org.apache.lucene.document.Field;
-import org.apache.lucene.index.IndexFileNames;
-import org.apache.lucene.index.IndexWriterConfig;
-import org.apache.lucene.index.RandomIndexWriter;
 import org.apache.lucene.store.CompoundFileDirectory;
 import org.apache.lucene.store.Directory;
 import org.apache.lucene.store.IndexInput;
@@ -41,7 +38,7 @@ public class TestAllFilesHaveCodecHeader
   public void test() throws Exception {
     Directory dir = newDirectory();
     IndexWriterConfig conf = newIndexWriterConfig(TEST_VERSION_CURRENT, new MockAnalyzer(random()));
-    conf.setCodec(new Lucene42Codec());
+    conf.setCodec(new Lucene46Codec());
     // riw should sometimes create docvalues fields, etc
     RandomIndexWriter riw = new RandomIndexWriter(random(), dir, conf);
     Document doc = new Document();

Modified: lucene/dev/branches/lucene4956/lucene/core/src/test/org/apache/lucene/index/TestBackwardsCompatibility.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene4956/lucene/core/src/test/org/apache/lucene/index/TestBackwardsCompatibility.java?rev=1534320&r1=1534319&r2=1534320&view=diff
==============================================================================
--- lucene/dev/branches/lucene4956/lucene/core/src/test/org/apache/lucene/index/TestBackwardsCompatibility.java (original)
+++ lucene/dev/branches/lucene4956/lucene/core/src/test/org/apache/lucene/index/TestBackwardsCompatibility.java Mon Oct 21 18:58:24 2013
@@ -37,8 +37,8 @@ import org.apache.lucene.document.Field;
 import org.apache.lucene.document.FieldType;
 import org.apache.lucene.document.FloatDocValuesField;
 import org.apache.lucene.document.IntField;
-import org.apache.lucene.document.NumericDocValuesField;
 import org.apache.lucene.document.LongField;
+import org.apache.lucene.document.NumericDocValuesField;
 import org.apache.lucene.document.SortedDocValuesField;
 import org.apache.lucene.document.SortedSetDocValuesField;
 import org.apache.lucene.document.StringField;
@@ -53,17 +53,19 @@ import org.apache.lucene.search.ScoreDoc
 import org.apache.lucene.search.TermQuery;
 import org.apache.lucene.store.BaseDirectoryWrapper;
 import org.apache.lucene.store.Directory;
+import org.apache.lucene.store.FSDirectory;
+import org.apache.lucene.store.NIOFSDirectory;
 import org.apache.lucene.store.RAMDirectory;
+import org.apache.lucene.store.SimpleFSDirectory;
 import org.apache.lucene.util.Bits;
 import org.apache.lucene.util.BytesRef;
 import org.apache.lucene.util.Constants;
-import org.apache.lucene.util.LuceneTestCase.SuppressCodecs;
 import org.apache.lucene.util.LuceneTestCase;
+import org.apache.lucene.util.LuceneTestCase.SuppressCodecs;
 import org.apache.lucene.util.StringHelper;
 import org.apache.lucene.util._TestUtil;
 import org.junit.AfterClass;
 import org.junit.BeforeClass;
-import org.junit.Ignore;
 
 /*
   Verify we can read the pre-5.0 file format, do searches
@@ -74,7 +76,7 @@ import org.junit.Ignore;
 // we won't even be running the actual code, only the impostor
 // @SuppressCodecs("Lucene4x")
 // Sep codec cannot yet handle the offsets in our 4.x index!
-@SuppressCodecs({"MockFixedIntBlock", "MockVariableIntBlock", "MockSep", "MockRandom", "Lucene40", "Lucene41"})
+@SuppressCodecs({"MockFixedIntBlock", "MockVariableIntBlock", "MockSep", "MockRandom", "Lucene40", "Lucene41", "Lucene42", "Lucene45"})
 public class TestBackwardsCompatibility extends LuceneTestCase {
 
   // Uncomment these cases & run them on an older Lucene version,
@@ -192,7 +194,23 @@ public class TestBackwardsCompatibility 
   };
   
   static Map<String,Directory> oldIndexDirs;
-  
+
+  /**
+   * Randomizes the use of some of hte constructor variations
+   */
+  private static IndexUpgrader newIndexUpgrader(Directory dir) {
+    final boolean streamType = random().nextBoolean();
+    final int choice = _TestUtil.nextInt(random(), 0, 2);
+    switch (choice) {
+      case 0: return new IndexUpgrader(dir, TEST_VERSION_CURRENT);
+      case 1: return new IndexUpgrader(dir, TEST_VERSION_CURRENT, 
+                                       streamType ? null : System.err, false);
+      case 2: return new IndexUpgrader(dir, newIndexWriterConfig(TEST_VERSION_CURRENT, null), false);
+      default: fail("case statement didn't get updated when random bounds changed");
+    }
+    return null; // never get here
+  }
+
   @BeforeClass
   public static void beforeClass() throws Exception {
     List<String> names = new ArrayList<String>(oldNames.length + oldSingleSegmentNames.length);
@@ -917,8 +935,7 @@ public class TestBackwardsCompatibility 
       }
       Directory dir = newDirectory(oldIndexDirs.get(name));
 
-      new IndexUpgrader(dir, newIndexWriterConfig(TEST_VERSION_CURRENT, null), false)
-        .upgrade();
+      newIndexUpgrader(dir).upgrade();
 
       checkAllSegmentsUpgraded(dir);
       
@@ -926,6 +943,52 @@ public class TestBackwardsCompatibility 
     }
   }
 
+  public void testCommandLineArgs() throws Exception {
+
+    for (String name : oldIndexDirs.keySet()) {
+      File dir = _TestUtil.getTempDir(name);
+      File dataFile = new File(TestBackwardsCompatibility.class.getResource("index." + name + ".zip").toURI());
+      _TestUtil.unzip(dataFile, dir);
+
+      String path = dir.getAbsolutePath();
+      
+      List<String> args = new ArrayList<String>();
+      if (random().nextBoolean()) {
+        args.add("-verbose");
+      }
+      if (random().nextBoolean()) {
+        args.add("-delete-prior-commits");
+      }
+      if (random().nextBoolean()) {
+        // TODO: need to better randomize this, but ...
+        //  - LuceneTestCase.FS_DIRECTORIES is private
+        //  - newFSDirectory returns BaseDirectoryWrapper
+        //  - BaseDirectoryWrapper doesn't expose delegate
+        Class<? extends FSDirectory> dirImpl = random().nextBoolean() ?
+          SimpleFSDirectory.class : NIOFSDirectory.class;
+
+        args.add("-dir-impl");
+        args.add(dirImpl.getName());
+      }
+      args.add(path);
+
+      IndexUpgrader upgrader = null;
+      try {
+        upgrader = IndexUpgrader.parseArgs(args.toArray(new String[0]));
+      } catch (Exception e) {
+        throw new AssertionError("unable to parse args: " + args, e);
+      }
+      upgrader.upgrade();
+      
+      Directory upgradedDir = newFSDirectory(dir);
+      try {
+        checkAllSegmentsUpgraded(upgradedDir);
+      } finally {
+        upgradedDir.close();
+      }
+    }
+  }
+
   public void testUpgradeOldSingleSegmentIndexWithAdditions() throws Exception {
     for (String name : oldSingleSegmentNames) {
       if (VERBOSE) {
@@ -963,8 +1026,7 @@ public class TestBackwardsCompatibility 
       // determine count of segments in modified index
       final int origSegCount = getNumberOfSegments(dir);
       
-      new IndexUpgrader(dir, newIndexWriterConfig(TEST_VERSION_CURRENT, null), false)
-        .upgrade();
+      newIndexUpgrader(dir).upgrade();
 
       final int segCount = checkAllSegmentsUpgraded(dir);
       assertEquals("Index must still contain the same number of segments, as only one segment was upgraded and nothing else merged",

Modified: lucene/dev/branches/lucene4956/lucene/core/src/test/org/apache/lucene/index/TestCheckIndex.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene4956/lucene/core/src/test/org/apache/lucene/index/TestCheckIndex.java?rev=1534320&r1=1534319&r2=1534320&view=diff
==============================================================================
--- lucene/dev/branches/lucene4956/lucene/core/src/test/org/apache/lucene/index/TestCheckIndex.java (original)
+++ lucene/dev/branches/lucene4956/lucene/core/src/test/org/apache/lucene/index/TestCheckIndex.java Mon Oct 21 18:58:24 2013
@@ -117,8 +117,10 @@ public class TestCheckIndex extends Luce
 
   public void testLuceneConstantVersion() throws IOException {
     // common-build.xml sets lucene.version
-    final String version = System.getProperty("lucene.version");
+    String version = System.getProperty("lucene.version");
     assertNotNull( "null version", version);
+    // remove anything after a "-" from the version string:
+    version = version.replaceAll("-.*$", "");
     final String constantVersion;
     String parts[] = Constants.LUCENE_MAIN_VERSION.split("\\.");
     if (parts.length == 4) {
@@ -130,7 +132,6 @@ public class TestCheckIndex extends Luce
       constantVersion = Constants.LUCENE_MAIN_VERSION;
     }
     assertTrue("Invalid version: "+version,
-               version.equals(constantVersion+"-SNAPSHOT") ||
                version.equals(constantVersion));
     assertTrue(Constants.LUCENE_VERSION + " should start with: "+version,
                Constants.LUCENE_VERSION.startsWith(version));

Modified: lucene/dev/branches/lucene4956/lucene/core/src/test/org/apache/lucene/index/TestCodecs.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene4956/lucene/core/src/test/org/apache/lucene/index/TestCodecs.java?rev=1534320&r1=1534319&r2=1534320&view=diff
==============================================================================
--- lucene/dev/branches/lucene4956/lucene/core/src/test/org/apache/lucene/index/TestCodecs.java (original)
+++ lucene/dev/branches/lucene4956/lucene/core/src/test/org/apache/lucene/index/TestCodecs.java Mon Oct 21 18:58:24 2013
@@ -25,15 +25,15 @@ import java.util.Random;
 
 import org.apache.lucene.analysis.MockAnalyzer;
 import org.apache.lucene.codecs.Codec;
-import org.apache.lucene.codecs.FieldsConsumer;
 import org.apache.lucene.codecs.FieldsProducer;
-import org.apache.lucene.codecs.PostingsConsumer;
-import org.apache.lucene.codecs.TermStats;
-import org.apache.lucene.codecs.TermsConsumer;
+import org.apache.lucene.codecs.lucene40.Lucene40RWCodec;
+import org.apache.lucene.codecs.lucene41.Lucene41RWCodec;
+import org.apache.lucene.codecs.lucene42.Lucene42RWCodec;
 import org.apache.lucene.codecs.mocksep.MockSepPostingsFormat;
 import org.apache.lucene.document.Document;
 import org.apache.lucene.document.Field.Store;
 import org.apache.lucene.document.FieldType;
+import org.apache.lucene.document.NumericDocValuesField;
 import org.apache.lucene.document.StringField;
 import org.apache.lucene.document.TextField;
 import org.apache.lucene.index.FieldInfo.DocValuesType;
@@ -44,11 +44,11 @@ import org.apache.lucene.search.PhraseQu
 import org.apache.lucene.search.Query;
 import org.apache.lucene.search.ScoreDoc;
 import org.apache.lucene.store.Directory;
+import org.apache.lucene.util.Bits;
 import org.apache.lucene.util.BytesRef;
 import org.apache.lucene.util.Constants;
 import org.apache.lucene.util.InfoStream;
 import org.apache.lucene.util.LuceneTestCase;
-import org.apache.lucene.util.OpenBitSet;
 import org.apache.lucene.util._TestUtil;
 import org.junit.BeforeClass;
 
@@ -141,22 +141,6 @@ public class TestCodecs extends LuceneTe
     public int compareTo(final FieldData other) {
       return fieldInfo.name.compareTo(other.fieldInfo.name);
     }
-
-    public void write(final FieldsConsumer consumer) throws Throwable {
-      Arrays.sort(terms);
-      final TermsConsumer termsConsumer = consumer.addField(fieldInfo);
-      long sumTotalTermCount = 0;
-      long sumDF = 0;
-      OpenBitSet visitedDocs = new OpenBitSet();
-      for (final TermData term : terms) {
-        for (int i = 0; i < term.docs.length; i++) {
-          visitedDocs.set(term.docs[i]);
-        }
-        sumDF += term.docs.length;
-        sumTotalTermCount += term.write(termsConsumer);
-      }
-      termsConsumer.finish(omitTF ? -1 : sumTotalTermCount, sumDF, (int) visitedDocs.cardinality());
-    }
   }
 
   class PositionData {
@@ -187,30 +171,6 @@ public class TestCodecs extends LuceneTe
     public int compareTo(final TermData o) {
       return text.compareTo(o.text);
     }
-
-    public long write(final TermsConsumer termsConsumer) throws Throwable {
-      final PostingsConsumer postingsConsumer = termsConsumer.startTerm(text);
-      long totTF = 0;
-      for(int i=0;i<docs.length;i++) {
-        final int termDocFreq;
-        if (field.omitTF) {
-          termDocFreq = -1;
-        } else {
-          termDocFreq = positions[i].length;
-        }
-        postingsConsumer.startDoc(docs[i], termDocFreq);
-        if (!field.omitTF) {
-          totTF += positions[i].length;
-          for(int j=0;j<positions[i].length;j++) {
-            final PositionData pos = positions[i][j];
-            postingsConsumer.addPosition(pos.pos, pos.payload, -1, -1);
-          }
-        }
-        postingsConsumer.finishDoc();
-      }
-      termsConsumer.finishTerm(text, new TermStats(docs.length, field.omitTF ? -1 : totTF));
-      return totTF;
-    }
   }
 
   final private static String SEGMENT = "0";
@@ -294,7 +254,7 @@ public class TestCodecs extends LuceneTe
     final Directory dir = newDirectory();
     this.write(fieldInfos, dir, fields);
     Codec codec = Codec.getDefault();
-    final SegmentInfo si = new SegmentInfo(dir, Constants.LUCENE_MAIN_VERSION, SEGMENT, 10000, false, codec, null, null);
+    final SegmentInfo si = new SegmentInfo(dir, Constants.LUCENE_MAIN_VERSION, SEGMENT, 10000, false, codec, null);
 
     final FieldsProducer reader = codec.postingsFormat().fieldsProducer(new SegmentReadState(dir, si, fieldInfos, newIOContext(random())));
 
@@ -351,8 +311,7 @@ public class TestCodecs extends LuceneTe
 
     this.write(fieldInfos, dir, fields);
     Codec codec = Codec.getDefault();
-    final SegmentInfo si = new SegmentInfo(dir, Constants.LUCENE_MAIN_VERSION, SEGMENT, 10000,
-                                           false, codec, null, null);
+    final SegmentInfo si = new SegmentInfo(dir, Constants.LUCENE_MAIN_VERSION, SEGMENT, 10000, false, codec, null);
 
     if (VERBOSE) {
       System.out.println("TEST: now read postings");
@@ -584,18 +543,16 @@ public class TestCodecs extends LuceneTe
           term = field.terms[upto];
           if (random().nextInt(3) == 1) {
             final DocsEnum docs;
-            final DocsEnum docsAndFreqs;
             final DocsAndPositionsEnum postings;
             if (!field.omitTF) {
               postings = termsEnum.docsAndPositions(null, null);
               if (postings != null) {
-                docs = docsAndFreqs = postings;
+                docs = postings;
               } else {
-                docs = docsAndFreqs = _TestUtil.docs(random(), termsEnum, null, null, DocsEnum.FLAG_FREQS);
+                docs = _TestUtil.docs(random(), termsEnum, null, null, DocsEnum.FLAG_FREQS);
               }
             } else {
               postings = null;
-              docsAndFreqs = null;
               docs = _TestUtil.docs(random(), termsEnum, null, null, DocsEnum.FLAG_NONE);
             }
             assertNotNull(docs);
@@ -653,18 +610,255 @@ public class TestCodecs extends LuceneTe
     }
   }
 
+  private static class DataFields extends Fields {
+    private final FieldData[] fields;
+
+    public DataFields(FieldData[] fields) {
+      // already sorted:
+      this.fields = fields;
+    }
+
+    @Override
+    public Iterator<String> iterator() {
+      return new Iterator<String>() {
+        int upto = -1;
+
+        @Override
+        public boolean hasNext() {
+          return upto+1 < fields.length;
+        }
+
+        @Override
+        public String next() {
+          upto++;
+          return fields[upto].fieldInfo.name;
+        }
+
+        @Override
+        public void remove() {
+          throw new UnsupportedOperationException();
+        }
+      };
+    }
+
+    @Override
+    public Terms terms(String field) {
+      // Slow linear search:
+      for(FieldData fieldData : fields) {
+        if (fieldData.fieldInfo.name.equals(field)) {
+          return new DataTerms(fieldData);
+        }
+      }
+      return null;
+    }
+
+    @Override
+    public int size() {
+      return fields.length;
+    }
+  }
+
+  private static class DataTerms extends Terms {
+    final FieldData fieldData;
+
+    public DataTerms(FieldData fieldData) {
+      this.fieldData = fieldData;
+    }
+
+    @Override
+    public TermsEnum iterator(TermsEnum reuse) {
+      return new DataTermsEnum(fieldData);
+    }
+
+    @Override
+    public long size() {
+      throw new UnsupportedOperationException();
+    }
+
+    @Override
+    public long getSumTotalTermFreq() {
+      throw new UnsupportedOperationException();
+    }
+
+    @Override
+    public long getSumDocFreq() {
+      throw new UnsupportedOperationException();
+    }
+
+    @Override
+    public int getDocCount() {
+      throw new UnsupportedOperationException();
+    }
+
+    @Override
+    public boolean hasFreqs() {
+      return fieldData.fieldInfo.getIndexOptions().compareTo(IndexOptions.DOCS_AND_FREQS) >= 0;
+    }
+
+    @Override
+    public boolean hasOffsets() {
+      return fieldData.fieldInfo.getIndexOptions().compareTo(IndexOptions.DOCS_AND_FREQS_AND_POSITIONS_AND_OFFSETS) >= 0;
+    }
+
+    @Override
+    public boolean hasPositions() {
+      return fieldData.fieldInfo.getIndexOptions().compareTo(IndexOptions.DOCS_AND_FREQS_AND_POSITIONS) >= 0;
+    }
+
+    @Override
+    public boolean hasPayloads() {
+      return fieldData.fieldInfo.hasPayloads();
+    }
+  }
+
+  private static class DataTermsEnum extends TermsEnum {
+    final FieldData fieldData;
+    private int upto = -1;
+
+    public DataTermsEnum(FieldData fieldData) {
+      this.fieldData = fieldData;
+    }
+
+    @Override
+    public BytesRef next() {
+      upto++;
+      if (upto == fieldData.terms.length) {
+        return null;
+      }
+
+      return term();
+    }
+
+    @Override
+    public BytesRef term() {
+      return fieldData.terms[upto].text;
+    }
+
+    @Override
+    public SeekStatus seekCeil(BytesRef text) {
+      // Stupid linear impl:
+      for(int i=0;i<fieldData.terms.length;i++) {
+        int cmp = fieldData.terms[i].text.compareTo(text);
+        if (cmp == 0) {
+          upto = i;
+          return SeekStatus.FOUND;
+        } else if (cmp > 0) {
+          upto = i;
+          return SeekStatus.NOT_FOUND;
+        }
+      }
+
+      return SeekStatus.END;
+    }
+
+    @Override
+    public void seekExact(long ord) {
+      throw new UnsupportedOperationException();
+    }
+
+    @Override
+    public long ord() {
+      throw new UnsupportedOperationException();
+    }
+
+    @Override
+    public int docFreq() {
+      throw new UnsupportedOperationException();
+    }
+  
+    @Override
+    public long totalTermFreq() {
+      throw new UnsupportedOperationException();
+    }
+
+    @Override
+    public DocsEnum docs(Bits liveDocs, DocsEnum reuse, int flags) {
+      assert liveDocs == null;
+      return new DataDocsAndPositionsEnum(fieldData.terms[upto]);
+    }
+
+    @Override
+    public DocsAndPositionsEnum docsAndPositions(Bits liveDocs, DocsAndPositionsEnum reuse, int flags) {
+      assert liveDocs == null;
+      return new DataDocsAndPositionsEnum(fieldData.terms[upto]);
+    }
+  }
+
+  private static class DataDocsAndPositionsEnum extends DocsAndPositionsEnum {
+    final TermData termData;
+    int docUpto = -1;
+    int posUpto;
+
+    public DataDocsAndPositionsEnum(TermData termData) {
+      this.termData = termData;
+    }
+
+    @Override
+    public long cost() {
+      throw new UnsupportedOperationException();
+    }
+
+    @Override
+    public int nextDoc() {
+      docUpto++;
+      if (docUpto == termData.docs.length) {
+        return NO_MORE_DOCS;
+      }
+      posUpto = -1;
+      return docID();
+    }
+
+    @Override
+    public int docID() {
+      return termData.docs[docUpto];
+    }
+
+    @Override
+    public int advance(int target) {
+      // Slow linear impl:
+      nextDoc();
+      while (docID() < target) {
+        nextDoc();
+      }
+
+      return docID();
+    }
+
+    @Override
+    public int freq() {
+      return termData.positions[docUpto].length;
+    }
+
+    @Override
+    public int nextPosition() {
+      posUpto++;
+      return termData.positions[docUpto][posUpto].pos;
+    }
+
+    @Override
+    public BytesRef getPayload() {
+      return termData.positions[docUpto][posUpto].payload;
+    }
+    
+    @Override
+    public int startOffset() {
+      throw new UnsupportedOperationException();
+    }
+
+    @Override
+    public int endOffset() {
+      throw new UnsupportedOperationException();
+    }
+  }
+
   private void write(final FieldInfos fieldInfos, final Directory dir, final FieldData[] fields) throws Throwable {
 
     final Codec codec = Codec.getDefault();
-    final SegmentInfo si = new SegmentInfo(dir, Constants.LUCENE_MAIN_VERSION, SEGMENT, 10000, false, codec, null, null);
+    final SegmentInfo si = new SegmentInfo(dir, Constants.LUCENE_MAIN_VERSION, SEGMENT, 10000, false, codec, null);
     final SegmentWriteState state = new SegmentWriteState(InfoStream.getDefault(), dir, si, fieldInfos, null, newIOContext(random()));
 
-    final FieldsConsumer consumer = codec.postingsFormat().fieldsConsumer(state);
     Arrays.sort(fields);
-    for (final FieldData field : fields) {
-      field.write(consumer);
-    }
-    consumer.close();
+    codec.postingsFormat().fieldsConsumer(state).write(new DataFields(fields));
   }
   
   public void testDocsOnlyFreq() throws Exception {
@@ -695,4 +889,30 @@ public class TestCodecs extends LuceneTe
     
     dir.close();
   }
+  
+  public void testDisableImpersonation() throws Exception {
+    Codec[] oldCodecs = new Codec[] { new Lucene40RWCodec(), new Lucene41RWCodec(), new Lucene42RWCodec() };
+    Directory dir = newDirectory();
+    IndexWriterConfig conf = newIndexWriterConfig(TEST_VERSION_CURRENT, new MockAnalyzer(random()));
+    conf.setCodec(oldCodecs[random().nextInt(oldCodecs.length)]);
+    IndexWriter writer = new IndexWriter(dir, conf);
+    
+    Document doc = new Document();
+    doc.add(new StringField("f", "bar", Store.YES));
+    doc.add(new NumericDocValuesField("n", 18L));
+    writer.addDocument(doc);
+    
+    OLD_FORMAT_IMPERSONATION_IS_ACTIVE = false;
+    try {
+      writer.close();
+      fail("should not have succeeded to impersonate an old format!");
+    } catch (UnsupportedOperationException e) {
+      writer.rollback();
+    } finally {
+      OLD_FORMAT_IMPERSONATION_IS_ACTIVE = true;
+    }
+    
+    dir.close();
+  }
+  
 }

Modified: lucene/dev/branches/lucene4956/lucene/core/src/test/org/apache/lucene/index/TestConcurrentMergeScheduler.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene4956/lucene/core/src/test/org/apache/lucene/index/TestConcurrentMergeScheduler.java?rev=1534320&r1=1534319&r2=1534320&view=diff
==============================================================================
--- lucene/dev/branches/lucene4956/lucene/core/src/test/org/apache/lucene/index/TestConcurrentMergeScheduler.java (original)
+++ lucene/dev/branches/lucene4956/lucene/core/src/test/org/apache/lucene/index/TestConcurrentMergeScheduler.java Mon Oct 21 18:58:24 2013
@@ -215,6 +215,7 @@ public class TestConcurrentMergeSchedule
     IndexWriter writer = new IndexWriter(
         directory,
         newIndexWriterConfig(TEST_VERSION_CURRENT, new MockAnalyzer(random())).
+            // Force excessive merging:
             setMaxBufferedDocs(2).
             setMergePolicy(newLogMergePolicy(100))
     );
@@ -249,7 +250,9 @@ public class TestConcurrentMergeSchedule
           directory,
           newIndexWriterConfig(TEST_VERSION_CURRENT, new MockAnalyzer(random())).
               setOpenMode(OpenMode.APPEND).
-              setMergePolicy(newLogMergePolicy(100))
+              setMergePolicy(newLogMergePolicy(100)).
+              // Force excessive merging:
+              setMaxBufferedDocs(2)
       );
     }
     writer.close();