Skip to content
New issue

Have a question about this project? Sign up for a free GitHub account to open an issue and contact its maintainers and the community.

By clicking “Sign up for GitHub”, you agree to our terms of service and privacy statement. We’ll occasionally send you account related emails.

Already on GitHub? Sign in to your account

[CCR] Improve shard follow task's retryable error handling #33371

Conversation

martijnvg
Copy link
Member

Improve the failure handling of retryable errors by retrying remote calls in
a exponential backoff like manner. The delay between a retry would not be
longer than the configured max retry delay. Also retryable errors will be
retried indefinitely.

Relates to #30086

@martijnvg martijnvg added >enhancement review v7.0.0 :Distributed/CCR Issues around the Cross Cluster State Replication features v6.5.0 labels Sep 4, 2018
@elasticmachine
Copy link
Collaborator

Pinging @elastic/es-distributed

Improve failure handling of retryable errors by retrying remote calls in
a exponential backoff like manner. The delay between a retry would not be
longer than the configured max retry delay. Also retryable errors will be
retried indefinitely.

Relates to elastic#30086
@martijnvg martijnvg force-pushed the ccr_improve_shard_follow_task_retryable_failure_handling branch from 05d0518 to a717f44 Compare September 4, 2018 10:11
Copy link
Member

@jasontedor jasontedor left a comment

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

I left a comment about the delay computation.

} else {
markAsFailed(e);
}
}

private boolean shouldRetry(Exception e) {
static long computeDelay(int currentRetry, long maxRetryDelayInMillis) {
long expectedBackOff = Math.round(10 * Math.exp(0.8d * currentRetry) - 1);
Copy link
Member

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

This is not quite what I think of as exponential backoff. The problem that I see with an implementation like this is that we can have a thundering herd problem. If there is some failure that causes a bunch of tasks to simultaneously fail (e.g., say that we have a bunch of outstanding fetch tasks waiting for a response, and the network connection breaks, failing all of them), all of the retries will keep waking up at the same time, itself potentially causing issues due to a herd.

Typically it would be that there is a random component in exponential backoff, to avoid this herding. As a first approach, what I suggest here is: choose a random value k between 0 and 2^number of retries - 1. Then retry after k * delay seconds. We can cap this at max retry delay.

Copy link
Member Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Right, I did think about this, but when looking at ExponentialBackoffIterator class to see how it was implemented elsewhere I decided to use that. I will change this.

Copy link
Member Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

I've changed how the delay is now computed. With this randomness the retying mechanism is much more solid.

Copy link
Member

@jasontedor jasontedor left a comment

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

I left a comment.

} else {
markAsFailed(e);
}
}

private boolean shouldRetry(Exception e) {
static long computeDelay(int currentRetry, long maxRetryDelayInMillis) {
long n = Math.round(Math.pow(2, currentRetry - 1));
Copy link
Member

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

I think that we need to guard against overflow here!

@@ -56,20 +57,22 @@
*/
public abstract class ShardFollowNodeTask extends AllocatedPersistentTask {

public static final SecureRandom RANDOM_INSTANCE = new SecureRandom();
Copy link
Member

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

We can use Randomness#get here.

"] times, aborting...", e));
}
if (shouldRetry(e) && isStopped() == false) {
LOGGER.debug(new ParameterizedMessage("{} error during follow shard task, retrying...", params.getFollowShardId()), e);
Copy link
Member

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

We should debug log here the number of times that we have retried up to here.

@martijnvg
Copy link
Member Author

@jasontedor I've updated the PR.

jasontedor and others added 3 commits September 11, 2018 08:48
* master: (91 commits)
  Preserve cluster settings on full restart tests (elastic#33590)
  Use IndexWriter.getFlushingBytes() rather than tracking it ourselves (elastic#33582)
  Fix upgrading of list settings (elastic#33589)
  Add read-only Engine (elastic#33563)
  HLRC: Add ML get categories API (elastic#33465)
  SQL: Adds MONTHNAME, DAYNAME and QUARTER functions (elastic#33411)
  Add predicate_token_filter (elastic#33431)
  Fix Replace function. Adds more tests to all string functions. (elastic#33478)
  [ML] Rename input_fields to column_names in file structure (elastic#33568)
  Add full cluster restart base class (elastic#33577)
  Validate list values for settings (elastic#33503)
  Copy and validatie soft-deletes setting on resize (elastic#33517)
  Test: Fix package name
  SQL: Fix result column names for arithmetic functions (elastic#33500)
  Upgrade to latest Lucene snapshot (elastic#33505)
  Enable not wiping cluster settings after REST test (elastic#33575)
  MINOR: Remove Dead Code in SearchScript (elastic#33569)
  [Test] Remove duplicate method in TestShardRouting (elastic#32815)
  mute test on windows
  Update beats template to include apm-server metrics (elastic#33286)
  ...
Copy link
Member

@jasontedor jasontedor left a comment

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

LGTM.

@jasontedor
Copy link
Member

jasontedor commented Sep 11, 2018

Separately I think we want some validation on max_retry_delay as today it can be negative and also set too large. Maybe:

diff --git a/x-pack/plugin/ccr/src/main/java/org/elasticsearch/xpack/ccr/action/FollowIndexAction.java b/x-pack/plugin/ccr/src/main/java/org/elasticsearch/xpack/ccr/action/FollowIndexAction.java
index 26a9d11eb32..4b235b55991 100644
--- a/x-pack/plugin/ccr/src/main/java/org/elasticsearch/xpack/ccr/action/FollowIndexAction.java
+++ b/x-pack/plugin/ccr/src/main/java/org/elasticsearch/xpack/ccr/action/FollowIndexAction.java
@@ -62,6 +62,8 @@ import java.util.concurrent.atomic.AtomicInteger;
 import java.util.concurrent.atomic.AtomicReferenceArray;
 import java.util.stream.Collectors;
 
+import static org.elasticsearch.action.ValidateActions.addValidationError;
+
 public class FollowIndexAction extends Action<AcknowledgedResponse> {
 
     public static final FollowIndexAction INSTANCE = new FollowIndexAction();
@@ -210,9 +212,19 @@ public class FollowIndexAction extends Action<AcknowledgedResponse> {
             return maxBatchOperationCount;
         }
 
+        private static final TimeValue MAX_MAX_RETRY_DELAY = TimeValue.timeValueMinutes(5);
+
         @Override
         public ActionRequestValidationException validate() {
-            return null;
+            ActionRequestValidationException e = null;
+            if (maxRetryDelay.millis() <= 0) {
+                e = addValidationError("max_retry_delay must be positive but was [" + maxRetryDelay.getStringRep() + "]", e);
+            }
+            if (maxRetryDelay.millis() > MAX_MAX_RETRY_DELAY.millis()) {
+                e = addValidationError(
+                        "max_retry_delay must be less than [" + MAX_MAX_RETRY_DELAY + "[ but was [" + maxRetryDelay.getStringRep() + "]", e);
+            }
+            return e;
         }
 
         @Override

What do you think?

@martijnvg martijnvg merged commit 96c49e5 into elastic:master Sep 12, 2018
martijnvg added a commit that referenced this pull request Sep 12, 2018
Improve failure handling of retryable errors by retrying remote calls in
a exponential backoff like manner. The delay between a retry would not be
longer than the configured max retry delay. Also retryable errors will be
retried indefinitely.

Relates to #30086
jasontedor added a commit to martijnvg/elasticsearch that referenced this pull request Sep 12, 2018
* master: (43 commits)
  [HLRC][ML] Add ML put datafeed API to HLRC (elastic#33603)
  Update AWS SDK to 1.11.406  in repository-s3 (elastic#30723)
  Expose CCR stats to monitoring (elastic#33617)
  [Docs] Update match-query.asciidoc (elastic#33610)
  TEST: Adjust rollback condition when shard is empty
  [CCR] Improve shard follow task's retryable error handling (elastic#33371)
  Forbid negative `weight` in Function Score Query (elastic#33390)
  Clarify context suggestions filtering and boosting (elastic#33601)
  Disable CCR REST endpoints if CCR disabled (elastic#33619)
  Lower version on full cluster restart settings test
  Upgrade remote cluster settings (elastic#33537)
  NETWORKING: http.publish_host Should Contain CNAME (elastic#32806)
  Add test coverage for global checkpoint listeners
  Reset replica engine to global checkpoint on promotion (elastic#33473)
  HLRC: ML Delete Forecast API (elastic#33526)
  Remove debug logging in full cluster restart tests (elastic#33612)
  Expose CCR to the transport client (elastic#33608)
  Mute testIndexDeletionWhenNodeRejoins
  SQL: Make Literal a NamedExpression (elastic#33583)
  [DOCS] Adds missing built-in user information (elastic#33585)
  ...
jasontedor added a commit to martijnvg/elasticsearch that referenced this pull request Sep 12, 2018
* master: (128 commits)
  [HLRC][ML] Add ML put datafeed API to HLRC (elastic#33603)
  Update AWS SDK to 1.11.406  in repository-s3 (elastic#30723)
  Expose CCR stats to monitoring (elastic#33617)
  [Docs] Update match-query.asciidoc (elastic#33610)
  TEST: Adjust rollback condition when shard is empty
  [CCR] Improve shard follow task's retryable error handling (elastic#33371)
  Forbid negative `weight` in Function Score Query (elastic#33390)
  Clarify context suggestions filtering and boosting (elastic#33601)
  Disable CCR REST endpoints if CCR disabled (elastic#33619)
  Lower version on full cluster restart settings test
  Upgrade remote cluster settings (elastic#33537)
  NETWORKING: http.publish_host Should Contain CNAME (elastic#32806)
  Add test coverage for global checkpoint listeners
  Reset replica engine to global checkpoint on promotion (elastic#33473)
  HLRC: ML Delete Forecast API (elastic#33526)
  Remove debug logging in full cluster restart tests (elastic#33612)
  Expose CCR to the transport client (elastic#33608)
  Mute testIndexDeletionWhenNodeRejoins
  SQL: Make Literal a NamedExpression (elastic#33583)
  [DOCS] Adds missing built-in user information (elastic#33585)
  ...
Sign up for free to join this conversation on GitHub. Already have an account? Sign in to comment
Labels
:Distributed/CCR Issues around the Cross Cluster State Replication features >enhancement
Projects
None yet
Development

Successfully merging this pull request may close these issues.

None yet

3 participants