You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@lucene.apache.org by ma...@apache.org on 2012/01/18 22:33:13 UTC

svn commit: r1233073 [2/2] - in /lucene/dev/branches/solrcloud/solr: core/src/java/org/apache/solr/client/solrj/embedded/ core/src/java/org/apache/solr/cloud/ core/src/java/org/apache/solr/core/ core/src/java/org/apache/solr/handler/ core/src/java/org/...

Modified: lucene/dev/branches/solrcloud/solr/core/src/test/org/apache/solr/cloud/FullSolrCloudTest.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/solrcloud/solr/core/src/test/org/apache/solr/cloud/FullSolrCloudTest.java?rev=1233073&r1=1233072&r2=1233073&view=diff
==============================================================================
--- lucene/dev/branches/solrcloud/solr/core/src/test/org/apache/solr/cloud/FullSolrCloudTest.java (original)
+++ lucene/dev/branches/solrcloud/solr/core/src/test/org/apache/solr/cloud/FullSolrCloudTest.java Wed Jan 18 21:33:12 2012
@@ -26,8 +26,6 @@ import java.util.HashMap;
 import java.util.List;
 import java.util.Map;
 import java.util.Set;
-import java.util.concurrent.CountDownLatch;
-import java.util.concurrent.TimeUnit;
 import java.util.concurrent.TimeoutException;
 import java.util.concurrent.atomic.AtomicInteger;
 
@@ -39,7 +37,6 @@ import org.apache.solr.client.solrj.impl
 import org.apache.solr.client.solrj.impl.CommonsHttpSolrServer;
 import org.apache.solr.client.solrj.request.UpdateRequest;
 import org.apache.solr.client.solrj.response.QueryResponse;
-import org.apache.solr.cloud.RecoveryStrat.RecoveryListener;
 import org.apache.solr.common.SolrException;
 import org.apache.solr.common.SolrInputDocument;
 import org.apache.solr.common.cloud.CloudState;
@@ -48,7 +45,6 @@ import org.apache.solr.common.cloud.ZkNo
 import org.apache.solr.common.cloud.ZkStateReader;
 import org.apache.solr.common.params.CommonParams;
 import org.apache.solr.common.params.ModifiableSolrParams;
-import org.apache.solr.servlet.SolrDispatchFilter;
 import org.apache.zookeeper.KeeperException;
 import org.junit.After;
 import org.junit.AfterClass;
@@ -161,8 +157,8 @@ public class FullSolrCloudTest extends A
   public FullSolrCloudTest() {
     fixShardCount = true;
     
-    shardCount = 6;
-    sliceCount = 3;
+    shardCount = atLeast(4);
+    sliceCount = atLeast(2);
     // TODO: for now, turn off stress because it uses regular clients, and we 
     // need the cloud client because we kill servers
     stress = 0;
@@ -492,7 +488,7 @@ public class FullSolrCloudTest extends A
     
     indexr("id", docId + 1, t1, "slip this doc in");
     
-    waitForRecovery(cloudJetty.jetty);
+    waitForRecoveriesToFinish(false);
     
     checkShardConsistency("shard1");
     
@@ -539,7 +535,7 @@ public class FullSolrCloudTest extends A
       SolrServerException {
     JettySolrRunner newReplica = createJettys(1).get(0);
     
-    waitForRecovery(newReplica);
+    waitForRecoveriesToFinish(false);
     
     // new server should be part of first shard
     // how many docs are on the new shard?
@@ -547,7 +543,7 @@ public class FullSolrCloudTest extends A
       if (VERBOSE) System.out.println("total:" + client.query(new SolrQuery("*:*")).getResults().getNumFound());
     }
 
-    checkShardConsistency(SHARD2);
+    checkShardConsistency("shard1");
 
     assertDocCounts(VERBOSE);
   }
@@ -580,7 +576,7 @@ public class FullSolrCloudTest extends A
           }
         }
       }
-      if (!sawLiveRecovering || cnt == 90) {
+      if (!sawLiveRecovering || cnt == 10) {
         if (!sawLiveRecovering) {
           if (VERBOSE) System.out.println("no one is recoverying");
         } else {
@@ -596,6 +592,10 @@ public class FullSolrCloudTest extends A
 
   private void brindDownShardIndexSomeDocsAndRecover() throws Exception,
       SolrServerException, IOException, InterruptedException {
+    
+    commit();
+    query("q", "*:*", "sort", "n_tl1 desc");
+    
     // kill a shard
     JettySolrRunner deadShard = chaosMonkey.stopShard(SHARD2, 0);
     
@@ -609,6 +609,12 @@ public class FullSolrCloudTest extends A
       // expected..
     }
     
+    commit();
+    query("q", "*:*", "sort", "n_tl1 desc");
+    
+//    long cloudClientDocs = cloudClient.query(new SolrQuery("*:*")).getResults().getNumFound();
+//    System.out.println("clouddocs:" + cloudClientDocs);
+    
     // try to index to a living shard at shard2
     // TODO: this can fail with connection refused !????
     index_specific(shardToClient.get(SHARD2).get(1), id, 1000, i1, 108, t1,
@@ -616,6 +622,8 @@ public class FullSolrCloudTest extends A
 
     commit();
     
+    checkShardConsistency(true, true);
+    
     query("q", "*:*", "sort", "n_tl1 desc");
     
     // try adding a doc with CloudSolrServer
@@ -665,7 +673,10 @@ public class FullSolrCloudTest extends A
 
     deadShard.start(true);
     
-    waitForRecovery(deadShard);
+    // make sure we have published we are recoverying
+    Thread.sleep(1500);
+    
+    waitForRecoveriesToFinish(false);
     
     List<SolrServer> s2c = shardToClient.get(SHARD2);
 
@@ -989,34 +1000,6 @@ public class FullSolrCloudTest extends A
     SolrQuery query = new SolrQuery("*:*");
     assertEquals("Doc Counts do not add up", controlCount, cloudClient.query(query).getResults().getNumFound());
   }
-  
-  protected void waitForRecovery(JettySolrRunner replica)
-      throws InterruptedException {
-    final CountDownLatch recoveryLatch = new CountDownLatch(1);
-    RecoveryStrat recoveryStrat = ((SolrDispatchFilter) replica.getDispatchFilter().getFilter()).getCores()
-        .getZkController().getRecoveryStrat();
-    
-    recoveryStrat.setRecoveryListener(new RecoveryListener() {
-      
-      @Override
-      public void startRecovery() {}
-      
-      @Override
-      public void finishedReplication() {}
-      
-      @Override
-      public void finishedRecovery() {
-        recoveryLatch.countDown();
-      }
-    });
-    
-    // wait for recovery to finish
-    // if it takes over n seconds, assume we didnt get our listener attached before
-    // recover started - it should be done before n though
-    if (!recoveryLatch.await(45, TimeUnit.SECONDS)) {
-      log.warn("Timed out waiting to be notified of replication");
-    }
-  }
 
   @Override
   protected QueryResponse queryServer(ModifiableSolrParams params) throws SolrServerException {  
@@ -1099,6 +1082,7 @@ public class FullSolrCloudTest extends A
     if (VERBOSE) {
       super.printLayout();
     }
+    ((CommonsHttpSolrServer) controlClient).shutdown();
     if (cloudClient != null) {
       cloudClient.close();
     }
@@ -1134,6 +1118,7 @@ public class FullSolrCloudTest extends A
       String url = "http://localhost:" + port + context + "/" + DEFAULT_COLLECTION;
       CommonsHttpSolrServer s = new CommonsHttpSolrServer(url);
       s.setConnectionTimeout(100); // 1/10th sec
+      s.setSoTimeout(15000);
       s.setDefaultMaxConnectionsPerHost(100);
       s.setMaxTotalConnections(100);
       return s;

Modified: lucene/dev/branches/solrcloud/solr/core/src/test/org/apache/solr/cloud/LeaderElectionIntegrationTest.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/solrcloud/solr/core/src/test/org/apache/solr/cloud/LeaderElectionIntegrationTest.java?rev=1233073&r1=1233072&r2=1233073&view=diff
==============================================================================
--- lucene/dev/branches/solrcloud/solr/core/src/test/org/apache/solr/cloud/LeaderElectionIntegrationTest.java (original)
+++ lucene/dev/branches/solrcloud/solr/core/src/test/org/apache/solr/cloud/LeaderElectionIntegrationTest.java Wed Jan 18 21:33:12 2012
@@ -160,6 +160,13 @@ public class LeaderElectionIntegrationTe
       
       leader = getLeader();
       int newLeaderPort = getLeaderPort(leader);
+      int retry = 0;
+      while (leaderPort == newLeaderPort) {
+        if (retry++ == 20) {
+          break;
+        }
+        Thread.sleep(1000);
+      }
       
       if (leaderPort == newLeaderPort) {
         fail("We didn't find a new leader! " + leaderPort + " was shutdown, but it's still showing as the leader");
@@ -216,7 +223,7 @@ public class LeaderElectionIntegrationTe
       ZkNodeProps props;
       try {
         reader.updateCloudState(true);
-        props = reader.getLeaderProps("collection1", "shard1");
+        props = reader.getLeaderProps("collection1", "shard1", 500);
         leader = props.get(ZkStateReader.NODE_NAME_PROP);
         if (leader != null) {
           break;

Modified: lucene/dev/branches/solrcloud/solr/core/src/test/org/apache/solr/cloud/LeaderElectionTest.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/solrcloud/solr/core/src/test/org/apache/solr/cloud/LeaderElectionTest.java?rev=1233073&r1=1233072&r2=1233073&view=diff
==============================================================================
--- lucene/dev/branches/solrcloud/solr/core/src/test/org/apache/solr/cloud/LeaderElectionTest.java (original)
+++ lucene/dev/branches/solrcloud/solr/core/src/test/org/apache/solr/cloud/LeaderElectionTest.java Wed Jan 18 21:33:12 2012
@@ -99,12 +99,12 @@ public class LeaderElectionTest extends 
       
         LeaderElector elector = new LeaderElector(ClientThread.this.zkClient);
         
-        ElectionContext context = new ShardLeaderElectionContext(null, "shard1",
+        ElectionContext context = new ShardLeaderElectionContextBase(elector, "shard1",
             "collection1", Integer.toString(nodeNumber), props, this.zkStateReader);
         
         try {
           elector.setup(context);
-          seq = elector.joinElection(context);
+          seq = elector.joinElection(context, null);
           electionDone = true;
           seqToThread.put(seq, this);
         } catch (InterruptedException e) {
@@ -142,11 +142,14 @@ public class LeaderElectionTest extends 
   @Test
   public void testBasic() throws Exception {
     LeaderElector elector = new LeaderElector(zkClient);
-    ZkNodeProps props = new ZkNodeProps(ZkStateReader.BASE_URL_PROP, "http://127.0.0.1/solr/", ZkStateReader.CORE_PROP, "");
-    ElectionContext context = new ShardLeaderElectionContext(null, "shard2", "collection1", "dummynode1", props, zkStateReader);
+    ZkNodeProps props = new ZkNodeProps(ZkStateReader.BASE_URL_PROP,
+        "http://127.0.0.1/solr/", ZkStateReader.CORE_PROP, "");
+    ElectionContext context = new ShardLeaderElectionContextBase(elector,
+        "shard2", "collection1", "dummynode1", props, zkStateReader);
     elector.setup(context);
-    elector.joinElection(context);
-    assertEquals("http://127.0.0.1/solr/", getLeaderUrl("collection1", "shard2"));
+    elector.joinElection(context, null);
+    assertEquals("http://127.0.0.1/solr/",
+        getLeaderUrl("collection1", "shard2"));
   }
   
   private String getLeaderUrl(final String collection, final String slice)

Modified: lucene/dev/branches/solrcloud/solr/core/src/test/org/apache/solr/cloud/OverseerTest.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/solrcloud/solr/core/src/test/org/apache/solr/cloud/OverseerTest.java?rev=1233073&r1=1233072&r2=1233073&view=diff
==============================================================================
--- lucene/dev/branches/solrcloud/solr/core/src/test/org/apache/solr/cloud/OverseerTest.java (original)
+++ lucene/dev/branches/solrcloud/solr/core/src/test/org/apache/solr/cloud/OverseerTest.java Wed Jan 18 21:33:12 2012
@@ -262,7 +262,7 @@ public class OverseerTest extends SolrTe
       
       //make sure leaders are in cloud state
       for (int i = 0; i < sliceCount; i++) {
-        assertNotNull(reader.getLeaderUrl("collection1", "shard" + (i + 1)));
+        assertNotNull(reader.getLeaderUrl("collection1", "shard" + (i + 1)), 15000);
       }
 
     } finally {
@@ -508,7 +508,7 @@ public class OverseerTest extends SolrTe
     LeaderElector overseerElector = new LeaderElector(zkClient);
     ElectionContext ec = new OverseerElectionContext(address.replaceAll("/", "_"), zkClient, reader);
     overseerElector.setup(ec);
-    overseerElector.joinElection(ec);
+    overseerElector.joinElection(ec, null);
     return zkClient;
   }
 }
\ No newline at end of file

Modified: lucene/dev/branches/solrcloud/solr/core/src/test/org/apache/solr/cloud/RecoveryZkTest.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/solrcloud/solr/core/src/test/org/apache/solr/cloud/RecoveryZkTest.java?rev=1233073&r1=1233072&r2=1233073&view=diff
==============================================================================
--- lucene/dev/branches/solrcloud/solr/core/src/test/org/apache/solr/cloud/RecoveryZkTest.java (original)
+++ lucene/dev/branches/solrcloud/solr/core/src/test/org/apache/solr/cloud/RecoveryZkTest.java Wed Jan 18 21:33:12 2012
@@ -79,7 +79,10 @@ public class RecoveryZkTest extends Full
     // bring shard replica up
     replica.start();
     
-    waitForRecovery(replica);
+    // make sure replication can start
+    Thread.sleep(1500);
+    
+    waitForRecoveriesToFinish(false);
     
     // stop indexing threads
     indexThread.safeStop();
@@ -88,10 +91,14 @@ public class RecoveryZkTest extends Full
     indexThread.join();
     indexThread2.join();
     
+    Thread.sleep(5000);
+    
     commit();
 
     // test that leader and replica have same doc count
     
+    checkShardConsistency("shard1", true); // nocommit: verbose
+    
     long client1Docs = shardToClient.get("shard1").get(0).query(new SolrQuery("*:*")).getResults().getNumFound();
     long client2Docs = shardToClient.get("shard1").get(1).query(new SolrQuery("*:*")).getResults().getNumFound();
     

Modified: lucene/dev/branches/solrcloud/solr/core/src/test/org/apache/solr/update/SolrCmdDistributorTest.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/solrcloud/solr/core/src/test/org/apache/solr/update/SolrCmdDistributorTest.java?rev=1233073&r1=1233072&r2=1233073&view=diff
==============================================================================
--- lucene/dev/branches/solrcloud/solr/core/src/test/org/apache/solr/update/SolrCmdDistributorTest.java (original)
+++ lucene/dev/branches/solrcloud/solr/core/src/test/org/apache/solr/update/SolrCmdDistributorTest.java Wed Jan 18 21:33:12 2012
@@ -26,13 +26,12 @@ import org.apache.solr.client.solrj.Solr
 import org.apache.solr.client.solrj.embedded.JettySolrRunner;
 import org.apache.solr.client.solrj.impl.CommonsHttpSolrServer;
 import org.apache.solr.common.SolrDocumentList;
-import org.apache.solr.common.SolrInputDocument;
 import org.apache.solr.common.cloud.ZkCoreNodeProps;
 import org.apache.solr.common.cloud.ZkNodeProps;
 import org.apache.solr.common.cloud.ZkStateReader;
 import org.apache.solr.common.params.ModifiableSolrParams;
-import org.apache.solr.update.SolrCmdDistributor.Response;
 import org.apache.solr.update.SolrCmdDistributor.Node;
+import org.apache.solr.update.SolrCmdDistributor.Response;
 import org.apache.solr.update.SolrCmdDistributor.StdNode;
 
 public class SolrCmdDistributorTest extends BaseDistributedSearchTestCase {
@@ -101,6 +100,7 @@ public class SolrCmdDistributorTest exte
     CommitUpdateCommand ccmd = new CommitUpdateCommand(null, false);
     cmdDistrib.distribCommit(ccmd, nodes, params);
     cmdDistrib.finish();
+
     Response response = cmdDistrib.getResponse();
     
     assertEquals(response.errors.toString(), 0, response.errors.size());
@@ -115,7 +115,7 @@ public class SolrCmdDistributorTest exte
     nodes.add(new StdNode(new ZkCoreNodeProps(nodeProps)));
     
     // add another 2 docs to control and 3 to client
-    
+    cmdDistrib = new SolrCmdDistributor();
     cmd.solrDoc = sdoc("id", 2);
     cmdDistrib.distribAdd(cmd, nodes, params);
     
@@ -148,10 +148,12 @@ public class SolrCmdDistributorTest exte
     DeleteUpdateCommand dcmd = new DeleteUpdateCommand(null);
     dcmd.id = "2";
     
+    cmdDistrib = new SolrCmdDistributor();
     cmdDistrib.distribDelete(dcmd, nodes, params);
     
     cmdDistrib.distribCommit(ccmd, nodes, params);
     cmdDistrib.finish();
+
     response = cmdDistrib.getResponse();
     
     assertEquals(response.errors.toString(), 0, response.errors.size());

Modified: lucene/dev/branches/solrcloud/solr/solrj/src/java/org/apache/solr/client/solrj/impl/CloudSolrServer.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/solrcloud/solr/solrj/src/java/org/apache/solr/client/solrj/impl/CloudSolrServer.java?rev=1233073&r1=1233072&r2=1233073&view=diff
==============================================================================
--- lucene/dev/branches/solrcloud/solr/solrj/src/java/org/apache/solr/client/solrj/impl/CloudSolrServer.java (original)
+++ lucene/dev/branches/solrcloud/solr/solrj/src/java/org/apache/solr/client/solrj/impl/CloudSolrServer.java Wed Jan 18 21:33:12 2012
@@ -28,6 +28,8 @@ import java.util.Random;
 import java.util.Set;
 import java.util.concurrent.TimeoutException;
 
+import org.apache.commons.httpclient.HttpClient;
+import org.apache.commons.httpclient.MultiThreadedHttpConnectionManager;
 import org.apache.solr.client.solrj.SolrRequest;
 import org.apache.solr.client.solrj.SolrServer;
 import org.apache.solr.client.solrj.SolrServerException;
@@ -53,12 +55,14 @@ public class CloudSolrServer extends Sol
   private String defaultCollection;
   private LBHttpSolrServer lbServer;
   Random rand = new Random();
-
+  private MultiThreadedHttpConnectionManager connManager;
   /**
    * @param zkHost The address of the zookeeper quorum containing the cloud state
    */
   public CloudSolrServer(String zkHost) throws MalformedURLException {
-      this(zkHost, new LBHttpSolrServer());
+      connManager = new MultiThreadedHttpConnectionManager();
+      this.zkHost = zkHost;
+      this.lbServer = new LBHttpSolrServer(new HttpClient(connManager));
   }
 
   /**
@@ -170,8 +174,8 @@ public class CloudSolrServer extends Sol
     }
 
     Collections.shuffle(urlList, rand);
-    // System.out.println("########################## MAKING REQUEST TO " + urlList);
-    // TODO: set distrib=true if we detected more than one shard?
+    //System.out.println("########################## MAKING REQUEST TO " + urlList);
+ 
     LBHttpSolrServer.Req req = new LBHttpSolrServer.Req(request, urlList);
     LBHttpSolrServer.Rsp rsp = lbServer.request(req);
     return rsp.getResponse();
@@ -185,6 +189,9 @@ public class CloudSolrServer extends Sol
         zkStateReader = null;
       }
     }
+    if (connManager != null) {
+      connManager.shutdown();
+    }
   }
 
   public LBHttpSolrServer getLbServer() {

Modified: lucene/dev/branches/solrcloud/solr/solrj/src/java/org/apache/solr/client/solrj/impl/CommonsHttpSolrServer.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/solrcloud/solr/solrj/src/java/org/apache/solr/client/solrj/impl/CommonsHttpSolrServer.java?rev=1233073&r1=1233072&r2=1233073&view=diff
==============================================================================
--- lucene/dev/branches/solrcloud/solr/solrj/src/java/org/apache/solr/client/solrj/impl/CommonsHttpSolrServer.java (original)
+++ lucene/dev/branches/solrcloud/solr/solrj/src/java/org/apache/solr/client/solrj/impl/CommonsHttpSolrServer.java Wed Jan 18 21:33:12 2012
@@ -139,6 +139,8 @@ public class CommonsHttpSolrServer exten
    * with single-part requests.
    */
   private boolean useMultiPartPost;
+
+  private boolean shutdownHttpClient = false;
   
   /**  
    * @param solrServerUrl The URL of the Solr server.  For 
@@ -204,6 +206,7 @@ public class CommonsHttpSolrServer exten
     }
 
     if (client == null) {
+      shutdownHttpClient  = true;
       _httpClient = new HttpClient(new MultiThreadedHttpConnectionManager()) ;
 
       // prevent retries  (note: this didn't work when set on mgr.. needed to be set on client)
@@ -669,4 +672,12 @@ public class CommonsHttpSolrServer exten
     req.setCommitWithin(commitWithinMs);
     return req.process(this);
   }
+  
+  public void shutdown() {
+    if (shutdownHttpClient && _httpClient != null
+        && _httpClient.getHttpConnectionManager() instanceof MultiThreadedHttpConnectionManager) {
+      ((MultiThreadedHttpConnectionManager) _httpClient
+          .getHttpConnectionManager()).shutdown();
+    }
+  }
 }

Modified: lucene/dev/branches/solrcloud/solr/solrj/src/java/org/apache/solr/client/solrj/impl/LBHttpSolrServer.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/solrcloud/solr/solrj/src/java/org/apache/solr/client/solrj/impl/LBHttpSolrServer.java?rev=1233073&r1=1233072&r2=1233073&view=diff
==============================================================================
--- lucene/dev/branches/solrcloud/solr/solrj/src/java/org/apache/solr/client/solrj/impl/LBHttpSolrServer.java (original)
+++ lucene/dev/branches/solrcloud/solr/solrj/src/java/org/apache/solr/client/solrj/impl/LBHttpSolrServer.java Wed Jan 18 21:33:12 2012
@@ -317,7 +317,7 @@ public class LBHttpSolrServer extends So
     if (ex == null) {
       throw new SolrServerException("No live SolrServers available to handle this request");
     } else {
-      throw new SolrServerException("No live SolrServers available to handle this request", ex);
+      throw new SolrServerException("No live SolrServers available to handle this request:" + zombieServers.keySet(), ex);
     }
 
   }
@@ -400,6 +400,12 @@ public class LBHttpSolrServer extends So
   public void setSoTimeout(int timeout) {
     httpClient.getParams().setSoTimeout(timeout);
   }
+  
+  public void shutdown() {
+    if (aliveCheckExecutor != null) {
+      aliveCheckExecutor.shutdownNow();
+    }
+  }
 
   /**
    * Tries to query a live server. A SolrServerException is thrown if all servers are dead.

Modified: lucene/dev/branches/solrcloud/solr/solrj/src/java/org/apache/solr/client/solrj/impl/StreamingUpdateSolrServer.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/solrcloud/solr/solrj/src/java/org/apache/solr/client/solrj/impl/StreamingUpdateSolrServer.java?rev=1233073&r1=1233072&r2=1233073&view=diff
==============================================================================
--- lucene/dev/branches/solrcloud/solr/solrj/src/java/org/apache/solr/client/solrj/impl/StreamingUpdateSolrServer.java (original)
+++ lucene/dev/branches/solrcloud/solr/solrj/src/java/org/apache/solr/client/solrj/impl/StreamingUpdateSolrServer.java Wed Jan 18 21:33:12 2012
@@ -304,7 +304,9 @@ public class StreamingUpdateSolrServer e
     log.error( "error", ex );
   }
   
+  @Override
   public void shutdown() {
+    super.shutdown();
     scheduler.shutdown();
     try {
       if (!scheduler.awaitTermination(60, TimeUnit.SECONDS)) {
@@ -320,6 +322,7 @@ public class StreamingUpdateSolrServer e
   
   
   public void shutdownNow() {
+    super.shutdown();
     scheduler.shutdownNow(); // Cancel currently executing tasks
     try {
       if (!scheduler.awaitTermination(30, TimeUnit.SECONDS)) log

Modified: lucene/dev/branches/solrcloud/solr/solrj/src/java/org/apache/solr/common/SolrException.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/solrcloud/solr/solrj/src/java/org/apache/solr/common/SolrException.java?rev=1233073&r1=1233072&r2=1233073&view=diff
==============================================================================
--- lucene/dev/branches/solrcloud/solr/solrj/src/java/org/apache/solr/common/SolrException.java (original)
+++ lucene/dev/branches/solrcloud/solr/solrj/src/java/org/apache/solr/common/SolrException.java Wed Jan 18 21:33:12 2012
@@ -135,6 +135,16 @@ public class SolrException extends Runti
     if (msg!=null) log(log,msg,e);
     else log(log,e);
   }
+  
+  public static void log(Logger log, String msg) {
+    String stackTrace = msg;
+    String ignore = doIgnore(stackTrace);
+    if (ignore != null) {
+      log.info(ignore);
+      return;
+    }
+    log.error(stackTrace);
+  }
 
 
   // public String toString() { return toStr(this); }  // oops, inf loop

Modified: lucene/dev/branches/solrcloud/solr/solrj/src/java/org/apache/solr/common/cloud/SolrZkClient.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/solrcloud/solr/solrj/src/java/org/apache/solr/common/cloud/SolrZkClient.java?rev=1233073&r1=1233072&r2=1233073&view=diff
==============================================================================
--- lucene/dev/branches/solrcloud/solr/solrj/src/java/org/apache/solr/common/cloud/SolrZkClient.java (original)
+++ lucene/dev/branches/solrcloud/solr/solrj/src/java/org/apache/solr/common/cloud/SolrZkClient.java Wed Jan 18 21:33:12 2012
@@ -39,6 +39,7 @@ import org.apache.solr.common.SolrExcept
 import org.apache.solr.common.cloud.ZkClientConnectionStrategy.ZkUpdate;
 import org.apache.zookeeper.CreateMode;
 import org.apache.zookeeper.KeeperException;
+import org.apache.zookeeper.KeeperException.NoNodeException;
 import org.apache.zookeeper.SolrZooKeeper;
 import org.apache.zookeeper.Watcher;
 import org.apache.zookeeper.ZooDefs;
@@ -595,8 +596,12 @@ public class SolrZkClient {
 
     for (String child : children) {
       if (!child.equals("quota")) {
-        printLayout(path + (path.equals("/") ? "" : "/") + child, indent + 1,
-            string);
+        try {
+          printLayout(path + (path.equals("/") ? "" : "/") + child, indent + 1,
+              string);
+        } catch (NoNodeException e) {
+          // must have gone away
+        }
       }
     }
 

Modified: lucene/dev/branches/solrcloud/solr/solrj/src/java/org/apache/solr/common/cloud/ZkStateReader.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/solrcloud/solr/solrj/src/java/org/apache/solr/common/cloud/ZkStateReader.java?rev=1233073&r1=1233072&r2=1233073&view=diff
==============================================================================
--- lucene/dev/branches/solrcloud/solr/solrj/src/java/org/apache/solr/common/cloud/ZkStateReader.java (original)
+++ lucene/dev/branches/solrcloud/solr/solrj/src/java/org/apache/solr/common/cloud/ZkStateReader.java Wed Jan 18 21:33:12 2012
@@ -414,6 +414,11 @@ public class ZkStateReader {
   
   public List<ZkCoreNodeProps> getReplicaProps(String collection,
       String shardId, String thisNodeName, String coreName) {
+    return getReplicaProps(collection, shardId, thisNodeName, coreName, null);
+  }
+  
+  public List<ZkCoreNodeProps> getReplicaProps(String collection,
+      String shardId, String thisNodeName, String coreName, String stateFilter) {
     CloudState cloudState = this.cloudState;
     if (cloudState == null) {
       return null;
@@ -437,7 +442,9 @@ public class ZkStateReader {
       ZkCoreNodeProps nodeProps = new ZkCoreNodeProps(entry.getValue());
       String coreNodeName = nodeProps.getNodeName() + "_" + coreName;
       if (cloudState.liveNodesContain(thisNodeName) && !coreNodeName.equals(thisNodeName + "_" + coreName)) {
-        nodes.add(nodeProps);
+        if (stateFilter == null || stateFilter.equals(nodeProps.getState())) {
+          nodes.add(nodeProps);
+        }
       }
     }
     if (nodes.size() == 0) {

Modified: lucene/dev/branches/solrcloud/solr/test-framework/src/java/org/apache/solr/BaseDistributedSearchTestCase.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/solrcloud/solr/test-framework/src/java/org/apache/solr/BaseDistributedSearchTestCase.java?rev=1233073&r1=1233072&r2=1233073&view=diff
==============================================================================
--- lucene/dev/branches/solrcloud/solr/test-framework/src/java/org/apache/solr/BaseDistributedSearchTestCase.java (original)
+++ lucene/dev/branches/solrcloud/solr/test-framework/src/java/org/apache/solr/BaseDistributedSearchTestCase.java Wed Jan 18 21:33:12 2012
@@ -32,7 +32,6 @@ import java.util.Set;
 
 import junit.framework.TestCase;
 
-import org.apache.solr.client.solrj.SolrResponse;
 import org.apache.solr.client.solrj.SolrServer;
 import org.apache.solr.client.solrj.SolrServerException;
 import org.apache.solr.client.solrj.embedded.JettySolrRunner;
@@ -46,7 +45,6 @@ import org.apache.solr.common.SolrInputD
 import org.apache.solr.common.params.ModifiableSolrParams;
 import org.apache.solr.common.params.SolrParams;
 import org.apache.solr.common.util.NamedList;
-import org.apache.solr.response.SolrQueryResponse;
 import org.apache.solr.schema.TrieDateField;
 import org.apache.solr.util.AbstractSolrTestCase;
 import org.junit.Test;
@@ -231,7 +229,9 @@ public abstract class BaseDistributedSea
 
   protected void destroyServers() throws Exception {
     controlJetty.stop();
+    ((CommonsHttpSolrServer) controlClient).shutdown();
     for (JettySolrRunner jetty : jettys) jetty.stop();
+    for (SolrServer client : clients) ((CommonsHttpSolrServer) client).shutdown();
     clients.clear();
     jettys.clear();
   }