You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@lucene.apache.org by rj...@apache.org on 2015/03/31 07:22:50 UTC

svn commit: r1670257 [10/39] - in /lucene/dev/branches/lucene6271: ./ dev-tools/ dev-tools/idea/.idea/libraries/ dev-tools/scripts/ lucene/ lucene/analysis/ lucene/analysis/common/ lucene/analysis/common/src/java/org/apache/lucene/analysis/miscellaneou...

Modified: lucene/dev/branches/lucene6271/lucene/core/src/java/org/apache/lucene/store/DataOutput.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene6271/lucene/core/src/java/org/apache/lucene/store/DataOutput.java?rev=1670257&r1=1670256&r2=1670257&view=diff
==============================================================================
--- lucene/dev/branches/lucene6271/lucene/core/src/java/org/apache/lucene/store/DataOutput.java (original)
+++ lucene/dev/branches/lucene6271/lucene/core/src/java/org/apache/lucene/store/DataOutput.java Tue Mar 31 05:22:40 2015
@@ -286,7 +286,9 @@ public abstract class DataOutput {
    * {@link #writeString(String) String}s.
    * 
    * @param map Input map. May be null (equivalent to an empty map)
+   * @deprecated Use {@link #writeMapOfStrings(Map)} instead.
    */
+  @Deprecated
   public void writeStringStringMap(Map<String,String> map) throws IOException {
     if (map == null) {
       writeInt(0);
@@ -298,6 +300,24 @@ public abstract class DataOutput {
       }
     }
   }
+  
+  /**
+   * Writes a String map.
+   * <p>
+   * First the size is written as an {@link #writeVInt(int) vInt},
+   * followed by each key-value pair written as two consecutive 
+   * {@link #writeString(String) String}s.
+   * 
+   * @param map Input map.
+   * @throws NullPointerException if {@code map} is null.
+   */
+  public void writeMapOfStrings(Map<String,String> map) throws IOException {
+    writeVInt(map.size());
+    for (Map.Entry<String, String> entry : map.entrySet()) {
+      writeString(entry.getKey());
+      writeString(entry.getValue());
+    }
+  }
 
   /**
    * Writes a String set.
@@ -307,7 +327,9 @@ public abstract class DataOutput {
    * {@link #writeString(String) String}.
    * 
    * @param set Input set. May be null (equivalent to an empty set)
+   * @deprecated Use {@link #writeMapOfStrings(Map)} instead.
    */
+  @Deprecated
   public void writeStringSet(Set<String> set) throws IOException {
     if (set == null) {
       writeInt(0);
@@ -318,4 +340,21 @@ public abstract class DataOutput {
       }
     }
   }
+  
+  /**
+   * Writes a String set.
+   * <p>
+   * First the size is written as an {@link #writeVInt(int) vInt},
+   * followed by each value written as a
+   * {@link #writeString(String) String}.
+   * 
+   * @param set Input set.
+   * @throws NullPointerException if {@code set} is null.
+   */
+  public void writeSetOfStrings(Set<String> set) throws IOException {
+    writeVInt(set.size());
+    for (String value : set) {
+      writeString(value);
+    }
+  }
 }

Modified: lucene/dev/branches/lucene6271/lucene/core/src/java/org/apache/lucene/store/IndexInput.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene6271/lucene/core/src/java/org/apache/lucene/store/IndexInput.java?rev=1670257&r1=1670256&r2=1670257&view=diff
==============================================================================
--- lucene/dev/branches/lucene6271/lucene/core/src/java/org/apache/lucene/store/IndexInput.java (original)
+++ lucene/dev/branches/lucene6271/lucene/core/src/java/org/apache/lucene/store/IndexInput.java Tue Mar 31 05:22:40 2015
@@ -88,7 +88,17 @@ public abstract class IndexInput extends
    * The slice is seeked to the beginning.
    */
   public abstract IndexInput slice(String sliceDescription, long offset, long length) throws IOException;
-  
+
+  /** Subclasses call this to get the String for resourceDescription of a slice of this {@code IndexInput}. */
+  protected String getFullSliceDescription(String sliceDescription) {
+    if (sliceDescription == null) {
+      // Clones pass null sliceDescription:
+      return toString();
+    } else {
+      return toString() + " [slice=" + sliceDescription + "]";
+    }
+  }
+
   /**
    * Creates a random-access slice of this index input, with the given offset and length. 
    * <p>

Modified: lucene/dev/branches/lucene6271/lucene/core/src/java/org/apache/lucene/store/MMapDirectory.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene6271/lucene/core/src/java/org/apache/lucene/store/MMapDirectory.java?rev=1670257&r1=1670256&r2=1670257&view=diff
==============================================================================
--- lucene/dev/branches/lucene6271/lucene/core/src/java/org/apache/lucene/store/MMapDirectory.java (original)
+++ lucene/dev/branches/lucene6271/lucene/core/src/java/org/apache/lucene/store/MMapDirectory.java Tue Mar 31 05:22:40 2015
@@ -25,6 +25,7 @@ import java.nio.channels.FileChannel.Map
 import java.nio.file.Path;
 import java.nio.file.StandardOpenOption;
 import java.security.AccessController;
+import java.security.PrivilegedAction;
 import java.security.PrivilegedExceptionAction;
 import java.security.PrivilegedActionException;
 import java.util.Locale;
@@ -158,18 +159,16 @@ public class MMapDirectory extends FSDir
   /**
    * <code>true</code>, if this platform supports unmapping mmapped files.
    */
-  public static final boolean UNMAP_SUPPORTED;
-  static {
-    boolean v;
+  public static final boolean UNMAP_SUPPORTED =
+      AccessController.doPrivileged((PrivilegedAction<Boolean>) MMapDirectory::checkUnmapSupported);
+  
+  private static boolean checkUnmapSupported() {
     try {
-      Class.forName("sun.misc.Cleaner");
-      Class.forName("java.nio.DirectByteBuffer")
-        .getMethod("cleaner");
-      v = true;
+      Class.forName("java.nio.DirectByteBuffer").getMethod("cleaner");
+      return true;
     } catch (Exception e) {
-      v = false;
+      return false;
     }
-    UNMAP_SUPPORTED = v;
   }
   
   /**
@@ -280,27 +279,19 @@ public class MMapDirectory extends FSDir
     return newIoe;
   }
   
-  private static final BufferCleaner CLEANER = new BufferCleaner() {
-    @Override
-    public void freeBuffer(final ByteBufferIndexInput parent, final ByteBuffer buffer) throws IOException {
-      try {
-        AccessController.doPrivileged(new PrivilegedExceptionAction<Void>() {
-          @Override
-          public Void run() throws Exception {
-            final Method getCleanerMethod = buffer.getClass()
-              .getMethod("cleaner");
-            getCleanerMethod.setAccessible(true);
-            final Object cleaner = getCleanerMethod.invoke(buffer);
-            if (cleaner != null) {
-              cleaner.getClass().getMethod("clean")
-                .invoke(cleaner);
-            }
-            return null;
-          }
-        });
-      } catch (PrivilegedActionException e) {
-        throw new IOException("Unable to unmap the mapped buffer: " + parent.toString(), e.getCause());
-      }
+  private static final BufferCleaner CLEANER = (ByteBufferIndexInput parent, ByteBuffer buffer) -> {
+    try {
+      AccessController.doPrivileged((PrivilegedExceptionAction<Void>) () -> {
+        final Method getCleanerMethod = buffer.getClass().getMethod("cleaner");
+        getCleanerMethod.setAccessible(true);
+        final Object cleaner = getCleanerMethod.invoke(buffer);
+        if (cleaner != null) {
+          cleaner.getClass().getMethod("clean").invoke(cleaner);
+        }
+        return null;
+      });
+    } catch (PrivilegedActionException e) {
+      throw new IOException("Unable to unmap the mapped buffer: " + parent.toString(), e.getCause());
     }
   };
 }

Modified: lucene/dev/branches/lucene6271/lucene/core/src/java/org/apache/lucene/store/MergeInfo.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene6271/lucene/core/src/java/org/apache/lucene/store/MergeInfo.java?rev=1670257&r1=1670256&r2=1670257&view=diff
==============================================================================
--- lucene/dev/branches/lucene6271/lucene/core/src/java/org/apache/lucene/store/MergeInfo.java (original)
+++ lucene/dev/branches/lucene6271/lucene/core/src/java/org/apache/lucene/store/MergeInfo.java Tue Mar 31 05:22:40 2015
@@ -23,7 +23,7 @@ package org.apache.lucene.store;
 
 public class MergeInfo {
   
-  public final int totalDocCount;
+  public final int totalMaxDoc;
   
   public final long estimatedMergeBytes;
   
@@ -40,8 +40,8 @@ public class MergeInfo {
    * 
    */
 
-  public MergeInfo(int totalDocCount, long estimatedMergeBytes, boolean isExternal, int mergeMaxNumSegments) {
-    this.totalDocCount = totalDocCount;
+  public MergeInfo(int totalMaxDoc, long estimatedMergeBytes, boolean isExternal, int mergeMaxNumSegments) {
+    this.totalMaxDoc = totalMaxDoc;
     this.estimatedMergeBytes = estimatedMergeBytes;
     this.isExternal = isExternal;
     this.mergeMaxNumSegments = mergeMaxNumSegments;
@@ -56,7 +56,7 @@ public class MergeInfo {
         + (int) (estimatedMergeBytes ^ (estimatedMergeBytes >>> 32));
     result = prime * result + (isExternal ? 1231 : 1237);
     result = prime * result + mergeMaxNumSegments;
-    result = prime * result + totalDocCount;
+    result = prime * result + totalMaxDoc;
     return result;
   }
 
@@ -75,14 +75,14 @@ public class MergeInfo {
       return false;
     if (mergeMaxNumSegments != other.mergeMaxNumSegments)
       return false;
-    if (totalDocCount != other.totalDocCount)
+    if (totalMaxDoc != other.totalMaxDoc)
       return false;
     return true;
   }
 
   @Override
   public String toString() {
-    return "MergeInfo [totalDocCount=" + totalDocCount
+    return "MergeInfo [totalMaxDoc=" + totalMaxDoc
         + ", estimatedMergeBytes=" + estimatedMergeBytes + ", isExternal="
         + isExternal + ", mergeMaxNumSegments=" + mergeMaxNumSegments + "]";
   }

Modified: lucene/dev/branches/lucene6271/lucene/core/src/java/org/apache/lucene/store/NIOFSDirectory.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene6271/lucene/core/src/java/org/apache/lucene/store/NIOFSDirectory.java?rev=1670257&r1=1670256&r2=1670257&view=diff
==============================================================================
--- lucene/dev/branches/lucene6271/lucene/core/src/java/org/apache/lucene/store/NIOFSDirectory.java (original)
+++ lucene/dev/branches/lucene6271/lucene/core/src/java/org/apache/lucene/store/NIOFSDirectory.java Tue Mar 31 05:22:40 2015
@@ -137,7 +137,7 @@ public class NIOFSDirectory extends FSDi
       if (offset < 0 || length < 0 || offset + length > this.length()) {
         throw new IllegalArgumentException("slice() " + sliceDescription + " out of bounds: "  + this);
       }
-      return new NIOFSIndexInput(sliceDescription, channel, off + offset, length, getBufferSize());
+      return new NIOFSIndexInput(getFullSliceDescription(sliceDescription), channel, off + offset, length, getBufferSize());
     }
 
     @Override

Modified: lucene/dev/branches/lucene6271/lucene/core/src/java/org/apache/lucene/store/RAMInputStream.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene6271/lucene/core/src/java/org/apache/lucene/store/RAMInputStream.java?rev=1670257&r1=1670256&r2=1670257&view=diff
==============================================================================
--- lucene/dev/branches/lucene6271/lucene/core/src/java/org/apache/lucene/store/RAMInputStream.java (original)
+++ lucene/dev/branches/lucene6271/lucene/core/src/java/org/apache/lucene/store/RAMInputStream.java Tue Mar 31 05:22:40 2015
@@ -128,8 +128,7 @@ public class RAMInputStream extends Inde
     if (offset < 0 || length < 0 || offset + length > this.length) {
       throw new IllegalArgumentException("slice() " + sliceDescription + " out of bounds: "  + this);
     }
-    final String newResourceDescription = (sliceDescription == null) ? toString() : (toString() + " [slice=" + sliceDescription + "]");
-    return new RAMInputStream(newResourceDescription, file, offset + length) {
+    return new RAMInputStream(getFullSliceDescription(sliceDescription), file, offset + length) {
       {
         seek(0L);
       }

Modified: lucene/dev/branches/lucene6271/lucene/core/src/java/org/apache/lucene/store/SimpleFSDirectory.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene6271/lucene/core/src/java/org/apache/lucene/store/SimpleFSDirectory.java?rev=1670257&r1=1670256&r2=1670257&view=diff
==============================================================================
--- lucene/dev/branches/lucene6271/lucene/core/src/java/org/apache/lucene/store/SimpleFSDirectory.java (original)
+++ lucene/dev/branches/lucene6271/lucene/core/src/java/org/apache/lucene/store/SimpleFSDirectory.java Tue Mar 31 05:22:40 2015
@@ -131,7 +131,7 @@ public class SimpleFSDirectory extends F
       if (offset < 0 || length < 0 || offset + length > this.length()) {
         throw new IllegalArgumentException("slice() " + sliceDescription + " out of bounds: "  + this);
       }
-      return new SimpleFSIndexInput(sliceDescription, channel, off + offset, length, getBufferSize());
+      return new SimpleFSIndexInput(getFullSliceDescription(sliceDescription), channel, off + offset, length, getBufferSize());
     }
 
     @Override

Modified: lucene/dev/branches/lucene6271/lucene/core/src/java/org/apache/lucene/util/BitDocIdSet.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene6271/lucene/core/src/java/org/apache/lucene/util/BitDocIdSet.java?rev=1670257&r1=1670256&r2=1670257&view=diff
==============================================================================
--- lucene/dev/branches/lucene6271/lucene/core/src/java/org/apache/lucene/util/BitDocIdSet.java (original)
+++ lucene/dev/branches/lucene6271/lucene/core/src/java/org/apache/lucene/util/BitDocIdSet.java Tue Mar 31 05:22:40 2015
@@ -61,12 +61,6 @@ public class BitDocIdSet extends DocIdSe
     return set;
   }
 
-  /** This DocIdSet implementation is cacheable. */
-  @Override
-  public boolean isCacheable() {
-    return true;
-  }
-
   @Override
   public long ramBytesUsed() {
     return BASE_RAM_BYTES_USED + set.ramBytesUsed();

Modified: lucene/dev/branches/lucene6271/lucene/core/src/java/org/apache/lucene/util/NotDocIdSet.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene6271/lucene/core/src/java/org/apache/lucene/util/NotDocIdSet.java?rev=1670257&r1=1670256&r2=1670257&view=diff
==============================================================================
--- lucene/dev/branches/lucene6271/lucene/core/src/java/org/apache/lucene/util/NotDocIdSet.java (original)
+++ lucene/dev/branches/lucene6271/lucene/core/src/java/org/apache/lucene/util/NotDocIdSet.java Tue Mar 31 05:22:40 2015
@@ -42,11 +42,6 @@ public final class NotDocIdSet extends D
   }
 
   @Override
-  public boolean isCacheable() {
-    return in.isCacheable();
-  }
-
-  @Override
   public Bits bits() throws IOException {
     final Bits inBits = in.bits();
     if (inBits == null) {

Modified: lucene/dev/branches/lucene6271/lucene/core/src/java/org/apache/lucene/util/NumericUtils.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene6271/lucene/core/src/java/org/apache/lucene/util/NumericUtils.java?rev=1670257&r1=1670256&r2=1670257&view=diff
==============================================================================
--- lucene/dev/branches/lucene6271/lucene/core/src/java/org/apache/lucene/util/NumericUtils.java (original)
+++ lucene/dev/branches/lucene6271/lucene/core/src/java/org/apache/lucene/util/NumericUtils.java Tue Mar 31 05:22:40 2015
@@ -28,7 +28,6 @@ import org.apache.lucene.index.FilterLea
 import org.apache.lucene.index.FilteredTermsEnum;
 import org.apache.lucene.index.Terms;
 import org.apache.lucene.index.TermsEnum;
-import org.apache.lucene.search.NumericRangeFilter;
 import org.apache.lucene.search.NumericRangeQuery; // for javadocs
 
 /**
@@ -57,7 +56,7 @@ import org.apache.lucene.search.NumericR
  * <p>For easy usage, the trie algorithm is implemented for indexing inside
  * {@link NumericTokenStream} that can index <code>int</code>, <code>long</code>,
  * <code>float</code>, and <code>double</code>. For querying,
- * {@link NumericRangeQuery} and {@link NumericRangeFilter} implement the query part
+ * {@link NumericRangeQuery} implements the query part
  * for the same data types.
  *
  * <p>This class can also be used, to generate lexicographically sortable (according to
@@ -74,7 +73,7 @@ public final class NumericUtils {
   /**
    * The default precision step used by {@link LongField},
    * {@link DoubleField}, {@link NumericTokenStream}, {@link
-   * NumericRangeQuery}, and {@link NumericRangeFilter}.
+   * NumericRangeQuery}.
    */
   public static final int PRECISION_STEP_DEFAULT = 16;
   

Modified: lucene/dev/branches/lucene6271/lucene/core/src/java/org/apache/lucene/util/RoaringDocIdSet.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene6271/lucene/core/src/java/org/apache/lucene/util/RoaringDocIdSet.java?rev=1670257&r1=1670256&r2=1670257&view=diff
==============================================================================
--- lucene/dev/branches/lucene6271/lucene/core/src/java/org/apache/lucene/util/RoaringDocIdSet.java (original)
+++ lucene/dev/branches/lucene6271/lucene/core/src/java/org/apache/lucene/util/RoaringDocIdSet.java Tue Mar 31 05:22:40 2015
@@ -243,11 +243,6 @@ public class RoaringDocIdSet extends Doc
   }
 
   @Override
-  public boolean isCacheable() {
-    return true;
-  }
-
-  @Override
   public long ramBytesUsed() {
     return ramBytesUsed;
   }

Modified: lucene/dev/branches/lucene6271/lucene/core/src/java/org/apache/lucene/util/StringHelper.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene6271/lucene/core/src/java/org/apache/lucene/util/StringHelper.java?rev=1670257&r1=1670256&r2=1670257&view=diff
==============================================================================
--- lucene/dev/branches/lucene6271/lucene/core/src/java/org/apache/lucene/util/StringHelper.java (original)
+++ lucene/dev/branches/lucene6271/lucene/core/src/java/org/apache/lucene/util/StringHelper.java Tue Mar 31 05:22:40 2015
@@ -22,6 +22,7 @@ import java.math.BigInteger;
 import java.nio.file.Files;
 import java.nio.file.Paths;
 import java.util.Arrays;
+import java.util.Locale;
 import java.util.Properties;
 
 /**
@@ -33,11 +34,11 @@ public abstract class StringHelper {
 
   /**
    * Compares two {@link BytesRef}, element by element, and returns the
-   * number of elements common to both arrays.
+   * number of elements common to both arrays (from the start of each).
    *
    * @param left The first {@link BytesRef} to compare
    * @param right The second {@link BytesRef} to compare
-   * @return The number of common elements.
+   * @return The number of common elements (from the start of each).
    */
   public static int bytesDifference(BytesRef left, BytesRef right) {
     int len = left.length < right.length ? left.length : right.length;
@@ -134,6 +135,34 @@ public abstract class StringHelper {
     return sliceEquals(ref, suffix, ref.length - suffix.length);
   }
   
+  /**
+   * Returns <code>true</code> iff the ref contains the given slice. Otherwise
+   * <code>false</code>.
+   * 
+   * @param ref
+   *          the {@link BytesRef} to test
+   * @param slice
+   *          the slice to look for
+   * @param ignoreCase
+   *          whether the comparison should be case-insensitive
+   * @return Returns <code>true</code> iff the ref contains the given slice.
+   *         Otherwise <code>false</code>.
+   */
+  public static boolean contains(BytesRef ref, BytesRef slice, boolean ignoreCase) {
+    if (ignoreCase) {
+      String s1 = ref.utf8ToString();
+      String s2 = slice.utf8ToString();
+      return s1.toLowerCase(Locale.ENGLISH).contains(s2.toLowerCase(Locale.ENGLISH));
+    } else {
+      for (int pos = 0; pos <= ref.length - slice.length; ++pos) {
+        if (sliceEquals(ref, slice, pos)) {
+          return true;
+        }
+      }
+    }
+    return false;
+  }
+  
   private static boolean sliceEquals(BytesRef sliceToTest, BytesRef other, int pos) {
     if (pos < 0 || sliceToTest.length - pos < other.length) {
       return false;
@@ -364,4 +393,20 @@ public abstract class StringHelper {
       return sb.toString();
     }
   }
+  
+  /** Just converts each int in the incoming {@link IntsRef} to each byte
+   *  in the returned {@link BytesRef}, throwing {@code IllegalArgumentException}
+   *  if any int value is out of bounds for a byte. */
+  public static BytesRef intsRefToBytesRef(IntsRef ints) {
+    byte[] bytes = new byte[ints.length];
+    for(int i=0;i<ints.length;i++) {
+      int x = ints.ints[ints.offset+i];
+      if (x < 0 || x > 255) {
+        throw new IllegalArgumentException("int at pos=" + i + " with value=" + x + " is out-of-bounds for byte");
+      }
+      bytes[i] = (byte) x;
+    }
+
+    return new BytesRef(bytes);
+  }
 }

Modified: lucene/dev/branches/lucene6271/lucene/core/src/java/org/apache/lucene/util/TimSorter.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene6271/lucene/core/src/java/org/apache/lucene/util/TimSorter.java?rev=1670257&r1=1670256&r2=1670257&view=diff
==============================================================================
--- lucene/dev/branches/lucene6271/lucene/core/src/java/org/apache/lucene/util/TimSorter.java (original)
+++ lucene/dev/branches/lucene6271/lucene/core/src/java/org/apache/lucene/util/TimSorter.java Tue Mar 31 05:22:40 2015
@@ -40,7 +40,7 @@ public abstract class TimSorter extends
 
   static final int MINRUN = 32;
   static final int THRESHOLD = 64;
-  static final int STACKSIZE = 40; // depends on MINRUN
+  static final int STACKSIZE = 49; // depends on MINRUN
   static final int MIN_GALLOP = 7;
 
   final int maxTempSlots;

Modified: lucene/dev/branches/lucene6271/lucene/core/src/java/org/apache/lucene/util/Version.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene6271/lucene/core/src/java/org/apache/lucene/util/Version.java?rev=1670257&r1=1670256&r2=1670257&view=diff
==============================================================================
--- lucene/dev/branches/lucene6271/lucene/core/src/java/org/apache/lucene/util/Version.java (original)
+++ lucene/dev/branches/lucene6271/lucene/core/src/java/org/apache/lucene/util/Version.java Tue Mar 31 05:22:40 2015
@@ -102,7 +102,9 @@ public final class Version {
     try {
       major = Integer.parseInt(token);
     } catch (NumberFormatException nfe) {
-      throw new ParseException("Failed to parse major version from \"" + token + "\" (got: " + version + ")", 0);
+      ParseException p = new ParseException("Failed to parse major version from \"" + token + "\" (got: " + version + ")", 0);
+      p.initCause(nfe);
+      throw p;
     }
 
     if (tokens.hasMoreTokens() == false) {
@@ -114,7 +116,9 @@ public final class Version {
     try {
       minor = Integer.parseInt(token);
     } catch (NumberFormatException nfe) {
-      throw new ParseException("Failed to parse minor version from \"" + token + "\" (got: " + version + ")", 0);
+      ParseException p = new ParseException("Failed to parse minor version from \"" + token + "\" (got: " + version + ")", 0);
+      p.initCause(nfe);
+      throw p;
     }
 
     int bugfix = 0;
@@ -125,7 +129,9 @@ public final class Version {
       try {
         bugfix = Integer.parseInt(token);
       } catch (NumberFormatException nfe) {
-        throw new ParseException("Failed to parse bugfix version from \"" + token + "\" (got: " + version + ")", 0);
+        ParseException p = new ParseException("Failed to parse bugfix version from \"" + token + "\" (got: " + version + ")", 0);
+        p.initCause(nfe);
+        throw p;
       }
 
       if (tokens.hasMoreTokens()) {
@@ -133,7 +139,9 @@ public final class Version {
         try {
           prerelease = Integer.parseInt(token);
         } catch (NumberFormatException nfe) {
-          throw new ParseException("Failed to parse prerelease version from \"" + token + "\" (got: " + version + ")", 0);
+          ParseException p = new ParseException("Failed to parse prerelease version from \"" + token + "\" (got: " + version + ")", 0);
+          p.initCause(nfe);
+          throw p;
         }
         if (prerelease == 0) {
           throw new ParseException("Invalid value " + prerelease + " for prerelease; should be 1 or 2 (got: " + version + ")", 0);

Modified: lucene/dev/branches/lucene6271/lucene/core/src/java/org/apache/lucene/util/automaton/Automaton.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene6271/lucene/core/src/java/org/apache/lucene/util/automaton/Automaton.java?rev=1670257&r1=1670256&r2=1670257&view=diff
==============================================================================
--- lucene/dev/branches/lucene6271/lucene/core/src/java/org/apache/lucene/util/automaton/Automaton.java (original)
+++ lucene/dev/branches/lucene6271/lucene/core/src/java/org/apache/lucene/util/automaton/Automaton.java Tue Mar 31 05:22:40 2015
@@ -347,6 +347,7 @@ public class Automaton implements Accoun
   
   /** How many transitions this state has. */
   public int getNumTransitions(int state) {
+    assert state >= 0;
     int count = states[2*state+1];
     if (count == -1) {
       return 0;

Modified: lucene/dev/branches/lucene6271/lucene/core/src/java/org/apache/lucene/util/automaton/CompiledAutomaton.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene6271/lucene/core/src/java/org/apache/lucene/util/automaton/CompiledAutomaton.java?rev=1670257&r1=1670256&r2=1670257&view=diff
==============================================================================
--- lucene/dev/branches/lucene6271/lucene/core/src/java/org/apache/lucene/util/automaton/CompiledAutomaton.java (original)
+++ lucene/dev/branches/lucene6271/lucene/core/src/java/org/apache/lucene/util/automaton/CompiledAutomaton.java Tue Mar 31 05:22:40 2015
@@ -24,9 +24,11 @@ import java.util.List;
 import org.apache.lucene.index.SingleTermsEnum;
 import org.apache.lucene.index.Terms;
 import org.apache.lucene.index.TermsEnum;
-import org.apache.lucene.search.PrefixTermsEnum;
 import org.apache.lucene.util.BytesRef;
 import org.apache.lucene.util.BytesRefBuilder;
+import org.apache.lucene.util.IntsRef;
+import org.apache.lucene.util.StringHelper;
+import org.apache.lucene.util.UnicodeUtil;
 
 /**
  * Immutable class holding compiled details for a given
@@ -47,8 +49,6 @@ public class CompiledAutomaton {
     ALL, 
     /** Automaton that accepts only a single fixed string. */
     SINGLE, 
-    /** Automaton that matches all Strings with a constant prefix. */
-    PREFIX, 
     /** Catch-all for any other automata. */
     NORMAL
   };
@@ -57,8 +57,7 @@ public class CompiledAutomaton {
   public final AUTOMATON_TYPE type;
 
   /** 
-   * For {@link AUTOMATON_TYPE#PREFIX}, this is the prefix term; 
-   * for {@link AUTOMATON_TYPE#SINGLE} this is the singleton term.
+   * For {@link AUTOMATON_TYPE#SINGLE} this is the singleton term.
    */
   public final BytesRef term;
 
@@ -79,7 +78,8 @@ public class CompiledAutomaton {
   /**
    * Shared common suffix accepted by the automaton. Only valid
    * for {@link AUTOMATON_TYPE#NORMAL}, and only when the
-   * automaton accepts an infinite language.
+   * automaton accepts an infinite language.  This will be null
+   * if the common prefix is length 0.
    */
   public final BytesRef commonSuffixRef;
 
@@ -101,7 +101,7 @@ public class CompiledAutomaton {
    *  possibly expensive operations to determine if the automaton is one
    *  the cases in {@link CompiledAutomaton.AUTOMATON_TYPE}. */
   public CompiledAutomaton(Automaton automaton, Boolean finite, boolean simplify) {
-    this(automaton, finite, simplify, Operations.DEFAULT_MAX_DETERMINIZED_STATES);
+    this(automaton, finite, simplify, Operations.DEFAULT_MAX_DETERMINIZED_STATES, false);
   }
 
 
@@ -114,7 +114,7 @@ public class CompiledAutomaton {
    *  TooComplexToDeterminizeException.
    */
   public CompiledAutomaton(Automaton automaton, Boolean finite, boolean simplify,
-      int maxDeterminizedStates) {
+                           int maxDeterminizedStates, boolean isBinary) {
     if (automaton.getNumStates() == 0) {
       automaton = new Automaton();
       automaton.createState();
@@ -135,8 +135,18 @@ public class CompiledAutomaton {
         this.automaton = null;
         this.finite = null;
         return;
+      }
+
+      boolean isTotal;
+
       // NOTE: only approximate, because automaton may not be minimal:
-      } else if (Operations.isTotal(automaton)) {
+      if (isBinary) {
+        isTotal = Operations.isTotal(automaton, 0, 0xff);
+      } else {
+        isTotal = Operations.isTotal(automaton);
+      }
+
+      if (isTotal) {
         // matches all possible strings
         type = AUTOMATON_TYPE.ALL;
         term = null;
@@ -145,43 +155,27 @@ public class CompiledAutomaton {
         this.automaton = null;
         this.finite = null;
         return;
-      } else {
+      }
 
-        automaton = Operations.determinize(automaton, maxDeterminizedStates);
+      automaton = Operations.determinize(automaton, maxDeterminizedStates);
 
-        final String commonPrefix = Operations.getCommonPrefix(automaton);
-        final String singleton;
+      IntsRef singleton = Operations.getSingleton(automaton);
 
-        if (commonPrefix.length() > 0 && Operations.sameLanguage(automaton, Automata.makeString(commonPrefix))) {
-          singleton = commonPrefix;
+      if (singleton != null) {
+        // matches a fixed string
+        type = AUTOMATON_TYPE.SINGLE;
+        commonSuffixRef = null;
+        runAutomaton = null;
+        this.automaton = null;
+        this.finite = null;
+
+        if (isBinary) {
+          term = StringHelper.intsRefToBytesRef(singleton);
         } else {
-          singleton = null;
+          term = new BytesRef(UnicodeUtil.newString(singleton.ints, singleton.offset, singleton.length));
         }
 
-        if (singleton != null) {
-          // matches a fixed string
-          type = AUTOMATON_TYPE.SINGLE;
-          term = new BytesRef(singleton);
-          commonSuffixRef = null;
-          runAutomaton = null;
-          this.automaton = null;
-          this.finite = null;
-          return;
-        } else if (commonPrefix.length() > 0) {
-          Automaton other = Operations.concatenate(Automata.makeString(commonPrefix), Automata.makeAnyString());
-          other = Operations.determinize(other, maxDeterminizedStates);
-          assert Operations.hasDeadStates(other) == false;
-          if (Operations.sameLanguage(automaton, other)) {
-            // matches a constant prefix
-            type = AUTOMATON_TYPE.PREFIX;
-            term = new BytesRef(commonPrefix);
-            commonSuffixRef = null;
-            runAutomaton = null;
-            this.automaton = null;
-            this.finite = null;
-            return;
-          }
-        }
+        return;
       }
     }
 
@@ -194,14 +188,31 @@ public class CompiledAutomaton {
       this.finite = finite;
     }
 
-    Automaton utf8 = new UTF32ToUTF8().convert(automaton);
+    Automaton binary;
+    if (isBinary) {
+      // Caller already built binary automaton themselves, e.g. PrefixQuery
+      // does this since it can be provided with a binary (not necessarily
+      // UTF8!) term:
+      binary = automaton;
+    } else {
+      // Incoming automaton is unicode, and we must convert to UTF8 to match what's in the index:
+      binary = new UTF32ToUTF8().convert(automaton);
+    }
+
     if (this.finite) {
       commonSuffixRef = null;
     } else {
       // NOTE: this is a very costly operation!  We should test if it's really warranted in practice...
-      commonSuffixRef = Operations.getCommonSuffixBytesRef(utf8, maxDeterminizedStates);
+      BytesRef suffix = Operations.getCommonSuffixBytesRef(binary, maxDeterminizedStates);
+      if (suffix.length == 0) {
+        commonSuffixRef = null;
+      } else {
+        commonSuffixRef = suffix;
+      }
     }
-    runAutomaton = new ByteRunAutomaton(utf8, true, maxDeterminizedStates);
+
+    // This will determinize the binary automaton for us:
+    runAutomaton = new ByteRunAutomaton(binary, true, maxDeterminizedStates);
 
     this.automaton = runAutomaton.automaton;
   }
@@ -285,10 +296,6 @@ public class CompiledAutomaton {
       return terms.iterator(null);
     case SINGLE:
       return new SingleTermsEnum(terms.iterator(null), term);
-    case PREFIX:
-      // TODO: this is very likely faster than .intersect,
-      // but we should test and maybe cutover
-      return new PrefixTermsEnum(terms.iterator(null), term);
     case NORMAL:
       return terms.intersect(this, null);
     default:
@@ -410,7 +417,7 @@ public class CompiledAutomaton {
     if (getClass() != obj.getClass()) return false;
     CompiledAutomaton other = (CompiledAutomaton) obj;
     if (type != other.type) return false;
-    if (type == AUTOMATON_TYPE.SINGLE || type == AUTOMATON_TYPE.PREFIX) {
+    if (type == AUTOMATON_TYPE.SINGLE) {
       if (!term.equals(other.term)) return false;
     } else if (type == AUTOMATON_TYPE.NORMAL) {
       if (!runAutomaton.equals(other.runAutomaton)) return false;

Modified: lucene/dev/branches/lucene6271/lucene/core/src/java/org/apache/lucene/util/automaton/Operations.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene6271/lucene/core/src/java/org/apache/lucene/util/automaton/Operations.java?rev=1670257&r1=1670256&r2=1670257&view=diff
==============================================================================
--- lucene/dev/branches/lucene6271/lucene/core/src/java/org/apache/lucene/util/automaton/Operations.java (original)
+++ lucene/dev/branches/lucene6271/lucene/core/src/java/org/apache/lucene/util/automaton/Operations.java Tue Mar 31 05:22:40 2015
@@ -834,11 +834,20 @@ final public class Operations {
    * Returns true if the given automaton accepts all strings.  The automaton must be minimized.
    */
   public static boolean isTotal(Automaton a) {
+    return isTotal(a, Character.MIN_CODE_POINT, Character.MAX_CODE_POINT);
+  }
+
+  /**
+   * Returns true if the given automaton accepts all strings for the specified min/max
+   * range of the alphabet.  The automaton must be minimized.
+   */
+  public static boolean isTotal(Automaton a, int minAlphabet, int maxAlphabet) {
     if (a.isAccept(0) && a.getNumTransitions(0) == 1) {
       Transition t = new Transition();
       a.getTransition(0, 0, t);
-      return t.dest == 0 && t.min == Character.MIN_CODE_POINT
-          && t.max == Character.MAX_CODE_POINT;
+      return t.dest == 0
+        && t.min == minAlphabet
+        && t.max == maxAlphabet;
     }
     return false;
   }
@@ -1054,7 +1063,7 @@ final public class Operations {
    * Returns the longest string that is a prefix of all accepted strings and
    * visits each state at most once.  The automaton must be deterministic.
    * 
-   * @return common prefix
+   * @return common prefix, which can be an empty (length 0) String (never null)
    */
   public static String getCommonPrefix(Automaton a) {
     if (a.isDeterministic() == false) {
@@ -1088,7 +1097,7 @@ final public class Operations {
    * Returns the longest BytesRef that is a prefix of all accepted strings and
    * visits each state at most once.  The automaton must be deterministic.
    * 
-   * @return common prefix
+   * @return common prefix, which can be an empty (length 0) BytesRef (never null)
    */
   public static BytesRef getCommonPrefixBytesRef(Automaton a) {
     BytesRefBuilder builder = new BytesRefBuilder();
@@ -1112,6 +1121,37 @@ final public class Operations {
     return builder.get();
   }
 
+  /** If this automaton accepts a single input, return it.  Else, return null.
+   *  The automaton must be deterministic. */
+  public static IntsRef getSingleton(Automaton a) {
+    if (a.isDeterministic() == false) {
+      throw new IllegalArgumentException("input automaton must be deterministic");
+    }
+    IntsRefBuilder builder = new IntsRefBuilder();
+    HashSet<Integer> visited = new HashSet<>();
+    int s = 0;
+    boolean done;
+    Transition t = new Transition();
+    while (true) {
+      visited.add(s);
+      if (a.isAccept(s) == false) {
+        if (a.getNumTransitions(s) == 1) {
+          a.getTransition(s, 0, t);
+          if (t.min == t.max && !visited.contains(t.dest)) {
+            builder.append(t.min);
+            s = t.dest;
+            continue;
+          }
+        }
+      } else if (a.getNumTransitions(s) == 0) {
+        return builder.get();
+      }
+
+      // Automaton accepts more than one string:
+      return null;
+    }
+  }
+
   /**
    * Returns the longest BytesRef that is a suffix of all accepted strings.
    * Worst case complexity: exponential in number of states (this calls
@@ -1119,7 +1159,7 @@ final public class Operations {
    * @param maxDeterminizedStates maximum number of states determinizing the
    *  automaton can result in.  Set higher to allow more complex queries and
    *  lower to prevent memory exhaustion.
-   * @return common suffix
+   * @return common suffix, which can be an empty (length 0) BytesRef (never null)
    */
   public static BytesRef getCommonSuffixBytesRef(Automaton a, int maxDeterminizedStates) {
     // reverse the language of the automaton, then reverse its common prefix.

Modified: lucene/dev/branches/lucene6271/lucene/core/src/java/org/apache/lucene/util/fst/FST.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene6271/lucene/core/src/java/org/apache/lucene/util/fst/FST.java?rev=1670257&r1=1670256&r2=1670257&view=diff
==============================================================================
--- lucene/dev/branches/lucene6271/lucene/core/src/java/org/apache/lucene/util/fst/FST.java (original)
+++ lucene/dev/branches/lucene6271/lucene/core/src/java/org/apache/lucene/util/fst/FST.java Tue Mar 31 05:22:40 2015
@@ -78,19 +78,18 @@ public final class FST<T> implements Acc
   /** Specifies allowed range of each int input label for
    *  this FST. */
   public static enum INPUT_TYPE {BYTE1, BYTE2, BYTE4};
-  public final INPUT_TYPE inputType;
 
-  final static int BIT_FINAL_ARC = 1 << 0;
-  final static int BIT_LAST_ARC = 1 << 1;
-  final static int BIT_TARGET_NEXT = 1 << 2;
+  static final int BIT_FINAL_ARC = 1 << 0;
+  static final int BIT_LAST_ARC = 1 << 1;
+  static final int BIT_TARGET_NEXT = 1 << 2;
 
   // TODO: we can free up a bit if we can nuke this:
-  final static int BIT_STOP_NODE = 1 << 3;
+  static final int BIT_STOP_NODE = 1 << 3;
 
   /** This flag is set if the arc has an output. */
-  public final static int BIT_ARC_HAS_OUTPUT = 1 << 4;
+  public static final int BIT_ARC_HAS_OUTPUT = 1 << 4;
 
-  final static int BIT_ARC_HAS_FINAL_OUTPUT = 1 << 5;
+  static final int BIT_ARC_HAS_FINAL_OUTPUT = 1 << 5;
 
   // Arcs are stored as fixed-size (per entry) array, so
   // that we can find an arc using binary search.  We do
@@ -98,56 +97,61 @@ public final class FST<T> implements Acc
 
   // If set, the target node is delta coded vs current
   // position:
-  private final static int BIT_TARGET_DELTA = 1 << 6;
+  private static final int BIT_TARGET_DELTA = 1 << 6;
 
   // We use this as a marker (because this one flag is
   // illegal by itself ...):
-  private final static byte ARCS_AS_FIXED_ARRAY = BIT_ARC_HAS_FINAL_OUTPUT;
+  private static final byte ARCS_AS_FIXED_ARRAY = BIT_ARC_HAS_FINAL_OUTPUT;
 
   /**
    * @see #shouldExpand(UnCompiledNode)
    */
-  final static int FIXED_ARRAY_SHALLOW_DISTANCE = 3; // 0 => only root node.
+  static final int FIXED_ARRAY_SHALLOW_DISTANCE = 3; // 0 => only root node.
 
   /**
    * @see #shouldExpand(UnCompiledNode)
    */
-  final static int FIXED_ARRAY_NUM_ARCS_SHALLOW = 5;
+  static final int FIXED_ARRAY_NUM_ARCS_SHALLOW = 5;
 
   /**
    * @see #shouldExpand(UnCompiledNode)
    */
-  final static int FIXED_ARRAY_NUM_ARCS_DEEP = 10;
+  static final int FIXED_ARRAY_NUM_ARCS_DEEP = 10;
 
   // Reused temporarily while building the FST:
   private int[] reusedBytesPerArc = new int[0];
 
   // Increment version to change it
-  private final static String FILE_FORMAT_NAME = "FST";
-  private final static int VERSION_START = 0;
+  private static final String FILE_FORMAT_NAME = "FST";
+  private static final int VERSION_START = 0;
 
   /** Changed numBytesPerArc for array'd case from byte to int. */
-  private final static int VERSION_INT_NUM_BYTES_PER_ARC = 1;
+  private static final int VERSION_INT_NUM_BYTES_PER_ARC = 1;
 
   /** Write BYTE2 labels as 2-byte short, not vInt. */
-  private final static int VERSION_SHORT_BYTE2_LABELS = 2;
+  private static final int VERSION_SHORT_BYTE2_LABELS = 2;
 
   /** Added optional packed format. */
-  private final static int VERSION_PACKED = 3;
+  private static final int VERSION_PACKED = 3;
 
   /** Changed from int to vInt for encoding arc targets. 
    *  Also changed maxBytesPerArc from int to vInt in the array case. */
-  private final static int VERSION_VINT_TARGET = 4;
+  private static final int VERSION_VINT_TARGET = 4;
 
-  private final static int VERSION_CURRENT = VERSION_VINT_TARGET;
+  private static final int VERSION_CURRENT = VERSION_VINT_TARGET;
 
   // Never serialized; just used to represent the virtual
   // final node w/ no arcs:
-  private final static long FINAL_END_NODE = -1;
+  private static final long FINAL_END_NODE = -1;
 
   // Never serialized; just used to represent the virtual
   // non-final node w/ no arcs:
-  private final static long NON_FINAL_END_NODE = 0;
+  private static final long NON_FINAL_END_NODE = 0;
+
+  /** If arc has this label then that arc is final/accepted */
+  public static final int END_LABEL = -1;
+
+  public final INPUT_TYPE inputType;
 
   // if non-null, this FST accepts the empty string and
   // produces this output
@@ -174,15 +178,12 @@ public final class FST<T> implements Acc
   private final boolean packed;
   private PackedInts.Reader nodeRefToAddress;
 
-  /** If arc has this label then that arc is final/accepted */
-  public static final int END_LABEL = -1;
-
   private final boolean allowArrayArcs;
 
   private Arc<T> cachedRootArcs[];
 
   /** Represents a single arc. */
-  public final static class Arc<T> {
+  public static final class Arc<T> {
     public int label;
     public T output;
 
@@ -604,8 +605,8 @@ public final class FST<T> implements Acc
    * Writes an automaton to a file. 
    */
   public void save(final Path path) throws IOException {
-    try (OutputStream os = Files.newOutputStream(path)) {
-      save(new OutputStreamDataOutput(new BufferedOutputStream(os)));
+    try (OutputStream os = new BufferedOutputStream(Files.newOutputStream(path))) {
+      save(new OutputStreamDataOutput(os));
     }
   }
 

Modified: lucene/dev/branches/lucene6271/lucene/core/src/java/org/apache/lucene/util/packed/EliasFanoDocIdSet.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene6271/lucene/core/src/java/org/apache/lucene/util/packed/EliasFanoDocIdSet.java?rev=1670257&r1=1670256&r2=1670257&view=diff
==============================================================================
--- lucene/dev/branches/lucene6271/lucene/core/src/java/org/apache/lucene/util/packed/EliasFanoDocIdSet.java (original)
+++ lucene/dev/branches/lucene6271/lucene/core/src/java/org/apache/lucene/util/packed/EliasFanoDocIdSet.java Tue Mar 31 05:22:40 2015
@@ -111,14 +111,6 @@ public class EliasFanoDocIdSet extends D
     };
   }
 
-  /** This DocIdSet implementation is cacheable.
-   * @return <code>true</code>
-   */
-  @Override
-  public boolean isCacheable() {
-    return true;
-  }
-
   @Override
   public boolean equals(Object other) {
     return ((other instanceof EliasFanoDocIdSet))

Modified: lucene/dev/branches/lucene6271/lucene/core/src/test/org/apache/lucene/TestDemo.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene6271/lucene/core/src/test/org/apache/lucene/TestDemo.java?rev=1670257&r1=1670256&r2=1670257&view=diff
==============================================================================
--- lucene/dev/branches/lucene6271/lucene/core/src/test/org/apache/lucene/TestDemo.java (original)
+++ lucene/dev/branches/lucene6271/lucene/core/src/test/org/apache/lucene/TestDemo.java Tue Mar 31 05:22:40 2015
@@ -77,5 +77,6 @@ public class TestDemo extends LuceneTest
 
     ireader.close();
     directory.close();
+    analyzer.close();
   }
 }

Modified: lucene/dev/branches/lucene6271/lucene/core/src/test/org/apache/lucene/document/TestDateTools.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene6271/lucene/core/src/test/org/apache/lucene/document/TestDateTools.java?rev=1670257&r1=1670256&r2=1670257&view=diff
==============================================================================
--- lucene/dev/branches/lucene6271/lucene/core/src/test/org/apache/lucene/document/TestDateTools.java (original)
+++ lucene/dev/branches/lucene6271/lucene/core/src/test/org/apache/lucene/document/TestDateTools.java Tue Mar 31 05:22:40 2015
@@ -5,11 +5,6 @@ import java.text.SimpleDateFormat;
 import java.util.*;
 
 import org.apache.lucene.util.LuceneTestCase;
-import org.junit.Rule;
-import org.junit.rules.RuleChain;
-import org.junit.rules.TestRule;
-
-import com.carrotsearch.randomizedtesting.rules.SystemPropertiesRestoreRule;
 
 /*
  * Licensed to the Apache Software Foundation (ASF) under one or more
@@ -28,10 +23,6 @@ import com.carrotsearch.randomizedtestin
  * limitations under the License.
  */
 public class TestDateTools extends LuceneTestCase {
-  @Rule
-  public TestRule testRules = 
-    RuleChain.outerRule(new SystemPropertiesRestoreRule());
-
   public void testStringToDate() throws ParseException {
     
     Date d = null;

Modified: lucene/dev/branches/lucene6271/lucene/core/src/test/org/apache/lucene/index/Test2BBinaryDocValues.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene6271/lucene/core/src/test/org/apache/lucene/index/Test2BBinaryDocValues.java?rev=1670257&r1=1670256&r2=1670257&view=diff
==============================================================================
--- lucene/dev/branches/lucene6271/lucene/core/src/test/org/apache/lucene/index/Test2BBinaryDocValues.java (original)
+++ lucene/dev/branches/lucene6271/lucene/core/src/test/org/apache/lucene/index/Test2BBinaryDocValues.java Tue Mar 31 05:22:40 2015
@@ -25,12 +25,12 @@ import org.apache.lucene.store.ByteArray
 import org.apache.lucene.store.ByteArrayDataOutput;
 import org.apache.lucene.store.MockDirectoryWrapper;
 import org.apache.lucene.util.BytesRef;
-import org.apache.lucene.util.LuceneTestCase;
 import org.apache.lucene.util.LuceneTestCase.Monster;
 import org.apache.lucene.util.LuceneTestCase.SuppressCodecs;
 import org.apache.lucene.util.TestUtil;
+import org.apache.lucene.util.LuceneTestCase.SuppressSysoutChecks;
+import org.apache.lucene.util.LuceneTestCase;
 import org.apache.lucene.util.TimeUnits;
-
 import com.carrotsearch.randomizedtesting.annotations.TimeoutSuite;
 
 @SuppressCodecs({"SimpleText", "Memory", "Direct"})
@@ -38,6 +38,7 @@ import com.carrotsearch.randomizedtestin
 // The six hour time was achieved on a Linux 3.13 system with these specs:
 // 3-core AMD at 2.5Ghz, 12 GB RAM, 5GB test heap, 2 test JVMs, 2TB SATA.
 @Monster("takes ~ 6 hours if the heap is 5gb")
+@SuppressSysoutChecks(bugUrl = "Stuff gets printed.")
 public class Test2BBinaryDocValues extends LuceneTestCase {
   
   // indexes IndexWriter.MAX_DOCS docs with a fixed binary field

Modified: lucene/dev/branches/lucene6271/lucene/core/src/test/org/apache/lucene/index/Test2BNumericDocValues.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene6271/lucene/core/src/test/org/apache/lucene/index/Test2BNumericDocValues.java?rev=1670257&r1=1670256&r2=1670257&view=diff
==============================================================================
--- lucene/dev/branches/lucene6271/lucene/core/src/test/org/apache/lucene/index/Test2BNumericDocValues.java (original)
+++ lucene/dev/branches/lucene6271/lucene/core/src/test/org/apache/lucene/index/Test2BNumericDocValues.java Tue Mar 31 05:22:40 2015
@@ -27,6 +27,7 @@ import org.apache.lucene.util.TestUtil;
 import org.apache.lucene.util.TimeUnits;
 import org.apache.lucene.util.LuceneTestCase.Monster;
 import org.apache.lucene.util.LuceneTestCase.SuppressCodecs;
+import org.apache.lucene.util.LuceneTestCase.SuppressSysoutChecks;
 
 import com.carrotsearch.randomizedtesting.annotations.TimeoutSuite;
 
@@ -35,6 +36,7 @@ import com.carrotsearch.randomizedtestin
 // The two hour time was achieved on a Linux 3.13 system with these specs:
 // 3-core AMD at 2.5Ghz, 12 GB RAM, 5GB test heap, 2 test JVMs, 2TB SATA.
 @Monster("takes ~ 2 hours if the heap is 5gb")
+@SuppressSysoutChecks(bugUrl = "Stuff gets printed")
 public class Test2BNumericDocValues extends LuceneTestCase {
   
   // indexes IndexWriter.MAX_DOCS docs with an increasing dv field

Modified: lucene/dev/branches/lucene6271/lucene/core/src/test/org/apache/lucene/index/Test2BSortedDocValues.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene6271/lucene/core/src/test/org/apache/lucene/index/Test2BSortedDocValues.java?rev=1670257&r1=1670256&r2=1670257&view=diff
==============================================================================
--- lucene/dev/branches/lucene6271/lucene/core/src/test/org/apache/lucene/index/Test2BSortedDocValues.java (original)
+++ lucene/dev/branches/lucene6271/lucene/core/src/test/org/apache/lucene/index/Test2BSortedDocValues.java Tue Mar 31 05:22:40 2015
@@ -28,6 +28,7 @@ import org.apache.lucene.util.TestUtil;
 import org.apache.lucene.util.TimeUnits;
 import org.apache.lucene.util.LuceneTestCase.Monster;
 import org.apache.lucene.util.LuceneTestCase.SuppressCodecs;
+import org.apache.lucene.util.LuceneTestCase.SuppressSysoutChecks;
 
 import com.carrotsearch.randomizedtesting.annotations.TimeoutSuite;
 
@@ -36,6 +37,7 @@ import com.carrotsearch.randomizedtestin
 // The six hour time was achieved on a Linux 3.13 system with these specs:
 // 3-core AMD at 2.5Ghz, 12 GB RAM, 5GB test heap, 2 test JVMs, 2TB SATA.
 @Monster("Takes ~ 6 hours if the heap is 5gb")
+@SuppressSysoutChecks(bugUrl = "Stuff gets printed")
 public class Test2BSortedDocValues extends LuceneTestCase {
   
   // indexes Integer.MAX_VALUE docs with a fixed binary field

Modified: lucene/dev/branches/lucene6271/lucene/core/src/test/org/apache/lucene/index/Test2BTerms.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene6271/lucene/core/src/test/org/apache/lucene/index/Test2BTerms.java?rev=1670257&r1=1670256&r2=1670257&view=diff
==============================================================================
--- lucene/dev/branches/lucene6271/lucene/core/src/test/org/apache/lucene/index/Test2BTerms.java (original)
+++ lucene/dev/branches/lucene6271/lucene/core/src/test/org/apache/lucene/index/Test2BTerms.java Tue Mar 31 05:22:40 2015
@@ -41,6 +41,7 @@ import org.apache.lucene.util.AttributeI
 import org.apache.lucene.util.BytesRef;
 import org.apache.lucene.util.LuceneTestCase.Monster;
 import org.apache.lucene.util.LuceneTestCase.SuppressCodecs;
+import org.apache.lucene.util.LuceneTestCase.SuppressSysoutChecks;
 import org.apache.lucene.util.LuceneTestCase;
 import org.apache.lucene.util.TestUtil;
 import org.apache.lucene.util.TimeUnits;
@@ -55,6 +56,7 @@ import com.carrotsearch.randomizedtestin
 @SuppressCodecs({ "SimpleText", "Memory", "Direct" })
 @Monster("very slow, use 5g minimum heap")
 @TimeoutSuite(millis = 80 * TimeUnits.HOUR) // effectively no limit
+@SuppressSysoutChecks(bugUrl = "Stuff gets printed")
 public class Test2BTerms extends LuceneTestCase {
 
   private final static int TOKEN_LEN = 5;

Modified: lucene/dev/branches/lucene6271/lucene/core/src/test/org/apache/lucene/index/TestAddIndexes.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene6271/lucene/core/src/test/org/apache/lucene/index/TestAddIndexes.java?rev=1670257&r1=1670256&r2=1670257&view=diff
==============================================================================
--- lucene/dev/branches/lucene6271/lucene/core/src/test/org/apache/lucene/index/TestAddIndexes.java (original)
+++ lucene/dev/branches/lucene6271/lucene/core/src/test/org/apache/lucene/index/TestAddIndexes.java Tue Mar 31 05:22:40 2015
@@ -339,7 +339,7 @@ public class TestAddIndexes extends Luce
 
     writer.addIndexes(aux);
     assertEquals(1040, writer.maxDoc());
-    assertEquals(1000, writer.getDocCount(0));
+    assertEquals(1000, writer.maxDoc(0));
     writer.close();
 
     // make sure the index is correct
@@ -368,7 +368,7 @@ public class TestAddIndexes extends Luce
 
     writer.addIndexes(aux);
     assertEquals(1032, writer.maxDoc());
-    assertEquals(1000, writer.getDocCount(0));
+    assertEquals(1000, writer.maxDoc(0));
     writer.close();
 
     // make sure the index is correct
@@ -396,7 +396,7 @@ public class TestAddIndexes extends Luce
 
     writer.addIndexes(aux, new MockDirectoryWrapper(random(), TestUtil.ramCopyOf(aux)));
     assertEquals(1060, writer.maxDoc());
-    assertEquals(1000, writer.getDocCount(0));
+    assertEquals(1000, writer.maxDoc(0));
     writer.close();
 
     // make sure the index is correct
@@ -438,7 +438,7 @@ public class TestAddIndexes extends Luce
     }
     writer.addIndexes(aux, new MockDirectoryWrapper(random(), TestUtil.ramCopyOf(aux)));
     assertEquals(1020, writer.maxDoc());
-    assertEquals(1000, writer.getDocCount(0));
+    assertEquals(1000, writer.maxDoc(0));
     writer.close();
     dir.close();
     aux.close();
@@ -498,7 +498,7 @@ public class TestAddIndexes extends Luce
 
     writer.addIndexes(aux, aux2);
     assertEquals(1040, writer.maxDoc());
-    assertEquals(1000, writer.getDocCount(0));
+    assertEquals(1000, writer.maxDoc(0));
     writer.close();
     dir.close();
     aux.close();

Modified: lucene/dev/branches/lucene6271/lucene/core/src/test/org/apache/lucene/index/TestCheckIndex.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene6271/lucene/core/src/test/org/apache/lucene/index/TestCheckIndex.java?rev=1670257&r1=1670256&r2=1670257&view=diff
==============================================================================
--- lucene/dev/branches/lucene6271/lucene/core/src/test/org/apache/lucene/index/TestCheckIndex.java (original)
+++ lucene/dev/branches/lucene6271/lucene/core/src/test/org/apache/lucene/index/TestCheckIndex.java Tue Mar 31 05:22:40 2015
@@ -24,9 +24,11 @@ import java.util.List;
 import java.util.ArrayList;
 
 import org.apache.lucene.util.IOUtils;
+import org.apache.lucene.util.LineFileDocs;
 import org.apache.lucene.util.LuceneTestCase;
 import org.apache.lucene.store.Directory;
 import org.apache.lucene.store.LockObtainFailedException;
+import org.apache.lucene.analysis.Analyzer;
 import org.apache.lucene.analysis.CannedTokenStream;
 import org.apache.lucene.analysis.MockAnalyzer;
 import org.apache.lucene.analysis.Token;
@@ -118,6 +120,54 @@ public class TestCheckIndex extends Luce
     dir.close(); // checkindex
   }
   
+  public void testChecksumsOnly() throws IOException {
+    LineFileDocs lf = new LineFileDocs(random());
+    Directory dir = newDirectory();
+    Analyzer analyzer = new MockAnalyzer(random());
+    IndexWriter iw = new IndexWriter(dir, newIndexWriterConfig(analyzer));
+    for (int i = 0; i < 100; i++) {
+      iw.addDocument(lf.nextDoc());
+    }
+    iw.addDocument(new Document());
+    iw.commit();
+    iw.close();
+    lf.close();
+    
+    ByteArrayOutputStream bos = new ByteArrayOutputStream(1024);
+    CheckIndex checker = new CheckIndex(dir);
+    checker.setInfoStream(new PrintStream(bos, false, IOUtils.UTF_8));
+    if (VERBOSE) checker.setInfoStream(System.out);
+    CheckIndex.Status indexStatus = checker.checkIndex();
+    assertTrue(indexStatus.clean);
+    checker.close();
+    dir.close();
+    analyzer.close();
+  }
+  
+  public void testChecksumsOnlyVerbose() throws IOException {
+    LineFileDocs lf = new LineFileDocs(random());
+    Directory dir = newDirectory();
+    Analyzer analyzer = new MockAnalyzer(random());
+    IndexWriter iw = new IndexWriter(dir, newIndexWriterConfig(analyzer));
+    for (int i = 0; i < 100; i++) {
+      iw.addDocument(lf.nextDoc());
+    }
+    iw.addDocument(new Document());
+    iw.commit();
+    iw.close();
+    lf.close();
+    
+    ByteArrayOutputStream bos = new ByteArrayOutputStream(1024);
+    CheckIndex checker = new CheckIndex(dir);
+    checker.setInfoStream(new PrintStream(bos, true, IOUtils.UTF_8));
+    if (VERBOSE) checker.setInfoStream(System.out);
+    CheckIndex.Status indexStatus = checker.checkIndex();
+    assertTrue(indexStatus.clean);
+    checker.close();
+    dir.close();
+    analyzer.close();
+  }
+  
   public void testObtainsLock() throws IOException {
     Directory dir = newDirectory();
     IndexWriter iw = new IndexWriter(dir, newIndexWriterConfig(null));

Modified: lucene/dev/branches/lucene6271/lucene/core/src/test/org/apache/lucene/index/TestCodecs.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene6271/lucene/core/src/test/org/apache/lucene/index/TestCodecs.java?rev=1670257&r1=1670256&r2=1670257&view=diff
==============================================================================
--- lucene/dev/branches/lucene6271/lucene/core/src/test/org/apache/lucene/index/TestCodecs.java (original)
+++ lucene/dev/branches/lucene6271/lucene/core/src/test/org/apache/lucene/index/TestCodecs.java Tue Mar 31 05:22:40 2015
@@ -19,6 +19,7 @@ package org.apache.lucene.index;
 
 import java.io.IOException;
 import java.util.Arrays;
+import java.util.Collections;
 import java.util.HashMap;
 import java.util.HashSet;
 import java.util.Iterator;
@@ -221,7 +222,7 @@ public class TestCodecs extends LuceneTe
     final FieldInfos fieldInfos = builder.finish();
     final Directory dir = newDirectory();
     Codec codec = Codec.getDefault();
-    final SegmentInfo si = new SegmentInfo(dir, Version.LATEST, SEGMENT, 10000, false, codec, null, StringHelper.randomId(), new HashMap<>());
+    final SegmentInfo si = new SegmentInfo(dir, Version.LATEST, SEGMENT, 10000, false, codec, Collections.emptyMap(), StringHelper.randomId(), new HashMap<>());
     
     this.write(si, fieldInfos, dir, fields);
     final FieldsProducer reader = codec.postingsFormat().fieldsProducer(new SegmentReadState(dir, si, fieldInfos, newIOContext(random())));
@@ -278,7 +279,7 @@ public class TestCodecs extends LuceneTe
     }
 
     Codec codec = Codec.getDefault();
-    final SegmentInfo si = new SegmentInfo(dir, Version.LATEST, SEGMENT, 10000, false, codec, null, StringHelper.randomId(), new HashMap<>());
+    final SegmentInfo si = new SegmentInfo(dir, Version.LATEST, SEGMENT, 10000, false, codec, Collections.emptyMap(), StringHelper.randomId(), new HashMap<>());
     this.write(si, fieldInfos, dir, fields);
 
     if (VERBOSE) {

Modified: lucene/dev/branches/lucene6271/lucene/core/src/test/org/apache/lucene/index/TestDemoParallelLeafReader.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene6271/lucene/core/src/test/org/apache/lucene/index/TestDemoParallelLeafReader.java?rev=1670257&r1=1670256&r2=1670257&view=diff
==============================================================================
--- lucene/dev/branches/lucene6271/lucene/core/src/test/org/apache/lucene/index/TestDemoParallelLeafReader.java (original)
+++ lucene/dev/branches/lucene6271/lucene/core/src/test/org/apache/lucene/index/TestDemoParallelLeafReader.java Tue Mar 31 05:22:40 2015
@@ -191,7 +191,7 @@ public class TestDemoParallelLeafReader
     }
 
     private class ParallelLeafDirectoryReader extends FilterDirectoryReader {
-      public ParallelLeafDirectoryReader(DirectoryReader in) {
+      public ParallelLeafDirectoryReader(DirectoryReader in) throws IOException {
         super(in, new FilterDirectoryReader.SubReaderWrapper() {
             final long currentSchemaGen = getCurrentSchemaGen();
             @Override
@@ -207,7 +207,7 @@ public class TestDemoParallelLeafReader
       }
 
       @Override
-      protected DirectoryReader doWrapDirectoryReader(DirectoryReader in) {
+      protected DirectoryReader doWrapDirectoryReader(DirectoryReader in) throws IOException {
         return new ParallelLeafDirectoryReader(in);
       }
 

Modified: lucene/dev/branches/lucene6271/lucene/core/src/test/org/apache/lucene/index/TestDoc.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene6271/lucene/core/src/test/org/apache/lucene/index/TestDoc.java?rev=1670257&r1=1670256&r2=1670257&view=diff
==============================================================================
--- lucene/dev/branches/lucene6271/lucene/core/src/test/org/apache/lucene/index/TestDoc.java (original)
+++ lucene/dev/branches/lucene6271/lucene/core/src/test/org/apache/lucene/index/TestDoc.java Tue Mar 31 05:22:40 2015
@@ -28,6 +28,7 @@ import java.nio.file.Files;
 import java.nio.file.Path;
 import java.util.Arrays;
 import java.util.Collection;
+import java.util.Collections;
 import java.util.HashMap;
 import java.util.HashSet;
 import java.util.LinkedList;
@@ -220,7 +221,7 @@ public class TestDoc extends LuceneTestC
 
     final Codec codec = Codec.getDefault();
     TrackingDirectoryWrapper trackingDir = new TrackingDirectoryWrapper(si1.info.dir);
-    final SegmentInfo si = new SegmentInfo(si1.info.dir, Version.LATEST, merged, -1, false, codec, null, StringHelper.randomId(), new HashMap<>());
+    final SegmentInfo si = new SegmentInfo(si1.info.dir, Version.LATEST, merged, -1, false, codec, Collections.emptyMap(), StringHelper.randomId(), new HashMap<>());
 
     SegmentMerger merger = new SegmentMerger(Arrays.<CodecReader>asList(r1, r2),
                                              si, InfoStream.getDefault(), trackingDir,

Modified: lucene/dev/branches/lucene6271/lucene/core/src/test/org/apache/lucene/index/TestIndexWriterMaxDocs.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene6271/lucene/core/src/test/org/apache/lucene/index/TestIndexWriterMaxDocs.java?rev=1670257&r1=1670256&r2=1670257&view=diff
==============================================================================
--- lucene/dev/branches/lucene6271/lucene/core/src/test/org/apache/lucene/index/TestIndexWriterMaxDocs.java (original)
+++ lucene/dev/branches/lucene6271/lucene/core/src/test/org/apache/lucene/index/TestIndexWriterMaxDocs.java Tue Mar 31 05:22:40 2015
@@ -17,8 +17,10 @@ package org.apache.lucene.index;
  * limitations under the License.
  */
 
+import java.io.IOException;
 import java.util.Arrays;
 import java.util.Collections;
+import java.util.concurrent.CountDownLatch;
 
 import org.apache.lucene.document.Document;
 import org.apache.lucene.document.Field;
@@ -28,11 +30,13 @@ import org.apache.lucene.search.SortFiel
 import org.apache.lucene.search.TermQuery;
 import org.apache.lucene.search.TopDocs;
 import org.apache.lucene.store.Directory;
+import org.apache.lucene.store.FilterDirectory;
+import org.apache.lucene.store.MockDirectoryWrapper;
+import org.apache.lucene.store.NoLockFactory;
 import org.apache.lucene.util.LuceneTestCase.SuppressCodecs;
 import org.apache.lucene.util.LuceneTestCase;
 import org.apache.lucene.util.TestUtil;
 import org.apache.lucene.util.TimeUnits;
-
 import com.carrotsearch.randomizedtesting.annotations.TimeoutSuite;
 
 @SuppressCodecs({ "SimpleText", "Memory", "Direct" })
@@ -93,7 +97,7 @@ public class TestIndexWriterMaxDocs exte
       try {
         w.addDocument(new Document());
         fail("didn't hit exception");
-      } catch (IllegalStateException ise) {
+      } catch (IllegalArgumentException iae) {
         // expected
       }
       w.close();
@@ -116,7 +120,7 @@ public class TestIndexWriterMaxDocs exte
       try {
         w.addDocuments(Collections.singletonList(new Document()));
         fail("didn't hit exception");
-      } catch (IllegalStateException ise) {
+      } catch (IllegalArgumentException iae) {
         // expected
       }
       w.close();
@@ -139,7 +143,7 @@ public class TestIndexWriterMaxDocs exte
       try {
         w.updateDocument(new Term("field", "foo"), new Document());
         fail("didn't hit exception");
-      } catch (IllegalStateException ise) {
+      } catch (IllegalArgumentException iae) {
         // expected
       }
       w.close();
@@ -162,7 +166,7 @@ public class TestIndexWriterMaxDocs exte
       try {
         w.updateDocuments(new Term("field", "foo"), Collections.singletonList(new Document()));
         fail("didn't hit exception");
-      } catch (IllegalStateException ise) {
+      } catch (IllegalArgumentException iae) {
         // expected
       }
       w.close();
@@ -201,7 +205,7 @@ public class TestIndexWriterMaxDocs exte
       try {
         w.addDocument(new Document());
         fail("didn't hit exception");
-      } catch (IllegalStateException ise) {
+      } catch (IllegalArgumentException iae) {
         // expected
       }
       w.close();
@@ -247,7 +251,7 @@ public class TestIndexWriterMaxDocs exte
       try {
         w.addDocument(new Document());
         fail("didn't hit exception");
-      } catch (IllegalStateException ise) {
+      } catch (IllegalArgumentException iae) {
         // expected
       }
       w.close();
@@ -273,7 +277,7 @@ public class TestIndexWriterMaxDocs exte
       try {
         w2.addIndexes(new Directory[] {dir});
         fail("didn't hit exception");
-      } catch (IllegalStateException ise) {
+      } catch (IllegalArgumentException iae) {
         // expected
       }
       assertEquals(1, w2.maxDoc());
@@ -281,7 +285,7 @@ public class TestIndexWriterMaxDocs exte
       try {
         TestUtil.addIndexesSlowly(w2, ir);
         fail("didn't hit exception");
-      } catch (IllegalStateException ise) {
+      } catch (IllegalArgumentException iae) {
         // expected
       }
       w2.close();
@@ -369,6 +373,103 @@ public class TestIndexWriterMaxDocs exte
     dir.close();
     dir2.close();
   }
+  
+  /** 
+   * LUCENE-6299: Test if addindexes(Dir[]) prevents exceeding max docs.
+   */
+  public void testAddTooManyIndexesDir() throws Exception {
+    // we cheat and add the same one over again... IW wants a write lock on each
+    Directory dir = newDirectory(random(), NoLockFactory.INSTANCE);
+    Document doc = new Document();
+    IndexWriter w = new IndexWriter(dir, new IndexWriterConfig(null));
+    for (int i = 0; i < 100000; i++) {
+      w.addDocument(doc);
+    }
+    w.forceMerge(1);
+    w.commit();
+    w.close();
+    
+    // wrap this with disk full, so test fails faster and doesn't fill up real disks.
+    MockDirectoryWrapper dir2 = newMockDirectory();
+    w = new IndexWriter(dir2, new IndexWriterConfig(null));
+    w.commit(); // don't confuse checkindex
+    dir2.setMaxSizeInBytes(dir2.sizeInBytes() + 65536); // 64KB
+    Directory dirs[] = new Directory[1 + (IndexWriter.MAX_DOCS / 100000)];
+    for (int i = 0; i < dirs.length; i++) {
+      // bypass iw check for duplicate dirs
+      dirs[i] = new FilterDirectory(dir) {};
+    }
+
+    try {
+      w.addIndexes(dirs);
+      fail("didn't get expected exception");
+    } catch (IllegalArgumentException expected) {
+      // pass
+    } catch (IOException fakeDiskFull) {
+      final Exception e;
+      if (fakeDiskFull.getMessage() != null && fakeDiskFull.getMessage().startsWith("fake disk full")) {
+        e = new RuntimeException("test failed: IW checks aren't working and we are executing addIndexes");
+        e.addSuppressed(fakeDiskFull);
+      } else {
+        e = fakeDiskFull;
+      }
+      throw e;
+    }
+    
+    w.close();
+    dir.close();
+    dir2.close();
+  }
+
+  /** 
+   * LUCENE-6299: Test if addindexes(CodecReader[]) prevents exceeding max docs.
+   */
+  public void testAddTooManyIndexesCodecReader() throws Exception {
+    // we cheat and add the same one over again... IW wants a write lock on each
+    Directory dir = newDirectory(random(), NoLockFactory.INSTANCE);
+    Document doc = new Document();
+    IndexWriter w = new IndexWriter(dir, new IndexWriterConfig(null));
+    for (int i = 0; i < 100000; i++) {
+      w.addDocument(doc);
+    }
+    w.forceMerge(1);
+    w.commit();
+    w.close();
+    
+    // wrap this with disk full, so test fails faster and doesn't fill up real disks.
+    MockDirectoryWrapper dir2 = newMockDirectory();
+    w = new IndexWriter(dir2, new IndexWriterConfig(null));
+    w.commit(); // don't confuse checkindex
+    dir2.setMaxSizeInBytes(dir2.sizeInBytes() + 65536); // 64KB
+    IndexReader r = DirectoryReader.open(dir);
+    CodecReader segReader = (CodecReader) r.leaves().get(0).reader();
+
+    CodecReader readers[] = new CodecReader[1 + (IndexWriter.MAX_DOCS / 100000)];
+    for (int i = 0; i < readers.length; i++) {
+      readers[i] = segReader;
+    }
+
+    try {
+      w.addIndexes(readers);
+      fail("didn't get expected exception");
+    } catch (IllegalArgumentException expected) {
+      // pass
+    } catch (IOException fakeDiskFull) {
+      final Exception e;
+      if (fakeDiskFull.getMessage() != null && fakeDiskFull.getMessage().startsWith("fake disk full")) {
+        e = new RuntimeException("test failed: IW checks aren't working and we are executing addIndexes");
+        e.addSuppressed(fakeDiskFull);
+      } else {
+        e = fakeDiskFull;
+      }
+      throw e;
+    }
+
+    r.close();
+    w.close();
+    dir.close();
+    dir2.close();
+  }
 
   public void testTooLargeMaxDocs() throws Exception {
     try {
@@ -378,4 +479,244 @@ public class TestIndexWriterMaxDocs exte
       // expected
     }
   }
+
+  // LUCENE-6299
+  public void testDeleteAll() throws Exception {
+    setIndexWriterMaxDocs(1);
+    try {
+      Directory dir = newDirectory();
+      IndexWriter w = new IndexWriter(dir, new IndexWriterConfig(null));
+      w.addDocument(new Document());
+      try {
+        w.addDocument(new Document());
+        fail("didn't hit exception");
+      } catch (IllegalArgumentException iae) {
+        // expected
+      }
+      w.deleteAll();
+      w.addDocument(new Document());
+      try {
+        w.addDocument(new Document());
+        fail("didn't hit exception");
+      } catch (IllegalArgumentException iae) {
+        // expected
+      }
+      w.close();
+      dir.close();
+    } finally {
+      restoreIndexWriterMaxDocs();
+    }
+  }
+
+  // LUCENE-6299
+  public void testDeleteAllAfterFlush() throws Exception {
+    setIndexWriterMaxDocs(2);
+    try {
+      Directory dir = newDirectory();
+      IndexWriter w = new IndexWriter(dir, new IndexWriterConfig(null));
+      w.addDocument(new Document());
+      w.getReader().close();
+      w.addDocument(new Document());
+      try {
+        w.addDocument(new Document());
+        fail("didn't hit exception");
+      } catch (IllegalArgumentException iae) {
+        // expected
+      }
+      w.deleteAll();
+      w.addDocument(new Document());
+      w.addDocument(new Document());
+      try {
+        w.addDocument(new Document());
+        fail("didn't hit exception");
+      } catch (IllegalArgumentException iae) {
+        // expected
+      }
+      w.close();
+      dir.close();
+    } finally {
+      restoreIndexWriterMaxDocs();
+    }
+  }
+
+  // LUCENE-6299
+  public void testDeleteAllAfterCommit() throws Exception {
+    setIndexWriterMaxDocs(2);
+    try {
+      Directory dir = newDirectory();
+      IndexWriter w = new IndexWriter(dir, new IndexWriterConfig(null));
+      w.addDocument(new Document());
+      w.commit();
+      w.addDocument(new Document());
+      try {
+        w.addDocument(new Document());
+        fail("didn't hit exception");
+      } catch (IllegalArgumentException iae) {
+        // expected
+      }
+      w.deleteAll();
+      w.addDocument(new Document());
+      w.addDocument(new Document());
+      try {
+        w.addDocument(new Document());
+        fail("didn't hit exception");
+      } catch (IllegalArgumentException iae) {
+        // expected
+      }
+      w.close();
+      dir.close();
+    } finally {
+      restoreIndexWriterMaxDocs();
+    }
+  }
+
+  // LUCENE-6299
+  public void testDeleteAllMultipleThreads() throws Exception {
+    int limit = TestUtil.nextInt(random(), 2, 10);
+    setIndexWriterMaxDocs(limit);
+    try {
+      Directory dir = newDirectory();
+      IndexWriter w = new IndexWriter(dir, new IndexWriterConfig(null));
+
+      CountDownLatch startingGun = new CountDownLatch(1);
+      Thread[] threads = new Thread[limit];
+      for(int i=0;i<limit;i++) {
+        threads[i] = new Thread() {
+          @Override
+          public void run() {
+            try {
+              startingGun.await();
+              w.addDocument(new Document());
+            } catch (Exception e) {
+              throw new RuntimeException(e);
+            }
+          }
+          };
+        threads[i].start();
+      }
+
+      startingGun.countDown();
+
+      for(Thread thread : threads) {
+        thread.join();
+      }
+
+      try {
+        w.addDocument(new Document());
+        fail("didn't hit exception");
+      } catch (IllegalArgumentException iae) {
+        // expected
+      }
+      w.deleteAll();
+      for(int i=0;i<limit;i++) {
+        w.addDocument(new Document());
+      }        
+      try {
+        w.addDocument(new Document());
+        fail("didn't hit exception");
+      } catch (IllegalArgumentException iae) {
+        // expected
+      }
+      w.close();
+      dir.close();
+    } finally {
+      restoreIndexWriterMaxDocs();
+    }
+  }
+
+  // LUCENE-6299
+  public void testDeleteAllAfterClose() throws Exception {
+    setIndexWriterMaxDocs(2);
+    try {
+      Directory dir = newDirectory();
+      IndexWriter w = new IndexWriter(dir, new IndexWriterConfig(null));
+      w.addDocument(new Document());
+      w.close();
+      w = new IndexWriter(dir, new IndexWriterConfig(null));
+      w.addDocument(new Document());
+      try {
+        w.addDocument(new Document());
+        fail("didn't hit exception");
+      } catch (IllegalArgumentException iae) {
+        // expected
+      }
+      w.deleteAll();
+      w.addDocument(new Document());
+      w.addDocument(new Document());
+      try {
+        w.addDocument(new Document());
+        fail("didn't hit exception");
+      } catch (IllegalArgumentException iae) {
+        // expected
+      }
+      w.close();
+      dir.close();
+    } finally {
+      restoreIndexWriterMaxDocs();
+    }
+  }
+
+  // LUCENE-6299
+  public void testAcrossTwoIndexWriters() throws Exception {
+    setIndexWriterMaxDocs(1);
+    try {
+      Directory dir = newDirectory();
+      IndexWriter w = new IndexWriter(dir, new IndexWriterConfig(null));
+      w.addDocument(new Document());
+      w.close();
+      w = new IndexWriter(dir, new IndexWriterConfig(null));
+      try {
+        w.addDocument(new Document());
+        fail("didn't hit exception");
+      } catch (IllegalArgumentException iae) {
+        // expected
+      }
+      w.close();
+      dir.close();
+    } finally {
+      restoreIndexWriterMaxDocs();
+    }
+  }
+
+  // LUCENE-6299
+  public void testCorruptIndexExceptionTooLarge() throws Exception {
+    Directory dir = newDirectory();
+    IndexWriter w = new IndexWriter(dir, new IndexWriterConfig(null));
+    w.addDocument(new Document());
+    w.addDocument(new Document());
+    w.close();
+
+    setIndexWriterMaxDocs(1);
+    try {       
+      DirectoryReader.open(dir);
+      fail("didn't hit exception");
+    } catch (CorruptIndexException cie) {
+      // expected
+    } finally {
+      restoreIndexWriterMaxDocs();
+    }
+
+    dir.close();
+  }
+  
+  // LUCENE-6299
+  public void testCorruptIndexExceptionTooLargeWriter() throws Exception {
+    Directory dir = newDirectory();
+    IndexWriter w = new IndexWriter(dir, new IndexWriterConfig(null));
+    w.addDocument(new Document());
+    w.addDocument(new Document());
+    w.close();
+
+    setIndexWriterMaxDocs(1);
+    try {       
+      new IndexWriter(dir, new IndexWriterConfig(null));
+      fail("didn't hit exception");
+    } catch (CorruptIndexException cie) {
+      // expected
+    } finally {
+      restoreIndexWriterMaxDocs();
+    }
+
+    dir.close();
+  }
 }

Modified: lucene/dev/branches/lucene6271/lucene/core/src/test/org/apache/lucene/index/TestIndexWriterMergePolicy.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene6271/lucene/core/src/test/org/apache/lucene/index/TestIndexWriterMergePolicy.java?rev=1670257&r1=1670256&r2=1670257&view=diff
==============================================================================
--- lucene/dev/branches/lucene6271/lucene/core/src/test/org/apache/lucene/index/TestIndexWriterMergePolicy.java (original)
+++ lucene/dev/branches/lucene6271/lucene/core/src/test/org/apache/lucene/index/TestIndexWriterMergePolicy.java Tue Mar 31 05:22:40 2015
@@ -248,7 +248,7 @@ public class TestIndexWriterMergePolicy
 
     int segmentCount = writer.getSegmentCount();
     for (int i = segmentCount - 1; i >= 0; i--) {
-      int docCount = writer.getDocCount(i);
+      int docCount = writer.maxDoc(i);
       assertTrue("docCount=" + docCount + " lowerBound=" + lowerBound + " upperBound=" + upperBound + " i=" + i + " segmentCount=" + segmentCount + " index=" + writer.segString() + " config=" + writer.getConfig(), docCount > lowerBound);
 
       if (docCount <= upperBound) {

Modified: lucene/dev/branches/lucene6271/lucene/core/src/test/org/apache/lucene/index/TestIndexWriterMerging.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene6271/lucene/core/src/test/org/apache/lucene/index/TestIndexWriterMerging.java?rev=1670257&r1=1670256&r2=1670257&view=diff
==============================================================================
--- lucene/dev/branches/lucene6271/lucene/core/src/test/org/apache/lucene/index/TestIndexWriterMerging.java (original)
+++ lucene/dev/branches/lucene6271/lucene/core/src/test/org/apache/lucene/index/TestIndexWriterMerging.java Tue Mar 31 05:22:40 2015
@@ -320,7 +320,7 @@ public class TestIndexWriterMerging exte
           break;
         }
         for(int i=0;i<merge.segments.size();i++) {
-          assert merge.segments.get(i).info.getDocCount() < 20;
+          assert merge.segments.get(i).info.maxDoc() < 20;
         }
         writer.merge(merge);
       }

Modified: lucene/dev/branches/lucene6271/lucene/core/src/test/org/apache/lucene/index/TestLogMergePolicy.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene6271/lucene/core/src/test/org/apache/lucene/index/TestLogMergePolicy.java?rev=1670257&r1=1670256&r2=1670257&view=diff
==============================================================================
--- lucene/dev/branches/lucene6271/lucene/core/src/test/org/apache/lucene/index/TestLogMergePolicy.java (original)
+++ lucene/dev/branches/lucene6271/lucene/core/src/test/org/apache/lucene/index/TestLogMergePolicy.java Tue Mar 31 05:22:40 2015
@@ -23,4 +23,8 @@ public class TestLogMergePolicy extends
     return newLogMergePolicy(random());
   }
 
+  public void testDefaultForcedMergeMB() {
+    LogByteSizeMergePolicy mp = new LogByteSizeMergePolicy();
+    assertTrue(mp.getMaxMergeMBForForcedMerge() > 0.0);
+  }
 }

Modified: lucene/dev/branches/lucene6271/lucene/core/src/test/org/apache/lucene/index/TestOmitTf.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene6271/lucene/core/src/test/org/apache/lucene/index/TestOmitTf.java?rev=1670257&r1=1670256&r2=1670257&view=diff
==============================================================================
--- lucene/dev/branches/lucene6271/lucene/core/src/test/org/apache/lucene/index/TestOmitTf.java (original)
+++ lucene/dev/branches/lucene6271/lucene/core/src/test/org/apache/lucene/index/TestOmitTf.java Tue Mar 31 05:22:40 2015
@@ -327,6 +327,10 @@ public class TestOmitTf extends LuceneTe
                     new CountingHitCollector() {
                       private Scorer scorer;
                       @Override
+                      public boolean needsScores() {
+                        return true;
+                      }
+                      @Override
                       public final void setScorer(Scorer scorer) {
                         this.scorer = scorer;
                       }
@@ -345,6 +349,10 @@ public class TestOmitTf extends LuceneTe
                     new CountingHitCollector() {
                       private Scorer scorer;
                       @Override
+                      public boolean needsScores() {
+                        return true;
+                      }
+                      @Override
                       public final void setScorer(Scorer scorer) {
                         this.scorer = scorer;
                       }
@@ -366,6 +374,10 @@ public class TestOmitTf extends LuceneTe
                     new CountingHitCollector() {
                       private Scorer scorer;
                       @Override
+                      public boolean needsScores() {
+                        return true;
+                      }
+                      @Override
                       public final void setScorer(Scorer scorer) {
                         this.scorer = scorer;
                       }
@@ -385,6 +397,10 @@ public class TestOmitTf extends LuceneTe
                     new CountingHitCollector() {
                       private Scorer scorer;
                       @Override
+                      public boolean needsScores() {
+                        return true;
+                      }
+                      @Override
                       public final void setScorer(Scorer scorer) {
                         this.scorer = scorer;
                       }

Modified: lucene/dev/branches/lucene6271/lucene/core/src/test/org/apache/lucene/index/TestSegmentMerger.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene6271/lucene/core/src/test/org/apache/lucene/index/TestSegmentMerger.java?rev=1670257&r1=1670256&r2=1670257&view=diff
==============================================================================
--- lucene/dev/branches/lucene6271/lucene/core/src/test/org/apache/lucene/index/TestSegmentMerger.java (original)
+++ lucene/dev/branches/lucene6271/lucene/core/src/test/org/apache/lucene/index/TestSegmentMerger.java Tue Mar 31 05:22:40 2015
@@ -19,6 +19,7 @@ package org.apache.lucene.index;
 
 import java.io.IOException;
 import java.util.Arrays;
+import java.util.Collections;
 import java.util.HashMap;
 
 import org.apache.lucene.codecs.Codec;
@@ -82,14 +83,14 @@ public class TestSegmentMerger extends L
 
   public void testMerge() throws IOException {
     final Codec codec = Codec.getDefault();
-    final SegmentInfo si = new SegmentInfo(mergedDir, Version.LATEST, mergedSegment, -1, false, codec, null, StringHelper.randomId(), new HashMap<>());
+    final SegmentInfo si = new SegmentInfo(mergedDir, Version.LATEST, mergedSegment, -1, false, codec, Collections.emptyMap(), StringHelper.randomId(), new HashMap<>());
 
     SegmentMerger merger = new SegmentMerger(Arrays.<CodecReader>asList(reader1, reader2),
                                              si, InfoStream.getDefault(), mergedDir,
                                              new FieldInfos.FieldNumbers(),
                                              newIOContext(random(), new IOContext(new MergeInfo(-1, -1, false, -1))));
     MergeState mergeState = merger.merge();
-    int docsMerged = mergeState.segmentInfo.getDocCount();
+    int docsMerged = mergeState.segmentInfo.maxDoc();
     assertTrue(docsMerged == 2);
     //Should be able to open a new SegmentReader against the new directory
     SegmentReader mergedReader = new SegmentReader(new SegmentCommitInfo(

Modified: lucene/dev/branches/lucene6271/lucene/core/src/test/org/apache/lucene/index/TestTermsEnum.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene6271/lucene/core/src/test/org/apache/lucene/index/TestTermsEnum.java?rev=1670257&r1=1670256&r2=1670257&view=diff
==============================================================================
--- lucene/dev/branches/lucene6271/lucene/core/src/test/org/apache/lucene/index/TestTermsEnum.java (original)
+++ lucene/dev/branches/lucene6271/lucene/core/src/test/org/apache/lucene/index/TestTermsEnum.java Tue Mar 31 05:22:40 2015
@@ -260,7 +260,7 @@ public class TestTermsEnum extends Lucen
         a = Automata.makeStringUnion(sortedAcceptTerms);
       }
       
-      final CompiledAutomaton c = new CompiledAutomaton(a, true, false, 1000000);
+      final CompiledAutomaton c = new CompiledAutomaton(a, true, false, 1000000, false);
 
       final BytesRef[] acceptTermsArray = new BytesRef[acceptTerms.size()];
       final Set<BytesRef> acceptTermsSet = new HashSet<>();

Modified: lucene/dev/branches/lucene6271/lucene/core/src/test/org/apache/lucene/search/TestAutomatonQuery.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene6271/lucene/core/src/test/org/apache/lucene/search/TestAutomatonQuery.java?rev=1670257&r1=1670256&r2=1670257&view=diff
==============================================================================
--- lucene/dev/branches/lucene6271/lucene/core/src/test/org/apache/lucene/search/TestAutomatonQuery.java (original)
+++ lucene/dev/branches/lucene6271/lucene/core/src/test/org/apache/lucene/search/TestAutomatonQuery.java Tue Mar 31 05:22:40 2015
@@ -96,13 +96,13 @@ public class TestAutomatonQuery extends
       throws IOException {
     AutomatonQuery query = new AutomatonQuery(newTerm("bogus"), automaton);
     
-    query.setRewriteMethod(MultiTermQuery.SCORING_BOOLEAN_QUERY_REWRITE);
+    query.setRewriteMethod(MultiTermQuery.SCORING_BOOLEAN_REWRITE);
     assertEquals(expected, automatonQueryNrHits(query));
     
-    query.setRewriteMethod(MultiTermQuery.CONSTANT_SCORE_FILTER_REWRITE);
+    query.setRewriteMethod(MultiTermQuery.CONSTANT_SCORE_REWRITE);
     assertEquals(expected, automatonQueryNrHits(query));
     
-    query.setRewriteMethod(MultiTermQuery.CONSTANT_SCORE_BOOLEAN_QUERY_REWRITE);
+    query.setRewriteMethod(MultiTermQuery.CONSTANT_SCORE_BOOLEAN_REWRITE);
     assertEquals(expected, automatonQueryNrHits(query));
   }
   
@@ -195,7 +195,6 @@ public class TestAutomatonQuery extends
     Automaton prefixAutomaton = Operations.concatenate(pfx, Automata.makeAnyString());
     AutomatonQuery aq = new AutomatonQuery(newTerm("bogus"), prefixAutomaton);
     Terms terms = MultiFields.getTerms(searcher.getIndexReader(), FN);
-    assertTrue(aq.getTermsEnum(terms) instanceof PrefixTermsEnum);
     assertEquals(3, automatonQueryNrHits(aq));
   }
   

Modified: lucene/dev/branches/lucene6271/lucene/core/src/test/org/apache/lucene/search/TestAutomatonQueryUnicode.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene6271/lucene/core/src/test/org/apache/lucene/search/TestAutomatonQueryUnicode.java?rev=1670257&r1=1670256&r2=1670257&view=diff
==============================================================================
--- lucene/dev/branches/lucene6271/lucene/core/src/test/org/apache/lucene/search/TestAutomatonQueryUnicode.java (original)
+++ lucene/dev/branches/lucene6271/lucene/core/src/test/org/apache/lucene/search/TestAutomatonQueryUnicode.java Tue Mar 31 05:22:40 2015
@@ -102,13 +102,13 @@ public class TestAutomatonQueryUnicode e
       throws IOException {
     AutomatonQuery query = new AutomatonQuery(newTerm("bogus"), automaton);
 
-    query.setRewriteMethod(MultiTermQuery.SCORING_BOOLEAN_QUERY_REWRITE);
+    query.setRewriteMethod(MultiTermQuery.SCORING_BOOLEAN_REWRITE);
     assertEquals(expected, automatonQueryNrHits(query));
 
-    query.setRewriteMethod(MultiTermQuery.CONSTANT_SCORE_FILTER_REWRITE);
+    query.setRewriteMethod(MultiTermQuery.CONSTANT_SCORE_REWRITE);
     assertEquals(expected, automatonQueryNrHits(query));
 
-    query.setRewriteMethod(MultiTermQuery.CONSTANT_SCORE_BOOLEAN_QUERY_REWRITE);
+    query.setRewriteMethod(MultiTermQuery.CONSTANT_SCORE_BOOLEAN_REWRITE);
     assertEquals(expected, automatonQueryNrHits(query));
   }