You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@lucene.apache.org by va...@apache.org on 2018/04/12 17:16:17 UTC

lucene-solr:branch_7x: SOLR-12065: A successful restore collection should mark the shard state as active and not buffering

Repository: lucene-solr
Updated Branches:
  refs/heads/branch_7x dc7f841e3 -> b25ae6779


SOLR-12065: A successful restore collection should mark the shard state as active and not buffering

(cherry picked from commit 7a57ca8)


Project: http://git-wip-us.apache.org/repos/asf/lucene-solr/repo
Commit: http://git-wip-us.apache.org/repos/asf/lucene-solr/commit/b25ae677
Tree: http://git-wip-us.apache.org/repos/asf/lucene-solr/tree/b25ae677
Diff: http://git-wip-us.apache.org/repos/asf/lucene-solr/diff/b25ae677

Branch: refs/heads/branch_7x
Commit: b25ae6779c9ad875eabd8b3e2a72b7108e740a33
Parents: dc7f841
Author: Varun Thacker <va...@apache.org>
Authored: Thu Apr 12 08:18:35 2018 -0700
Committer: Varun Thacker <va...@apache.org>
Committed: Thu Apr 12 10:14:40 2018 -0700

----------------------------------------------------------------------
 solr/CHANGES.txt                                |  5 ++-
 .../solr/cloud/api/collections/RestoreCmd.java  | 26 +++++++++---
 .../AbstractCloudBackupRestoreTestCase.java     | 42 ++++++++++++++------
 3 files changed, 55 insertions(+), 18 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/b25ae677/solr/CHANGES.txt
----------------------------------------------------------------------
diff --git a/solr/CHANGES.txt b/solr/CHANGES.txt
index 0721dd8..d5d2234 100644
--- a/solr/CHANGES.txt
+++ b/solr/CHANGES.txt
@@ -123,6 +123,9 @@ Bug Fixes
 * SOLR-12190: Need to properly escape output in GraphMLResponseWriter. (yonik)
 
 * SOLR-12214: Leader may skip publish itself as ACTIVE when its last published state is DOWN (Cao Manh Dat)
+
+* SOLR-12065: A successful restore collection should mark the shard state as active and not buffering
+  (Rohit, Varun Thacker)
  
 Optimizations
 ----------------------
@@ -1971,7 +1974,7 @@ Bug Fixes
 
 * SOLR-11024: ParallelStream should set the StreamContext when constructing SolrStreams (Joel Bernstein)
 
-* SOLR-10908: CloudSolrStream.toExpression incorrectly handles fq clauses (Rohit Singh via Erick Erickson)
+* SOLR-10908: CloudSolrStream.toExpression incorrectly handles fq clauses (Rohit via Erick Erickson)
 
 * SOLR-11177: CoreContainer.load needs to send lazily loaded core descriptors to the proper list rather than send
   them all to the transient lists. (Erick Erickson)

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/b25ae677/solr/core/src/java/org/apache/solr/cloud/api/collections/RestoreCmd.java
----------------------------------------------------------------------
diff --git a/solr/core/src/java/org/apache/solr/cloud/api/collections/RestoreCmd.java b/solr/core/src/java/org/apache/solr/cloud/api/collections/RestoreCmd.java
index 1823fe3..ca7c2d6 100644
--- a/solr/core/src/java/org/apache/solr/cloud/api/collections/RestoreCmd.java
+++ b/solr/core/src/java/org/apache/solr/cloud/api/collections/RestoreCmd.java
@@ -258,7 +258,6 @@ public class RestoreCmd implements OverseerCollectionMessageHandler.Cmd {
           propMap.put(ASYNC, asyncId);
         }
         ocmh.addPropertyParams(message, propMap);
-
         ocmh.addReplica(clusterState, new ZkNodeProps(propMap), new NamedList(), null);
       }
 
@@ -272,11 +271,31 @@ public class RestoreCmd implements OverseerCollectionMessageHandler.Cmd {
         params.set(NAME, "snapshot." + slice.getName());
         params.set(CoreAdminParams.BACKUP_LOCATION, backupPath.toASCIIString());
         params.set(CoreAdminParams.BACKUP_REPOSITORY, repo);
-
         ocmh.sliceCmd(clusterState, params, null, slice, shardHandler, asyncId, requestMap);
       }
       ocmh.processResponses(new NamedList(), shardHandler, true, "Could not restore core", asyncId, requestMap);
 
+
+      for (Slice s: restoreCollection.getSlices()) {
+        for (Replica r : s.getReplicas()) {
+          String nodeName = r.getNodeName();
+          String coreNodeName = r.getCoreName();
+          Replica.State stateRep  = r.getState();
+
+          log.debug("Calling REQUESTAPPLYUPDATES on: nodeName={}, coreNodeName={}, state={}"
+              , nodeName, coreNodeName, stateRep.name());
+
+          ModifiableSolrParams params = new ModifiableSolrParams();
+          params.set(CoreAdminParams.ACTION, CoreAdminParams.CoreAdminAction.REQUESTAPPLYUPDATES.toString());
+          params.set(CoreAdminParams.NAME, coreNodeName);
+
+          ocmh.sendShardRequest(nodeName, params, shardHandler, asyncId, requestMap);
+        }
+
+        ocmh.processResponses(new NamedList(), shardHandler, true, "REQUESTAPPLYUPDATES calls did not succeed", asyncId, requestMap);
+
+      }
+
       //Mark all shards in ACTIVE STATE
       {
         HashMap<String, Object> propMap = new HashMap<>();
@@ -288,9 +307,6 @@ public class RestoreCmd implements OverseerCollectionMessageHandler.Cmd {
         inQueue.offer(Utils.toJSON(new ZkNodeProps(propMap)));
       }
 
-      //refresh the location copy of collection state
-      restoreCollection = zkStateReader.getClusterState().getCollection(restoreCollectionName);
-
       if (totalReplicasPerShard > 1) {
         log.info("Adding replicas to restored collection={}", restoreCollection);
         for (Slice slice : restoreCollection.getSlices()) {

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/b25ae677/solr/core/src/test/org/apache/solr/cloud/api/collections/AbstractCloudBackupRestoreTestCase.java
----------------------------------------------------------------------
diff --git a/solr/core/src/test/org/apache/solr/cloud/api/collections/AbstractCloudBackupRestoreTestCase.java b/solr/core/src/test/org/apache/solr/cloud/api/collections/AbstractCloudBackupRestoreTestCase.java
index 058814c..46e6faf 100644
--- a/solr/core/src/test/org/apache/solr/cloud/api/collections/AbstractCloudBackupRestoreTestCase.java
+++ b/solr/core/src/test/org/apache/solr/cloud/api/collections/AbstractCloudBackupRestoreTestCase.java
@@ -93,12 +93,12 @@ public abstract class AbstractCloudBackupRestoreTestCase extends SolrCloudTestCa
     replFactor = TestUtil.nextInt(random(), 1, 2);
     numTlogReplicas = TestUtil.nextInt(random(), 0, 1);
     numPullReplicas = TestUtil.nextInt(random(), 0, 1);
-    
+
     CollectionAdminRequest.Create create = isImplicit ?
-      // NOTE: use shard list with same # of shards as NUM_SHARDS; we assume this later
-      CollectionAdminRequest.createCollectionWithImplicitRouter(getCollectionName(), "conf1", "shard1,shard2", replFactor, numTlogReplicas, numPullReplicas) :
-      CollectionAdminRequest.createCollection(getCollectionName(), "conf1", NUM_SHARDS, replFactor, numTlogReplicas, numPullReplicas);
-    
+        // NOTE: use shard list with same # of shards as NUM_SHARDS; we assume this later
+        CollectionAdminRequest.createCollectionWithImplicitRouter(getCollectionName(), "conf1", "shard1,shard2", replFactor, numTlogReplicas, numPullReplicas) :
+        CollectionAdminRequest.createCollection(getCollectionName(), "conf1", NUM_SHARDS, replFactor, numTlogReplicas, numPullReplicas);
+
     if (NUM_SHARDS * (replFactor + numTlogReplicas + numPullReplicas) > cluster.getJettySolrRunners().size() || random().nextBoolean()) {
       create.setMaxShardsPerNode((int)Math.ceil(NUM_SHARDS * (replFactor + numTlogReplicas + numPullReplicas) / cluster.getJettySolrRunners().size()));//just to assert it survives the restoration
       if (doSplitShardOperation) {
@@ -122,7 +122,7 @@ public abstract class AbstractCloudBackupRestoreTestCase extends SolrCloudTestCa
     CloudSolrClient solrClient = cluster.getSolrClient();
     create.process(solrClient);
 
-    indexDocs(getCollectionName());
+    indexDocs(getCollectionName(), false);
 
     if (doSplitShardOperation) {
       // shard split the first shard
@@ -197,23 +197,29 @@ public abstract class AbstractCloudBackupRestoreTestCase extends SolrCloudTestCa
     return cluster.getSolrClient().getZkStateReader().getClusterState().getCollection(collectionName).getActiveSlices().size();
   }
 
-  private void indexDocs(String collectionName) throws Exception {
+  private int indexDocs(String collectionName, boolean useUUID) throws Exception {
     Random random = new Random(docsSeed);// use a constant seed for the whole test run so that we can easily re-index.
     int numDocs = random.nextInt(100);
     if (numDocs == 0) {
       log.info("Indexing ZERO test docs");
-      return;
+      return 0;
     }
+
     List<SolrInputDocument> docs = new ArrayList<>(numDocs);
     for (int i=0; i<numDocs; i++) {
       SolrInputDocument doc = new SolrInputDocument();
-      doc.addField("id", i);
+      doc.addField("id", ((useUUID == true) ? java.util.UUID.randomUUID().toString() : i));
       doc.addField("shard_s", "shard" + (1 + random.nextInt(NUM_SHARDS))); // for implicit router
       docs.add(doc);
     }
+
     CloudSolrClient client = cluster.getSolrClient();
-    client.add(collectionName, docs);// batch
+    client.add(collectionName, docs); //batch
     client.commit(collectionName);
+
+    log.info("Indexed {} docs to collection: {}", numDocs, collectionName);
+
+    return numDocs;
   }
 
   private void testBackupAndRestore(String collectionName) throws Exception {
@@ -257,7 +263,7 @@ public abstract class AbstractCloudBackupRestoreTestCase extends SolrCloudTestCa
         // may need to increase maxShardsPerNode (e.g. if it was shard split, then now we need more)
         restore.setMaxShardsPerNode((int)Math.ceil(backupCollection.getReplicas().size()/cluster.getJettySolrRunners().size()));
       }
-      
+
 
       if (rarely()) { // Try with createNodeSet configuration
         int nodeSetSize = cluster.getJettySolrRunners().size() / 2;
@@ -298,7 +304,7 @@ public abstract class AbstractCloudBackupRestoreTestCase extends SolrCloudTestCa
     //Re-index same docs (should be identical docs given same random seed) and test we have the same result.  Helps
     //  test we reconstituted the hash ranges / doc router.
     if (!(restoreCollection.getRouter() instanceof ImplicitDocRouter) && random().nextBoolean()) {
-      indexDocs(restoreCollectionName);
+      indexDocs(restoreCollectionName, false);
       assertEquals(origShardToDocCount, getShardToDocCountMap(client, restoreCollection));
     }
 
@@ -327,6 +333,18 @@ public abstract class AbstractCloudBackupRestoreTestCase extends SolrCloudTestCa
 
     assertEquals("Restore collection should use stateFormat=2", 2, restoreCollection.getStateFormat());
 
+    //SOLR-12605: Add more docs after restore is complete to see if they are getting added fine
+    //explicitly querying the leaders. If we use CloudSolrClient there is no guarantee that we'll hit a nrtReplica
+    {
+      Map<String, Integer> restoredCollectionPerShardCount =  getShardToDocCountMap(client, restoreCollection);
+      long restoredCollectionDocCount = restoredCollectionPerShardCount.values().stream().mapToInt(Number::intValue).sum();
+      int numberNewDocsIndexed = indexDocs(restoreCollectionName, true);
+      Map<String, Integer> restoredCollectionPerShardCountAfterIndexing = getShardToDocCountMap(client, restoreCollection);
+      int restoredCollectionFinalDocCount = restoredCollectionPerShardCountAfterIndexing.values().stream().mapToInt(Number::intValue).sum();
+
+      log.info("Original doc count in restored collection:" + restoredCollectionDocCount + ", number of newly added documents to the restored collection: " + numberNewDocsIndexed + ", after indexing: " + restoredCollectionFinalDocCount);
+      assertEquals((restoredCollectionDocCount + numberNewDocsIndexed), restoredCollectionFinalDocCount);
+    }
 
     // assert added core properties:
     // DWS: did via manual inspection.