You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@lucene.apache.org by th...@apache.org on 2015/04/08 17:25:39 UTC

svn commit: r1672105 - in /lucene/dev/branches/lucene_solr_5_1: ./ solr/ solr/core/ solr/core/src/test/org/apache/solr/cloud/ solr/test-framework/ solr/test-framework/src/java/org/apache/solr/cloud/

Author: thelabdude
Date: Wed Apr  8 15:25:38 2015
New Revision: 1672105

URL: http://svn.apache.org/r1672105
Log:
SOLR-6944: Add BadApple annotation to ReplicationFactorTest and increase socket timeout in SocketProxy from 10 to 100

Modified:
    lucene/dev/branches/lucene_solr_5_1/   (props changed)
    lucene/dev/branches/lucene_solr_5_1/solr/   (props changed)
    lucene/dev/branches/lucene_solr_5_1/solr/core/   (props changed)
    lucene/dev/branches/lucene_solr_5_1/solr/core/src/test/org/apache/solr/cloud/LeaderInitiatedRecoveryOnCommitTest.java
    lucene/dev/branches/lucene_solr_5_1/solr/core/src/test/org/apache/solr/cloud/ReplicationFactorTest.java
    lucene/dev/branches/lucene_solr_5_1/solr/test-framework/   (props changed)
    lucene/dev/branches/lucene_solr_5_1/solr/test-framework/src/java/org/apache/solr/cloud/SocketProxy.java

Modified: lucene/dev/branches/lucene_solr_5_1/solr/core/src/test/org/apache/solr/cloud/LeaderInitiatedRecoveryOnCommitTest.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene_solr_5_1/solr/core/src/test/org/apache/solr/cloud/LeaderInitiatedRecoveryOnCommitTest.java?rev=1672105&r1=1672104&r2=1672105&view=diff
==============================================================================
--- lucene/dev/branches/lucene_solr_5_1/solr/core/src/test/org/apache/solr/cloud/LeaderInitiatedRecoveryOnCommitTest.java (original)
+++ lucene/dev/branches/lucene_solr_5_1/solr/core/src/test/org/apache/solr/cloud/LeaderInitiatedRecoveryOnCommitTest.java Wed Apr  8 15:25:38 2015
@@ -17,11 +17,12 @@ package org.apache.solr.cloud;
  * limitations under the License.
  */
 
+import org.apache.http.NoHttpResponseException;
 import org.apache.solr.client.solrj.embedded.JettySolrRunner;
 import org.apache.solr.client.solrj.impl.HttpSolrClient;
 import org.apache.solr.client.solrj.request.CollectionAdminRequest;
+import org.apache.solr.common.SolrException;
 import org.apache.solr.common.cloud.Replica;
-import org.apache.solr.common.cloud.ZkCoreNodeProps;
 import org.junit.Test;
 
 import java.io.File;
@@ -64,7 +65,10 @@ public class LeaderInitiatedRecoveryOnCo
   }
 
   private void multiShardTest() throws Exception {
-    // create a collection that has 1 shard and 3 replicas
+
+    log.info("Running multiShardTest");
+
+    // create a collection that has 2 shard and 2 replicas
     String testCollectionName = "c8n_2x2_commits";
     createCollection(testCollectionName, 2, 2, 1);
     cloudClient.setDefaultCollection(testCollectionName);
@@ -76,16 +80,17 @@ public class LeaderInitiatedRecoveryOnCo
             + printClusterStateInfo(),
         notLeaders.size() == 1);
 
+    log.info("All replicas active for "+testCollectionName);
+
     // let's put the leader in its own partition, no replicas can contact it now
     Replica leader = cloudClient.getZkStateReader().getLeaderRetry(testCollectionName, "shard1");
+    log.info("Creating partition to leader at "+leader.getCoreUrl());
     SocketProxy leaderProxy = getProxyForReplica(leader);
     leaderProxy.close();
 
     // let's find the leader of shard2 and ask him to commit
     Replica shard2Leader = cloudClient.getZkStateReader().getLeaderRetry(testCollectionName, "shard2");
-    try (HttpSolrClient server = new HttpSolrClient(ZkCoreNodeProps.getCoreUrl(shard2Leader.getStr("base_url"), shard2Leader.getStr("core")))) {
-      server.commit();
-    }
+    sendCommitWithRetry(shard2Leader);
 
     Thread.sleep(sleepMsBeforeHealPartition);
 
@@ -93,6 +98,7 @@ public class LeaderInitiatedRecoveryOnCo
     leader = cloudClient.getZkStateReader().getLeaderRetry(testCollectionName, "shard1");
     assertEquals("Leader was not active", "active", leader.getStr("state"));
 
+    log.info("Healing partitioned replica at "+leader.getCoreUrl());
     leaderProxy.reopen();
     Thread.sleep(sleepMsBeforeHealPartition);
 
@@ -105,9 +111,13 @@ public class LeaderInitiatedRecoveryOnCo
       // don't fail the test
       log.warn("Could not delete collection {} after test completed", testCollectionName);
     }
+
+    log.info("multiShardTest completed OK");
   }
 
   private void oneShardTest() throws Exception {
+    log.info("Running oneShardTest");
+
     // create a collection that has 1 shard and 3 replicas
     String testCollectionName = "c8n_1x3_commits";
     createCollection(testCollectionName, 1, 3, 1);
@@ -120,22 +130,23 @@ public class LeaderInitiatedRecoveryOnCo
             + printClusterStateInfo(),
         notLeaders.size() == 2);
 
+    log.info("All replicas active for "+testCollectionName);
+
     // let's put the leader in its own partition, no replicas can contact it now
     Replica leader = cloudClient.getZkStateReader().getLeaderRetry(testCollectionName, "shard1");
+    log.info("Creating partition to leader at "+leader.getCoreUrl());
     SocketProxy leaderProxy = getProxyForReplica(leader);
     leaderProxy.close();
 
     Replica replica = notLeaders.get(0);
-    try (HttpSolrClient client = new HttpSolrClient(ZkCoreNodeProps.getCoreUrl(replica.getStr("base_url"), replica.getStr("core")))) {
-      client.commit();
-    }
-
+    sendCommitWithRetry(replica);
     Thread.sleep(sleepMsBeforeHealPartition);
 
     cloudClient.getZkStateReader().updateClusterState(true); // get the latest state
     leader = cloudClient.getZkStateReader().getLeaderRetry(testCollectionName, "shard1");
     assertEquals("Leader was not active", "active", leader.getStr("state"));
 
+    log.info("Healing partitioned replica at "+leader.getCoreUrl());
     leaderProxy.reopen();
     Thread.sleep(sleepMsBeforeHealPartition);
 
@@ -148,6 +159,8 @@ public class LeaderInitiatedRecoveryOnCo
       // don't fail the test
       log.warn("Could not delete collection {} after test completed", testCollectionName);
     }
+
+    log.info("oneShardTest completed OK");
   }
 
   /**
@@ -160,4 +173,29 @@ public class LeaderInitiatedRecoveryOnCo
     return createProxiedJetty(solrHome, dataDir, shardList, solrConfigOverride, schemaOverride);
   }
 
+  protected void sendCommitWithRetry(Replica replica) throws Exception {
+    String replicaCoreUrl = replica.getCoreUrl();
+    log.info("Sending commit request to: "+replicaCoreUrl);
+    long startMs = System.currentTimeMillis();
+    try (HttpSolrClient client = new HttpSolrClient(replicaCoreUrl)) {
+      try {
+        client.commit();
+
+        long tookMs = System.currentTimeMillis() - startMs;
+        log.info("Sent commit request to "+replicaCoreUrl+" OK, took: "+tookMs);
+      } catch (Exception exc) {
+        Throwable rootCause = SolrException.getRootCause(exc);
+        if (rootCause instanceof NoHttpResponseException) {
+          log.warn("No HTTP response from sending commit request to "+replicaCoreUrl+
+              "; will re-try after waiting 3 seconds");
+          Thread.sleep(3000);
+          client.commit();
+          log.info("Second attempt at sending commit to "+replicaCoreUrl+" succeeded.");
+        } else {
+          throw exc;
+        }
+      }
+    }
+  }
+
 }

Modified: lucene/dev/branches/lucene_solr_5_1/solr/core/src/test/org/apache/solr/cloud/ReplicationFactorTest.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene_solr_5_1/solr/core/src/test/org/apache/solr/cloud/ReplicationFactorTest.java?rev=1672105&r1=1672104&r2=1672105&view=diff
==============================================================================
--- lucene/dev/branches/lucene_solr_5_1/solr/core/src/test/org/apache/solr/cloud/ReplicationFactorTest.java (original)
+++ lucene/dev/branches/lucene_solr_5_1/solr/core/src/test/org/apache/solr/cloud/ReplicationFactorTest.java Wed Apr  8 15:25:38 2015
@@ -23,6 +23,7 @@ import java.util.ArrayList;
 import java.util.List;
 import java.util.Locale;
 
+import org.apache.lucene.util.LuceneTestCase;
 import org.apache.lucene.util.LuceneTestCase.Slow;
 import org.apache.solr.SolrTestCaseJ4.SuppressSSL;
 import org.apache.solr.client.solrj.embedded.JettySolrRunner;
@@ -47,6 +48,7 @@ import org.slf4j.LoggerFactory;
  */
 @Slow
 @SuppressSSL(bugUrl = "https://issues.apache.org/jira/browse/SOLR-5776")
+@LuceneTestCase.BadApple(bugUrl = "https://issues.apache.org/jira/browse/SOLR-6944")
 public class ReplicationFactorTest extends AbstractFullDistribZkTestBase {
   
   private static final transient Logger log = 

Modified: lucene/dev/branches/lucene_solr_5_1/solr/test-framework/src/java/org/apache/solr/cloud/SocketProxy.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene_solr_5_1/solr/test-framework/src/java/org/apache/solr/cloud/SocketProxy.java?rev=1672105&r1=1672104&r2=1672105&view=diff
==============================================================================
--- lucene/dev/branches/lucene_solr_5_1/solr/test-framework/src/java/org/apache/solr/cloud/SocketProxy.java (original)
+++ lucene/dev/branches/lucene_solr_5_1/solr/test-framework/src/java/org/apache/solr/cloud/SocketProxy.java Wed Apr  8 15:25:38 2015
@@ -48,6 +48,9 @@ public class SocketProxy {
   private static final transient Logger log = LoggerFactory.getLogger(SocketProxy.class);
   
   public static final int ACCEPT_TIMEOUT_MILLIS = 100;
+
+  // should be as large as the HttpShardHandlerFactory socket timeout ... or larger?
+  public static final int PUMP_SOCKET_TIMEOUT_MS = 100 * 1000;
   
   private URI proxyUrl;
   private URI target;
@@ -148,7 +151,7 @@ public class SocketProxy {
     synchronized (this.connections) {
       connections = new ArrayList<Bridge>(this.connections);
     }
-    log.warn("Closing " + connections.size()+" connections to: "+getUrl());
+    log.warn("Closing " + connections.size()+" connections to: "+getUrl()+", target: "+target);
     for (Bridge con : connections) {
       closeConnection(con);
     }
@@ -338,7 +341,7 @@ public class SocketProxy {
         byte[] buf = new byte[1024];
 
         try {
-          src.setSoTimeout(10 * 1000);
+          src.setSoTimeout(PUMP_SOCKET_TIMEOUT_MS);
         } catch (SocketException e) {
           log.error("Failed to set socket timeout on "+src+" due to: "+e);
           throw new RuntimeException(e);