You are viewing a plain text version of this content. The canonical link for it is here.
Posted to dev@lucene.apache.org by "Jeff Miller (JIRA)" <ji...@apache.org> on 2017/03/08 23:00:40 UTC

[jira] [Commented] (SOLR-10249) Allow index fetching to return a detailed result instead of a true/false value

    [ https://issues.apache.org/jira/browse/SOLR-10249?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=15902139#comment-15902139 ] 

Jeff Miller commented on SOLR-10249:
------------------------------------

Diffs for my local testing

diff --git a/solr/core/src/java/org/apache/solr/cloud/RecoveryStrategy.java b/solr/core/src/java/org/apache/solr/cloud/RecoveryStrategy.java
index 90e515a..2483e69 100644
--- a/solr/core/src/java/org/apache/solr/cloud/RecoveryStrategy.java
+++ b/solr/core/src/java/org/apache/solr/cloud/RecoveryStrategy.java
@@ -153,7 +153,7 @@ public class RecoveryStrategy extends Thread implements Closeable {
     solrParams.set(ReplicationHandler.MASTER_URL, leaderUrl);
     
     if (isClosed()) return; // we check closed on return
-    boolean success = replicationHandler.doFetch(solrParams, false);
+    boolean success = replicationHandler.doFetch(solrParams, false).getStatus();
     
     if (!success) {
       throw new SolrException(ErrorCode.SERVER_ERROR, "Replication for recovery failed.");
diff --git a/solr/core/src/java/org/apache/solr/handler/CdcrRequestHandler.java b/solr/core/src/java/org/apache/solr/handler/CdcrRequestHandler.java
index f706637..a65299a 100644
--- a/solr/core/src/java/org/apache/solr/handler/CdcrRequestHandler.java
+++ b/solr/core/src/java/org/apache/solr/handler/CdcrRequestHandler.java
@@ -754,7 +754,7 @@ public class CdcrRequestHandler extends RequestHandlerBase implements SolrCoreAw
         // we do not want the raw tlog files from the source
         solrParams.set(ReplicationHandler.TLOG_FILES, false);
 
-        success = replicationHandler.doFetch(solrParams, false);
+        success = replicationHandler.doFetch(solrParams, false).getStatus();
 
         // this is required because this callable can race with HttpSolrCall#destroy
         // which clears the request info.
diff --git a/solr/core/src/java/org/apache/solr/handler/IndexFetcher.java b/solr/core/src/java/org/apache/solr/handler/IndexFetcher.java
index b9d9f51..281e660 100644
--- a/solr/core/src/java/org/apache/solr/handler/IndexFetcher.java
+++ b/solr/core/src/java/org/apache/solr/handler/IndexFetcher.java
@@ -106,6 +106,8 @@ import static org.apache.solr.common.params.CommonParams.JAVABIN;
 import static org.apache.solr.common.params.CommonParams.NAME;
 import static org.apache.solr.handler.ReplicationHandler.*;
 
+import com.google.common.base.Strings;
+
 /**
  * <p> Provides functionality of downloading changed index files as well as config files and a timer for scheduling fetches from the
  * master. </p>
@@ -161,6 +163,52 @@ public class IndexFetcher {
 
   private Integer soTimeout;
 
+  private static final String INTERRUPT_RESPONSE_MESSAGE = "Interrupted while waiting for modify lock";
+
+  public static class IndexFetchResult {
+    private final String message;
+    private final boolean status;
+    private final Throwable exception;
+
+    public static final String FAILED_BY_INTERRUPT_MESSAGE = "Fetching index failed by interrupt";
+    public static final String FAILED_BY_EXCEPTION_MESSAGE = "Fetching index failed by exception";
+
+    /** pre-defined results */
+    public static final IndexFetchResult ALREADY_IN_SYNC = new IndexFetchResult("Local index commit is already in sync with peer", true, null);
+    public static final IndexFetchResult INDEX_FETCH_FAILURE = new IndexFetchResult("Fetching lastest index is failed", false, null);
+    public static final IndexFetchResult INDEX_FETCH_SUCCESS = new IndexFetchResult("Fetching latest index is successful", true, null);
+    public static final IndexFetchResult LOCK_OBTAIN_FAILED = new IndexFetchResult("Obtaining SnapPuller lock failed", false, null);
+    public static final IndexFetchResult MASTER_VERSION_ZERO = new IndexFetchResult("Index in peer is empty and never committed yet", true, null);
+    public static final IndexFetchResult NO_INDEX_COMMIT_EXIST = new IndexFetchResult("No IndexCommit in local index", false, null);
+    public static final IndexFetchResult PEER_INDEX_COMMIT_DELETED = new IndexFetchResult("No files to download because IndexCommit in peer was deleted", false, null);
+    // SFDC: adding a new failure result when replication is aborted because of local activity
+    public static final IndexFetchResult LOCAL_ACTIVITY_DURING_REPLICATION = new IndexFetchResult("Local index modification during replication", false, null);
+
+    IndexFetchResult(String message, boolean status, Throwable exception) {
+      this.message = message;
+      this.status = status;
+      this.exception = exception;
+    }
+
+    /*
+     * @return exception thrown if failed by exception or interrupt, otherwise null
+     */
+    public Throwable getException() {
+      return this.exception;
+    }
+
+    /*
+     * @return true if index fetch was successful, false otherwise
+     */
+    public boolean getStatus() {
+      return this.status;
+    }
+
+    public String getMessage() {
+      return this.message;
+    }
+  }
+
   private static HttpClient createHttpClient(SolrCore core, String httpBasicAuthUser, String httpBasicAuthPassword, boolean useCompression) {
     final ModifiableSolrParams httpClientParams = new ModifiableSolrParams();
     httpClientParams.set(HttpClientUtil.PROP_BASIC_AUTH_USER, httpBasicAuthUser);
@@ -264,7 +312,7 @@ public class IndexFetcher {
     }
   }
 
-  boolean fetchLatestIndex(boolean forceReplication) throws IOException, InterruptedException {
+  IndexFetchResult fetchLatestIndex(boolean forceReplication) throws IOException, InterruptedException {
     return fetchLatestIndex(forceReplication, false);
   }
 
@@ -277,7 +325,7 @@ public class IndexFetcher {
    * @return true on success, false if slave is already in sync
    * @throws IOException if an exception occurs
    */
-  boolean fetchLatestIndex(boolean forceReplication, boolean forceCoreReload) throws IOException, InterruptedException {
+  IndexFetchResult fetchLatestIndex(boolean forceReplication, boolean forceCoreReload) throws IOException, InterruptedException {
 
     boolean cleanupDone = false;
     boolean successfulInstall = false;
@@ -302,9 +350,16 @@ public class IndexFetcher {
       try {
         response = getLatestVersion();
       } catch (Exception e) {
-        LOG.error("Master at: " + masterUrl + " is not available. Index fetch failed. Exception: " + e.getMessage());
-        return false;
-      }
+        final String errorMsg = e.getMessage();
+        if (!Strings.isNullOrEmpty(errorMsg) && errorMsg.contains(INTERRUPT_RESPONSE_MESSAGE)) {
+            LOG.warn("Master at: " + masterUrl + " is not available. Index fetch failed by interrupt. Exception: " + errorMsg);
+            return new IndexFetchResult(IndexFetchResult.FAILED_BY_INTERRUPT_MESSAGE, false, e);
+        } else {
+            LOG.warn("Master at: " + masterUrl + " is not available. Index fetch failed by exception: " + errorMsg);
+            return new IndexFetchResult(IndexFetchResult.FAILED_BY_EXCEPTION_MESSAGE, false, e);
+        }
+    }
+
       long latestVersion = (Long) response.get(CMD_INDEX_VERSION);
       long latestGeneration = (Long) response.get(GENERATION);
 
@@ -320,7 +375,7 @@ public class IndexFetcher {
           searcherRefCounted = solrCore.getNewestSearcher(false);
           if (searcherRefCounted == null) {
             LOG.warn("No open searcher found - fetch aborted");
-            return false;
+            return IndexFetchResult.NO_INDEX_COMMIT_EXIST;
           }
           commit = searcherRefCounted.get().getIndexReader().getIndexCommit();
         } finally {
@@ -347,7 +402,7 @@ public class IndexFetcher {
 
         //there is nothing to be replicated
         successfulInstall = true;
-        return true;
+        return IndexFetchResult.MASTER_VERSION_ZERO;
       }
 
       // TODO: Should we be comparing timestamps (across machines) here?
@@ -355,14 +410,14 @@ public class IndexFetcher {
         //master and slave are already in sync just return
         LOG.info("Slave in sync with master.");
         successfulInstall = true;
-        return true;
+        return IndexFetchResult.ALREADY_IN_SYNC;
       }
       LOG.info("Starting replication process");
       // get the list of files first
       fetchFileList(latestGeneration);
       // this can happen if the commit point is deleted before we fetch the file list.
       if (filesToDownload.isEmpty()) {
-        return false;
+        return IndexFetchResult.PEER_INDEX_COMMIT_DELETED;
       }
       LOG.info("Number of files in latest index in master: " + filesToDownload.size());
       if (tlogFilesToDownload != null) {
@@ -550,14 +605,14 @@ public class IndexFetcher {
           LOG.warn(
               "Replication attempt was not successful - trying a full index replication reloadCore={}",
               reloadCore);
-          successfulInstall = fetchLatestIndex(true, reloadCore);
+          successfulInstall = fetchLatestIndex(true, reloadCore).getStatus();
         }
 
         markReplicationStop();
-        return successfulInstall;
+        return successfulInstall ? IndexFetchResult.INDEX_FETCH_SUCCESS : IndexFetchResult.INDEX_FETCH_FAILURE;
       } catch (ReplicationHandlerException e) {
         LOG.error("User aborted Replication");
-        return false;
+        return new IndexFetchResult(IndexFetchResult.FAILED_BY_EXCEPTION_MESSAGE, false, e);
       } catch (SolrException e) {
         throw e;
       } catch (InterruptedException e) {
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 84e1ba2..becec83 100644
--- a/solr/core/src/java/org/apache/solr/handler/ReplicationHandler.java
+++ b/solr/core/src/java/org/apache/solr/handler/ReplicationHandler.java
@@ -88,6 +88,7 @@ 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.request.SolrQueryRequest;
 import org.apache.solr.response.SolrQueryResponse;
 import org.apache.solr.search.SolrIndexSearcher;
@@ -381,10 +382,10 @@ public class ReplicationHandler extends RequestHandlerBase implements SolrCoreAw
 
   private volatile IndexFetcher currentIndexFetcher;
 
-  public boolean doFetch(SolrParams solrParams, boolean forceReplication) {
+  public IndexFetchResult doFetch(SolrParams solrParams, boolean forceReplication) {
     String masterUrl = solrParams == null ? null : solrParams.get(MASTER_URL);
     if (!indexFetchLock.tryLock())
-      return false;
+      return IndexFetchResult.LOCK_OBTAIN_FAILED;
     try {
       if (masterUrl != null) {
         if (currentIndexFetcher != null && currentIndexFetcher != pollingIndexFetcher) {
@@ -400,17 +401,16 @@ public class ReplicationHandler extends RequestHandlerBase implements SolrCoreAw
       if (currentIndexFetcher != pollingIndexFetcher) {
         currentIndexFetcher.destroy();
       }
+      return new IndexFetchResult(IndexFetchResult.FAILED_BY_EXCEPTION_MESSAGE, false, e);
     } finally {
       if (pollingIndexFetcher != null) {
        if( currentIndexFetcher != pollingIndexFetcher) {
          currentIndexFetcher.destroy();
        }
-        
         currentIndexFetcher = pollingIndexFetcher;
       }
       indexFetchLock.unlock();
     }
-    return false;
   }
 
   boolean isReplicating() {


> Allow index fetching to return a detailed result instead of a true/false value
> ------------------------------------------------------------------------------
>
>                 Key: SOLR-10249
>                 URL: https://issues.apache.org/jira/browse/SOLR-10249
>             Project: Solr
>          Issue Type: Improvement
>      Security Level: Public(Default Security Level. Issues are Public) 
>          Components: replication (java)
>    Affects Versions: 6.4.1
>         Environment: Any
>            Reporter: Jeff Miller
>            Priority: Trivial
>              Labels: easyfix, newbie
>             Fix For: 6.4
>
>   Original Estimate: 3h
>  Remaining Estimate: 3h
>
> This gives us the ability to see into why a replication might of failed and act on it if we need to.  We use this enhancement for logging conditions so we can quantify what is happening with replication, get success rates, etc.
> The idea is to create a public static class IndexFetchResult as an inner class to IndexFetcher that has strings that hold statuses that could occur while fetching an index.



--
This message was sent by Atlassian JIRA
(v6.3.15#6346)

---------------------------------------------------------------------
To unsubscribe, e-mail: dev-unsubscribe@lucene.apache.org
For additional commands, e-mail: dev-help@lucene.apache.org