You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@lucene.apache.org by mi...@apache.org on 2015/02/09 00:53:25 UTC

svn commit: r1658277 [38/38] - in /lucene/dev/branches/lucene6005: ./ dev-tools/ dev-tools/idea/solr/contrib/dataimporthandler/ dev-tools/idea/solr/contrib/velocity/ dev-tools/maven/lucene/replicator/ dev-tools/maven/solr/ dev-tools/maven/solr/contrib/...

Modified: lucene/dev/branches/lucene6005/solr/test-framework/src/java/org/apache/solr/cloud/AbstractFullDistribZkTestBase.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene6005/solr/test-framework/src/java/org/apache/solr/cloud/AbstractFullDistribZkTestBase.java?rev=1658277&r1=1658276&r2=1658277&view=diff
==============================================================================
--- lucene/dev/branches/lucene6005/solr/test-framework/src/java/org/apache/solr/cloud/AbstractFullDistribZkTestBase.java (original)
+++ lucene/dev/branches/lucene6005/solr/test-framework/src/java/org/apache/solr/cloud/AbstractFullDistribZkTestBase.java Sun Feb  8 23:53:14 2015
@@ -55,9 +55,7 @@ import org.apache.solr.core.SolrResource
 import org.apache.solr.servlet.SolrDispatchFilter;
 import org.apache.solr.update.DirectUpdateHandler2;
 import org.apache.zookeeper.CreateMode;
-import org.junit.After;
 import org.junit.AfterClass;
-import org.junit.Before;
 import org.junit.BeforeClass;
 import org.noggit.CharArr;
 import org.noggit.JSONWriter;
@@ -95,18 +93,18 @@ import static org.apache.solr.common.clo
 @Slow
 public abstract class AbstractFullDistribZkTestBase extends AbstractDistribZkTestBase {
   static Logger log = LoggerFactory.getLogger(AbstractFullDistribZkTestBase.class);
-  
+
   @BeforeClass
   public static void beforeFullSolrCloudTest() {
     // shorten the log output more for this test type
     if (formatter != null) formatter.setShorterFormat();
   }
-  
+
   public static final String SHARD1 = "shard1";
   public static final String SHARD2 = "shard2";
-  
+
   protected boolean printLayoutOnTearDown = false;
-  
+
   String t1 = "a_t";
   String i1 = "a_i1";
   String tlong = "other_tl1";
@@ -122,14 +120,14 @@ public abstract class AbstractFullDistri
   protected Map<String,List<CloudJettyRunner>> shardToJetty = new HashMap<>();
   private AtomicInteger jettyIntCntr = new AtomicInteger(0);
   protected ChaosMonkey chaosMonkey;
-  
+
   protected Map<String,CloudJettyRunner> shardToLeaderJetty = new HashMap<>();
   private boolean cloudInit;
   protected boolean checkCreatedVsState;
   protected boolean useJettyDataDir = true;
 
   protected Map<URI,SocketProxy> proxies = new HashMap<URI,SocketProxy>();
-  
+
   public static class CloudJettyRunner {
     public JettySolrRunner jetty;
     public String nodeName;
@@ -160,19 +158,19 @@ public abstract class AbstractFullDistri
       return "CloudJettyRunner [url=" + url + "]";
     }
   }
-  
+
   static class CloudSolrServerClient {
     SolrClient solrClient;
     String shardName;
     int port;
     public ZkNodeProps info;
-    
+
     public CloudSolrServerClient() {}
     
     public CloudSolrServerClient(SolrClient client) {
       this.solrClient = client;
     }
-    
+
     @Override
     public int hashCode() {
       final int prime = 31;
@@ -180,7 +178,7 @@ public abstract class AbstractFullDistri
       result = prime * result + ((solrClient == null) ? 0 : solrClient.hashCode());
       return result;
     }
-    
+
     @Override
     public boolean equals(Object obj) {
       if (this == obj) return true;
@@ -192,69 +190,67 @@ public abstract class AbstractFullDistri
       } else if (!solrClient.equals(other.solrClient)) return false;
       return true;
     }
-    
+
   }
-  
-  @Before
+
   @Override
-  public void setUp() throws Exception {
-    super.setUp();
+  public void distribSetUp() throws Exception {
+    super.distribSetUp();
     // ignoreException(".*");
     if (sliceCount > 0) {
       System.setProperty("numShards", Integer.toString(sliceCount));
     } else {
       System.clearProperty("numShards");
     }
-    
+
     if (isSSLMode()) {
       System.clearProperty("urlScheme");
       ZkStateReader zkStateReader = new ZkStateReader(zkServer.getZkAddress(),
           AbstractZkTestCase.TIMEOUT, AbstractZkTestCase.TIMEOUT);
       try {
         zkStateReader.getZkClient().create(ZkStateReader.CLUSTER_PROPS,
-          ZkStateReader.toJSON(Collections.singletonMap("urlScheme","https")), 
+          ZkStateReader.toJSON(Collections.singletonMap("urlScheme","https")),
           CreateMode.PERSISTENT, true);
       } finally {
         zkStateReader.close();
       }
     }
   }
-  
+
   @BeforeClass
   public static void beforeClass() {
     System.setProperty("solrcloud.update.delay", "0");
   }
-  
+
   @AfterClass
   public static void afterClass() throws Exception {
     System.clearProperty("solrcloud.update.delay");
     System.clearProperty("genericCoreNodeNames");
   }
-  
+
   public AbstractFullDistribZkTestBase() {
-    fixShardCount = true;
-    
-    shardCount = 4;
     sliceCount = 2;
+    fixShardCount(4);
+
     // TODO: for now, turn off stress because it uses regular clients, and we
     // need the cloud client because we kill servers
     stress = 0;
-    
+
     useExplicitNodeNames = random().nextBoolean();
   }
-  
+
   protected String getDataDir(String dataDir) throws IOException {
     return dataDir;
   }
-  
+
   protected void initCloud() throws Exception {
     assert(cloudInit == false);
     cloudInit = true;
     cloudClient = createCloudClient(DEFAULT_COLLECTION);
     cloudClient.connect();
-    
+
     ZkStateReader zkStateReader = cloudClient.getZkStateReader();
-    
+
     chaosMonkey = new ChaosMonkey(zkServer, zkStateReader, DEFAULT_COLLECTION,
         shardToJetty, shardToLeaderJetty);
   }
@@ -265,23 +261,26 @@ public abstract class AbstractFullDistri
     if (defaultCollection != null) client.setDefaultCollection(defaultCollection);
     client.getLbClient().getHttpClient().getParams()
         .setParameter(CoreConnectionPNames.CONNECTION_TIMEOUT, 30000);
+    client.getLbClient().getHttpClient().getParams()
+    .setParameter(CoreConnectionPNames.SO_TIMEOUT, 60000);
     return client;
   }
-  
+
   @Override
   protected void createServers(int numServers) throws Exception {
-    
+
     System.setProperty("collection", "control_collection");
 
     // we want hashes by default for the control, so set to 1 shard as opposed to leaving unset
     String oldNumShards = System.getProperty(ZkStateReader.NUM_SHARDS_PROP);
     System.setProperty(ZkStateReader.NUM_SHARDS_PROP, "1");
-    
+
     try {
-      
-      File controlJettyDir = createTempDir().toFile();
+
+      File controlJettyDir = createTempDir("control").toFile();
       setupJettySolrHome(controlJettyDir);
-      
+
+      System.setProperty("coreRootDirectory", controlJettyDir.toPath().resolve("cores").toString());
       controlJetty = createJetty(controlJettyDir, useJettyDataDir ? getDataDir(testDir
           + "/control/data") : null); // don't pass shard name... let it default to
                                // "shard1"
@@ -302,7 +301,7 @@ public abstract class AbstractFullDistri
                                           // cloudClient
         return;
       }
-      
+
     } finally {
       System.clearProperty("collection");
       if (oldNumShards != null) {
@@ -314,9 +313,9 @@ public abstract class AbstractFullDistri
 
 
     initCloud();
-    
+
     createJettys(numServers, checkCreatedVsState).size();
-    
+
     int cnt = getTotalReplicas(DEFAULT_COLLECTION);
     if (cnt > 0) {
       waitForCollection(cloudClient.getZkStateReader(), DEFAULT_COLLECTION, sliceCount);
@@ -345,21 +344,17 @@ public abstract class AbstractFullDistri
       Thread.sleep(500);
     }
   }
-  
+
   protected List<JettySolrRunner> createJettys(int numJettys) throws Exception {
     return createJettys(numJettys, false);
   }
 
-  protected int defaultStateFormat = 1 + random().nextInt(2);
+  protected String defaultStateFormat = String.valueOf( 1 + random().nextInt(2));
 
-  protected int getStateFormat()  {
+  protected String getStateFormat()  {
     String stateFormat = System.getProperty("tests.solr.stateFormat", null);
     if (stateFormat != null)  {
-      if ("2".equals(stateFormat)) {
-        return defaultStateFormat = 2;
-      } else if ("1".equals(stateFormat))  {
-        return defaultStateFormat = 1;
-      }
+      defaultStateFormat = stateFormat;
     }
     return defaultStateFormat; // random
   }
@@ -375,7 +370,7 @@ public abstract class AbstractFullDistri
     List<SolrClient> clients = new ArrayList<>();
     StringBuilder sb = new StringBuilder();
 
-    if (getStateFormat() == 2) {
+    if ("2".equals(getStateFormat())) {
       log.info("Creating collection1 with stateFormat=2");
       SolrZkClient zkClient = new SolrZkClient(zkServer.getZkAddress(),
           AbstractZkTestCase.TIMEOUT, AbstractZkTestCase.TIMEOUT);
@@ -391,33 +386,34 @@ public abstract class AbstractFullDistri
       if (sb.length() > 0) sb.append(',');
       int cnt = this.jettyIntCntr.incrementAndGet();
 
-      File jettyDir = createTempDir().toFile();
+      File jettyDir = createTempDir("shard-" + i).toFile();
 
       jettyDir.mkdirs();
       setupJettySolrHome(jettyDir);
       log.info("create jetty " + i);
+      System.setProperty("coreRootDirectory", jettyDir.toPath().resolve("cores").toString());
       JettySolrRunner j = createJetty(jettyDir, useJettyDataDir ? getDataDir(testDir + "/jetty"
           + cnt) : null, null, "solrconfig.xml", null);
       jettys.add(j);
       SolrClient client = createNewSolrClient(j.getLocalPort());
       clients.add(client);
     }
-  
+
     this.jettys.addAll(jettys);
     this.clients.addAll(clients);
-    
+
     int numShards = getTotalReplicas(DEFAULT_COLLECTION);
     if (checkCreatedVsState) {
       // now wait until we see that the number of shards in the cluster state
       // matches what we expect
       int retries = 0;
-      while (numShards != shardCount) {
+      while (numShards != getShardCount()) {
         numShards = getTotalReplicas(DEFAULT_COLLECTION);
-        if (numShards == shardCount) break;
+        if (numShards == getShardCount()) break;
         if (retries++ == 60) {
           printLayoutOnTearDown = true;
           fail("Shards in the state does not match what we set:" + numShards
-              + " vs " + shardCount);
+              + " vs " + getShardCount());
         }
         Thread.sleep(500);
       }
@@ -432,7 +428,7 @@ public abstract class AbstractFullDistri
     if (numShards > 0) {
       updateMappingsFromZk(this.jettys, this.clients);
     }
-    
+
     // build the shard string
     for (int i = 1; i <= numJettys / 2; i++) {
       JettySolrRunner j = this.jettys.get(i);
@@ -442,7 +438,7 @@ public abstract class AbstractFullDistri
       sb.append("|").append(buildUrl(j2.getLocalPort()));
     }
     shards = sb.toString();
-    
+
     return jettys;
   }
 
@@ -494,30 +490,30 @@ public abstract class AbstractFullDistri
     }
     return cnt;
   }
-  
+
   public JettySolrRunner createJetty(String dataDir, String ulogDir, String shardList,
       String solrConfigOverride) throws Exception {
-    
+
     JettySolrRunner jetty = new JettySolrRunner(getSolrHome(), context, 0,
         solrConfigOverride, null, false, getExtraServlets(), sslConfig, getExtraRequestFilters());
     jetty.setShards(shardList);
     jetty.setDataDir(getDataDir(dataDir));
     jetty.start();
-    
+
     return jetty;
   }
-  
+
   public JettySolrRunner createJetty(File solrHome, String dataDir, String shardList, String solrConfigOverride, String schemaOverride) throws Exception {
     // randomly test a relative solr.home path
     if (random().nextBoolean()) {
       solrHome = getRelativeSolrHomePath(solrHome);
     }
-    
+
     JettySolrRunner jetty = new JettySolrRunner(solrHome.getPath(), context, 0, solrConfigOverride, schemaOverride, false, getExtraServlets(), sslConfig, getExtraRequestFilters());
     jetty.setShards(shardList);
     jetty.setDataDir(getDataDir(dataDir));
     jetty.start();
-    
+
     return jetty;
   }
 
@@ -598,7 +594,7 @@ public abstract class AbstractFullDistri
   private File getRelativeSolrHomePath(File solrHome) {
     String path = SolrResourceLoader.normalizeDir(new File(".").getAbsolutePath());
     String base = new File(solrHome.getPath()).getAbsolutePath();
-    
+
     if (base.startsWith(".")) {
       base = base.replaceFirst("\\.", new File(".").getName());
     }
@@ -606,14 +602,14 @@ public abstract class AbstractFullDistri
     if (path.endsWith(File.separator + ".")) {
       path = path.substring(0, path.length() - 2);
     }
-    
+
     int splits = path.split("\\" + File.separator).length;
-    
+
     StringBuilder p = new StringBuilder();
     for (int i = 0; i < splits - 2; i++) {
       p.append("..").append(File.separator);
-    }   
-    
+    }
+
     String prefix = FilenameUtils.getPrefix(path);
     if (base.startsWith(prefix)) {
       base = base.substring(prefix.length());
@@ -632,7 +628,7 @@ public abstract class AbstractFullDistri
     zkStateReader.updateClusterState(true);
     cloudJettys.clear();
     shardToJetty.clear();
-    
+
     ClusterState clusterState = zkStateReader.getClusterState();
     DocCollection coll = clusterState.getCollection(DEFAULT_COLLECTION);
 
@@ -645,28 +641,28 @@ public abstract class AbstractFullDistri
         for (Replica replica : slice.getReplicas()) {
           int port = new URI(((HttpSolrClient) client).getBaseURL())
               .getPort();
-          
+
           if (replica.getStr(ZkStateReader.BASE_URL_PROP).contains(":" + port)) {
             CloudSolrServerClient csc = new CloudSolrServerClient();
             csc.solrClient = client;
             csc.port = port;
             csc.shardName = replica.getStr(ZkStateReader.NODE_NAME_PROP);
             csc.info = replica;
-            
+
             theClients .add(csc);
-            
+
             break nextClient;
           }
         }
       }
     }
- 
+
     for (JettySolrRunner jetty : jettys) {
       int port = jetty.getLocalPort();
       if (port == -1) {
         throw new RuntimeException("Cannot find the port for jetty");
       }
-      
+
       nextJetty:
       for (Slice slice : coll.getSlices()) {
         Set<Entry<String,Replica>> entries = slice.getReplicasMap().entrySet();
@@ -696,7 +692,7 @@ public abstract class AbstractFullDistri
         }
       }
     }
-    
+
     // # of jetties may not match replicas in shard here, because we don't map
     // jetties that are not running - every shard should have at least one
     // running jetty though
@@ -706,13 +702,13 @@ public abstract class AbstractFullDistri
       if (!allowOverSharding) {
         assertNotNull("Test setup problem: We found no jetties for shard: "
             + slice.getName() + " just:" + shardToJetty.keySet(), jetties);
-        
+
         assertEquals("slice:" + slice.getName(), slice.getReplicas().size(),
             jetties.size());
       }
     }
   }
-  
+
   private CloudSolrServerClient findClientByPort(int port, List<CloudSolrServerClient> theClients) {
     for (CloudSolrServerClient client : theClients) {
       if (client.port == port) {
@@ -724,7 +720,7 @@ public abstract class AbstractFullDistri
 
   @Override
   protected void setDistributedParams(ModifiableSolrParams params) {
-    
+
     if (r.nextBoolean()) {
       // don't set shards, let that be figured out from the cloud state
     } else {
@@ -737,19 +733,19 @@ public abstract class AbstractFullDistri
       params.set("shards", sb.toString());
     }
   }
-  
+
   @Override
   protected void indexDoc(SolrInputDocument doc) throws IOException,
       SolrServerException {
-    
+
     UpdateRequest req = new UpdateRequest();
     req.add(doc);
     req.setParam("CONTROL", "TRUE");
     req.process(controlClient);
-    
+
     // if we wanted to randomly pick a client - but sometimes they may be
     // down...
-    
+
     // boolean pick = random.nextBoolean();
     //
     // int which = (doc.getField(id).toString().hashCode() & 0x7fffffff) %
@@ -762,13 +758,13 @@ public abstract class AbstractFullDistri
     //
     // HttpSolrServer client = (HttpSolrServer)
     // clients.get(which);
-    
+
     UpdateRequest ureq = new UpdateRequest();
     ureq.add(doc);
     // ureq.setParam(UpdateParams.UPDATE_CHAIN, DISTRIB_UPDATE_CHAIN);
     ureq.process(cloudClient);
   }
-  
+
   @Override
   protected void index_specific(int serverNumber, Object... fields)
       throws Exception {
@@ -780,7 +776,7 @@ public abstract class AbstractFullDistri
     
     HttpSolrClient client = (HttpSolrClient) clients
         .get(serverNumber);
-    
+
     UpdateRequest ureq = new UpdateRequest();
     ureq.add(doc);
     // ureq.setParam("update.chain", DISTRIB_UPDATE_CHAIN);
@@ -793,16 +789,16 @@ public abstract class AbstractFullDistri
     for (int i = 0; i < fields.length; i += 2) {
       doc.addField((String) (fields[i]), fields[i + 1]);
     }
-    
+
     UpdateRequest ureq = new UpdateRequest();
     ureq.add(doc);
     // ureq.setParam("update.chain", DISTRIB_UPDATE_CHAIN);
     ureq.process(client);
-    
+
     // add to control second in case adding to shards fails
     controlClient.add(doc);
   }
-  
+
   @Override
   protected void del(String q) throws Exception {
     controlClient.deleteByQuery(q);
@@ -816,25 +812,25 @@ public abstract class AbstractFullDistri
     }
      ***/
   }// serial commit...
-  
+
   protected void waitForRecoveriesToFinish(boolean verbose)
       throws Exception {
     ZkStateReader zkStateReader = cloudClient.getZkStateReader();
     super.waitForRecoveriesToFinish(DEFAULT_COLLECTION, zkStateReader, verbose);
   }
-  
+
   protected void waitForRecoveriesToFinish(String collection, boolean verbose)
       throws Exception {
     ZkStateReader zkStateReader = cloudClient.getZkStateReader();
     super.waitForRecoveriesToFinish(collection, zkStateReader, verbose);
   }
-  
+
   protected void waitForRecoveriesToFinish(boolean verbose, int timeoutSeconds)
       throws Exception {
     ZkStateReader zkStateReader = cloudClient.getZkStateReader();
     super.waitForRecoveriesToFinish(DEFAULT_COLLECTION, zkStateReader, verbose, true, timeoutSeconds);
   }
-  
+
   protected void checkQueries() throws Exception {
 
     handle.put("_version_", SKIPVAL);
@@ -858,7 +854,7 @@ public abstract class AbstractFullDistri
       query("q", "*:*", "sort", f + " desc");
       query("q", "*:*", "sort", f + " asc");
     }
-    
+
     // these queries should be exactly ordered and scores should exactly match
     query("q", "*:*", "sort", i1 + " desc");
     query("q", "*:*", "sort", i1 + " asc");
@@ -875,30 +871,30 @@ public abstract class AbstractFullDistri
     handle.remove("maxScore");
     query("q", "{!func}" + i1, "fl", "*,score"); // even scores should match
                                                  // exactly here
-    
+
     handle.put("highlighting", UNORDERED);
     handle.put("response", UNORDERED);
-    
+
     handle.put("maxScore", SKIPVAL);
     query("q", "quick");
     query("q", "all", "fl", "id", "start", "0");
     query("q", "all", "fl", "foofoofoo", "start", "0"); // no fields in returned
                                                         // docs
     query("q", "all", "fl", "id", "start", "100");
-    
+
     handle.put("score", SKIPVAL);
     query("q", "quick", "fl", "*,score");
     query("q", "all", "fl", "*,score", "start", "1");
     query("q", "all", "fl", "*,score", "start", "100");
-    
+
     query("q", "now their fox sat had put", "fl", "*,score", "hl", "true",
         "hl.fl", t1);
-    
+
     query("q", "now their fox sat had put", "fl", "foofoofoo", "hl", "true",
         "hl.fl", t1);
-    
+
     query("q", "matchesnothing", "fl", "*,score");
-    
+
     query("q", "*:*", "rows", 100, "facet", "true", "facet.field", t1);
     query("q", "*:*", "rows", 100, "facet", "true", "facet.field", t1,
         "facet.limit", -1, "facet.sort", "count");
@@ -916,11 +912,11 @@ public abstract class AbstractFullDistri
         "facet.offset", 1);
     query("q", "*:*", "rows", 100, "facet", "true", "facet.field", t1,
         "facet.mincount", 2);
-    
+
     // test faceting multiple things at once
     query("q", "*:*", "rows", 100, "facet", "true", "facet.query", "quick",
         "facet.query", "all", "facet.query", "*:*", "facet.field", t1);
-    
+
     // test filter tagging, facet exclusion, and naming (multi-select facet
     // support)
     query("q", "*:*", "rows", 100, "facet", "true", "facet.query",
@@ -933,16 +929,16 @@ public abstract class AbstractFullDistri
         "{!ex=t1}SubjectTerms_mfacet", "fq",
         "{!tag=t1}SubjectTerms_mfacet:(test 1)", "facet.limit", "10",
         "facet.mincount", "1");
-    
+
     // test field that is valid in schema but missing in all shards
     query("q", "*:*", "rows", 100, "facet", "true", "facet.field",
         missingField, "facet.mincount", 2);
     // test field that is valid in schema and missing in some shards
     query("q", "*:*", "rows", 100, "facet", "true", "facet.field", oddField,
         "facet.mincount", 2);
-    
+
     query("q", "*:*", "sort", i1 + " desc", "stats", "true", "stats.field", i1);
-    
+
     // Try to get better coverage for refinement queries by turning off over
     // requesting.
     // This makes it much more likely that we may not get the top facet values
@@ -956,15 +952,15 @@ public abstract class AbstractFullDistri
         "{!key='a b/c \\' \\} foo'}" + t1, "facet.limit", 5,
         "facet.shard.limit", 5);
     handle.remove("facet_fields");
-    
+
     query("q", "*:*", "sort", "n_tl1 desc");
-    
+
     // index the same document to two shards and make sure things
     // don't blow up.
     // assumes first n clients are first n shards
     if (clients.size() >= 2) {
       index(id, 100, i1, 107, t1, "oh no, a duplicate!");
-      for (int i = 0; i < shardCount; i++) {
+      for (int i = 0; i < getShardCount(); i++) {
         index_specific(i, id, 100, i1, 107, t1, "oh no, a duplicate!");
       }
       commit();
@@ -973,7 +969,7 @@ public abstract class AbstractFullDistri
       query("q", "*:*", "rows", 100);
     }
   }
-  
+
   protected void indexAbunchOfDocs() throws Exception {
     indexr(id, 2, i1, 50, t1, "to come to the aid of their country.");
     indexr(id, 3, i1,  2, t1, "how now brown cow");
@@ -993,7 +989,7 @@ public abstract class AbstractFullDistri
         "Great works are performed, not by strength, but by perseverance.");
     indexr(id, 13, i1, 232, t1, "no eggs on wall, lesson learned",
         oddField, "odd man out");
-    
+
     indexr(id, 14, "SubjectTerms_mfacet", new String[] {"mathematical models",
         "mathematical analysis"});
     indexr(id, 15, "SubjectTerms_mfacet", new String[] {"test 1", "test 2",
@@ -1005,19 +1001,19 @@ public abstract class AbstractFullDistri
       vals[i] = "test " + i;
     }
     indexr(id, 17, "SubjectTerms_mfacet", vals);
-    
+
     for (int i = 100; i < 150; i++) {
       indexr(id, i);
     }
   }
-  
+
   /**
-   * Executes a query against each live and active replica of the specified shard 
+   * Executes a query against each live and active replica of the specified shard
    * and aserts that the results are identical.
    *
    * @see #queryAndCompare
    */
-  public QueryResponse queryAndCompareReplicas(SolrParams params, String shard) 
+  public QueryResponse queryAndCompareReplicas(SolrParams params, String shard)
     throws Exception {
 
     ArrayList<SolrClient> shardClients = new ArrayList<>(7);
@@ -1042,9 +1038,9 @@ public abstract class AbstractFullDistri
 
   /**
    * For each Shard, executes a query against each live and active replica of that shard
-   * and asserts that the results are identical for each replica of the same shard.  
-   * Because results are not compared between replicas of different shards, this method 
-   * should be safe for comparing the results of any query, even if it contains 
+   * and asserts that the results are identical for each replica of the same shard.
+   * Because results are not compared between replicas of different shards, this method
+   * should be safe for comparing the results of any query, even if it contains
    * "distrib=false", because the replicas should all be identical.
    *
    * @see AbstractFullDistribZkTestBase#queryAndCompareReplicas(SolrParams, String)
@@ -1058,25 +1054,25 @@ public abstract class AbstractFullDistri
     }
   }
 
-  /** 
-   * Returns a non-null string if replicas within the same shard do not have a 
-   * consistent number of documents. 
+  /**
+   * Returns a non-null string if replicas within the same shard do not have a
+   * consistent number of documents.
    */
   protected void checkShardConsistency(String shard) throws Exception {
     checkShardConsistency(shard, false, false);
   }
 
-  /** 
-   * Returns a non-null string if replicas within the same shard do not have a 
+  /**
+   * Returns a non-null string if replicas within the same shard do not have a
    * consistent number of documents.
-   * If expectFailure==false, the exact differences found will be logged since 
+   * If expectFailure==false, the exact differences found will be logged since
    * this would be an unexpected failure.
-   * verbose causes extra debugging into to be displayed, even if everything is 
+   * verbose causes extra debugging into to be displayed, even if everything is
    * consistent.
    */
   protected String checkShardConsistency(String shard, boolean expectFailure, boolean verbose)
       throws Exception {
-    
+
     List<CloudJettyRunner> solrJetties = shardToJetty.get(shard);
     if (solrJetties == null) {
       throw new RuntimeException("shard not found:" + shard + " keys:"
@@ -1112,7 +1108,7 @@ public abstract class AbstractFullDistri
             + e.getMessage() + "\n");
         continue;
       }
-      
+
       boolean live = false;
       String nodeName = props.getStr(ZkStateReader.NODE_NAME_PROP);
       if (zkStateReader.getClusterState().liveNodesContain(nodeName)) {
@@ -1149,19 +1145,19 @@ public abstract class AbstractFullDistri
       }
     }
     return failMessage;
-    
+
   }
-  
+
   public void showCounts() {
     Set<String> theShards = shardToJetty.keySet();
-    
+
     for (String shard : theShards) {
       List<CloudJettyRunner> solrJetties = shardToJetty.get(shard);
-      
+
       for (CloudJettyRunner cjetty : solrJetties) {
         ZkNodeProps props = cjetty.info;
         System.err.println("PROPS:" + props);
-        
+
         try {
           SolrParams query = params("q", "*:*", "rows", "0", "distrib",
               "false", "tests", "checkShardConsistency"); // "tests" is just a
@@ -1187,11 +1183,11 @@ public abstract class AbstractFullDistri
           live = true;
         }
         System.err.println(" live:" + live);
-        
+
       }
     }
   }
-  
+
   protected void randomlyEnableAutoSoftCommit() {
     if (r.nextBoolean()) {
       enableAutoSoftCommit(1000);
@@ -1199,7 +1195,7 @@ public abstract class AbstractFullDistri
       log.info("Not turning on auto soft commit");
     }
   }
-  
+
   protected void enableAutoSoftCommit(int time) {
     log.info("Turning on auto soft commit: " + time);
     for (List<CloudJettyRunner> jettyList : shardToJetty.values()) {
@@ -1228,7 +1224,7 @@ public abstract class AbstractFullDistri
       throws Exception {
     checkShardConsistency(checkVsControl, verbose, null, null);
   }
-  
+
   /* Checks shard consistency and optionally checks against the control shard.
    * The test will be failed if differences are found.
    */
@@ -1236,7 +1232,7 @@ public abstract class AbstractFullDistri
       throws Exception {
 
     updateMappingsFromZk(jettys, clients, true);
-    
+
     Set<String> theShards = shardToJetty.keySet();
     String failMessage = null;
     for (String shard : theShards) {
@@ -1245,7 +1241,7 @@ public abstract class AbstractFullDistri
         failMessage = shardFailMessage;
       }
     }
-    
+
     if (failMessage != null) {
       fail(failMessage);
     }
@@ -1260,7 +1256,7 @@ public abstract class AbstractFullDistri
     SolrDocumentList cloudDocList = cloudClient.query(q).getResults();
     long cloudClientDocs = cloudDocList.getNumFound();
 
-    
+
     // now check that the right # are on each shard
     theShards = shardToJetty.keySet();
     int cnt = 0;
@@ -1314,7 +1310,7 @@ public abstract class AbstractFullDistri
     }
     return null;
   }
-  
+
   protected void assertDocCounts(boolean verbose) throws Exception {
     // TODO: as we create the clients, we should build a map from shard to
     // node/client
@@ -1324,7 +1320,7 @@ public abstract class AbstractFullDistri
         + "\n\n");
     long controlCount = controlClient.query(new SolrQuery("*:*")).getResults()
         .getNumFound();
-    
+
     // do some really inefficient mapping...
     ZkStateReader zk = new ZkStateReader(zkServer.getZkAddress(), 10000,
         AbstractZkTestCase.TIMEOUT);
@@ -1337,12 +1333,12 @@ public abstract class AbstractFullDistri
     } finally {
       zk.close();
     }
-    
+
     if (slices == null) {
       throw new RuntimeException("Could not find collection "
           + DEFAULT_COLLECTION + " in " + clusterState.getCollections());
     }
-    
+
     for (CloudJettyRunner cjetty : cloudJettys) {
       CloudSolrServerClient client = cjetty.client;
       for (Map.Entry<String,Slice> slice : slices.entrySet()) {
@@ -1368,7 +1364,7 @@ public abstract class AbstractFullDistri
         query.set("distrib", false);
         count = client.solrClient.query(query).getResults().getNumFound();
       }
-      
+
       if (verbose) System.err.println("client docs:" + count + "\n\n");
     }
     if (verbose) System.err.println("control docs:"
@@ -1378,39 +1374,39 @@ public abstract class AbstractFullDistri
     assertEquals("Doc Counts do not add up", controlCount,
         cloudClient.query(query).getResults().getNumFound());
   }
-  
+
   @Override
   protected QueryResponse queryServer(ModifiableSolrParams params)
       throws SolrServerException {
-    
+
     if (r.nextBoolean()) params.set("collection", DEFAULT_COLLECTION);
-    
+
     QueryResponse rsp = cloudClient.query(params);
     return rsp;
   }
-  
+
   static abstract class StopableThread extends Thread {
     public StopableThread(String name) {
       super(name);
     }
     public abstract void safeStop();
   }
-  
+
   class StopableSearchThread extends StopableThread {
     private volatile boolean stop = false;
     protected final AtomicInteger queryFails = new AtomicInteger();
     private String[] QUERIES = new String[] {"to come","their country","aid","co*"};
-    
+
     public StopableSearchThread() {
       super("StopableSearchThread");
       setDaemon(true);
     }
-    
+
     @Override
     public void run() {
       Random random = random();
       int numSearches = 0;
-      
+
       while (true && !stop) {
         numSearches++;
         try {
@@ -1431,43 +1427,43 @@ public abstract class AbstractFullDistri
           Thread.currentThread().interrupt();
         }
       }
-      
+
       System.err.println("num searches done:" + numSearches + " with " + queryFails + " fails");
     }
-    
+
     @Override
     public void safeStop() {
       stop = true;
     }
-    
+
     public int getFails() {
       return queryFails.get();
     }
-    
+
   };
-  
+
   public void waitForThingsToLevelOut(int waitForRecTimeSeconds) throws Exception {
     log.info("Wait for recoveries to finish - wait " + waitForRecTimeSeconds + " for each attempt");
     int cnt = 0;
     boolean retry = false;
     do {
       waitForRecoveriesToFinish(VERBOSE, waitForRecTimeSeconds);
-      
+
       try {
         commit();
       } catch (Throwable t) {
         t.printStackTrace();
         // we don't care if this commit fails on some nodes
       }
-      
+
       updateMappingsFromZk(jettys, clients);
-      
+
       Set<String> theShards = shardToJetty.keySet();
       String failMessage = null;
       for (String shard : theShards) {
         failMessage = checkShardConsistency(shard, true, false);
       }
-      
+
       if (failMessage != null) {
         log.info("shard inconsistency - waiting ...");
         retry = true;
@@ -1475,12 +1471,12 @@ public abstract class AbstractFullDistri
         retry = false;
       }
       cnt++;
-      if (cnt > 20) break;
+      if (cnt > 30) break;
       Thread.sleep(2000);
     } while (retry);
   }
-  
-  
+
+
   public void waitForNoShardInconsistency() throws Exception {
     log.info("Wait for no shard inconsistency");
     int cnt = 0;
@@ -1492,15 +1488,20 @@ public abstract class AbstractFullDistri
         t.printStackTrace();
         // we don't care if this commit fails on some nodes
       }
-      
+
       updateMappingsFromZk(jettys, clients);
-      
+
       Set<String> theShards = shardToJetty.keySet();
       String failMessage = null;
       for (String shard : theShards) {
-        failMessage = checkShardConsistency(shard, true, false);
+        try {
+          failMessage = checkShardConsistency(shard, true, true);
+        } catch (Exception e) {
+          // we might hit a node we just stopped
+          failMessage="hit exception:" + e.getMessage();
+        }
       }
-      
+
       if (failMessage != null) {
         log.info("shard inconsistency - waiting ...");
         retry = true;
@@ -1508,7 +1509,7 @@ public abstract class AbstractFullDistri
         retry = false;
       }
       cnt++;
-      if (cnt > 20) break;
+      if (cnt > 40) break;
       Thread.sleep(2000);
     } while (retry);
   }
@@ -1526,42 +1527,41 @@ public abstract class AbstractFullDistri
   }
 
   @Override
-  @After
-  public void tearDown() throws Exception {
+  public void distribTearDown() throws Exception {
     if (VERBOSE || printLayoutOnTearDown) {
       super.printLayout();
     }
     if (commondCloudSolrClient != null) {
-      commondCloudSolrClient.shutdown();
+      commondCloudSolrClient.close();
     }
     if (controlClient != null) {
-      ((HttpSolrClient) controlClient).shutdown();
+      controlClient.close();
     }
     if (cloudClient != null) {
-      cloudClient.shutdown();
+      cloudClient.close();
     }
     if (controlClientCloud != null) {
-      controlClientCloud.shutdown();
+      controlClientCloud.close();
     }
-    super.tearDown();
-    
+    super.distribTearDown();
+
     System.clearProperty("zkHost");
     System.clearProperty("numShards");
 
-    // close socket proxies after super.tearDown
+    // close socket proxies after super.distribTearDown
     if (!proxies.isEmpty()) {
       for (SocketProxy proxy : proxies.values()) {
         proxy.close();
       }
     }
   }
-  
+
   @Override
   protected void commit() throws Exception {
     controlClient.commit();
     cloudClient.commit();
   }
-  
+
   @Override
   protected void destroyServers() throws Exception {
     if (controlJetty != null) {
@@ -1574,10 +1574,9 @@ public abstract class AbstractFullDistri
         log.error("", e);
       }
     }
-    clients.clear();
-    jettys.clear();
+    super.destroyServers();
   }
-  
+
   protected CollectionAdminResponse createCollection(String collectionName, int numShards, int replicationFactor, int maxShardsPerNode) throws SolrServerException, IOException {
     return createCollection(null, collectionName, numShards, replicationFactor, maxShardsPerNode, null, null);
   }
@@ -1615,9 +1614,9 @@ public abstract class AbstractFullDistri
       collectionInfos.put(collectionName, list);
     }
     params.set("name", collectionName);
-    if (getStateFormat() == 2) {
-      log.info("Creating collection with stateFormat=2: " + collectionName);
-      params.set(DocCollection.STATE_FORMAT, "2");
+    if ("1".equals(getStateFormat()) ) {
+      log.info("Creating collection with stateFormat=1: " + collectionName);
+      params.set(DocCollection.STATE_FORMAT, "1");
     }
     SolrRequest request = new QueryRequest(params);
     request.setPath("/admin/collections");
@@ -1625,11 +1624,8 @@ public abstract class AbstractFullDistri
     CollectionAdminResponse res = new CollectionAdminResponse();
     if (client == null) {
       final String baseUrl = getBaseUrl((HttpSolrClient) clients.get(clientIndex));
-      SolrClient adminClient = createNewSolrClient("", baseUrl);
-      try {
+      try (SolrClient adminClient = createNewSolrClient("", baseUrl)) {
         res.setResponse(adminClient.request(request));
-      } finally {
-        if (adminClient != null) adminClient.shutdown();
       }
     } else {
       res.setResponse(client.request(request));
@@ -1649,7 +1645,7 @@ public abstract class AbstractFullDistri
         MAX_SHARDS_PER_NODE, maxShardsPerNode),
         client);
   }
-  
+
   protected CollectionAdminResponse createCollection(Map<String, List<Integer>> collectionInfos,
                                                      String collectionName, int numShards, int replicationFactor, int maxShardsPerNode, SolrClient client, String createNodeSetStr, String configName) throws SolrServerException, IOException {
 
@@ -1698,7 +1694,7 @@ public abstract class AbstractFullDistri
         0, client.getBaseURL().length()
             - DEFAULT_COLLECTION.length() - 1);
   }
-  
+
   protected SolrInputDocument getDoc(Object... fields) throws Exception {
     SolrInputDocument doc = new SolrInputDocument();
     addFields(doc, fields);
@@ -1712,7 +1708,7 @@ public abstract class AbstractFullDistri
     // The Math.min thing is here, because we expect replication-factor to be reduced to if there are not enough live nodes to spread all shards of a collection over different nodes
     int expectedShardsPerSlice = numShardsNumReplicaList.get(1);
     int expectedTotalShards = expectedSlices * expectedShardsPerSlice;
-    
+
 //      Map<String,DocCollection> collections = clusterState
 //          .getCollectionStates();
       if (clusterState.hasCollection(collectionName)) {
@@ -1731,14 +1727,14 @@ public abstract class AbstractFullDistri
         totalShards += slices.get(sliceName).getReplicas().size();
       }
       if (totalShards != expectedTotalShards) {
-        return "Found new collection " + collectionName + " with correct number of slices, but mismatch on number of shards. Expected: " + expectedTotalShards + ", actual: " + totalShards; 
+        return "Found new collection " + collectionName + " with correct number of slices, but mismatch on number of shards. Expected: " + expectedTotalShards + ", actual: " + totalShards;
         }
       return null;
     } else {
       return "Could not find new collection " + collectionName;
     }
   }
-  
+
   protected void checkForCollection(String collectionName,
       List<Integer> numShardsNumReplicaList,
       List<String> nodesAllowedToRunShards) throws Exception {
@@ -1776,7 +1772,7 @@ public abstract class AbstractFullDistri
     }
     return commondCloudSolrClient;
   }
-  
+
   public static String getUrlFromZk(ClusterState clusterState, String collection) {
     Map<String,Slice> slices = clusterState.getCollection(collection).getSlicesMap();
 
@@ -1856,12 +1852,11 @@ public abstract class AbstractFullDistri
         .getBaseURL();
     baseUrl = baseUrl.substring(0, baseUrl.length() - "collection1".length());
 
-    HttpSolrClient baseClient = new HttpSolrClient(baseUrl);
-    baseClient.setConnectionTimeout(15000);
-    baseClient.setSoTimeout(60000 * 5);
-    NamedList r = baseClient.request(request);
-    baseClient.shutdown();
-    return r;
+    try (HttpSolrClient baseClient = new HttpSolrClient(baseUrl)) {
+      baseClient.setConnectionTimeout(15000);
+      baseClient.setSoTimeout(60000 * 5);
+      return baseClient.request(request);
+    }
   }
 
   protected void createCollection(String collName,
@@ -1881,9 +1876,9 @@ public abstract class AbstractFullDistri
 
   protected List<Replica> ensureAllReplicasAreActive(String testCollectionName, String shardId, int shards, int rf, int maxWaitSecs) throws Exception {
     long startMs = System.currentTimeMillis();
-    
+
     Map<String,Replica> notLeaders = new HashMap<String,Replica>();
-    
+
     ZkStateReader zkr = cloudClient.getZkStateReader();
     zkr.updateClusterState(true); // force the state to be fresh
 
@@ -1898,19 +1893,20 @@ public abstract class AbstractFullDistri
       // refresh state every 2 secs
       if (waitMs % 2000 == 0)
         cloudClient.getZkStateReader().updateClusterState(true);
-      
+
       cs = cloudClient.getZkStateReader().getClusterState();
       assertNotNull(cs);
       Slice shard = cs.getSlice(testCollectionName, shardId);
       assertNotNull("No Slice for "+shardId, shard);
       allReplicasUp = true; // assume true
       Collection<Replica> replicas = shard.getReplicas();
-      assertTrue(replicas.size() == rf);
+      assertTrue("Did not find correct number of replicas. Expected:" + rf + " Found:" + replicas.size(), replicas.size() == rf);
+      
       leader = shard.getLeader();
       assertNotNull(leader);
       log.info("Found "+replicas.size()+" replicas and leader on "+
         leader.getNodeName()+" for "+shardId+" in "+testCollectionName);
-      
+
       // ensure all replicas are "active" and identify the non-leader replica
       for (Replica replica : replicas) {
         String replicaState = replica.getStr(ZkStateReader.STATE_PROP);
@@ -1918,11 +1914,11 @@ public abstract class AbstractFullDistri
           log.info("Replica " + replica.getName() + " is currently " + replicaState);
           allReplicasUp = false;
         }
-        
-        if (!leader.equals(replica)) 
+
+        if (!leader.equals(replica))
           notLeaders.put(replica.getName(), replica);
       }
-      
+
       if (!allReplicasUp) {
         try {
           Thread.sleep(500L);
@@ -1930,22 +1926,22 @@ public abstract class AbstractFullDistri
         waitMs += 500L;
       }
     } // end while
-    
-    if (!allReplicasUp) 
+
+    if (!allReplicasUp)
       fail("Didn't see all replicas for shard "+shardId+" in "+testCollectionName+
           " come up within " + maxWaitMs + " ms! ClusterState: " + printClusterStateInfo());
-    
-    if (notLeaders.isEmpty()) 
+
+    if (notLeaders.isEmpty())
       fail("Didn't isolate any replicas that are not the leader! ClusterState: " + printClusterStateInfo());
-    
+
     long diffMs = (System.currentTimeMillis() - startMs);
     log.info("Took " + diffMs + " ms to see all replicas become active.");
-    
+
     List<Replica> replicas = new ArrayList<Replica>();
     replicas.addAll(notLeaders.values());
     return replicas;
-  }  
-  
+  }
+
   protected String printClusterStateInfo() throws Exception {
     return printClusterStateInfo(null);
   }

Modified: lucene/dev/branches/lucene6005/solr/test-framework/src/java/org/apache/solr/cloud/AbstractZkTestCase.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene6005/solr/test-framework/src/java/org/apache/solr/cloud/AbstractZkTestCase.java?rev=1658277&r1=1658276&r2=1658277&view=diff
==============================================================================
--- lucene/dev/branches/lucene6005/solr/test-framework/src/java/org/apache/solr/cloud/AbstractZkTestCase.java (original)
+++ lucene/dev/branches/lucene6005/solr/test-framework/src/java/org/apache/solr/cloud/AbstractZkTestCase.java Sun Feb  8 23:53:14 2015
@@ -17,10 +17,6 @@ package org.apache.solr.cloud;
  * limitations under the License.
  */
 
-import java.io.File;
-import java.util.HashMap;
-import java.util.Map;
-
 import org.apache.solr.SolrTestCaseJ4;
 import org.apache.solr.common.cloud.SolrZkClient;
 import org.apache.solr.common.cloud.ZkNodeProps;
@@ -31,6 +27,10 @@ import org.junit.BeforeClass;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
+import java.io.File;
+import java.util.HashMap;
+import java.util.Map;
+
 /**
  * Base test class for ZooKeeper tests.
  */
@@ -74,7 +74,7 @@ public abstract class AbstractZkTestCase
     
     buildZooKeeper(zkServer.getZkHost(), zkServer.getZkAddress(), SOLRHOME,
         "solrconfig.xml", "schema.xml");
-    
+
     initCore("solrconfig.xml", "schema.xml");
   }
 

Modified: lucene/dev/branches/lucene6005/solr/test-framework/src/java/org/apache/solr/cloud/ChaosMonkey.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene6005/solr/test-framework/src/java/org/apache/solr/cloud/ChaosMonkey.java?rev=1658277&r1=1658276&r2=1658277&view=diff
==============================================================================
--- lucene/dev/branches/lucene6005/solr/test-framework/src/java/org/apache/solr/cloud/ChaosMonkey.java (original)
+++ lucene/dev/branches/lucene6005/solr/test-framework/src/java/org/apache/solr/cloud/ChaosMonkey.java Sun Feb  8 23:53:14 2015
@@ -218,9 +218,11 @@ public class ChaosMonkey {
       if (filter != null) {
         CoreContainer cores = ((SolrDispatchFilter) filter).getCores();
         if (cores != null) {
-          int zklocalport = ((InetSocketAddress) cores.getZkController()
-              .getZkClient().getSolrZooKeeper().getSocketAddress()).getPort();
-          IpTables.blockPort(zklocalport);
+          if (cores.isZooKeeperAware()) {
+            int zklocalport = ((InetSocketAddress) cores.getZkController()
+                .getZkClient().getSolrZooKeeper().getSocketAddress()).getPort();
+            IpTables.blockPort(zklocalport);
+          }
         }
       }
     }
@@ -591,9 +593,11 @@ public class ChaosMonkey {
       if (filter != null) {
         CoreContainer cores = ((SolrDispatchFilter) filter).getCores();
         if (cores != null) {
-          int zklocalport = ((InetSocketAddress) cores.getZkController()
-              .getZkClient().getSolrZooKeeper().getSocketAddress()).getPort();
-          IpTables.unblockPort(zklocalport);
+          if (cores.isZooKeeperAware()) {
+            int zklocalport = ((InetSocketAddress) cores.getZkController()
+                .getZkClient().getSolrZooKeeper().getSocketAddress()).getPort();
+            IpTables.unblockPort(zklocalport);
+          }
         }
       }
     }

Modified: lucene/dev/branches/lucene6005/solr/test-framework/src/java/org/apache/solr/cloud/MiniSolrCloudCluster.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene6005/solr/test-framework/src/java/org/apache/solr/cloud/MiniSolrCloudCluster.java?rev=1658277&r1=1658276&r2=1658277&view=diff
==============================================================================
--- lucene/dev/branches/lucene6005/solr/test-framework/src/java/org/apache/solr/cloud/MiniSolrCloudCluster.java (original)
+++ lucene/dev/branches/lucene6005/solr/test-framework/src/java/org/apache/solr/cloud/MiniSolrCloudCluster.java Sun Feb  8 23:53:14 2015
@@ -17,16 +17,9 @@ package org.apache.solr.cloud;
  * limitations under the License.
  */
 
-import java.io.File;
-import java.io.IOException;
-import java.util.Collections;
-import java.util.LinkedList;
-import java.util.List;
-import java.util.Map;
-import java.util.SortedMap;
-
 import org.apache.solr.client.solrj.SolrServerException;
 import org.apache.solr.client.solrj.embedded.JettySolrRunner;
+import org.apache.solr.client.solrj.embedded.SSLConfig;
 import org.apache.solr.client.solrj.impl.CloudSolrClient;
 import org.apache.solr.client.solrj.request.QueryRequest;
 import org.apache.solr.common.cloud.SolrZkClient;
@@ -39,6 +32,14 @@ import org.eclipse.jetty.servlet.Servlet
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
+import java.io.File;
+import java.io.IOException;
+import java.util.Collections;
+import java.util.LinkedList;
+import java.util.List;
+import java.util.Map;
+import java.util.SortedMap;
+
 public class MiniSolrCloudCluster {
   
   private static Logger log = LoggerFactory.getLogger(MiniSolrCloudCluster.class);
@@ -60,6 +61,23 @@ public class MiniSolrCloudCluster {
   public MiniSolrCloudCluster(int numServers, String hostContext, File baseDir, File solrXml,
       SortedMap<ServletHolder, String> extraServlets,
       SortedMap<Class, String> extraRequestFilters) throws Exception {
+    this(numServers, hostContext, baseDir, solrXml, extraServlets, extraRequestFilters, null);
+  }
+
+  /**
+   * "Mini" SolrCloud cluster to be used for testing
+   * @param numServers number of Solr servers to start
+   * @param hostContext context path of Solr servers used by Jetty
+   * @param baseDir base directory that the mini cluster should be run from
+   * @param solrXml solr.xml file to be uploaded to ZooKeeper
+   * @param extraServlets Extra servlets to be started by Jetty
+   * @param extraRequestFilters extra filters to be started by Jetty
+   * @param sslConfig SSL configuration
+   */
+  public MiniSolrCloudCluster(int numServers, String hostContext, File baseDir, File solrXml,
+      SortedMap<ServletHolder, String> extraServlets,
+      SortedMap<Class, String> extraRequestFilters,
+      SSLConfig sslConfig) throws Exception {
     testDir = baseDir;
 
     String zkDir = testDir.getAbsolutePath() + File.separator
@@ -78,7 +96,11 @@ public class MiniSolrCloudCluster {
 
     jettys = new LinkedList<JettySolrRunner>();
     for (int i = 0; i < numServers; ++i) {
-      startJettySolrRunner(hostContext, extraServlets, extraRequestFilters);
+      if (sslConfig == null) {
+        startJettySolrRunner(hostContext, extraServlets, extraRequestFilters);
+      } else {
+        startJettySolrRunner(hostContext, extraServlets, extraRequestFilters, sslConfig);
+      }
     }
     
     solrClient = buildSolrClient();
@@ -108,9 +130,23 @@ public class MiniSolrCloudCluster {
   public JettySolrRunner startJettySolrRunner(String hostContext,
       SortedMap<ServletHolder, String> extraServlets,
       SortedMap<Class, String> extraRequestFilters) throws Exception {
+    return startJettySolrRunner(hostContext, extraServlets, extraRequestFilters, null);
+  }
+
+  /**
+   * Start a new Solr instance
+   * @param hostContext context path of Solr servers used by Jetty
+   * @param extraServlets Extra servlets to be started by Jetty
+   * @param extraRequestFilters extra filters to be started by Jetty
+   * @param sslConfig SSL configuration
+   * @return new Solr instance
+   */
+  public JettySolrRunner startJettySolrRunner(String hostContext,
+      SortedMap<ServletHolder, String> extraServlets,
+      SortedMap<Class, String> extraRequestFilters, SSLConfig sslConfig) throws Exception {
     String context = getHostContextSuitableForServletContext(hostContext);
-    JettySolrRunner jetty = new JettySolrRunner(testDir.getAbsolutePath(), context, 0, null, null,
-      true, extraServlets, null, extraRequestFilters);
+    JettySolrRunner jetty = new JettySolrRunner(testDir.getAbsolutePath(), context,
+      0, null, null, true, extraServlets, sslConfig, extraRequestFilters);
     jetty.start();
     jettys.add(jetty);
     return jetty;
@@ -160,7 +196,7 @@ public class MiniSolrCloudCluster {
    */
   public void shutdown() throws Exception {
     try {
-      solrClient.shutdown();
+      solrClient.close();
       for (int i = jettys.size() - 1; i >= 0; --i) {
         stopJettySolrRunner(i);
       }

Modified: lucene/dev/branches/lucene6005/solr/test-framework/src/java/org/apache/solr/cloud/ZkTestServer.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene6005/solr/test-framework/src/java/org/apache/solr/cloud/ZkTestServer.java?rev=1658277&r1=1658276&r2=1658277&view=diff
==============================================================================
--- lucene/dev/branches/lucene6005/solr/test-framework/src/java/org/apache/solr/cloud/ZkTestServer.java (original)
+++ lucene/dev/branches/lucene6005/solr/test-framework/src/java/org/apache/solr/cloud/ZkTestServer.java Sun Feb  8 23:53:14 2015
@@ -287,6 +287,11 @@ public class ZkTestServer {
     public void runFromConfig(ServerConfig config) throws IOException {
       log.info("Starting server");
       try {
+        // ZooKeeper maintains a static collection of AuthenticationProviders, so
+        // we make sure the SASL provider is loaded so that it can be used in
+        // subsequent tests.
+        System.setProperty("zookeeper.authProvider.1",
+          "org.apache.zookeeper.server.auth.SASLAuthenticationProvider");
         // Note that this thread isn't going to be doing anything else,
         // so rather than spawning another thread, we will just call
         // run() in this thread.
@@ -419,7 +424,7 @@ public class ZkTestServer {
 
   public void run() throws InterruptedException {
     log.info("STARTING ZK TEST SERVER");
-    // we don't call super.setUp
+    // we don't call super.distribSetUp
     zooThread = new Thread() {
       
       @Override

Modified: lucene/dev/branches/lucene6005/solr/test-framework/src/java/org/apache/solr/core/MockDirectoryFactory.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene6005/solr/test-framework/src/java/org/apache/solr/core/MockDirectoryFactory.java?rev=1658277&r1=1658276&r2=1658277&view=diff
==============================================================================
--- lucene/dev/branches/lucene6005/solr/test-framework/src/java/org/apache/solr/core/MockDirectoryFactory.java (original)
+++ lucene/dev/branches/lucene6005/solr/test-framework/src/java/org/apache/solr/core/MockDirectoryFactory.java Sun Feb  8 23:53:14 2015
@@ -25,7 +25,6 @@ import org.apache.lucene.store.LockFacto
 import org.apache.lucene.store.MockDirectoryWrapper;
 import org.apache.lucene.store.NRTCachingDirectory;
 import org.apache.lucene.store.NoLockFactory;
-import org.apache.lucene.store.RateLimitedDirectoryWrapper;
 import org.apache.lucene.store.TrackingDirectoryWrapper;
 import org.apache.lucene.util.LuceneTestCase;
 
@@ -86,9 +85,6 @@ public class MockDirectoryFactory extend
     if (dir instanceof NRTCachingDirectory) {
       cdir = ((NRTCachingDirectory)dir).getDelegate();
     }
-    if (cdir instanceof RateLimitedDirectoryWrapper) {
-      cdir = ((RateLimitedDirectoryWrapper)dir).getDelegate();
-    }
     if (cdir instanceof TrackingDirectoryWrapper) {
       cdir = ((TrackingDirectoryWrapper)dir).getDelegate();
     }

Modified: lucene/dev/branches/lucene6005/solr/test-framework/src/java/org/apache/solr/core/MockFSDirectoryFactory.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene6005/solr/test-framework/src/java/org/apache/solr/core/MockFSDirectoryFactory.java?rev=1658277&r1=1658276&r2=1658277&view=diff
==============================================================================
--- lucene/dev/branches/lucene6005/solr/test-framework/src/java/org/apache/solr/core/MockFSDirectoryFactory.java (original)
+++ lucene/dev/branches/lucene6005/solr/test-framework/src/java/org/apache/solr/core/MockFSDirectoryFactory.java Sun Feb  8 23:53:14 2015
@@ -25,7 +25,6 @@ import org.apache.lucene.store.Directory
 import org.apache.lucene.store.LockFactory;
 import org.apache.lucene.store.MockDirectoryWrapper;
 import org.apache.lucene.store.NRTCachingDirectory;
-import org.apache.lucene.store.RateLimitedDirectoryWrapper;
 import org.apache.lucene.store.TrackingDirectoryWrapper;
 import org.apache.lucene.util.LuceneTestCase;
 
@@ -69,9 +68,6 @@ public class MockFSDirectoryFactory exte
     if (dir instanceof NRTCachingDirectory) {
       cdir = ((NRTCachingDirectory)dir).getDelegate();
     }
-    if (cdir instanceof RateLimitedDirectoryWrapper) {
-      cdir = ((RateLimitedDirectoryWrapper)dir).getDelegate();
-    }
     if (cdir instanceof TrackingDirectoryWrapper) {
       cdir = ((TrackingDirectoryWrapper)dir).getDelegate();
     }

Modified: lucene/dev/branches/lucene6005/solr/test-framework/src/java/org/apache/solr/util/RestTestBase.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene6005/solr/test-framework/src/java/org/apache/solr/util/RestTestBase.java?rev=1658277&r1=1658276&r2=1658277&view=diff
==============================================================================
--- lucene/dev/branches/lucene6005/solr/test-framework/src/java/org/apache/solr/util/RestTestBase.java (original)
+++ lucene/dev/branches/lucene6005/solr/test-framework/src/java/org/apache/solr/util/RestTestBase.java Sun Feb  8 23:53:14 2015
@@ -16,12 +16,6 @@ package org.apache.solr.util;
  * limitations under the License.
  */
 
-import java.io.IOException;
-import java.util.Map;
-import java.util.SortedMap;
-
-import javax.xml.xpath.XPathExpressionException;
-
 import org.apache.solr.JSONTestUtil;
 import org.apache.solr.SolrJettyTestBase;
 import org.apache.solr.common.SolrException;
@@ -29,18 +23,25 @@ import org.apache.solr.common.params.Mul
 import org.apache.solr.common.util.StrUtils;
 import org.apache.solr.servlet.SolrRequestParsers;
 import org.eclipse.jetty.servlet.ServletHolder;
+import org.junit.AfterClass;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 import org.xml.sax.SAXException;
 
-import org.junit.AfterClass;
+import javax.xml.xpath.XPathExpressionException;
+import java.io.IOException;
+import java.util.Map;
+import java.util.SortedMap;
 
 abstract public class RestTestBase extends SolrJettyTestBase {
   private static final Logger log = LoggerFactory.getLogger(RestTestBase.class);
   protected static RestTestHarness restTestHarness;
 
   @AfterClass
-  public static void cleanUpHarness() {
+  public static void cleanUpHarness() throws IOException {
+    if (restTestHarness != null) {
+      restTestHarness.close();
+    }
     restTestHarness = null;
   }
 
@@ -53,7 +54,7 @@ abstract public class RestTestBase exten
     restTestHarness = new RestTestHarness(new RESTfulServerProvider() {
       @Override
       public String getBaseURL() {
-        return jetty.getBaseUrl().toString();
+        return jetty.getBaseUrl().toString() + "/" + DEFAULT_TEST_CORENAME;
       }
     });
   }

Modified: lucene/dev/branches/lucene6005/solr/test-framework/src/java/org/apache/solr/util/RestTestHarness.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene6005/solr/test-framework/src/java/org/apache/solr/util/RestTestHarness.java?rev=1658277&r1=1658276&r2=1658277&view=diff
==============================================================================
--- lucene/dev/branches/lucene6005/solr/test-framework/src/java/org/apache/solr/util/RestTestHarness.java (original)
+++ lucene/dev/branches/lucene6005/solr/test-framework/src/java/org/apache/solr/util/RestTestHarness.java Sun Feb  8 23:53:14 2015
@@ -16,15 +16,7 @@ package org.apache.solr.util;
  * limitations under the License.
  */
 
-import java.io.IOException;
-import java.net.URLEncoder;
-import java.nio.charset.StandardCharsets;
-
-import javax.xml.xpath.XPathConstants;
-import javax.xml.xpath.XPathExpressionException;
-
 import org.apache.http.HttpEntity;
-import org.apache.http.client.HttpClient;
 import org.apache.http.client.methods.HttpDelete;
 import org.apache.http.client.methods.HttpGet;
 import org.apache.http.client.methods.HttpPost;
@@ -32,16 +24,24 @@ import org.apache.http.client.methods.Ht
 import org.apache.http.client.methods.HttpUriRequest;
 import org.apache.http.entity.ContentType;
 import org.apache.http.entity.StringEntity;
+import org.apache.http.impl.client.CloseableHttpClient;
 import org.apache.http.util.EntityUtils;
 import org.apache.solr.client.solrj.impl.HttpClientUtil;
 import org.apache.solr.common.params.ModifiableSolrParams;
 
+import javax.xml.xpath.XPathConstants;
+import javax.xml.xpath.XPathExpressionException;
+import java.io.Closeable;
+import java.io.IOException;
+import java.net.URLEncoder;
+import java.nio.charset.StandardCharsets;
+
 /**
  * Facilitates testing Solr's REST API via a provided embedded Jetty
  */
-public class RestTestHarness extends BaseTestHarness {
+public class RestTestHarness extends BaseTestHarness implements Closeable {
   private RESTfulServerProvider serverProvider;
-  private HttpClient httpClient = HttpClientUtil.createClient(new
+  private CloseableHttpClient httpClient = HttpClientUtil.createClient(new
       ModifiableSolrParams());
   
   public RestTestHarness(RESTfulServerProvider serverProvider) {
@@ -51,6 +51,10 @@ public class RestTestHarness extends Bas
   public String getBaseURL() {
     return serverProvider.getBaseURL();
   }
+
+  public String getAdminURL() {
+    return getBaseURL().replace("/collection1", "");
+  }
   
   /**
    * Validates an XML "query" response against an array of XPath test strings
@@ -96,6 +100,10 @@ public class RestTestHarness extends Bas
     return getResponse(new HttpGet(getBaseURL() + request));
   }
 
+  public String adminQuery(String request) throws Exception {
+    return getResponse(new HttpGet(getAdminURL() + request));
+  }
+
   /**
    * Processes a PUT request using a URL path (with no context path) + optional query params,
    * e.g. "/schema/fields/newfield", PUTs the given content, and returns the response content.
@@ -151,17 +159,25 @@ public class RestTestHarness extends Bas
     }
   }
 
-
+  public String checkAdminResponseStatus(String xml, String code) throws Exception {
+    try {
+      String response = adminQuery(xml);
+      String valid = validateXPath(response, "//int[@name='status']="+code );
+      return (null == valid) ? null : response;
+    } catch (XPathExpressionException e) {
+      throw new RuntimeException("?!? static xpath has bug?", e);
+    }
+  }
   /**
    * Reloads the first core listed in the response to the core admin handler STATUS command
    */
   @Override
   public void reload() throws Exception {
     String coreName = (String)evaluateXPath
-        (query("/admin/cores?action=STATUS"),
+        (adminQuery("/admin/cores?action=STATUS"),
          "//lst[@name='status']/lst[1]/str[@name='name']",
          XPathConstants.STRING);
-    String xml = checkResponseStatus("/admin/cores?action=RELOAD&core=" + coreName, "0");
+    String xml = checkAdminResponseStatus("/admin/cores?action=RELOAD&core=" + coreName, "0");
     if (null != xml) {
       throw new RuntimeException("RELOAD failed:\n" + xml);
     }
@@ -195,4 +211,9 @@ public class RestTestHarness extends Bas
       EntityUtils.consumeQuietly(entity);
     }
   }
+
+  @Override
+  public void close() throws IOException {
+    httpClient.close();
+  }
 }

Modified: lucene/dev/branches/lucene6005/solr/test-framework/src/java/org/apache/solr/util/SSLTestConfig.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene6005/solr/test-framework/src/java/org/apache/solr/util/SSLTestConfig.java?rev=1658277&r1=1658276&r2=1658277&view=diff
==============================================================================
--- lucene/dev/branches/lucene6005/solr/test-framework/src/java/org/apache/solr/util/SSLTestConfig.java (original)
+++ lucene/dev/branches/lucene6005/solr/test-framework/src/java/org/apache/solr/util/SSLTestConfig.java Sun Feb  8 23:53:14 2015
@@ -39,7 +39,7 @@ import org.eclipse.jetty.util.security.C
 
 public class SSLTestConfig extends SSLConfig {
   public static File TEST_KEYSTORE = ExternalPaths.SERVER_HOME == null ? null
-      : new File(ExternalPaths.SERVER_HOME, "../etc/solrtest.keystore");
+      : new File(ExternalPaths.SERVER_HOME, "../etc/test/solrtest.keystore");
   
   private static String TEST_KEYSTORE_PATH = TEST_KEYSTORE != null
       && TEST_KEYSTORE.exists() ? TEST_KEYSTORE.getAbsolutePath() : null;
@@ -90,7 +90,7 @@ public class SSLTestConfig extends SSLCo
   
   private class SSLHttpClientConfigurer extends HttpClientConfigurer {
     @SuppressWarnings("deprecation")
-    protected void configure(DefaultHttpClient httpClient, SolrParams config) {
+    public void configure(DefaultHttpClient httpClient, SolrParams config) {
       super.configure(httpClient, config);
       SchemeRegistry registry = httpClient.getConnectionManager().getSchemeRegistry();
       // Make sure no tests cheat by using HTTP

Modified: lucene/dev/branches/lucene6005/solr/test-framework/src/java/org/apache/solr/util/TestHarness.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene6005/solr/test-framework/src/java/org/apache/solr/util/TestHarness.java?rev=1658277&r1=1658276&r2=1658277&view=diff
==============================================================================
--- lucene/dev/branches/lucene6005/solr/test-framework/src/java/org/apache/solr/util/TestHarness.java (original)
+++ lucene/dev/branches/lucene6005/solr/test-framework/src/java/org/apache/solr/util/TestHarness.java Sun Feb  8 23:53:14 2015
@@ -17,13 +17,17 @@
 
 package org.apache.solr.util;
 
+import com.google.common.collect.ImmutableList;
+import org.apache.solr.SolrTestCaseJ4;
 import org.apache.solr.common.SolrException;
 import org.apache.solr.common.params.CommonParams;
 import org.apache.solr.common.util.NamedList;
 import org.apache.solr.common.util.NamedList.NamedListEntry;
 import org.apache.solr.core.ConfigSolr;
-import org.apache.solr.core.ConfigSolrXmlOld;
 import org.apache.solr.core.CoreContainer;
+import org.apache.solr.core.CoreDescriptor;
+import org.apache.solr.core.CoresLocator;
+import org.apache.solr.core.PluginInfo;
 import org.apache.solr.core.SolrConfig;
 import org.apache.solr.core.SolrCore;
 import org.apache.solr.core.SolrResourceLoader;
@@ -42,6 +46,7 @@ import java.io.File;
 import java.io.IOException;
 import java.io.StringWriter;
 import java.util.HashMap;
+import java.util.List;
 import java.util.Map;
 
 /**
@@ -57,7 +62,7 @@ import java.util.Map;
  *
  */
 public class TestHarness extends BaseTestHarness {
-  String coreName;
+  public String coreName;
   protected volatile CoreContainer container;
   public UpdateRequestHandler updater;
  
@@ -78,11 +83,10 @@ public class TestHarness extends BaseTes
   }
   
   /**
-   * Creates a SolrConfig object for the 
-   * {@link ConfigSolrXmlOld#DEFAULT_DEFAULT_CORE_NAME} core using {@link #createConfig(String,String,String)}
+   * Creates a SolrConfig object for the default test core using {@link #createConfig(String,String,String)}
    */
   public static SolrConfig createConfig(String solrHome, String confFile) {
-    return createConfig(solrHome, ConfigSolrXmlOld.DEFAULT_DEFAULT_CORE_NAME, confFile);
+    return createConfig(solrHome, SolrTestCaseJ4.DEFAULT_TEST_CORENAME, confFile);
   }
 
   /**
@@ -116,7 +120,7 @@ public class TestHarness extends BaseTes
   public TestHarness( String dataDirectory,
                       SolrConfig solrConfig,
                       IndexSchema indexSchema) {
-      this(ConfigSolrXmlOld.DEFAULT_DEFAULT_CORE_NAME, dataDirectory, solrConfig, indexSchema);
+      this(SolrTestCaseJ4.DEFAULT_TEST_CORENAME, dataDirectory, solrConfig, indexSchema);
   }
 
   /**
@@ -126,21 +130,9 @@ public class TestHarness extends BaseTes
    * @param indexSchema schema resource name
    */
   public TestHarness(String coreName, String dataDir, String solrConfig, String indexSchema) {
-    try {
-      if (coreName == null)
-        coreName = ConfigSolrXmlOld.DEFAULT_DEFAULT_CORE_NAME;
-      this.coreName = coreName;
-
-      SolrResourceLoader loader = new SolrResourceLoader(SolrResourceLoader.locateSolrHome());
-      ConfigSolr config = getTestHarnessConfig(loader, coreName, dataDir, solrConfig, indexSchema);
-      container = new CoreContainer(loader, config);
-      container.load();
-
-      updater = new UpdateRequestHandler();
-      updater.init( null );
-    } catch (Exception e) {
-      throw new RuntimeException(e);
-    }
+    this(new TestConfigSolr(new SolrResourceLoader(SolrResourceLoader.locateSolrHome()),
+                                                    coreName, dataDir, solrConfig, indexSchema));
+    this.coreName = (coreName == null) ? SolrTestCaseJ4.DEFAULT_TEST_CORENAME : coreName;
   }
 
   public TestHarness(String coreName, String dataDir, SolrConfig solrConfig, IndexSchema indexSchema) {
@@ -162,38 +154,81 @@ public class TestHarness extends BaseTes
    * @param solrXml the text of a solrxml
    */
   public TestHarness(SolrResourceLoader loader, String solrXml) {
-    this(loader, ConfigSolr.fromString(loader, solrXml));
+    this(ConfigSolr.fromString(loader, solrXml));
   }
 
   /**
-   * Create a TestHarness using a specific resource loader and config
-   * @param loader the SolrResourceLoader to use
+   * Create a TestHarness using a specific config
    * @param config the ConfigSolr to use
    */
-  public TestHarness(SolrResourceLoader loader, ConfigSolr config) {
-    container = new CoreContainer(loader, config);
+  public TestHarness(ConfigSolr config) {
+    container = new CoreContainer(config);
     container.load();
     updater = new UpdateRequestHandler();
     updater.init(null);
   }
 
-  private static ConfigSolr getTestHarnessConfig(SolrResourceLoader loader, String coreName, String dataDir,
-                                                 String solrConfig, String schema) {
-    String solrxml = "<?xml version=\"1.0\" encoding=\"UTF-8\" ?>\n"
-        + "<solr persistent=\"false\">\n"
-        + "  <cores adminPath=\"/admin/cores\" defaultCoreName=\""
-        + ConfigSolrXmlOld.DEFAULT_DEFAULT_CORE_NAME
-        + "\""
-        + " host=\"${host:}\" hostPort=\"${hostPort:}\" hostContext=\"${hostContext:}\""
-        + " distribUpdateSoTimeout=\"30000\""
-        + " zkClientTimeout=\"${zkClientTimeout:30000}\" distribUpdateConnTimeout=\"30000\""
-        + ">\n"
-        + "    <core name=\"" + coreName + "\" config=\"" + solrConfig
-        + "\" schema=\"" + schema + "\" dataDir=\"" + dataDir
-        + "\" transient=\"false\" loadOnStartup=\"true\""
-        + " shard=\"${shard:shard1}\" collection=\"${collection:collection1}\" instanceDir=\"" + coreName + "/\" />\n"
-        + "  </cores>\n" + "</solr>";
-    return ConfigSolr.fromString(loader, solrxml);
+  public static class TestConfigSolr extends ConfigSolr {
+
+    final CoresLocator locator;
+
+    public TestConfigSolr(String coreName, String dataDir, String solrConfig, String schema) {
+      this(new SolrResourceLoader(SolrResourceLoader.locateSolrHome()), coreName, dataDir, solrConfig, schema);
+    }
+
+    public TestConfigSolr(SolrResourceLoader loader, String coreName, String dataDir, String solrConfig, String schema) {
+      super(loader);
+      locator = new TestCoresLocator(coreName, dataDir, solrConfig, schema);
+    }
+
+    @Override
+    public CoresLocator getCoresLocator() {
+      return locator;
+    }
+
+    @Override
+    public PluginInfo getShardHandlerFactoryPluginInfo() {
+      return null;
+    }
+
+    @Override
+    protected String getProperty(CfgProp key) {
+      switch (key) {
+        case SOLR_HOST: return System.getProperty("host");
+        case SOLR_HOSTPORT: return System.getProperty("hostPort", "");
+        case SOLR_HOSTCONTEXT: return System.getProperty("hostContext", "");
+        case SOLR_DISTRIBUPDATESOTIMEOUT: return "30000";
+        case SOLR_ZKCLIENTTIMEOUT: return System.getProperty("zkClientTimeout", "30000");
+        case SOLR_DISTRIBUPDATECONNTIMEOUT: return "30000";
+        case SOLR_SHARESCHEMA: return System.getProperty("shareSchema", "false");
+      }
+      return null;
+    }
+  }
+
+  public static class TestCoresLocator extends ReadOnlyCoresLocator {
+
+    final String coreName;
+    final String dataDir;
+    final String solrConfig;
+    final String schema;
+
+    public TestCoresLocator(String coreName, String dataDir, String solrConfig, String schema) {
+      this.coreName = coreName == null ? SolrTestCaseJ4.DEFAULT_TEST_CORENAME : coreName;
+      this.dataDir = dataDir;
+      this.schema = schema;
+      this.solrConfig = solrConfig;
+    }
+
+    @Override
+    public List<CoreDescriptor> discover(CoreContainer cc) {
+      return ImmutableList.of(new CoreDescriptor(cc, coreName, coreName,
+          CoreDescriptor.CORE_DATADIR, dataDir,
+          CoreDescriptor.CORE_CONFIG, solrConfig,
+          CoreDescriptor.CORE_SCHEMA, schema,
+          CoreDescriptor.CORE_COLLECTION, System.getProperty("collection", "collection1"),
+          CoreDescriptor.CORE_SHARD, System.getProperty("shard", "shard1")));
+    }
   }
   
   public CoreContainer getCoreContainer() {
@@ -289,7 +324,8 @@ public class TestHarness extends BaseTes
    * @see LocalSolrQueryRequest
    */
   public String query(String handler, SolrQueryRequest req) throws Exception {
-    try (SolrCore core = getCoreInc()) {
+    try {
+      SolrCore core = req.getCore();
       SolrQueryResponse rsp = new SolrQueryResponse();
       SolrRequestInfo.setRequestInfo(new SolrRequestInfo(req, rsp));
       core.execute(core.getRequestHandler(handler),req,rsp);

Modified: lucene/dev/branches/lucene6005/solr/webapp/build.xml
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene6005/solr/webapp/build.xml?rev=1658277&r1=1658276&r2=1658277&view=diff
==============================================================================
--- lucene/dev/branches/lucene6005/solr/webapp/build.xml (original)
+++ lucene/dev/branches/lucene6005/solr/webapp/build.xml Sun Feb  8 23:53:14 2015
@@ -40,7 +40,7 @@
   <target name="compile-core"/>
   <target name="compile-test"/>
 
-  <target name="dist"
+  <target name="server-war"
           description="Creates the Solr WAR Distribution file."
           depends="test, init-dist, dist-core, dist-solrj, lucene-jars-to-solr">
     <build-manifest title="Apache Solr Search Server"

Modified: lucene/dev/branches/lucene6005/solr/webapp/web/admin.html
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene6005/solr/webapp/web/admin.html?rev=1658277&r1=1658276&r2=1658277&view=diff
==============================================================================
--- lucene/dev/branches/lucene6005/solr/webapp/web/admin.html (original)
+++ lucene/dev/branches/lucene6005/solr/webapp/web/admin.html Sun Feb  8 23:53:14 2015
@@ -138,9 +138,9 @@ limitations under the License.
                     
           <li class="documentation"><a href="http://lucene.apache.org/solr/"><span>Documentation</span></a></li>
           <li class="issues"><a href="http://issues.apache.org/jira/browse/SOLR"><span>Issue Tracker</span></a></li>
-          <li class="irc"><a href="http://webchat.freenode.net/?channels=#solr"><span>IRC Channel</span></a></li>
-          <li class="mailinglist"><a href="http://wiki.apache.org/solr/UsingMailingLists"><span>Community forum</span></a></li>
-          <li class="wiki-query-syntax"><a href="http://wiki.apache.org/solr/SolrQuerySyntax"><span>Solr Query Syntax</span></a></li>
+          <li class="irc"><a href="https://wiki.apache.org/solr/IRCChannels"><span>IRC Channel</span></a></li>
+          <li class="mailinglist"><a href="http://lucene.apache.org/solr/resources.html#community"><span>Community forum</span></a></li>
+          <li class="wiki-query-syntax"><a href="https://cwiki.apache.org/confluence/display/solr/Query+Syntax+and+Parsing"><span>Solr Query Syntax</span></a></li>
                     
         </ul>