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

[43/50] lucene-solr:jira/solr-11458-2: SOLR-11616: Snapshot the segments more robustly such that segments created during a backup does does not fail the operation

SOLR-11616: Snapshot the segments more robustly such that segments created during a backup does does not fail the
operation


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

Branch: refs/heads/jira/solr-11458-2
Commit: 864ce90d2cd9bfae66506f38823278738afe6c4a
Parents: 8c855fa
Author: Varun Thacker <va...@apache.org>
Authored: Fri Dec 1 18:38:09 2017 -0800
Committer: Varun Thacker <va...@apache.org>
Committed: Fri Dec 1 19:30:29 2017 -0800

----------------------------------------------------------------------
 solr/CHANGES.txt                                |  3 +
 .../apache/solr/handler/ReplicationHandler.java | 42 +++--------
 .../org/apache/solr/handler/SnapShooter.java    | 75 +++++++++++++-------
 .../apache/solr/handler/admin/BackupCoreOp.java |  6 --
 .../org/apache/solr/handler/TestCoreBackup.java | 61 ++++++++++++++++
 5 files changed, 121 insertions(+), 66 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/864ce90d/solr/CHANGES.txt
----------------------------------------------------------------------
diff --git a/solr/CHANGES.txt b/solr/CHANGES.txt
index 1387fd1..e02e1cf 100644
--- a/solr/CHANGES.txt
+++ b/solr/CHANGES.txt
@@ -156,6 +156,9 @@ Bug Fixes
 * SOLR-11256: The queue size for ConcurrentUpdateSolrClient should default to 10 instead of throwing an
   IllegalArgumentException. (Jason Gerlowski, Anshum Gupta)
 
+* SOLR-11616: Snapshot the segments more robustly such that segments created during a backup does does not fail the
+  operation (Varun Thacker)
+
 Optimizations
 ----------------------
 * SOLR-11285: Refactor autoscaling framework to avoid direct references to Zookeeper and Solr

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/864ce90d/solr/core/src/java/org/apache/solr/handler/ReplicationHandler.java
----------------------------------------------------------------------
diff --git a/solr/core/src/java/org/apache/solr/handler/ReplicationHandler.java b/solr/core/src/java/org/apache/solr/handler/ReplicationHandler.java
index e986960..0101559 100644
--- a/solr/core/src/java/org/apache/solr/handler/ReplicationHandler.java
+++ b/solr/core/src/java/org/apache/solr/handler/ReplicationHandler.java
@@ -82,7 +82,6 @@ import org.apache.solr.common.util.SimpleOrderedMap;
 import org.apache.solr.common.util.StrUtils;
 import org.apache.solr.common.util.SuppressForbidden;
 import org.apache.solr.core.CloseHook;
-import static org.apache.solr.core.Config.assertWarnOrFail;
 import org.apache.solr.core.CoreContainer;
 import org.apache.solr.core.DirectoryFactory.DirContext;
 import org.apache.solr.core.IndexDeletionPolicyWrapper;
@@ -91,10 +90,9 @@ import org.apache.solr.core.SolrDeletionPolicy;
 import org.apache.solr.core.SolrEventListener;
 import org.apache.solr.core.backup.repository.BackupRepository;
 import org.apache.solr.core.backup.repository.LocalFileSystemRepository;
-import org.apache.solr.core.snapshots.SolrSnapshotMetaDataManager;
+import org.apache.solr.handler.IndexFetcher.IndexFetchResult;
 import org.apache.solr.metrics.MetricsMap;
 import org.apache.solr.metrics.SolrMetricManager;
-import org.apache.solr.handler.IndexFetcher.IndexFetchResult;
 import org.apache.solr.request.SolrQueryRequest;
 import org.apache.solr.response.SolrQueryResponse;
 import org.apache.solr.search.SolrIndexSearcher;
@@ -111,6 +109,7 @@ import org.slf4j.LoggerFactory;
 import org.slf4j.MDC;
 
 import static org.apache.solr.common.params.CommonParams.NAME;
+import static org.apache.solr.core.Config.assertWarnOrFail;
 
 /**
  * <p> A Handler which provides a REST API for replication and serves replication requests from Slaves. </p>
@@ -530,41 +529,18 @@ public class ReplicationHandler extends RequestHandlerBase implements SolrCoreAw
     return status;
   }
 
-  private void doSnapShoot(SolrParams params, SolrQueryResponse rsp,
-      SolrQueryRequest req) {
+  private void doSnapShoot(SolrParams params, SolrQueryResponse rsp, SolrQueryRequest req) {
     try {
       int numberToKeep = params.getInt(NUMBER_BACKUPS_TO_KEEP_REQUEST_PARAM, 0);
       if (numberToKeep > 0 && numberBackupsToKeep > 0) {
-        throw new SolrException(ErrorCode.BAD_REQUEST, "Cannot use "
-            + NUMBER_BACKUPS_TO_KEEP_REQUEST_PARAM + " if "
-            + NUMBER_BACKUPS_TO_KEEP_INIT_PARAM
-            + " was specified in the configuration.");
+        throw new SolrException(ErrorCode.BAD_REQUEST, "Cannot use " + NUMBER_BACKUPS_TO_KEEP_REQUEST_PARAM +
+            " if " + NUMBER_BACKUPS_TO_KEEP_INIT_PARAM + " was specified in the configuration.");
       }
       numberToKeep = Math.max(numberToKeep, numberBackupsToKeep);
       if (numberToKeep < 1) {
         numberToKeep = Integer.MAX_VALUE;
       }
 
-      IndexCommit indexCommit = null;
-      String commitName = params.get(CoreAdminParams.COMMIT_NAME);
-      if (commitName != null) {
-        SolrSnapshotMetaDataManager snapshotMgr = core.getSnapshotMetaDataManager();
-        Optional<IndexCommit> commit = snapshotMgr.getIndexCommitByName(commitName);
-        if(commit.isPresent()) {
-          indexCommit = commit.get();
-        } else {
-          throw new SolrException(ErrorCode.BAD_REQUEST, "Unable to find an index commit with name " + commitName +
-              " for core " + core.getName());
-        }
-      } else {
-        IndexDeletionPolicyWrapper delPolicy = core.getDeletionPolicy();
-        indexCommit = delPolicy.getLatestCommit();
-
-        if (indexCommit == null) {
-          indexCommit = req.getSearcher().getIndexReader().getIndexCommit();
-        }
-      }
-
       String location = params.get(CoreAdminParams.BACKUP_LOCATION);
       String repoName = params.get(CoreAdminParams.BACKUP_REPOSITORY);
       CoreContainer cc = core.getCoreContainer();
@@ -586,12 +562,12 @@ public class ReplicationHandler extends RequestHandlerBase implements SolrCoreAw
 
       // small race here before the commit point is saved
       URI locationUri = repo.createURI(location);
+      String commitName = params.get(CoreAdminParams.COMMIT_NAME);
       SnapShooter snapShooter = new SnapShooter(repo, core, locationUri, params.get(NAME), commitName);
       snapShooter.validateCreateSnapshot();
-      snapShooter.createSnapAsync(indexCommit, numberToKeep, (nl) -> snapShootDetails = nl);
-
+      snapShooter.createSnapAsync(numberToKeep, (nl) -> snapShootDetails = nl);
     } catch (Exception e) {
-      LOG.warn("Exception during creating a snapshot", e);
+      LOG.error("Exception during creating a snapshot", e);
       rsp.add("exception", e);
     }
   }
@@ -1441,7 +1417,7 @@ public class ReplicationHandler extends RequestHandlerBase implements SolrCoreAw
             }
             SnapShooter snapShooter = new SnapShooter(core, null, null);
             snapShooter.validateCreateSnapshot();
-            snapShooter.createSnapAsync(currentCommitPoint, numberToKeep, (nl) -> snapShootDetails = nl);
+            snapShooter.createSnapAsync(numberToKeep, (nl) -> snapShootDetails = nl);
           } catch (Exception e) {
             LOG.error("Exception while snapshooting", e);
           }

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/864ce90d/solr/core/src/java/org/apache/solr/handler/SnapShooter.java
----------------------------------------------------------------------
diff --git a/solr/core/src/java/org/apache/solr/handler/SnapShooter.java b/solr/core/src/java/org/apache/solr/handler/SnapShooter.java
index a6e8110..79634f9 100644
--- a/solr/core/src/java/org/apache/solr/handler/SnapShooter.java
+++ b/solr/core/src/java/org/apache/solr/handler/SnapShooter.java
@@ -43,8 +43,6 @@ import org.apache.solr.core.backup.repository.BackupRepository;
 import org.apache.solr.core.backup.repository.BackupRepository.PathType;
 import org.apache.solr.core.backup.repository.LocalFileSystemRepository;
 import org.apache.solr.core.snapshots.SolrSnapshotMetaDataManager;
-import org.apache.solr.search.SolrIndexSearcher;
-import org.apache.solr.util.RefCounted;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
@@ -151,35 +149,57 @@ public class SnapShooter {
   }
 
   public NamedList createSnapshot() throws Exception {
-    RefCounted<SolrIndexSearcher> searcher = solrCore.getSearcher();
-    try {
-      if (commitName != null) {
-        SolrSnapshotMetaDataManager snapshotMgr = solrCore.getSnapshotMetaDataManager();
-        Optional<IndexCommit> commit = snapshotMgr.getIndexCommitByName(commitName);
-        if(commit.isPresent()) {
-          return createSnapshot(commit.get());
-        }
-        throw new SolrException(ErrorCode.SERVER_ERROR, "Unable to find an index commit with name " + commitName +
-            " for core " + solrCore.getName());
-      } else {
-        //TODO should we try solrCore.getDeletionPolicy().getLatestCommit() first?
-        IndexDeletionPolicyWrapper deletionPolicy = solrCore.getDeletionPolicy();
-        IndexCommit indexCommit = searcher.get().getIndexReader().getIndexCommit();
-        deletionPolicy.saveCommitPoint(indexCommit.getGeneration());
-        try {
-          return createSnapshot(indexCommit);
-        } finally {
-          deletionPolicy.releaseCommitPoint(indexCommit.getGeneration());
-        }
+    IndexCommit indexCommit;
+    if (commitName != null) {
+      indexCommit = getIndexCommitFromName();
+      return createSnapshot(indexCommit);
+    } else {
+      indexCommit = getIndexCommit();
+      IndexDeletionPolicyWrapper deletionPolicy = solrCore.getDeletionPolicy();
+      deletionPolicy.saveCommitPoint(indexCommit.getGeneration());
+      try {
+        return createSnapshot(indexCommit);
+      } finally {
+        deletionPolicy.releaseCommitPoint(indexCommit.getGeneration());
       }
-    } finally {
-      searcher.decref();
     }
   }
 
-  public void createSnapAsync(final IndexCommit indexCommit, final int numberToKeep, Consumer<NamedList> result) {
-    solrCore.getDeletionPolicy().saveCommitPoint(indexCommit.getGeneration());
+  private IndexCommit getIndexCommit() throws IOException {
+    IndexDeletionPolicyWrapper delPolicy = solrCore.getDeletionPolicy();
+    IndexCommit indexCommit = delPolicy.getLatestCommit();
+
+    if (indexCommit == null) {
+      indexCommit = solrCore.getSearcher().get().getIndexReader().getIndexCommit();
+    }
+    return indexCommit;
+  }
+
+  private IndexCommit getIndexCommitFromName() throws IOException {
+    assert commitName !=null;
+    IndexCommit indexCommit;
+    SolrSnapshotMetaDataManager snapshotMgr = solrCore.getSnapshotMetaDataManager();
+    Optional<IndexCommit> commit = snapshotMgr.getIndexCommitByName(commitName);
+    if (commit.isPresent()) {
+      indexCommit = commit.get();
+    } else {
+      throw new SolrException(ErrorCode.BAD_REQUEST, "Unable to find an index commit with name " + commitName +
+          " for core " + solrCore.getName());
+    }
+    return indexCommit;
+  }
+
+  public void createSnapAsync(final int numberToKeep, Consumer<NamedList> result) throws IOException {
+    IndexCommit indexCommit;
+    if (commitName != null) {
+      indexCommit = getIndexCommitFromName();
+    } else {
+      indexCommit = getIndexCommit();
+    }
+    createSnapAsync(indexCommit, numberToKeep, result);
+  }
 
+  private void createSnapAsync(final IndexCommit indexCommit, final int numberToKeep, Consumer<NamedList> result) {
     //TODO should use Solr's ExecutorUtil
     new Thread(() -> {
       try {
@@ -187,7 +207,7 @@ public class SnapShooter {
       } catch (Exception e) {
         LOG.error("Exception while creating snapshot", e);
         NamedList snapShootDetails = new NamedList<>();
-        snapShootDetails.add("snapShootException", e.getMessage());
+        snapShootDetails.add("exception", e.getMessage());
         result.accept(snapShootDetails);
       } finally {
         solrCore.getDeletionPolicy().releaseCommitPoint(indexCommit.getGeneration());
@@ -205,6 +225,7 @@ public class SnapShooter {
 
   // note: remember to reserve the indexCommit first so it won't get deleted concurrently
   protected NamedList createSnapshot(final IndexCommit indexCommit) throws Exception {
+    assert indexCommit != null;
     LOG.info("Creating backup snapshot " + (snapshotName == null ? "<not named>" : snapshotName) + " at " + baseSnapDirPath);
     boolean success = false;
     try {

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/864ce90d/solr/core/src/java/org/apache/solr/handler/admin/BackupCoreOp.java
----------------------------------------------------------------------
diff --git a/solr/core/src/java/org/apache/solr/handler/admin/BackupCoreOp.java b/solr/core/src/java/org/apache/solr/handler/admin/BackupCoreOp.java
index e083cfd..3f8e1f7 100644
--- a/solr/core/src/java/org/apache/solr/handler/admin/BackupCoreOp.java
+++ b/solr/core/src/java/org/apache/solr/handler/admin/BackupCoreOp.java
@@ -20,7 +20,6 @@ package org.apache.solr.handler.admin;
 import java.net.URI;
 import java.util.Optional;
 
-import org.apache.solr.cloud.ZkController;
 import org.apache.solr.common.SolrException;
 import org.apache.solr.common.params.CoreAdminParams;
 import org.apache.solr.common.params.SolrParams;
@@ -34,11 +33,6 @@ import static org.apache.solr.common.params.CommonParams.NAME;
 class BackupCoreOp implements CoreAdminHandler.CoreAdminOp {
   @Override
   public void execute(CoreAdminHandler.CallInfo it) throws Exception {
-    ZkController zkController = it.handler.coreContainer.getZkController();
-    if (zkController == null) {
-      throw new SolrException(SolrException.ErrorCode.BAD_REQUEST, "Internal SolrCloud API");
-    }
-
     final SolrParams params = it.req.getParams();
     String cname = params.get(CoreAdminParams.CORE);
     if (cname == null) {

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/864ce90d/solr/core/src/test/org/apache/solr/handler/TestCoreBackup.java
----------------------------------------------------------------------
diff --git a/solr/core/src/test/org/apache/solr/handler/TestCoreBackup.java b/solr/core/src/test/org/apache/solr/handler/TestCoreBackup.java
new file mode 100644
index 0000000..5d707dc
--- /dev/null
+++ b/solr/core/src/test/org/apache/solr/handler/TestCoreBackup.java
@@ -0,0 +1,61 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.solr.handler;
+
+import org.apache.lucene.util.TestUtil;
+import org.apache.solr.SolrTestCaseJ4;
+import org.apache.solr.common.params.CoreAdminParams;
+import org.apache.solr.core.CoreContainer;
+import org.apache.solr.handler.admin.CoreAdminHandler;
+import org.apache.solr.response.SolrQueryResponse;
+import org.junit.BeforeClass;
+import org.junit.Test;
+
+public class TestCoreBackup extends SolrTestCaseJ4 {
+
+  @BeforeClass
+  public static void beforeClass() throws Exception {
+    initCore("solrconfig.xml", "schema.xml");
+  }
+
+  @Test
+  public void testBackupWithDocsNotSearchable() throws Exception {
+    //See SOLR-11616 to see when this issue can be triggered
+
+    assertU(adoc("id", "1"));
+    assertU(commit());
+
+    assertU(adoc("id", "2"));
+
+    assertU(commit("openSearcher", "false"));
+    assertQ(req("q", "*:*"), "//result[@numFound='1']");
+
+    //call backup
+    String location = createTempDir().toFile().getAbsolutePath();
+    String snapshotName = TestUtil.randomSimpleString(random(), 1, 5);
+
+    final CoreContainer cores = h.getCoreContainer();
+    final CoreAdminHandler admin = new CoreAdminHandler(cores);
+    SolrQueryResponse resp = new SolrQueryResponse();
+    admin.handleRequestBody
+        (req(CoreAdminParams.ACTION, CoreAdminParams.CoreAdminAction.BACKUPCORE.toString(),
+            "core", DEFAULT_TEST_COLLECTION_NAME, "name", snapshotName, "location", location)
+            , resp);
+    assertNull("Backup should have succeeded", resp.getException());
+
+  }
+}