You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@lucene.apache.org by da...@apache.org on 2018/12/07 09:57:21 UTC

[06/37] lucene-solr:jira/http2: SOLR-6117: Unify ReplicationHandler error handling

SOLR-6117: Unify ReplicationHandler error handling

Prior to this commit, ReplicationHandler had a few inconsistencies
in how it reported errors:
  - Sometimes the 'status' field was used as an enum (e.g. 'success'
    v. 'failure'.  Elsewhere it is used to hold error messages.
  - Sometimes an explicit 'message' field was used, but often not.
  - Sometimes a stack trace was provided in place of an error message.

This commit tweaks the various error cases in ReplicationHandler to
report errors consistently.  'status' is always an enum-type value.  A
'message' field is provided for all errors, with an optional 'exception'
field.


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

Branch: refs/heads/jira/http2
Commit: c951578fcabfd5520f425156b0fa35f35811554d
Parents: 16241f4
Author: Jason Gerlowski <ge...@apache.org>
Authored: Sun Dec 2 16:08:41 2018 -0500
Committer: Jason Gerlowski <ge...@apache.org>
Committed: Sun Dec 2 16:23:04 2018 -0500

----------------------------------------------------------------------
 solr/CHANGES.txt                                |   4 +
 .../apache/solr/handler/ReplicationHandler.java | 188 +++++++++++--------
 .../test/org/apache/solr/MinimalSchemaTest.java |   3 +-
 .../solr/handler/TestReplicationHandler.java    |  91 ++++++---
 4 files changed, 183 insertions(+), 103 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/c951578f/solr/CHANGES.txt
----------------------------------------------------------------------
diff --git a/solr/CHANGES.txt b/solr/CHANGES.txt
index 22b6721..d5240be 100644
--- a/solr/CHANGES.txt
+++ b/solr/CHANGES.txt
@@ -80,6 +80,10 @@ Improvements
  If you don't use nested documents then _root_ can be removed from the schema.  (Dr Oleg Savrasov, Moshe Bla,
  David Smiley, Mikhail Khludnev)
 
+* SOLR-6117: The response format has changed slightly for ReplicationHandler error-cases.  All errors now have a non-200
+  'status' field, a 'message' field giving more details on the error, and an optional 'exception' field.  (Shalin Mangar,
+  Jason Gerlowski)
+
 Optimizations
 ----------------------
 

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/c951578f/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 241b6cd..3dc53bf 100644
--- a/solr/core/src/java/org/apache/solr/handler/ReplicationHandler.java
+++ b/solr/core/src/java/org/apache/solr/handler/ReplicationHandler.java
@@ -240,12 +240,8 @@ public class ReplicationHandler extends RequestHandlerBase implements SolrCoreAw
   public void handleRequestBody(SolrQueryRequest req, SolrQueryResponse rsp) throws Exception {
     rsp.setHttpCaching(false);
     final SolrParams solrParams = req.getParams();
-    String command = solrParams.get(COMMAND);
-    if (command == null) {
-      rsp.add(STATUS, OK_STATUS);
-      rsp.add("message", "No command");
-      return;
-    }
+    String command = solrParams.required().get(COMMAND);
+
     // This command does not give the current index version of the master
     // It gives the current 'replicateable' index version
     if (command.equals(CMD_INDEX_VERSION)) {
@@ -267,11 +263,13 @@ public class ReplicationHandler extends RequestHandlerBase implements SolrCoreAw
         core.getDeletionPolicy().setReserveDuration(commitPoint.getGeneration(), reserveCommitDuration);
         rsp.add(CMD_INDEX_VERSION, IndexDeletionPolicyWrapper.getCommitTimestamp(commitPoint));
         rsp.add(GENERATION, commitPoint.getGeneration());
+        rsp.add(STATUS, OK_STATUS);
       } else {
         // This happens when replication is not configured to happen after startup and no commit/optimize
         // has happened yet.
         rsp.add(CMD_INDEX_VERSION, 0L);
         rsp.add(GENERATION, 0L);
+        rsp.add(STATUS, OK_STATUS);
       }
     } else if (command.equals(CMD_GET_FILE)) {
       getFileStream(solrParams, rsp);
@@ -279,57 +277,28 @@ public class ReplicationHandler extends RequestHandlerBase implements SolrCoreAw
       getFileList(solrParams, rsp);
     } else if (command.equalsIgnoreCase(CMD_BACKUP)) {
       doSnapShoot(new ModifiableSolrParams(solrParams), rsp, req);
-      rsp.add(STATUS, OK_STATUS);
     } else if (command.equalsIgnoreCase(CMD_RESTORE)) {
       restore(new ModifiableSolrParams(solrParams), rsp, req);
-      rsp.add(STATUS, OK_STATUS);
     } else if (command.equalsIgnoreCase(CMD_RESTORE_STATUS)) {
-      rsp.add(CMD_RESTORE_STATUS, getRestoreStatus());
+      populateRestoreStatus(rsp);
     } else if (command.equalsIgnoreCase(CMD_DELETE_BACKUP)) {
-      deleteSnapshot(new ModifiableSolrParams(solrParams));
-      rsp.add(STATUS, OK_STATUS);
+      deleteSnapshot(new ModifiableSolrParams(solrParams), rsp);
     } else if (command.equalsIgnoreCase(CMD_FETCH_INDEX)) {
-      String masterUrl = solrParams.get(MASTER_URL);
-      if (!isSlave && masterUrl == null) {
-        rsp.add(STATUS,ERR_STATUS);
-        rsp.add("message","No slave configured or no 'masterUrl' Specified");
-        return;
-      }
-      final SolrParams paramsCopy = new ModifiableSolrParams(solrParams);
-      Thread fetchThread = new Thread(() -> doFetch(paramsCopy, false), "explicit-fetchindex-cmd") ;
-      fetchThread.setDaemon(false);
-      fetchThread.start();
-      if (solrParams.getBool(WAIT, false)) {
-        fetchThread.join();
-      }
-      rsp.add(STATUS, OK_STATUS);
+      fetchIndex(solrParams, rsp);
     } else if (command.equalsIgnoreCase(CMD_DISABLE_POLL)) {
-      if (pollingIndexFetcher != null){
-        disablePoll();
-        rsp.add(STATUS, OK_STATUS);
-      } else {
-        rsp.add(STATUS, ERR_STATUS);
-        rsp.add("message","No slave configured");
-      }
+      disablePoll(rsp);
     } else if (command.equalsIgnoreCase(CMD_ENABLE_POLL)) {
-      if (pollingIndexFetcher != null){
-        enablePoll();
-        rsp.add(STATUS, OK_STATUS);
-      }else {
-        rsp.add(STATUS,ERR_STATUS);
-        rsp.add("message","No slave configured");
-      }
+      enablePoll(rsp);
     } else if (command.equalsIgnoreCase(CMD_ABORT_FETCH)) {
-      if (abortFetch()){
+      if (abortFetch()) {
         rsp.add(STATUS, OK_STATUS);
       } else {
-        rsp.add(STATUS,ERR_STATUS);
-        rsp.add("message","No slave configured");
+        reportErrorOnResponse(rsp, "No slave configured", null);
       }
     } else if (command.equals(CMD_SHOW_COMMITS)) {
-      rsp.add(CMD_SHOW_COMMITS, getCommits());
+      populateCommitInfo(rsp);
     } else if (command.equals(CMD_DETAILS)) {
-      rsp.add(CMD_DETAILS, getReplicationDetails(solrParams.getBool("slave", true)));
+      getReplicationDetails(rsp, solrParams.getBool("slave", true));
     } else if (CMD_ENABLE_REPL.equalsIgnoreCase(command)) {
       replicationEnabled.set(true);
       rsp.add(STATUS, OK_STATUS);
@@ -339,6 +308,14 @@ public class ReplicationHandler extends RequestHandlerBase implements SolrCoreAw
     }
   }
 
+  private void reportErrorOnResponse(SolrQueryResponse response, String message, Exception e) {
+    response.add(STATUS, ERR_STATUS);
+    response.add(MESSAGE, message);
+    if (e != null) {
+      response.add(EXCEPTION, e);
+    }
+  }
+
   public boolean abortFetch() {
     IndexFetcher fetcher = currentIndexFetcher;
     if (fetcher != null){
@@ -349,15 +326,41 @@ public class ReplicationHandler extends RequestHandlerBase implements SolrCoreAw
     }
   }
 
-  private void deleteSnapshot(ModifiableSolrParams params) {
-    String name = params.get(NAME);
-    if(name == null) {
-      throw new SolrException(ErrorCode.BAD_REQUEST, "Missing mandatory param: name");
-    }
+  private void deleteSnapshot(ModifiableSolrParams params, SolrQueryResponse rsp) {
+    String name = params.required().get(NAME);
 
     SnapShooter snapShooter = new SnapShooter(core, params.get(CoreAdminParams.BACKUP_LOCATION), params.get(NAME));
     snapShooter.validateDeleteSnapshot();
     snapShooter.deleteSnapAsync(this);
+    rsp.add(STATUS, OK_STATUS);
+  }
+
+  private void fetchIndex(SolrParams solrParams, SolrQueryResponse rsp) throws InterruptedException {
+    String masterUrl = solrParams.get(MASTER_URL);
+    if (!isSlave && masterUrl == null) {
+      reportErrorOnResponse(rsp, "No slave configured or no 'masterUrl' specified", null);
+      return;
+    }
+    final SolrParams paramsCopy = new ModifiableSolrParams(solrParams);
+    final IndexFetchResult[] results = new IndexFetchResult[1];
+    Thread fetchThread = new Thread(() -> {
+      IndexFetchResult result = doFetch(paramsCopy, false);
+      results[0] = result;
+    }, "explicit-fetchindex-cmd") ;
+    fetchThread.setDaemon(false);
+    fetchThread.start();
+    if (solrParams.getBool(WAIT, false)) {
+      fetchThread.join();
+      if (results[0] == null) {
+        reportErrorOnResponse(rsp, "Unable to determine result of synchronous index fetch", null);
+      } else if (results[0].getSuccessful()) {
+        rsp.add(STATUS, OK_STATUS);
+      } else {
+        reportErrorOnResponse(rsp, results[0].getMessage(), null);
+      }
+    } else {
+      rsp.add(STATUS, OK_STATUS);
+    }
   }
 
   private List<NamedList<Object>> getCommits() {
@@ -496,6 +499,7 @@ public class ReplicationHandler extends RequestHandlerBase implements SolrCoreAw
       MDC.put("RestoreCore.backupName", name);
       restoreFuture = restoreExecutor.submit(restoreCore);
       currentRestoreName = name;
+      rsp.add(STATUS, OK_STATUS);
     } finally {
       MDC.remove("RestoreCore.core");
       MDC.remove("RestoreCore.backupLocation");
@@ -503,31 +507,42 @@ public class ReplicationHandler extends RequestHandlerBase implements SolrCoreAw
     }
   }
 
-  private NamedList<Object> getRestoreStatus() {
-    NamedList<Object> status = new SimpleOrderedMap<>();
-
+  private void populateRestoreStatus(SolrQueryResponse rsp) {
+    NamedList<Object> restoreStatus = new SimpleOrderedMap<>();
     if (restoreFuture == null) {
-      status.add(STATUS, "No restore actions in progress");
-      return status;
+      restoreStatus.add(STATUS, "No restore actions in progress");
+      rsp.add(CMD_RESTORE_STATUS, restoreStatus);
+      rsp.add(STATUS, OK_STATUS);
+      return;
     }
 
-    status.add("snapshotName", currentRestoreName);
+    restoreStatus.add("snapshotName", currentRestoreName);
     if (restoreFuture.isDone()) {
       try {
         boolean success = restoreFuture.get();
         if (success) {
-          status.add(STATUS, SUCCESS);
+          restoreStatus.add(STATUS, SUCCESS);
         } else {
-          status.add(STATUS, FAILED);
+          restoreStatus.add(STATUS, FAILED);
         }
       } catch (Exception e) {
-        status.add(STATUS, FAILED);
-        status.add(EXCEPTION, e.getMessage());
+        restoreStatus.add(STATUS, FAILED);
+        restoreStatus.add(EXCEPTION, e.getMessage());
+        rsp.add(CMD_RESTORE_STATUS, restoreStatus);
+        reportErrorOnResponse(rsp, "Unable to read restorestatus", e);
+        return;
       }
     } else {
-      status.add(STATUS, "In Progress");
+      restoreStatus.add(STATUS, "In Progress");
     }
-    return status;
+
+    rsp.add(CMD_RESTORE_STATUS, restoreStatus);
+    rsp.add(STATUS, OK_STATUS);
+  }
+
+  private void populateCommitInfo(SolrQueryResponse rsp) {
+    rsp.add(CMD_SHOW_COMMITS, getCommits());
+    rsp.add(STATUS, OK_STATUS);
   }
 
   private void doSnapShoot(SolrParams params, SolrQueryResponse rsp, SolrQueryRequest req) {
@@ -567,9 +582,12 @@ public class ReplicationHandler extends RequestHandlerBase implements SolrCoreAw
       SnapShooter snapShooter = new SnapShooter(repo, core, locationUri, params.get(NAME), commitName);
       snapShooter.validateCreateSnapshot();
       snapShooter.createSnapAsync(numberToKeep, (nl) -> snapShootDetails = nl);
+      rsp.add(STATUS, OK_STATUS);
+    } catch (SolrException e) {
+      throw e;
     } catch (Exception e) {
-      log.error("Exception during creating a snapshot", e);
-      rsp.add("exception", e);
+      log.error("Exception while creating a snapshot", e);
+      reportErrorOnResponse(rsp, "Error encountered while creating a snapshot: " + e.getMessage(), e);
     }
   }
 
@@ -593,21 +611,17 @@ public class ReplicationHandler extends RequestHandlerBase implements SolrCoreAw
     } else {
       rsp.add(FILE_STREAM, new DirectoryFileStream(solrParams));
     }
+    rsp.add(STATUS, OK_STATUS);
   }
 
   @SuppressWarnings("unchecked")
   private void getFileList(SolrParams solrParams, SolrQueryResponse rsp) {
-    String v = solrParams.get(GENERATION);
-    if (v == null) {
-      rsp.add("status", "no index generation specified");
-      return;
-    }
+    String v = solrParams.required().get(GENERATION);
     long gen = Long.parseLong(v);
     IndexCommit commit = core.getDeletionPolicy().getCommitPoint(gen);
 
-    //System.out.println("ask for files for gen:" + commit.getGeneration() + core.getCoreDescriptor().getCoreContainer().getZkController().getNodeName());
     if (commit == null) {
-      rsp.add("status", "invalid index generation");
+      reportErrorOnResponse(rsp, "invalid index generation", null);
       return;
     }
 
@@ -629,6 +643,7 @@ public class ReplicationHandler extends RequestHandlerBase implements SolrCoreAw
               long checksum = CodecUtil.retrieveChecksum(in);
               fileMeta.put(CHECKSUM, checksum);
             } catch (Exception e) {
+              //TODO Should this trigger a larger error?
               log.warn("Could not read checksum from index file: " + file, e);
             }
           }
@@ -647,15 +662,16 @@ public class ReplicationHandler extends RequestHandlerBase implements SolrCoreAw
           try {
             fileMeta.put(CHECKSUM, CodecUtil.retrieveChecksum(in));
           } catch (Exception e) {
+            //TODO Should this trigger a larger error?
             log.warn("Could not read checksum from index file: " + infos.getSegmentsFileName(), e);
           }
         }
       }
       result.add(fileMeta);
     } catch (IOException e) {
-      rsp.add("status", "unable to get file names for given index generation");
-      rsp.add(EXCEPTION, e);
       log.error("Unable to get file names for indexCommit generation: " + gen, e);
+      reportErrorOnResponse(rsp, "unable to get file names for given index generation", e);
+      return;
     } finally {
       if (dir != null) {
         try {
@@ -674,9 +690,9 @@ public class ReplicationHandler extends RequestHandlerBase implements SolrCoreAw
         rsp.add(TLOG_FILES, tlogfiles);
       }
       catch (IOException e) {
-        rsp.add("status", "unable to get tlog file names for given index generation");
-        rsp.add(EXCEPTION, e);
         log.error("Unable to get tlog file names for indexCommit generation: " + gen, e);
+        reportErrorOnResponse(rsp, "unable to get tlog file names for given index generation", e);
+        return;
       }
     }
 
@@ -685,6 +701,7 @@ public class ReplicationHandler extends RequestHandlerBase implements SolrCoreAw
     log.debug("Adding config files to list: " + includeConfFiles);
     //if configuration files need to be included get their details
     rsp.add(CONF_FILES, getConfFileInfoFromCache(confFileNameAlias, confFileInfoCache));
+    rsp.add(STATUS, OK_STATUS);
   }
 
   /**
@@ -773,17 +790,23 @@ public class ReplicationHandler extends RequestHandlerBase implements SolrCoreAw
     }
   }
 
-  void disablePoll() {
-    if (isSlave) {
+  private void disablePoll(SolrQueryResponse rsp) {
+    if (pollingIndexFetcher != null){
       pollDisabled.set(true);
       log.info("inside disable poll, value of pollDisabled = " + pollDisabled);
+      rsp.add(STATUS, OK_STATUS);
+    } else {
+      reportErrorOnResponse(rsp, "No slave configured", null);
     }
   }
 
-  void enablePoll() {
-    if (isSlave) {
+  private void enablePoll(SolrQueryResponse rsp) {
+    if (pollingIndexFetcher != null){
       pollDisabled.set(false);
       log.info("inside enable poll, value of pollDisabled = " + pollDisabled);
+      rsp.add(STATUS, OK_STATUS);
+    } else {
+      reportErrorOnResponse(rsp, "No slave configured", null);
     }
   }
 
@@ -886,10 +909,11 @@ public class ReplicationHandler extends RequestHandlerBase implements SolrCoreAw
         tag, true, "replicationEnabled", getCategory().toString(), scope);
   }
 
+  //TODO Should a failure retrieving any piece of info mark the overall request as a failure?  Is there a core set of values that are required to make a response here useful?
   /**
    * Used for showing statistics and progress information.
    */
-  private NamedList<Object> getReplicationDetails(boolean showSlaveDetails) {
+  private NamedList<Object> getReplicationDetails(SolrQueryResponse rsp, boolean showSlaveDetails) {
     NamedList<Object> details = new SimpleOrderedMap<>();
     NamedList<Object> master = new SimpleOrderedMap<>();
     NamedList<Object> slave = new SimpleOrderedMap<>();
@@ -1047,6 +1071,10 @@ public class ReplicationHandler extends RequestHandlerBase implements SolrCoreAw
     if (snapshotStats != null)
       details.add(CMD_BACKUP, snapshotStats);
 
+    if (rsp.getValues().get(STATUS) == null) {
+      rsp.add(STATUS, OK_STATUS);
+    }
+    rsp.add(CMD_DETAILS, details);
     return details;
   }
 
@@ -1727,6 +1755,8 @@ public class ReplicationHandler extends RequestHandlerBase implements SolrCoreAw
 
   public static final String STATUS = "status";
 
+  public static final String MESSAGE = "message";
+
   public static final String COMMAND = "command";
 
   public static final String CMD_DETAILS = "details";

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/c951578f/solr/core/src/test/org/apache/solr/MinimalSchemaTest.java
----------------------------------------------------------------------
diff --git a/solr/core/src/test/org/apache/solr/MinimalSchemaTest.java b/solr/core/src/test/org/apache/solr/MinimalSchemaTest.java
index 4c06438..a5fc545 100644
--- a/solr/core/src/test/org/apache/solr/MinimalSchemaTest.java
+++ b/solr/core/src/test/org/apache/solr/MinimalSchemaTest.java
@@ -116,7 +116,8 @@ public class MinimalSchemaTest extends SolrTestCaseJ4 {
             handler.startsWith("/stream") ||
             handler.startsWith("/terms") ||
             handler.startsWith("/analysis/")||
-            handler.startsWith("/debug/")
+            handler.startsWith("/debug/") ||
+            handler.startsWith("/replication")
             ) {
           continue;
         }

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/c951578f/solr/core/src/test/org/apache/solr/handler/TestReplicationHandler.java
----------------------------------------------------------------------
diff --git a/solr/core/src/test/org/apache/solr/handler/TestReplicationHandler.java b/solr/core/src/test/org/apache/solr/handler/TestReplicationHandler.java
index 0cd2c04..e3a9377 100644
--- a/solr/core/src/test/org/apache/solr/handler/TestReplicationHandler.java
+++ b/solr/core/src/test/org/apache/solr/handler/TestReplicationHandler.java
@@ -49,6 +49,7 @@ import org.apache.solr.BaseDistributedSearchTestCase;
 import org.apache.solr.SolrTestCaseJ4;
 import org.apache.solr.SolrTestCaseJ4.SuppressSSL;
 import org.apache.solr.client.solrj.SolrClient;
+import org.apache.solr.client.solrj.SolrQuery;
 import org.apache.solr.client.solrj.SolrServerException;
 import org.apache.solr.client.solrj.embedded.JettyConfig;
 import org.apache.solr.client.solrj.embedded.JettySolrRunner;
@@ -79,6 +80,8 @@ import org.junit.Test;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
+import static org.junit.matchers.JUnitMatchers.containsString;
+
 /**
  * Test for ReplicationHandler
  *
@@ -223,8 +226,7 @@ public class TestReplicationHandler extends SolrTestCaseJ4 {
     QueryRequest req = new QueryRequest(params);
 
     NamedList<Object> res = s.request(req);
-
-    assertNotNull("null response from server", res);
+    assertReplicationResponseSucceeded(res);
 
     @SuppressWarnings("unchecked") NamedList<Object> details 
       = (NamedList<Object>) res.get("details");
@@ -234,23 +236,6 @@ public class TestReplicationHandler extends SolrTestCaseJ4 {
     return details;
   }
   
-//  private NamedList<Object> getCommits(SolrClient s) throws Exception {
-//    
-//
-//    ModifiableSolrParams params = new ModifiableSolrParams();
-//    params.set("command","commits");
-//    params.set("_trace","getCommits");
-//    params.set("qt",ReplicationHandler.PATH);
-//    QueryRequest req = new QueryRequest(params);
-//
-//    NamedList<Object> res = s.request(req);
-//
-//    assertNotNull("null response from server", res);
-//
-//
-//    return res;
-//  }
-  
   private NamedList<Object> getIndexVersion(SolrClient s) throws Exception {
     
     ModifiableSolrParams params = new ModifiableSolrParams();
@@ -260,9 +245,7 @@ public class TestReplicationHandler extends SolrTestCaseJ4 {
     QueryRequest req = new QueryRequest(params);
 
     NamedList<Object> res = s.request(req);
-
-    assertNotNull("null response from server", res);
-
+    assertReplicationResponseSucceeded(res);
 
     return res;
   }
@@ -1083,12 +1066,13 @@ public class TestReplicationHandler extends SolrTestCaseJ4 {
     params.set("command", "indexversion");
     QueryRequest req = new QueryRequest(params);
     NamedList<Object> resp = client1.request(req);
-    
+    assertReplicationResponseSucceeded(resp);
     Long version = (Long) resp.get("indexversion");
     assertEquals(maxVersionClient1, version);
     
     // check vs /replication?command=indexversion call
     resp = client2.request(req);
+    assertReplicationResponseSucceeded(resp);
     version = (Long) resp.get("indexversion");
     assertEquals(maxVersionClient2, version);
   }
@@ -1481,6 +1465,61 @@ public class TestReplicationHandler extends SolrTestCaseJ4 {
       }
     }
   }
+
+  @Test
+  public void testFileListShouldReportErrorsWhenTheyOccur() throws Exception {
+    SolrQuery q = new SolrQuery();
+    q.add("qt", "/replication")
+        .add("wt", "json")
+        .add("command", "filelist")
+        .add("generation", "-1"); // A 'generation' value not matching any commit point should cause error.
+    QueryResponse response = slaveClient.query(q);
+    NamedList<Object> resp = response.getResponse();
+    assertNotNull(resp);
+    assertEquals("ERROR", resp.get("status"));
+    assertEquals("invalid index generation", resp.get("message"));
+  }
+
+  @Test
+  public void testFetchIndexShouldReportErrorsWhenTheyOccur() throws Exception  {
+    int masterPort = masterJetty.getLocalPort();
+    masterJetty.stop();
+    SolrQuery q = new SolrQuery();
+    q.add("qt", "/replication")
+        .add("wt", "json")
+        .add("wait", "true")
+        .add("command", "fetchindex")
+        .add("masterUrl", buildUrl(masterPort));
+    QueryResponse response = slaveClient.query(q);
+    NamedList<Object> resp = response.getResponse();
+    assertNotNull(resp);
+    assertEquals("Fetch index with wait=true should have returned an error response", "ERROR", resp.get("status"));
+  }
+
+  @Test
+  public void testShouldReportErrorWhenRequiredCommandArgMissing() throws Exception {
+    SolrQuery q = new SolrQuery();
+    q.add("qt", "/replication")
+        .add("wt", "json");
+    SolrException thrown = expectThrows(SolrException.class, () -> {
+      slaveClient.query(q);
+    });
+    assertEquals(SolrException.ErrorCode.BAD_REQUEST.code, thrown.code());
+    assertThat(thrown.getMessage(), containsString("Missing required parameter: command"));
+  }
+
+  @Test
+  public void testShouldReportErrorWhenDeletingBackupButNameMissing() {
+    SolrQuery q = new SolrQuery();
+    q.add("qt", "/replication")
+        .add("wt", "json")
+        .add("command", "deletebackup");
+    SolrException thrown = expectThrows(SolrException.class, () -> {
+      slaveClient.query(q);
+    });
+    assertEquals(SolrException.ErrorCode.BAD_REQUEST.code, thrown.code());
+    assertThat(thrown.getMessage(), containsString("Missing required parameter: name"));
+  }
   
   private class AddExtraDocs implements Runnable {
 
@@ -1583,6 +1622,12 @@ public class TestReplicationHandler extends SolrTestCaseJ4 {
       return min; // compilation neccessity
     }
   }
+
+  private void assertReplicationResponseSucceeded(NamedList response) {
+    assertNotNull("null response from server", response);
+    assertNotNull("Expected replication response to have 'status' field", response.get("status"));
+    assertEquals("OK", response.get("status"));
+  }
   
   private static String buildUrl(int port) {
     return buildUrl(port, context);