-
Notifications
You must be signed in to change notification settings - Fork 513
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
Retry bulk chunk requests to Elasticsearch 2, 5, and 6 #1626
Changes from 3 commits
7146ab9
9605842
00c299f
b4d95d1
2bd5492
87e7805
d8c1724
File filter
Filter by extension
Conversations
Jump to
Diff view
Diff view
There are no files selected for viewing
Original file line number | Diff line number | Diff line change |
---|---|---|
|
@@ -122,63 +122,115 @@ public static<T> Bound withMaxBulkRequestSize(int maxBulkRequestSize) { | |
return new Bound<>().withMaxBulkRequestSize(maxBulkRequestSize); | ||
} | ||
|
||
/** | ||
* Returns a transform for writing to Elasticsearch cluster. | ||
* | ||
* @param maxRetries Maximum number of retries to attempt for saving any single chunk of bulk | ||
* requests to the Elasticsearch cluster. | ||
*/ | ||
public static <T> Bound withMaxRetries(int maxRetries) { | ||
return new Bound<>().withMaxRetries(maxRetries); | ||
} | ||
|
||
/** | ||
* Returns a transform for writing to Elasticsearch cluster. | ||
* | ||
* @param retryPause Number of seconds to wait between successive retry attempts. | ||
*/ | ||
public static <T> Bound withRetryPause(int retryPause) { | ||
return new Bound<>().withRetryPause(retryPause); | ||
} | ||
|
||
public static class Bound<T> extends PTransform<PCollection<T>, PDone> { | ||
private static final int CHUNK_SIZE = 3000; | ||
private static final int DEFAULT_RETRIES = 3; | ||
private static final int DEFAULT_RETRY_PAUSE = 5; | ||
|
||
private final String clusterName; | ||
private final InetSocketAddress[] servers; | ||
private final Duration flushInterval; | ||
private final SerializableFunction<T, Iterable<ActionRequest<?>>> toActionRequests; | ||
private final long numOfShard; | ||
private final int maxBulkRequestSize; | ||
private final int maxRetries; | ||
private final int retryPause; | ||
private final ThrowingConsumer<BulkExecutionException> error; | ||
|
||
private Bound(final String clusterName, | ||
final InetSocketAddress[] servers, | ||
final Duration flushInterval, | ||
final SerializableFunction<T, Iterable<ActionRequest<?>>> toActionRequests, | ||
final long numOfShard, | ||
final int maxBulkRequestSize, | ||
final ThrowingConsumer<BulkExecutionException> error) { | ||
final InetSocketAddress[] servers, | ||
final Duration flushInterval, | ||
final SerializableFunction<T, Iterable<ActionRequest<?>>> toActionRequests, | ||
final long numOfShard, | ||
final int maxBulkRequestSize, | ||
int maxRetries, int retryPause, final ThrowingConsumer<BulkExecutionException> error) { | ||
this.clusterName = clusterName; | ||
this.servers = servers; | ||
this.flushInterval = flushInterval; | ||
this.toActionRequests = toActionRequests; | ||
this.numOfShard = numOfShard; | ||
this.maxBulkRequestSize = maxBulkRequestSize; | ||
this.maxRetries = maxRetries; | ||
this.retryPause = retryPause; | ||
this.error = error; | ||
} | ||
|
||
Bound() { | ||
this(null, null, null, null, 0, CHUNK_SIZE, defaultErrorHandler()); | ||
this(null, null, null, null, 0, CHUNK_SIZE, DEFAULT_RETRIES, DEFAULT_RETRY_PAUSE, | ||
defaultErrorHandler()); | ||
} | ||
|
||
public Bound<T> withClusterName(String clusterName) { | ||
return new Bound<>(clusterName, servers, flushInterval, toActionRequests, numOfShard, maxBulkRequestSize, error); | ||
return new Bound<>(clusterName, servers, flushInterval, toActionRequests, numOfShard, | ||
maxBulkRequestSize, | ||
maxRetries, retryPause, error); | ||
} | ||
|
||
public Bound<T> withServers(InetSocketAddress[] servers) { | ||
return new Bound<>(clusterName, servers, flushInterval, toActionRequests, numOfShard, maxBulkRequestSize, error); | ||
return new Bound<>(clusterName, servers, flushInterval, toActionRequests, numOfShard, | ||
maxBulkRequestSize, | ||
maxRetries, retryPause, error); | ||
} | ||
|
||
public Bound<T> withFlushInterval(Duration flushInterval) { | ||
return new Bound<>(clusterName, servers, flushInterval, toActionRequests, numOfShard, maxBulkRequestSize, error); | ||
return new Bound<>(clusterName, servers, flushInterval, toActionRequests, numOfShard, | ||
maxBulkRequestSize, | ||
maxRetries, retryPause, error); | ||
} | ||
|
||
public Bound<T> withFunction(SerializableFunction<T, Iterable<ActionRequest<?>>> toIndexRequest) { | ||
return new Bound<>(clusterName, servers, flushInterval, toIndexRequest, numOfShard, maxBulkRequestSize, error); | ||
return new Bound<>(clusterName, servers, flushInterval, toIndexRequest, numOfShard, | ||
maxBulkRequestSize, | ||
maxRetries, retryPause, error); | ||
} | ||
|
||
public Bound<T> withNumOfShard(long numOfShard) { | ||
return new Bound<>(clusterName, servers, flushInterval, toActionRequests, numOfShard, maxBulkRequestSize, error); | ||
return new Bound<>(clusterName, servers, flushInterval, toActionRequests, numOfShard, | ||
maxBulkRequestSize, | ||
maxRetries, retryPause, error); | ||
} | ||
|
||
public Bound<T> withError(ThrowingConsumer<BulkExecutionException> error) { | ||
return new Bound<>(clusterName, servers, flushInterval, toActionRequests, numOfShard, maxBulkRequestSize, error); | ||
return new Bound<>(clusterName, servers, flushInterval, toActionRequests, numOfShard, | ||
maxBulkRequestSize, | ||
maxRetries, retryPause, error); | ||
} | ||
|
||
public Bound<T> withMaxBulkRequestSize(int maxBulkRequestSize) { | ||
return new Bound<>(clusterName, servers, flushInterval, toActionRequests, numOfShard, maxBulkRequestSize, error); | ||
return new Bound<>(clusterName, servers, flushInterval, toActionRequests, numOfShard, | ||
maxBulkRequestSize, | ||
maxRetries, retryPause, error); | ||
} | ||
|
||
public Bound<T> withMaxRetries(int maxRetries) { | ||
return new Bound<>(clusterName, servers, flushInterval, toActionRequests, numOfShard, | ||
maxBulkRequestSize, | ||
maxRetries, retryPause, error); | ||
} | ||
|
||
public Bound<T> withRetryPause(int retryPause) { | ||
return new Bound<>(clusterName, servers, flushInterval, toActionRequests, numOfShard, | ||
maxBulkRequestSize, | ||
maxRetries, retryPause, error); | ||
} | ||
|
||
@Override | ||
|
@@ -189,6 +241,8 @@ public PDone expand(final PCollection<T> input) { | |
checkNotNull(flushInterval); | ||
checkArgument(numOfShard > 0); | ||
checkArgument(maxBulkRequestSize > 0); | ||
checkArgument(maxRetries >= 0); | ||
checkArgument(retryPause >= 0); | ||
input | ||
.apply("Assign To Shard", ParDo.of(new AssignToShard<>(numOfShard))) | ||
.apply("Re-Window to Global Window", Window.<KV<Long, T>>into(new GlobalWindows()) | ||
|
@@ -199,9 +253,10 @@ public PDone expand(final PCollection<T> input) { | |
.discardingFiredPanes() | ||
.withTimestampCombiner(TimestampCombiner.END_OF_WINDOW)) | ||
.apply(GroupByKey.create()) | ||
.apply("Write to Elasticesarch", | ||
.apply("Write to Elasticesearch", | ||
ParDo.of(new ElasticsearchWriter<> | ||
(clusterName, servers, maxBulkRequestSize, toActionRequests, error))); | ||
(clusterName, servers, maxBulkRequestSize, toActionRequests, error, | ||
maxRetries, retryPause))); | ||
return PDone.in(input.getPipeline()); | ||
} | ||
} | ||
|
@@ -227,18 +282,24 @@ private static class ElasticsearchWriter<T> extends DoFn<KV<Long, Iterable<T>>, | |
private final SerializableFunction<T, Iterable<ActionRequest<?>>> toActionRequests; | ||
private final ThrowingConsumer<BulkExecutionException> error; | ||
private final int maxBulkRequestSize; | ||
private final int maxRetries; | ||
private final int retryPause; | ||
|
||
public ElasticsearchWriter(String clusterName, | ||
InetSocketAddress[] servers, | ||
int maxBulkRequestSize, | ||
SerializableFunction<T, Iterable<ActionRequest<?>>> toActionRequests, | ||
ThrowingConsumer<BulkExecutionException> error) { | ||
InetSocketAddress[] servers, | ||
int maxBulkRequestSize, | ||
SerializableFunction<T, Iterable<ActionRequest<?>>> toActionRequests, | ||
ThrowingConsumer<BulkExecutionException> error, int maxRetries, int retryPause) { | ||
this.maxBulkRequestSize = maxBulkRequestSize; | ||
this.maxRetries = maxRetries; | ||
this.retryPause = retryPause; | ||
this.clientSupplier = new ClientSupplier(clusterName, servers); | ||
this.toActionRequests = toActionRequests; | ||
this.error = error; | ||
} | ||
|
||
|
||
@SuppressWarnings("Duplicates") | ||
@ProcessElement | ||
public void processElement(ProcessContext c) throws Exception { | ||
final Iterable<T> values = c.element().getValue(); | ||
|
@@ -259,11 +320,46 @@ public void processElement(ProcessContext c) throws Exception { | |
Iterables.partition(actionRequests::iterator, maxBulkRequestSize); | ||
|
||
chunks.forEach(chunk -> { | ||
int attempts = 0; | ||
Exception exception = null; | ||
|
||
while (attempts <= maxRetries) { | ||
try { | ||
if (attempts > 0) { | ||
// Sleep on subsequent attempts. | ||
Thread.sleep(retryPause * 1000); | ||
} | ||
|
||
final BulkRequest bulkRequest = new BulkRequest().add(chunk).refresh(false); | ||
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. @soundofjw no need for There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. Yes, though it should be noted that when we are indexing with a refresh interval of There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. Yeah refresh at indexing time can happen do to several things ... document sizes, nr documents, queues... I guess at some point we can offer support, but let's not do it in the same PR, next one! 😄 |
||
final BulkResponse bulkItemResponse = clientSupplier.get().bulk(bulkRequest).get(); | ||
if (bulkItemResponse.hasFailures()) { | ||
exception = new BulkExecutionException(bulkItemResponse); | ||
} else { | ||
exception = null; | ||
break; | ||
} | ||
} catch (Exception e) { | ||
exception = e; | ||
} finally { | ||
if (exception != null) { | ||
LOG.error( | ||
"ElasticsearchWriter: Failed to bulk save chunk of " + | ||
Objects.toString(chunk.size()) + " items, attempts remaining: " + | ||
Objects.toString(maxRetries - attempts), | ||
exception); | ||
} | ||
attempts += 1; | ||
} | ||
} | ||
|
||
try { | ||
final BulkRequest bulkRequest = new BulkRequest().add(chunk); | ||
final BulkResponse bulkItemResponse = clientSupplier.get().bulk(bulkRequest).get(); | ||
if (bulkItemResponse.hasFailures()) { | ||
error.accept(new BulkExecutionException(bulkItemResponse)); | ||
if (exception != null) { | ||
if (exception instanceof BulkExecutionException) { | ||
// This may result in no exception being thrown, depending on callback. | ||
error.accept((BulkExecutionException) exception); | ||
} else { | ||
throw exception; | ||
} | ||
} | ||
} catch (Exception e) { | ||
throw new RuntimeException(e); | ||
|
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
@soundofjw can you undo the unrelated changes, keep the format?