You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@lucene.apache.org by dw...@apache.org on 2014/04/04 12:27:14 UTC

svn commit: r1584603 [6/12] - in /lucene/dev/branches/solr5914: ./ dev-tools/ dev-tools/idea/solr/core/src/test/ lucene/ lucene/analysis/ lucene/analysis/common/ lucene/analysis/common/src/java/org/apache/lucene/analysis/br/ lucene/analysis/common/src/...

Modified: lucene/dev/branches/solr5914/lucene/core/src/java/org/apache/lucene/store/BufferedIndexOutput.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/solr5914/lucene/core/src/java/org/apache/lucene/store/BufferedIndexOutput.java?rev=1584603&r1=1584602&r2=1584603&view=diff
==============================================================================
--- lucene/dev/branches/solr5914/lucene/core/src/java/org/apache/lucene/store/BufferedIndexOutput.java (original)
+++ lucene/dev/branches/solr5914/lucene/core/src/java/org/apache/lucene/store/BufferedIndexOutput.java Fri Apr  4 10:27:05 2014
@@ -18,6 +18,7 @@ package org.apache.lucene.store;
  */
 
 import java.io.IOException;
+import java.util.zip.CRC32;
 
 /** Base implementation class for buffered {@link IndexOutput}. */
 public abstract class BufferedIndexOutput extends IndexOutput {
@@ -28,6 +29,7 @@ public abstract class BufferedIndexOutpu
   private final byte[] buffer;
   private long bufferStart = 0;           // position in file of buffer
   private int bufferPosition = 0;         // position in buffer
+  private final CRC32 crc = new CRC32();
 
   /**
    * Creates a new {@link BufferedIndexOutput} with the default buffer size
@@ -75,6 +77,7 @@ public abstract class BufferedIndexOutpu
         if (bufferPosition > 0)
           flush();
         // and write data at once
+        crc.update(b, offset, length);
         flushBuffer(b, offset, length);
         bufferStart += length;
       } else {
@@ -99,6 +102,7 @@ public abstract class BufferedIndexOutpu
 
   @Override
   public void flush() throws IOException {
+    crc.update(buffer, 0, bufferPosition);
     flushBuffer(buffer, bufferPosition);
     bufferStart += bufferPosition;
     bufferPosition = 0;
@@ -141,4 +145,9 @@ public abstract class BufferedIndexOutpu
     return bufferSize;
   }
 
+  @Override
+  public long getChecksum() throws IOException {
+    flush();
+    return crc.getValue();
+  }
 }

Modified: lucene/dev/branches/solr5914/lucene/core/src/java/org/apache/lucene/store/ChecksumIndexInput.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/solr5914/lucene/core/src/java/org/apache/lucene/store/ChecksumIndexInput.java?rev=1584603&r1=1584602&r2=1584603&view=diff
==============================================================================
--- lucene/dev/branches/solr5914/lucene/core/src/java/org/apache/lucene/store/ChecksumIndexInput.java (original)
+++ lucene/dev/branches/solr5914/lucene/core/src/java/org/apache/lucene/store/ChecksumIndexInput.java Fri Apr  4 10:27:05 2014
@@ -1,5 +1,7 @@
 package org.apache.lucene.store;
 
+import java.io.IOException;
+
 /*
  * Licensed to the Apache Software Foundation (ASF) under one or more
  * contributor license agreements.  See the NOTICE file distributed with
@@ -17,61 +19,24 @@ package org.apache.lucene.store;
  * limitations under the License.
  */
 
-import java.io.IOException;
-import java.util.zip.CRC32;
-import java.util.zip.Checksum;
-
-/** Reads bytes through to a primary IndexInput, computing
- *  checksum as it goes. Note that you cannot use seek().
- *
- * @lucene.internal
+/** 
+ * Extension of IndexInput, computing checksum as it goes. 
+ * Callers can retrieve the checksum via {@link #getChecksum()}.
  */
-public class ChecksumIndexInput extends IndexInput {
-  IndexInput main;
-  Checksum digest;
-
-  public ChecksumIndexInput(IndexInput main) {
-    super("ChecksumIndexInput(" + main + ")");
-    this.main = main;
-    digest = new CRC32();
-  }
-
-  @Override
-  public byte readByte() throws IOException {
-    final byte b = main.readByte();
-    digest.update(b);
-    return b;
-  }
-
-  @Override
-  public void readBytes(byte[] b, int offset, int len)
-    throws IOException {
-    main.readBytes(b, offset, len);
-    digest.update(b, offset, len);
-  }
-
+public abstract class ChecksumIndexInput extends IndexInput {
   
-  public long getChecksum() {
-    return digest.getValue();
+  /** resourceDescription should be a non-null, opaque string
+   *  describing this resource; it's returned from
+   *  {@link #toString}. */
+  protected ChecksumIndexInput(String resourceDescription) {
+    super(resourceDescription);
   }
 
-  @Override
-  public void close() throws IOException {
-    main.close();
-  }
-
-  @Override
-  public long getFilePointer() {
-    return main.getFilePointer();
-  }
+  /** Returns the current checksum value */
+  public abstract long getChecksum() throws IOException;
 
   @Override
   public void seek(long pos) {
     throw new UnsupportedOperationException();
   }
-
-  @Override
-  public long length() {
-    return main.length();
-  }
 }

Modified: lucene/dev/branches/solr5914/lucene/core/src/java/org/apache/lucene/store/CompoundFileDirectory.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/solr5914/lucene/core/src/java/org/apache/lucene/store/CompoundFileDirectory.java?rev=1584603&r1=1584602&r2=1584603&view=diff
==============================================================================
--- lucene/dev/branches/solr5914/lucene/core/src/java/org/apache/lucene/store/CompoundFileDirectory.java (original)
+++ lucene/dev/branches/solr5914/lucene/core/src/java/org/apache/lucene/store/CompoundFileDirectory.java Fri Apr  4 10:27:05 2014
@@ -52,14 +52,15 @@ import java.io.IOException;
  * </ul>
  * <p>Description:</p>
  * <ul>
- *   <li>Compound (.cfs) --&gt; Header, FileData <sup>FileCount</sup></li>
+ *   <li>Compound (.cfs) --&gt; Header, FileData <sup>FileCount</sup>, Footer</li>
  *   <li>Compound Entry Table (.cfe) --&gt; Header, FileCount, &lt;FileName,
  *       DataOffset, DataLength&gt; <sup>FileCount</sup></li>
  *   <li>Header --&gt; {@link CodecUtil#writeHeader CodecHeader}</li>
  *   <li>FileCount --&gt; {@link DataOutput#writeVInt VInt}</li>
- *   <li>DataOffset,DataLength --&gt; {@link DataOutput#writeLong UInt64}</li>
+ *   <li>DataOffset,DataLength,Checksum --&gt; {@link DataOutput#writeLong UInt64}</li>
  *   <li>FileName --&gt; {@link DataOutput#writeString String}</li>
  *   <li>FileData --&gt; raw file data</li>
+ *   <li>Footer --&gt; {@link CodecUtil#writeFooter CodecFooter}</li>
  * </ul>
  * <p>Notes:</p>
  * <ul>
@@ -87,6 +88,7 @@ public final class CompoundFileDirectory
   private static final Map<String,FileEntry> SENTINEL = Collections.emptyMap();
   private final CompoundFileWriter writer;
   private final IndexInputSlicer handle;
+  private int version;
   
   /**
    * Create a new CompoundFileDirectory.
@@ -120,15 +122,15 @@ public final class CompoundFileDirectory
   }
 
   /** Helper method that reads CFS entries from an input stream */
-  private static final Map<String, FileEntry> readEntries(Directory dir, String name) throws IOException {
+  private final Map<String, FileEntry> readEntries(Directory dir, String name) throws IOException {
     IOException priorE = null;
-    IndexInput entriesStream = null;
+    ChecksumIndexInput entriesStream = null;
     try {
       final String entriesFileName = IndexFileNames.segmentFileName(
                                             IndexFileNames.stripExtension(name), "",
                                              IndexFileNames.COMPOUND_FILE_ENTRIES_EXTENSION);
-      entriesStream = dir.openInput(entriesFileName, IOContext.READONCE);
-      CodecUtil.checkHeader(entriesStream, CompoundFileWriter.ENTRY_CODEC, CompoundFileWriter.VERSION_START, CompoundFileWriter.VERSION_START);
+      entriesStream = dir.openChecksumInput(entriesFileName, IOContext.READONCE);
+      version = CodecUtil.checkHeader(entriesStream, CompoundFileWriter.ENTRY_CODEC, CompoundFileWriter.VERSION_START, CompoundFileWriter.VERSION_CURRENT);
       final int numEntries = entriesStream.readVInt();
       final Map<String, FileEntry> mapping = new HashMap<>(numEntries);
       for (int i = 0; i < numEntries; i++) {
@@ -141,8 +143,10 @@ public final class CompoundFileDirectory
         fileEntry.offset = entriesStream.readLong();
         fileEntry.length = entriesStream.readLong();
       }
-      if (entriesStream.getFilePointer() != entriesStream.length()) {
-        throw new CorruptIndexException("did not read all bytes from file \"" + entriesFileName + "\": read " + entriesStream.getFilePointer() + " vs size " + entriesStream.length() + " (resource: " + entriesStream + ")");
+      if (version >= CompoundFileWriter.VERSION_CHECKSUM) {
+        CodecUtil.checkFooter(entriesStream);
+      } else {
+        CodecUtil.checkEOF(entriesStream);
       }
       return mapping;
     } catch (IOException ioe) {

Modified: lucene/dev/branches/solr5914/lucene/core/src/java/org/apache/lucene/store/CompoundFileWriter.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/solr5914/lucene/core/src/java/org/apache/lucene/store/CompoundFileWriter.java?rev=1584603&r1=1584602&r2=1584603&view=diff
==============================================================================
--- lucene/dev/branches/solr5914/lucene/core/src/java/org/apache/lucene/store/CompoundFileWriter.java (original)
+++ lucene/dev/branches/solr5914/lucene/core/src/java/org/apache/lucene/store/CompoundFileWriter.java Fri Apr  4 10:27:05 2014
@@ -54,7 +54,8 @@ final class CompoundFileWriter implement
   // versioning for the .cfs file
   static final String DATA_CODEC = "CompoundFileWriterData";
   static final int VERSION_START = 0;
-  static final int VERSION_CURRENT = VERSION_START;
+  static final int VERSION_CHECKSUM = 1;
+  static final int VERSION_CURRENT = VERSION_CHECKSUM;
 
   // versioning for the .cfe file
   static final String ENTRY_CODEC = "CompoundFileWriterEntries";
@@ -140,6 +141,7 @@ final class CompoundFileWriter implement
       // open the compound stream
       getOutput();
       assert dataOut != null;
+      CodecUtil.writeFooter(dataOut);
     } catch (IOException e) {
       priorException = e;
     } finally {
@@ -202,6 +204,7 @@ final class CompoundFileWriter implement
       entryOut.writeLong(fe.offset);
       entryOut.writeLong(fe.length);
     }
+    CodecUtil.writeFooter(entryOut);
   }
 
   IndexOutput createOutput(String name, IOContext context) throws IOException {
@@ -342,6 +345,11 @@ final class CompoundFileWriter implement
       writtenBytes += length;
       delegate.writeBytes(b, offset, length);
     }
+
+    @Override
+    public long getChecksum() throws IOException {
+      return delegate.getChecksum();
+    }
   }
 
 }

Modified: lucene/dev/branches/solr5914/lucene/core/src/java/org/apache/lucene/store/DataInput.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/solr5914/lucene/core/src/java/org/apache/lucene/store/DataInput.java?rev=1584603&r1=1584602&r2=1584603&view=diff
==============================================================================
--- lucene/dev/branches/solr5914/lucene/core/src/java/org/apache/lucene/store/DataInput.java (original)
+++ lucene/dev/branches/solr5914/lucene/core/src/java/org/apache/lucene/store/DataInput.java Fri Apr  4 10:27:05 2014
@@ -18,13 +18,12 @@ package org.apache.lucene.store;
  */
 
 import java.io.IOException;
+import java.nio.charset.StandardCharsets;
 import java.util.HashMap;
 import java.util.HashSet;
 import java.util.Map;
 import java.util.Set;
 
-import org.apache.lucene.util.IOUtils;
-
 /**
  * Abstract base class for performing read operations of Lucene's low-level
  * data types.
@@ -188,7 +187,7 @@ public abstract class DataInput implemen
     int length = readVInt();
     final byte[] bytes = new byte[length];
     readBytes(bytes, 0, length);
-    return new String(bytes, 0, length, IOUtils.CHARSET_UTF_8);
+    return new String(bytes, 0, length, StandardCharsets.UTF_8);
   }
 
   /** Returns a clone of this stream.

Modified: lucene/dev/branches/solr5914/lucene/core/src/java/org/apache/lucene/store/Directory.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/solr5914/lucene/core/src/java/org/apache/lucene/store/Directory.java?rev=1584603&r1=1584602&r2=1584603&view=diff
==============================================================================
--- lucene/dev/branches/solr5914/lucene/core/src/java/org/apache/lucene/store/Directory.java (original)
+++ lucene/dev/branches/solr5914/lucene/core/src/java/org/apache/lucene/store/Directory.java Fri Apr  4 10:27:05 2014
@@ -100,7 +100,12 @@ public abstract class Directory implemen
    * <p>Throws {@link FileNotFoundException} or {@link NoSuchFileException}
    * if the file does not exist.
    */
-  public abstract IndexInput openInput(String name, IOContext context) throws IOException; 
+  public abstract IndexInput openInput(String name, IOContext context) throws IOException;
+  
+  /** Returns a stream reading an existing file, computing checksum as it reads */
+  public ChecksumIndexInput openChecksumInput(String name, IOContext context) throws IOException {
+    return new BufferedChecksumIndexInput(openInput(name, context));
+  }
   
   /** Construct a {@link Lock}.
    * @param name the name of the lock file

Modified: lucene/dev/branches/solr5914/lucene/core/src/java/org/apache/lucene/store/IndexOutput.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/solr5914/lucene/core/src/java/org/apache/lucene/store/IndexOutput.java?rev=1584603&r1=1584602&r2=1584603&view=diff
==============================================================================
--- lucene/dev/branches/solr5914/lucene/core/src/java/org/apache/lucene/store/IndexOutput.java (original)
+++ lucene/dev/branches/solr5914/lucene/core/src/java/org/apache/lucene/store/IndexOutput.java Fri Apr  4 10:27:05 2014
@@ -43,6 +43,8 @@ public abstract class IndexOutput extend
    */
   public abstract long getFilePointer();
 
+  /** Returns the current checksum of bytes written so far */
+  public abstract long getChecksum() throws IOException;
 
   /** The number of bytes in the file. */
   public abstract long length() throws IOException;

Modified: lucene/dev/branches/solr5914/lucene/core/src/java/org/apache/lucene/store/RAMOutputStream.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/solr5914/lucene/core/src/java/org/apache/lucene/store/RAMOutputStream.java?rev=1584603&r1=1584602&r2=1584603&view=diff
==============================================================================
--- lucene/dev/branches/solr5914/lucene/core/src/java/org/apache/lucene/store/RAMOutputStream.java (original)
+++ lucene/dev/branches/solr5914/lucene/core/src/java/org/apache/lucene/store/RAMOutputStream.java Fri Apr  4 10:27:05 2014
@@ -18,6 +18,8 @@ package org.apache.lucene.store;
  */
 
 import java.io.IOException;
+import java.util.zip.CRC32;
+import java.util.zip.Checksum;
 
 /**
  * A memory-resident {@link IndexOutput} implementation.
@@ -35,6 +37,8 @@ public class RAMOutputStream extends Ind
   private int bufferPosition;
   private long bufferStart;
   private int bufferLength;
+  
+  private Checksum crc = new BufferedChecksum(new CRC32());
 
   /** Construct an empty output buffer. */
   public RAMOutputStream() {
@@ -95,6 +99,7 @@ public class RAMOutputStream extends Ind
     bufferStart = 0;
     bufferLength = 0;
     file.setLength(0);
+    crc.reset();
   }
 
   @Override
@@ -113,12 +118,14 @@ public class RAMOutputStream extends Ind
       currentBufferIndex++;
       switchCurrentBuffer();
     }
+    crc.update(b);
     currentBuffer[bufferPosition++] = b;
   }
 
   @Override
   public void writeBytes(byte[] b, int offset, int len) throws IOException {
     assert b != null;
+    crc.update(b, offset, len);
     while (len > 0) {
       if (bufferPosition ==  bufferLength) {
         currentBufferIndex++;
@@ -165,5 +172,10 @@ public class RAMOutputStream extends Ind
   /** Returns byte usage of all buffers. */
   public long sizeInBytes() {
     return (long) file.numBuffers() * (long) BUFFER_SIZE;
-  }  
+  }
+
+  @Override
+  public long getChecksum() throws IOException {
+    return crc.getValue();
+  }
 }

Modified: lucene/dev/branches/solr5914/lucene/core/src/java/org/apache/lucene/util/IOUtils.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/solr5914/lucene/core/src/java/org/apache/lucene/util/IOUtils.java?rev=1584603&r1=1584602&r2=1584603&view=diff
==============================================================================
--- lucene/dev/branches/solr5914/lucene/core/src/java/org/apache/lucene/util/IOUtils.java (original)
+++ lucene/dev/branches/solr5914/lucene/core/src/java/org/apache/lucene/util/IOUtils.java Fri Apr  4 10:27:05 2014
@@ -31,6 +31,7 @@ import java.io.Reader;
 import java.nio.charset.Charset;
 import java.nio.charset.CharsetDecoder;
 import java.nio.charset.CodingErrorAction;
+import java.nio.charset.StandardCharsets;
 
 /** This class emulates the new Java 7 "Try-With-Resources" statement.
  * Remove once Lucene is on Java 7.
@@ -38,16 +39,21 @@ import java.nio.charset.CodingErrorActio
 public final class IOUtils {
   
   /**
-   * UTF-8 charset string
-   * @see Charset#forName(String)
+   * UTF-8 {@link Charset} instance to prevent repeated
+   * {@link Charset#forName(String)} lookups
+   * @deprecated Use {@link StandardCharsets#UTF_8} instead.
    */
-  public static final String UTF_8 = "UTF-8";
+  @Deprecated
+  public static final Charset CHARSET_UTF_8 = StandardCharsets.UTF_8;
   
   /**
-   * UTF-8 {@link Charset} instance to prevent repeated
-   * {@link Charset#forName(String)} lookups
+   * UTF-8 charset string.
+   * <p>Where possible, use {@link StandardCharsets#UTF_8} instead,
+   * as using the String constant may slow things down.
+   * @see StandardCharsets#UTF_8
    */
-  public static final Charset CHARSET_UTF_8 = Charset.forName("UTF-8");
+  public static final String UTF_8 = StandardCharsets.UTF_8.name();
+  
   private IOUtils() {} // no instance
 
   /**

Modified: lucene/dev/branches/solr5914/lucene/core/src/java/org/apache/lucene/util/SPIClassIterator.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/solr5914/lucene/core/src/java/org/apache/lucene/util/SPIClassIterator.java?rev=1584603&r1=1584602&r2=1584603&view=diff
==============================================================================
--- lucene/dev/branches/solr5914/lucene/core/src/java/org/apache/lucene/util/SPIClassIterator.java (original)
+++ lucene/dev/branches/solr5914/lucene/core/src/java/org/apache/lucene/util/SPIClassIterator.java Fri Apr  4 10:27:05 2014
@@ -22,6 +22,7 @@ import java.io.InputStream;
 import java.io.BufferedReader;
 import java.io.InputStreamReader;
 import java.net.URL;
+import java.nio.charset.StandardCharsets;
 import java.util.ArrayList;
 import java.util.Collections;
 import java.util.Enumeration;
@@ -91,7 +92,7 @@ public final class SPIClassIterator<S> i
         final InputStream in = url.openStream();
         IOException priorE = null;
         try {
-          final BufferedReader reader = new BufferedReader(new InputStreamReader(in, IOUtils.CHARSET_UTF_8));
+          final BufferedReader reader = new BufferedReader(new InputStreamReader(in, StandardCharsets.UTF_8));
           String line;
           while ((line = reader.readLine()) != null) {
             final int pos = line.indexOf('#');

Modified: lucene/dev/branches/solr5914/lucene/core/src/java/org/apache/lucene/util/UnicodeUtil.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/solr5914/lucene/core/src/java/org/apache/lucene/util/UnicodeUtil.java?rev=1584603&r1=1584602&r2=1584603&view=diff
==============================================================================
--- lucene/dev/branches/solr5914/lucene/core/src/java/org/apache/lucene/util/UnicodeUtil.java (original)
+++ lucene/dev/branches/solr5914/lucene/core/src/java/org/apache/lucene/util/UnicodeUtil.java Fri Apr  4 10:27:05 2014
@@ -1,5 +1,7 @@
 package org.apache.lucene.util;
 
+import java.nio.charset.StandardCharsets;
+
 /*
  * Licensed to the Apache Software Foundation (ASF) under one or more
  * contributor license agreements.  See the NOTICE file distributed with
@@ -88,7 +90,7 @@ package org.apache.lucene.util;
 /**
  * Class to encode java's UTF16 char[] into UTF8 byte[]
  * without always allocating a new byte[] as
- * String.getBytes("UTF-8") does.
+ * String.getBytes(StandardCharsets.UTF_8) does.
  *
  * @lucene.internal
  */
@@ -292,7 +294,7 @@ public final class UnicodeUtil {
   private static boolean matches(char[] source, int offset, int length, byte[] result, int upto) {
     try {
       String s1 = new String(source, offset, length);
-      String s2 = new String(result, 0, upto, "UTF-8");
+      String s2 = new String(result, 0, upto, StandardCharsets.UTF_8);
       if (!s1.equals(s2)) {
         //System.out.println("DIFF: s1 len=" + s1.length());
         //for(int i=0;i<s1.length();i++)
@@ -318,7 +320,7 @@ public final class UnicodeUtil {
   private static boolean matches(String source, int offset, int length, byte[] result, int upto) {
     try {
       String s1 = source.substring(offset, offset+length);
-      String s2 = new String(result, 0, upto, "UTF-8");
+      String s2 = new String(result, 0, upto, StandardCharsets.UTF_8);
       if (!s1.equals(s2)) {
         // Allow a difference if s1 is not valid UTF-16
 

Modified: lucene/dev/branches/solr5914/lucene/core/src/java/org/apache/lucene/util/fst/FST.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/solr5914/lucene/core/src/java/org/apache/lucene/util/fst/FST.java?rev=1584603&r1=1584602&r2=1584603&view=diff
==============================================================================
--- lucene/dev/branches/solr5914/lucene/core/src/java/org/apache/lucene/util/fst/FST.java (original)
+++ lucene/dev/branches/solr5914/lucene/core/src/java/org/apache/lucene/util/fst/FST.java Fri Apr  4 10:27:05 2014
@@ -25,6 +25,7 @@ import java.io.FileOutputStream;
 import java.io.IOException;
 import java.io.InputStream;
 import java.io.OutputStream;
+import java.nio.charset.StandardCharsets;
 import java.util.HashMap;
 import java.util.Map;
 
@@ -379,7 +380,7 @@ public final class FST<T> {
 
     /*
     if (bytes.length == 665) {
-      Writer w = new OutputStreamWriter(new FileOutputStream("out.dot"), "UTF-8");
+      Writer w = new OutputStreamWriter(new FileOutputStream("out.dot"), StandardCharsets.UTF_8);
       Util.toDot(this, w, false, false);
       w.close();
       System.out.println("Wrote FST to out.dot");

Modified: lucene/dev/branches/solr5914/lucene/core/src/test/org/apache/lucene/codecs/compressing/AbstractTestLZ4CompressionMode.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/solr5914/lucene/core/src/test/org/apache/lucene/codecs/compressing/AbstractTestLZ4CompressionMode.java?rev=1584603&r1=1584602&r2=1584603&view=diff
==============================================================================
--- lucene/dev/branches/solr5914/lucene/core/src/test/org/apache/lucene/codecs/compressing/AbstractTestLZ4CompressionMode.java (original)
+++ lucene/dev/branches/solr5914/lucene/core/src/test/org/apache/lucene/codecs/compressing/AbstractTestLZ4CompressionMode.java Fri Apr  4 10:27:05 2014
@@ -18,6 +18,7 @@ package org.apache.lucene.codecs.compres
  */
 
 import java.io.IOException;
+import java.nio.charset.StandardCharsets;
 
 import com.carrotsearch.randomizedtesting.generators.RandomInts;
 
@@ -81,7 +82,7 @@ public abstract class AbstractTestLZ4Com
 
   public void testShortLiteralsAndMatchs() throws IOException {
     // literals and matchs lengths <= 15
-    final byte[] decompressed = "1234562345673456745678910123".getBytes("UTF-8");
+    final byte[] decompressed = "1234562345673456745678910123".getBytes(StandardCharsets.UTF_8);
     test(decompressed);
   }
 

Modified: lucene/dev/branches/solr5914/lucene/core/src/test/org/apache/lucene/codecs/perfield/TestPerFieldPostingsFormat2.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/solr5914/lucene/core/src/test/org/apache/lucene/codecs/perfield/TestPerFieldPostingsFormat2.java?rev=1584603&r1=1584602&r2=1584603&view=diff
==============================================================================
--- lucene/dev/branches/solr5914/lucene/core/src/test/org/apache/lucene/codecs/perfield/TestPerFieldPostingsFormat2.java (original)
+++ lucene/dev/branches/solr5914/lucene/core/src/test/org/apache/lucene/codecs/perfield/TestPerFieldPostingsFormat2.java Fri Apr  4 10:27:05 2014
@@ -23,7 +23,7 @@ import org.apache.lucene.codecs.Codec;
 import org.apache.lucene.codecs.PostingsFormat;
 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.memory.MemoryPostingsFormat;
 import org.apache.lucene.codecs.pulsing.Pulsing41PostingsFormat;
 import org.apache.lucene.codecs.simpletext.SimpleTextPostingsFormat;
 import org.apache.lucene.document.Document;
@@ -44,7 +44,6 @@ import org.apache.lucene.search.TopDocs;
 import org.apache.lucene.store.Directory;
 import org.apache.lucene.util.LuceneTestCase;
 import org.apache.lucene.util.TestUtil;
-import org.apache.lucene.util.TestUtil;
 import org.junit.Test;
 
 /**
@@ -204,14 +203,14 @@ public class TestPerFieldPostingsFormat2
   public static class MockCodec extends Lucene46Codec {
     final PostingsFormat lucene40 = new Lucene41PostingsFormat();
     final PostingsFormat simpleText = new SimpleTextPostingsFormat();
-    final PostingsFormat mockSep = new MockSepPostingsFormat();
+    final PostingsFormat memory = new MemoryPostingsFormat();
     
     @Override
     public PostingsFormat getPostingsFormatForField(String field) {
       if (field.equals("id")) {
         return simpleText;
       } else if (field.equals("content")) {
-        return mockSep;
+        return memory;
       } else {
         return lucene40;
       }

Modified: lucene/dev/branches/solr5914/lucene/core/src/test/org/apache/lucene/document/TestBinaryDocument.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/solr5914/lucene/core/src/test/org/apache/lucene/document/TestBinaryDocument.java?rev=1584603&r1=1584602&r2=1584603&view=diff
==============================================================================
--- lucene/dev/branches/solr5914/lucene/core/src/test/org/apache/lucene/document/TestBinaryDocument.java (original)
+++ lucene/dev/branches/solr5914/lucene/core/src/test/org/apache/lucene/document/TestBinaryDocument.java Fri Apr  4 10:27:05 2014
@@ -1,5 +1,7 @@
 package org.apache.lucene.document;
 
+import java.nio.charset.StandardCharsets;
+
 import org.apache.lucene.index.IndexReader;
 import org.apache.lucene.index.IndexableField;
 import org.apache.lucene.index.RandomIndexWriter;
@@ -38,7 +40,7 @@ public class TestBinaryDocument extends 
   {
     FieldType ft = new FieldType();
     ft.setStored(true);
-    StoredField binaryFldStored = new StoredField("binaryStored", binaryValStored.getBytes("UTF-8"));
+    StoredField binaryFldStored = new StoredField("binaryStored", binaryValStored.getBytes(StandardCharsets.UTF_8));
     Field stringFldStored = new Field("stringStored", binaryValStored, ft);
 
     Document doc = new Document();
@@ -63,7 +65,7 @@ public class TestBinaryDocument extends 
     /** fetch the binary stored field and compare it's content with the original one */
     BytesRef bytes = docFromReader.getBinaryValue("binaryStored");
     assertNotNull(bytes);
-    String binaryFldStoredTest = new String(bytes.bytes, bytes.offset, bytes.length, "UTF-8");
+    String binaryFldStoredTest = new String(bytes.bytes, bytes.offset, bytes.length, StandardCharsets.UTF_8);
     assertTrue(binaryFldStoredTest.equals(binaryValStored));
     
     /** fetch the string field and compare it's content with the original one */
@@ -76,7 +78,7 @@ public class TestBinaryDocument extends 
   }
   
   public void testCompressionTools() throws Exception {
-    StoredField binaryFldCompressed = new StoredField("binaryCompressed", CompressionTools.compress(binaryValCompressed.getBytes("UTF-8")));
+    StoredField binaryFldCompressed = new StoredField("binaryCompressed", CompressionTools.compress(binaryValCompressed.getBytes(StandardCharsets.UTF_8)));
     StoredField stringFldCompressed = new StoredField("stringCompressed", CompressionTools.compressString(binaryValCompressed));
     
     Document doc = new Document();
@@ -95,7 +97,7 @@ public class TestBinaryDocument extends 
     assertTrue(docFromReader != null);
     
     /** fetch the binary compressed field and compare it's content with the original one */
-    String binaryFldCompressedTest = new String(CompressionTools.decompress(docFromReader.getBinaryValue("binaryCompressed")), "UTF-8");
+    String binaryFldCompressedTest = new String(CompressionTools.decompress(docFromReader.getBinaryValue("binaryCompressed")), StandardCharsets.UTF_8);
     assertTrue(binaryFldCompressedTest.equals(binaryValCompressed));
     assertTrue(CompressionTools.decompressString(docFromReader.getBinaryValue("stringCompressed")).equals(binaryValCompressed));
 

Modified: lucene/dev/branches/solr5914/lucene/core/src/test/org/apache/lucene/document/TestDocument.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/solr5914/lucene/core/src/test/org/apache/lucene/document/TestDocument.java?rev=1584603&r1=1584602&r2=1584603&view=diff
==============================================================================
--- lucene/dev/branches/solr5914/lucene/core/src/test/org/apache/lucene/document/TestDocument.java (original)
+++ lucene/dev/branches/solr5914/lucene/core/src/test/org/apache/lucene/document/TestDocument.java Fri Apr  4 10:27:05 2014
@@ -19,6 +19,7 @@ package org.apache.lucene.document;
 
 import java.io.IOException;
 import java.io.StringReader;
+import java.nio.charset.StandardCharsets;
 import java.util.List;
 
 import org.apache.lucene.analysis.MockTokenizer;
@@ -53,8 +54,8 @@ public class TestDocument extends Lucene
     FieldType ft = new FieldType();
     ft.setStored(true);
     Field stringFld = new Field("string", binaryVal, ft);
-    StoredField binaryFld = new StoredField("binary", binaryVal.getBytes("UTF-8"));
-    StoredField binaryFld2 = new StoredField("binary", binaryVal2.getBytes("UTF-8"));
+    StoredField binaryFld = new StoredField("binary", binaryVal.getBytes(StandardCharsets.UTF_8));
+    StoredField binaryFld2 = new StoredField("binary", binaryVal2.getBytes(StandardCharsets.UTF_8));
     
     doc.add(stringFld);
     doc.add(binaryFld);

Modified: lucene/dev/branches/solr5914/lucene/core/src/test/org/apache/lucene/document/TestField.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/solr5914/lucene/core/src/test/org/apache/lucene/document/TestField.java?rev=1584603&r1=1584602&r2=1584603&view=diff
==============================================================================
--- lucene/dev/branches/solr5914/lucene/core/src/test/org/apache/lucene/document/TestField.java (original)
+++ lucene/dev/branches/solr5914/lucene/core/src/test/org/apache/lucene/document/TestField.java Fri Apr  4 10:27:05 2014
@@ -18,6 +18,8 @@ package org.apache.lucene.document;
  */
 
 import java.io.StringReader;
+import java.nio.charset.StandardCharsets;
+
 import org.apache.lucene.analysis.CannedTokenStream;
 import org.apache.lucene.analysis.Token;
 import org.apache.lucene.util.BytesRef;
@@ -184,7 +186,7 @@ public class TestField extends LuceneTes
 
     trySetBoost(field);
     trySetByteValue(field);
-    field.setBytesValue("fubar".getBytes("UTF-8"));
+    field.setBytesValue("fubar".getBytes(StandardCharsets.UTF_8));
     field.setBytesValue(new BytesRef("baz"));
     trySetDoubleValue(field);
     trySetIntValue(field);
@@ -203,7 +205,7 @@ public class TestField extends LuceneTes
 
     trySetBoost(field);
     trySetByteValue(field);
-    field.setBytesValue("fubar".getBytes("UTF-8"));
+    field.setBytesValue("fubar".getBytes(StandardCharsets.UTF_8));
     field.setBytesValue(new BytesRef("baz"));
     trySetDoubleValue(field);
     trySetIntValue(field);
@@ -294,15 +296,15 @@ public class TestField extends LuceneTes
   
   public void testStoredFieldBytes() throws Exception {
     Field fields[] = new Field[] {
-        new StoredField("foo", "bar".getBytes("UTF-8")),
-        new StoredField("foo", "bar".getBytes("UTF-8"), 0, 3),
+        new StoredField("foo", "bar".getBytes(StandardCharsets.UTF_8)),
+        new StoredField("foo", "bar".getBytes(StandardCharsets.UTF_8), 0, 3),
         new StoredField("foo", new BytesRef("bar")),
     };
     
     for (Field field : fields) {
       trySetBoost(field);
       trySetByteValue(field);
-      field.setBytesValue("baz".getBytes("UTF-8"));
+      field.setBytesValue("baz".getBytes(StandardCharsets.UTF_8));
       field.setBytesValue(new BytesRef("baz"));
       trySetDoubleValue(field);
       trySetIntValue(field);

Modified: lucene/dev/branches/solr5914/lucene/core/src/test/org/apache/lucene/index/TestAddIndexes.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/solr5914/lucene/core/src/test/org/apache/lucene/index/TestAddIndexes.java?rev=1584603&r1=1584602&r2=1584603&view=diff
==============================================================================
--- lucene/dev/branches/solr5914/lucene/core/src/test/org/apache/lucene/index/TestAddIndexes.java (original)
+++ lucene/dev/branches/solr5914/lucene/core/src/test/org/apache/lucene/index/TestAddIndexes.java Fri Apr  4 10:27:05 2014
@@ -1064,14 +1064,14 @@ public class TestAddIndexes extends Luce
   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");
+    private final PostingsFormat memoryFormat = PostingsFormat.forName("Memory");
 
     @Override
     public PostingsFormat getPostingsFormatForField(String field) {
       if (field.equals("id")) {
         return simpleTextFormat;
       } else if (field.equals("content")) {
-        return mockSepFormat;
+        return memoryFormat;
       } else {
         return defaultFormat;
       }

Modified: lucene/dev/branches/solr5914/lucene/core/src/test/org/apache/lucene/index/TestAllFilesHaveCodecHeader.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/solr5914/lucene/core/src/test/org/apache/lucene/index/TestAllFilesHaveCodecHeader.java?rev=1584603&r1=1584602&r2=1584603&view=diff
==============================================================================
--- lucene/dev/branches/solr5914/lucene/core/src/test/org/apache/lucene/index/TestAllFilesHaveCodecHeader.java (original)
+++ lucene/dev/branches/solr5914/lucene/core/src/test/org/apache/lucene/index/TestAllFilesHaveCodecHeader.java Fri Apr  4 10:27:05 2014
@@ -24,6 +24,7 @@ import org.apache.lucene.codecs.CodecUti
 import org.apache.lucene.codecs.lucene46.Lucene46Codec;
 import org.apache.lucene.document.Document;
 import org.apache.lucene.document.Field;
+import org.apache.lucene.document.NumericDocValuesField;
 import org.apache.lucene.store.CompoundFileDirectory;
 import org.apache.lucene.store.Directory;
 import org.apache.lucene.store.IndexInput;
@@ -39,14 +40,15 @@ public class TestAllFilesHaveCodecHeader
     Directory dir = newDirectory();
     IndexWriterConfig conf = newIndexWriterConfig(TEST_VERSION_CURRENT, new MockAnalyzer(random()));
     conf.setCodec(new Lucene46Codec());
-    // riw should sometimes create docvalues fields, etc
     RandomIndexWriter riw = new RandomIndexWriter(random(), dir, conf);
     Document doc = new Document();
     // these fields should sometimes get term vectors, etc
     Field idField = newStringField("id", "", Field.Store.NO);
     Field bodyField = newTextField("body", "", Field.Store.NO);
+    Field dvField = new NumericDocValuesField("dv", 5);
     doc.add(idField);
     doc.add(bodyField);
+    doc.add(dvField);
     for (int i = 0; i < 100; i++) {
       idField.setStringValue(Integer.toString(i));
       bodyField.setStringValue(TestUtil.randomUnicodeString(random()));
@@ -54,6 +56,10 @@ public class TestAllFilesHaveCodecHeader
       if (random().nextInt(7) == 0) {
         riw.commit();
       }
+      // TODO: we should make a new format with a clean header...
+      // if (random().nextInt(20) == 0) {
+      //  riw.deleteDocuments(new Term("id", Integer.toString(i)));
+      // }
     }
     riw.close();
     checkHeaders(dir);

Modified: lucene/dev/branches/solr5914/lucene/core/src/test/org/apache/lucene/index/TestBackwardsCompatibility.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/solr5914/lucene/core/src/test/org/apache/lucene/index/TestBackwardsCompatibility.java?rev=1584603&r1=1584602&r2=1584603&view=diff
==============================================================================
--- lucene/dev/branches/solr5914/lucene/core/src/test/org/apache/lucene/index/TestBackwardsCompatibility.java (original)
+++ lucene/dev/branches/solr5914/lucene/core/src/test/org/apache/lucene/index/TestBackwardsCompatibility.java Fri Apr  4 10:27:05 2014
@@ -60,6 +60,7 @@ import org.apache.lucene.store.SimpleFSD
 import org.apache.lucene.util.Bits;
 import org.apache.lucene.util.BytesRef;
 import org.apache.lucene.util.Constants;
+import org.apache.lucene.util.IOUtils;
 import org.apache.lucene.util.LuceneTestCase;
 import org.apache.lucene.util.LuceneTestCase.SuppressCodecs;
 import org.apache.lucene.util.StringHelper;
@@ -76,7 +77,7 @@ import org.junit.BeforeClass;
 // 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", "Lucene42", "Lucene45"})
+@SuppressCodecs({"Lucene40", "Lucene41", "Lucene42", "Lucene45"})
 public class TestBackwardsCompatibility extends LuceneTestCase {
 
   // Uncomment these cases & run them on an older Lucene version,
@@ -292,13 +293,13 @@ public class TestBackwardsCompatibility 
       
       ByteArrayOutputStream bos = new ByteArrayOutputStream(1024);
       CheckIndex checker = new CheckIndex(dir);
-      checker.setInfoStream(new PrintStream(bos, false, "UTF-8"));
+      checker.setInfoStream(new PrintStream(bos, false, IOUtils.UTF_8));
       CheckIndex.Status indexStatus = checker.checkIndex();
       assertFalse(indexStatus.clean);
-      assertTrue(bos.toString("UTF-8").contains(IndexFormatTooOldException.class.getName()));
+      assertTrue(bos.toString(IOUtils.UTF_8).contains(IndexFormatTooOldException.class.getName()));
 
       dir.close();
-      TestUtil.rmDir(oldIndxeDir);
+      TestUtil.rm(oldIndxeDir);
     }
   }
   
@@ -597,7 +598,7 @@ public class TestBackwardsCompatibility 
   public File createIndex(String dirName, boolean doCFS, boolean fullyMerged) throws IOException {
     // we use a real directory name that is not cleaned up, because this method is only used to create backwards indexes:
     File indexDir = new File("/tmp/idx", dirName);
-    TestUtil.rmDir(indexDir);
+    TestUtil.rm(indexDir);
     Directory dir = newFSDirectory(indexDir);
     LogByteSizeMergePolicy mp = new LogByteSizeMergePolicy();
     mp.setNoCFSRatio(doCFS ? 1.0 : 0.0);
@@ -646,7 +647,7 @@ public class TestBackwardsCompatibility 
 
     String outputDirName = "lucene.backwardscompat0.index";
     File outputDir = TestUtil.createTempDir(outputDirName);
-    TestUtil.rmDir(outputDir);
+    TestUtil.rm(outputDir);
 
     try {
       Directory dir = newFSDirectory(outputDir);
@@ -704,7 +705,7 @@ public class TestBackwardsCompatibility 
       }
       dir.close();
     } finally {
-      TestUtil.rmDir(outputDir);
+      TestUtil.rm(outputDir);
     }
   }
 

Modified: lucene/dev/branches/solr5914/lucene/core/src/test/org/apache/lucene/index/TestBagOfPositions.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/solr5914/lucene/core/src/test/org/apache/lucene/index/TestBagOfPositions.java?rev=1584603&r1=1584602&r2=1584603&view=diff
==============================================================================
--- lucene/dev/branches/solr5914/lucene/core/src/test/org/apache/lucene/index/TestBagOfPositions.java (original)
+++ lucene/dev/branches/solr5914/lucene/core/src/test/org/apache/lucene/index/TestBagOfPositions.java Fri Apr  4 10:27:05 2014
@@ -87,7 +87,7 @@ public class TestBagOfPositions extends 
     if (options == 0) {
       fieldType.setIndexOptions(IndexOptions.DOCS_AND_FREQS); // we dont actually need positions
       fieldType.setStoreTermVectors(true); // but enforce term vectors when we do this so we check SOMETHING
-    } else if (options == 1 && !doesntSupportOffsets.contains(TestUtil.getPostingsFormat("field"))) {
+    } else if (options == 1) {
       fieldType.setIndexOptions(IndexOptions.DOCS_AND_FREQS_AND_POSITIONS_AND_OFFSETS);
     }
     // else just positions

Modified: lucene/dev/branches/solr5914/lucene/core/src/test/org/apache/lucene/index/TestCheckIndex.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/solr5914/lucene/core/src/test/org/apache/lucene/index/TestCheckIndex.java?rev=1584603&r1=1584602&r2=1584603&view=diff
==============================================================================
--- lucene/dev/branches/solr5914/lucene/core/src/test/org/apache/lucene/index/TestCheckIndex.java (original)
+++ lucene/dev/branches/solr5914/lucene/core/src/test/org/apache/lucene/index/TestCheckIndex.java Fri Apr  4 10:27:05 2014
@@ -20,9 +20,11 @@ package org.apache.lucene.index;
 import java.io.IOException;
 import java.io.ByteArrayOutputStream;
 import java.io.PrintStream;
+import java.nio.charset.StandardCharsets;
 import java.util.List;
 import java.util.ArrayList;
 
+import org.apache.lucene.util.IOUtils;
 import org.apache.lucene.util.LuceneTestCase;
 import org.apache.lucene.store.Directory;
 import org.apache.lucene.analysis.CannedTokenStream;
@@ -54,12 +56,12 @@ public class TestCheckIndex extends Luce
 
     ByteArrayOutputStream bos = new ByteArrayOutputStream(1024);
     CheckIndex checker = new CheckIndex(dir);
-    checker.setInfoStream(new PrintStream(bos, false, "UTF-8"));
+    checker.setInfoStream(new PrintStream(bos, false, IOUtils.UTF_8));
     if (VERBOSE) checker.setInfoStream(System.out);
     CheckIndex.Status indexStatus = checker.checkIndex();
     if (indexStatus.clean == false) {
       System.out.println("CheckIndex failed");
-      System.out.println(bos.toString("UTF-8"));
+      System.out.println(bos.toString(IOUtils.UTF_8));
       fail();
     }
     

Modified: lucene/dev/branches/solr5914/lucene/core/src/test/org/apache/lucene/index/TestCodecs.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/solr5914/lucene/core/src/test/org/apache/lucene/index/TestCodecs.java?rev=1584603&r1=1584602&r2=1584603&view=diff
==============================================================================
--- lucene/dev/branches/solr5914/lucene/core/src/test/org/apache/lucene/index/TestCodecs.java (original)
+++ lucene/dev/branches/solr5914/lucene/core/src/test/org/apache/lucene/index/TestCodecs.java Fri Apr  4 10:27:05 2014
@@ -29,7 +29,6 @@ import org.apache.lucene.codecs.FieldsPr
 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;
@@ -336,54 +335,6 @@ public class TestCodecs extends LuceneTe
     dir.close();
   }
 
-  public void testSepPositionAfterMerge() throws IOException {
-    final Directory dir = newDirectory();
-    final IndexWriterConfig config = newIndexWriterConfig(TEST_VERSION_CURRENT,
-      new MockAnalyzer(random()));
-    config.setMergePolicy(newLogMergePolicy());
-    config.setCodec(TestUtil.alwaysPostingsFormat(new MockSepPostingsFormat()));
-    final IndexWriter writer = new IndexWriter(dir, config);
-
-    try {
-      final PhraseQuery pq = new PhraseQuery();
-      pq.add(new Term("content", "bbb"));
-      pq.add(new Term("content", "ccc"));
-
-      final Document doc = new Document();
-      FieldType customType = new FieldType(TextField.TYPE_NOT_STORED);
-      customType.setOmitNorms(true);
-      doc.add(newField("content", "aaa bbb ccc ddd", customType));
-
-      // add document and force commit for creating a first segment
-      writer.addDocument(doc);
-      writer.commit();
-
-      ScoreDoc[] results = this.search(writer, pq, 5);
-      assertEquals(1, results.length);
-      assertEquals(0, results[0].doc);
-
-      // add document and force commit for creating a second segment
-      writer.addDocument(doc);
-      writer.commit();
-
-      // at this point, there should be at least two segments
-      results = this.search(writer, pq, 5);
-      assertEquals(2, results.length);
-      assertEquals(0, results[0].doc);
-
-      writer.forceMerge(1);
-
-      // optimise to merge the segments.
-      results = this.search(writer, pq, 5);
-      assertEquals(2, results.length);
-      assertEquals(0, results[0].doc);
-    }
-    finally {
-      writer.close();
-      dir.close();
-    }
-  }
-
   private ScoreDoc[] search(final IndexWriter writer, final Query q, final int n) throws IOException {
     final IndexReader reader = writer.getReader();
     final IndexSearcher searcher = newSearcher(reader);

Modified: lucene/dev/branches/solr5914/lucene/core/src/test/org/apache/lucene/index/TestDoc.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/solr5914/lucene/core/src/test/org/apache/lucene/index/TestDoc.java?rev=1584603&r1=1584602&r2=1584603&view=diff
==============================================================================
--- lucene/dev/branches/solr5914/lucene/core/src/test/org/apache/lucene/index/TestDoc.java (original)
+++ lucene/dev/branches/solr5914/lucene/core/src/test/org/apache/lucene/index/TestDoc.java Fri Apr  4 10:27:05 2014
@@ -25,6 +25,7 @@ import java.io.OutputStreamWriter;
 import java.io.PrintWriter;
 import java.io.StringWriter;
 import java.io.Writer;
+import java.nio.charset.StandardCharsets;
 import java.util.Arrays;
 import java.util.Collection;
 import java.util.HashSet;
@@ -89,7 +90,7 @@ public class TestDoc extends LuceneTestC
             File f = new File(workDir, name);
             if (f.exists()) f.delete();
 
-            fw = new OutputStreamWriter(new FileOutputStream(f), "UTF-8");
+            fw = new OutputStreamWriter(new FileOutputStream(f), StandardCharsets.UTF_8);
             pw = new PrintWriter(fw);
             pw.println(text);
             return f;
@@ -200,7 +201,7 @@ public class TestDoc extends LuceneTestC
    {
       File file = new File(workDir, fileName);
       Document doc = new Document();
-      InputStreamReader is = new InputStreamReader(new FileInputStream(file), "UTF-8");
+      InputStreamReader is = new InputStreamReader(new FileInputStream(file), StandardCharsets.UTF_8);
       doc.add(new TextField("contents", is));
       writer.addDocument(doc);
       writer.commit();
@@ -221,7 +222,7 @@ public class TestDoc extends LuceneTestC
 
       SegmentMerger merger = new SegmentMerger(Arrays.<AtomicReader>asList(r1, r2),
           si, InfoStream.getDefault(), trackingDir,
-          MergeState.CheckAbort.NONE, new FieldInfos.FieldNumbers(), context);
+          MergeState.CheckAbort.NONE, new FieldInfos.FieldNumbers(), context, true);
 
       MergeState mergeState = merger.merge();
       r1.close();

Modified: lucene/dev/branches/solr5914/lucene/core/src/test/org/apache/lucene/index/TestDocInverterPerFieldErrorInfo.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/solr5914/lucene/core/src/test/org/apache/lucene/index/TestDocInverterPerFieldErrorInfo.java?rev=1584603&r1=1584602&r2=1584603&view=diff
==============================================================================
--- lucene/dev/branches/solr5914/lucene/core/src/test/org/apache/lucene/index/TestDocInverterPerFieldErrorInfo.java (original)
+++ lucene/dev/branches/solr5914/lucene/core/src/test/org/apache/lucene/index/TestDocInverterPerFieldErrorInfo.java Fri Apr  4 10:27:05 2014
@@ -26,6 +26,7 @@ import org.apache.lucene.document.Docume
 import org.apache.lucene.document.FieldType;
 import org.apache.lucene.document.TextField;
 import org.apache.lucene.store.Directory;
+import org.apache.lucene.util.IOUtils;
 import org.apache.lucene.util.LuceneTestCase;
 import org.apache.lucene.util.PrintStreamInfoStream;
 import org.junit.Test;
@@ -70,7 +71,7 @@ public class TestDocInverterPerFieldErro
     IndexWriter writer;
     IndexWriterConfig c = new IndexWriterConfig(TEST_VERSION_CURRENT, new ThrowingAnalyzer());
     final ByteArrayOutputStream infoBytes = new ByteArrayOutputStream();
-    PrintStream infoPrintStream = new PrintStream(infoBytes, true, "utf-8");
+    PrintStream infoPrintStream = new PrintStream(infoBytes, true, IOUtils.UTF_8);
     PrintStreamInfoStream printStreamInfoStream = new PrintStreamInfoStream(infoPrintStream);
     c.setInfoStream(printStreamInfoStream);
     writer = new IndexWriter(dir, c);
@@ -81,7 +82,7 @@ public class TestDocInverterPerFieldErro
       fail("Failed to fail.");
     } catch(BadNews badNews) {
       infoPrintStream.flush();
-      String infoStream = new String(infoBytes.toByteArray(), "utf-8");
+      String infoStream = new String(infoBytes.toByteArray(), IOUtils.UTF_8);
       assertTrue(infoStream.contains("distinctiveFieldName"));
     }
 
@@ -95,7 +96,7 @@ public class TestDocInverterPerFieldErro
     IndexWriter writer;
     IndexWriterConfig c = new IndexWriterConfig(TEST_VERSION_CURRENT, new ThrowingAnalyzer());
     final ByteArrayOutputStream infoBytes = new ByteArrayOutputStream();
-    PrintStream infoPrintStream = new PrintStream(infoBytes, true, "utf-8");
+    PrintStream infoPrintStream = new PrintStream(infoBytes, true, IOUtils.UTF_8);
     PrintStreamInfoStream printStreamInfoStream = new PrintStreamInfoStream(infoPrintStream);
     c.setInfoStream(printStreamInfoStream);
     writer = new IndexWriter(dir, c);
@@ -107,7 +108,7 @@ public class TestDocInverterPerFieldErro
       fail("Unwanted exception");
     }
     infoPrintStream.flush();
-    String infoStream = new String(infoBytes.toByteArray(), "utf-8");
+    String infoStream = new String(infoBytes.toByteArray(), IOUtils.UTF_8);
     assertFalse(infoStream.contains("boringFieldName"));
 
     writer.close();

Modified: lucene/dev/branches/solr5914/lucene/core/src/test/org/apache/lucene/index/TestIndexWriterDelete.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/solr5914/lucene/core/src/test/org/apache/lucene/index/TestIndexWriterDelete.java?rev=1584603&r1=1584602&r2=1584603&view=diff
==============================================================================
--- lucene/dev/branches/solr5914/lucene/core/src/test/org/apache/lucene/index/TestIndexWriterDelete.java (original)
+++ lucene/dev/branches/solr5914/lucene/core/src/test/org/apache/lucene/index/TestIndexWriterDelete.java Fri Apr  4 10:27:05 2014
@@ -20,6 +20,7 @@ package org.apache.lucene.index;
 import java.io.ByteArrayOutputStream;
 import java.io.IOException;
 import java.io.PrintStream;
+import java.nio.charset.StandardCharsets;
 import java.util.ArrayList;
 import java.util.Collections;
 import java.util.List;
@@ -43,6 +44,7 @@ import org.apache.lucene.search.TermQuer
 import org.apache.lucene.store.Directory;
 import org.apache.lucene.store.MockDirectoryWrapper;
 import org.apache.lucene.store.RAMDirectory;
+import org.apache.lucene.util.IOUtils;
 import org.apache.lucene.util.LuceneTestCase;
 import org.apache.lucene.util.TestUtil;
 
@@ -1173,10 +1175,10 @@ public class TestIndexWriterDelete exten
 
     ByteArrayOutputStream bos = new ByteArrayOutputStream(1024);
     CheckIndex checker = new CheckIndex(dir);
-    checker.setInfoStream(new PrintStream(bos, false, "UTF-8"), false);
+    checker.setInfoStream(new PrintStream(bos, false, IOUtils.UTF_8), false);
     CheckIndex.Status indexStatus = checker.checkIndex(null);
     assertTrue(indexStatus.clean);
-    String s = bos.toString("UTF-8");
+    String s = bos.toString(IOUtils.UTF_8);
 
     // Segment should have deletions:
     assertTrue(s.contains("has deletions"));
@@ -1185,10 +1187,10 @@ public class TestIndexWriterDelete exten
     w.close();
 
     bos = new ByteArrayOutputStream(1024);
-    checker.setInfoStream(new PrintStream(bos, false, "UTF-8"), false);
+    checker.setInfoStream(new PrintStream(bos, false, IOUtils.UTF_8), false);
     indexStatus = checker.checkIndex(null);
     assertTrue(indexStatus.clean);
-    s = bos.toString("UTF-8");
+    s = bos.toString(IOUtils.UTF_8);
     assertFalse(s.contains("has deletions"));
     dir.close();
   }

Modified: lucene/dev/branches/solr5914/lucene/core/src/test/org/apache/lucene/index/TestIndexWriterUnicode.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/solr5914/lucene/core/src/test/org/apache/lucene/index/TestIndexWriterUnicode.java?rev=1584603&r1=1584602&r2=1584603&view=diff
==============================================================================
--- lucene/dev/branches/solr5914/lucene/core/src/test/org/apache/lucene/index/TestIndexWriterUnicode.java (original)
+++ lucene/dev/branches/solr5914/lucene/core/src/test/org/apache/lucene/index/TestIndexWriterUnicode.java Fri Apr  4 10:27:05 2014
@@ -18,6 +18,7 @@ package org.apache.lucene.index;
  */
 
 import java.io.IOException;
+import java.nio.charset.StandardCharsets;
 import java.util.HashSet;
 import java.util.Iterator;
 import java.util.Random;
@@ -181,7 +182,7 @@ public class TestIndexWriterUnicode exte
 
       UnicodeUtil.UTF16toUTF8(buffer, 0, 20, utf8);
       if (!hasIllegal) {
-        byte[] b = new String(buffer, 0, 20).getBytes("UTF-8");
+        byte[] b = new String(buffer, 0, 20).getBytes(StandardCharsets.UTF_8);
         assertEquals(b.length, utf8.length);
         for(int i=0;i<b.length;i++)
           assertEquals(b[i], utf8.bytes[i]);
@@ -217,13 +218,13 @@ public class TestIndexWriterUnicode exte
       UnicodeUtil.UTF16toUTF8(chars, 0, len, utf8);
 
       String s1 = new String(chars, 0, len);
-      String s2 = new String(utf8.bytes, 0, utf8.length, "UTF-8");
+      String s2 = new String(utf8.bytes, 0, utf8.length, StandardCharsets.UTF_8);
       assertEquals("codepoint " + ch, s1, s2);
 
       UnicodeUtil.UTF8toUTF16(utf8.bytes, 0, utf8.length, utf16);
       assertEquals("codepoint " + ch, s1, new String(utf16.chars, 0, utf16.length));
 
-      byte[] b = s1.getBytes("UTF-8");
+      byte[] b = s1.getBytes(StandardCharsets.UTF_8);
       assertEquals(utf8.length, b.length);
       for(int j=0;j<utf8.length;j++)
         assertEquals(utf8.bytes[j], b[j]);

Modified: lucene/dev/branches/solr5914/lucene/core/src/test/org/apache/lucene/index/TestNorms.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/solr5914/lucene/core/src/test/org/apache/lucene/index/TestNorms.java?rev=1584603&r1=1584602&r2=1584603&view=diff
==============================================================================
--- lucene/dev/branches/solr5914/lucene/core/src/test/org/apache/lucene/index/TestNorms.java (original)
+++ lucene/dev/branches/solr5914/lucene/core/src/test/org/apache/lucene/index/TestNorms.java Fri Apr  4 10:27:05 2014
@@ -76,7 +76,6 @@ public class TestNorms extends LuceneTes
   public void testCustomEncoder() throws Exception {
     Directory dir = newDirectory();
     MockAnalyzer analyzer = new MockAnalyzer(random());
-    analyzer.setMaxTokenLength(TestUtil.nextInt(random(), 1, IndexWriter.MAX_TERM_LENGTH));
 
     IndexWriterConfig config = newIndexWriterConfig(TEST_VERSION_CURRENT, analyzer);
     config.setSimilarity(new CustomNormEncodingSimilarity());
@@ -128,8 +127,10 @@ public class TestNorms extends LuceneTes
 
   public void buildIndex(Directory dir) throws IOException {
     Random random = random();
+    MockAnalyzer analyzer = new MockAnalyzer(random());
+    analyzer.setMaxTokenLength(TestUtil.nextInt(random(), 1, IndexWriter.MAX_TERM_LENGTH));
     IndexWriterConfig config = newIndexWriterConfig(TEST_VERSION_CURRENT,
-        new MockAnalyzer(random()));
+        analyzer);
     Similarity provider = new MySimProvider();
     config.setSimilarity(provider);
     RandomIndexWriter writer = new RandomIndexWriter(random, dir, config);

Modified: lucene/dev/branches/solr5914/lucene/core/src/test/org/apache/lucene/index/TestPayloads.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/solr5914/lucene/core/src/test/org/apache/lucene/index/TestPayloads.java?rev=1584603&r1=1584602&r2=1584603&view=diff
==============================================================================
--- lucene/dev/branches/solr5914/lucene/core/src/test/org/apache/lucene/index/TestPayloads.java (original)
+++ lucene/dev/branches/solr5914/lucene/core/src/test/org/apache/lucene/index/TestPayloads.java Fri Apr  4 10:27:05 2014
@@ -20,6 +20,7 @@ package org.apache.lucene.index;
 import java.io.IOException;
 import java.io.StringReader;
 import java.nio.charset.Charset;
+import java.nio.charset.StandardCharsets;
 import java.util.ArrayList;
 import java.util.HashMap;
 import java.util.List;
@@ -72,7 +73,7 @@ public class TestPayloads extends Lucene
         // enabled in only some documents
         d.add(newTextField("f3", "This field has payloads in some docs", Field.Store.NO));
         // only add payload data for field f2
-        analyzer.setPayloadData("f2", "somedata".getBytes("UTF-8"), 0, 1);
+        analyzer.setPayloadData("f2", "somedata".getBytes(StandardCharsets.UTF_8), 0, 1);
         writer.addDocument(d);
         // flush
         writer.close();
@@ -95,8 +96,8 @@ public class TestPayloads extends Lucene
         d.add(newTextField("f2", "This field has payloads in all docs", Field.Store.NO));
         d.add(newTextField("f3", "This field has payloads in some docs", Field.Store.NO));
         // add payload data for field f2 and f3
-        analyzer.setPayloadData("f2", "somedata".getBytes("UTF-8"), 0, 1);
-        analyzer.setPayloadData("f3", "somedata".getBytes("UTF-8"), 0, 3);
+        analyzer.setPayloadData("f2", "somedata".getBytes(StandardCharsets.UTF_8), 0, 1);
+        analyzer.setPayloadData("f3", "somedata".getBytes(StandardCharsets.UTF_8), 0, 3);
         writer.addDocument(d);
 
         // force merge
@@ -297,7 +298,8 @@ public class TestPayloads extends Lucene
         
     }
     
-    static final Charset utf8 = Charset.forName("UTF-8");
+    static final Charset utf8 = StandardCharsets.UTF_8;
+    
     private void generateRandomData(byte[] data) {
       // this test needs the random data to be valid unicode
       String s = TestUtil.randomFixedByteLengthUnicodeString(random(), data.length);

Modified: lucene/dev/branches/solr5914/lucene/core/src/test/org/apache/lucene/index/TestPostingsOffsets.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/solr5914/lucene/core/src/test/org/apache/lucene/index/TestPostingsOffsets.java?rev=1584603&r1=1584602&r2=1584603&view=diff
==============================================================================
--- lucene/dev/branches/solr5914/lucene/core/src/test/org/apache/lucene/index/TestPostingsOffsets.java (original)
+++ lucene/dev/branches/solr5914/lucene/core/src/test/org/apache/lucene/index/TestPostingsOffsets.java Fri Apr  4 10:27:05 2014
@@ -43,13 +43,11 @@ import org.apache.lucene.util.BytesRef;
 import org.apache.lucene.util.English;
 import org.apache.lucene.util.IOUtils;
 import org.apache.lucene.util.LuceneTestCase;
-import org.apache.lucene.util.LuceneTestCase.SuppressCodecs;
 import org.apache.lucene.util.TestUtil;
 
 // TODO: we really need to test indexingoffsets, but then getting only docs / docs + freqs.
 // not all codecs store prx separate...
 // TODO: fix sep codec to index offsets so we can greatly reduce this list!
-@SuppressCodecs({"MockFixedIntBlock", "MockVariableIntBlock", "MockSep", "MockRandom"})
 public class TestPostingsOffsets extends LuceneTestCase {
   IndexWriterConfig iwc;
   

Modified: lucene/dev/branches/solr5914/lucene/core/src/test/org/apache/lucene/index/TestSegmentMerger.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/solr5914/lucene/core/src/test/org/apache/lucene/index/TestSegmentMerger.java?rev=1584603&r1=1584602&r2=1584603&view=diff
==============================================================================
--- lucene/dev/branches/solr5914/lucene/core/src/test/org/apache/lucene/index/TestSegmentMerger.java (original)
+++ lucene/dev/branches/solr5914/lucene/core/src/test/org/apache/lucene/index/TestSegmentMerger.java Fri Apr  4 10:27:05 2014
@@ -83,7 +83,7 @@ public class TestSegmentMerger extends L
 
     SegmentMerger merger = new SegmentMerger(Arrays.<AtomicReader>asList(reader1, reader2),
         si, InfoStream.getDefault(), mergedDir,
-        MergeState.CheckAbort.NONE, new FieldInfos.FieldNumbers(), newIOContext(random()));
+        MergeState.CheckAbort.NONE, new FieldInfos.FieldNumbers(), newIOContext(random()), true);
     MergeState mergeState = merger.merge();
     int docsMerged = mergeState.segmentInfo.getDocCount();
     assertTrue(docsMerged == 2);

Modified: lucene/dev/branches/solr5914/lucene/core/src/test/org/apache/lucene/search/TestFieldCache.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/solr5914/lucene/core/src/test/org/apache/lucene/search/TestFieldCache.java?rev=1584603&r1=1584602&r2=1584603&view=diff
==============================================================================
--- lucene/dev/branches/solr5914/lucene/core/src/test/org/apache/lucene/search/TestFieldCache.java (original)
+++ lucene/dev/branches/solr5914/lucene/core/src/test/org/apache/lucene/search/TestFieldCache.java Fri Apr  4 10:27:05 2014
@@ -60,6 +60,7 @@ import org.apache.lucene.search.FieldCac
 import org.apache.lucene.store.Directory;
 import org.apache.lucene.util.Bits;
 import org.apache.lucene.util.BytesRef;
+import org.apache.lucene.util.IOUtils;
 import org.apache.lucene.util.LuceneTestCase;
 import org.apache.lucene.util.NumericUtils;
 import org.apache.lucene.util.TestUtil;
@@ -139,7 +140,7 @@ public class TestFieldCache extends Luce
     try {
       FieldCache cache = FieldCache.DEFAULT;
       ByteArrayOutputStream bos = new ByteArrayOutputStream(1024);
-      cache.setInfoStream(new PrintStream(bos, false, "UTF-8"));
+      cache.setInfoStream(new PrintStream(bos, false, IOUtils.UTF_8));
       cache.getDoubles(reader, "theDouble", false);
       cache.getFloats(reader, "theDouble", new FieldCache.FloatParser() {
         @Override
@@ -151,7 +152,7 @@ public class TestFieldCache extends Luce
           return NumericUtils.sortableIntToFloat((int) NumericUtils.prefixCodedToLong(term));
         }
       }, false);
-      assertTrue(bos.toString("UTF-8").indexOf("WARNING") != -1);
+      assertTrue(bos.toString(IOUtils.UTF_8).indexOf("WARNING") != -1);
     } finally {
       FieldCache.DEFAULT.purgeAllCaches();
     }

Modified: lucene/dev/branches/solr5914/lucene/core/src/test/org/apache/lucene/search/TestPositionIncrement.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/solr5914/lucene/core/src/test/org/apache/lucene/search/TestPositionIncrement.java?rev=1584603&r1=1584602&r2=1584603&view=diff
==============================================================================
--- lucene/dev/branches/solr5914/lucene/core/src/test/org/apache/lucene/search/TestPositionIncrement.java (original)
+++ lucene/dev/branches/solr5914/lucene/core/src/test/org/apache/lucene/search/TestPositionIncrement.java Fri Apr  4 10:27:05 2014
@@ -20,6 +20,7 @@ package org.apache.lucene.search;
 import java.io.IOException;
 import java.io.Reader;
 import java.io.StringReader;
+import java.nio.charset.StandardCharsets;
 import java.util.Collection;
 
 import org.apache.lucene.analysis.*;
@@ -249,7 +250,7 @@ public class TestPositionIncrement exten
       for (byte[] bytes : payloads) {
         count++;
         if (VERBOSE) {
-          System.out.println("  payload: " + new String(bytes, "UTF-8"));
+          System.out.println("  payload: " + new String(bytes, StandardCharsets.UTF_8));
         }
       }
     }
@@ -276,7 +277,7 @@ public class TestPositionIncrement exten
     Collection<byte[]> pls = psu.getPayloadsForQuery(snq);
     count = pls.size();
     for (byte[] bytes : pls) {
-      String s = new String(bytes, "UTF-8");
+      String s = new String(bytes, StandardCharsets.UTF_8);
       //System.out.println(s);
       sawZero |= s.equals("pos: 0");
     }

Modified: lucene/dev/branches/solr5914/lucene/core/src/test/org/apache/lucene/search/spans/TestBasics.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/solr5914/lucene/core/src/test/org/apache/lucene/search/spans/TestBasics.java?rev=1584603&r1=1584602&r2=1584603&view=diff
==============================================================================
--- lucene/dev/branches/solr5914/lucene/core/src/test/org/apache/lucene/search/spans/TestBasics.java (original)
+++ lucene/dev/branches/solr5914/lucene/core/src/test/org/apache/lucene/search/spans/TestBasics.java Fri Apr  4 10:27:05 2014
@@ -18,6 +18,7 @@ package org.apache.lucene.search.spans;
  */
 
 import java.io.IOException;
+import java.nio.charset.StandardCharsets;
 import java.util.ArrayList;
 import java.util.Collection;
 import java.util.Collections;
@@ -80,7 +81,7 @@ public class TestBasics extends LuceneTe
     @Override
     public boolean incrementToken() throws IOException {
       if (input.incrementToken()) {
-        payloadAttr.setPayload(new BytesRef(("pos: " + pos).getBytes("UTF-8")));
+        payloadAttr.setPayload(new BytesRef(("pos: " + pos).getBytes(StandardCharsets.UTF_8)));
         pos++;
         return true;
       } else {
@@ -482,7 +483,7 @@ public class TestBasics extends LuceneTe
   @Test
   public void testSpanPayloadCheck() throws Exception {
     SpanTermQuery term1 = new SpanTermQuery(new Term("field", "five"));
-    BytesRef pay = new BytesRef(("pos: " + 5).getBytes("UTF-8"));
+    BytesRef pay = new BytesRef(("pos: " + 5).getBytes(StandardCharsets.UTF_8));
     SpanQuery query = new SpanPayloadCheckQuery(term1, Collections.singletonList(pay.bytes));
     checkHits(query, new int[]
       {1125, 1135, 1145, 1155, 1165, 1175, 1185, 1195, 1225, 1235, 1245, 1255, 1265, 1275, 1285, 1295, 1325, 1335, 1345, 1355, 1365, 1375, 1385, 1395, 1425, 1435, 1445, 1455, 1465, 1475, 1485, 1495, 1525, 1535, 1545, 1555, 1565, 1575, 1585, 1595, 1625, 1635, 1645, 1655, 1665, 1675, 1685, 1695, 1725, 1735, 1745, 1755, 1765, 1775, 1785, 1795, 1825, 1835, 1845, 1855, 1865, 1875, 1885, 1895, 1925, 1935, 1945, 1955, 1965, 1975, 1985, 1995});
@@ -497,8 +498,8 @@ public class TestBasics extends LuceneTe
     clauses[0] = term1;
     clauses[1] = term2;
     snq = new SpanNearQuery(clauses, 0, true);
-    pay = new BytesRef(("pos: " + 0).getBytes("UTF-8"));
-    pay2 = new BytesRef(("pos: " + 1).getBytes("UTF-8"));
+    pay = new BytesRef(("pos: " + 0).getBytes(StandardCharsets.UTF_8));
+    pay2 = new BytesRef(("pos: " + 1).getBytes(StandardCharsets.UTF_8));
     list = new ArrayList<>();
     list.add(pay.bytes);
     list.add(pay2.bytes);
@@ -510,9 +511,9 @@ public class TestBasics extends LuceneTe
     clauses[1] = term2;
     clauses[2] = new SpanTermQuery(new Term("field", "five"));
     snq = new SpanNearQuery(clauses, 0, true);
-    pay = new BytesRef(("pos: " + 0).getBytes("UTF-8"));
-    pay2 = new BytesRef(("pos: " + 1).getBytes("UTF-8"));
-    BytesRef pay3 = new BytesRef(("pos: " + 2).getBytes("UTF-8"));
+    pay = new BytesRef(("pos: " + 0).getBytes(StandardCharsets.UTF_8));
+    pay2 = new BytesRef(("pos: " + 1).getBytes(StandardCharsets.UTF_8));
+    BytesRef pay3 = new BytesRef(("pos: " + 2).getBytes(StandardCharsets.UTF_8));
     list = new ArrayList<>();
     list.add(pay.bytes);
     list.add(pay2.bytes);
@@ -541,10 +542,10 @@ public class TestBasics extends LuceneTe
     checkHits(query, new int[]{1103, 1203,1303,1403,1503,1603,1703,1803,1903});
 
     Collection<byte[]> payloads = new ArrayList<>();
-    BytesRef pay = new BytesRef(("pos: " + 0).getBytes("UTF-8"));
-    BytesRef pay2 = new BytesRef(("pos: " + 1).getBytes("UTF-8"));
-    BytesRef pay3 = new BytesRef(("pos: " + 3).getBytes("UTF-8"));
-    BytesRef pay4 = new BytesRef(("pos: " + 4).getBytes("UTF-8"));
+    BytesRef pay = new BytesRef(("pos: " + 0).getBytes(StandardCharsets.UTF_8));
+    BytesRef pay2 = new BytesRef(("pos: " + 1).getBytes(StandardCharsets.UTF_8));
+    BytesRef pay3 = new BytesRef(("pos: " + 3).getBytes(StandardCharsets.UTF_8));
+    BytesRef pay4 = new BytesRef(("pos: " + 4).getBytes(StandardCharsets.UTF_8));
     payloads.add(pay.bytes);
     payloads.add(pay2.bytes);
     payloads.add(pay3.bytes);

Modified: lucene/dev/branches/solr5914/lucene/core/src/test/org/apache/lucene/search/spans/TestPayloadSpans.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/solr5914/lucene/core/src/test/org/apache/lucene/search/spans/TestPayloadSpans.java?rev=1584603&r1=1584602&r2=1584603&view=diff
==============================================================================
--- lucene/dev/branches/solr5914/lucene/core/src/test/org/apache/lucene/search/spans/TestPayloadSpans.java (original)
+++ lucene/dev/branches/solr5914/lucene/core/src/test/org/apache/lucene/search/spans/TestPayloadSpans.java Fri Apr  4 10:27:05 2014
@@ -19,6 +19,7 @@ package org.apache.lucene.search.spans;
 import java.io.IOException;
 import java.io.Reader;
 import java.io.StringReader;
+import java.nio.charset.StandardCharsets;
 import java.util.Collection;
 import java.util.HashSet;
 import java.util.Set;
@@ -276,7 +277,7 @@ public class TestPayloadSpans extends Lu
         Collection<byte[]> payloads = spans.getPayload();
 
         for (final byte [] payload : payloads) {
-          payloadSet.add(new String(payload, "UTF-8"));
+          payloadSet.add(new String(payload, StandardCharsets.UTF_8));
         }
       }
     }
@@ -311,7 +312,7 @@ public class TestPayloadSpans extends Lu
       while (spans.next()) {
         Collection<byte[]> payloads = spans.getPayload();
         for (final byte[] payload : payloads) {
-          payloadSet.add(new String(payload, "UTF-8"));
+          payloadSet.add(new String(payload, StandardCharsets.UTF_8));
         }
       }
     }
@@ -347,7 +348,7 @@ public class TestPayloadSpans extends Lu
         Collection<byte[]> payloads = spans.getPayload();
 
         for (final byte [] payload : payloads) {
-          payloadSet.add(new String(payload, "UTF-8"));
+          payloadSet.add(new String(payload, StandardCharsets.UTF_8));
         }
       }
     }
@@ -382,7 +383,7 @@ public class TestPayloadSpans extends Lu
     if(VERBOSE) {
       System.out.println("Num payloads:" + payloads.size());
       for (final byte [] bytes : payloads) {
-        System.out.println(new String(bytes, "UTF-8"));
+        System.out.println(new String(bytes, StandardCharsets.UTF_8));
       }
     }
     reader.close();
@@ -455,7 +456,7 @@ public class TestPayloadSpans extends Lu
           System.out.println("payloads for span:" + payload.size());
           for (final byte [] bytes : payload) {
             System.out.println("doc:" + spans.doc() + " s:" + spans.start() + " e:" + spans.end() + " "
-              + new String(bytes, "UTF-8"));
+              + new String(bytes, StandardCharsets.UTF_8));
           }
         }
 

Modified: lucene/dev/branches/solr5914/lucene/core/src/test/org/apache/lucene/store/TestFilterDirectory.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/solr5914/lucene/core/src/test/org/apache/lucene/store/TestFilterDirectory.java?rev=1584603&r1=1584602&r2=1584603&view=diff
==============================================================================
--- lucene/dev/branches/solr5914/lucene/core/src/test/org/apache/lucene/store/TestFilterDirectory.java (original)
+++ lucene/dev/branches/solr5914/lucene/core/src/test/org/apache/lucene/store/TestFilterDirectory.java Fri Apr  4 10:27:05 2014
@@ -30,12 +30,13 @@ public class TestFilterDirectory extends
   public void testOverrides() throws Exception {
     // verify that all methods of Directory are overridden by FilterDirectory,
     // except those under the 'exclude' list
-    Set<String> exclude = new HashSet<>();
-    exclude.add("copy");
-    exclude.add("createSlicer");
+    Set<Method> exclude = new HashSet<>();
+    exclude.add(Directory.class.getMethod("copy", Directory.class, String.class, String.class, IOContext.class));
+    exclude.add(Directory.class.getMethod("createSlicer", String.class, IOContext.class));
+    exclude.add(Directory.class.getMethod("openChecksumInput", String.class, IOContext.class));
     for (Method m : FilterDirectory.class.getMethods()) {
       if (m.getDeclaringClass() == Directory.class) {
-        assertTrue("method " + m.getName() + " not overridden!", exclude.contains(m.getName()));
+        assertTrue("method " + m.getName() + " not overridden!", exclude.contains(m));
       }
     }
   }

Modified: lucene/dev/branches/solr5914/lucene/core/src/test/org/apache/lucene/util/TestMaxFailuresRule.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/solr5914/lucene/core/src/test/org/apache/lucene/util/TestMaxFailuresRule.java?rev=1584603&r1=1584602&r2=1584603&view=diff
==============================================================================
--- lucene/dev/branches/solr5914/lucene/core/src/test/org/apache/lucene/util/TestMaxFailuresRule.java (original)
+++ lucene/dev/branches/solr5914/lucene/core/src/test/org/apache/lucene/util/TestMaxFailuresRule.java Fri Apr  4 10:27:05 2014
@@ -17,8 +17,11 @@ package org.apache.lucene.util;
  * limitations under the License.
  */
 
+import java.util.concurrent.CountDownLatch;
+
 import org.apache.lucene.util.junitcompat.WithNestedTests;
 import org.junit.Assert;
+import org.junit.BeforeClass;
 import org.junit.Rule;
 import org.junit.Test;
 import org.junit.runner.Description;
@@ -27,8 +30,13 @@ import org.junit.runner.Result;
 import org.junit.runner.notification.Failure;
 import org.junit.runner.notification.RunListener;
 
-import com.carrotsearch.randomizedtesting.SysGlobals;
 import com.carrotsearch.randomizedtesting.annotations.Repeat;
+import com.carrotsearch.randomizedtesting.annotations.ThreadLeakAction;
+import com.carrotsearch.randomizedtesting.annotations.ThreadLeakLingering;
+import com.carrotsearch.randomizedtesting.annotations.ThreadLeakScope;
+import com.carrotsearch.randomizedtesting.annotations.ThreadLeakScope.Scope;
+import com.carrotsearch.randomizedtesting.annotations.ThreadLeakZombies;
+import com.carrotsearch.randomizedtesting.annotations.ThreadLeakZombies.Consequence;
 import com.carrotsearch.randomizedtesting.rules.SystemPropertiesInvariantRule;
 import com.carrotsearch.randomizedtesting.rules.SystemPropertiesRestoreRule;
 
@@ -66,48 +74,118 @@ public class TestMaxFailuresRule extends
 
   @Test
   public void testMaxFailures() {
-    TestRuleIgnoreAfterMaxFailures newRule = new TestRuleIgnoreAfterMaxFailures(2);
-    TestRuleIgnoreAfterMaxFailures prevRule = LuceneTestCase.replaceMaxFailureRule(newRule);
-    System.clearProperty(SysGlobals.SYSPROP_ITERATIONS());
-    try {
-      JUnitCore core = new JUnitCore();
-      final StringBuilder results = new StringBuilder();
-      core.addListener(new RunListener() {
-        char lastTest;
-
-        @Override
-        public void testStarted(Description description) throws Exception {
-          lastTest = 'S'; // success.
-        }
-
-        @Override
-        public void testAssumptionFailure(Failure failure) {
-          lastTest = 'A'; // assumption failure.
-        }
-
-        @Override
-        public void testFailure(Failure failure) throws Exception {
-          lastTest = 'F'; // failure
-        }
-
-        @Override
-        public void testFinished(Description description) throws Exception {
-          results.append(lastTest);
-        }
-      });
-
-      Result result = core.run(Nested.class);
-      Assert.assertEquals(500, result.getRunCount());
-      Assert.assertEquals(0, result.getIgnoreCount());
-      Assert.assertEquals(2, result.getFailureCount());
-
-      // Make sure we had exactly two failures followed by assumption-failures
-      // resulting from ignored tests.
-      Assert.assertTrue(results.toString(), 
-          results.toString().matches("(S*F){2}A+"));
+    LuceneTestCase.replaceMaxFailureRule(new TestRuleIgnoreAfterMaxFailures(2));
+    JUnitCore core = new JUnitCore();
+    final StringBuilder results = new StringBuilder();
+    core.addListener(new RunListener() {
+      char lastTest;
+
+      @Override
+      public void testStarted(Description description) throws Exception {
+        lastTest = 'S'; // success.
+      }
+
+      @Override
+      public void testAssumptionFailure(Failure failure) {
+        lastTest = 'A'; // assumption failure.
+      }
 
-    } finally {
-      LuceneTestCase.replaceMaxFailureRule(prevRule);
+      @Override
+      public void testFailure(Failure failure) throws Exception {
+        lastTest = 'F'; // failure
+      }
+
+      @Override
+      public void testFinished(Description description) throws Exception {
+        results.append(lastTest);
+      }
+    });
+
+    Result result = core.run(Nested.class);
+    Assert.assertEquals(500, result.getRunCount());
+    Assert.assertEquals(0, result.getIgnoreCount());
+    Assert.assertEquals(2, result.getFailureCount());
+
+    // Make sure we had exactly two failures followed by assumption-failures
+    // resulting from ignored tests.
+    Assert.assertTrue(results.toString(), 
+        results.toString().matches("(S*F){2}A+"));
+  }
+
+  @ThreadLeakZombies(Consequence.IGNORE_REMAINING_TESTS)
+  @ThreadLeakAction({ThreadLeakAction.Action.WARN})
+  @ThreadLeakScope(Scope.TEST)
+  @ThreadLeakLingering(linger = 500)
+  public static class Nested2 extends WithNestedTests.AbstractNestedTest {
+    public static final int TOTAL_ITERS = 10;
+    public static CountDownLatch die;
+    public static Thread zombie;
+    public static int testNum;
+    
+    @BeforeClass
+    public static void setup() {
+      assert zombie == null;
+      die = new CountDownLatch(1);
+      testNum = 0;
+    }
+
+    @Repeat(iterations = TOTAL_ITERS)
+    public void testLeaveZombie() {
+      if (++testNum == 2) {
+        zombie = new Thread() {
+          @Override
+          public void run() {
+            while (true) {
+              try {
+                die.await();
+                return;
+              } catch (Exception e) { /* ignore */ }
+            }
+          }
+        };
+        zombie.start();
+      }
     }
   }
+
+  @Test
+  public void testZombieThreadFailures() throws Exception {
+    LuceneTestCase.replaceMaxFailureRule(new TestRuleIgnoreAfterMaxFailures(1));
+    JUnitCore core = new JUnitCore();
+    final StringBuilder results = new StringBuilder();
+    core.addListener(new RunListener() {
+      char lastTest;
+
+      @Override
+      public void testStarted(Description description) throws Exception {
+        lastTest = 'S'; // success.
+      }
+
+      @Override
+      public void testAssumptionFailure(Failure failure) {
+        lastTest = 'A'; // assumption failure.
+      }
+
+      @Override
+      public void testFailure(Failure failure) throws Exception {
+        lastTest = 'F'; // failure
+        System.out.println(failure.getMessage());
+      }
+
+      @Override
+      public void testFinished(Description description) throws Exception {
+        results.append(lastTest);
+      }
+    });
+
+    Result result = core.run(Nested2.class);
+    if (Nested2.die != null) {
+      Nested2.die.countDown();
+      Nested2.zombie.join();
+    }
+    
+    super.prevSysOut.println(results.toString());
+    Assert.assertEquals(Nested2.TOTAL_ITERS, result.getRunCount());
+    Assert.assertEquals(results.toString(), "SFAAAAAAAA", results.toString());
+  }  
 }

Modified: lucene/dev/branches/solr5914/lucene/core/src/test/org/apache/lucene/util/TestOfflineSorter.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/solr5914/lucene/core/src/test/org/apache/lucene/util/TestOfflineSorter.java?rev=1584603&r1=1584602&r2=1584603&view=diff
==============================================================================
--- lucene/dev/branches/solr5914/lucene/core/src/test/org/apache/lucene/util/TestOfflineSorter.java (original)
+++ lucene/dev/branches/solr5914/lucene/core/src/test/org/apache/lucene/util/TestOfflineSorter.java Fri Apr  4 10:27:05 2014
@@ -21,6 +21,7 @@ import java.io.DataInputStream;
 import java.io.File;
 import java.io.FileInputStream;
 import java.io.IOException;
+import java.nio.charset.StandardCharsets;
 import java.util.ArrayList;
 import java.util.Arrays;
 import java.util.Comparator;
@@ -43,14 +44,14 @@ public class TestOfflineSorter extends L
   public void setUp() throws Exception {
     super.setUp();
     tempDir = TestUtil.createTempDir("mergesort");
-    TestUtil.rmDir(tempDir);
+    TestUtil.rm(tempDir);
     tempDir.mkdirs();
   }
   
   @Override
   public void tearDown() throws Exception {
     if (tempDir != null)
-      TestUtil.rmDir(tempDir);
+      TestUtil.rm(tempDir);
     super.tearDown();
   }
 
@@ -60,7 +61,7 @@ public class TestOfflineSorter extends L
 
   public void testSingleLine() throws Exception {
     checkSort(new OfflineSorter(), new byte [][] {
-        "Single line only.".getBytes("UTF-8")
+        "Single line only.".getBytes(StandardCharsets.UTF_8)
     });
   }
 

Modified: lucene/dev/branches/solr5914/lucene/core/src/test/org/apache/lucene/util/automaton/TestDeterminizeLexicon.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/solr5914/lucene/core/src/test/org/apache/lucene/util/automaton/TestDeterminizeLexicon.java?rev=1584603&r1=1584602&r2=1584603&view=diff
==============================================================================
--- lucene/dev/branches/solr5914/lucene/core/src/test/org/apache/lucene/util/automaton/TestDeterminizeLexicon.java (original)
+++ lucene/dev/branches/solr5914/lucene/core/src/test/org/apache/lucene/util/automaton/TestDeterminizeLexicon.java Fri Apr  4 10:27:05 2014
@@ -17,6 +17,7 @@ package org.apache.lucene.util.automaton
  * limitations under the License.
  */
 
+import java.nio.charset.StandardCharsets;
 import java.util.ArrayList;
 import java.util.Collections;
 import java.util.List;
@@ -56,7 +57,7 @@ public class TestDeterminizeLexicon exte
     }
     final ByteRunAutomaton lexByte = new ByteRunAutomaton(lex);
     for (String s : terms) {
-      byte bytes[] = s.getBytes("UTF-8");
+      byte bytes[] = s.getBytes(StandardCharsets.UTF_8);
       assertTrue(lexByte.run(bytes, 0, bytes.length));
     }
   }

Modified: lucene/dev/branches/solr5914/lucene/core/src/test/org/apache/lucene/util/automaton/TestUTF32ToUTF8.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/solr5914/lucene/core/src/test/org/apache/lucene/util/automaton/TestUTF32ToUTF8.java?rev=1584603&r1=1584602&r2=1584603&view=diff
==============================================================================
--- lucene/dev/branches/solr5914/lucene/core/src/test/org/apache/lucene/util/automaton/TestUTF32ToUTF8.java (original)
+++ lucene/dev/branches/solr5914/lucene/core/src/test/org/apache/lucene/util/automaton/TestUTF32ToUTF8.java Fri Apr  4 10:27:05 2014
@@ -22,6 +22,7 @@ import org.apache.lucene.util.TestUtil;
 import org.apache.lucene.util.BytesRef;
 import org.apache.lucene.util.UnicodeUtil;
 
+import java.nio.charset.StandardCharsets;
 import java.util.Random;
 
 public class TestUTF32ToUTF8 extends LuceneTestCase {
@@ -184,7 +185,7 @@ public class TestUTF32ToUTF8 extends Luc
 
     assertTrue(cra.run(input));
     
-    byte[] bytes = input.getBytes("UTF-8");
+    byte[] bytes = input.getBytes(StandardCharsets.UTF_8);
     assertTrue(bra.run(bytes, 0, bytes.length)); // this one fails!
   }
   
@@ -197,7 +198,7 @@ public class TestUTF32ToUTF8 extends Luc
 
     assertTrue(cra.run(input));
     
-    byte[] bytes = input.getBytes("UTF-8");
+    byte[] bytes = input.getBytes(StandardCharsets.UTF_8);
     assertTrue(bra.run(bytes, 0, bytes.length));
   }
   
@@ -232,7 +233,7 @@ public class TestUTF32ToUTF8 extends Luc
           throw e;
         }
       }
-      byte bytes[] = string.getBytes("UTF-8");
+      byte bytes[] = string.getBytes(StandardCharsets.UTF_8);
       assertEquals(cra.run(string), bra.run(bytes, 0, bytes.length));
     }
   }

Modified: lucene/dev/branches/solr5914/lucene/core/src/test/org/apache/lucene/util/fst/TestFSTs.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/solr5914/lucene/core/src/test/org/apache/lucene/util/fst/TestFSTs.java?rev=1584603&r1=1584602&r2=1584603&view=diff
==============================================================================
--- lucene/dev/branches/solr5914/lucene/core/src/test/org/apache/lucene/util/fst/TestFSTs.java (original)
+++ lucene/dev/branches/solr5914/lucene/core/src/test/org/apache/lucene/util/fst/TestFSTs.java Fri Apr  4 10:27:05 2014
@@ -63,6 +63,7 @@ import java.io.InputStreamReader;
 import java.io.OutputStreamWriter;
 import java.io.StringWriter;
 import java.io.Writer;
+import java.nio.charset.StandardCharsets;
 import java.util.ArrayList;
 import java.util.Arrays;
 import java.util.Collections;
@@ -474,7 +475,7 @@ public class TestFSTs extends LuceneTest
     protected abstract T getOutput(IntsRef input, int ord) throws IOException;
 
     public void run(int limit, boolean verify, boolean verifyByOutput) throws IOException {
-      BufferedReader is = new BufferedReader(new InputStreamReader(new FileInputStream(wordsFileIn), "UTF-8"), 65536);
+      BufferedReader is = new BufferedReader(new InputStreamReader(new FileInputStream(wordsFileIn), StandardCharsets.UTF_8), 65536);
       try {
         final IntsRef intsRef = new IntsRef(10);
         long tStart = System.currentTimeMillis();
@@ -517,7 +518,7 @@ public class TestFSTs extends LuceneTest
 
         System.out.println(ord + " terms; " + fst.getNodeCount() + " nodes; " + fst.getArcCount() + " arcs; " + fst.getArcWithOutputCount() + " arcs w/ output; tot size " + fst.sizeInBytes());
         if (fst.getNodeCount() < 100) {
-          Writer w = new OutputStreamWriter(new FileOutputStream("out.dot"), "UTF-8");
+          Writer w = new OutputStreamWriter(new FileOutputStream("out.dot"), StandardCharsets.UTF_8);
           Util.toDot(fst, w, false, false);
           w.close();
           System.out.println("Wrote FST to out.dot");
@@ -544,7 +545,7 @@ public class TestFSTs extends LuceneTest
         while(true) {
           for(int iter=0;iter<2;iter++) {
             is.close();
-            is = new BufferedReader(new InputStreamReader(new FileInputStream(wordsFileIn), "UTF-8"), 65536);
+            is = new BufferedReader(new InputStreamReader(new FileInputStream(wordsFileIn), StandardCharsets.UTF_8), 65536);
 
             ord = 0;
             tStart = System.currentTimeMillis();