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 [8/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/test-framework/src/java/org/apache/lucene/index/RandomCodec.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/solr5914/lucene/test-framework/src/java/org/apache/lucene/index/RandomCodec.java?rev=1584603&r1=1584602&r2=1584603&view=diff
==============================================================================
--- lucene/dev/branches/solr5914/lucene/test-framework/src/java/org/apache/lucene/index/RandomCodec.java (original)
+++ lucene/dev/branches/solr5914/lucene/test-framework/src/java/org/apache/lucene/index/RandomCodec.java Fri Apr  4 10:27:05 2014
@@ -46,10 +46,7 @@ import org.apache.lucene.codecs.memory.F
 import org.apache.lucene.codecs.memory.FSTPulsing41PostingsFormat;
 import org.apache.lucene.codecs.memory.MemoryDocValuesFormat;
 import org.apache.lucene.codecs.memory.MemoryPostingsFormat;
-import org.apache.lucene.codecs.mockintblock.MockFixedIntBlockPostingsFormat;
-import org.apache.lucene.codecs.mockintblock.MockVariableIntBlockPostingsFormat;
 import org.apache.lucene.codecs.mockrandom.MockRandomPostingsFormat;
-import org.apache.lucene.codecs.mocksep.MockSepPostingsFormat;
 import org.apache.lucene.codecs.nestedpulsing.NestedPulsingPostingsFormat;
 import org.apache.lucene.codecs.pulsing.Pulsing41PostingsFormat;
 import org.apache.lucene.codecs.simpletext.SimpleTextDocValuesFormat;
@@ -142,9 +139,6 @@ public class RandomCodec extends Lucene4
         //with a choice of concrete PostingsFormats. Maybe useful to have a generic means of marking and dealing 
         //with such "wrapper" classes?
         new TestBloomFilteredLucene41Postings(),                
-        new MockSepPostingsFormat(),
-        new MockFixedIntBlockPostingsFormat(TestUtil.nextInt(random, 1, 2000)),
-        new MockVariableIntBlockPostingsFormat( TestUtil.nextInt(random, 1, 127)),
         new MockRandomPostingsFormat(random),
         new NestedPulsingPostingsFormat(),
         new Lucene41WithOrds(TestUtil.nextInt(random, 1, 1000)),

Modified: lucene/dev/branches/solr5914/lucene/test-framework/src/java/org/apache/lucene/store/MockIndexOutputWrapper.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/solr5914/lucene/test-framework/src/java/org/apache/lucene/store/MockIndexOutputWrapper.java?rev=1584603&r1=1584602&r2=1584603&view=diff
==============================================================================
--- lucene/dev/branches/solr5914/lucene/test-framework/src/java/org/apache/lucene/store/MockIndexOutputWrapper.java (original)
+++ lucene/dev/branches/solr5914/lucene/test-framework/src/java/org/apache/lucene/store/MockIndexOutputWrapper.java Fri Apr  4 10:27:05 2014
@@ -166,6 +166,11 @@ public class MockIndexOutputWrapper exte
   }
 
   @Override
+  public long getChecksum() throws IOException {
+    return delegate.getChecksum();
+  }
+
+  @Override
   public String toString() {
     return "MockIndexOutputWrapper(" + delegate + ")";
   }

Modified: lucene/dev/branches/solr5914/lucene/test-framework/src/java/org/apache/lucene/util/LineFileDocs.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/solr5914/lucene/test-framework/src/java/org/apache/lucene/util/LineFileDocs.java?rev=1584603&r1=1584602&r2=1584603&view=diff
==============================================================================
--- lucene/dev/branches/solr5914/lucene/test-framework/src/java/org/apache/lucene/util/LineFileDocs.java (original)
+++ lucene/dev/branches/solr5914/lucene/test-framework/src/java/org/apache/lucene/util/LineFileDocs.java Fri Apr  4 10:27:05 2014
@@ -29,6 +29,7 @@ import java.nio.channels.Channels;
 import java.nio.channels.FileChannel;
 import java.nio.charset.CharsetDecoder;
 import java.nio.charset.CodingErrorAction;
+import java.nio.charset.StandardCharsets;
 import java.util.Random;
 import java.util.concurrent.atomic.AtomicInteger;
 import java.util.zip.GZIPInputStream;
@@ -133,7 +134,7 @@ public class LineFileDocs implements Clo
       } while (b >= 0 && b != 13 && b != 10);
     }
     
-    CharsetDecoder decoder = IOUtils.CHARSET_UTF_8.newDecoder()
+    CharsetDecoder decoder = StandardCharsets.UTF_8.newDecoder()
         .onMalformedInput(CodingErrorAction.REPORT)
         .onUnmappableCharacter(CodingErrorAction.REPORT);
     reader = new BufferedReader(new InputStreamReader(is, decoder), BUFFER_SIZE);

Modified: lucene/dev/branches/solr5914/lucene/test-framework/src/java/org/apache/lucene/util/LuceneTestCase.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/solr5914/lucene/test-framework/src/java/org/apache/lucene/util/LuceneTestCase.java?rev=1584603&r1=1584602&r2=1584603&view=diff
==============================================================================
--- lucene/dev/branches/solr5914/lucene/test-framework/src/java/org/apache/lucene/util/LuceneTestCase.java (original)
+++ lucene/dev/branches/solr5914/lucene/test-framework/src/java/org/apache/lucene/util/LuceneTestCase.java Fri Apr  4 10:27:05 2014
@@ -126,7 +126,8 @@ import static com.carrotsearch.randomize
   JUnit4MethodProvider.class
 })
 @Listeners({
-  RunListenerPrintReproduceInfo.class
+  RunListenerPrintReproduceInfo.class,
+  FailureMarker.class
 })
 @SeedDecorators({MixWithSuiteName.class}) // See LUCENE-3995 for rationale.
 @ThreadLeakScope(Scope.SUITE)
@@ -314,14 +315,6 @@ public abstract class LuceneTestCase ext
     CORE_DIRECTORIES.add("RAMDirectory");
   };
   
-  protected static final Set<String> doesntSupportOffsets = new HashSet<>(Arrays.asList(
-    "Lucene3x",
-    "MockFixedIntBlock",
-    "MockVariableIntBlock",
-    "MockSep",
-    "MockRandom"
-  ));
-  
   // -----------------------------------------------------------------
   // Fields initialized in class or instance rules.
   // -----------------------------------------------------------------
@@ -353,8 +346,7 @@ public abstract class LuceneTestCase ext
   /**
    * Suite failure marker (any error in the test or suite scope).
    */
-  public final static TestRuleMarkFailure suiteFailureMarker = 
-      new TestRuleMarkFailure();
+  public static TestRuleMarkFailure suiteFailureMarker;
 
   /**
    * Ignore tests after hitting a designated number of initial failures. This
@@ -419,7 +411,7 @@ public abstract class LuceneTestCase ext
   public static TestRule classRules = RuleChain
     .outerRule(new TestRuleIgnoreTestSuites())
     .around(ignoreAfterMaxFailures)
-    .around(suiteFailureMarker)
+    .around(suiteFailureMarker = new TestRuleMarkFailure())
     .around(new TestRuleAssertionsRequired())
     .around(new StaticFieldsInvariantRule(STATIC_LEAK_THRESHOLD, true) {
       @Override
@@ -800,6 +792,7 @@ public abstract class LuceneTestCase ext
     }
     c.setUseCompoundFile(r.nextBoolean());
     c.setReaderPooling(r.nextBoolean());
+    c.setCheckIntegrityAtMerge(r.nextBoolean());
     return c;
   }
 

Modified: lucene/dev/branches/solr5914/lucene/test-framework/src/java/org/apache/lucene/util/TestRuleIgnoreAfterMaxFailures.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/solr5914/lucene/test-framework/src/java/org/apache/lucene/util/TestRuleIgnoreAfterMaxFailures.java?rev=1584603&r1=1584602&r2=1584603&view=diff
==============================================================================
--- lucene/dev/branches/solr5914/lucene/test-framework/src/java/org/apache/lucene/util/TestRuleIgnoreAfterMaxFailures.java (original)
+++ lucene/dev/branches/solr5914/lucene/test-framework/src/java/org/apache/lucene/util/TestRuleIgnoreAfterMaxFailures.java Fri Apr  4 10:27:05 2014
@@ -40,11 +40,6 @@ public final class TestRuleIgnoreAfterMa
    * Maximum failures. Package scope for tests.
    */
   int maxFailures;
-
-  /**
-   * Current count of failures. Package scope for tests.
-   */
-  int failuresSoFar;
   
   /**
    * @param maxFailures
@@ -61,19 +56,13 @@ public final class TestRuleIgnoreAfterMa
     return new Statement() {
       @Override
       public void evaluate() throws Throwable {
+        int failuresSoFar = FailureMarker.getFailures();
         if (failuresSoFar >= maxFailures) {
           RandomizedTest.assumeTrue("Ignored, failures limit reached (" + 
               failuresSoFar + " >= " + maxFailures + ").", false);
         }
 
-        try {
-          s.evaluate();
-        } catch (Throwable t) {
-          if (!TestRuleMarkFailure.isAssumption(t)) {
-            failuresSoFar++;
-          }
-          throw t;
-        }
+        s.evaluate();
       }
     };
   }

Modified: lucene/dev/branches/solr5914/lucene/test-framework/src/java/org/apache/lucene/util/TestUtil.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/solr5914/lucene/test-framework/src/java/org/apache/lucene/util/TestUtil.java?rev=1584603&r1=1584602&r2=1584603&view=diff
==============================================================================
--- lucene/dev/branches/solr5914/lucene/test-framework/src/java/org/apache/lucene/util/TestUtil.java (original)
+++ lucene/dev/branches/solr5914/lucene/test-framework/src/java/org/apache/lucene/util/TestUtil.java Fri Apr  4 10:27:05 2014
@@ -103,42 +103,38 @@ public final class TestUtil {
   private static final int GET_TEMP_DIR_RETRY_THRESHOLD = 1000;
   
   /**
-   * Deletes a directory and everything underneath it.
+   * Deletes a file or a directory (and everything underneath it).
    */
-  public static void rmDir(File dir) throws IOException {
-    if (dir.exists()) {
-      if (dir.isFile() && !dir.delete()) {
-        throw new IOException("could not delete " + dir);
-      }
-      for (File f : dir.listFiles()) {
-        if (f.isDirectory()) {
-          rmDir(f);
-        } else {
-          if (!f.delete()) {
-            throw new IOException("could not delete " + f);
-          }
-        }
+  public static void rm(File location) throws IOException {
+    if (!location.exists()) {
+      return;
+    }
+
+    if (location.isDirectory()) {
+      for (File f : location.listFiles()) {
+        rm(f);
       }
-      if (!dir.delete()) {
-        throw new IOException("could not delete " + dir);
+    } else {
+      if (!location.delete()) {
+        throw new IOException("Could not delete: " + location.getAbsolutePath());
       }
     }
+
+    assert !location.exists();
   }
 
   /** 
-   * Convenience method: Unzip zipName + ".zip" under destDir, removing destDir first 
+   * Convenience method: Unzip zipName + ".zip" into destDir, cleaning up 
+   * destDir first. 
    */
   public static void unzip(File zipName, File destDir) throws IOException {
-    
-    ZipFile zipFile = new ZipFile(zipName);
-    
-    Enumeration<? extends ZipEntry> entries = zipFile.entries();
-    
-    rmDir(destDir);
-
+    rm(destDir);
     destDir.mkdir();
     LuceneTestCase.closeAfterSuite(new CloseableFile(destDir, LuceneTestCase.suiteFailureMarker));
 
+    ZipFile zipFile = new ZipFile(zipName);
+    Enumeration<? extends ZipEntry> entries = zipFile.entries();
+
     while (entries.hasMoreElements()) {
       ZipEntry entry = entries.nextElement();
       
@@ -189,15 +185,15 @@ public final class TestUtil {
     ByteArrayOutputStream bos = new ByteArrayOutputStream(1024);
     CheckIndex checker = new CheckIndex(dir);
     checker.setCrossCheckTermVectors(crossCheckTermVectors);
-    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);
     if (indexStatus == null || indexStatus.clean == false) {
       System.out.println("CheckIndex failed");
-      System.out.println(bos.toString("UTF-8"));
+      System.out.println(bos.toString(IOUtils.UTF_8));
       throw new RuntimeException("CheckIndex failed");
     } else {
       if (LuceneTestCase.INFOSTREAM) {
-        System.out.println(bos.toString("UTF-8"));
+        System.out.println(bos.toString(IOUtils.UTF_8));
       }
       return indexStatus;
     }
@@ -213,8 +209,9 @@ public final class TestUtil {
   
   public static void checkReader(AtomicReader reader, boolean crossCheckTermVectors) throws IOException {
     ByteArrayOutputStream bos = new ByteArrayOutputStream(1024);
-    PrintStream infoStream = new PrintStream(bos, false, "UTF-8");
+    PrintStream infoStream = new PrintStream(bos, false, IOUtils.UTF_8);
 
+    reader.checkIntegrity();
     FieldNormStatus fieldNormStatus = CheckIndex.testFieldNorms(reader, infoStream);
     TermIndexStatus termIndexStatus = CheckIndex.testPostings(reader, infoStream);
     StoredFieldStatus storedFieldStatus = CheckIndex.testStoredFields(reader, infoStream);
@@ -227,11 +224,11 @@ public final class TestUtil {
       termVectorStatus.error != null ||
       docValuesStatus.error != null) {
       System.out.println("CheckReader failed");
-      System.out.println(bos.toString("UTF-8"));
+      System.out.println(bos.toString(IOUtils.UTF_8));
       throw new RuntimeException("CheckReader failed");
     } else {
       if (LuceneTestCase.INFOSTREAM) {
-        System.out.println(bos.toString("UTF-8"));
+        System.out.println(bos.toString(IOUtils.UTF_8));
       }
     }
   }

Modified: lucene/dev/branches/solr5914/lucene/test-framework/src/java/org/apache/lucene/util/ThrottledIndexOutput.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/solr5914/lucene/test-framework/src/java/org/apache/lucene/util/ThrottledIndexOutput.java?rev=1584603&r1=1584602&r2=1584603&view=diff
==============================================================================
--- lucene/dev/branches/solr5914/lucene/test-framework/src/java/org/apache/lucene/util/ThrottledIndexOutput.java (original)
+++ lucene/dev/branches/solr5914/lucene/test-framework/src/java/org/apache/lucene/util/ThrottledIndexOutput.java Fri Apr  4 10:27:05 2014
@@ -145,4 +145,9 @@ public class ThrottledIndexOutput extend
   public void copyBytes(DataInput input, long numBytes) throws IOException {
     delegate.copyBytes(input, numBytes);
   }
+
+  @Override
+  public long getChecksum() throws IOException {
+    return delegate.getChecksum();
+  }
 }

Modified: lucene/dev/branches/solr5914/lucene/test-framework/src/java/org/apache/lucene/util/fst/FSTTester.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/solr5914/lucene/test-framework/src/java/org/apache/lucene/util/fst/FSTTester.java?rev=1584603&r1=1584602&r2=1584603&view=diff
==============================================================================
--- lucene/dev/branches/solr5914/lucene/test-framework/src/java/org/apache/lucene/util/fst/FSTTester.java (original)
+++ lucene/dev/branches/solr5914/lucene/test-framework/src/java/org/apache/lucene/util/fst/FSTTester.java Fri Apr  4 10:27:05 2014
@@ -21,6 +21,7 @@ import java.io.FileOutputStream;
 import java.io.IOException;
 import java.io.OutputStreamWriter;
 import java.io.Writer;
+import java.nio.charset.StandardCharsets;
 import java.util.ArrayList;
 import java.util.Collections;
 import java.util.HashMap;
@@ -328,7 +329,7 @@ public class FSTTester<T> {
     }
 
     if (LuceneTestCase.VERBOSE && pairs.size() <= 20 && fst != null) {
-      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("SAVED out.dot");

Modified: lucene/dev/branches/solr5914/lucene/test-framework/src/resources/META-INF/services/org.apache.lucene.codecs.PostingsFormat
URL: http://svn.apache.org/viewvc/lucene/dev/branches/solr5914/lucene/test-framework/src/resources/META-INF/services/org.apache.lucene.codecs.PostingsFormat?rev=1584603&r1=1584602&r2=1584603&view=diff
==============================================================================
--- lucene/dev/branches/solr5914/lucene/test-framework/src/resources/META-INF/services/org.apache.lucene.codecs.PostingsFormat (original)
+++ lucene/dev/branches/solr5914/lucene/test-framework/src/resources/META-INF/services/org.apache.lucene.codecs.PostingsFormat Fri Apr  4 10:27:05 2014
@@ -13,10 +13,7 @@
 #  See the License for the specific language governing permissions and
 #  limitations under the License.
 
-org.apache.lucene.codecs.mockintblock.MockFixedIntBlockPostingsFormat
-org.apache.lucene.codecs.mockintblock.MockVariableIntBlockPostingsFormat
 org.apache.lucene.codecs.mockrandom.MockRandomPostingsFormat
-org.apache.lucene.codecs.mocksep.MockSepPostingsFormat
 org.apache.lucene.codecs.nestedpulsing.NestedPulsingPostingsFormat
 org.apache.lucene.codecs.ramonly.RAMOnlyPostingsFormat
 org.apache.lucene.codecs.lucene41ords.Lucene41WithOrds

Modified: lucene/dev/branches/solr5914/lucene/tools/src/java/org/apache/lucene/dependencies/GetMavenDependenciesTask.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/solr5914/lucene/tools/src/java/org/apache/lucene/dependencies/GetMavenDependenciesTask.java?rev=1584603&r1=1584602&r2=1584603&view=diff
==============================================================================
--- lucene/dev/branches/solr5914/lucene/tools/src/java/org/apache/lucene/dependencies/GetMavenDependenciesTask.java (original)
+++ lucene/dev/branches/solr5914/lucene/tools/src/java/org/apache/lucene/dependencies/GetMavenDependenciesTask.java Fri Apr  4 10:27:05 2014
@@ -38,9 +38,9 @@ import java.io.InputStream;
 import java.io.InputStreamReader;
 import java.io.OutputStreamWriter;
 import java.io.Reader;
-import java.io.UnsupportedEncodingException;
 import java.io.Writer;
 import java.nio.charset.Charset;
+import java.nio.charset.StandardCharsets;
 import java.util.ArrayList;
 import java.util.Arrays;
 import java.util.Collection;
@@ -58,6 +58,7 @@ import java.util.TreeMap;
 import java.util.TreeSet;
 import java.util.regex.Matcher;
 import java.util.regex.Pattern;
+
 import javax.xml.parsers.DocumentBuilder;
 import javax.xml.parsers.DocumentBuilderFactory;
 import javax.xml.parsers.ParserConfigurationException;
@@ -91,17 +92,19 @@ public class GetMavenDependenciesTask ex
   private static final Properties allProperties = new Properties();
   private static final Set<String> modulesWithSeparateCompileAndTestPOMs = new HashSet<>();
 
-  private static final Set<String>  optionalExternalDependencies = new HashSet<>();
+  private static final Set<String> globalOptionalExternalDependencies = new HashSet<>();
+  private static final Map<String,Set<String>> perModuleOptionalExternalDependencies = new HashMap<>();
   static {
     // Add modules here that have split compile and test POMs
     // - they need compile-scope deps to also be test-scope deps.
     modulesWithSeparateCompileAndTestPOMs.addAll
         (Arrays.asList("lucene-core", "lucene-codecs", "solr-core", "solr-solrj"));
 
-    // Add external dependencies here that should be optional (i.e., not invoke Maven's transitive dep mechanism).
+    // Add external dependencies here that should be optional for all modules
+    // (i.e., not invoke Maven's transitive dependency mechanism).
     // Format is "groupId:artifactId"
-    optionalExternalDependencies.addAll(Arrays.asList
-        ("org.slf4j:jcl-over-slf4j", "org.slf4j:jul-to-slf4j", "org.slf4j:slf4j-api", "org.slf4j:slf4j-log4j12"));
+    globalOptionalExternalDependencies.addAll(Arrays.asList
+        ("org.slf4j:jcl-over-slf4j", "org.slf4j:jul-to-slf4j", "org.slf4j:slf4j-log4j12"));
   }
 
   private final XPath xpath = XPathFactory.newInstance().newXPath();
@@ -151,7 +154,7 @@ public class GetMavenDependenciesTask ex
   }
 
   public void setVerbose(boolean verbose) {
-    verboseLevel = (verbose ? Project.MSG_INFO : Project.MSG_VERBOSE);
+    verboseLevel = (verbose ? Project.MSG_VERBOSE : Project.MSG_INFO);
   }
 
   public void setCentralizedVersionsFile(File file) {
@@ -200,12 +203,10 @@ public class GetMavenDependenciesTask ex
     Writer writer = null;
     try {
       FileOutputStream outputStream = new FileOutputStream(mavenDependenciesFiltersFile);
-      writer = new OutputStreamWriter(outputStream, "ISO-8859-1");
+      writer = new OutputStreamWriter(outputStream, StandardCharsets.ISO_8859_1);
       allProperties.store(writer, null);
     } catch (FileNotFoundException e) {
       throw new BuildException("Can't find file: '" + mavenDependenciesFiltersFile.getPath() + "'", e);
-    } catch (UnsupportedEncodingException e) {
-      throw new BuildException(e);
     } catch (IOException e) {
       throw new BuildException("Exception writing out '" + mavenDependenciesFiltersFile.getPath() + "'", e);
     } finally {
@@ -242,7 +243,7 @@ public class GetMavenDependenciesTask ex
       } catch (BuildException e) {
         throw e;
       } catch (Exception e) {
-        throw new BuildException("Exception reading file " + ivyXmlFile.getPath(), e);
+        throw new BuildException("Exception reading file " + ivyXmlFile.getPath() + ": " + e, e);
       }
     }
     addSharedExternalDependencies();
@@ -258,10 +259,10 @@ public class GetMavenDependenciesTask ex
     // Delay adding shared compile-scope dependencies until after all have been processed,
     // so dependency sharing is limited to a depth of one.
     Map<String,SortedSet<ExternalDependency>> sharedDependencies = new HashMap<>();
-    for (String artifactId : interModuleExternalCompileScopeDependencies.keySet()) {
+    for (String module : interModuleExternalCompileScopeDependencies.keySet()) {
       TreeSet<ExternalDependency> deps = new TreeSet<>();
-      sharedDependencies.put(artifactId, deps);
-      Set<String> moduleDependencies = interModuleExternalCompileScopeDependencies.get(artifactId);
+      sharedDependencies.put(module, deps);
+      Set<String> moduleDependencies = interModuleExternalCompileScopeDependencies.get(module);
       if (null != moduleDependencies) {
         for (String otherArtifactId : moduleDependencies) {
           SortedSet<ExternalDependency> otherExtDeps = allExternalDependencies.get(otherArtifactId); 
@@ -275,13 +276,13 @@ public class GetMavenDependenciesTask ex
         }
       }
     }
-    for (String artifactId : interModuleExternalTestScopeDependencies.keySet()) {
-      SortedSet<ExternalDependency> deps = sharedDependencies.get(artifactId);
+    for (String module : interModuleExternalTestScopeDependencies.keySet()) {
+      SortedSet<ExternalDependency> deps = sharedDependencies.get(module);
       if (null == deps) {
         deps = new TreeSet<>();
-        sharedDependencies.put(artifactId, deps);
+        sharedDependencies.put(module, deps);
       }
-      Set<String> moduleDependencies = interModuleExternalTestScopeDependencies.get(artifactId);
+      Set<String> moduleDependencies = interModuleExternalTestScopeDependencies.get(module);
       if (null != moduleDependencies) {
         for (String otherArtifactId : moduleDependencies) {
           int testScopePos = otherArtifactId.indexOf(":test");
@@ -295,8 +296,8 @@ public class GetMavenDependenciesTask ex
             for (ExternalDependency otherDep : otherExtDeps) {
               if (otherDep.isTestDependency == isTestScope) {
                 if (  ! deps.contains(otherDep)
-                   && (  null == allExternalDependencies.get(artifactId)
-                      || ! allExternalDependencies.get(artifactId).contains(otherDep))) { 
+                   && (  null == allExternalDependencies.get(module)
+                      || ! allExternalDependencies.get(module).contains(otherDep))) {
                   // Add test-scope clone only if it's not already a compile-scope dependency. 
                   ExternalDependency otherDepTestScope = new ExternalDependency
                       (otherDep.groupId, otherDep.artifactId, otherDep.classifier, true, otherDep.isOptional);
@@ -308,13 +309,21 @@ public class GetMavenDependenciesTask ex
         }
       }
     }
-    for (String artifactId : sharedDependencies.keySet()) {
-      SortedSet<ExternalDependency> deps = allExternalDependencies.get(artifactId);
+    for (String module : sharedDependencies.keySet()) {
+      SortedSet<ExternalDependency> deps = allExternalDependencies.get(module);
       if (null == deps) {
         deps = new TreeSet<>();
-        allExternalDependencies.put(artifactId, deps);
+        allExternalDependencies.put(module, deps);
+      }
+      for (ExternalDependency dep : sharedDependencies.get(module)) {
+        String dependencyCoordinate = dep.groupId + ":" + dep.artifactId;
+        if (globalOptionalExternalDependencies.contains(dependencyCoordinate)
+            || (perModuleOptionalExternalDependencies.containsKey(module)
+                && perModuleOptionalExternalDependencies.get(module).contains(dependencyCoordinate))) {
+          dep = new ExternalDependency(dep.groupId, dep.artifactId, dep.classifier, dep.isTestDependency, true);
+        }
+        deps.add(dep);
       }
-      deps.addAll(sharedDependencies.get(artifactId));
     }
   }
 
@@ -615,7 +624,7 @@ public class GetMavenDependenciesTask ex
       // Pattern.compile("(lucene|solr)/build/(.*)/classes/java");
       String artifact = matcher.group(2);
       artifact = artifact.replace('/', '-');
-      artifact = artifact.replace("analysis-", "analyzers-");
+      artifact = artifact.replace("(?<!solr-)analysis-", "analyzers-");
       if ("lucene".equals(matcher.group(1))) {
         artifactId.append("lucene-");
       }
@@ -672,7 +681,9 @@ public class GetMavenDependenciesTask ex
       }
       String conf = dependency.getAttribute("conf");
       boolean confContainsTest = conf.contains("test");
-      boolean isOptional = optionalExternalDependencies.contains(dependencyCoordinate);
+      boolean isOptional = globalOptionalExternalDependencies.contains(dependencyCoordinate)
+          || ( perModuleOptionalExternalDependencies.containsKey(module)
+              && perModuleOptionalExternalDependencies.get(module).contains(dependencyCoordinate));
       SortedSet<ExternalDependency> deps = allExternalDependencies.get(module);
       if (null == deps) {
         deps = new TreeSet<>();
@@ -808,7 +819,7 @@ public class GetMavenDependenciesTask ex
       throw new BuildException("Properties file does not exist: " + file.getPath());
     }
     // Properties files are encoded as Latin-1
-    final Reader reader = new InputStreamReader(stream, Charset.forName("ISO-8859-1"));
+    final Reader reader = new InputStreamReader(stream, StandardCharsets.ISO_8859_1);
     final Properties properties = new Properties(); 
     try {
       properties.load(reader);

Modified: lucene/dev/branches/solr5914/lucene/tools/src/java/org/apache/lucene/validation/LibVersionsCheckTask.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/solr5914/lucene/tools/src/java/org/apache/lucene/validation/LibVersionsCheckTask.java?rev=1584603&r1=1584602&r2=1584603&view=diff
==============================================================================
--- lucene/dev/branches/solr5914/lucene/tools/src/java/org/apache/lucene/validation/LibVersionsCheckTask.java (original)
+++ lucene/dev/branches/solr5914/lucene/tools/src/java/org/apache/lucene/validation/LibVersionsCheckTask.java Fri Apr  4 10:27:05 2014
@@ -33,6 +33,7 @@ import org.xml.sax.helpers.DefaultHandle
 import org.xml.sax.helpers.XMLReaderFactory;
 
 import javax.xml.parsers.ParserConfigurationException;
+
 import java.io.BufferedReader;
 import java.io.File;
 import java.io.FileInputStream;
@@ -44,6 +45,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;
 import java.util.Iterator;
 import java.util.LinkedHashMap;
 import java.util.Locale;
@@ -188,7 +190,7 @@ public class LibVersionsCheckTask extend
           + centralizedVersionsFile.getPath());
     }
     // Properties files are encoded as Latin-1
-    final Reader reader = new InputStreamReader(stream, Charset.forName("ISO-8859-1"));
+    final Reader reader = new InputStreamReader(stream, StandardCharsets.ISO_8859_1);
     final BufferedReader bufferedReader = new BufferedReader(reader);
     
     String line = null;

Modified: lucene/dev/branches/solr5914/lucene/tools/src/java/org/apache/lucene/validation/LicenseCheckTask.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/solr5914/lucene/tools/src/java/org/apache/lucene/validation/LicenseCheckTask.java?rev=1584603&r1=1584602&r2=1584603&view=diff
==============================================================================
--- lucene/dev/branches/solr5914/lucene/tools/src/java/org/apache/lucene/validation/LicenseCheckTask.java (original)
+++ lucene/dev/branches/solr5914/lucene/tools/src/java/org/apache/lucene/validation/LicenseCheckTask.java Fri Apr  4 10:27:05 2014
@@ -22,14 +22,13 @@ import java.io.FileInputStream;
 import java.io.BufferedReader;
 import java.io.InputStreamReader;
 import java.io.IOException;
-
+import java.nio.charset.StandardCharsets;
 import java.util.ArrayList;
 import java.util.Iterator;
 import java.util.LinkedHashMap;
 import java.util.List;
 import java.util.Locale;
 import java.util.Map;
-
 import java.security.DigestInputStream;
 import java.security.MessageDigest;
 import java.security.NoSuchAlgorithmException;
@@ -302,7 +301,7 @@ outer:
     BufferedReader reader = null;
     try {
       reader = new BufferedReader(new InputStreamReader
-                                  (new FileInputStream(f), "UTF-8"));
+                                  (new FileInputStream(f), StandardCharsets.UTF_8));
       try {
         String checksum = reader.readLine();
         if (null == checksum || 0 == checksum.length()) {

Modified: lucene/dev/branches/solr5914/solr/CHANGES.txt
URL: http://svn.apache.org/viewvc/lucene/dev/branches/solr5914/solr/CHANGES.txt?rev=1584603&r1=1584602&r2=1584603&view=diff
==============================================================================
--- lucene/dev/branches/solr5914/solr/CHANGES.txt (original)
+++ lucene/dev/branches/solr5914/solr/CHANGES.txt Fri Apr  4 10:27:05 2014
@@ -146,9 +146,6 @@ New Features
   improve logging and force refresh cluster state every 15 seconds.
   (Timothy Potter via shalin)
   
- * SOLR-5228: Don't require <field> or <dynamicField> be inside of <fields> -- or 
-   that <fieldType> be inside of <types>. (Erick Erickson)
-
 * SOLR-5749: A new Overseer status collection API exposes overseer queue sizes, timing
   statistics, success and error counts and last N failures per operation. (shalin)
 
@@ -159,6 +156,21 @@ New Features
 * SOLR-4478: Allow cores to use configuration from a configsets directory
   outside their instance directory. (Alan Woodward, Erick Erickson)
 
+* SOLR-5466: A new List collections and cluster status API which clients can use
+  to read collection and shard information instead of reading data directly from ZooKeeper.
+  (Dave Seltzer, Varun Thacker, Vitaliy Zhovtyuk, Erick Erickson, shalin)
+
+* SOLR-5795: New DocExpirationUpdateProcessorFactory supports computing an expiration 
+  date for documents from the "TTL" expression, as well as automatically deleting expired 
+  documents on a periodic basis. (hossman)
+
+* SOLR-5829: Allow ExpandComponent to accept query and filter query parameters
+  (Joel Bernstein)
+
+* SOLR-5654: Create a synonym filter factory that is (re)configurable, and
+  capable of reporting its configuration, via REST API.
+  (Tim Potter via Steve Rowe)
+
 Bug Fixes
 ----------------------
 
@@ -167,6 +179,20 @@ Bug Fixes
 
 * SOLR-5893: On restarting overseer designate , move itself to front of the queue (Noble Paul)
 
+* SOLR-5915: Attempts to specify the parserImpl for
+  solr.PreAnalyzedField fieldtype failed.  (Mike McCandless)
+
+* SOLR-5943: SolrCmdDistributor does not distribute the openSearcher parameter.
+  (ludovic Boutros via shalin)
+
+* SOLR-5951: Fixed SolrDispatchFilter to throw useful exception on startup if
+  SLF4j logging jars are missing.  (Uwe Schindler, Hossman, Shawn Heisey)
+
+* SOLR-5950: Maven config: make the org.slf4j:slf4j-api dependency transitive
+  (i.e., not optional) in all modules in which it's a dependency, including
+  solrj, except for the WAR, where it will remain optional.
+  (Uwe Schindler, Steve Rowe)
+
 Optimizations
 ----------------------
 * SOLR-1880: Distributed Search skips GET_FIELDS stage if EXECUTE_QUERY
@@ -243,6 +269,12 @@ Other Changes
 * SOLR-5914: Cleanup and fix Solr's test cleanup code. 
  (Mark Miller, Uwe Schindler)
 
+* SOLR-5934: LBHttpSolrServer exception handling improvement and small test
+  improvements. (Gregory Chanan via Mark Miller)
+
+* SOLR-5773: CollapsingQParserPlugin should make elevated documents the 
+  group head. (David Boychuck, Joel Bernstein)
+
 ==================  4.7.1  ==================
 
 Versions of Major Components
@@ -344,6 +376,9 @@ Bug Fixes
 * SOLR-5906: Collection create API ignores property.instanceDir parameter.
   (Varun Thacker, shalin)
 
+* SOLR-5920: Distributed sort on DateField, BoolField and BCD{Int,Long,Str}Field
+  returns string cast exception (Eric Bus, AJ Lemke, hossman, Steve Rowe)
+
 Other Changes
 ---------------------
 

Modified: lucene/dev/branches/solr5914/solr/common-build.xml
URL: http://svn.apache.org/viewvc/lucene/dev/branches/solr5914/solr/common-build.xml?rev=1584603&r1=1584602&r2=1584603&view=diff
==============================================================================
--- lucene/dev/branches/solr5914/solr/common-build.xml (original)
+++ lucene/dev/branches/solr5914/solr/common-build.xml Fri Apr  4 10:27:05 2014
@@ -445,6 +445,12 @@
   </macrodef>
 
   <!-- Solr contrib targets -->
+  <target name="-compile-analysis-extras">
+    <ant dir="${common-solr.dir}/contrib/analysis-extras" target="compile" inheritAll="false">
+      <propertyset refid="uptodate.and.compiled.properties"/>
+    </ant>
+  </target>
+
   <target name="compile-contrib" description="Compile contrib modules">
   	<contrib-crawl target="compile-core"/>
   </target>

Modified: lucene/dev/branches/solr5914/solr/contrib/analysis-extras/src/java/org/apache/solr/schema/ICUCollationField.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/solr5914/solr/contrib/analysis-extras/src/java/org/apache/solr/schema/ICUCollationField.java?rev=1584603&r1=1584602&r2=1584603&view=diff
==============================================================================
--- lucene/dev/branches/solr5914/solr/contrib/analysis-extras/src/java/org/apache/solr/schema/ICUCollationField.java (original)
+++ lucene/dev/branches/solr5914/solr/contrib/analysis-extras/src/java/org/apache/solr/schema/ICUCollationField.java Fri Apr  4 10:27:05 2014
@@ -43,7 +43,6 @@ import org.apache.lucene.util.Version;
 import org.apache.lucene.analysis.util.ResourceLoader;
 import org.apache.solr.common.SolrException;
 import org.apache.solr.common.SolrException.ErrorCode;
-import org.apache.solr.common.util.Base64;
 import org.apache.solr.response.TextResponseWriter;
 import org.apache.solr.search.QParser;
 
@@ -303,20 +302,11 @@ public class ICUCollationField extends F
 
   @Override
   public Object marshalSortValue(Object value) {
-    if (null == value) {
-      return null;
-    }
-    final BytesRef val = (BytesRef)value;
-    return Base64.byteArrayToBase64(val.bytes, val.offset, val.length);
+    return marshalBase64SortValue(value);
   }
 
   @Override
   public Object unmarshalSortValue(Object value) {
-    if (null == value) {
-      return null;
-    }
-    final String val = (String)value;
-    final byte[] bytes = Base64.base64ToByteArray(val);
-    return new BytesRef(bytes);
+    return unmarshalBase64SortValue(value);
   }
 }

Modified: lucene/dev/branches/solr5914/solr/contrib/dataimporthandler/src/java/org/apache/solr/handler/dataimport/FieldReaderDataSource.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/solr5914/solr/contrib/dataimporthandler/src/java/org/apache/solr/handler/dataimport/FieldReaderDataSource.java?rev=1584603&r1=1584602&r2=1584603&view=diff
==============================================================================
--- lucene/dev/branches/solr5914/solr/contrib/dataimporthandler/src/java/org/apache/solr/handler/dataimport/FieldReaderDataSource.java (original)
+++ lucene/dev/branches/solr5914/solr/contrib/dataimporthandler/src/java/org/apache/solr/handler/dataimport/FieldReaderDataSource.java Fri Apr  4 10:27:05 2014
@@ -18,10 +18,12 @@ package org.apache.solr.handler.dataimpo
 
 import static org.apache.solr.handler.dataimport.DataImportHandlerException.SEVERE;
 import static org.apache.solr.handler.dataimport.DataImportHandlerException.wrapAndThrow;
+
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
 import java.io.*;
+import java.nio.charset.StandardCharsets;
 import java.sql.Blob;
 import java.sql.Clob;
 import java.sql.SQLException;
@@ -106,7 +108,7 @@ public class FieldReaderDataSource exten
   private Reader getReader(Blob blob)
           throws SQLException, UnsupportedEncodingException {
     if (encoding == null) {
-      return (new InputStreamReader(blob.getBinaryStream(), "UTF-8"));
+      return (new InputStreamReader(blob.getBinaryStream(), StandardCharsets.UTF_8));
     } else {
       return (new InputStreamReader(blob.getBinaryStream(), encoding));
     }

Modified: lucene/dev/branches/solr5914/solr/contrib/dataimporthandler/src/java/org/apache/solr/handler/dataimport/FileDataSource.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/solr5914/solr/contrib/dataimporthandler/src/java/org/apache/solr/handler/dataimport/FileDataSource.java?rev=1584603&r1=1584602&r2=1584603&view=diff
==============================================================================
--- lucene/dev/branches/solr5914/solr/contrib/dataimporthandler/src/java/org/apache/solr/handler/dataimport/FileDataSource.java (original)
+++ lucene/dev/branches/solr5914/solr/contrib/dataimporthandler/src/java/org/apache/solr/handler/dataimport/FileDataSource.java Fri Apr  4 10:27:05 2014
@@ -17,10 +17,12 @@
 package org.apache.solr.handler.dataimport;
 
 import java.io.*;
+import java.nio.charset.StandardCharsets;
 import java.util.Properties;
 
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
+
 import static org.apache.solr.handler.dataimport.DataImportHandlerException.wrapAndThrow;
 import static org.apache.solr.handler.dataimport.DataImportHandlerException.SEVERE;
 
@@ -138,7 +140,7 @@ public class FileDataSource extends Data
   protected Reader openStream(File file) throws FileNotFoundException,
           UnsupportedEncodingException {
     if (encoding == null) {
-      return new InputStreamReader(new FileInputStream(file), "UTF-8");
+      return new InputStreamReader(new FileInputStream(file), StandardCharsets.UTF_8);
     } else {
       return new InputStreamReader(new FileInputStream(file), encoding);
     }

Modified: lucene/dev/branches/solr5914/solr/contrib/dataimporthandler/src/java/org/apache/solr/handler/dataimport/SimplePropertiesWriter.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/solr5914/solr/contrib/dataimporthandler/src/java/org/apache/solr/handler/dataimport/SimplePropertiesWriter.java?rev=1584603&r1=1584602&r2=1584603&view=diff
==============================================================================
--- lucene/dev/branches/solr5914/solr/contrib/dataimporthandler/src/java/org/apache/solr/handler/dataimport/SimplePropertiesWriter.java (original)
+++ lucene/dev/branches/solr5914/solr/contrib/dataimporthandler/src/java/org/apache/solr/handler/dataimport/SimplePropertiesWriter.java Fri Apr  4 10:27:05 2014
@@ -26,6 +26,7 @@ import java.io.InputStream;
 import java.io.InputStreamReader;
 import java.io.OutputStreamWriter;
 import java.io.Writer;
+import java.nio.charset.StandardCharsets;
 import java.text.ParseException;
 import java.text.SimpleDateFormat;
 import java.util.Date;
@@ -192,7 +193,7 @@ public class SimplePropertiesWriter exte
         filePath += File.separator;
       }
       filePath += filename;
-      propOutput = new OutputStreamWriter(new FileOutputStream(filePath), IOUtils.CHARSET_UTF_8);
+      propOutput = new OutputStreamWriter(new FileOutputStream(filePath), StandardCharsets.UTF_8);
       existingProps.store(propOutput, null);
       log.info("Wrote last indexed time to " + filename);
     } catch (Exception e) {
@@ -214,7 +215,7 @@ public class SimplePropertiesWriter exte
       }
       filePath += filename;
       propInput = new FileInputStream(filePath);
-      props.load(new InputStreamReader(propInput, IOUtils.CHARSET_UTF_8));
+      props.load(new InputStreamReader(propInput, StandardCharsets.UTF_8));
       log.info("Read " + filename);
     } catch (Exception e) {
       log.warn("Unable to read: " + filename);

Modified: lucene/dev/branches/solr5914/solr/contrib/dataimporthandler/src/java/org/apache/solr/handler/dataimport/SolrWriter.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/solr5914/solr/contrib/dataimporthandler/src/java/org/apache/solr/handler/dataimport/SolrWriter.java?rev=1584603&r1=1584602&r2=1584603&view=diff
==============================================================================
--- lucene/dev/branches/solr5914/solr/contrib/dataimporthandler/src/java/org/apache/solr/handler/dataimport/SolrWriter.java (original)
+++ lucene/dev/branches/solr5914/solr/contrib/dataimporthandler/src/java/org/apache/solr/handler/dataimport/SolrWriter.java Fri Apr  4 10:27:05 2014
@@ -28,6 +28,7 @@ import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
 import java.io.*;
+import java.nio.charset.StandardCharsets;
 
 /**
  * <p> Writes documents to SOLR. </p>
@@ -147,7 +148,7 @@ public class SolrWriter extends DIHWrite
 
       }
     }
-    return new String(baos.toByteArray(), "UTF-8");
+    return new String(baos.toByteArray(), StandardCharsets.UTF_8);
   }
 
   static String getDocCount() {

Modified: lucene/dev/branches/solr5914/solr/contrib/dataimporthandler/src/java/org/apache/solr/handler/dataimport/URLDataSource.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/solr5914/solr/contrib/dataimporthandler/src/java/org/apache/solr/handler/dataimport/URLDataSource.java?rev=1584603&r1=1584602&r2=1584603&view=diff
==============================================================================
--- lucene/dev/branches/solr5914/solr/contrib/dataimporthandler/src/java/org/apache/solr/handler/dataimport/URLDataSource.java (original)
+++ lucene/dev/branches/solr5914/solr/contrib/dataimporthandler/src/java/org/apache/solr/handler/dataimport/URLDataSource.java Fri Apr  4 10:27:05 2014
@@ -24,6 +24,7 @@ import java.io.InputStreamReader;
 import java.io.Reader;
 import java.net.URL;
 import java.net.URLConnection;
+import java.nio.charset.StandardCharsets;
 import java.util.Properties;
 import java.util.regex.Matcher;
 import java.util.regex.Pattern;
@@ -140,7 +141,7 @@ public class URLDataSource extends DataS
 
   public static final String BASE_URL = "baseUrl";
 
-  public static final String UTF_8 = "UTF-8";
+  public static final String UTF_8 = StandardCharsets.UTF_8.name();
 
   public static final String CONNECTION_TIMEOUT_FIELD_NAME = "connectionTimeout";
 

Modified: lucene/dev/branches/solr5914/solr/contrib/dataimporthandler/src/java/org/apache/solr/handler/dataimport/UrlEvaluator.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/solr5914/solr/contrib/dataimporthandler/src/java/org/apache/solr/handler/dataimport/UrlEvaluator.java?rev=1584603&r1=1584602&r2=1584603&view=diff
==============================================================================
--- lucene/dev/branches/solr5914/solr/contrib/dataimporthandler/src/java/org/apache/solr/handler/dataimport/UrlEvaluator.java (original)
+++ lucene/dev/branches/solr5914/solr/contrib/dataimporthandler/src/java/org/apache/solr/handler/dataimport/UrlEvaluator.java Fri Apr  4 10:27:05 2014
@@ -4,6 +4,7 @@ import static org.apache.solr.handler.da
 import static org.apache.solr.handler.dataimport.DataImportHandlerException.wrapAndThrow;
 
 import java.net.URLEncoder;
+import java.nio.charset.StandardCharsets;
 import java.util.List;
 
 /*

Modified: lucene/dev/branches/solr5914/solr/contrib/dataimporthandler/src/java/org/apache/solr/handler/dataimport/ZKPropertiesWriter.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/solr5914/solr/contrib/dataimporthandler/src/java/org/apache/solr/handler/dataimport/ZKPropertiesWriter.java?rev=1584603&r1=1584602&r2=1584603&view=diff
==============================================================================
--- lucene/dev/branches/solr5914/solr/contrib/dataimporthandler/src/java/org/apache/solr/handler/dataimport/ZKPropertiesWriter.java (original)
+++ lucene/dev/branches/solr5914/solr/contrib/dataimporthandler/src/java/org/apache/solr/handler/dataimport/ZKPropertiesWriter.java Fri Apr  4 10:27:05 2014
@@ -18,10 +18,10 @@ package org.apache.solr.handler.dataimpo
 
 import java.io.StringReader;
 import java.io.StringWriter;
+import java.nio.charset.StandardCharsets;
 import java.util.Map;
 import java.util.Properties;
 
-import org.apache.lucene.util.IOUtils;
 import org.apache.solr.common.cloud.SolrZkClient;
 import org.apache.zookeeper.KeeperException.NodeExistsException;
 import org.slf4j.Logger;
@@ -67,7 +67,7 @@ public class ZKPropertiesWriter extends 
     StringWriter output = new StringWriter();
     try {
       existing.store(output, null);
-      byte[] bytes = output.toString().getBytes(IOUtils.CHARSET_UTF_8);
+      byte[] bytes = output.toString().getBytes(StandardCharsets.UTF_8);
       if (!zkClient.exists(path, false)) {
         try {
           zkClient.makePath(path, false);
@@ -90,7 +90,7 @@ public class ZKPropertiesWriter extends 
     try {
       byte[] data = zkClient.getData(path, null, null, false);
       if (data != null) {
-        props.load(new StringReader(new String(data, "UTF-8")));
+        props.load(new StringReader(new String(data, StandardCharsets.UTF_8)));
       }
     } catch (Exception e) {
       log.warn(

Modified: lucene/dev/branches/solr5914/solr/contrib/dataimporthandler/src/test/org/apache/solr/handler/dataimport/TestBuiltInEvaluators.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/solr5914/solr/contrib/dataimporthandler/src/test/org/apache/solr/handler/dataimport/TestBuiltInEvaluators.java?rev=1584603&r1=1584602&r2=1584603&view=diff
==============================================================================
--- lucene/dev/branches/solr5914/solr/contrib/dataimporthandler/src/test/org/apache/solr/handler/dataimport/TestBuiltInEvaluators.java (original)
+++ lucene/dev/branches/solr5914/solr/contrib/dataimporthandler/src/test/org/apache/solr/handler/dataimport/TestBuiltInEvaluators.java Fri Apr  4 10:27:05 2014
@@ -20,6 +20,7 @@ import org.junit.Before;
 import org.junit.Test;
 
 import java.net.URLEncoder;
+import java.nio.charset.StandardCharsets;
 import java.text.SimpleDateFormat;
 import java.util.*;
 
@@ -30,7 +31,7 @@ import java.util.*;
  * @since solr 1.3
  */
 public class TestBuiltInEvaluators extends AbstractDataImportHandlerTestCase {
-  private static final String ENCODING = "UTF-8";
+  private static final String ENCODING = StandardCharsets.UTF_8.name();
 
   VariableResolver resolver;
 

Modified: lucene/dev/branches/solr5914/solr/contrib/dataimporthandler/src/test/org/apache/solr/handler/dataimport/TestDocBuilder2.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/solr5914/solr/contrib/dataimporthandler/src/test/org/apache/solr/handler/dataimport/TestDocBuilder2.java?rev=1584603&r1=1584602&r2=1584603&view=diff
==============================================================================
--- lucene/dev/branches/solr5914/solr/contrib/dataimporthandler/src/test/org/apache/solr/handler/dataimport/TestDocBuilder2.java (original)
+++ lucene/dev/branches/solr5914/solr/contrib/dataimporthandler/src/test/org/apache/solr/handler/dataimport/TestDocBuilder2.java Fri Apr  4 10:27:05 2014
@@ -26,6 +26,8 @@ import org.junit.BeforeClass;
 import org.junit.Ignore;
 import org.junit.Test;
 
+import java.nio.charset.StandardCharsets;
+
 /**
  * <p>
  * Test for DocBuilder using the test harness
@@ -243,14 +245,14 @@ public class TestDocBuilder2 extends Abs
 
     Map<String, String> params = createMap("baseDir", tmpdir.getAbsolutePath());
 
-    createFile(tmpdir, "a.xml", "a.xml".getBytes("UTF-8"), true);
-    createFile(tmpdir, "b.xml", "b.xml".getBytes("UTF-8"), true);
-    createFile(tmpdir, "c.props", "c.props".getBytes("UTF-8"), true);
+    createFile(tmpdir, "a.xml", "a.xml".getBytes(StandardCharsets.UTF_8), true);
+    createFile(tmpdir, "b.xml", "b.xml".getBytes(StandardCharsets.UTF_8), true);
+    createFile(tmpdir, "c.props", "c.props".getBytes(StandardCharsets.UTF_8), true);
     runFullImport(dataConfigFileList, params);
     assertQ(req("*:*"), "//*[@numFound='3']");
 
     // Add a new file after a full index is done
-    createFile(tmpdir, "t.xml", "t.xml".getBytes("UTF-8"), false);
+    createFile(tmpdir, "t.xml", "t.xml".getBytes(StandardCharsets.UTF_8), false);
     runFullImport(dataConfigFileList, params);
     // we should find only 1 because by default clean=true is passed
     // and this particular import should find only one file t.xml

Modified: lucene/dev/branches/solr5914/solr/contrib/dataimporthandler/src/test/org/apache/solr/handler/dataimport/TestFileListEntityProcessor.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/solr5914/solr/contrib/dataimporthandler/src/test/org/apache/solr/handler/dataimport/TestFileListEntityProcessor.java?rev=1584603&r1=1584602&r2=1584603&view=diff
==============================================================================
--- lucene/dev/branches/solr5914/solr/contrib/dataimporthandler/src/test/org/apache/solr/handler/dataimport/TestFileListEntityProcessor.java (original)
+++ lucene/dev/branches/solr5914/solr/contrib/dataimporthandler/src/test/org/apache/solr/handler/dataimport/TestFileListEntityProcessor.java Fri Apr  4 10:27:05 2014
@@ -23,6 +23,7 @@ import org.junit.Test;
 
 import java.io.File;
 import java.io.IOException;
+import java.nio.charset.StandardCharsets;
 import java.text.SimpleDateFormat;
 import java.util.*;
 
@@ -41,9 +42,9 @@ public class TestFileListEntityProcessor
   public void testSimple() throws IOException {
     File tmpdir = createTempDir();
 
-    createFile(tmpdir, "a.xml", "a.xml".getBytes("UTF-8"), false);
-    createFile(tmpdir, "b.xml", "b.xml".getBytes("UTF-8"), false);
-    createFile(tmpdir, "c.props", "c.props".getBytes("UTF-8"), false);
+    createFile(tmpdir, "a.xml", "a.xml".getBytes(StandardCharsets.UTF_8), false);
+    createFile(tmpdir, "b.xml", "b.xml".getBytes(StandardCharsets.UTF_8), false);
+    createFile(tmpdir, "c.props", "c.props".getBytes(StandardCharsets.UTF_8), false);
     Map attrs = createMap(
             FileListEntityProcessor.FILE_NAME, "xml$",
             FileListEntityProcessor.BASE_DIR, tmpdir.getAbsolutePath());
@@ -69,19 +70,19 @@ public class TestFileListEntityProcessor
 
     long minLength = Long.MAX_VALUE;
     String smallestFile = "";
-    byte[] content = "abcdefgij".getBytes("UTF-8");
+    byte[] content = "abcdefgij".getBytes(StandardCharsets.UTF_8);
     createFile(tmpdir, "a.xml", content, false);
     if (minLength > content.length) {
       minLength = content.length;
       smallestFile = "a.xml";
     }
-    content = "abcdefgij".getBytes("UTF-8");
+    content = "abcdefgij".getBytes(StandardCharsets.UTF_8);
     createFile(tmpdir, "b.xml", content, false);
     if (minLength > content.length) {
       minLength = content.length;
       smallestFile = "b.xml";
     }
-    content = "abc".getBytes("UTF-8");
+    content = "abc".getBytes(StandardCharsets.UTF_8);
     createFile(tmpdir, "c.props", content, false);
     if (minLength > content.length) {
       minLength = content.length;
@@ -135,9 +136,9 @@ public class TestFileListEntityProcessor
   public void testNTOT() throws IOException {
     File tmpdir = createTempDir();
 
-    createFile(tmpdir, "a.xml", "a.xml".getBytes("UTF-8"), true);
-    createFile(tmpdir, "b.xml", "b.xml".getBytes("UTF-8"), true);
-    createFile(tmpdir, "c.props", "c.props".getBytes("UTF-8"), true);
+    createFile(tmpdir, "a.xml", "a.xml".getBytes(StandardCharsets.UTF_8), true);
+    createFile(tmpdir, "b.xml", "b.xml".getBytes(StandardCharsets.UTF_8), true);
+    createFile(tmpdir, "c.props", "c.props".getBytes(StandardCharsets.UTF_8), true);
     Map attrs = createMap(
             FileListEntityProcessor.FILE_NAME, "xml$",
             FileListEntityProcessor.BASE_DIR, tmpdir.getAbsolutePath(),
@@ -159,7 +160,7 @@ public class TestFileListEntityProcessor
     VariableResolver resolver = new VariableResolver();
     String lastMod = new SimpleDateFormat("yyyy-MM-dd HH:mm:ss", Locale.ROOT).format(new Date(System.currentTimeMillis() - 50000));
     resolver.addNamespace("a", createMap("x", lastMod));
-    createFile(tmpdir, "t.xml", "t.xml".getBytes("UTF-8"), false);
+    createFile(tmpdir, "t.xml", "t.xml".getBytes(StandardCharsets.UTF_8), false);
     fList = getFiles(resolver, attrs);
     assertEquals(1, fList.size());
     assertEquals("File name must be t.xml", new File(tmpdir, "t.xml").getAbsolutePath(), fList.get(0));
@@ -170,9 +171,9 @@ public class TestFileListEntityProcessor
     File tmpdir = createTempDir();
     File childdir = new File(tmpdir + "/child" );
     childdir.mkdir();
-    createFile(childdir, "a.xml", "a.xml".getBytes("UTF-8"), true);
-    createFile(childdir, "b.xml", "b.xml".getBytes("UTF-8"), true);
-    createFile(childdir, "c.props", "c.props".getBytes("UTF-8"), true);
+    createFile(childdir, "a.xml", "a.xml".getBytes(StandardCharsets.UTF_8), true);
+    createFile(childdir, "b.xml", "b.xml".getBytes(StandardCharsets.UTF_8), true);
+    createFile(childdir, "c.props", "c.props".getBytes(StandardCharsets.UTF_8), true);
     Map attrs = createMap(
             FileListEntityProcessor.FILE_NAME, "^.*\\.xml$",
             FileListEntityProcessor.BASE_DIR, childdir.getAbsolutePath(),

Modified: lucene/dev/branches/solr5914/solr/contrib/dataimporthandler/src/test/org/apache/solr/handler/dataimport/TestFileListWithLineEntityProcessor.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/solr5914/solr/contrib/dataimporthandler/src/test/org/apache/solr/handler/dataimport/TestFileListWithLineEntityProcessor.java?rev=1584603&r1=1584602&r2=1584603&view=diff
==============================================================================
--- lucene/dev/branches/solr5914/solr/contrib/dataimporthandler/src/test/org/apache/solr/handler/dataimport/TestFileListWithLineEntityProcessor.java (original)
+++ lucene/dev/branches/solr5914/solr/contrib/dataimporthandler/src/test/org/apache/solr/handler/dataimport/TestFileListWithLineEntityProcessor.java Fri Apr  4 10:27:05 2014
@@ -1,6 +1,7 @@
 package org.apache.solr.handler.dataimport;
 
 import java.io.File;
+import java.nio.charset.StandardCharsets;
 
 import org.apache.lucene.util.LuceneTestCase;
 import org.apache.lucene.util.TestUtil;
@@ -32,9 +33,9 @@ public class TestFileListWithLineEntityP
   
   public void test() throws Exception {
     File tmpdir = TestUtil.createTempDir(LuceneTestCase.getTestClass().getSimpleName());
-    createFile(tmpdir, "a.txt", "a line one\na line two\na line three".getBytes("UTF-8"), false);
-    createFile(tmpdir, "b.txt", "b line one\nb line two".getBytes("UTF-8"), false);
-    createFile(tmpdir, "c.txt", "c line one\nc line two\nc line three\nc line four".getBytes("UTF-8"), false);
+    createFile(tmpdir, "a.txt", "a line one\na line two\na line three".getBytes(StandardCharsets.UTF_8), false);
+    createFile(tmpdir, "b.txt", "b line one\nb line two".getBytes(StandardCharsets.UTF_8), false);
+    createFile(tmpdir, "c.txt", "c line one\nc line two\nc line three\nc line four".getBytes(StandardCharsets.UTF_8), false);
     
     String config = generateConfig(tmpdir);
     LocalSolrQueryRequest request = lrf.makeRequest(

Modified: lucene/dev/branches/solr5914/solr/contrib/dataimporthandler/src/test/org/apache/solr/handler/dataimport/TestXPathEntityProcessor.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/solr5914/solr/contrib/dataimporthandler/src/test/org/apache/solr/handler/dataimport/TestXPathEntityProcessor.java?rev=1584603&r1=1584602&r2=1584603&view=diff
==============================================================================
--- lucene/dev/branches/solr5914/solr/contrib/dataimporthandler/src/test/org/apache/solr/handler/dataimport/TestXPathEntityProcessor.java (original)
+++ lucene/dev/branches/solr5914/solr/contrib/dataimporthandler/src/test/org/apache/solr/handler/dataimport/TestXPathEntityProcessor.java Fri Apr  4 10:27:05 2014
@@ -19,6 +19,7 @@ package org.apache.solr.handler.dataimpo
 import java.io.File;
 import java.io.Reader;
 import java.io.StringReader;
+import java.nio.charset.StandardCharsets;
 import java.util.ArrayList;
 import java.util.List;
 import java.util.Map;
@@ -44,7 +45,7 @@ public class TestXPathEntityProcessor ex
   public void withFieldsAndXpath() throws Exception {
     File tmpdir = createTempDir();
     
-    createFile(tmpdir, "x.xsl", xsl.getBytes("UTF-8"), false);
+    createFile(tmpdir, "x.xsl", xsl.getBytes(StandardCharsets.UTF_8), false);
     Map entityAttrs = createMap("name", "e", "url", "cd.xml",
             XPathEntityProcessor.FOR_EACH, "/catalog/cd");
     List fields = new ArrayList();
@@ -332,7 +333,7 @@ public class TestXPathEntityProcessor ex
   @Test
   public void withDefaultSolrAndXsl() throws Exception {
     File tmpdir = createTempDir();
-    AbstractDataImportHandlerTestCase.createFile(tmpdir, "x.xsl", xsl.getBytes("UTF-8"),
+    AbstractDataImportHandlerTestCase.createFile(tmpdir, "x.xsl", xsl.getBytes(StandardCharsets.UTF_8),
             false);
 
     Map entityAttrs = createMap("name", "e",

Modified: lucene/dev/branches/solr5914/solr/contrib/extraction/src/java/org/apache/solr/handler/extraction/RegexRulesPasswordProvider.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/solr5914/solr/contrib/extraction/src/java/org/apache/solr/handler/extraction/RegexRulesPasswordProvider.java?rev=1584603&r1=1584602&r2=1584603&view=diff
==============================================================================
--- lucene/dev/branches/solr5914/solr/contrib/extraction/src/java/org/apache/solr/handler/extraction/RegexRulesPasswordProvider.java (original)
+++ lucene/dev/branches/solr5914/solr/contrib/extraction/src/java/org/apache/solr/handler/extraction/RegexRulesPasswordProvider.java Fri Apr  4 10:27:05 2014
@@ -20,6 +20,7 @@ package org.apache.solr.handler.extracti
 import java.io.BufferedReader;
 import java.io.IOException;
 import java.io.InputStream;
+import java.nio.charset.StandardCharsets;
 import java.util.LinkedHashMap;
 import java.util.Map.Entry;
 import java.util.regex.Pattern;
@@ -73,7 +74,7 @@ public class RegexRulesPasswordProvider 
    */
   public static LinkedHashMap<Pattern,String> parseRulesFile(InputStream is) {
     LinkedHashMap<Pattern,String> rules = new LinkedHashMap<>();
-    BufferedReader br = new BufferedReader(IOUtils.getDecodingReader(is, IOUtils.CHARSET_UTF_8));
+    BufferedReader br = new BufferedReader(IOUtils.getDecodingReader(is, StandardCharsets.UTF_8));
     String line;
     try {
       int linenum = 0;

Modified: lucene/dev/branches/solr5914/solr/contrib/langid/src/java/org/apache/solr/update/processor/LangDetectLanguageIdentifierUpdateProcessorFactory.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/solr5914/solr/contrib/langid/src/java/org/apache/solr/update/processor/LangDetectLanguageIdentifierUpdateProcessorFactory.java?rev=1584603&r1=1584602&r2=1584603&view=diff
==============================================================================
--- lucene/dev/branches/solr5914/solr/contrib/langid/src/java/org/apache/solr/update/processor/LangDetectLanguageIdentifierUpdateProcessorFactory.java (original)
+++ lucene/dev/branches/solr5914/solr/contrib/langid/src/java/org/apache/solr/update/processor/LangDetectLanguageIdentifierUpdateProcessorFactory.java Fri Apr  4 10:27:05 2014
@@ -22,6 +22,7 @@ import java.io.IOException;
 import java.io.InputStream;
 import java.io.InputStreamReader;
 import java.nio.charset.Charset;
+import java.nio.charset.StandardCharsets;
 import java.util.ArrayList;
 import java.util.List;
 
@@ -126,10 +127,9 @@ public class LangDetectLanguageIdentifie
     }
     loaded = true;
     List<String> profileData = new ArrayList<>();
-    Charset encoding = Charset.forName("UTF-8");
     for (String language : languages) {
       InputStream stream = LangDetectLanguageIdentifierUpdateProcessor.class.getResourceAsStream("langdetect-profiles/" + language);
-      BufferedReader reader = new BufferedReader(new InputStreamReader(stream, encoding));
+      BufferedReader reader = new BufferedReader(new InputStreamReader(stream, StandardCharsets.UTF_8));
       profileData.add(new String(IOUtils.toCharArray(reader)));
       reader.close();
     }

Modified: lucene/dev/branches/solr5914/solr/contrib/map-reduce/src/java/org/apache/solr/hadoop/MapReduceIndexerTool.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/solr5914/solr/contrib/map-reduce/src/java/org/apache/solr/hadoop/MapReduceIndexerTool.java?rev=1584603&r1=1584602&r2=1584603&view=diff
==============================================================================
--- lucene/dev/branches/solr5914/solr/contrib/map-reduce/src/java/org/apache/solr/hadoop/MapReduceIndexerTool.java (original)
+++ lucene/dev/branches/solr5914/solr/contrib/map-reduce/src/java/org/apache/solr/hadoop/MapReduceIndexerTool.java Fri Apr  4 10:27:05 2014
@@ -31,6 +31,7 @@ import java.io.Writer;
 import java.net.URISyntaxException;
 import java.net.URL;
 import java.net.URLClassLoader;
+import java.nio.charset.StandardCharsets;
 import java.text.NumberFormat;
 import java.util.ArrayList;
 import java.util.Arrays;
@@ -82,7 +83,6 @@ import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 import org.kitesdk.morphline.base.Fields;
 
-import com.google.common.base.Charsets;
 import com.google.common.base.Preconditions;
 import com.google.common.io.ByteStreams;
 
@@ -924,7 +924,7 @@ public class MapReduceIndexerTool extend
     FileSystem fs = fullInputList.getFileSystem(conf);
     FSDataOutputStream out = fs.create(fullInputList);
     try {
-      Writer writer = new BufferedWriter(new OutputStreamWriter(out, "UTF-8"));
+      Writer writer = new BufferedWriter(new OutputStreamWriter(out, StandardCharsets.UTF_8));
       
       for (Path inputFile : inputFiles) {
         FileSystem inputFileFs = inputFile.getFileSystem(conf);
@@ -949,7 +949,7 @@ public class MapReduceIndexerTool extend
           in = inputList.getFileSystem(conf).open(inputList);
         }
         try {
-          BufferedReader reader = new BufferedReader(new InputStreamReader(in, "UTF-8"));
+          BufferedReader reader = new BufferedReader(new InputStreamReader(in, StandardCharsets.UTF_8));
           String line;
           while ((line = reader.readLine()) != null) {
             writer.write(line + "\n");
@@ -988,7 +988,7 @@ public class MapReduceIndexerTool extend
   
   private void randomizeFewInputFiles(FileSystem fs, Path outputStep2Dir, Path fullInputList) throws IOException {    
     List<String> lines = new ArrayList();
-    BufferedReader reader = new BufferedReader(new InputStreamReader(fs.open(fullInputList), "UTF-8"));
+    BufferedReader reader = new BufferedReader(new InputStreamReader(fs.open(fullInputList), StandardCharsets.UTF_8));
     try {
       String line;
       while ((line = reader.readLine()) != null) {
@@ -1001,7 +1001,7 @@ public class MapReduceIndexerTool extend
     Collections.shuffle(lines, new Random(421439783L)); // constant seed for reproducability
     
     FSDataOutputStream out = fs.create(new Path(outputStep2Dir, FULL_INPUT_LIST));
-    Writer writer = new BufferedWriter(new OutputStreamWriter(out, "UTF-8"));
+    Writer writer = new BufferedWriter(new OutputStreamWriter(out, StandardCharsets.UTF_8));
     try {
       for (String line : lines) {
         writer.write(line + "\n");
@@ -1135,7 +1135,7 @@ public class MapReduceIndexerTool extend
    * turnaround during trial & debug sessions
    */
   private void dryRun(MorphlineMapRunner runner, FileSystem fs, Path fullInputList) throws IOException {    
-    BufferedReader reader = new BufferedReader(new InputStreamReader(fs.open(fullInputList), "UTF-8"));
+    BufferedReader reader = new BufferedReader(new InputStreamReader(fs.open(fullInputList), StandardCharsets.UTF_8));
     try {
       String line;
       while ((line = reader.readLine()) != null) {
@@ -1154,7 +1154,7 @@ public class MapReduceIndexerTool extend
     int numFiles = 0;
     FSDataOutputStream out = fs.create(fullInputList);
     try {
-      Writer writer = new BufferedWriter(new OutputStreamWriter(out, "UTF-8"));
+      Writer writer = new BufferedWriter(new OutputStreamWriter(out, StandardCharsets.UTF_8));
       for (FileStatus stat : dirs) {
         LOG.debug("Adding path {}", stat.getPath());
         Path dir = new Path(stat.getPath(), "data/index");
@@ -1263,7 +1263,7 @@ public class MapReduceIndexerTool extend
       byte[] bytes = ByteStreams.toByteArray(in);
       in.close();
       Preconditions.checkArgument(bytes.length > 0);
-      int solrShard = Integer.parseInt(new String(bytes, Charsets.UTF_8));
+      int solrShard = Integer.parseInt(new String(bytes, StandardCharsets.UTF_8));
       if (!delete(solrShardNumberFile, false, fs)) {
         return false;
       }

Modified: lucene/dev/branches/solr5914/solr/contrib/map-reduce/src/java/org/apache/solr/hadoop/SolrOutputFormat.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/solr5914/solr/contrib/map-reduce/src/java/org/apache/solr/hadoop/SolrOutputFormat.java?rev=1584603&r1=1584602&r2=1584603&view=diff
==============================================================================
--- lucene/dev/branches/solr5914/solr/contrib/map-reduce/src/java/org/apache/solr/hadoop/SolrOutputFormat.java (original)
+++ lucene/dev/branches/solr5914/solr/contrib/map-reduce/src/java/org/apache/solr/hadoop/SolrOutputFormat.java Fri Apr  4 10:27:05 2014
@@ -22,6 +22,7 @@ import java.io.FileOutputStream;
 import java.io.IOException;
 import java.io.InputStream;
 import java.net.URI;
+import java.nio.charset.StandardCharsets;
 import java.util.Arrays;
 import java.util.HashSet;
 import java.util.Locale;
@@ -242,7 +243,7 @@ public class SolrOutputFormat<K, V> exte
     
     ZipEntry ze = new ZipEntry("solr.xml");
     zos.putNextEntry(ze);
-    zos.write("<cores><core name=\"collection1\" instanceDir=\".\"/></cores>".getBytes("UTF-8"));
+    zos.write("<cores><core name=\"collection1\" instanceDir=\".\"/></cores>".getBytes(StandardCharsets.UTF_8));
     zos.flush();
     zos.closeEntry();
     zos.close();

Modified: lucene/dev/branches/solr5914/solr/contrib/map-reduce/src/java/org/apache/solr/hadoop/ToolRunnerHelpFormatter.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/solr5914/solr/contrib/map-reduce/src/java/org/apache/solr/hadoop/ToolRunnerHelpFormatter.java?rev=1584603&r1=1584602&r2=1584603&view=diff
==============================================================================
--- lucene/dev/branches/solr5914/solr/contrib/map-reduce/src/java/org/apache/solr/hadoop/ToolRunnerHelpFormatter.java (original)
+++ lucene/dev/branches/solr5914/solr/contrib/map-reduce/src/java/org/apache/solr/hadoop/ToolRunnerHelpFormatter.java Fri Apr  4 10:27:05 2014
@@ -24,6 +24,7 @@ import java.io.PrintWriter;
 import java.io.StringReader;
 import java.io.StringWriter;
 import java.io.UnsupportedEncodingException;
+import java.nio.charset.StandardCharsets;
 
 import net.sourceforge.argparse4j.ArgumentParsers;
 import net.sourceforge.argparse4j.helper.ASCIITextWidthCounter;
@@ -42,7 +43,7 @@ class ToolRunnerHelpFormatter {
     String msg;
     try {
       ToolRunner.printGenericCommandUsage(new PrintStream(bout, true, "UTF-8"));
-      msg = new String(bout.toByteArray(), "UTF-8");
+      msg = new String(bout.toByteArray(), StandardCharsets.UTF_8);
     } catch (UnsupportedEncodingException e) {
       throw new RuntimeException(e); // unreachable
     }

Modified: lucene/dev/branches/solr5914/solr/contrib/map-reduce/src/java/org/apache/solr/hadoop/TreeMergeOutputFormat.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/solr5914/solr/contrib/map-reduce/src/java/org/apache/solr/hadoop/TreeMergeOutputFormat.java?rev=1584603&r1=1584602&r2=1584603&view=diff
==============================================================================
--- lucene/dev/branches/solr5914/solr/contrib/map-reduce/src/java/org/apache/solr/hadoop/TreeMergeOutputFormat.java (original)
+++ lucene/dev/branches/solr5914/solr/contrib/map-reduce/src/java/org/apache/solr/hadoop/TreeMergeOutputFormat.java Fri Apr  4 10:27:05 2014
@@ -20,6 +20,7 @@ import java.io.IOException;
 import java.io.OutputStream;
 import java.io.OutputStreamWriter;
 import java.io.Writer;
+import java.nio.charset.StandardCharsets;
 import java.util.ArrayList;
 import java.util.List;
 import java.util.concurrent.TimeUnit;
@@ -43,7 +44,6 @@ import org.apache.solr.store.hdfs.HdfsDi
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
-import com.google.common.base.Charsets;
 import com.google.common.base.Preconditions;
 
 /**
@@ -188,7 +188,7 @@ public class TreeMergeOutputFormat exten
       LOG.debug("Merging into outputShardNum: " + outputShardNum + " from taskId: " + taskId);
       Path shardNumberFile = new Path(workDir.getParent().getParent(), TreeMergeMapper.SOLR_SHARD_NUMBER);
       OutputStream out = shardNumberFile.getFileSystem(context.getConfiguration()).create(shardNumberFile);
-      Writer writer = new OutputStreamWriter(out, Charsets.UTF_8);
+      Writer writer = new OutputStreamWriter(out, StandardCharsets.UTF_8);
       writer.write(String.valueOf(outputShardNum));
       writer.flush();
       writer.close();

Modified: lucene/dev/branches/solr5914/solr/contrib/map-reduce/src/java/org/apache/solr/hadoop/UnbufferedDataInputInputStream.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/solr5914/solr/contrib/map-reduce/src/java/org/apache/solr/hadoop/UnbufferedDataInputInputStream.java?rev=1584603&r1=1584602&r2=1584603&view=diff
==============================================================================
--- lucene/dev/branches/solr5914/solr/contrib/map-reduce/src/java/org/apache/solr/hadoop/UnbufferedDataInputInputStream.java (original)
+++ lucene/dev/branches/solr5914/solr/contrib/map-reduce/src/java/org/apache/solr/hadoop/UnbufferedDataInputInputStream.java Fri Apr  4 10:27:05 2014
@@ -22,6 +22,7 @@ import java.io.DataInput;
 import java.io.DataInputStream;
 import java.io.IOException;
 import java.io.InputStreamReader;
+import java.nio.charset.StandardCharsets;
 
 public class UnbufferedDataInputInputStream extends org.apache.solr.common.util.DataInputInputStream {
   private final DataInputStream in;
@@ -97,7 +98,7 @@ public class UnbufferedDataInputInputStr
 
   @Override
   public String readLine() throws IOException {
-    BufferedReader reader = new BufferedReader(new InputStreamReader(in, "UTF-8"));
+    BufferedReader reader = new BufferedReader(new InputStreamReader(in, StandardCharsets.UTF_8));
     return reader.readLine();
   }
 

Modified: lucene/dev/branches/solr5914/solr/contrib/map-reduce/src/test/org/apache/solr/hadoop/MapReduceIndexerToolArgumentParserTest.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/solr5914/solr/contrib/map-reduce/src/test/org/apache/solr/hadoop/MapReduceIndexerToolArgumentParserTest.java?rev=1584603&r1=1584602&r2=1584603&view=diff
==============================================================================
--- lucene/dev/branches/solr5914/solr/contrib/map-reduce/src/test/org/apache/solr/hadoop/MapReduceIndexerToolArgumentParserTest.java (original)
+++ lucene/dev/branches/solr5914/solr/contrib/map-reduce/src/test/org/apache/solr/hadoop/MapReduceIndexerToolArgumentParserTest.java Fri Apr  4 10:27:05 2014
@@ -20,6 +20,7 @@ import java.io.ByteArrayOutputStream;
 import java.io.File;
 import java.io.PrintStream;
 import java.io.UnsupportedEncodingException;
+import java.nio.charset.StandardCharsets;
 import java.util.Arrays;
 import java.util.Collections;
 
@@ -191,7 +192,7 @@ public class MapReduceIndexerToolArgumen
   public void testArgsParserHelp() throws UnsupportedEncodingException  {
     String[] args = new String[] { "--help" };
     assertEquals(new Integer(0), parser.parseArgs(args, conf, opts));
-    String helpText = new String(bout.toByteArray(), "UTF-8");
+    String helpText = new String(bout.toByteArray(), StandardCharsets.UTF_8);
     assertTrue(helpText.contains("MapReduce batch job driver that "));
     assertTrue(helpText.contains("bin/hadoop command"));
     assertEquals(0, berr.toByteArray().length);
@@ -458,9 +459,9 @@ public class MapReduceIndexerToolArgumen
   
   private void assertArgumentParserException(String[] args) throws UnsupportedEncodingException {
     assertEquals("should have returned fail code", new Integer(1), parser.parseArgs(args, conf, opts));
-    assertEquals("no sys out expected:" + new String(bout.toByteArray(), "UTF-8"), 0, bout.toByteArray().length);
+    assertEquals("no sys out expected:" + new String(bout.toByteArray(), StandardCharsets.UTF_8), 0, bout.toByteArray().length);
     String usageText;
-    usageText = new String(berr.toByteArray(), "UTF-8");
+    usageText = new String(berr.toByteArray(), StandardCharsets.UTF_8);
 
     assertTrue("should start with usage msg \"usage: hadoop \":" + usageText, usageText.startsWith("usage: hadoop "));
   }

Modified: lucene/dev/branches/solr5914/solr/contrib/map-reduce/src/test/org/apache/solr/hadoop/MorphlineBasicMiniMRTest.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/solr5914/solr/contrib/map-reduce/src/test/org/apache/solr/hadoop/MorphlineBasicMiniMRTest.java?rev=1584603&r1=1584602&r2=1584603&view=diff
==============================================================================
--- lucene/dev/branches/solr5914/solr/contrib/map-reduce/src/test/org/apache/solr/hadoop/MorphlineBasicMiniMRTest.java (original)
+++ lucene/dev/branches/solr5914/solr/contrib/map-reduce/src/test/org/apache/solr/hadoop/MorphlineBasicMiniMRTest.java Fri Apr  4 10:27:05 2014
@@ -22,6 +22,7 @@ import java.io.OutputStream;
 import java.io.OutputStreamWriter;
 import java.io.Writer;
 import java.lang.reflect.Array;
+import java.nio.charset.StandardCharsets;
 import java.util.Arrays;
 
 import org.apache.commons.io.FileUtils;
@@ -308,7 +309,7 @@ public class MorphlineBasicMiniMRTest ex
     assertTrue(fs.mkdirs(inDir));
     Path INPATH = new Path(inDir, "input.txt");
     OutputStream os = fs.create(INPATH);
-    Writer wr = new OutputStreamWriter(os, "UTF-8");
+    Writer wr = new OutputStreamWriter(os, StandardCharsets.UTF_8);
     wr.write(DATADIR + "/" + inputAvroFile);
     wr.close();
 

Modified: lucene/dev/branches/solr5914/solr/contrib/map-reduce/src/test/org/apache/solr/hadoop/MorphlineGoLiveMiniMRTest.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/solr5914/solr/contrib/map-reduce/src/test/org/apache/solr/hadoop/MorphlineGoLiveMiniMRTest.java?rev=1584603&r1=1584602&r2=1584603&view=diff
==============================================================================
--- lucene/dev/branches/solr5914/solr/contrib/map-reduce/src/test/org/apache/solr/hadoop/MorphlineGoLiveMiniMRTest.java (original)
+++ lucene/dev/branches/solr5914/solr/contrib/map-reduce/src/test/org/apache/solr/hadoop/MorphlineGoLiveMiniMRTest.java Fri Apr  4 10:27:05 2014
@@ -24,6 +24,7 @@ import java.io.UnsupportedEncodingExcept
 import java.io.Writer;
 import java.lang.reflect.Array;
 import java.net.URI;
+import java.nio.charset.StandardCharsets;
 import java.util.ArrayList;
 import java.util.Arrays;
 import java.util.Collection;
@@ -679,7 +680,7 @@ public class MorphlineGoLiveMiniMRTest e
       Path dataDir, String localFile) throws IOException, UnsupportedEncodingException {
     Path INPATH = new Path(inDir, "input.txt");
     OutputStream os = fs.create(INPATH);
-    Writer wr = new OutputStreamWriter(os, "UTF-8");
+    Writer wr = new OutputStreamWriter(os, StandardCharsets.UTF_8);
     wr.write(DATADIR + File.separator + localFile);
     wr.close();
     

Modified: lucene/dev/branches/solr5914/solr/contrib/velocity/src/java/org/apache/solr/response/SolrParamResourceLoader.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/solr5914/solr/contrib/velocity/src/java/org/apache/solr/response/SolrParamResourceLoader.java?rev=1584603&r1=1584602&r2=1584603&view=diff
==============================================================================
--- lucene/dev/branches/solr5914/solr/contrib/velocity/src/java/org/apache/solr/response/SolrParamResourceLoader.java (original)
+++ lucene/dev/branches/solr5914/solr/contrib/velocity/src/java/org/apache/solr/response/SolrParamResourceLoader.java Fri Apr  4 10:27:05 2014
@@ -25,7 +25,7 @@ import org.apache.commons.collections.Ex
 
 import java.io.ByteArrayInputStream;
 import java.io.InputStream;
-import java.io.UnsupportedEncodingException;
+import java.nio.charset.StandardCharsets;
 import java.util.HashMap;
 import java.util.Iterator;
 import java.util.Map;
@@ -57,11 +57,7 @@ public class SolrParamResourceLoader ext
   @Override
   public InputStream getResourceStream(String s) throws ResourceNotFoundException {
     String template = templates.get(s);
-    try {
-      return template == null ? null : new ByteArrayInputStream(template.getBytes("UTF-8"));
-    } catch (UnsupportedEncodingException e) {
-      throw new RuntimeException(e); // may not happen
-    }
+    return template == null ? null : new ByteArrayInputStream(template.getBytes(StandardCharsets.UTF_8));
   }
 
   @Override

Modified: lucene/dev/branches/solr5914/solr/contrib/velocity/src/java/org/apache/solr/response/VelocityResponseWriter.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/solr5914/solr/contrib/velocity/src/java/org/apache/solr/response/VelocityResponseWriter.java?rev=1584603&r1=1584602&r2=1584603&view=diff
==============================================================================
--- lucene/dev/branches/solr5914/solr/contrib/velocity/src/java/org/apache/solr/response/VelocityResponseWriter.java (original)
+++ lucene/dev/branches/solr5914/solr/contrib/velocity/src/java/org/apache/solr/response/VelocityResponseWriter.java Fri Apr  4 10:27:05 2014
@@ -17,21 +17,29 @@
 
 package org.apache.solr.response;
 
-import org.apache.lucene.util.IOUtils;
+import java.io.IOException;
+import java.io.InputStream;
+import java.io.InputStreamReader;
+import java.io.StringWriter;
+import java.io.Writer;
+import java.nio.charset.StandardCharsets;
+import java.util.Properties;
+
 import org.apache.solr.client.solrj.SolrResponse;
 import org.apache.solr.client.solrj.response.QueryResponse;
 import org.apache.solr.client.solrj.response.SolrResponseBase;
-import org.apache.solr.common.SolrException;
 import org.apache.solr.common.util.NamedList;
 import org.apache.solr.request.SolrQueryRequest;
 import org.apache.velocity.Template;
 import org.apache.velocity.VelocityContext;
 import org.apache.velocity.app.VelocityEngine;
 import org.apache.velocity.runtime.RuntimeConstants;
-import org.apache.velocity.tools.generic.*;
-
-import java.io.*;
-import java.util.Properties;
+import org.apache.velocity.tools.generic.ComparisonDateTool;
+import org.apache.velocity.tools.generic.EscapeTool;
+import org.apache.velocity.tools.generic.ListTool;
+import org.apache.velocity.tools.generic.MathTool;
+import org.apache.velocity.tools.generic.NumberTool;
+import org.apache.velocity.tools.generic.SortTool;
 
 public class VelocityResponseWriter implements QueryResponseWriter {
 
@@ -132,7 +140,7 @@ public class VelocityResponseWriter impl
         try {
           is = resourceLoader.getResourceStream(propFile);
           Properties props = new Properties();
-          props.load(new InputStreamReader(is, IOUtils.CHARSET_UTF_8));
+          props.load(new InputStreamReader(is, StandardCharsets.UTF_8));
           engine.init(props);
         }
         finally {

Modified: lucene/dev/branches/solr5914/solr/core/build.xml
URL: http://svn.apache.org/viewvc/lucene/dev/branches/solr5914/solr/core/build.xml?rev=1584603&r1=1584602&r2=1584603&view=diff
==============================================================================
--- lucene/dev/branches/solr5914/solr/core/build.xml (original)
+++ lucene/dev/branches/solr5914/solr/core/build.xml Fri Apr  4 10:27:05 2014
@@ -32,9 +32,14 @@
 
   <target name="compile-core" depends="compile-solrj,common-solr.compile-core"/>
 
+  <target name="compile-test" depends="jar-analyzers-icu,-compile-analysis-extras,common-solr.compile-test"/>
+
   <path id="test.classpath">
     <path refid="solr.test.base.classpath"/>
     <fileset dir="${test.lib.dir}" includes="*.jar"/>
+    <pathelement location="${analyzers-icu.jar}"/>
+    <pathelement location="${common-solr.dir}/build/contrib/solr-analysis-extras/classes/java"/>
+    <fileset dir="${common-solr.dir}/contrib/analysis-extras/lib" includes="icu4j*.jar"/>
   </path>
 
   <!-- specialized to ONLY depend on solrj -->

Modified: lucene/dev/branches/solr5914/solr/core/src/java/org/apache/solr/analytics/accumulator/BasicAccumulator.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/solr5914/solr/core/src/java/org/apache/solr/analytics/accumulator/BasicAccumulator.java?rev=1584603&r1=1584602&r2=1584603&view=diff
==============================================================================
--- lucene/dev/branches/solr5914/solr/core/src/java/org/apache/solr/analytics/accumulator/BasicAccumulator.java (original)
+++ lucene/dev/branches/solr5914/solr/core/src/java/org/apache/solr/analytics/accumulator/BasicAccumulator.java Fri Apr  4 10:27:05 2014
@@ -18,8 +18,10 @@
 package org.apache.solr.analytics.accumulator;
 
 import java.io.IOException;
+import java.util.ArrayList;
 import java.util.Collections;
 import java.util.Date;
+import java.util.List;
 import java.util.Set;
 
 import org.apache.lucene.index.AtomicReaderContext;
@@ -35,6 +37,8 @@ import org.apache.solr.common.util.Named
 import org.apache.solr.schema.TrieDateField;
 import org.apache.solr.search.DocSet;
 import org.apache.solr.search.SolrIndexSearcher;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
 
 import com.google.common.base.Supplier;
 
@@ -42,6 +46,7 @@ import com.google.common.base.Supplier;
  * A <code>BasicAccumulator</code> manages the ValueCounters and Expressions without regard to Facets.
  */
 public class BasicAccumulator extends ValueAccumulator {
+  private static final Logger log = LoggerFactory.getLogger(BasicAccumulator.class);
   protected final SolrIndexSearcher searcher;
   protected final AnalyticsRequest request;
   protected final DocSet docs;
@@ -57,14 +62,16 @@ public class BasicAccumulator extends Va
     this.searcher = searcher;
     this.docs = docs;
     this.request = request;
-    statsCollectorArraySupplier = StatsCollectorSupplierFactory.create(searcher.getSchema(), request);
+    final List<ExpressionRequest> exRequests = new ArrayList<ExpressionRequest>(request.getExpressions()); // make a copy here
+    Collections.sort(exRequests);
+    log.info("Processing request '"+request.getName()+"'");
+    statsCollectorArraySupplier = StatsCollectorSupplierFactory.create(searcher.getSchema(), exRequests);
     statsCollectors = statsCollectorArraySupplier.get();
-    int size = request.getExpressions().size();
+    int size = exRequests.size();
     expressionNames = new String[size];
     expressionStrings = new String[size];
     int count = 0;
-    Collections.sort(request.getExpressions());
-    for (ExpressionRequest expRequest : request.getExpressions()) {
+    for (ExpressionRequest expRequest : exRequests) {
       expressionNames[count] = expRequest.getName();
       expressionStrings[count++] = expRequest.getExpressionString();
     }

Modified: lucene/dev/branches/solr5914/solr/core/src/java/org/apache/solr/analytics/accumulator/FacetingAccumulator.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/solr5914/solr/core/src/java/org/apache/solr/analytics/accumulator/FacetingAccumulator.java?rev=1584603&r1=1584602&r2=1584603&view=diff
==============================================================================
--- lucene/dev/branches/solr5914/solr/core/src/java/org/apache/solr/analytics/accumulator/FacetingAccumulator.java (original)
+++ lucene/dev/branches/solr5914/solr/core/src/java/org/apache/solr/analytics/accumulator/FacetingAccumulator.java Fri Apr  4 10:27:05 2014
@@ -29,6 +29,7 @@ import java.util.List;
 import java.util.Map;
 import java.util.Map.Entry;
 import java.util.Set;
+import java.util.TreeMap;
 
 import org.apache.lucene.index.AtomicReaderContext;
 import org.apache.lucene.search.Filter;
@@ -98,7 +99,7 @@ public class FacetingAccumulator extends
     List<RangeFacetRequest> rangeFreqs = request.getRangeFacets();
     List<QueryFacetRequest> queryFreqs = request.getQueryFacets();
 
-    this.fieldFacetExpressions = new LinkedHashMap<>(fieldFreqs.size());
+    this.fieldFacetExpressions = new TreeMap<>();
     this.rangeFacetExpressions = new LinkedHashMap<>(rangeFreqs.size());
     this.queryFacetExpressions = new LinkedHashMap<>(queryFreqs.size());
     this.fieldFacetCollectors = new LinkedHashMap<>(fieldFreqs.size());
@@ -120,8 +121,8 @@ public class FacetingAccumulator extends
       final SchemaField ff = fr.getField();
       final FieldFacetAccumulator facc = FieldFacetAccumulator.create(searcher, this, ff);
       facetAccumulators.add(facc);
-      fieldFacetExpressions.put(freq.getName(), new LinkedHashMap<String,Expression[]>() );
-      fieldFacetCollectors.put(freq.getName(), new LinkedHashMap<String,StatsCollector[]>());
+      fieldFacetExpressions.put(freq.getName(), new TreeMap<String, Expression[]>() );
+      fieldFacetCollectors.put(freq.getName(), new TreeMap<String,StatsCollector[]>());
     }
     /**
      * For each range and query facet request add a bucket to the corresponding

Modified: lucene/dev/branches/solr5914/solr/core/src/java/org/apache/solr/analytics/expression/BaseExpression.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/solr5914/solr/core/src/java/org/apache/solr/analytics/expression/BaseExpression.java?rev=1584603&r1=1584602&r2=1584603&view=diff
==============================================================================
--- lucene/dev/branches/solr5914/solr/core/src/java/org/apache/solr/analytics/expression/BaseExpression.java (original)
+++ lucene/dev/branches/solr5914/solr/core/src/java/org/apache/solr/analytics/expression/BaseExpression.java Fri Apr  4 10:27:05 2014
@@ -35,7 +35,10 @@ public class BaseExpression extends Expr
   }
   
   public Comparable getValue() {
-    return statsCollector.getStat(stat);
+    if(statsCollector.getStatsList().contains(stat)) {
+      return statsCollector.getStat(stat);
+    }
+    return null;
   }
 }
 /**

Modified: lucene/dev/branches/solr5914/solr/core/src/java/org/apache/solr/analytics/request/AnalyticsStats.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/solr5914/solr/core/src/java/org/apache/solr/analytics/request/AnalyticsStats.java?rev=1584603&r1=1584602&r2=1584603&view=diff
==============================================================================
--- lucene/dev/branches/solr5914/solr/core/src/java/org/apache/solr/analytics/request/AnalyticsStats.java (original)
+++ lucene/dev/branches/solr5914/solr/core/src/java/org/apache/solr/analytics/request/AnalyticsStats.java Fri Apr  4 10:27:05 2014
@@ -33,6 +33,8 @@ import org.apache.solr.common.util.Named
 import org.apache.solr.request.SolrQueryRequest;
 import org.apache.solr.search.DocSet;
 import org.apache.solr.search.SolrIndexSearcher;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
 
 /**
  * Class which computes the set of {@link AnalyticsRequest}s.
@@ -43,6 +45,7 @@ public class AnalyticsStats {
   protected SolrIndexSearcher searcher;
   protected SolrQueryRequest req;
   protected AnalyticsStatisticsCollector statsCollector;
+  private static final Logger log = LoggerFactory.getLogger(AnalyticsStats.class);
   
   public AnalyticsStats(SolrQueryRequest req, DocSet docs, SolrParams params, AnalyticsStatisticsCollector statsCollector) {
     this.req = req;
@@ -69,6 +72,10 @@ public class AnalyticsStats {
       return res;
     }
     statsCollector.addRequests(requests.size());
+    
+    // Get filter to all docs
+    Filter filter = docs.getTopFilter();
+    
     // Computing each Analytics Request Seperately
     for( AnalyticsRequest areq : requests ){
       // The Accumulator which will control the statistics generation
@@ -84,7 +91,7 @@ public class AnalyticsStats {
           accumulator = FacetingAccumulator.create(searcher, docs, areq, req);
         }
       } catch (IOException e) {
-        System.err.println(e.getMessage());
+        log.warn("Analytics request '"+areq.getName()+"' failed", e);
         continue;
       }
 
@@ -96,7 +103,6 @@ public class AnalyticsStats {
       statsCollector.addQueries(((BasicAccumulator)accumulator).getNumQueries());
       
       // Loop through the documents returned by the query and add to accumulator
-      Filter filter = docs.getTopFilter();
       List<AtomicReaderContext> contexts = searcher.getTopReaderContext().leaves();
       for (int leafNum = 0; leafNum < contexts.size(); leafNum++) {
         AtomicReaderContext context = contexts.get(leafNum);