You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@lucene.apache.org by sa...@apache.org on 2013/05/22 17:51:12 UTC

svn commit: r1485259 [5/5] - in /lucene/dev/branches/lucene4956: ./ dev-tools/ dev-tools/idea/.idea/ dev-tools/idea/.idea/libraries/ dev-tools/idea/lucene/replicator/ dev-tools/maven/ dev-tools/maven/lucene/ dev-tools/maven/lucene/replicator/ dev-tools...

Modified: lucene/dev/branches/lucene4956/solr/core/src/java/org/apache/solr/update/processor/DistributedUpdateProcessor.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene4956/solr/core/src/java/org/apache/solr/update/processor/DistributedUpdateProcessor.java?rev=1485259&r1=1485258&r2=1485259&view=diff
==============================================================================
--- lucene/dev/branches/lucene4956/solr/core/src/java/org/apache/solr/update/processor/DistributedUpdateProcessor.java (original)
+++ lucene/dev/branches/lucene4956/solr/core/src/java/org/apache/solr/update/processor/DistributedUpdateProcessor.java Wed May 22 15:51:08 2013
@@ -322,7 +322,11 @@ public class DistributedUpdateProcessor 
     boolean localIsLeader = cloudDescriptor.isLeader();
     if (DistribPhase.FROMLEADER == phase && localIsLeader && from != null) { // from will be null on log replay
       String fromShard = req.getParams().get("distrib.from.parent");
-      if (fromShard != null)  {
+      if (fromShard != null) {
+        if (!Slice.CONSTRUCTION.equals(mySlice.getState()))  {
+          throw new SolrException(ErrorCode.SERVICE_UNAVAILABLE,
+              "Request says it is coming from parent shard leader but we are not in construction state");
+        }
         // shard splitting case -- check ranges to see if we are a sub-shard
         Slice fromSlice = zkController.getClusterState().getCollection(collection).getSlice(fromShard);
         DocRouter.Range parentRange = fromSlice.getRange();
@@ -331,12 +335,12 @@ public class DistributedUpdateProcessor 
           throw new SolrException(ErrorCode.SERVICE_UNAVAILABLE,
               "Request says it is coming from parent shard leader but parent hash range is not superset of my range");
         }
-      } else  {
-      log.error("Request says it is coming from leader, but we are the leader: " + req.getParamString());
-      throw new SolrException(ErrorCode.SERVICE_UNAVAILABLE, "Request says it is coming from leader, but we are the leader");
-    }
+      } else {
+        log.error("Request says it is coming from leader, but we are the leader: " + req.getParamString());
+        throw new SolrException(ErrorCode.SERVICE_UNAVAILABLE, "Request says it is coming from leader, but we are the leader");
+      }
     }
-    
+
     if (isLeader && !localIsLeader) {
       log.error("ClusterState says we are the leader, but locally we don't think so");
       throw new SolrException(ErrorCode.SERVICE_UNAVAILABLE, "ClusterState says we are the leader, but locally we don't think so");

Modified: lucene/dev/branches/lucene4956/solr/core/src/test-files/log4j.properties
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene4956/solr/core/src/test-files/log4j.properties?rev=1485259&r1=1485258&r2=1485259&view=diff
==============================================================================
--- lucene/dev/branches/lucene4956/solr/core/src/test-files/log4j.properties (original)
+++ lucene/dev/branches/lucene4956/solr/core/src/test-files/log4j.properties Wed May 22 15:51:08 2013
@@ -2,8 +2,8 @@
 log4j.rootLogger=INFO, CONSOLE
 
 log4j.appender.CONSOLE=org.apache.log4j.ConsoleAppender
-
-log4j.appender.CONSOLE.layout=org.apache.log4j.PatternLayout
+log4j.appender.CONSOLE.Target=System.err
+log4j.appender.CONSOLE.layout=org.apache.solr.util.SolrLogLayout
 log4j.appender.CONSOLE.layout.ConversionPattern=%-5p - %d{yyyy-MM-dd HH:mm:ss.SSS}; %C; %m\n
 
 log4j.logger.org.apache.zookeeper=WARN

Modified: lucene/dev/branches/lucene4956/solr/core/src/test/org/apache/solr/cloud/ChaosMonkeyShardSplitTest.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene4956/solr/core/src/test/org/apache/solr/cloud/ChaosMonkeyShardSplitTest.java?rev=1485259&r1=1485258&r2=1485259&view=diff
==============================================================================
--- lucene/dev/branches/lucene4956/solr/core/src/test/org/apache/solr/cloud/ChaosMonkeyShardSplitTest.java (original)
+++ lucene/dev/branches/lucene4956/solr/core/src/test/org/apache/solr/cloud/ChaosMonkeyShardSplitTest.java Wed May 22 15:51:08 2013
@@ -63,7 +63,7 @@ public class ChaosMonkeyShardSplitTest e
     waitForThingsToLevelOut(15);
 
     ClusterState clusterState = cloudClient.getZkStateReader().getClusterState();
-    DocRouter router = clusterState.getCollection(AbstractDistribZkTestBase.DEFAULT_COLLECTION).getRouter();
+    final DocRouter router = clusterState.getCollection(AbstractDistribZkTestBase.DEFAULT_COLLECTION).getRouter();
     Slice shard1 = clusterState.getSlice(AbstractDistribZkTestBase.DEFAULT_COLLECTION, SHARD1);
     DocRouter.Range shard1Range = shard1.getRange() != null ? shard1.getRange() : router.fullRange();
     final List<DocRouter.Range> ranges = router.partitionRange(2, shard1Range);
@@ -78,16 +78,17 @@ public class ChaosMonkeyShardSplitTest e
     try {
       del("*:*");
       for (int id = 0; id < 100; id++) {
-        indexAndUpdateCount(ranges, docCounts, id);
+        indexAndUpdateCount(router, ranges, docCounts, String.valueOf(id));
       }
       commit();
 
       indexThread = new Thread() {
         @Override
         public void run() {
-          for (int id = 101; id < atLeast(401); id++) {
+          int max = atLeast(401);
+          for (int id = 101; id < max; id++) {
             try {
-              indexAndUpdateCount(ranges, docCounts, id);
+              indexAndUpdateCount(router, ranges, docCounts, String.valueOf(id));
               Thread.sleep(atLeast(25));
             } catch (Exception e) {
               log.error("Exception while adding doc", e);

Modified: lucene/dev/branches/lucene4956/solr/core/src/test/org/apache/solr/cloud/ShardSplitTest.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene4956/solr/core/src/test/org/apache/solr/cloud/ShardSplitTest.java?rev=1485259&r1=1485258&r2=1485259&view=diff
==============================================================================
--- lucene/dev/branches/lucene4956/solr/core/src/test/org/apache/solr/cloud/ShardSplitTest.java (original)
+++ lucene/dev/branches/lucene4956/solr/core/src/test/org/apache/solr/cloud/ShardSplitTest.java Wed May 22 15:51:08 2013
@@ -28,13 +28,15 @@ import org.apache.solr.client.solrj.requ
 import org.apache.solr.client.solrj.response.QueryResponse;
 import org.apache.solr.common.SolrDocument;
 import org.apache.solr.common.cloud.ClusterState;
+import org.apache.solr.common.cloud.CompositeIdRouter;
 import org.apache.solr.common.cloud.DocRouter;
+import org.apache.solr.common.cloud.HashBasedRouter;
+import org.apache.solr.common.cloud.PlainIdRouter;
 import org.apache.solr.common.cloud.Slice;
 import org.apache.solr.common.cloud.ZkCoreNodeProps;
 import org.apache.solr.common.cloud.ZkStateReader;
 import org.apache.solr.common.params.CollectionParams;
 import org.apache.solr.common.params.ModifiableSolrParams;
-import org.apache.solr.common.util.Hash;
 import org.apache.solr.handler.admin.CollectionsHandler;
 import org.apache.solr.update.DirectUpdateHandler2;
 import org.apache.zookeeper.KeeperException;
@@ -52,6 +54,10 @@ public class ShardSplitTest extends Basi
   public static final String SHARD1_0 = SHARD1 + "_0";
   public static final String SHARD1_1 = SHARD1 + "_1";
 
+  public ShardSplitTest() {
+    schemaString = "schema15.xml";      // we need a string id
+  }
+
   @Override
   @Before
   public void setUp() throws Exception {
@@ -94,7 +100,7 @@ public class ShardSplitTest extends Basi
     waitForThingsToLevelOut(15);
 
     ClusterState clusterState = cloudClient.getZkStateReader().getClusterState();
-    DocRouter router = clusterState.getCollection(AbstractDistribZkTestBase.DEFAULT_COLLECTION).getRouter();
+    final DocRouter router = clusterState.getCollection(AbstractDistribZkTestBase.DEFAULT_COLLECTION).getRouter();
     Slice shard1 = clusterState.getSlice(AbstractDistribZkTestBase.DEFAULT_COLLECTION, SHARD1);
     DocRouter.Range shard1Range = shard1.getRange() != null ? shard1.getRange() : router.fullRange();
     final List<DocRouter.Range> ranges = router.partitionRange(2, shard1Range);
@@ -102,17 +108,19 @@ public class ShardSplitTest extends Basi
     int numReplicas = shard1.getReplicas().size();
 
     del("*:*");
-    for (int id = 0; id < 100; id++) {
-      indexAndUpdateCount(ranges, docCounts, id);
+    for (int id = 0; id <= 100; id++) {
+      String shardKey = "" + (char)('a' + (id % 26)); // See comment in ShardRoutingTest for hash distribution
+      indexAndUpdateCount(router, ranges, docCounts, shardKey + "!" + String.valueOf(id));
     }
     commit();
 
     Thread indexThread = new Thread() {
       @Override
       public void run() {
-        for (int id = 101; id < atLeast(401); id++) {
+        int max = atLeast(401);
+        for (int id = 101; id < max; id++) {
           try {
-            indexAndUpdateCount(ranges, docCounts, id);
+            indexAndUpdateCount(router, ranges, docCounts, String.valueOf(id));
             Thread.sleep(atLeast(25));
           } catch (Exception e) {
             log.error("Exception while adding doc", e);
@@ -200,12 +208,14 @@ public class ShardSplitTest extends Basi
     baseServer.request(request);
   }
 
-  protected void indexAndUpdateCount(List<DocRouter.Range> ranges, int[] docCounts, int id) throws Exception {
-    indexr("id", id);
+  protected void indexAndUpdateCount(DocRouter router, List<DocRouter.Range> ranges, int[] docCounts, String id) throws Exception {
+    index("id", id);
 
-    // todo - hook in custom hashing
-    byte[] bytes = String.valueOf(id).getBytes("UTF-8");
-    int hash = Hash.murmurhash3_x86_32(bytes, 0, bytes.length, 0);
+    int hash = 0;
+    if (router instanceof HashBasedRouter) {
+      HashBasedRouter hashBasedRouter = (HashBasedRouter) router;
+      hash = hashBasedRouter.sliceHash(id, null, null);
+    }
     for (int i = 0; i < ranges.size(); i++) {
       DocRouter.Range range = ranges.get(i);
       if (range.includes(hash))

Modified: lucene/dev/branches/lucene4956/solr/core/src/test/org/apache/solr/cloud/ZkCLITest.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene4956/solr/core/src/test/org/apache/solr/cloud/ZkCLITest.java?rev=1485259&r1=1485258&r2=1485259&view=diff
==============================================================================
--- lucene/dev/branches/lucene4956/solr/core/src/test/org/apache/solr/cloud/ZkCLITest.java (original)
+++ lucene/dev/branches/lucene4956/solr/core/src/test/org/apache/solr/cloud/ZkCLITest.java Wed May 22 15:51:08 2013
@@ -18,9 +18,12 @@ package org.apache.solr.cloud;
  */
 
 import java.io.File;
+import java.util.Collection;
 import java.util.List;
 
 import org.apache.commons.io.FileUtils;
+import org.apache.commons.io.filefilter.RegexFileFilter;
+import org.apache.commons.io.filefilter.TrueFileFilter;
 import org.apache.solr.SolrTestCaseJ4;
 import org.apache.solr.common.cloud.SolrZkClient;
 import org.apache.solr.common.cloud.ZkNodeProps;
@@ -186,6 +189,19 @@ public class ZkCLITest extends SolrTestC
     List<String> zkFiles = zkClient.getChildren(ZkController.CONFIGS_ZKNODE + "/" + confsetname, null, true);
     assertEquals(files.length, zkFiles.size());
     
+    File sourceConfDir = new File(ExternalPaths.EXAMPLE_HOME + File.separator + "collection1"
+            + File.separator + "conf");
+    // filter out all directories starting with . (e.g. .svn)
+    Collection<File> sourceFiles = FileUtils.listFiles(sourceConfDir, TrueFileFilter.INSTANCE, new RegexFileFilter("[^\\.].*"));
+    for (File sourceFile :sourceFiles){
+        int indexOfRelativePath = sourceFile.getAbsolutePath().lastIndexOf("collection1" + File.separator + "conf");
+        String relativePathofFile = sourceFile.getAbsolutePath().substring(indexOfRelativePath + 17, sourceFile.getAbsolutePath().length());
+        File downloadedFile = new File(confDir,relativePathofFile);
+        assertTrue(downloadedFile.getAbsolutePath() + " does not exist source:" + sourceFile.getAbsolutePath(), downloadedFile.exists());
+        assertTrue("Content didn't change",FileUtils.contentEquals(sourceFile,downloadedFile));
+    }
+    
+   
     // test reset zk
     args = new String[] {"-zkhost", zkServer.getZkAddress(), "-cmd",
         "clear", "/"};

Modified: lucene/dev/branches/lucene4956/solr/core/src/test/org/apache/solr/core/ResourceLoaderTest.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene4956/solr/core/src/test/org/apache/solr/core/ResourceLoaderTest.java?rev=1485259&r1=1485258&r2=1485259&view=diff
==============================================================================
--- lucene/dev/branches/lucene4956/solr/core/src/test/org/apache/solr/core/ResourceLoaderTest.java (original)
+++ lucene/dev/branches/lucene4956/solr/core/src/test/org/apache/solr/core/ResourceLoaderTest.java Wed May 22 15:51:08 2013
@@ -22,6 +22,7 @@ import junit.framework.Assert;
 import org.apache.lucene.util.LuceneTestCase;
 import org.apache.lucene.analysis.core.KeywordTokenizerFactory;
 import org.apache.lucene.analysis.ngram.NGramFilterFactory;
+import org.apache.lucene.util._TestUtil;
 import org.apache.solr.common.SolrException;
 import org.apache.solr.handler.admin.LukeRequestHandler;
 import org.apache.solr.handler.component.FacetComponent;
@@ -30,11 +31,15 @@ import org.apache.lucene.analysis.util.R
 import org.apache.solr.util.plugin.SolrCoreAware;
 
 import java.io.File;
+import java.io.FileFilter;
+import java.io.FileOutputStream;
 import java.io.InputStream;
 import java.nio.charset.CharacterCodingException;
 import java.util.Arrays;
 import java.util.HashMap;
 import java.util.List;
+import java.util.jar.JarEntry;
+import java.util.jar.JarOutputStream;
 
 public class ResourceLoaderTest extends LuceneTestCase 
 {
@@ -131,4 +136,51 @@ public class ResourceLoaderTest extends 
       assertTrue(expected.getCause() instanceof CharacterCodingException);
     }
   }
+
+  public void testClassLoaderLibs() throws Exception {
+    File tmpRoot = _TestUtil.getTempDir("testClassLoaderLibs");
+
+    File lib = new File(tmpRoot, "lib");
+    lib.mkdirs();
+
+    JarOutputStream jar1 = new JarOutputStream(new FileOutputStream(new File(lib, "jar1.jar")));
+    jar1.putNextEntry(new JarEntry("aLibFile"));
+    jar1.closeEntry();
+    jar1.close();
+
+    File otherLib = new File(tmpRoot, "otherLib");
+    otherLib.mkdirs();
+
+    JarOutputStream jar2 = new JarOutputStream(new FileOutputStream(new File(otherLib, "jar2.jar")));
+    jar2.putNextEntry(new JarEntry("explicitFile"));
+    jar2.closeEntry();
+    jar2.close();
+    JarOutputStream jar3 = new JarOutputStream(new FileOutputStream(new File(otherLib, "jar3.jar")));
+    jar3.putNextEntry(new JarEntry("otherFile"));
+    jar3.closeEntry();
+    jar3.close();
+
+    SolrResourceLoader loader = new SolrResourceLoader(tmpRoot.getAbsolutePath());
+
+    // ./lib is accessible by default
+    assertNotNull(loader.getClassLoader().getResource("aLibFile"));
+
+    // file filter works (and doesn't add other files in the same dir)
+    final File explicitFileJar = new File(otherLib, "jar2.jar").getAbsoluteFile();
+    loader.addToClassLoader("otherLib",
+        new FileFilter() {
+          @Override
+          public boolean accept(File pathname) {
+            return pathname.equals(explicitFileJar);
+          }
+        }, false);
+    assertNotNull(loader.getClassLoader().getResource("explicitFile"));
+    assertNull(loader.getClassLoader().getResource("otherFile"));
+
+
+    // null file filter means accept all (making otherFile accessible)
+    loader.addToClassLoader("otherLib", null, false);
+    assertNotNull(loader.getClassLoader().getResource("otherFile"));
+    loader.close();
+  }
 }

Modified: lucene/dev/branches/lucene4956/solr/core/src/test/org/apache/solr/core/TestCoreContainer.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene4956/solr/core/src/test/org/apache/solr/core/TestCoreContainer.java?rev=1485259&r1=1485258&r2=1485259&view=diff
==============================================================================
--- lucene/dev/branches/lucene4956/solr/core/src/test/org/apache/solr/core/TestCoreContainer.java (original)
+++ lucene/dev/branches/lucene4956/solr/core/src/test/org/apache/solr/core/TestCoreContainer.java Wed May 22 15:51:08 2013
@@ -24,11 +24,14 @@ import java.io.IOException;
 import java.io.OutputStreamWriter;
 import java.util.ArrayList;
 import java.util.List;
+import java.util.jar.JarEntry;
+import java.util.jar.JarOutputStream;
 
 import javax.xml.parsers.ParserConfigurationException;
 
 import org.apache.commons.io.FileUtils;
 import org.apache.lucene.util.IOUtils;
+import org.apache.lucene.util._TestUtil;
 import org.apache.solr.SolrTestCaseJ4;
 import org.junit.AfterClass;
 import org.junit.BeforeClass;
@@ -334,6 +337,55 @@ public class TestCoreContainer extends S
       cc.shutdown();
     }
   }
+
+  @Test
+  public void testSharedLib() throws Exception {
+    File tmpRoot = _TestUtil.getTempDir("testSharedLib");
+
+    File lib = new File(tmpRoot, "lib");
+    lib.mkdirs();
+
+    JarOutputStream jar1 = new JarOutputStream(new FileOutputStream(new File(lib, "jar1.jar")));
+    jar1.putNextEntry(new JarEntry("defaultSharedLibFile"));
+    jar1.closeEntry();
+    jar1.close();
+
+    File customLib = new File(tmpRoot, "customLib");
+    customLib.mkdirs();
+
+    JarOutputStream jar2 = new JarOutputStream(new FileOutputStream(new File(customLib, "jar2.jar")));
+    jar2.putNextEntry(new JarEntry("customSharedLibFile"));
+    jar2.closeEntry();
+    jar2.close();
+
+    FileUtils.writeStringToFile(new File(tmpRoot, "default-lib-solr.xml"), "<solr><cores/></solr>", "UTF-8");
+    FileUtils.writeStringToFile(new File(tmpRoot, "explicit-lib-solr.xml"), "<solr sharedLib=\"lib\"><cores/></solr>", "UTF-8");
+    FileUtils.writeStringToFile(new File(tmpRoot, "custom-lib-solr.xml"), "<solr sharedLib=\"customLib\"><cores/></solr>", "UTF-8");
+
+    final CoreContainer cc1 = new CoreContainer(tmpRoot.getAbsolutePath());
+    cc1.load(tmpRoot.getAbsolutePath(), new File(tmpRoot, "default-lib-solr.xml"));
+    try {
+      cc1.loader.openResource("defaultSharedLibFile").close();
+    } finally {
+      cc1.shutdown();
+    }
+
+    final CoreContainer cc2 = new CoreContainer(tmpRoot.getAbsolutePath());
+    cc2.load(tmpRoot.getAbsolutePath(), new File(tmpRoot, "explicit-lib-solr.xml"));
+    try {
+      cc2.loader.openResource("defaultSharedLibFile").close();
+    } finally {
+      cc2.shutdown();
+    }
+
+    final CoreContainer cc3 = new CoreContainer(tmpRoot.getAbsolutePath());
+    cc3.load(tmpRoot.getAbsolutePath(), new File(tmpRoot, "custom-lib-solr.xml"));
+    try {
+      cc3.loader.openResource("customSharedLibFile").close();
+    } finally {
+      cc3.shutdown();
+    }
+  }
   
   private static final String EMPTY_SOLR_XML ="<?xml version=\"1.0\" encoding=\"UTF-8\" ?>\n" +
       "<solr persistent=\"false\">\n" +

Modified: lucene/dev/branches/lucene4956/solr/core/src/test/org/apache/solr/core/TestCoreDiscovery.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene4956/solr/core/src/test/org/apache/solr/core/TestCoreDiscovery.java?rev=1485259&r1=1485258&r2=1485259&view=diff
==============================================================================
--- lucene/dev/branches/lucene4956/solr/core/src/test/org/apache/solr/core/TestCoreDiscovery.java (original)
+++ lucene/dev/branches/lucene4956/solr/core/src/test/org/apache/solr/core/TestCoreDiscovery.java Wed May 22 15:51:08 2013
@@ -24,6 +24,7 @@ import java.util.Properties;
 import org.apache.commons.io.FileUtils;
 import org.apache.lucene.util.IOUtils;
 import org.apache.solr.SolrTestCaseJ4;
+import org.apache.solr.common.SolrException;
 import org.junit.After;
 import org.junit.BeforeClass;
 import org.junit.Test;
@@ -88,10 +89,9 @@ public class TestCoreDiscovery extends S
 
   }
 
-  private void addCoreWithProps(Properties stockProps) throws Exception {
+  private void addCoreWithProps(String name, Properties stockProps) throws Exception {
 
-    File propFile = new File(solrHomeDirectory,
-        stockProps.getProperty(CoreDescriptor.CORE_NAME) + File.separator + SolrCoreDiscoverer.CORE_PROP_FILE);
+    File propFile = new File(new File(solrHomeDirectory, name), SolrCoreDiscoverer.CORE_PROP_FILE);
     File parent = propFile.getParentFile();
     assertTrue("Failed to mkdirs for " + parent.getAbsolutePath(), parent.mkdirs());
     addCoreWithProps(stockProps, propFile);
@@ -127,16 +127,17 @@ public class TestCoreDiscovery extends S
     setMeUp();
 
     // name, isLazy, loadOnStartup
-    addCoreWithProps(makeCorePropFile("core1", false, true, "dataDir=core1"));
-    addCoreWithProps(makeCorePropFile("core2", false, false, "dataDir=core2"));
+    addCoreWithProps("core1", makeCorePropFile("core1", false, true, "dataDir=core1"));
+    addCoreWithProps("core2", makeCorePropFile("core2", false, false, "dataDir=core2"));
 
     // I suspect what we're adding in here is a "configset" rather than a schema or solrconfig.
     //
-    addCoreWithProps(makeCorePropFile("lazy1", true, false, "dataDir=lazy1"));
+    addCoreWithProps("lazy1", makeCorePropFile("lazy1", true, false, "dataDir=lazy1"));
 
     CoreContainer cc = init();
     try {
-      assertNull("defaultCore no longer allowed in solr.xml", cc.getDefaultCoreName());
+      assertEquals(CoreContainer.DEFAULT_DEFAULT_CORE_NAME,
+                   cc.getDefaultCoreName());
 
       TestLazyCores.checkInCores(cc, "core1");
       TestLazyCores.checkNotInCores(cc, "lazy1", "core2", "collection1");
@@ -170,6 +171,30 @@ public class TestCoreDiscovery extends S
   }
 
   @Test
+  public void testDuplicateNames() throws Exception {
+    setMeUp();
+
+    // name, isLazy, loadOnStartup
+    addCoreWithProps("core1", makeCorePropFile("core1", false, true));
+    addCoreWithProps("core2", makeCorePropFile("core2", false, false, "name=core1"));
+    CoreContainer cc = null;
+    try {
+      cc = init();
+      fail("Should have thrown exception in testDuplicateNames");
+    } catch (SolrException se) {
+      assertTrue("Should have seen an exception because two cores had the same name",
+          "Core  + desc.getName() + \" defined twice".indexOf(se.getMessage()) != -1);
+      assertTrue("/core1 should have been mentioned in the message", "/core1".indexOf(se.getMessage()) != -1);
+      assertTrue("/core2 should have been mentioned in the message", "/core2".indexOf(se.getMessage()) != -1);
+    } finally {
+      if (cc != null) {
+        cc.shutdown();
+      }
+    }
+  }
+
+
+  @Test
   public void testAlternateCoreDir() throws Exception {
     File alt = new File(TEMP_DIR, "alternateCoreDir");
     if (alt.exists()) FileUtils.deleteDirectory(alt);
@@ -192,7 +217,29 @@ public class TestCoreDiscovery extends S
       if (alt.exists()) FileUtils.deleteDirectory(alt);
     }
   }
-
+  @Test
+  public void testNoCoreDir() throws Exception {
+    File noCoreDir = new File(TEMP_DIR, "noCoreDir");
+    if (noCoreDir.exists()) FileUtils.deleteDirectory(noCoreDir);
+    noCoreDir.mkdirs();
+    setMeUp(noCoreDir.getAbsolutePath());
+    addCoreWithProps(makeCorePropFile("core1", false, true),
+        new File(noCoreDir, "core1" + File.separator + SolrCoreDiscoverer.CORE_PROP_FILE));
+    addCoreWithProps(makeCorePropFile("core2", false, false),
+        new File(noCoreDir, "core2" + File.separator + SolrCoreDiscoverer.CORE_PROP_FILE));
+    CoreContainer cc = init();
+    try {
+      SolrCore core1 = cc.getCore("core1");
+      SolrCore core2 = cc.getCore("core2");
+      assertNotNull(core1);
+      assertNotNull(core2);
+      core1.close();
+      core2.close();
+    } finally {
+      cc.shutdown();
+      if (noCoreDir.exists()) FileUtils.deleteDirectory(noCoreDir);
+    }
+  }
   // For testing whether finding a solr.xml overrides looking at solr.properties
   private final static String SOLR_XML = "<solr> " +
       "<int name=\"transientCacheSize\">2</int> " +

Modified: lucene/dev/branches/lucene4956/solr/core/src/test/org/apache/solr/request/TestFaceting.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene4956/solr/core/src/test/org/apache/solr/request/TestFaceting.java?rev=1485259&r1=1485258&r2=1485259&view=diff
==============================================================================
--- lucene/dev/branches/lucene4956/solr/core/src/test/org/apache/solr/request/TestFaceting.java (original)
+++ lucene/dev/branches/lucene4956/solr/core/src/test/org/apache/solr/request/TestFaceting.java Wed May 22 15:51:08 2013
@@ -494,6 +494,122 @@ public class TestFaceting extends SolrTe
               ,"//lst[@name='bar']/int[@name='Chauvinist'][.='1']"
               ,"//lst[@name='bar']/int[@name='Obnoxious'][.='1']"
               );
+
+      assertQ("localparams in one facet variant should not affect defaults in another: facet.sort vs facet.missing",
+                  req("q", "id:[42 TO 47]"
+                          ,"rows","0"
+                          ,"facet", "true"
+                          ,"fq", "id:[42 TO 45]"
+                          ,"facet.field", "{!key=foo " +
+                              "facet.sort=index" +
+                          "}"+fname
+                          ,"facet.field", "{!key=bar " +
+                              "facet.missing=true" +
+                          "}"+fname
+                          )
+                      // foo is in index order w/o missing
+                      ,"*[count(//lst[@name='foo']/int)=4]"
+                  ,"//lst[@name='foo']/int[1][@name='Chauvinist'][.='1']"
+                  ,"//lst[@name='foo']/int[2][@name='Obnoxious'][.='1']"
+                  ,"//lst[@name='foo']/int[3][@name='Pig'][.='0']"
+                  ,"//lst[@name='foo']/int[4][@name='Tool'][.='2']"
+                  // bar is in count order by default and includes missing
+                  ,"*[count(//lst[@name='bar']/int)=5]"
+                  ,"//lst[@name='bar']/int[1][@name='Tool'][.='2']"
+                  // don't assume tie breaker for slots 3 & 4, behavior undefined?
+                  ,"//lst[@name='bar']/int[4][@name='Pig'][.='0']"
+                  ,"//lst[@name='bar']/int[5][not(@name)][.='1']"
+                  );
+
+      assertQ("localparams in one facet variant should not affect defaults in another: facet.mincount",
+                  req("q", "id:[42 TO 47]"
+                          ,"rows","0"
+                          ,"facet", "true"
+                          ,"fq", "id:[42 TO 45]"
+                          ,"facet.field", "{!key=foo " +
+                              "facet.mincount=2" +
+                          "}"+fname
+                          ,"facet.field", "{!key=bar}"+fname
+                          )
+                      // only Tool for foo
+                      ,"*[count(//lst[@name='foo']/int)=1]"
+                  ,"//lst[@name='foo']/int[1][@name='Tool'][.='2']"
+                  // all for bar
+                  ,"*[count(//lst[@name='bar']/int)=4]"
+                  ,"//lst[@name='bar']/int[1][@name='Tool'][.='2']"
+                  // don't assume tie breaker for slots 3 & 4, behavior undefined?
+                  ,"//lst[@name='bar']/int[4][@name='Pig'][.='0']"
+                  );
+
+      assertQ("localparams in one facet variant should not affect defaults in another: facet.missing",
+                  req("q", "id:[42 TO 47]"
+                          ,"rows","0"
+                          ,"facet", "true"
+                          ,"fq", "id:[42 TO 45]"
+                          ,"facet.field", "{!key=foo " +
+                              "facet.missing=true" +
+                          "}"+fname
+                          ,"facet.field", "{!key=bar}"+fname
+                          )
+                      // foo includes missing
+                      ,"*[count(//lst[@name='foo']/int)=5]"
+                  ,"//lst[@name='foo']/int[1][@name='Tool'][.='2']"
+                  // don't assume tie breaker for slots 3 & 4, behavior undefined?
+                  ,"//lst[@name='foo']/int[4][@name='Pig'][.='0']"
+                  ,"//lst[@name='foo']/int[5][not(@name)][.='1']"
+                  // bar does not
+                  ,"*[count(//lst[@name='bar']/int)=4]"
+                  ,"//lst[@name='bar']/int[1][@name='Tool'][.='2']"
+                  // don't assume tie breaker for slots 3 & 4, behavior undefined?
+                  ,"//lst[@name='bar']/int[4][@name='Pig'][.='0']"
+                  );
+
+      assertQ("checking facets when local facet.prefix param used after regular/raw field faceting",
+          req("q", "*:*"
+              ,"facet", "true"
+              ,"facet.field", fname
+              ,"facet.field", "{!key=foo " +
+              "facet.prefix=T "+
+              "}"+fname
+          )
+          ,"*[count(//doc)=6]"
+          ,"*[count(//lst[@name='" + fname + "']/int)=4]"
+          ,"*[count(//lst[@name='foo']/int)=1]"
+          ,"//lst[@name='foo']/int[@name='Tool'][.='2']"
+      );
+
+      assertQ("checking facets when local facet.prefix param used before regular/raw field faceting",
+          req("q", "*:*"
+              ,"facet", "true"
+              ,"facet.field", "{!key=foo " +
+              "facet.prefix=T "+
+              "}"+fname
+              ,"facet.field", fname
+          )
+          ,"*[count(//doc)=6]"
+          ,"*[count(//lst[@name='" + fname + "']/int)=4]"
+          ,"*[count(//lst[@name='foo']/int)=1]"
+          ,"//lst[@name='foo']/int[@name='Tool'][.='2']"
+      );
+
+      final String foo_range_facet = "{!key=foo facet.range.gap=2}val_i";
+      final String val_range_facet = "val_i";
+      for (boolean toggle : new boolean[] { true, false }) {
+          assertQ("local gap param mixed w/raw range faceting: " + toggle,
+                      req("q", "*:*"
+                              ,"facet", "true"
+                              ,"rows", "0"
+                              ,"facet.range.start", "0"
+                              ,"facet.range.end", "10"
+                              ,"facet.range.gap", "1"
+                              ,"facet.range", (toggle ? foo_range_facet : val_range_facet)
+                              ,"facet.range", (toggle ? val_range_facet : foo_range_facet)
+                              )
+                          ,"*[count(//lst[@name='val_i']/lst[@name='counts']/int)=10]"
+                      ,"*[count(//lst[@name='foo']/lst[@name='counts']/int)=5]"
+                      );
+        }
+
       clearIndex();
       assertU(commit());
   }

Modified: lucene/dev/branches/lucene4956/solr/core/src/test/org/apache/solr/search/QueryEqualityTest.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene4956/solr/core/src/test/org/apache/solr/search/QueryEqualityTest.java?rev=1485259&r1=1485258&r2=1485259&view=diff
==============================================================================
--- lucene/dev/branches/lucene4956/solr/core/src/test/org/apache/solr/search/QueryEqualityTest.java (original)
+++ lucene/dev/branches/lucene4956/solr/core/src/test/org/apache/solr/search/QueryEqualityTest.java Wed May 22 15:51:08 2013
@@ -32,8 +32,8 @@ import org.junit.BeforeClass;
 
 /**
  * Sanity checks that queries (generated by the QParser and ValueSourceParser 
- * framework) are appropraitely {@link Object#equals} and 
- * {@link Object#hashCode()} equivilent.  If you are adding a new default 
+ * framework) are appropriately {@link Object#equals} and 
+ * {@link Object#hashCode()} equivalent.  If you are adding a new default 
  * QParser or ValueSourceParser, you will most likely get a failure from
  * {@link #testParserCoverage} until you add a new test method to this class.
  *
@@ -76,7 +76,7 @@ public class QueryEqualityTest extends S
 
 
   public void testDateMathParsingEquality() throws Exception {
-    // regardless of parser, these should all be equivilent queries
+    // regardless of parser, these should all be equivalent queries
     assertQueryEquals
       (null
        ,"{!lucene}f_tdt:2013-09-11T00\\:00\\:00Z"
@@ -704,6 +704,18 @@ public class QueryEqualityTest extends S
     assertFuncEquals("sleep(1,5)", "sleep(1,5)");
     assertFuncEquals("threadid()", "threadid()");
   }
+  
+  // TODO: more tests
+  public void testQueryMaxScore() throws Exception {
+    assertQueryEquals("maxscore", "{!maxscore}A OR B OR C",
+                      "A OR B OR C");
+    assertQueryEquals("maxscore", "{!maxscore}A AND B",
+                      "A AND B");
+    assertQueryEquals("maxscore", "{!maxscore}apache -solr",
+        "apache  -solr", "apache -solr ");
+    assertQueryEquals("maxscore", "+apache +solr", "apache AND solr",
+        "+apache +solr");
+  }
 
   /**
    * this test does not assert anything itself, it simply toggles a static 

Modified: lucene/dev/branches/lucene4956/solr/core/src/test/org/apache/solr/spelling/SpellCheckCollatorTest.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene4956/solr/core/src/test/org/apache/solr/spelling/SpellCheckCollatorTest.java?rev=1485259&r1=1485258&r2=1485259&view=diff
==============================================================================
--- lucene/dev/branches/lucene4956/solr/core/src/test/org/apache/solr/spelling/SpellCheckCollatorTest.java (original)
+++ lucene/dev/branches/lucene4956/solr/core/src/test/org/apache/solr/spelling/SpellCheckCollatorTest.java Wed May 22 15:51:08 2013
@@ -42,21 +42,24 @@ import org.junit.Test;
 public class SpellCheckCollatorTest extends SolrTestCaseJ4 {
   @BeforeClass
   public static void beforeClass() throws Exception {
-     initCore("solrconfig-spellcheckcomponent.xml", "schema.xml");
-    assertNull(h.validateUpdate(adoc("id", "0", "lowerfilt", "faith hope and love")));
+    initCore("solrconfig-spellcheckcomponent.xml", "schema.xml");
+    assertNull(h.validateUpdate(adoc("id", "0", "lowerfilt", "faith hope and love", "teststop", "metanoia")));
     assertNull(h.validateUpdate(adoc("id", "1", "lowerfilt", "faith hope and loaves")));
     assertNull(h.validateUpdate(adoc("id", "2", "lowerfilt", "fat hops and loaves")));
-    assertNull(h.validateUpdate(adoc("id", "3", "lowerfilt", "faith of homer")));
+    assertNull(h.validateUpdate(adoc("id", "3", "lowerfilt", "faith of homer", "teststop", "metanoia")));
     assertNull(h.validateUpdate(adoc("id", "4", "lowerfilt", "fat of homer")));
     assertNull(h.validateUpdate(adoc("id", "5", "lowerfilt1", "peace")));
     assertNull(h.validateUpdate(adoc("id", "6", "lowerfilt", "hyphenated word")));
-     assertNull(h.validateUpdate(adoc("id", "7", "teststop", "Jane filled out a form at Charles De Gaulle")));
-     assertNull(h.validateUpdate(adoc("id", "8", "teststop", "Dick flew from Heathrow")));
-     assertNull(h.validateUpdate(adoc("id", "9", "teststop", "Jane is stuck in customs because Spot chewed up the form")));
-     assertNull(h.validateUpdate(adoc("id", "10", "teststop", "Once in Paris Dick built a fire on the hearth")));
-     assertNull(h.validateUpdate(adoc("id", "11", "teststop", "Dick waited for Jane as he watched the sparks flow upward")));
-     assertNull(h.validateUpdate(adoc("id", "12", "teststop", "This June parisian rendez-vous is ruined because of a customs snafu")));
-     assertNull(h.validateUpdate(adoc("id", "13", "teststop", "partisan political machine")));
+    assertNull(h.validateUpdate(adoc("id", "7", "teststop", "Jane filled out a form at Charles De Gaulle")));
+    assertNull(h.validateUpdate(adoc("id", "8", "teststop", "Dick flew from Heathrow")));
+    assertNull(h.validateUpdate(adoc("id", "9", "teststop", "Jane is stuck in customs because Spot chewed up the form")));
+    assertNull(h.validateUpdate(adoc("id", "10", "teststop", "Once in Paris Dick built a fire on the hearth")));
+    assertNull(h.validateUpdate(adoc("id", "11", "teststop", "Dick waited for Jane as he watched the sparks flow upward")));
+    assertNull(h.validateUpdate(adoc("id", "12", "teststop", "This June parisian rendez-vous is ruined because of a customs snafu")));
+    assertNull(h.validateUpdate(adoc("id", "13", "teststop", "partisan political machine", "teststop", "metanoia")));
+    assertNull(h.validateUpdate(adoc("id", "14", "teststop", "metanoia")));
+    assertNull(h.validateUpdate(adoc("id", "15", "teststop", "metanoia")));
+    assertNull(h.validateUpdate(adoc("id", "16", "teststop", "metanoia")));
     assertNull(h.validateUpdate(commit()));
   }
 
@@ -430,4 +433,71 @@ public class SpellCheckCollatorTest exte
       );
     }
   }
+  @Test
+  public void testEstimatedHitCounts() throws Exception {
+   assertQ(
+        req(
+          SpellCheckComponent.COMPONENT_NAME, "true",
+          SpellCheckComponent.SPELLCHECK_DICT, "direct",
+          SpellingParams.SPELLCHECK_COUNT, "1",   
+          SpellingParams.SPELLCHECK_COLLATE, "true",
+          SpellingParams.SPELLCHECK_MAX_COLLATION_TRIES, "1",
+          SpellingParams.SPELLCHECK_MAX_COLLATIONS, "1",
+          SpellingParams.SPELLCHECK_COLLATE_EXTENDED_RESULTS, "true",          
+          "qt", "spellCheckCompRH",          
+          CommonParams.Q, "teststop:metnoia"
+        ),
+        "//lst[@name='spellcheck']/lst[@name='suggestions']/lst[@name='collation']/str[@name='collationQuery']='teststop:metanoia'",
+        "//lst[@name='spellcheck']/lst[@name='suggestions']/lst[@name='collation']/int[@name='hits']=6"        
+      );
+    assertQ(
+        req(
+          SpellCheckComponent.COMPONENT_NAME, "true",
+          SpellCheckComponent.SPELLCHECK_DICT, "direct",
+          SpellingParams.SPELLCHECK_COUNT, "1",   
+          SpellingParams.SPELLCHECK_COLLATE, "true",
+          SpellingParams.SPELLCHECK_MAX_COLLATION_TRIES, "1",
+          SpellingParams.SPELLCHECK_MAX_COLLATIONS, "1",
+          SpellingParams.SPELLCHECK_COLLATE_EXTENDED_RESULTS, "true",
+          SpellingParams.SPELLCHECK_COLLATE_MAX_COLLECT_DOCS, "1",
+          "qt", "spellCheckCompRH",          
+          CommonParams.Q, "teststop:metnoia"
+        ),
+        "//lst[@name='spellcheck']/lst[@name='suggestions']/lst[@name='collation']/str[@name='collationQuery']='teststop:metanoia'",
+        "//lst[@name='spellcheck']/lst[@name='suggestions']/lst[@name='collation']/int[@name='hits']=17"        
+      );
+    assertQ(
+        req(
+          SpellCheckComponent.COMPONENT_NAME, "true",
+          SpellCheckComponent.SPELLCHECK_DICT, "direct",
+          SpellingParams.SPELLCHECK_COUNT, "1",   
+          SpellingParams.SPELLCHECK_COLLATE, "true",
+          SpellingParams.SPELLCHECK_MAX_COLLATION_TRIES, "1",
+          SpellingParams.SPELLCHECK_MAX_COLLATIONS, "1",
+          SpellingParams.SPELLCHECK_COLLATE_EXTENDED_RESULTS, "true",
+          SpellingParams.SPELLCHECK_COLLATE_MAX_COLLECT_DOCS, "3",
+          "qt", "spellCheckCompRH",          
+          CommonParams.Q, "teststop:metnoia"
+        ),
+        "//lst[@name='spellcheck']/lst[@name='suggestions']/lst[@name='collation']/str[@name='collationQuery']='teststop:metanoia'",
+        "//lst[@name='spellcheck']/lst[@name='suggestions']/lst[@name='collation']/int[@name='hits']=4"        
+      );
+    assertQ(
+        req(
+          SpellCheckComponent.COMPONENT_NAME, "true",
+          SpellCheckComponent.SPELLCHECK_DICT, "direct",
+          SpellingParams.SPELLCHECK_COUNT, "1",   
+          SpellingParams.SPELLCHECK_COLLATE, "true",
+          SpellingParams.SPELLCHECK_MAX_COLLATION_TRIES, "1",
+          SpellingParams.SPELLCHECK_MAX_COLLATIONS, "1",
+          SpellingParams.SPELLCHECK_COLLATE_EXTENDED_RESULTS, "true",
+          SpellingParams.SPELLCHECK_COLLATE_MAX_COLLECT_DOCS, "100",
+          "qt", "spellCheckCompRH",          
+          CommonParams.Q, "teststop:metnoia"
+        ),
+        "//lst[@name='spellcheck']/lst[@name='suggestions']/lst[@name='collation']/str[@name='collationQuery']='teststop:metanoia'",
+        "//lst[@name='spellcheck']/lst[@name='suggestions']/lst[@name='collation']/int[@name='hits']=6"        
+      );
+  }  
+  
 }

Modified: lucene/dev/branches/lucene4956/solr/core/src/test/org/apache/solr/update/SolrIndexSplitterTest.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene4956/solr/core/src/test/org/apache/solr/update/SolrIndexSplitterTest.java?rev=1485259&r1=1485258&r2=1485259&view=diff
==============================================================================
--- lucene/dev/branches/lucene4956/solr/core/src/test/org/apache/solr/update/SolrIndexSplitterTest.java (original)
+++ lucene/dev/branches/lucene4956/solr/core/src/test/org/apache/solr/update/SolrIndexSplitterTest.java Wed May 22 15:51:08 2013
@@ -95,7 +95,7 @@ public class SolrIndexSplitterTest exten
       request = lrf.makeRequest("q", "dummy");
 
       SplitIndexCommand command = new SplitIndexCommand(request,
-          Lists.newArrayList(indexDir1.getAbsolutePath(), indexDir2.getAbsolutePath()), null, ranges);
+          Lists.newArrayList(indexDir1.getAbsolutePath(), indexDir2.getAbsolutePath()), null, ranges, new PlainIdRouter());
       new SolrIndexSplitter(command).split();
 
       Directory directory = h.getCore().getDirectoryFactory().get(indexDir1.getAbsolutePath(),
@@ -148,7 +148,7 @@ public class SolrIndexSplitterTest exten
       try {
         request = lrf.makeRequest("q", "dummy");
 
-        SplitIndexCommand command = new SplitIndexCommand(request, null, Lists.newArrayList(core1, core2), ranges);
+        SplitIndexCommand command = new SplitIndexCommand(request, null, Lists.newArrayList(core1, core2), ranges, new PlainIdRouter());
         new SolrIndexSplitter(command).split();
       } finally {
         if (request != null) request.close();
@@ -185,7 +185,7 @@ public class SolrIndexSplitterTest exten
       request = lrf.makeRequest("q", "dummy");
 
       SplitIndexCommand command = new SplitIndexCommand(request,
-          Lists.newArrayList(indexDir1.getAbsolutePath(), indexDir2.getAbsolutePath(), indexDir3.getAbsolutePath()), null, null);
+          Lists.newArrayList(indexDir1.getAbsolutePath(), indexDir2.getAbsolutePath(), indexDir3.getAbsolutePath()), null, null, new PlainIdRouter());
       new SolrIndexSplitter(command).split();
 
       directory = h.getCore().getDirectoryFactory().get(indexDir1.getAbsolutePath(),

Modified: lucene/dev/branches/lucene4956/solr/core/src/test/org/apache/solr/update/processor/RegexBoostProcessorTest.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene4956/solr/core/src/test/org/apache/solr/update/processor/RegexBoostProcessorTest.java?rev=1485259&r1=1485258&r2=1485259&view=diff
==============================================================================
--- lucene/dev/branches/lucene4956/solr/core/src/test/org/apache/solr/update/processor/RegexBoostProcessorTest.java (original)
+++ lucene/dev/branches/lucene4956/solr/core/src/test/org/apache/solr/update/processor/RegexBoostProcessorTest.java Wed May 22 15:51:08 2013
@@ -24,6 +24,7 @@ import org.apache.solr.request.SolrQuery
 import org.apache.solr.response.SolrQueryResponse;
 import org.apache.solr.servlet.SolrRequestParsers;
 import org.apache.solr.update.AddUpdateCommand;
+import org.junit.AfterClass;
 import org.junit.Before;
 import org.junit.BeforeClass;
 import org.junit.Test;
@@ -50,6 +51,15 @@ public class RegexBoostProcessorTest ext
     factory.init(parameters.toNamedList());
     reProcessor = (RegexpBoostProcessor) factory.getInstance(req, resp, null);
   }
+  
+  @AfterClass
+  public static void tearDownAfterClass() throws Exception {
+    // null static members for gc
+    reProcessor = null;
+    _parser = null;
+    parameters = null;
+    factory = null;
+  }
 
   @Before
   public void setUp() throws Exception {

Modified: lucene/dev/branches/lucene4956/solr/example/cloud-scripts/zkcli.bat
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene4956/solr/example/cloud-scripts/zkcli.bat?rev=1485259&r1=1485258&r2=1485259&view=diff
==============================================================================
--- lucene/dev/branches/lucene4956/solr/example/cloud-scripts/zkcli.bat (original)
+++ lucene/dev/branches/lucene4956/solr/example/cloud-scripts/zkcli.bat Wed May 22 15:51:08 2013
@@ -8,5 +8,4 @@ REM  Find location of this script
 set SDIR=%~dp0
 if "%SDIR:~-1%"=="\" set SDIR=%SDIR:~0,-1%
 
-     
-"%JVM%" -classpath "%SDIR%\..\solr-webapp\webapp\WEB-INF\lib\*;%SDIR%\..\lib\ext" org.apache.solr.cloud.ZkCLI %*
+"%JVM%" -Dlog4j.configuration=file:%SDIR%\log4j.properties -classpath "%SDIR%\..\solr-webapp\webapp\WEB-INF\lib\*;%SDIR%\..\lib\ext\*" org.apache.solr.cloud.ZkCLI %*

Modified: lucene/dev/branches/lucene4956/solr/example/cloud-scripts/zkcli.sh
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene4956/solr/example/cloud-scripts/zkcli.sh?rev=1485259&r1=1485258&r2=1485259&view=diff
==============================================================================
--- lucene/dev/branches/lucene4956/solr/example/cloud-scripts/zkcli.sh (original)
+++ lucene/dev/branches/lucene4956/solr/example/cloud-scripts/zkcli.sh Wed May 22 15:51:08 2013
@@ -9,6 +9,5 @@ JVM="java"
 
 sdir="`dirname \"$0\"`"
 
-
-$JVM  -classpath "$sdir/../solr-webapp/webapp/WEB-INF/lib/*:$sdir/../lib/ext/*" org.apache.solr.cloud.ZkCLI ${1+"$@"}
+PATH=$JAVA_HOME/bin:$PATH $JVM -Dlog4j.configuration=file:$sdir/log4j.properties -classpath "$sdir/../solr-webapp/webapp/WEB-INF/lib/*:$sdir/../lib/ext/*" org.apache.solr.cloud.ZkCLI ${1+"$@"}
 

Modified: lucene/dev/branches/lucene4956/solr/example/example-DIH/solr/rss/conf/rss-data-config.xml
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene4956/solr/example/example-DIH/solr/rss/conf/rss-data-config.xml?rev=1485259&r1=1485258&r2=1485259&view=diff
==============================================================================
--- lucene/dev/branches/lucene4956/solr/example/example-DIH/solr/rss/conf/rss-data-config.xml (original)
+++ lucene/dev/branches/lucene4956/solr/example/example-DIH/solr/rss/conf/rss-data-config.xml Wed May 22 15:51:08 2013
@@ -5,22 +5,22 @@
                 pk="link"
                 url="http://rss.slashdot.org/Slashdot/slashdot"
                 processor="XPathEntityProcessor"
-                forEach="/rss/channel | /rss/item"
+                forEach="/rss/channel/item"
                 transformer="DateFormatTransformer">
 				
             <field column="source" xpath="/rss/channel/title" commonField="true" />
             <field column="source-link" xpath="/rss/channel/link" commonField="true" />
             <field column="subject" xpath="/rss/channel/subject" commonField="true" />
 			
-            <field column="title" xpath="/rss/item/title" />
-            <field column="link" xpath="/rss/item/link" />
-            <field column="description" xpath="/rss/item/description" />
-            <field column="creator" xpath="/rss/item/creator" />
-            <field column="item-subject" xpath="/rss/item/subject" />
-            <field column="date" xpath="/rss/item/date" dateTimeFormat="yyyy-MM-dd'T'HH:mm:ss" />
-            <field column="slash-department" xpath="/rss/item/department" />
-            <field column="slash-section" xpath="/rss/item/section" />
-            <field column="slash-comments" xpath="/rss/item/comments" />
+            <field column="title" xpath="/rss/channel/item/title" />
+            <field column="link" xpath="/rss/channel/item/link" />
+            <field column="description" xpath="/rss/channel/item/description" />
+            <field column="creator" xpath="/rss/channel/item/creator" />
+            <field column="item-subject" xpath="/rss/channel/item/subject" />
+            <field column="date" xpath="/rss/channel/item/date" dateTimeFormat="yyyy-MM-dd'T'HH:mm:ss" />
+            <field column="slash-department" xpath="/rss/channel/item/department" />
+            <field column="slash-section" xpath="/rss/channel/item/section" />
+            <field column="slash-comments" xpath="/rss/channel/item/comments" />
         </entity>
     </document>
 </dataConfig>

Modified: lucene/dev/branches/lucene4956/solr/solrj/src/java/org/apache/solr/client/solrj/impl/LBHttpSolrServer.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene4956/solr/solrj/src/java/org/apache/solr/client/solrj/impl/LBHttpSolrServer.java?rev=1485259&r1=1485258&r2=1485259&view=diff
==============================================================================
--- lucene/dev/branches/lucene4956/solr/solrj/src/java/org/apache/solr/client/solrj/impl/LBHttpSolrServer.java (original)
+++ lucene/dev/branches/lucene4956/solr/solrj/src/java/org/apache/solr/client/solrj/impl/LBHttpSolrServer.java Wed May 22 15:51:08 2013
@@ -80,7 +80,7 @@ public class LBHttpSolrServer extends So
   private final Map<String, ServerWrapper> aliveServers = new LinkedHashMap<String, ServerWrapper>();
   // access to aliveServers should be synchronized on itself
   
-  private final Map<String, ServerWrapper> zombieServers = new ConcurrentHashMap<String, ServerWrapper>();
+  protected final Map<String, ServerWrapper> zombieServers = new ConcurrentHashMap<String, ServerWrapper>();
 
   // changes to aliveServers are reflected in this array, no need to synchronize
   private volatile ServerWrapper[] aliveServerList = new ServerWrapper[0];
@@ -99,7 +99,7 @@ public class LBHttpSolrServer extends So
     solrQuery.setRows(0);
   }
 
-  private static class ServerWrapper {
+  protected static class ServerWrapper {
     final HttpSolrServer solrServer;
 
     long lastUsed;     // last time used for a real request
@@ -335,8 +335,7 @@ public class LBHttpSolrServer extends So
 
   }
 
-  private Exception addZombie(HttpSolrServer server,
-      Exception e) {
+  protected Exception addZombie(HttpSolrServer server, Exception e) {
 
     ServerWrapper wrapper;
 

Modified: lucene/dev/branches/lucene4956/solr/solrj/src/java/org/apache/solr/common/cloud/CompositeIdRouter.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene4956/solr/solrj/src/java/org/apache/solr/common/cloud/CompositeIdRouter.java?rev=1485259&r1=1485258&r2=1485259&view=diff
==============================================================================
--- lucene/dev/branches/lucene4956/solr/solrj/src/java/org/apache/solr/common/cloud/CompositeIdRouter.java (original)
+++ lucene/dev/branches/lucene4956/solr/solrj/src/java/org/apache/solr/common/cloud/CompositeIdRouter.java Wed May 22 15:51:08 2013
@@ -36,7 +36,7 @@ public class CompositeIdRouter extends H
   private int separator = '!';
 
   // separator used to optionally specify number of bits to allocate toward first part.
-  private int bitsSepartor = '/';
+  private int bitsSeparator = '/';
   private int bits = 16;
   private int mask1 = 0xffff0000;
   private int mask2 = 0x0000ffff;
@@ -59,7 +59,7 @@ public class CompositeIdRouter extends H
   }
 
   @Override
-  protected int sliceHash(String id, SolrInputDocument doc, SolrParams params) {
+  public int sliceHash(String id, SolrInputDocument doc, SolrParams params) {
     int idx = id.indexOf(separator);
     if (idx < 0) {
       return Hash.murmurhash3_x86_32(id, 0, id.length(), 0);
@@ -69,7 +69,7 @@ public class CompositeIdRouter extends H
     int m2 = mask2;
 
     String part1 = id.substring(0,idx);
-    int commaIdx = part1.indexOf(bitsSepartor);
+    int commaIdx = part1.indexOf(bitsSeparator);
     if (commaIdx > 0) {
       int firstBits = getBits(part1, commaIdx);
       if (firstBits >= 0) {
@@ -105,7 +105,7 @@ public class CompositeIdRouter extends H
     int m2 = mask2;
 
     String part1 = id.substring(0,idx);
-    int bitsSepIdx = part1.indexOf(bitsSepartor);
+    int bitsSepIdx = part1.indexOf(bitsSeparator);
     if (bitsSepIdx > 0) {
       int firstBits = getBits(part1, bitsSepIdx);
       if (firstBits >= 0) {

Modified: lucene/dev/branches/lucene4956/solr/solrj/src/java/org/apache/solr/common/cloud/HashBasedRouter.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene4956/solr/solrj/src/java/org/apache/solr/common/cloud/HashBasedRouter.java?rev=1485259&r1=1485258&r2=1485259&view=diff
==============================================================================
--- lucene/dev/branches/lucene4956/solr/solrj/src/java/org/apache/solr/common/cloud/HashBasedRouter.java (original)
+++ lucene/dev/branches/lucene4956/solr/solrj/src/java/org/apache/solr/common/cloud/HashBasedRouter.java Wed May 22 15:51:08 2013
@@ -42,7 +42,7 @@ public abstract class HashBasedRouter ex
     return range != null && range.includes(hash);
   }
 
-  protected int sliceHash(String id, SolrInputDocument sdoc, SolrParams params) {
+  public int sliceHash(String id, SolrInputDocument sdoc, SolrParams params) {
     return Hash.murmurhash3_x86_32(id, 0, id.length(), 0);
   }
 

Modified: lucene/dev/branches/lucene4956/solr/solrj/src/java/org/apache/solr/common/cloud/SolrZkClient.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene4956/solr/solrj/src/java/org/apache/solr/common/cloud/SolrZkClient.java?rev=1485259&r1=1485258&r2=1485259&view=diff
==============================================================================
--- lucene/dev/branches/lucene4956/solr/solrj/src/java/org/apache/solr/common/cloud/SolrZkClient.java (original)
+++ lucene/dev/branches/lucene4956/solr/solrj/src/java/org/apache/solr/common/cloud/SolrZkClient.java Wed May 22 15:51:08 2013
@@ -311,13 +311,13 @@ public class SolrZkClient {
   
   public void makePath(String path, File file, boolean failOnExists, boolean retryOnConnLoss)
       throws IOException, KeeperException, InterruptedException {
-    makePath(path, FileUtils.readFileToString(file).getBytes("UTF-8"),
+    makePath(path, FileUtils.readFileToByteArray(file),
         CreateMode.PERSISTENT, null, failOnExists, retryOnConnLoss);
   }
   
   public void makePath(String path, File file, boolean retryOnConnLoss) throws IOException,
       KeeperException, InterruptedException {
-    makePath(path, FileUtils.readFileToString(file).getBytes("UTF-8"), retryOnConnLoss);
+    makePath(path, FileUtils.readFileToByteArray(file), retryOnConnLoss);
   }
   
   public void makePath(String path, CreateMode createMode, boolean retryOnConnLoss) throws KeeperException,

Modified: lucene/dev/branches/lucene4956/solr/solrj/src/java/org/apache/solr/common/params/SpellingParams.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene4956/solr/solrj/src/java/org/apache/solr/common/params/SpellingParams.java?rev=1485259&r1=1485258&r2=1485259&view=diff
==============================================================================
--- lucene/dev/branches/lucene4956/solr/solrj/src/java/org/apache/solr/common/params/SpellingParams.java (original)
+++ lucene/dev/branches/lucene4956/solr/solrj/src/java/org/apache/solr/common/params/SpellingParams.java Wed May 22 15:51:08 2013
@@ -129,13 +129,32 @@ public interface SpellingParams {
    * </p>
    */
   public static final String SPELLCHECK_MAX_COLLATION_EVALUATIONS = SPELLCHECK_PREFIX + "maxCollationEvaluations";
-  
+  /**
+   * <p>
+   * For use with {@link SpellingParams#SPELLCHECK_MAX_COLLATION_TRIES} and 
+   * {@link SpellingParams#SPELLCHECK_COLLATE_EXTENDED_RESULTS}.
+   * A performance optimization in cases where the exact number of hits a collation would return is not needed.  
+   * Specify "0" to return the exact # of hits, otherwise give the maximum documents Lucene should collect 
+   * with which to base an estimate.  The higher the value the more likely the estimates will be accurate 
+   * (at expense of performance). 
+   * </p>
+   * 
+   * <p>
+   * The default is 0 (report exact hit-counts) when {@link SpellingParams#SPELLCHECK_COLLATE_EXTENDED_RESULTS} is TRUE.
+   * When {@link SpellingParams#SPELLCHECK_COLLATE_EXTENDED_RESULTS} is FALSE, this optimization is always performed.
+   * </p>
+   */
+  public static final String SPELLCHECK_COLLATE_MAX_COLLECT_DOCS = SPELLCHECK_PREFIX + "collateMaxCollectDocs";
   /**
    * <p>
    * Whether to use the Extended Results Format for collations. 
    * Includes "before>after" pairs to easily allow clients to generate messages like "no results for PORK.  did you mean POLK?"
    * Also indicates the # of hits each collation will return on re-query.  Default=false, which retains 1.4-compatible output.
    * </p>
+   * <p>
+   * Note: that if {@link SpellingParams#SPELLCHECK_COLLATE_MAX_COLLECT_DOCS} is set to a value greater than 0, 
+   * then the hit counts returned by this will be estimated.
+   * </p>
    */
   public static final String SPELLCHECK_COLLATE_EXTENDED_RESULTS = SPELLCHECK_PREFIX + "collateExtendedResults";
   

Modified: lucene/dev/branches/lucene4956/solr/solrj/src/java/org/apache/solr/common/util/ContentStreamBase.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene4956/solr/solrj/src/java/org/apache/solr/common/util/ContentStreamBase.java?rev=1485259&r1=1485258&r2=1485259&view=diff
==============================================================================
--- lucene/dev/branches/lucene4956/solr/solrj/src/java/org/apache/solr/common/util/ContentStreamBase.java (original)
+++ lucene/dev/branches/lucene4956/solr/solrj/src/java/org/apache/solr/common/util/ContentStreamBase.java Wed May 22 15:51:08 2013
@@ -233,4 +233,27 @@ public abstract class ContentStreamBase 
   public void setSourceInfo(String sourceInfo) {
     this.sourceInfo = sourceInfo;
   }
+  
+  /**
+   * Construct a <code>ContentStream</code> from a <code>File</code>
+   */
+  public static class ByteArrayStream extends ContentStreamBase
+  {
+    private final byte[] bytes;
+    
+    public ByteArrayStream( byte[] bytes, String source ) {
+      this.bytes = bytes; 
+      
+      this.contentType = null;
+      name = source;
+      size = new Long(bytes.length);
+      sourceInfo = source;
+    }
+
+
+    @Override
+    public InputStream getStream() throws IOException {
+      return new ByteArrayInputStream( bytes );
+    }
+  }  
 }

Modified: lucene/dev/branches/lucene4956/solr/solrj/src/java/org/apache/solr/common/util/NamedList.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene4956/solr/solrj/src/java/org/apache/solr/common/util/NamedList.java?rev=1485259&r1=1485258&r2=1485259&view=diff
==============================================================================
--- lucene/dev/branches/lucene4956/solr/solrj/src/java/org/apache/solr/common/util/NamedList.java (original)
+++ lucene/dev/branches/lucene4956/solr/solrj/src/java/org/apache/solr/common/util/NamedList.java Wed May 22 15:51:08 2013
@@ -17,10 +17,11 @@
 
 package org.apache.solr.common.util;
 
-import java.util.*;
 import java.io.Serializable;
-
-
+import java.util.ArrayList;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Map;
 
 /**
  * A simple container class for modeling an ordered list of name/value pairs.
@@ -47,9 +48,10 @@ import java.io.Serializable;
  * or simply use a regular {@link Map}
  * </p>
  *
- *
  */
 public class NamedList<T> implements Cloneable, Serializable, Iterable<Map.Entry<String,T>> {
+
+  private static final long serialVersionUID = 1957981902839867821L;
   protected final List<Object> nvPairs;
 
   /** Creates an empty instance */
@@ -57,7 +59,6 @@ public class NamedList<T> implements Clo
     nvPairs = new ArrayList<Object>();
   }
 
-
   /**
    * Creates a NamedList instance containing the "name,value" pairs contained in the
    * Entry[].
@@ -80,8 +81,8 @@ public class NamedList<T> implements Clo
    * pairwise names/values.
    *
    * <p>
-   * When using this constructor, runtime typesafety is only guaranteed if the all
-   * even numbered elements of the input list are of type "T".
+   * When using this constructor, runtime type safety is only guaranteed if
+   * all even numbered elements of the input list are of type "T".
    * </p>
    *
    * @param nameValuePairs underlying List which should be used to implement a NamedList
@@ -153,6 +154,7 @@ public class NamedList<T> implements Clo
 
   /**
    * Modifies the value of the pair at the specified index.
+   *
    * @return the value that used to be at index
    */
   public T setVal(int idx, T val) {
@@ -165,6 +167,7 @@ public class NamedList<T> implements Clo
 
   /**
    * Removes the name/value pair at the specified index.
+   *
    * @return the value at the index removed
    */
   public T remove(int idx) {
@@ -203,10 +206,11 @@ public class NamedList<T> implements Clo
    * NOTE: this runs in linear time (it scans starting at the
    * beginning of the list until it finds the first pair with
    * the specified name).
+   *
    * @return null if not found or if the value stored was null.
    * @see #indexOf
    * @see #get(String,int)
-   * 
+   *
    */
   public T get(String name) {
     return get(name,0);
@@ -219,6 +223,7 @@ public class NamedList<T> implements Clo
    * NOTE: this runs in linear time (it scans starting at the
    * specified position until it finds the first pair with
    * the specified name).
+   *
    * @return null if not found or if the value stored was null.
    * @see #indexOf
    */
@@ -237,6 +242,7 @@ public class NamedList<T> implements Clo
 
   /**
    * Gets the values for the the specified name
+   *
    * @param name Name
    * @return List of values
    */
@@ -252,6 +258,71 @@ public class NamedList<T> implements Clo
     return result;
   }
 
+  /**
+   * Recursively parses the NamedList structure to arrive at a specific element.
+   * As you descend the NamedList tree, the last element can be any type,
+   * including NamedList, but the previous elements MUST be NamedList objects
+   * themselves. A null value is returned if the indicated hierarchy doesn't
+   * exist, but NamedList allows null values so that could be the actual value
+   * at the end of the path.
+   * 
+   * This method is particularly useful for parsing the response from Solr's
+   * /admin/mbeans handler, but it also works for any complex structure.
+   * 
+   * Explicitly casting the return value is recommended. An even safer option is
+   * to accept the return value as an object and then check its type.
+   * 
+   * Usage examples:
+   * 
+   * String coreName = (String) response.findRecursive
+   * ("solr-mbeans", "CORE", "core", "stats", "coreName");
+   * long numDoc = (long) response.findRecursive
+   * ("solr-mbeans", "CORE", "searcher", "stats", "numDocs");
+   * 
+   * @param args
+   *          One or more strings specifying the tree to navigate.
+   * @return the last entry in the given path hierarchy, null if not found.
+   */
+  public Object findRecursive(String... args) {
+    NamedList<?> currentList = null;
+    Object value = null;
+    for (int i = 0; i < args.length; i++) {
+      String key = args[i];
+      /*
+       * The first time through the loop, the current list is null, so we assign
+       * it to this list. Then we retrieve the first key from this list and
+       * assign it to value.
+       * 
+       * On the next loop, we check whether the retrieved value is a NamedList.
+       * If it is, then we drop down to that NamedList, grab the value of the
+       * next key, and start the loop over. If it is not a NamedList, then we
+       * assign the value to null and break out of the loop.
+       * 
+       * Assigning the value to null and then breaking out of the loop seems
+       * like the wrong thing to do, but there's a very simple reason that it
+       * works: If we have reached the last key, then the loop ends naturally
+       * after we retrieve the value, and that code is never executed.
+       */
+      if (currentList == null) {
+        currentList = this;
+      } else {
+        if (value instanceof NamedList) {
+          currentList = (NamedList<?>) value;
+        } else {
+          value = null;
+          break;
+        }
+      }
+      /*
+       * We do not need to do a null check on currentList for the following
+       * assignment. The instanceof check above will fail if the current list is
+       * null, and if that happens, the loop will end before this point.
+       */
+      value = currentList.get(key, 0);
+    }
+    return value;
+  }
+
   @Override
   public String toString() {
     StringBuilder sb = new StringBuilder();
@@ -269,12 +340,12 @@ public class NamedList<T> implements Clo
   }
 
   /**
-   *
+   * 
    * Helper class implementing Map.Entry&lt;String, T&gt; to store the key-value
-   * relationship in NamedList (the keys of which are String-s) 
+   * relationship in NamedList (the keys of which are String-s)
    */
-  public static final class NamedListEntry<T> implements Map.Entry<String, T> {
-
+  public static final class NamedListEntry<T> implements Map.Entry<String,T> {
+    
     public NamedListEntry() {
 
     }
@@ -291,7 +362,7 @@ public class NamedList<T> implements Clo
 
     @Override
     public T getValue() {
-      return  value;
+      return value;
     }
 
     @Override
@@ -310,8 +381,8 @@ public class NamedList<T> implements Clo
    * Iterates over the Map and sequentially adds it's key/value pairs
    */
   public boolean addAll(Map<String,T> args) {
-    for( Map.Entry<String, T> entry : args.entrySet() ) {
-      add( entry.getKey(), entry.getValue() );
+    for (Map.Entry<String, T> entry : args.entrySet() ) {
+      add(entry.getKey(), entry.getValue());
     }
     return args.size()>0;
   }
@@ -332,7 +403,6 @@ public class NamedList<T> implements Clo
     return new NamedList<T>(newList);
   }
 
-
   //----------------------------------------------------------------------------
   // Iterable interface
   //----------------------------------------------------------------------------
@@ -364,14 +434,12 @@ public class NamedList<T> implements Clo
           }
 
           @Override
-          @SuppressWarnings("unchecked")
           public T getValue() {
             return list.getVal( index );
           }
 
           @Override
-          public String toString()
-          {
+          public String toString() {
             return getKey()+"="+getValue();
           }
 
@@ -391,7 +459,7 @@ public class NamedList<T> implements Clo
     return iter;
   }
 
-  /** 
+  /**
    * NOTE: this runs in linear time (it scans starting at the
    * beginning of the list until it finds the first pair with
    * the specified name).
@@ -402,7 +470,6 @@ public class NamedList<T> implements Clo
     return null;
   }
 
-
   public void clear() {
     nvPairs.clear();
   }
@@ -415,7 +482,7 @@ public class NamedList<T> implements Clo
   @Override
   public boolean equals(Object obj) {
     if (!(obj instanceof NamedList)) return false;
-    NamedList nl = (NamedList) obj;
+    NamedList<?> nl = (NamedList<?>) obj;
     return this.nvPairs.equals(nl.nvPairs);
   }
 }

Modified: lucene/dev/branches/lucene4956/solr/solrj/src/test-files/solrj/log4j.properties
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene4956/solr/solrj/src/test-files/solrj/log4j.properties?rev=1485259&r1=1485258&r2=1485259&view=diff
==============================================================================
--- lucene/dev/branches/lucene4956/solr/solrj/src/test-files/solrj/log4j.properties (original)
+++ lucene/dev/branches/lucene4956/solr/solrj/src/test-files/solrj/log4j.properties Wed May 22 15:51:08 2013
@@ -2,8 +2,8 @@
 log4j.rootLogger=INFO, CONSOLE
 
 log4j.appender.CONSOLE=org.apache.log4j.ConsoleAppender
-
-log4j.appender.CONSOLE.layout=org.apache.log4j.PatternLayout
+log4j.appender.CONSOLE.Target=System.err
+log4j.appender.CONSOLE.layout=org.apache.solr.util.SolrLogLayout
 log4j.appender.CONSOLE.layout.ConversionPattern=%-5p - %d{yyyy-MM-dd HH:mm:ss.SSS}; %C; %m\n
 
-log4j.logger.org.apache.zookeeper=WARN
\ No newline at end of file
+log4j.logger.org.apache.zookeeper=WARN

Modified: lucene/dev/branches/lucene4956/solr/solrj/src/test/org/apache/solr/common/util/NamedListTest.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene4956/solr/solrj/src/test/org/apache/solr/common/util/NamedListTest.java?rev=1485259&r1=1485258&r2=1485259&view=diff
==============================================================================
--- lucene/dev/branches/lucene4956/solr/solrj/src/test/org/apache/solr/common/util/NamedListTest.java (original)
+++ lucene/dev/branches/lucene4956/solr/solrj/src/test/org/apache/solr/common/util/NamedListTest.java Wed May 22 15:51:08 2013
@@ -29,4 +29,95 @@ public class NamedListTest extends Lucen
     assertEquals("value1", value);
     assertEquals(1, nl.size());
   }
+  
+  public void testRecursive() {
+    // key1
+    // key2
+    // - key2a
+    // - key2b
+    // --- key2b1
+    // --- key2b2
+    // - key2c
+    // - k2int1
+    // key3
+    // - key3a
+    // --- key3a1
+    // --- key3a2
+    // --- key3a3
+    // - key3b
+    // - key3c
+    
+    // this is a varied NL structure.
+    NamedList<String> nl2b = new NamedList<String>();
+    nl2b.add("key2b1", "value2b1");
+    nl2b.add("key2b2", "value2b2");
+    NamedList<String> nl3a = new NamedList<String>();
+    nl3a.add("key3a1", "value3a1");
+    nl3a.add("key3a2", "value3a2");
+    nl3a.add("key3a3", "value3a3");
+    NamedList<Object> nl2 = new NamedList<Object>();
+    nl2.add("key2a", "value2a");
+    nl2.add("key2b", nl2b);
+    nl2.add("k2int1", (int) 5);
+    NamedList<Object> nl3 = new NamedList<Object>();
+    nl3.add("key3a", nl3a);
+    nl3.add("key3b", "value3b");
+    nl3.add("key3c", "value3c");
+    nl3.add("key3c", "value3c2");
+    NamedList<Object> nl = new NamedList<Object>();
+    nl.add("key1", "value1");
+    nl.add("key2", nl2);
+    nl.add("key3", nl3);
+    
+    // Simple three-level checks.
+    String test1 = (String) nl.findRecursive("key2", "key2b", "key2b2");
+    assertEquals("value2b2", test1);
+    String test2 = (String) nl.findRecursive("key3", "key3a", "key3a3");
+    assertEquals("value3a3", test2);
+    // Two-level check.
+    String test3 = (String) nl.findRecursive("key3", "key3c");
+    assertEquals("value3c", test3);
+    // Checking that invalid values return null.
+    String test4 = (String) nl.findRecursive("key3", "key3c", "invalid");
+    assertEquals(null, test4);
+    String test5 = (String) nl.findRecursive("key3", "invalid", "invalid");
+    assertEquals(null, test5);
+    String test6 = (String) nl.findRecursive("invalid", "key3c");
+    assertEquals(null, test6);
+    // Verify that retrieved NamedList objects have the right type.
+    Object test7 = nl.findRecursive("key2", "key2b");
+    assertTrue(test7 instanceof NamedList);
+    // Integer check.
+    int test8 = (Integer) nl.findRecursive("key2", "k2int1");
+    assertEquals(5, test8);
+    // Check that a single argument works the same as get(String).
+    String test9 = (String) nl.findRecursive("key1");
+    assertEquals("value1", test9);
+    // enl == explicit nested list
+    //
+    // key1
+    // - key1a
+    // - key1b
+    // key2 (null list)
+    NamedList<NamedList<String>> enl = new NamedList<NamedList<String>>();
+    NamedList<String> enlkey1 = new NamedList<String>();
+    NamedList<String> enlkey2 = null;
+    enlkey1.add("key1a", "value1a");
+    enlkey1.add("key1b", "value1b");
+    enl.add("key1", enlkey1);
+    enl.add("key2", enlkey2);
+    
+    // Tests that are very similar to the test above, just repeated
+    // on the explicitly nested object type.
+    String enltest1 = (String) enl.findRecursive("key1", "key1a");
+    assertEquals("value1a", enltest1);
+    String enltest2 = (String) enl.findRecursive("key1", "key1b");
+    assertEquals("value1b", enltest2);
+    // Verify that when a null value is stored, the standard get method
+    // says it is null, then check the recursive method.
+    Object enltest3 = enl.get("key2");
+    assertNull(enltest3);
+    Object enltest4 = enl.findRecursive("key2");
+    assertNull(enltest4);
+  }
 }

Modified: lucene/dev/branches/lucene4956/solr/test-framework/ivy.xml
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene4956/solr/test-framework/ivy.xml?rev=1485259&r1=1485258&r2=1485259&view=diff
==============================================================================
--- lucene/dev/branches/lucene4956/solr/test-framework/ivy.xml (original)
+++ lucene/dev/branches/lucene4956/solr/test-framework/ivy.xml Wed May 22 15:51:08 2013
@@ -32,8 +32,8 @@
       <dependency org="org.apache.ant" name="ant" rev="1.8.2" transitive="false" />
 
       <dependency org="junit" name="junit" rev="4.10" transitive="false" conf="default->*;junit4-stdalone->*" />
-      <dependency org="com.carrotsearch.randomizedtesting" name="junit4-ant" rev="2.0.9" transitive="false" conf="default->*;junit4-stdalone->*" />
-      <dependency org="com.carrotsearch.randomizedtesting" name="randomizedtesting-runner" rev="2.0.9" transitive="false" conf="default->*;junit4-stdalone->*" />
+      <dependency org="com.carrotsearch.randomizedtesting" name="junit4-ant" rev="2.0.10" transitive="false" conf="default->*;junit4-stdalone->*" />
+      <dependency org="com.carrotsearch.randomizedtesting" name="randomizedtesting-runner" rev="2.0.10" transitive="false" conf="default->*;junit4-stdalone->*" />
 
       <exclude org="*" ext="*" matcher="regexp" type="${ivy.exclude.types}"/> 
     </dependencies>

Modified: lucene/dev/branches/lucene4956/solr/test-framework/src/java/org/apache/solr/util/TestHarness.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene4956/solr/test-framework/src/java/org/apache/solr/util/TestHarness.java?rev=1485259&r1=1485258&r2=1485259&view=diff
==============================================================================
--- lucene/dev/branches/lucene4956/solr/test-framework/src/java/org/apache/solr/util/TestHarness.java (original)
+++ lucene/dev/branches/lucene4956/solr/test-framework/src/java/org/apache/solr/util/TestHarness.java Wed May 22 15:51:08 2013
@@ -183,8 +183,8 @@ public class TestHarness extends BaseTes
         String solrHome = SolrResourceLoader.locateSolrHome();
         container = new CoreContainer(new SolrResourceLoader(solrHome)) {
           {
-            String hostPort = System.getProperty("hostPort");
-            String hostContext = "solr";
+            String hostPort = System.getProperty("hostPort", "8983");
+            String hostContext = System.getProperty("hostContext", "solr");
             defaultCoreName = CoreContainer.DEFAULT_DEFAULT_CORE_NAME;
             initShardHandler();
             zkSys.initZooKeeper(this, solrHome, System.getProperty("zkHost"), 30000, hostPort, hostContext, null, "30000", 30000, 30000);