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

CC-1059 Changed ES connector to use exponential backoff with jitter #115

Closed
wants to merge 1 commit into from
Closed
Show file tree
Hide file tree
Changes from all commits
Commits
File filter

Filter by extension

Filter by extension

Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
Original file line number Diff line number Diff line change
Expand Up @@ -19,6 +19,7 @@
import org.apache.kafka.common.config.AbstractConfig;
import org.apache.kafka.common.config.ConfigDef;
import org.apache.kafka.common.config.ConfigDef.Importance;
import org.apache.kafka.common.config.ConfigDef.Range;
import org.apache.kafka.common.config.ConfigDef.Type;
import org.apache.kafka.common.config.ConfigDef.Width;

Expand Down Expand Up @@ -61,17 +62,25 @@ public class ElasticsearchSinkConnectorConfig extends AbstractConfig {
private static final String MAX_RETRIES_DOC =
"The maximum number of retries that are allowed for failed indexing requests. If the retry "
+ "attempts are exhausted the task will fail.";
public static final String RETRY_BACKOFF_MS_CONFIG = "retry.backoff.ms";
private static final String RETRY_BACKOFF_MS_DOC =
"How long to wait in milliseconds before attempting to retry a failed indexing request. "
+ "This avoids retrying in a tight loop under failure scenarios.";

public static final String TYPE_NAME_CONFIG = "type.name";
private static final String TYPE_NAME_DOC = "The Elasticsearch type name to use when indexing.";
public static final String TOPIC_INDEX_MAP_CONFIG = "topic.index.map";
private static final String TOPIC_INDEX_MAP_DOC =
"A map from Kafka topic name to the destination Elasticsearch index, represented as a list "
+ "of ``topic:index`` pairs.";

public static final String MIN_RETRY_BACKOFF_MS_CONFIG = "retry.backoff.ms";
private static final String MIN_RETRY_BACKOFF_MS_DOC =
"The minimum time to wait in milliseconds before attempting to retry a failed indexing request. "
+ "Actual time to wait is computed using exponential backoff with jitter within minimum and maximum values. "
+ "This avoids retrying in a tight loop under failure scenarios and avoids thundering herd problems with the source.";
public static final String MAX_RETRY_BACKOFF_MS_CONFIG = "max.retry.backoff.ms";
private static final String MAX_RETRY_BACKOFF_MS_DOC =
"The maximum time to wait in milliseconds before attempting to retry a failed indexing request. "
+ "Actual time to wait is computed using exponential backoff with jitter within minimum and maximum values. "
+ "This avoids retrying in a tight loop under failure scenarios and avoids thundering herd problems with the source.";

public static final String KEY_IGNORE_CONFIG = "key.ignore";
public static final String TOPIC_KEY_IGNORE_CONFIG = "topic.key.ignore";
public static final String SCHEMA_IGNORE_CONFIG = "schema.ignore";
Expand Down Expand Up @@ -178,16 +187,28 @@ private static void addConnectorConfigs(ConfigDef configDef) {
Width.SHORT,
"Max Retries"
).define(
RETRY_BACKOFF_MS_CONFIG,
MIN_RETRY_BACKOFF_MS_CONFIG,
Type.LONG,
100L,
Range.atLeast(0),
Importance.LOW,
RETRY_BACKOFF_MS_DOC,
MIN_RETRY_BACKOFF_MS_DOC,
group,
++order,
Width.SHORT,
"Retry Backoff (ms)"
);
).define(
MAX_RETRY_BACKOFF_MS_CONFIG,
Type.LONG,
10000L,
Range.atLeast(0),
Importance.LOW,
MAX_RETRY_BACKOFF_MS_DOC,
group,
++order,
Width.SHORT,
"Max Retry Backoff (ms)"
);
}

private static void addConversionConfigs(ConfigDef configDef) {
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -83,8 +83,10 @@ public void start(Map<String, String> props, JestClient client) {
config.getLong(ElasticsearchSinkConnectorConfig.LINGER_MS_CONFIG);
int maxInFlightRequests =
config.getInt(ElasticsearchSinkConnectorConfig.MAX_IN_FLIGHT_REQUESTS_CONFIG);
long retryBackoffMs =
config.getLong(ElasticsearchSinkConnectorConfig.RETRY_BACKOFF_MS_CONFIG);
long minRetryBackoffMs =
config.getLong(ElasticsearchSinkConnectorConfig.MIN_RETRY_BACKOFF_MS_CONFIG);
long maxRetryBackoffMs =
config.getLong(ElasticsearchSinkConnectorConfig.MAX_RETRY_BACKOFF_MS_CONFIG);
int maxRetry =
config.getInt(ElasticsearchSinkConnectorConfig.MAX_RETRIES_CONFIG);
boolean dropInvalidMessage =
Expand Down Expand Up @@ -114,7 +116,8 @@ public void start(Map<String, String> props, JestClient client) {
.setMaxInFlightRequests(maxInFlightRequests)
.setBatchSize(batchSize)
.setLingerMs(lingerMs)
.setRetryBackoffMs(retryBackoffMs)
.setMinRetryBackoffMs(minRetryBackoffMs)
.setMaxRetryBackoffMs(maxRetryBackoffMs)
.setMaxRetry(maxRetry)
.setDropInvalidMessage(dropInvalidMessage);

Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -67,7 +67,8 @@ public class ElasticsearchWriter {
int batchSize,
long lingerMs,
int maxRetries,
long retryBackoffMs,
long minRetryBackoffMs,
long maxRetryBackoffMs,
boolean dropInvalidMessage
) {
this.client = client;
Expand All @@ -88,7 +89,8 @@ public class ElasticsearchWriter {
batchSize,
lingerMs,
maxRetries,
retryBackoffMs
minRetryBackoffMs,
maxRetryBackoffMs
);

existingMappings = new HashSet<>();
Expand All @@ -108,7 +110,8 @@ public static class Builder {
private int batchSize;
private long lingerMs;
private int maxRetry;
private long retryBackoffMs;
private long minRetryBackoffMs;
private long maxRetryBackoffMs;
private boolean dropInvalidMessage;

public Builder(JestClient client) {
Expand Down Expand Up @@ -167,8 +170,13 @@ public Builder setMaxRetry(int maxRetry) {
return this;
}

public Builder setRetryBackoffMs(long retryBackoffMs) {
this.retryBackoffMs = retryBackoffMs;
public Builder setMinRetryBackoffMs(long retryBackoffMs) {
this.minRetryBackoffMs = retryBackoffMs;
return this;
}

public Builder setMaxRetryBackoffMs(long retryBackoffMs) {
this.maxRetryBackoffMs = retryBackoffMs;
return this;
}

Expand All @@ -192,7 +200,8 @@ public ElasticsearchWriter build() {
batchSize,
lingerMs,
maxRetry,
retryBackoffMs,
minRetryBackoffMs,
maxRetryBackoffMs,
dropInvalidMessage
);
}
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -30,6 +30,7 @@
import java.util.concurrent.Executors;
import java.util.concurrent.Future;
import java.util.concurrent.ThreadFactory;
import java.util.concurrent.ThreadLocalRandom;
import java.util.concurrent.TimeUnit;
import java.util.concurrent.atomic.AtomicInteger;
import java.util.concurrent.atomic.AtomicLong;
Expand All @@ -51,7 +52,8 @@ public class BulkProcessor<R, B> {
private final int batchSize;
private final long lingerMs;
private final int maxRetries;
private final long retryBackoffMs;
private final long minRetryBackoffMs;
private final long maxRetryBackoffMs;

private final Thread farmer;
private final ExecutorService executor;
Expand All @@ -75,15 +77,17 @@ public BulkProcessor(
int batchSize,
long lingerMs,
int maxRetries,
long retryBackoffMs
long minRetryBackoffMs,
long maxRetryBackoffMs
) {
this.time = time;
this.bulkClient = bulkClient;
this.maxBufferedRecords = maxBufferedRecords;
this.batchSize = batchSize;
this.lingerMs = lingerMs;
this.maxRetries = maxRetries;
this.retryBackoffMs = retryBackoffMs;
this.minRetryBackoffMs = minRetryBackoffMs;
this.maxRetryBackoffMs = maxRetryBackoffMs;

unsentRecords = new ArrayDeque<>(maxBufferedRecords);

Expand Down Expand Up @@ -357,28 +361,36 @@ private BulkResponse execute() throws Exception {
);
throw e;
}
for (int remainingRetries = maxRetries; true; remainingRetries--) {
int retryAttempt = 0;
int attempts = 0;
while (true) {
boolean retriable = true;
++attempts;
try {
log.trace("Executing batch {} of {} records", batchId, batch.size());
final BulkResponse bulkRsp = bulkClient.execute(bulkReq);
if (bulkRsp.isSucceeded()) {
if (attempts > 1) {
log.debug("Completed batch {} of {} records with attempt {}/{}", batchId, batch.size(), attempts, maxRetries + 1);
}
return bulkRsp;
}
retriable = bulkRsp.isRetriable();
throw new ConnectException("Bulk request failed: " + bulkRsp.getErrorInfo());
} catch (Exception e) {
if (retriable && remainingRetries > 0) {
log.warn(
"Failed to execute batch {} of {} records, retrying after {} ms",
batchId,
batch.size(),
retryBackoffMs,
e
);
time.sleep(retryBackoffMs);
if (retriable && retryAttempt < maxRetries) {
++retryAttempt;
long sleepTimeMs = computeRetryWaitTimeInMillis(retryAttempt, minRetryBackoffMs, maxRetryBackoffMs);
if (log.isDebugEnabled()) {
log.debug("Failed to execute batch {} of {} records with attempt {}/{}, will attempt retry after {} ms",
batchId, batch.size(), attempts, maxRetries + 1, sleepTimeMs, e);
} else {
log.warn("Failed to execute batch {} of {} records with attempt {}/{}, will attempt retry after {} ms: {}",
batchId, batch.size(), attempts, maxRetries + 1, sleepTimeMs, e.getMessage());
}
time.sleep(minRetryBackoffMs);
} else {
log.error("Failed to execute batch {} of {} records", batchId, batch.size(), e);
log.error("Failed to execute batch {} of {} records after total of {} attempt(s)", batchId, batch.size(), attempts, e);
throw e;
}
}
Expand All @@ -387,6 +399,31 @@ private BulkResponse execute() throws Exception {

}

/**
* Compute the time to sleep using exponential backoff with jitter. This method computes the normal exponential backoff
* based upon the {@code minRetryBackoffMs} and the {@code retryAttempt}, bounding the result to be within {@code minRetryBackoffMs}
* and {@code maxRetryBackoffMs}, and then choosing a random value within that range.
* <p>
* The purposes of using exponential backoff is to give the ES service time to recover when it becomes overwhelmed.
* Adding jitter attempts to prevent a thundering herd, where large numbers of requests from many tasks overwhelm the
* ES service, and without randomization all tasks retry at the same time. Randomization should spread the retries
* out and should reduce the overall time required to complete all attempts.
* See <a href="https://www.awsarchitectureblog.com/2015/03/backoff.html">this blog post</a> for details.
*
* @param retryAttempt the retry attempt number, starting with 1 for the first retry
* @param minRetryBackoffMs the minimum time to wait before retrying; assumed to be 0 if value is negative
* @param maxRetryBackoffMs the maximum amount of time to wait before retrying
* @return the time in milliseconds to wait before the next retry attempt, in the range {@code minRetryBackoffMs}
* and {@code maxRetryBackoffMs} (inclusive), or {@code minRetryBackoffMs} if {@code minRangeBackoffMs}
* is greater than or equal to {@code maxRetryBackoffMs}
*/
protected static long computeRetryWaitTimeInMillis(int retryAttempt, long minRetryBackoffMs, long maxRetryBackoffMs) {
if (minRetryBackoffMs >= maxRetryBackoffMs) return minRetryBackoffMs; // this was the original value and likely what they're setting
if (minRetryBackoffMs < 0) minRetryBackoffMs = 0;
long nextMaxTimeMs = Math.max(minRetryBackoffMs + 1, Math.min(maxRetryBackoffMs, minRetryBackoffMs * 2 ^ retryAttempt));
return ThreadLocalRandom.current().nextLong(minRetryBackoffMs, nextMaxTimeMs);
}

private synchronized void onBatchCompletion(int batchSize) {
inFlightRecords -= batchSize;
assert inFlightRecords >= 0;
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -377,7 +377,8 @@ private ElasticsearchWriter initWriter(JestClient client, boolean ignoreKey, Set
.setMaxInFlightRequests(1)
.setBatchSize(2)
.setLingerMs(1000)
.setRetryBackoffMs(1000)
.setMinRetryBackoffMs(1000)
.setMaxRetryBackoffMs(10000)
.setMaxRetry(3)
.setDropInvalidMessage(dropInvalidMessage)
.build();
Expand Down
Loading