You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@lucene.apache.org by cm...@apache.org on 2013/08/11 14:19:39 UTC

svn commit: r1512909 [32/38] - in /lucene/dev/branches/lucene4956: ./ dev-tools/ dev-tools/eclipse/ dev-tools/idea/.idea/libraries/ dev-tools/idea/lucene/suggest/ dev-tools/idea/solr/contrib/dataimporthandler/ dev-tools/idea/solr/core/src/test/ dev-too...

Modified: lucene/dev/branches/lucene4956/solr/core/src/test/org/apache/solr/cloud/OverseerTest.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene4956/solr/core/src/test/org/apache/solr/cloud/OverseerTest.java?rev=1512909&r1=1512908&r2=1512909&view=diff
==============================================================================
--- lucene/dev/branches/lucene4956/solr/core/src/test/org/apache/solr/cloud/OverseerTest.java (original)
+++ lucene/dev/branches/lucene4956/solr/core/src/test/org/apache/solr/cloud/OverseerTest.java Sun Aug 11 12:19:13 2013
@@ -36,6 +36,7 @@ import org.apache.lucene.util.LuceneTest
 import org.apache.solr.SolrTestCaseJ4;
 import org.apache.solr.common.cloud.ClusterState;
 import org.apache.solr.common.cloud.DocCollection;
+import org.apache.solr.common.cloud.Replica;
 import org.apache.solr.common.cloud.Slice;
 import org.apache.solr.common.cloud.SolrZkClient;
 import org.apache.solr.common.cloud.ZkNodeProps;
@@ -98,7 +99,7 @@ public class OverseerTest extends SolrTe
       zkClient.close();
     }
     
-    public String publishState(String coreName, String stateName, int numShards)
+    public String publishState(String coreName, String coreNodeName, String stateName, int numShards)
         throws KeeperException, InterruptedException, IOException {
       if (stateName == null) {
         ElectionContext ec = electionContext.remove(coreName);
@@ -108,6 +109,7 @@ public class OverseerTest extends SolrTe
         ZkNodeProps m = new ZkNodeProps(Overseer.QUEUE_OPERATION, "deletecore",
             ZkStateReader.NODE_NAME_PROP, nodeName,
             ZkStateReader.CORE_NAME_PROP, coreName,
+            ZkStateReader.CORE_NODE_NAME_PROP, coreNodeName,
             ZkStateReader.COLLECTION_PROP, collection);
             DistributedQueue q = Overseer.getInQueue(zkClient);
             q.offer(ZkStateReader.toJSON(m));
@@ -117,6 +119,7 @@ public class OverseerTest extends SolrTe
         ZkStateReader.STATE_PROP, stateName,
         ZkStateReader.NODE_NAME_PROP, nodeName,
         ZkStateReader.CORE_NAME_PROP, coreName,
+        ZkStateReader.CORE_NODE_NAME_PROP, coreNodeName,
         ZkStateReader.COLLECTION_PROP, collection,
         ZkStateReader.NUM_SHARDS_PROP, Integer.toString(numShards),
         ZkStateReader.BASE_URL_PROP, "http://" + nodeName
@@ -126,7 +129,8 @@ public class OverseerTest extends SolrTe
       }
       
       for (int i = 0; i < 120; i++) {
-        String shardId = getShardId(coreName);
+        String shardId = getShardId("http://" + nodeName
+            + "/solr/", coreName);
         if (shardId != null) {
           try {
             zkClient.makePath("/collections/" + collection + "/leader_elect/"
@@ -136,7 +140,8 @@ public class OverseerTest extends SolrTe
               "http://" + nodeName + "/solr/", ZkStateReader.NODE_NAME_PROP,
               nodeName, ZkStateReader.CORE_NAME_PROP, coreName,
               ZkStateReader.SHARD_ID_PROP, shardId,
-              ZkStateReader.COLLECTION_PROP, collection);
+              ZkStateReader.COLLECTION_PROP, collection,
+              ZkStateReader.CORE_NODE_NAME_PROP, coreNodeName);
           ShardLeaderElectionContextBase ctx = new ShardLeaderElectionContextBase(
               elector, shardId, collection, nodeName + "_" + coreName, props,
               zkStateReader);
@@ -148,13 +153,18 @@ public class OverseerTest extends SolrTe
       return null;
     }
     
-    private String getShardId(final String coreName) {
+    private String getShardId(final String baseUrl, final String coreName) {
       Map<String,Slice> slices = zkStateReader.getClusterState().getSlicesMap(
           collection);
       if (slices != null) {
         for (Slice slice : slices.values()) {
-          if (slice.getReplicasMap().containsKey(nodeName + "_" + coreName)) {
-            return slice.getName();
+          for (Replica replica : slice.getReplicas()) {
+            // TODO: for really large clusters, we could 'index' on this
+            String rbaseUrl = replica.getStr(ZkStateReader.BASE_URL_PROP);
+            String rcore = replica.getStr(ZkStateReader.CORE_NAME_PROP);
+            if (baseUrl.equals(rbaseUrl) && coreName.equals(rcore)) {
+              return slice.getName();
+            }
           }
         }
       }
@@ -202,7 +212,7 @@ public class OverseerTest extends SolrTe
       final int numShards=6;
       
       for (int i = 0; i < numShards; i++) {
-        assertNotNull("shard got no id?", zkController.publishState("core" + (i+1), ZkStateReader.ACTIVE, 3));
+        assertNotNull("shard got no id?", zkController.publishState("core" + (i+1), "node" + (i+1), ZkStateReader.ACTIVE, 3));
       }
 
       assertEquals(2, reader.getClusterState().getSlice("collection1", "shard1").getReplicasMap().size());
@@ -277,7 +287,7 @@ public class OverseerTest extends SolrTe
             final String coreName = "core" + slot;
             
             try {
-              ids[slot]=controllers[slot % nodeCount].publishState(coreName, ZkStateReader.ACTIVE, sliceCount);
+              ids[slot]=controllers[slot % nodeCount].publishState(coreName, "node" + slot, ZkStateReader.ACTIVE, sliceCount);
             } catch (Throwable e) {
               e.printStackTrace();
               fail("register threw exception:" + e.getClass());
@@ -440,7 +450,7 @@ public class OverseerTest extends SolrTe
 
       assertEquals(reader.getClusterState().toString(), ZkStateReader.RECOVERING,
           reader.getClusterState().getSlice("collection1", "shard1").getReplicasMap()
-              .get("node1_core1").getStr(ZkStateReader.STATE_PROP));
+              .get("core_node1").getStr(ZkStateReader.STATE_PROP));
 
       //publish node state (active)
       m = new ZkNodeProps(Overseer.QUEUE_OPERATION, "state",
@@ -471,7 +481,7 @@ public class OverseerTest extends SolrTe
     while(maxIterations-->0) {
       Slice slice = reader.getClusterState().getSlice("collection1", "shard1");
       if(slice!=null) {
-        coreState = slice.getReplicasMap().get("node1_core1").getStr(ZkStateReader.STATE_PROP);
+        coreState = slice.getReplicasMap().get("core_node1").getStr(ZkStateReader.STATE_PROP);
         if(coreState.equals(expectedState)) {
           return;
         }
@@ -523,14 +533,14 @@ public class OverseerTest extends SolrTe
       overseerClient = electNewOverseer(server.getZkAddress());
 
       Thread.sleep(1000);
-      mockController.publishState("core1", ZkStateReader.RECOVERING, 1);
+      mockController.publishState("core1", "core_node1", ZkStateReader.RECOVERING, 1);
 
       waitForCollections(reader, "collection1");
       verifyStatus(reader, ZkStateReader.RECOVERING);
 
       int version = getClusterStateVersion(controllerClient);
       
-      mockController.publishState("core1", ZkStateReader.ACTIVE, 1);
+      mockController.publishState("core1", "core_node1", ZkStateReader.ACTIVE, 1);
       
       while(version == getClusterStateVersion(controllerClient));
 
@@ -539,7 +549,7 @@ public class OverseerTest extends SolrTe
       overseerClient.close();
       Thread.sleep(1000); //wait for overseer to get killed
 
-      mockController.publishState("core1", ZkStateReader.RECOVERING, 1);
+      mockController.publishState("core1",  "core_node1", ZkStateReader.RECOVERING, 1);
       version = getClusterStateVersion(controllerClient);
       
       overseerClient = electNewOverseer(server.getZkAddress());
@@ -553,7 +563,7 @@ public class OverseerTest extends SolrTe
       assertEquals("Shard count does not match", 1, reader.getClusterState()
           .getSlice("collection1", "shard1").getReplicasMap().size());
       version = getClusterStateVersion(controllerClient);
-      mockController.publishState("core1", null,1);
+      mockController.publishState("core1", "core_node1", null,1);
       while(version == getClusterStateVersion(controllerClient));
       Thread.sleep(500);
       assertFalse("collection1 should be gone after publishing the null state", reader.getClusterState().getCollections().contains("collection1"));
@@ -641,16 +651,16 @@ public class OverseerTest extends SolrTe
       for (int i = 0; i < atLeast(4); i++) {
         killCounter.incrementAndGet(); //for each round allow 1 kill
         mockController = new MockZKController(server.getZkAddress(), "node1", "collection1");
-        mockController.publishState("core1", "state1",1);
+        mockController.publishState("core1", "node1", "state1",1);
         if(mockController2!=null) {
           mockController2.close();
           mockController2 = null;
         }
-        mockController.publishState("core1", "state2",1);
+        mockController.publishState("core1", "node1","state2",1);
         mockController2 = new MockZKController(server.getZkAddress(), "node2", "collection1");
-        mockController.publishState("core1", "state1",1);
+        mockController.publishState("core1", "node1", "state1",1);
         verifyShardLeader(reader, "collection1", "shard1", "core1");
-        mockController2.publishState("core4", "state2" ,1);
+        mockController2.publishState("core4", "node2", "state2" ,1);
         mockController.close();
         mockController = null;
         verifyShardLeader(reader, "collection1", "shard1", "core4");
@@ -697,7 +707,7 @@ public class OverseerTest extends SolrTe
       
       overseerClient = electNewOverseer(server.getZkAddress());
 
-      mockController.publishState("core1", ZkStateReader.RECOVERING, 1);
+      mockController.publishState("core1", "core_node1", ZkStateReader.RECOVERING, 1);
 
       waitForCollections(reader, "collection1");
       
@@ -708,7 +718,7 @@ public class OverseerTest extends SolrTe
       int version = getClusterStateVersion(controllerClient);
       
       mockController = new MockZKController(server.getZkAddress(), "node1", "collection1");
-      mockController.publishState("core1", ZkStateReader.RECOVERING, 1);
+      mockController.publishState("core1", "core_node1", ZkStateReader.RECOVERING, 1);
 
       while (version == getClusterStateVersion(controllerClient));
       
@@ -718,7 +728,7 @@ public class OverseerTest extends SolrTe
       int numFound = 0;
       for (DocCollection collection : state.getCollectionStates().values()) {
         for (Slice slice : collection.getSlices()) {
-          if (slice.getReplicasMap().get("node1_core1") != null) {
+          if (slice.getReplicasMap().get("core_node1") != null) {
             numFound++;
           }
         }
@@ -761,7 +771,7 @@ public class OverseerTest extends SolrTe
       
       overseerClient = electNewOverseer(server.getZkAddress());
 
-      mockController.publishState("core1", ZkStateReader.RECOVERING, 12);
+      mockController.publishState("core1", "node1", ZkStateReader.RECOVERING, 12);
 
       waitForCollections(reader, "collection1");
       

Modified: lucene/dev/branches/lucene4956/solr/core/src/test/org/apache/solr/cloud/RecoveryZkTest.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene4956/solr/core/src/test/org/apache/solr/cloud/RecoveryZkTest.java?rev=1512909&r1=1512908&r2=1512909&view=diff
==============================================================================
--- lucene/dev/branches/lucene4956/solr/core/src/test/org/apache/solr/cloud/RecoveryZkTest.java (original)
+++ lucene/dev/branches/lucene4956/solr/core/src/test/org/apache/solr/cloud/RecoveryZkTest.java Sun Aug 11 12:19:13 2013
@@ -86,19 +86,25 @@ public class RecoveryZkTest extends Abst
     indexThread.join();
     indexThread2.join();
     
-    Thread.sleep(500);
+    Thread.sleep(1000);
   
-    waitForThingsToLevelOut(30);
+    waitForThingsToLevelOut(45);
     
     Thread.sleep(2000);
     
     waitForThingsToLevelOut(30);
     
+    Thread.sleep(5000);
+    
     waitForRecoveriesToFinish(DEFAULT_COLLECTION, zkStateReader, false, true);
 
     // test that leader and replica have same doc count
     
-    checkShardConsistency("shard1", false, false);
+    String fail = checkShardConsistency("shard1", false, false);
+    if (fail != null) {
+      fail(fail);
+    }
+    
     SolrQuery query = new SolrQuery("*:*");
     query.setParam("distrib", "false");
     long client1Docs = shardToJetty.get("shard1").get(0).client.solrClient.query(query).getResults().getNumFound();

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=1512909&r1=1512908&r2=1512909&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 Sun Aug 11 12:19:13 2013
@@ -17,6 +17,15 @@ package org.apache.solr.cloud;
  * limitations under the License.
  */
 
+import java.io.IOException;
+import java.net.MalformedURLException;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Map;
+import java.util.Random;
+import java.util.Set;
+
 import org.apache.http.params.CoreConnectionPNames;
 import org.apache.solr.client.solrj.SolrQuery;
 import org.apache.solr.client.solrj.SolrRequest;
@@ -28,10 +37,9 @@ 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.Replica;
 import org.apache.solr.common.cloud.Slice;
 import org.apache.solr.common.cloud.ZkCoreNodeProps;
 import org.apache.solr.common.cloud.ZkStateReader;
@@ -39,16 +47,9 @@ import org.apache.solr.common.params.Col
 import org.apache.solr.common.params.ModifiableSolrParams;
 import org.apache.solr.handler.admin.CollectionsHandler;
 import org.apache.solr.update.DirectUpdateHandler2;
-import org.apache.zookeeper.KeeperException;
 import org.junit.After;
 import org.junit.Before;
 
-import java.io.IOException;
-import java.net.MalformedURLException;
-import java.util.HashMap;
-import java.util.List;
-import java.util.Map;
-
 public class ShardSplitTest extends BasicDistributedZkTest {
 
   public static final String SHARD1_0 = SHARD1 + "_0";
@@ -110,34 +111,64 @@ public class ShardSplitTest extends Basi
     del("*:*");
     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));
+      indexAndUpdateCount(router, ranges, docCounts, shardKey + "!" + String.valueOf(id), id);
     }
     commit();
 
     Thread indexThread = new Thread() {
       @Override
       public void run() {
-        int max = atLeast(401);
+        Random random = random();
+        int max = atLeast(random, 401);
+        int sleep = atLeast(random, 25);
+        log.info("SHARDSPLITTEST: Going to add " + max + " number of docs at 1 doc per " + sleep + "ms");
+        Set<String> deleted = new HashSet<String>();
         for (int id = 101; id < max; id++) {
           try {
-            indexAndUpdateCount(router, ranges, docCounts, String.valueOf(id));
-            Thread.sleep(atLeast(25));
+            indexAndUpdateCount(router, ranges, docCounts, String.valueOf(id), id);
+            Thread.sleep(sleep);
+            if (usually(random))  {
+              String delId = String.valueOf(random.nextInt(id - 101 + 1) + 101);
+              if (deleted.contains(delId))  continue;
+              try {
+                deleteAndUpdateCount(router, ranges, docCounts, delId);
+                deleted.add(delId);
+              } catch (Exception e) {
+                log.error("Exception while deleting docs", e);
+              }
+            }
           } catch (Exception e) {
-            log.error("Exception while adding doc", e);
+            log.error("Exception while adding docs", e);
           }
         }
       }
     };
     indexThread.start();
 
-    splitShard(SHARD1);
-
-    log.info("Layout after split: \n");
-    printLayout();
-
-    indexThread.join();
-
-    commit();
+    try {
+      for (int i = 0; i < 3; i++) {
+        try {
+          splitShard(SHARD1);
+          log.info("Layout after split: \n");
+          printLayout();
+          break;
+        } catch (HttpSolrServer.RemoteSolrException e) {
+          if (e.code() != 500)  {
+            throw e;
+          }
+          log.error("SPLITSHARD failed. " + (i < 2 ? " Retring split" : ""), e);
+          if (i == 2) {
+            fail("SPLITSHARD was not successful even after three tries");
+          }
+        }
+      }
+    } finally {
+      try {
+        indexThread.join();
+      } catch (InterruptedException e) {
+        log.error("Indexing thread interrupted", e);
+      }
+    }
 
     checkDocCountsAndShardStates(docCounts, numReplicas);
 
@@ -147,25 +178,7 @@ public class ShardSplitTest extends Basi
     //waitForThingsToLevelOut(15);
   }
 
-  protected void checkDocCountsAndShardStates(int[] docCounts, int numReplicas) throws SolrServerException, KeeperException, InterruptedException {
-    SolrQuery query = new SolrQuery("*:*").setRows(1000).setFields("id", "_version_");
-    query.set("distrib", false);
-
-    ZkCoreNodeProps shard1_0 = getLeaderUrlFromZk(AbstractDistribZkTestBase.DEFAULT_COLLECTION, SHARD1_0);
-    HttpSolrServer shard1_0Server = new HttpSolrServer(shard1_0.getCoreUrl());
-    QueryResponse response = shard1_0Server.query(query);
-    long shard10Count = response.getResults().getNumFound();
-
-    ZkCoreNodeProps shard1_1 = getLeaderUrlFromZk(AbstractDistribZkTestBase.DEFAULT_COLLECTION, SHARD1_1);
-    HttpSolrServer shard1_1Server = new HttpSolrServer(shard1_1.getCoreUrl());
-    QueryResponse response2 = shard1_1Server.query(query);
-    long shard11Count = response2.getResults().getNumFound();
-
-    logDebugHelp(docCounts, response, shard10Count, response2, shard11Count);
-
-    assertEquals("Wrong doc count on shard1_0", docCounts[0], shard10Count);
-    assertEquals("Wrong doc count on shard1_1", docCounts[1], shard11Count);
-
+  protected void checkDocCountsAndShardStates(int[] docCounts, int numReplicas) throws Exception {
     ClusterState clusterState = null;
     Slice slice1_0 = null, slice1_1 = null;
     int i = 0;
@@ -188,6 +201,51 @@ public class ShardSplitTest extends Basi
     assertEquals("shard1_1 is not active", Slice.ACTIVE, slice1_1.getState());
     assertEquals("Wrong number of replicas created for shard1_0", numReplicas, slice1_0.getReplicas().size());
     assertEquals("Wrong number of replicas created for shard1_1", numReplicas, slice1_1.getReplicas().size());
+
+    commit();
+
+    // can't use checkShardConsistency because it insists on jettys and clients for each shard
+    checkSubShardConsistency(SHARD1_0);
+    checkSubShardConsistency(SHARD1_1);
+
+    SolrQuery query = new SolrQuery("*:*").setRows(1000).setFields("id", "_version_");
+    query.set("distrib", false);
+
+    ZkCoreNodeProps shard1_0 = getLeaderUrlFromZk(AbstractDistribZkTestBase.DEFAULT_COLLECTION, SHARD1_0);
+    HttpSolrServer shard1_0Server = new HttpSolrServer(shard1_0.getCoreUrl());
+    QueryResponse response = shard1_0Server.query(query);
+    long shard10Count = response.getResults().getNumFound();
+
+    ZkCoreNodeProps shard1_1 = getLeaderUrlFromZk(AbstractDistribZkTestBase.DEFAULT_COLLECTION, SHARD1_1);
+    HttpSolrServer shard1_1Server = new HttpSolrServer(shard1_1.getCoreUrl());
+    QueryResponse response2 = shard1_1Server.query(query);
+    long shard11Count = response2.getResults().getNumFound();
+
+    logDebugHelp(docCounts, response, shard10Count, response2, shard11Count);
+
+    assertEquals("Wrong doc count on shard1_0", docCounts[0], shard10Count);
+    assertEquals("Wrong doc count on shard1_1", docCounts[1], shard11Count);
+  }
+
+  protected void checkSubShardConsistency(String shard) throws SolrServerException {
+    SolrQuery query = new SolrQuery("*:*").setRows(1000).setFields("id", "_version_");
+    query.set("distrib", false);
+
+    ClusterState clusterState = cloudClient.getZkStateReader().getClusterState();
+    Slice slice = clusterState.getSlice(AbstractDistribZkTestBase.DEFAULT_COLLECTION, shard);
+    long[] numFound = new long[slice.getReplicasMap().size()];
+    int c = 0;
+    for (Replica replica : slice.getReplicas()) {
+      String coreUrl = new ZkCoreNodeProps(replica).getCoreUrl();
+      HttpSolrServer server = new HttpSolrServer(coreUrl);
+      QueryResponse response = server.query(query);
+      numFound[c++] = response.getResults().getNumFound();
+      log.info("Shard: " + shard + " Replica: {} has {} docs", coreUrl, String.valueOf(response.getResults().getNumFound()));
+      assertTrue("Shard: " + shard + " Replica: " + coreUrl + " has 0 docs", response.getResults().getNumFound() > 0);
+    }
+    for (int i = 0; i < slice.getReplicasMap().size(); i++) {
+      assertEquals(shard + " is not consistent", numFound[0], numFound[i]);
+    }
   }
 
   protected void splitShard(String shardId) throws SolrServerException, IOException {
@@ -208,9 +266,26 @@ public class ShardSplitTest extends Basi
     baseServer.request(request);
   }
 
-  protected void indexAndUpdateCount(DocRouter router, List<DocRouter.Range> ranges, int[] docCounts, String id) throws Exception {
-    index("id", id);
+  protected void indexAndUpdateCount(DocRouter router, List<DocRouter.Range> ranges, int[] docCounts, String id, int n) throws Exception {
+    index("id", id, "n_ti", n);
+
+    int idx = getHashRangeIdx(router, ranges, docCounts, id);
+    if (idx != -1)  {
+      docCounts[idx]++;
+    }
+  }
+
+  protected void deleteAndUpdateCount(DocRouter router, List<DocRouter.Range> ranges, int[] docCounts, String id) throws Exception {
+    controlClient.deleteById(id);
+    cloudClient.deleteById(id);
+
+    int idx = getHashRangeIdx(router, ranges, docCounts, id);
+    if (idx != -1)  {
+      docCounts[idx]--;
+    }
+  }
 
+  private int getHashRangeIdx(DocRouter router, List<DocRouter.Range> ranges, int[] docCounts, String id) {
     int hash = 0;
     if (router instanceof HashBasedRouter) {
       HashBasedRouter hashBasedRouter = (HashBasedRouter) router;
@@ -219,8 +294,9 @@ public class ShardSplitTest extends Basi
     for (int i = 0; i < ranges.size(); i++) {
       DocRouter.Range range = ranges.get(i);
       if (range.includes(hash))
-        docCounts[i]++;
+        return i;
     }
+    return -1;
   }
 
   protected void logDebugHelp(int[] docCounts, QueryResponse response, long shard10Count, QueryResponse response2, long shard11Count) {

Modified: lucene/dev/branches/lucene4956/solr/core/src/test/org/apache/solr/cloud/SliceStateUpdateTest.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene4956/solr/core/src/test/org/apache/solr/cloud/SliceStateUpdateTest.java?rev=1512909&r1=1512908&r2=1512909&view=diff
==============================================================================
--- lucene/dev/branches/lucene4956/solr/core/src/test/org/apache/solr/cloud/SliceStateUpdateTest.java (original)
+++ lucene/dev/branches/lucene4956/solr/core/src/test/org/apache/solr/cloud/SliceStateUpdateTest.java Sun Aug 11 12:19:13 2013
@@ -28,7 +28,6 @@ import org.apache.solr.common.cloud.Solr
 import org.apache.solr.common.cloud.ZkNodeProps;
 import org.apache.solr.common.cloud.ZkStateReader;
 import org.apache.solr.core.CoreContainer;
-import org.apache.solr.core.CoreContainer.Initializer;
 import org.apache.zookeeper.CreateMode;
 import org.junit.AfterClass;
 import org.junit.BeforeClass;
@@ -62,8 +61,6 @@ public class SliceStateUpdateTest extend
 
   private File dataDir3;
 
-  private Initializer init2;
-
   @BeforeClass
   public static void beforeClass() {
     System.setProperty("solrcloud.skip.autorecovery", "true");
@@ -117,20 +114,21 @@ public class SliceStateUpdateTest extend
 
     System.setProperty("solr.solr.home", TEST_HOME());
     System.setProperty("hostPort", "1661");
-    CoreContainer.Initializer init1 = new CoreContainer.Initializer();
     System.setProperty("solr.data.dir", SliceStateUpdateTest.this.dataDir1.getAbsolutePath());
-    container1 = init1.initialize();
+    container1 = new CoreContainer();
 
     System.clearProperty("hostPort");
 
     System.setProperty("hostPort", "1662");
-    init2 = new CoreContainer.Initializer();
     System.setProperty("solr.data.dir", SliceStateUpdateTest.this.dataDir2.getAbsolutePath());
-    container2 = init2.initialize();
+    container2 = new CoreContainer();
     System.clearProperty("hostPort");
 
     System.clearProperty("solr.solr.home");
 
+    container1.load();
+    container2.load();
+
     log.info("####SETUP_END " + getTestName());
 
   }

Modified: lucene/dev/branches/lucene4956/solr/core/src/test/org/apache/solr/cloud/SyncSliceTest.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene4956/solr/core/src/test/org/apache/solr/cloud/SyncSliceTest.java?rev=1512909&r1=1512908&r2=1512909&view=diff
==============================================================================
--- lucene/dev/branches/lucene4956/solr/core/src/test/org/apache/solr/cloud/SyncSliceTest.java (original)
+++ lucene/dev/branches/lucene4956/solr/core/src/test/org/apache/solr/cloud/SyncSliceTest.java Sun Aug 11 12:19:13 2013
@@ -67,7 +67,6 @@ public class SyncSliceTest extends Abstr
     super.setUp();
     // we expect this time of exception as shards go up and down...
     //ignoreException(".*");
-    useFactory(null);
     System.setProperty("numShards", Integer.toString(sliceCount));
   }
   
@@ -94,7 +93,7 @@ public class SyncSliceTest extends Abstr
     handle.put("QTime", SKIPVAL);
     handle.put("timestamp", SKIPVAL);
     
-    waitForThingsToLevelOut(15);
+    waitForThingsToLevelOut(30);
 
     del("*:*");
     List<CloudJettyRunner> skipServers = new ArrayList<CloudJettyRunner>();

Modified: lucene/dev/branches/lucene4956/solr/core/src/test/org/apache/solr/cloud/TestMultiCoreConfBootstrap.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene4956/solr/core/src/test/org/apache/solr/cloud/TestMultiCoreConfBootstrap.java?rev=1512909&r1=1512908&r2=1512909&view=diff
==============================================================================
--- lucene/dev/branches/lucene4956/solr/core/src/test/org/apache/solr/cloud/TestMultiCoreConfBootstrap.java (original)
+++ lucene/dev/branches/lucene4956/solr/core/src/test/org/apache/solr/cloud/TestMultiCoreConfBootstrap.java Sun Aug 11 12:19:13 2013
@@ -17,8 +17,6 @@
 
 package org.apache.solr.cloud;
 
-import java.io.File;
-
 import org.apache.solr.SolrTestCaseJ4;
 import org.apache.solr.common.cloud.SolrZkClient;
 import org.apache.solr.core.CoreContainer;
@@ -30,6 +28,8 @@ import org.junit.Test;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
+import java.io.File;
+
 public class TestMultiCoreConfBootstrap extends SolrTestCaseJ4 {
   protected static Logger log = LoggerFactory.getLogger(TestMultiCoreConfBootstrap.class);
   protected CoreContainer cores = null;
@@ -99,7 +99,7 @@ public class TestMultiCoreConfBootstrap 
   @Test
   public void testMultiCoreConfBootstrap() throws Exception {
     System.setProperty("bootstrap_conf", "true");
-    cores = new CoreContainer(home, new File(home, "solr.xml"));
+    cores = CoreContainer.createAndLoad(home, new File(home, "solr.xml"));
     SolrZkClient zkclient = cores.getZkController().getZkClient();
     // zkclient.printLayoutToStdOut();
     

Modified: lucene/dev/branches/lucene4956/solr/core/src/test/org/apache/solr/cloud/TestZkChroot.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene4956/solr/core/src/test/org/apache/solr/cloud/TestZkChroot.java?rev=1512909&r1=1512908&r2=1512909&view=diff
==============================================================================
--- lucene/dev/branches/lucene4956/solr/core/src/test/org/apache/solr/cloud/TestZkChroot.java (original)
+++ lucene/dev/branches/lucene4956/solr/core/src/test/org/apache/solr/cloud/TestZkChroot.java Sun Aug 11 12:19:13 2013
@@ -17,8 +17,6 @@ package org.apache.solr.cloud;
  * limitations under the License.
  */
 
-import java.io.File;
-
 import org.apache.solr.SolrTestCaseJ4;
 import org.apache.solr.common.cloud.SolrZkClient;
 import org.apache.solr.common.cloud.ZooKeeperException;
@@ -31,6 +29,8 @@ import org.junit.Test;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
+import java.io.File;
+
 public class TestZkChroot extends SolrTestCaseJ4 {
   protected static Logger log = LoggerFactory.getLogger(TestZkChroot.class);
   protected CoreContainer cores = null;
@@ -91,7 +91,7 @@ public class TestZkChroot extends SolrTe
     SolrZkClient zkClient2 = null;
     
     try {
-      cores = new CoreContainer(home, new File(home, "solr.xml"));
+      cores = CoreContainer.createAndLoad(home, new File(home, "solr.xml"));
       zkClient = cores.getZkController().getZkClient();
       
       assertTrue(zkClient.exists("/clusterstate.json", true));
@@ -122,7 +122,7 @@ public class TestZkChroot extends SolrTe
           AbstractZkTestCase.TIMEOUT);
       assertFalse("Path '" + chroot + "' should not exist before the test",
           zkClient.exists(chroot, true));
-      cores = new CoreContainer(home, new File(home, "solr.xml"));
+      cores = CoreContainer.createAndLoad(home, new File(home, "solr.xml"));
       fail("There should be a zk exception, as the initial path doesn't exist");
     } catch (ZooKeeperException e) {
       // expected
@@ -150,7 +150,7 @@ public class TestZkChroot extends SolrTe
           AbstractZkTestCase.TIMEOUT);
       assertFalse("Path '" + chroot + "' should not exist before the test",
           zkClient.exists(chroot, true));
-      cores = new CoreContainer(home, new File(home, "solr.xml"));
+      cores = CoreContainer.createAndLoad(home, new File(home, "solr.xml"));
       assertTrue(
           "solrconfig.xml should have been uploaded to zk to the correct config directory",
           zkClient.exists(chroot + ZkController.CONFIGS_ZKNODE + "/"
@@ -176,7 +176,7 @@ public class TestZkChroot extends SolrTe
       assertTrue(zkClient.exists(chroot, true));
       assertFalse(zkClient.exists(chroot + "/clusterstate.json", true));
       
-      cores = new CoreContainer(home, new File(home, "solr.xml"));
+      cores = CoreContainer.createAndLoad(home, new File(home, "solr.xml"));
       assertTrue(zkClient.exists(chroot + "/clusterstate.json", true));
     } finally {
       if (cores != null) cores.shutdown();

Modified: lucene/dev/branches/lucene4956/solr/core/src/test/org/apache/solr/cloud/UnloadDistributedZkTest.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene4956/solr/core/src/test/org/apache/solr/cloud/UnloadDistributedZkTest.java?rev=1512909&r1=1512908&r2=1512909&view=diff
==============================================================================
--- lucene/dev/branches/lucene4956/solr/core/src/test/org/apache/solr/cloud/UnloadDistributedZkTest.java (original)
+++ lucene/dev/branches/lucene4956/solr/core/src/test/org/apache/solr/cloud/UnloadDistributedZkTest.java Sun Aug 11 12:19:13 2013
@@ -91,7 +91,7 @@ public class UnloadDistributedZkTest ext
     createCmd.setCollection(collection);
     String coreDataDir = dataDir.getAbsolutePath() + File.separator
         + System.currentTimeMillis() + collection + "1";
-    createCmd.setDataDir(coreDataDir);
+    createCmd.setDataDir(getDataDir(coreDataDir));
     createCmd.setNumShards(2);
     
     SolrServer client = clients.get(0);
@@ -107,7 +107,7 @@ public class UnloadDistributedZkTest ext
     createCmd.setCollection(collection);
     coreDataDir = dataDir.getAbsolutePath() + File.separator
         + System.currentTimeMillis() + collection + "2";
-    createCmd.setDataDir(coreDataDir);
+    createCmd.setDataDir(getDataDir(coreDataDir));
     
     server.request(createCmd);
     
@@ -171,7 +171,7 @@ public class UnloadDistributedZkTest ext
     createCmd.setCollection("unloadcollection");
     createCmd.setNumShards(1);
     String core1DataDir = dataDir.getAbsolutePath() + File.separator + System.currentTimeMillis() + "unloadcollection1" + "_1n";
-    createCmd.setDataDir(core1DataDir);
+    createCmd.setDataDir(getDataDir(core1DataDir));
     server.request(createCmd);
     
     ZkStateReader zkStateReader = getCommonCloudSolrServer().getZkStateReader();
@@ -189,7 +189,7 @@ public class UnloadDistributedZkTest ext
     createCmd.setCoreName("unloadcollection2");
     createCmd.setCollection("unloadcollection");
     String core2dataDir = dataDir.getAbsolutePath() + File.separator + System.currentTimeMillis() + "unloadcollection1" + "_2n";
-    createCmd.setDataDir(core2dataDir);
+    createCmd.setDataDir(getDataDir(core2dataDir));
     server.request(createCmd);
     
     zkStateReader.updateClusterState(true);
@@ -227,7 +227,7 @@ public class UnloadDistributedZkTest ext
     createCmd.setCoreName("unloadcollection3");
     createCmd.setCollection("unloadcollection");
     String core3dataDir = dataDir.getAbsolutePath() + File.separator + System.currentTimeMillis() + "unloadcollection" + "_3n";
-    createCmd.setDataDir(core3dataDir);
+    createCmd.setDataDir(getDataDir(core3dataDir));
     server.request(createCmd);
     
     waitForRecoveriesToFinish("unloadcollection", zkStateReader, false);
@@ -296,7 +296,7 @@ public class UnloadDistributedZkTest ext
     createCmd.setCoreName("unloadcollection4");
     createCmd.setCollection("unloadcollection");
     String core4dataDir = dataDir.getAbsolutePath() + File.separator + System.currentTimeMillis() + "unloadcollection" + "_4n";
-    createCmd.setDataDir(core4dataDir);
+    createCmd.setDataDir(getDataDir(core4dataDir));
     server.request(createCmd);
     
     waitForRecoveriesToFinish("unloadcollection", zkStateReader, false);
@@ -334,7 +334,7 @@ public class UnloadDistributedZkTest ext
     createCmd = new Create();
     createCmd.setCoreName(leaderProps.getCoreName());
     createCmd.setCollection("unloadcollection");
-    createCmd.setDataDir(core1DataDir);
+    createCmd.setDataDir(getDataDir(core1DataDir));
     server.request(createCmd);
 
     waitForRecoveriesToFinish("unloadcollection", zkStateReader, false);

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=1512909&r1=1512908&r2=1512909&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 Sun Aug 11 12:19:13 2013
@@ -138,6 +138,19 @@ public class ZkCLITest extends SolrTestC
 
     assertTrue(zkClient.exists("/path/mynewpath", true));
   }
+
+  @Test
+  public void testPut() throws Exception {
+    // test bootstrap_conf
+    String data = "my data";
+    String[] args = new String[] {"-zkhost", zkServer.getZkAddress(), "-cmd",
+        "put", "/data.txt", data};
+    ZkCLI.main(args);
+
+    zkClient.getData("/data.txt", null, null, true);
+
+    assertArrayEquals(zkClient.getData("/data.txt", null, null, true), data.getBytes("UTF-8"));
+  }
   
   @Test
   public void testList() throws Exception {

Modified: lucene/dev/branches/lucene4956/solr/core/src/test/org/apache/solr/cloud/ZkControllerTest.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene4956/solr/core/src/test/org/apache/solr/cloud/ZkControllerTest.java?rev=1512909&r1=1512908&r2=1512909&view=diff
==============================================================================
--- lucene/dev/branches/lucene4956/solr/core/src/test/org/apache/solr/cloud/ZkControllerTest.java (original)
+++ lucene/dev/branches/lucene4956/solr/core/src/test/org/apache/solr/cloud/ZkControllerTest.java Sun Aug 11 12:19:13 2013
@@ -17,11 +17,6 @@ package org.apache.solr.cloud;
  * the License.
  */
 
-import java.io.File;
-import java.util.HashMap;
-import java.util.List;
-import java.util.Map;
-
 import org.apache.lucene.util.LuceneTestCase.Slow;
 import org.apache.solr.SolrTestCaseJ4;
 import org.apache.solr.common.cloud.SolrZkClient;
@@ -34,6 +29,11 @@ import org.apache.zookeeper.CreateMode;
 import org.junit.BeforeClass;
 import org.junit.Test;
 
+import java.io.File;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+
 @Slow
 public class ZkControllerTest extends SolrTestCaseJ4 {
 
@@ -163,7 +163,7 @@ public class ZkControllerTest extends So
       cc = getCoreContainer();
       
       ZkController zkController = new ZkController(cc, server.getZkAddress(), TIMEOUT, 10000,
-          "127.0.0.1", "8983", "solr", "0", 10000, 10000, new CurrentCoreDescriptorProvider() {
+          "127.0.0.1", "8983", "solr", 0, true, 10000, 10000, new CurrentCoreDescriptorProvider() {
             
             @Override
             public List<CoreDescriptor> getCurrentDescriptors() {
@@ -203,7 +203,7 @@ public class ZkControllerTest extends So
       cc = getCoreContainer();
       
       zkController = new ZkController(cc, server.getZkAddress(),
-          TIMEOUT, 10000, "127.0.0.1", "8983", "solr", "0", 10000, 10000, new CurrentCoreDescriptorProvider() {
+          TIMEOUT, 10000, "127.0.0.1", "8983", "solr", 0, true, 10000, 10000, new CurrentCoreDescriptorProvider() {
             
             @Override
             public List<CoreDescriptor> getCurrentDescriptors() {
@@ -240,12 +240,8 @@ public class ZkControllerTest extends So
   }
 
   private CoreContainer getCoreContainer() {
-    CoreContainer cc = new CoreContainer(TEMP_DIR.getAbsolutePath()) {
-      {
-        initShardHandler();
-      }
-    };
-    
+    CoreContainer cc = new CoreContainer(TEMP_DIR.getAbsolutePath());
+    cc.load();
     return cc;
   }
 

Modified: lucene/dev/branches/lucene4956/solr/core/src/test/org/apache/solr/core/AbstractBadConfigTestBase.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene4956/solr/core/src/test/org/apache/solr/core/AbstractBadConfigTestBase.java?rev=1512909&r1=1512908&r2=1512909&view=diff
==============================================================================
--- lucene/dev/branches/lucene4956/solr/core/src/test/org/apache/solr/core/AbstractBadConfigTestBase.java (original)
+++ lucene/dev/branches/lucene4956/solr/core/src/test/org/apache/solr/core/AbstractBadConfigTestBase.java Sun Aug 11 12:19:13 2013
@@ -18,9 +18,8 @@
 package org.apache.solr.core;
 
 import org.apache.solr.SolrTestCaseJ4;
-import org.apache.solr.common.SolrException;
-import org.apache.solr.common.SolrException.ErrorCode;
 
+import java.util.Map;
 import java.util.regex.Pattern;
 
 public abstract class AbstractBadConfigTestBase extends SolrTestCaseJ4 {
@@ -50,26 +49,37 @@ public abstract class AbstractBadConfigT
 
     ignoreException(Pattern.quote(errString));
     try {
+
       if (null == solrHome) {
         initCore( solrconfigFile, schemaFile );
       } else {
         initCore( solrconfigFile, schemaFile, solrHome );
       }
-    } catch (Exception e) {
-      for (Throwable t = e; t != null; t = t.getCause()) {
-        // short circuit out if we found what we expected
-        if (t.getMessage() != null && -1 != t.getMessage().indexOf(errString)) return;
-      }
 
-      // otherwise, rethrow it, possibly completley unrelated
-      throw new SolrException
-        (ErrorCode.SERVER_ERROR, 
-         "Unexpected error, expected error matching: " + errString, e);
-    } finally {
+      CoreContainer cc = h.getCoreContainer();
+      for (Map.Entry<String, Exception> entry : cc.getCoreInitFailures().entrySet()) {
+        if (matches(entry.getValue(), errString))
+          return;
+      }
+    }
+    catch (Exception e) {
+      if (matches(e, errString))
+        return;
+      throw e;
+    }
+    finally {
       deleteCore();
       resetExceptionIgnores();
     }
     fail("Did not encounter any exception from: " + solrconfigFile + " using " + schemaFile);
   }
 
+  private static boolean matches(Exception e, String errString) {
+    for (Throwable t = e; t != null; t = t.getCause()) {
+      if (t.getMessage() != null && -1 != t.getMessage().indexOf(errString))
+        return true;
+    }
+    return false;
+  }
+
 }

Modified: lucene/dev/branches/lucene4956/solr/core/src/test/org/apache/solr/core/AlternateDirectoryTest.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene4956/solr/core/src/test/org/apache/solr/core/AlternateDirectoryTest.java?rev=1512909&r1=1512908&r2=1512909&view=diff
==============================================================================
--- lucene/dev/branches/lucene4956/solr/core/src/test/org/apache/solr/core/AlternateDirectoryTest.java (original)
+++ lucene/dev/branches/lucene4956/solr/core/src/test/org/apache/solr/core/AlternateDirectoryTest.java Sun Aug 11 12:19:13 2013
@@ -25,21 +25,29 @@ import org.apache.solr.SolrTestCaseJ4;
 import org.junit.BeforeClass;
 import org.junit.Test;
 
+/**
+ * test that configs can override the DirectoryFactory and 
+ * IndexReaderFactory used in solr.
+ */
 public class AlternateDirectoryTest extends SolrTestCaseJ4 {
   @BeforeClass
   public static void beforeClass() throws Exception {
     initCore("solrconfig-altdirectory.xml", "schema.xml");
   }
 
-  /**
-   * Simple test to ensure that alternate IndexReaderFactory is being used.
-   */
-  @Test
   public void testAltDirectoryUsed() throws Exception {
     assertQ(req("q","*:*","qt","standard"));
     assertTrue(TestFSDirectoryFactory.openCalled);
     assertTrue(TestIndexReaderFactory.newReaderCalled);
   }
+  
+  public void testAltReaderUsed() throws Exception {
+    IndexReaderFactory readerFactory = h.getCore().getIndexReaderFactory();
+    assertNotNull("Factory is null", readerFactory);
+    assertEquals("readerFactory is wrong class",
+                 AlternateDirectoryTest.TestIndexReaderFactory.class.getName(), 
+                 readerFactory.getClass().getName());
+  }
 
   static public class TestFSDirectoryFactory extends StandardDirectoryFactory {
     public static volatile boolean openCalled = false;

Modified: lucene/dev/branches/lucene4956/solr/core/src/test/org/apache/solr/core/CoreContainerCoreInitFailuresTest.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene4956/solr/core/src/test/org/apache/solr/core/CoreContainerCoreInitFailuresTest.java?rev=1512909&r1=1512908&r2=1512909&view=diff
==============================================================================
--- lucene/dev/branches/lucene4956/solr/core/src/test/org/apache/solr/core/CoreContainerCoreInitFailuresTest.java (original)
+++ lucene/dev/branches/lucene4956/solr/core/src/test/org/apache/solr/core/CoreContainerCoreInitFailuresTest.java Sun Aug 11 12:19:13 2013
@@ -17,23 +17,17 @@
 
 package org.apache.solr.core;
 
-import java.util.Map;
-import java.util.Collection;
-import java.util.regex.Pattern;
-
-import java.io.File;
-
-import org.apache.solr.common.SolrException;
-import org.apache.solr.SolrTestCaseJ4;
-
-import org.apache.lucene.util.IOUtils;
-
 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.xml.sax.SAXParseException;
 
-import org.junit.Before;
-import org.junit.After;
+import java.io.File;
+import java.util.Collection;
+import java.util.Map;
+import java.util.regex.Pattern;
 
 public class CoreContainerCoreInitFailuresTest extends SolrTestCaseJ4 {
   
@@ -44,8 +38,7 @@ public class CoreContainerCoreInitFailur
     // would be nice to do this in an @Before method,
     // but junit doesn't let @Before methods have test names
     solrHome = new File(TEMP_DIR, this.getClass().getName() + "_" + dirSuffix);
-    assertTrue("Failed to mkdirs solrhome", solrHome.mkdirs());
-    cc = new CoreContainer(solrHome.getAbsolutePath());
+    assertTrue("Failed to mkdirs solrhome [" + solrHome + "]", solrHome.mkdirs());
   }
 
   @After
@@ -68,7 +61,7 @@ public class CoreContainerCoreInitFailur
     Map<String,Exception> failures = null;
     Collection<String> cores = null;
     Exception fail = null;
-    
+
     init("empty_flow");
 
     // solr.xml
@@ -77,7 +70,8 @@ public class CoreContainerCoreInitFailur
 
     // ----
     // init the CoreContainer
-    cc.load(solrHome.getAbsolutePath(), solrXml);
+    cc = new CoreContainer(solrHome.getAbsolutePath());
+    cc.load();
 
     // check that we have the cores we expect
     cores = cc.getCoreNames();
@@ -150,22 +144,23 @@ public class CoreContainerCoreInitFailur
     FileUtils.write(solrXml, BAD_SOLR_XML, IOUtils.CHARSET_UTF_8.toString());
 
     // our "ok" collection
-    FileUtils.copyFile(getFile("solr/collection1/conf/solrconfig-basic.xml"),
+    FileUtils.copyFile(getFile("solr/collection1/conf/solrconfig-defaults.xml"),
                        FileUtils.getFile(solrHome, "col_ok", "conf", "solrconfig.xml"));
     FileUtils.copyFile(getFile("solr/collection1/conf/schema-minimal.xml"),
                        FileUtils.getFile(solrHome, "col_ok", "conf", "schema.xml"));
-    
+
     // our "bad" collection
     ignoreException(Pattern.quote("DummyMergePolicy"));
     FileUtils.copyFile(getFile("solr/collection1/conf/bad-mp-solrconfig.xml"),
                        FileUtils.getFile(solrHome, "col_bad", "conf", "solrconfig.xml"));
     FileUtils.copyFile(getFile("solr/collection1/conf/schema-minimal.xml"),
                        FileUtils.getFile(solrHome, "col_bad", "conf", "schema.xml"));
-    
-    
+
+
     // -----
     // init the  CoreContainer with the mix of ok/bad cores
-    cc.load(solrHome.getAbsolutePath(), solrXml);
+    cc = new CoreContainer(solrHome.getAbsolutePath());
+    cc.load();
     
     // check that we have the cores we expect
     cores = cc.getCoreNames();
@@ -198,7 +193,7 @@ public class CoreContainerCoreInitFailur
 
     // -----
     // "fix" the bad collection
-    FileUtils.copyFile(getFile("solr/collection1/conf/solrconfig-basic.xml"),
+    FileUtils.copyFile(getFile("solr/collection1/conf/solrconfig-defaults.xml"),
                        FileUtils.getFile(solrHome, "col_bad", "conf", "solrconfig.xml"));
     final CoreDescriptor fixed = new CoreDescriptor(cc, "col_bad", "col_bad");
     cc.register("col_bad", cc.create(fixed), false);
@@ -293,8 +288,8 @@ public class CoreContainerCoreInitFailur
       fail("corrupt solrconfig.xml failed to trigger exception from reload");
     } catch (SolrException e) {
       assertTrue("We're supposed to have a wrapped SAXParserException here, but we don't",
-          e.getCause() instanceof SAXParseException);
-      SAXParseException se = (SAXParseException)e.getCause();
+          e.getCause().getCause() instanceof SAXParseException);
+      SAXParseException se = (SAXParseException)e.getCause().getCause();
       assertTrue("reload exception doesn't refer to slrconfig.xml " + se.getSystemId(),
           0 < se.getSystemId().indexOf("solrconfig.xml"));
 
@@ -318,13 +313,13 @@ public class CoreContainerCoreInitFailur
     fail = failures.get("col_bad");
     assertNotNull("null failure for test core", fail);
     assertTrue("init failure isn't SAXParseException",
-               fail instanceof SAXParseException);
+               fail.getCause() instanceof SAXParseException);
     assertTrue("init failure doesn't mention problem: " + fail.toString(),
-               0 < ((SAXParseException)fail).getSystemId().indexOf("solrconfig.xml"));
+               0 < ((SAXParseException)fail.getCause()).getSystemId().indexOf("solrconfig.xml"));
 
     // ----
     // fix col_bad's config (again) and RELOAD to fix failure
-    FileUtils.copyFile(getFile("solr/collection1/conf/solrconfig-basic.xml"),
+    FileUtils.copyFile(getFile("solr/collection1/conf/solrconfig-defaults.xml"),
                        FileUtils.getFile(solrHome, "col_bad", "conf", "solrconfig.xml"));
     cc.reload("col_bad");
     

Modified: lucene/dev/branches/lucene4956/solr/core/src/test/org/apache/solr/core/OpenCloseCoreStressTest.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene4956/solr/core/src/test/org/apache/solr/core/OpenCloseCoreStressTest.java?rev=1512909&r1=1512908&r2=1512909&view=diff
==============================================================================
--- lucene/dev/branches/lucene4956/solr/core/src/test/org/apache/solr/core/OpenCloseCoreStressTest.java (original)
+++ lucene/dev/branches/lucene4956/solr/core/src/test/org/apache/solr/core/OpenCloseCoreStressTest.java Sun Aug 11 12:19:13 2013
@@ -207,9 +207,10 @@ public class OpenCloseCoreStressTest ext
         }
       } while (secondsRemaining > 0);
 
-      assertTrue("We didn't index any documents, somethings really messsed up", cumulativeDocs > 0);
+      assertTrue("We didn't index any documents, somethings really messed up", cumulativeDocs > 0);
     } catch (Exception e) {
       e.printStackTrace();
+      fail("Caught unexpected exception");
     }
   }
 
@@ -241,6 +242,8 @@ public class OpenCloseCoreStressTest ext
     FileUtils.copyFile(new File(testConf, "schema-tiny.xml"), new File(conf, "schema-tiny.xml"));
 
     FileUtils.copyFile(new File(testConf, "solrconfig-minimal.xml"), new File(conf, "solrconfig-minimal.xml"));
+    FileUtils.copyFile(new File(testConf, "solrconfig.snippet.randomindexconfig.xml"),
+        new File(conf, "solrconfig.snippet.randomindexconfig.xml"));
 
     if (!oldStyle) {
       FileUtils.copyFile(new File(testSrcRoot, "conf/core.properties"), new File(coreDir, "core.properties"));
@@ -479,7 +482,7 @@ class Queries {
     try {
       QueryResponse response = server.query(params);
       numFound = response.getResults().getNumFound();
-    } catch (SolrServerException e) {
+    } catch (Exception e) {
       e.printStackTrace();
     }
     return numFound;

Modified: lucene/dev/branches/lucene4956/solr/core/src/test/org/apache/solr/core/QueryResultKeyTest.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene4956/solr/core/src/test/org/apache/solr/core/QueryResultKeyTest.java?rev=1512909&r1=1512908&r2=1512909&view=diff
==============================================================================
--- lucene/dev/branches/lucene4956/solr/core/src/test/org/apache/solr/core/QueryResultKeyTest.java (original)
+++ lucene/dev/branches/lucene4956/solr/core/src/test/org/apache/solr/core/QueryResultKeyTest.java Sun Aug 11 12:19:13 2013
@@ -38,7 +38,7 @@ public class QueryResultKeyTest extends 
     // the hashcode should be the same even when the list
     // of filters is in a different order
     
-    Sort sort = new Sort(new SortField("test", SortField.Type.BYTE));
+    Sort sort = new Sort(new SortField("test", SortField.Type.INT));
     List<Query> filters = new ArrayList<Query>();
     filters.add(new TermQuery(new Term("test", "field")));
     filters.add(new TermQuery(new Term("test2", "field2")));

Modified: lucene/dev/branches/lucene4956/solr/core/src/test/org/apache/solr/core/SolrCoreCheckLockOnStartupTest.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene4956/solr/core/src/test/org/apache/solr/core/SolrCoreCheckLockOnStartupTest.java?rev=1512909&r1=1512908&r2=1512909&view=diff
==============================================================================
--- lucene/dev/branches/lucene4956/solr/core/src/test/org/apache/solr/core/SolrCoreCheckLockOnStartupTest.java (original)
+++ lucene/dev/branches/lucene4956/solr/core/src/test/org/apache/solr/core/SolrCoreCheckLockOnStartupTest.java Sun Aug 11 12:19:13 2013
@@ -29,6 +29,7 @@ import org.junit.Before;
 import org.junit.Test;
 
 import java.io.File;
+import java.util.Map;
 
 public class SolrCoreCheckLockOnStartupTest extends SolrTestCaseJ4 {
 
@@ -59,17 +60,17 @@ public class SolrCoreCheckLockOnStartupT
     //creates a new IndexWriter without releasing the lock yet
     IndexWriter indexWriter = new IndexWriter(directory, new IndexWriterConfig(Version.LUCENE_40, null));
 
+    ignoreException("locked");
     try {
+      System.setProperty("solr.tests.lockType","simple");
       //opening a new core on the same index
-      initCore("solrconfig-simplelock.xml", "schema.xml");
+      initCore("solrconfig-basic.xml", "schema.xml");
+      if (checkForCoreInitException(LockObtainFailedException.class))
+        return;
       fail("Expected " + LockObtainFailedException.class.getSimpleName());
-    } catch (Throwable t) {
-      assertTrue(t instanceof RuntimeException);
-      assertNotNull(t.getCause());
-      assertTrue(t.getCause() instanceof RuntimeException);
-      assertNotNull(t.getCause().getCause());
-      assertTrue(t.getCause().getCause().toString(), t.getCause().getCause() instanceof LockObtainFailedException);
     } finally {
+      System.clearProperty("solr.tests.lockType");
+      unIgnoreException("locked");
       indexWriter.close();
       directory.close();
       deleteCore();
@@ -79,24 +80,37 @@ public class SolrCoreCheckLockOnStartupT
   @Test
   public void testNativeLockErrorOnStartup() throws Exception {
 
-    Directory directory = newFSDirectory(new File(dataDir, "index"), new NativeFSLockFactory());
+    File indexDir = new File(dataDir, "index");
+    log.info("Acquiring lock on {}", indexDir.getAbsolutePath());
+    Directory directory = newFSDirectory(indexDir, new NativeFSLockFactory());
     //creates a new IndexWriter without releasing the lock yet
     IndexWriter indexWriter = new IndexWriter(directory, new IndexWriterConfig(Version.LUCENE_40, null));
 
+    ignoreException("locked");
     try {
+      System.setProperty("solr.tests.lockType","native");
       //opening a new core on the same index
-      initCore("solrconfig-nativelock.xml", "schema.xml");
+      initCore("solrconfig-basic.xml", "schema.xml");
+      CoreContainer cc = h.getCoreContainer();
+      if (checkForCoreInitException(LockObtainFailedException.class))
+        return;
       fail("Expected " + LockObtainFailedException.class.getSimpleName());
-    } catch(Throwable t) {
-      assertTrue(t instanceof RuntimeException);
-      assertNotNull(t.getCause());
-      assertTrue(t.getCause() instanceof RuntimeException);
-      assertNotNull(t.getCause().getCause());
-      assertTrue(t.getCause().getCause() instanceof  LockObtainFailedException);
     } finally {
+      System.clearProperty("solr.tests.lockType");
+      unIgnoreException("locked");
       indexWriter.close();
       directory.close();
       deleteCore();
     }
   }
+
+  private boolean checkForCoreInitException(Class<? extends Exception> clazz) {
+    for (Map.Entry<String, Exception> entry : h.getCoreContainer().getCoreInitFailures().entrySet()) {
+      for (Throwable t = entry.getValue(); t != null; t = t.getCause()) {
+        if (clazz.isInstance(t))
+          return true;
+      }
+    }
+    return false;
+  }
 }

Modified: lucene/dev/branches/lucene4956/solr/core/src/test/org/apache/solr/core/TestArbitraryIndexDir.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene4956/solr/core/src/test/org/apache/solr/core/TestArbitraryIndexDir.java?rev=1512909&r1=1512908&r2=1512909&view=diff
==============================================================================
--- lucene/dev/branches/lucene4956/solr/core/src/test/org/apache/solr/core/TestArbitraryIndexDir.java (original)
+++ lucene/dev/branches/lucene4956/solr/core/src/test/org/apache/solr/core/TestArbitraryIndexDir.java Sun Aug 11 12:19:13 2013
@@ -19,6 +19,8 @@ package org.apache.solr.core;
 import java.io.File;
 import java.io.FileOutputStream;
 import java.io.IOException;
+import java.io.OutputStreamWriter;
+import java.io.Writer;
 import java.util.Properties;
 
 import javax.xml.parsers.ParserConfigurationException;
@@ -30,6 +32,7 @@ import org.apache.lucene.document.TextFi
 import org.apache.lucene.index.IndexWriter;
 import org.apache.lucene.index.IndexWriterConfig;
 import org.apache.lucene.store.Directory;
+import org.apache.lucene.util.IOUtils;
 import org.apache.lucene.util.Version;
 import org.apache.solr.common.SolrException;
 import org.apache.solr.common.params.CommonParams;
@@ -50,6 +53,7 @@ public class TestArbitraryIndexDir exten
   static String savedFactory;
   @BeforeClass
   public static void beforeClass() {
+    System.setProperty("enable.update.log", "false"); // schema12 doesn't support _version_
     savedFactory = System.getProperty("solr.DirectoryFactory");
     System.setProperty("solr.directoryFactory", "org.apache.solr.core.MockFSDirectoryFactory");
   }
@@ -96,15 +100,15 @@ public class TestArbitraryIndexDir exten
     File newDir = new File(h.getCore().getDataDir() + "index_temp");
     newDir.mkdirs();
     p.put("index", newDir.getName());
-    FileOutputStream os = null;
+    Writer os = null;
     try {
-      os = new FileOutputStream(idxprops);
+      os = new OutputStreamWriter(new FileOutputStream(idxprops), IOUtils.CHARSET_UTF_8);
       p.store(os, "index properties");
     } catch (Exception e) {
       throw new SolrException(SolrException.ErrorCode.SERVER_ERROR,
           "Unable to write " + SnapPuller.INDEX_PROPERTIES, e);
     } finally {
-      if (os != null) os.close();
+      IOUtils.closeWhileHandlingException(os);
     }
 
     //add a doc in the new index dir

Modified: lucene/dev/branches/lucene4956/solr/core/src/test/org/apache/solr/core/TestBadConfig.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene4956/solr/core/src/test/org/apache/solr/core/TestBadConfig.java?rev=1512909&r1=1512908&r2=1512909&view=diff
==============================================================================
--- lucene/dev/branches/lucene4956/solr/core/src/test/org/apache/solr/core/TestBadConfig.java (original)
+++ lucene/dev/branches/lucene4956/solr/core/src/test/org/apache/solr/core/TestBadConfig.java Sun Aug 11 12:19:13 2013
@@ -27,6 +27,15 @@ public class TestBadConfig extends Abstr
     assertConfigs("bad_solrconfig.xml","schema.xml","unset.sys.property");
   }
 
+  public void testMultipleIndexConfigs() throws Exception {
+      assertConfigs("bad-solrconfig-multiple-indexconfigs.xml", "schema12.xml",
+                    "indexConfig");
+  }
+  public void testMultipleCFS() throws Exception {
+      assertConfigs("bad-solrconfig-multiple-cfs.xml", "schema12.xml",
+                    "useCompoundFile");
+  }
+
   public void testUpdateLogButNoVersionField() throws Exception {
     
     System.setProperty("enable.update.log", "true");

Modified: lucene/dev/branches/lucene4956/solr/core/src/test/org/apache/solr/core/TestConfig.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene4956/solr/core/src/test/org/apache/solr/core/TestConfig.java?rev=1512909&r1=1512908&r2=1512909&view=diff
==============================================================================
--- lucene/dev/branches/lucene4956/solr/core/src/test/org/apache/solr/core/TestConfig.java (original)
+++ lucene/dev/branches/lucene4956/solr/core/src/test/org/apache/solr/core/TestConfig.java Sun Aug 11 12:19:13 2013
@@ -114,35 +114,30 @@ public class TestConfig extends SolrTest
     assertTrue(handler.getHiddenFiles().contains("PROTWORDS.TXT"));
   }
 
+  // If defaults change, add test methods to cover each version
   @Test
-  public void testTermIndexInterval() throws Exception {
-    RefCounted<IndexWriter> iw = ((DirectUpdateHandler2) h.getCore()
-        .getUpdateHandler()).getSolrCoreState().getIndexWriter(h.getCore());
-    int interval = 0;
-    try {
-      IndexWriter writer = iw.get();
-      interval = writer.getConfig().getTermIndexInterval();
-    } finally {
-      iw.decref();
-    }
-    assertEquals(256, interval);
-  }
+  public void testDefaults() throws Exception {
+
+    SolrConfig sc = new SolrConfig(new SolrResourceLoader("solr/collection1"), "solrconfig-defaults.xml", null);
+    SolrIndexConfig sic = sc.indexConfig;
+    assertEquals("default ramBufferSizeMB", 100.0D, sic.ramBufferSizeMB, 0.0D);
+    assertEquals("default LockType", SolrIndexConfig.LOCK_TYPE_NATIVE, sic.lockType);
+    assertEquals("default useCompoundFile", false, sic.useCompoundFile);
 
-  @Test
-  public void testTermIndexDivisor() throws Exception {
-    IndexReaderFactory irf = h.getCore().getIndexReaderFactory();
-    StandardIndexReaderFactory sirf = (StandardIndexReaderFactory) irf;
-    assertEquals(12, sirf.termInfosIndexDivisor);
   }
 
-  // If defaults change, add test methods to cover each version
-  @Test
-  public void testDefaults() throws Exception {
+
+  // sanity check that sys propertis are working as expected
+  public void testSanityCheckTestSysPropsAreUsed() throws Exception {
+
     SolrConfig sc = new SolrConfig(new SolrResourceLoader("solr/collection1"), "solrconfig-basic.xml", null);
     SolrIndexConfig sic = sc.indexConfig;
-    assertTrue("default ramBufferSizeMB should be 100", sic.ramBufferSizeMB == 100);
-    assertTrue("default useCompoundFile should be false", sic.useCompoundFile == false);
-    assertTrue("default LockType should be native", sic.lockType.equals(SolrIndexConfig.LOCK_TYPE_NATIVE));
+
+    assertEquals("ramBufferSizeMB sysprop", 
+                 Double.parseDouble(System.getProperty("solr.tests.ramBufferSizeMB")), 
+                                    sic.ramBufferSizeMB, 0.0D);
+    assertEquals("useCompoundFile sysprop", 
+                 Boolean.parseBoolean(System.getProperty("useCompoundFile")), sic.useCompoundFile);
   }
 
 }

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=1512909&r1=1512908&r2=1512909&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 Sun Aug 11 12:19:13 2013
@@ -17,6 +17,16 @@
 
 package org.apache.solr.core;
 
+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;
+import org.junit.Test;
+import org.xml.sax.SAXException;
+
+import javax.xml.parsers.ParserConfigurationException;
 import java.io.BufferedWriter;
 import java.io.File;
 import java.io.FileOutputStream;
@@ -27,17 +37,6 @@ 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;
-import org.junit.Test;
-import org.xml.sax.SAXException;
-
 public class TestCoreContainer extends SolrTestCaseJ4 {
 
   private static String oldSolrHome;
@@ -70,9 +69,10 @@ public class TestCoreContainer extends S
     assertTrue("Failed to mkdirs workDir", solrHomeDirectory.mkdirs());
 
     FileUtils.copyDirectory(new File(SolrTestCaseJ4.TEST_HOME()), solrHomeDirectory);
+    System.out.println("Using solrconfig from " + new File(SolrTestCaseJ4.TEST_HOME()).getAbsolutePath());
 
     CoreContainer ret = new CoreContainer(solrHomeDirectory.getAbsolutePath());
-    ret.load(solrHomeDirectory.getAbsolutePath(), new File(solrHomeDirectory, "solr.xml"));
+    ret.load();
     return ret;
   }
 
@@ -81,7 +81,6 @@ public class TestCoreContainer extends S
     System.setProperty("shareSchema", "true");
     final CoreContainer cores = init("_shareSchema");
     try {
-      cores.setPersistent(false);
       assertTrue(cores.isShareSchema());
       
       CoreDescriptor descriptor1 = new CoreDescriptor(cores, "core1", "./collection1");
@@ -143,118 +142,7 @@ public class TestCoreContainer extends S
 
   }
 
-  @Test
-  public void testPersist() throws Exception {
-    final File workDir = new File(TEMP_DIR, this.getClass().getName()
-        + "_persist");
-    if (workDir.exists()) {
-      FileUtils.deleteDirectory(workDir);
-    }
-    assertTrue("Failed to mkdirs workDir", workDir.mkdirs());
-    
-    final CoreContainer cores = h.getCoreContainer();
-
-    cores.setPersistent(true); // is this needed since we make explicit calls?
-
-    String instDir = null;
-    {
-      SolrCore template = null;
-      try {
-        template = cores.getCore("collection1");
-        instDir = template.getCoreDescriptor().getInstanceDir();
-      } finally {
-        if (null != template) template.close();
-      }
-    }
-    
-    final File instDirFile = new File(instDir);
-    assertTrue("instDir doesn't exist: " + instDir, instDirFile.exists());
-    
-    // sanity check the basic persistence of the default init
-    
-    final File oneXml = new File(workDir, "1.solr.xml");
-    cores.persistFile(oneXml);
-
-    assertXmlFile(oneXml, "/solr[@persistent='true']",
-        "/solr/cores[@defaultCoreName='collection1' and not(@transientCacheSize)]",
-        "/solr/cores/core[@name='collection1' and @instanceDir='" + instDir +
-        "' and @transient='false' and @loadOnStartup='true' ]", "1=count(/solr/cores/core)");
-
-    // create some new cores and sanity check the persistence
-    
-    final File dataXfile = new File(workDir, "dataX");
-    final String dataX = dataXfile.getAbsolutePath();
-    assertTrue("dataXfile mkdirs failed: " + dataX, dataXfile.mkdirs());
-    
-    final File instYfile = new File(workDir, "instY");
-    FileUtils.copyDirectory(instDirFile, instYfile);
-    
-    // :HACK: dataDir leaves off trailing "/", but instanceDir uses it
-    final String instY = instYfile.getAbsolutePath() + "/";
-    
-    final CoreDescriptor xd = new CoreDescriptor(cores, "X", instDir);
-    xd.setDataDir(dataX);
-    
-    final CoreDescriptor yd = new CoreDescriptor(cores, "Y", instY);
-    
-    SolrCore x = null;
-    SolrCore y = null;
-    try {
-      x = cores.create(xd);
-      y = cores.create(yd);
-      cores.register(x, false);
-      cores.register(y, false);
-      
-      assertEquals("cores not added?", 3, cores.getCoreNames().size());
-      
-      final File twoXml = new File(workDir, "2.solr.xml");
-
-      cores.persistFile(twoXml);
-
-      assertXmlFile(twoXml, "/solr[@persistent='true']",
-          "/solr/cores[@defaultCoreName='collection1']",
-          "/solr/cores/core[@name='collection1' and @instanceDir='" + instDir
-              + "']", "/solr/cores/core[@name='X' and @instanceDir='" + instDir
-              + "' and @dataDir='" + dataX + "']",
-          "/solr/cores/core[@name='Y' and @instanceDir='" + instY + "']",
-          "3=count(/solr/cores/core)");
 
-      // Test for saving implicit properties, we should not do this.
-      assertXmlFile(twoXml, "/solr/cores/core[@name='X' and not(@solr.core.instanceDir) and not (@solr.core.configName)]");
-
-      // delete a core, check persistence again
-      assertNotNull("removing X returned null", cores.remove("X"));
-      
-      final File threeXml = new File(workDir, "3.solr.xml");
-      cores.persistFile(threeXml);
-      
-      assertXmlFile(threeXml, "/solr[@persistent='true']",
-          "/solr/cores[@defaultCoreName='collection1']",
-          "/solr/cores/core[@name='collection1' and @instanceDir='" + instDir + "']",
-          "/solr/cores/core[@name='Y' and @instanceDir='" + instY + "']",
-          "2=count(/solr/cores/core)");
-      
-      // sanity check that persisting w/o changes has no changes
-      
-      final File fourXml = new File(workDir, "4.solr.xml");
-      cores.persistFile(fourXml);
-      
-      assertTrue("3 and 4 should be identical files",
-          FileUtils.contentEquals(threeXml, fourXml));
-      
-    } finally {
-      // y is closed by the container, but
-      // x has been removed from the container
-      if (x != null) {
-        try {
-          x.close();
-        } catch (Exception e) {
-          log.error("", e);
-        }
-      }
-    }
-  }
-  
 
   @Test
   public void testNoCores() throws IOException, ParserConfigurationException, SAXException {
@@ -262,14 +150,8 @@ public class TestCoreContainer extends S
     File solrHomeDirectory = new File(TEMP_DIR, this.getClass().getName()
         + "_noCores");
     SetUpHome(solrHomeDirectory, EMPTY_SOLR_XML);
-    CoreContainer.Initializer init = new CoreContainer.Initializer();
-    CoreContainer cores = null;
-    try {
-      cores = init.initialize();
-    }
-    catch(Exception e) {
-      fail("CoreContainer not created" + e.getMessage());
-    }
+    CoreContainer cores = new CoreContainer(solrHomeDirectory.getAbsolutePath());
+    cores.load();
     try {
       //assert zero cores
       assertEquals("There should not be cores", 0, cores.getCores().size());
@@ -322,7 +204,6 @@ public class TestCoreContainer extends S
   public void testClassLoaderHierarchy() throws Exception {
     final CoreContainer cc = init("_classLoaderHierarchy");
     try {
-      cc.setPersistent(false);
       ClassLoader sharedLoader = cc.loader.getClassLoader();
       ClassLoader contextLoader = Thread.currentThread().getContextClassLoader();
       assertSame(contextLoader, sharedLoader.getParent());
@@ -362,24 +243,21 @@ public class TestCoreContainer extends S
     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"));
+    final CoreContainer cc1 = CoreContainer.createAndLoad(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"));
+    final CoreContainer cc2 = CoreContainer.createAndLoad(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"));
+    final CoreContainer cc3 = CoreContainer.createAndLoad(tmpRoot.getAbsolutePath(), new File(tmpRoot, "custom-lib-solr.xml"));
     try {
       cc3.loader.openResource("customSharedLibFile").close();
     } finally {
@@ -392,22 +270,4 @@ public class TestCoreContainer extends S
       "  <cores adminPath=\"/admin/cores\" transientCacheSize=\"32\" >\n" +
       "  </cores>\n" +
       "</solr>";
-
-  private static final String SOLR_XML_SAME_NAME ="<?xml version=\"1.0\" encoding=\"UTF-8\" ?>\n" +
-      "<solr persistent=\"false\">\n" +
-      "  <cores adminPath=\"/admin/cores\" transientCacheSize=\"32\" >\n" +
-      "    <core name=\"core1\" instanceDir=\"core1\" dataDir=\"core1\"/> \n" +
-      "    <core name=\"core1\" instanceDir=\"core2\" dataDir=\"core2\"/> \n " +
-      "  </cores>\n" +
-      "</solr>";
-
-  private static final String SOLR_XML_SAME_DATADIR ="<?xml version=\"1.0\" encoding=\"UTF-8\" ?>\n" +
-      "<solr persistent=\"false\">\n" +
-      "  <cores adminPath=\"/admin/cores\" transientCacheSize=\"32\" >\n" +
-      "    <core name=\"core2\" instanceDir=\"core2\" dataDir=\"../samedatadir\" schema=\"schema-tiny.xml\" config=\"solrconfig-minimal.xml\" /> \n" +
-      "    <core name=\"core1\" instanceDir=\"core2\" dataDir=\"../samedatadir\" schema=\"schema-tiny.xml\" config=\"solrconfig-minimal.xml\"  /> \n " +
-      "  </cores>\n" +
-      "</solr>";
-
-
 }

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=1512909&r1=1512908&r2=1512909&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 Sun Aug 11 12:19:13 2013
@@ -17,10 +17,6 @@ package org.apache.solr.core;
  * limitations under the License.
  */
 
-import java.io.File;
-import java.io.FileOutputStream;
-import java.util.Properties;
-
 import org.apache.commons.io.FileUtils;
 import org.apache.lucene.util.IOUtils;
 import org.apache.solr.SolrTestCaseJ4;
@@ -29,6 +25,12 @@ import org.junit.After;
 import org.junit.BeforeClass;
 import org.junit.Test;
 
+import java.io.File;
+import java.io.FileOutputStream;
+import java.io.OutputStreamWriter;
+import java.io.Writer;
+import java.util.Properties;
+
 public class TestCoreDiscovery extends SolrTestCaseJ4 {
 
   @BeforeClass
@@ -78,7 +80,7 @@ public class TestCoreDiscovery extends S
 
   private void addCoreWithProps(Properties stockProps, File propFile) throws Exception {
     if (!propFile.getParentFile().exists()) propFile.getParentFile().mkdirs();
-    FileOutputStream out = new FileOutputStream(propFile);
+    Writer out = new OutputStreamWriter(new FileOutputStream(propFile), IOUtils.CHARSET_UTF_8);
     try {
       stockProps.store(out, null);
     } finally {
@@ -91,7 +93,7 @@ public class TestCoreDiscovery extends S
 
   private void addCoreWithProps(String name, Properties stockProps) throws Exception {
 
-    File propFile = new File(new File(solrHomeDirectory, name), SolrCoreDiscoverer.CORE_PROP_FILE);
+    File propFile = new File(new File(solrHomeDirectory, name), CorePropertiesLocator.PROPERTIES_FILENAME);
     File parent = propFile.getParentFile();
     assertTrue("Failed to mkdirs for " + parent.getAbsolutePath(), parent.mkdirs());
     addCoreWithProps(stockProps, propFile);
@@ -102,14 +104,12 @@ public class TestCoreDiscovery extends S
     assertTrue("Failed to mkdirs for " + confDir.getAbsolutePath(), confDir.mkdirs());
     FileUtils.copyFile(new File(top, "schema-tiny.xml"), new File(confDir, "schema-tiny.xml"));
     FileUtils.copyFile(new File(top, "solrconfig-minimal.xml"), new File(confDir, "solrconfig-minimal.xml"));
+    FileUtils.copyFile(new File(top, "solrconfig.snippet.randomindexconfig.xml"), new File(confDir, "solrconfig.snippet.randomindexconfig.xml"));
   }
 
   private CoreContainer init() throws Exception {
-
-    CoreContainer.Initializer init = new CoreContainer.Initializer();
-
-    final CoreContainer cores = init.initialize();
-    cores.setPersistent(false);
+    final CoreContainer cores = new CoreContainer();
+    cores.load();
     return cores;
   }
 
@@ -136,7 +136,7 @@ public class TestCoreDiscovery extends S
 
     CoreContainer cc = init();
     try {
-      assertEquals(CoreContainer.DEFAULT_DEFAULT_CORE_NAME,
+      assertEquals(ConfigSolrXmlOld.DEFAULT_DEFAULT_CORE_NAME,
                    cc.getDefaultCoreName());
 
       TestLazyCores.checkInCores(cc, "core1");
@@ -146,17 +146,17 @@ public class TestCoreDiscovery extends S
 
       // Let's assert we did the right thing for implicit properties too.
       CoreDescriptor desc = core1.getCoreDescriptor();
-      assertEquals("core1", desc.getProperty("solr.core.name"));
-
-      // Prove we're ignoring this even though it's set in the properties file
-      assertFalse("InstanceDir should be ignored", desc.getProperty("solr.core.instanceDir").contains("totallybogus"));
+      assertEquals("core1", desc.getName());
 
       // This is too long and ugly to put in. Besides, it varies.
-      assertNotNull(desc.getProperty("solr.core.instanceDir"));
+      assertNotNull(desc.getInstanceDir());
+
+      // Prove we're ignoring this even though it's set in the properties file
+      assertFalse("InstanceDir should be ignored", desc.getInstanceDir().contains("totallybogus"));
 
-      assertEquals("core1", desc.getProperty("solr.core.dataDir"));
-      assertEquals("solrconfig-minimal.xml", desc.getProperty("solr.core.configName"));
-      assertEquals("schema-tiny.xml", desc.getProperty("solr.core.schemaName"));
+      assertEquals("core1", desc.getDataDir());
+      assertEquals("solrconfig-minimal.xml", desc.getConfigName());
+      assertEquals("schema-tiny.xml", desc.getSchemaName());
 
       SolrCore core2 = cc.getCore("core2");
       SolrCore lazy1 = cc.getCore("lazy1");
@@ -182,10 +182,13 @@ public class TestCoreDiscovery extends S
       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);
+      String message = se.getMessage();
+      assertTrue("Wrong exception thrown on duplicate core names",
+          message.indexOf("Found multiple cores with the name [core1]") != -1);
+      assertTrue(File.separator + "core1 should have been mentioned in the message: " + message,
+          message.indexOf(File.separator + "core1") != -1);
+      assertTrue(File.separator + "core2 should have been mentioned in the message:" + message,
+          message.indexOf(File.separator + "core2") != -1);
     } finally {
       if (cc != null) {
         cc.shutdown();
@@ -201,9 +204,9 @@ public class TestCoreDiscovery extends S
     alt.mkdirs();
     setMeUp(alt.getAbsolutePath());
     addCoreWithProps(makeCorePropFile("core1", false, true, "dataDir=core1"),
-        new File(alt, "core1" + File.separator + SolrCoreDiscoverer.CORE_PROP_FILE));
+        new File(alt, "core1" + File.separator + CorePropertiesLocator.PROPERTIES_FILENAME));
     addCoreWithProps(makeCorePropFile("core2", false, false, "dataDir=core2"),
-        new File(alt, "core2" + File.separator + SolrCoreDiscoverer.CORE_PROP_FILE));
+        new File(alt, "core2" + File.separator + CorePropertiesLocator.PROPERTIES_FILENAME));
     CoreContainer cc = init();
     try {
       SolrCore core1 = cc.getCore("core1");
@@ -224,9 +227,9 @@ public class TestCoreDiscovery extends S
     noCoreDir.mkdirs();
     setMeUp(noCoreDir.getAbsolutePath());
     addCoreWithProps(makeCorePropFile("core1", false, true),
-        new File(noCoreDir, "core1" + File.separator + SolrCoreDiscoverer.CORE_PROP_FILE));
+        new File(noCoreDir, "core1" + File.separator + CorePropertiesLocator.PROPERTIES_FILENAME));
     addCoreWithProps(makeCorePropFile("core2", false, false),
-        new File(noCoreDir, "core2" + File.separator + SolrCoreDiscoverer.CORE_PROP_FILE));
+        new File(noCoreDir, "core2" + File.separator + CorePropertiesLocator.PROPERTIES_FILENAME));
     CoreContainer cc = init();
     try {
       SolrCore core1 = cc.getCore("core1");

Modified: lucene/dev/branches/lucene4956/solr/core/src/test/org/apache/solr/core/TestJmxIntegration.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene4956/solr/core/src/test/org/apache/solr/core/TestJmxIntegration.java?rev=1512909&r1=1512908&r2=1512909&view=diff
==============================================================================
--- lucene/dev/branches/lucene4956/solr/core/src/test/org/apache/solr/core/TestJmxIntegration.java (original)
+++ lucene/dev/branches/lucene4956/solr/core/src/test/org/apache/solr/core/TestJmxIntegration.java Sun Aug 11 12:19:13 2013
@@ -23,9 +23,8 @@ import javax.management.*;
 import org.apache.lucene.util.Constants;
 import org.apache.solr.core.JmxMonitoredMap.SolrDynamicMBean;
 import org.apache.solr.util.AbstractSolrTestCase;
-import org.junit.After;
 import org.junit.Assume;
-import org.junit.Before;
+import org.junit.AfterClass;
 import org.junit.BeforeClass;
 import org.junit.Ignore;
 import org.junit.Test;
@@ -38,59 +37,78 @@ import org.junit.Test;
  */
 public class TestJmxIntegration extends AbstractSolrTestCase {
 
-  
+  private static MBeanServer mbeanServer = null;
+
   @BeforeClass
   public static void beforeClass() throws Exception {
-    initCore("solrconfig.xml", "schema.xml");
-  }
-
-  @Override
-  @Before
-  public void setUp() throws Exception {
     // Make sure that at least one MBeanServer is available
-    MBeanServer mbeanServer = ManagementFactory.getPlatformMBeanServer();
-    super.setUp();
-  }
+    // prior to initializing the core
+    //
+    // (test configs are setup to use existing server if any, 
+    // otherwise skip JMX)
+    MBeanServer platformServer = ManagementFactory.getPlatformMBeanServer();
+
+    initCore("solrconfig.xml", "schema.xml");
 
-  @Override
-  @After
-  public void tearDown() throws Exception {
-    super.tearDown();
+    // we should be able to se that the core has JmxIntegration enabled
+    assertTrue("JMX not enabled",
+               h.getCore().getSolrConfig().jmxConfig.enabled);
+    // and we should be able to see that the the monitor map found 
+    // a JMX server to use, which refers to the server we started
+
+    Map registry = h.getCore().getInfoRegistry();
+    assertTrue("info registry is not a JMX monitored map",
+               registry instanceof JmxMonitoredMap);
+    mbeanServer = ((JmxMonitoredMap)registry).getServer();
+
+    assertNotNull("No JMX server found by monitor map",
+                  mbeanServer);
+
+    // NOTE: we can't garuntee that "mbeanServer == platformServer"
+    // the JVM may have mutiple MBean servers funning when the test started
+    // and the contract of not specifying one when configuring solr with
+    // <jmx /> is that it will use whatever the "first" MBean server 
+    // returned by the JVM is.
+  }
+
+  @AfterClass
+  public static void afterClass() throws Exception {
+    mbeanServer = null;
   }
 
   @Test
   public void testJmxRegistration() throws Exception {
-    List<MBeanServer> servers = MBeanServerFactory.findMBeanServer(null);
-    log.info("Servers in testJmxRegistration: " + servers);
-    assertNotNull("MBeanServers were null", servers);
-    assertFalse("No MBeanServer was found", servers.isEmpty());
-
-    MBeanServer mbeanServer = servers.get(0);
     assertTrue("No MBeans found in server", mbeanServer.getMBeanCount() > 0);
 
     Set<ObjectInstance> objects = mbeanServer.queryMBeans(null, null);
-    assertFalse("No SolrInfoMBean objects found in mbean server", objects
+    assertFalse("No objects found in mbean server", objects
             .isEmpty());
+    int numDynamicMbeans = 0;
     for (ObjectInstance o : objects) {
+      assertNotNull("Null name on: " + o.toString(), o.getObjectName());
       MBeanInfo mbeanInfo = mbeanServer.getMBeanInfo(o.getObjectName());
       if (mbeanInfo.getClassName().endsWith(SolrDynamicMBean.class.getName())) {
-        assertTrue("No Attributes found for mbean: " + mbeanInfo, mbeanInfo
-                .getAttributes().length > 0);
+        numDynamicMbeans++;
+        MBeanAttributeInfo[] attrs = mbeanInfo.getAttributes();
+        assertTrue("No Attributes found for mbean: " + mbeanInfo, 
+                   0 < attrs.length);
+        for (MBeanAttributeInfo attr : attrs) {
+          // ensure every advertised attribute is gettable
+          try {
+            Object trash = mbeanServer.getAttribute(o.getObjectName(), attr.getName());
+          } catch (javax.management.AttributeNotFoundException e) {
+            throw new RuntimeException("Unable to featch attribute for " + o.getObjectName()
+                                       + ": " + attr.getName(), e);
+          }
+        }
       }
     }
+    assertTrue("No SolrDynamicMBeans found", 0 < numDynamicMbeans);
   }
 
   @Test
   public void testJmxUpdate() throws Exception {
 
-    // Workaround for SOLR-4418 (this test fails with "No
-    // mbean found for SolrIndexSearcher" on IBM J9 6.0 and 7.0):
-    Assume.assumeTrue(!"IBM Corporation".equals(Constants.JVM_VENDOR));
-
-    List<MBeanServer> servers = MBeanServerFactory.findMBeanServer(null);
-    log.info("Servers in testJmxUpdate: " + servers);
-    log.info(h.getCore().getInfoRegistry().toString());
-
     SolrInfoMBean bean = null;
     // wait until searcher is registered
     for (int i=0; i<100; i++) {
@@ -100,7 +118,7 @@ public class TestJmxIntegration extends 
     }
     if (bean==null) throw new RuntimeException("searcher was never registered");
     ObjectName searcher = getObjectName("searcher", bean);
-    MBeanServer mbeanServer = servers.get(0);
+
     log.info("Mbeans in server: " + mbeanServer.queryNames(null, null));
 
     assertFalse("No mbean found for SolrIndexSearcher", mbeanServer.queryMBeans(searcher, null).isEmpty());
@@ -115,8 +133,6 @@ public class TestJmxIntegration extends 
 
   @Test @Ignore("timing problem? https://issues.apache.org/jira/browse/SOLR-2715")
   public void testJmxOnCoreReload() throws Exception {
-    List<MBeanServer> servers = MBeanServerFactory.findMBeanServer(null);
-    MBeanServer mbeanServer = servers.get(0);
 
     String coreName = h.getCore().getName();