lucene-commits mailing list archives

Site index · List index
Message view « Date » · « Thread »
Top « Date » · « Thread »
From hoss...@apache.org
Subject [lucene-solr] branch branch_8x updated: Harden TestPolicyCloud
Date Thu, 11 Jul 2019 00:59:10 GMT
This is an automated email from the ASF dual-hosted git repository.

hossman pushed a commit to branch branch_8x
in repository https://gitbox.apache.org/repos/asf/lucene-solr.git


The following commit(s) were added to refs/heads/branch_8x by this push:
     new 81b2e06  Harden TestPolicyCloud
81b2e06 is described below

commit 81b2e06ffe6bddcd8d25b24c79683281da85baee
Author: Chris Hostetter <hossman@apache.org>
AuthorDate: Wed Jul 10 17:16:59 2019 -0700

    Harden TestPolicyCloud
    
    - ensure all collections/replicas are active
    
    - tighten assertions around expected replica locations
    
    - eliminate some redundent code
    
    These changes should also help ensure we don't get (more) spurious failures due to SOLR-13616
    
    (cherry picked from commit 8a277cab7d15c03ff59577efceb6a0cb281d095a)
---
 .../solr/cloud/autoscaling/TestPolicyCloud.java    | 243 ++++++++++++++-------
 1 file changed, 169 insertions(+), 74 deletions(-)

diff --git a/solr/core/src/test/org/apache/solr/cloud/autoscaling/TestPolicyCloud.java b/solr/core/src/test/org/apache/solr/cloud/autoscaling/TestPolicyCloud.java
index c60f9d9..5d7981f 100644
--- a/solr/core/src/test/org/apache/solr/cloud/autoscaling/TestPolicyCloud.java
+++ b/solr/core/src/test/org/apache/solr/cloud/autoscaling/TestPolicyCloud.java
@@ -16,10 +16,9 @@
  */
 package org.apache.solr.cloud.autoscaling;
 
-import java.io.IOException;
-import java.io.StringWriter;
 import java.lang.invoke.MethodHandles;
 import java.nio.charset.StandardCharsets;
+import java.util.ArrayList;
 import java.util.Arrays;
 import java.util.Collections;
 import java.util.List;
@@ -33,12 +32,10 @@ import com.google.common.collect.ImmutableSet;
 import org.apache.lucene.util.Constants;
 import org.apache.lucene.util.LuceneTestCase;
 import org.apache.solr.client.solrj.SolrRequest;
-import org.apache.solr.client.solrj.SolrServerException;
 import org.apache.solr.client.solrj.cloud.DistributedQueueFactory;
 import org.apache.solr.client.solrj.cloud.SolrCloudManager;
 import org.apache.solr.client.solrj.cloud.autoscaling.AutoScalingConfig;
 import org.apache.solr.client.solrj.cloud.autoscaling.Policy;
-import org.apache.solr.client.solrj.cloud.autoscaling.PolicyHelper;
 import org.apache.solr.client.solrj.cloud.autoscaling.ReplicaInfo;
 import org.apache.solr.client.solrj.cloud.autoscaling.Row;
 import org.apache.solr.client.solrj.cloud.autoscaling.Variable.Type;
@@ -52,14 +49,15 @@ import org.apache.solr.cloud.CloudTestUtils.AutoScalingRequest;
 import org.apache.solr.cloud.OverseerTaskProcessor;
 import org.apache.solr.cloud.SolrCloudTestCase;
 import org.apache.solr.cloud.ZkDistributedQueueFactory;
+import org.apache.solr.common.cloud.CollectionStatePredicate;
 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.ZkStateReader;
 import org.apache.solr.common.util.NamedList;
 import org.apache.solr.common.util.TimeSource;
 import org.apache.solr.common.util.Utils;
 import org.apache.solr.util.TimeOut;
-import org.apache.zookeeper.KeeperException;
 import org.junit.After;
 import org.junit.BeforeClass;
 import org.junit.rules.ExpectedException;
@@ -118,10 +116,9 @@ public class TestPolicyCloud extends SolrCloudTestCase {
     commands =  "{ set-cluster-policy: [ {cores: '<2', node: '#ANY'} ] }";
     cluster.getSolrClient().request(AutoScalingRequest.create(SolrRequest.METHOD.POST, commands));
     CollectionAdminRequest.createCollection(collectionName, "conf", 2, 1).process(cluster.getSolrClient());
-    SolrClientCloudManager scm = new SolrClientCloudManager(new ZkDistributedQueueFactory(cluster.getSolrClient().getZkStateReader().getZkClient()),
cluster.getSolrClient());
-    Policy.Session session = scm.getDistribStateManager().getAutoScalingConfig().getPolicy().createSession(scm);
-    System.out.println(Utils.writeJson(PolicyHelper.getDiagnostics(session), new StringWriter(),
true).toString());
-
+    
+    cluster.waitForActiveCollection(collectionName, 2, 2);
+    
   }
 
   public void testDataProviderPerReplicaDetails() throws Exception {
@@ -176,75 +173,163 @@ public class TestPolicyCloud extends SolrCloudTestCase {
     }
     assertTrue(count.get() > 0);
 
-    CollectionAdminRequest.deleteCollection("perReplicaDataColl").process(cluster.getSolrClient());
-
   }
-
+  
+  private static CollectionStatePredicate expectAllReplicasOnSpecificNode
+    (final String expectedNodeName,
+     final int expectedSliceCount,
+     final int expectedReplicaCount) {
+
+    return (liveNodes, collection) -> {
+      if (null == collection || expectedSliceCount != collection.getSlices().size()) {
+        return false;
+      }
+      int actualReplicaCount = 0;
+      for (Slice slice : collection) {
+        for (Replica replica : slice) {
+          if ( ! (replica.isActive(liveNodes)
+                  && expectedNodeName.equals(replica.getNodeName())) ) {
+            return false;
+          }
+          actualReplicaCount++;
+        }
+      }
+      return expectedReplicaCount == actualReplicaCount;
+    };
+  }
+  
   public void testCreateCollectionAddReplica() throws Exception  {
-    JettySolrRunner jetty = cluster.getRandomJetty(random());
-    int port = jetty.getLocalPort();
+    final JettySolrRunner jetty = cluster.getRandomJetty(random());
+    final String jettyNodeName = jetty.getNodeName();
+    final int port = jetty.getLocalPort();
 
-    String commands =  "{set-policy :{c1 : [{replica:0 , shard:'#EACH', port: '!" + port
+ "'}]}}";
+    final String commands =  "{set-policy :{c1 : [{replica:0 , shard:'#EACH', port: '!" +
port + "'}]}}";
     cluster.getSolrClient().request(AutoScalingRequest.create(SolrRequest.METHOD.POST, commands));
 
-    String collectionName = "testCreateCollectionAddReplica";
+    final String collectionName = "testCreateCollectionAddReplica";
+    log.info("Creating collection {}", collectionName);
     CollectionAdminRequest.createCollection(collectionName, "conf", 1, 1)
         .setPolicy("c1")
         .process(cluster.getSolrClient());
 
-    getCollectionState(collectionName).forEachReplica((s, replica) -> assertEquals(jetty.getNodeName(),
replica.getNodeName()));
+    waitForState("Should have found exactly one replica, only on expected jetty: " +
+                 jettyNodeName + "/" + port,
+                 collectionName, expectAllReplicasOnSpecificNode(jettyNodeName, 1, 1),
+                 120, TimeUnit.SECONDS);
 
-    CollectionAdminRequest.addReplicaToShard(collectionName, "shard1").process(cluster.getSolrClient());
-    waitForState("Timed out waiting to see 2 replicas for collection: " + collectionName,
-        collectionName, (liveNodes, collectionState) -> collectionState.getReplicas().size()
== 2);
+    log.info("Adding replica to {}", collectionName);
+    CollectionAdminRequest.addReplicaToShard(collectionName, "shard1")
+      .process(cluster.getSolrClient());
+    
+    waitForState("Should have found exactly two replicas, only on expected jetty: " +
+                 jettyNodeName + "/" + port,
+                 collectionName, expectAllReplicasOnSpecificNode(jettyNodeName, 1, 2),
+                 120, TimeUnit.SECONDS);
 
-    getCollectionState(collectionName).forEachReplica((s, replica) -> assertEquals(jetty.getNodeName(),
replica.getNodeName()));
   }
 
   public void testCreateCollectionSplitShard() throws Exception  {
-    JettySolrRunner firstNode = cluster.getRandomJetty(random());
-    int firstNodePort = firstNode.getLocalPort();
 
-    JettySolrRunner secondNode = null;
-    while (true)  {
-      secondNode = cluster.getRandomJetty(random());
-      if (secondNode.getLocalPort() != firstNodePort)  break;
-    }
-    int secondNodePort = secondNode.getLocalPort();
+    final List<JettySolrRunner> shuffledJetties = new ArrayList<>(cluster.getJettySolrRunners());
+    Collections.shuffle(shuffledJetties, random());
+    assertTrue(2 < shuffledJetties.size()); // sanity check test setup
+    
+    final JettySolrRunner firstNode = shuffledJetties.get(0);
+    final JettySolrRunner secondNode = shuffledJetties.get(1);
+
+    final int firstNodePort = firstNode.getLocalPort();
+    final int secondNodePort = secondNode.getLocalPort();
+    assertNotEquals(firstNodePort, secondNodePort);
+    
+    final String commands =  "{set-policy :{c1 : [{replica:1 , shard:'#EACH', port: '" +
+      firstNodePort + "'}, {replica:1, shard:'#EACH', port:'" + secondNodePort + "'}]}}";
+
+    final String firstNodeName = firstNode.getNodeName();
+    final String secondNodeName = secondNode.getNodeName();
+    assertNotEquals(firstNodeName, secondNodeName);
 
-    String commands =  "{set-policy :{c1 : [{replica:1 , shard:'#EACH', port: '" + firstNodePort
+ "'}, {replica:1, shard:'#EACH', port:'" + secondNodePort + "'}]}}";
-    NamedList<Object> response = cluster.getSolrClient().request(AutoScalingRequest.create(SolrRequest.METHOD.POST,
commands));
+    final NamedList<Object> response = cluster.getSolrClient()
+      .request(AutoScalingRequest.create(SolrRequest.METHOD.POST, commands));
     assertEquals("success", response.get("result"));
 
-    String collectionName = "testCreateCollectionSplitShard";
+    // through out the test, every shard shuld have 2 replicas, one on each of these two
nodes
+    final Set<String> expectedNodeNames = ImmutableSet.of(firstNodeName, secondNodeName);
+    
+    final String collectionName = "testCreateCollectionSplitShard";
+    log.info("Creating collection {}", collectionName);
     CollectionAdminRequest.createCollection(collectionName, "conf", 1, 2)
         .setPolicy("c1")
         .process(cluster.getSolrClient());
-    cluster.waitForActiveCollection(collectionName, 1, 2);
-    DocCollection docCollection = getCollectionState(collectionName);
-    List<Replica> list = docCollection.getReplicas(firstNode.getNodeName());
-    int replicasOnNode1 = list != null ? list.size() : 0;
-    list = docCollection.getReplicas(secondNode.getNodeName());
-    int replicasOnNode2 = list != null ? list.size() : 0;
-
-    assertEquals("Expected exactly one replica of collection on node with port: " + firstNodePort,
1, replicasOnNode1);
-    assertEquals("Expected exactly one replica of collection on node with port: " + secondNodePort,
1, replicasOnNode2);
-
-    CollectionAdminRequest.splitShard(collectionName).setShardName("shard1").process(cluster.getSolrClient());
-
-    waitForState("Timed out waiting to see 6 replicas for collection: " + collectionName,
-        collectionName, (liveNodes, collectionState) -> collectionState.getReplicas().size()
== 6);
-
-    docCollection = getCollectionState(collectionName);
-    list = docCollection.getReplicas(firstNode.getNodeName());
-    replicasOnNode1 = list != null ? list.size() : 0;
-    list = docCollection.getReplicas(secondNode.getNodeName());
-    replicasOnNode2 = list != null ? list.size() : 0;
-
-    assertEquals("Expected exactly three replica of collection on node with port: " + firstNodePort,
3, replicasOnNode1);
-    assertEquals("Expected exactly three replica of collection on node with port: " + secondNodePort,
3, replicasOnNode2);
-    CollectionAdminRequest.deleteCollection(collectionName).process(cluster.getSolrClient());
-
+                   
+    waitForState("Should have found exactly 1 slice w/2 live Replicas, one on each expected
jetty: " +
+                 firstNodeName + "/" + firstNodePort + " & " +  secondNodeName + "/"
+ secondNodePort,
+                 collectionName, (liveNodes, collection) -> {
+                   // short circut if collection is deleted
+                   // or we some how have the wrong number of slices
+                   if (null == collection || 1 != collection.getSlices().size()) {
+                     return false;
+                   }
+                   // Note: only 1 slices, but simpler to loop then extract...
+                   for (Slice slice : collection.getSlices()) {
+                     // short circut if our slice isn't active, or has wrong # replicas
+                     if (Slice.State.ACTIVE != slice.getState()
+                         || 2 != slice.getReplicas().size()) {
+                       return false;
+                     }
+                     // make sure our replicas are fully live...
+                     final List<Replica> liveReplicas = slice.getReplicas
+                       ((r) -> r.isActive(liveNodes));
+                     if (2 != liveReplicas.size()) {
+                       return false;
+                     }
+                     // now the main check we care about: were the replicas split up on
+                     // the expected nodes...
+                     if (! expectedNodeNames.equals(ImmutableSet.of
+                                                  (liveReplicas.get(0).getNodeName(),
+                                                   liveReplicas.get(1).getNodeName()))) {
+                       return false;
+                     }
+                   }
+                   return true;
+                 });
+
+    log.info("Splitting (single) Shard on collection {}", collectionName);
+    CollectionAdminRequest.splitShard(collectionName).setShardName("shard1")
+      .process(cluster.getSolrClient());
+
+    waitForState("Should have found exactly 3 shards (1 inactive) each w/two live Replicas,
" +
+                 "one on each expected jetty: " +
+                 firstNodeName + "/" + firstNodePort + " & " +  secondNodeName + "/"
+ secondNodePort,
+                 collectionName, (liveNodes, collection) -> {
+                   // short circut if collection is deleted
+                   // or we some how have the wrong number of (active) slices
+                   if (null == collection
+                       || 3 != collection.getSlices().size()
+                       || 2 != collection.getActiveSlices().size()) {
+                     return false;
+                   }
+                   // Note: we're checking all slices, even the inactive (split) slice...
+                   for (Slice slice : collection.getSlices()) {
+                     // short circut if our slice has wrong # replicas
+                     if (2 != slice.getReplicas().size()) {
+                       return false;
+                     }
+                     // make sure our replicas are fully live...
+                     final List<Replica> liveReplicas = slice.getReplicas
+                       ((r) -> r.isActive(liveNodes));
+                     if (2 != liveReplicas.size()) {
+                       return false;
+                     }
+                     // now the main check we care about: were the replicas split up on
+                     // the expected nodes...
+                     if (! expectedNodeNames.equals(ImmutableSet.of
+                                                    (liveReplicas.get(0).getNodeName(),
+                                                     liveReplicas.get(1).getNodeName())))
{
+                       return false;
+                     }
+                   }
+                   return true;
+                 });
   }
 
   public void testMetricsTag() throws Exception {
@@ -275,10 +360,11 @@ public class TestPolicyCloud extends SolrCloudTestCase {
     req = AutoScalingRequest.create(SolrRequest.METHOD.POST, setClusterPolicyCommand);
     solrClient.request(req);
 
-    //org.eclipse.jetty.server.handler.DefaultHandler.2xx-responses
-    CollectionAdminRequest.createCollection("metricsTest", "conf", 1, 1)
+    final String collectionName = "metrics_tags";
+    CollectionAdminRequest.createCollection(collectionName, "conf", 1, 1)
         .process(cluster.getSolrClient());
-    DocCollection collection = getCollectionState("metricsTest");
+    cluster.waitForActiveCollection(collectionName, 1, 1);
+    DocCollection collection = getCollectionState(collectionName);
     DistributedQueueFactory queueFactory = new ZkDistributedQueueFactory(cluster.getZkClient());
     try (SolrCloudManager provider = new SolrClientCloudManager(queueFactory, solrClient))
{
       List<String> tags = Arrays.asList("metrics:solr.node:ADMIN./admin/authorization.clientErrors:count",
@@ -324,13 +410,14 @@ public class TestPolicyCloud extends SolrCloudTestCase {
     assertEquals("full json:" + Utils.toJSONString(json), "!" + tlogPort,
         Utils.getObjectByPath(json, true, "cluster-policy[2]/port"));
 
-    CollectionAdminRequest.createCollectionWithImplicitRouter("policiesTest", "conf", "s1",
1, 1, 1)
+    final String collectionName = "addshard_with_reptype_using_policy";
+    CollectionAdminRequest.createCollectionWithImplicitRouter(collectionName, "conf", "s1",
1, 1, 1)
         .setMaxShardsPerNode(-1)
         .process(cluster.getSolrClient());
     
-    cluster.waitForActiveCollection("policiesTest", 1, 3);
+    cluster.waitForActiveCollection(collectionName, 1, 3);
 
-    DocCollection coll = getCollectionState("policiesTest");
+    DocCollection coll = getCollectionState(collectionName);
 
 
     BiConsumer<String, Replica> verifyReplicas = (s, replica) -> {
@@ -352,12 +439,12 @@ public class TestPolicyCloud extends SolrCloudTestCase {
     };
     coll.forEachReplica(verifyReplicas);
 
-    CollectionAdminRequest.createShard("policiesTest", "s3").
+    CollectionAdminRequest.createShard(collectionName, "s3").
         process(cluster.getSolrClient());
     
-    cluster.waitForActiveCollection("policiesTest", 2, 6);
+    cluster.waitForActiveCollection(collectionName, 2, 6);
     
-    coll = getCollectionState("policiesTest");
+    coll = getCollectionState(collectionName);
     assertEquals(3, coll.getSlice("s3").getReplicas().size());
     coll.forEachReplica(verifyReplicas);
   }
@@ -371,27 +458,35 @@ public class TestPolicyCloud extends SolrCloudTestCase {
     Map<String, Object> json = Utils.getJson(cluster.getZkClient(), ZkStateReader.SOLR_AUTOSCALING_CONF_PATH,
true);
     assertEquals("full json:"+ Utils.toJSONString(json) , "#EACH",
         Utils.getObjectByPath(json, true, "/policies/c1[0]/shard"));
-    CollectionAdminRequest.createCollectionWithImplicitRouter("policiesTest", "conf", "s1,s2",
1)
+
+    final String collectionName = "addshard_using_policy";
+    CollectionAdminRequest.createCollectionWithImplicitRouter(collectionName, "conf", "s1,s2",
1)
         .setPolicy("c1")
         .process(cluster.getSolrClient());
 
-    DocCollection coll = getCollectionState("policiesTest");
+    cluster.waitForActiveCollection(collectionName, 2, 2);
+    DocCollection coll = getCollectionState(collectionName);
     assertEquals("c1", coll.getPolicyName());
     assertEquals(2,coll.getReplicas().size());
     coll.forEachReplica((s, replica) -> assertEquals(jetty.getNodeName(), replica.getNodeName()));
-    CollectionAdminRequest.createShard("policiesTest", "s3").process(cluster.getSolrClient());
-    coll = getCollectionState("policiesTest");
+    
+    CollectionAdminRequest.createShard(collectionName, "s3").process(cluster.getSolrClient());
+
+    cluster.waitForActiveCollection(collectionName, 3, 3);
+
+    coll = getCollectionState(collectionName);
     assertEquals(1, coll.getSlice("s3").getReplicas().size());
     coll.getSlice("s3").forEach(replica -> assertEquals(jetty.getNodeName(), replica.getNodeName()));
   }
 
-  public void testDataProvider() throws IOException, SolrServerException, KeeperException,
InterruptedException {
-    CollectionAdminRequest.createCollectionWithImplicitRouter("policiesTest", "conf", "shard1",
2)
+  public void testDataProvider() throws Exception {
+    final String collectionName = "data_provider";
+    CollectionAdminRequest.createCollectionWithImplicitRouter(collectionName, "conf", "shard1",
2)
         .process(cluster.getSolrClient());
     
-    cluster.waitForActiveCollection("policiesTest", 1, 2);
+    cluster.waitForActiveCollection(collectionName, 1, 2);
     
-    DocCollection rulesCollection = getCollectionState("policiesTest");
+    DocCollection rulesCollection = getCollectionState(collectionName);
 
     try (SolrCloudManager cloudManager = new SolrClientCloudManager(new ZkDistributedQueueFactory(cluster.getZkClient()),
cluster.getSolrClient())) {
       Map<String, Object> val = cloudManager.getNodeStateProvider().getNodeValues(rulesCollection.getReplicas().get(0).getNodeName(),
Arrays.asList(


Mime
View raw message