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

Retry bulk chunk requests to Elasticsearch 2, 5, and 6 #1626

Closed
wants to merge 7 commits 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.
Jump to
Jump to file
Failed to load files.
Diff view
Diff view
Original file line number Diff line number Diff line change
Expand Up @@ -43,6 +43,8 @@
import org.apache.beam.sdk.transforms.windowing.Repeatedly;
import org.apache.beam.sdk.transforms.windowing.TimestampCombiner;
import org.apache.beam.sdk.transforms.windowing.Window;
import org.apache.beam.sdk.util.BackOff;
import org.apache.beam.sdk.util.FluentBackoff;
import org.apache.beam.sdk.values.KV;
import org.apache.beam.sdk.values.PCollection;
import org.apache.beam.sdk.values.PDone;
Expand Down Expand Up @@ -122,15 +124,38 @@ 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,
Expand All @@ -139,46 +164,77 @@ private Bound(final String clusterName,
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
Expand All @@ -189,6 +245,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())
Expand All @@ -199,9 +257,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());
}
}
Expand All @@ -226,19 +285,34 @@ private static class ElasticsearchWriter<T> extends DoFn<KV<Long, Iterable<T>>,
private final ClientSupplier clientSupplier;
private final SerializableFunction<T, Iterable<ActionRequest<?>>> toActionRequests;
private final ThrowingConsumer<BulkExecutionException> error;
private FluentBackoff backoffConfig;
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.clientSupplier = new ClientSupplier(clusterName, servers);
this.toActionRequests = toActionRequests;
this.error = error;
this.maxRetries = maxRetries;
this.retryPause = retryPause;
}

@Setup
public void setup() throws Exception {
this.backoffConfig = FluentBackoff.DEFAULT
Copy link
Contributor

Choose a reason for hiding this comment

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

@soundofjw This is actually needs to be inside a

@Setup 
public void setup() throws Exception {
...
}

.withMaxRetries(this.maxRetries)
.withInitialBackoff(Duration.standardSeconds(this.retryPause));
}

@SuppressWarnings("Duplicates")
@ProcessElement
public void processElement(ProcessContext c) throws Exception {
final Iterable<T> values = c.element().getValue();
Expand All @@ -259,11 +333,50 @@ public void processElement(ProcessContext c) throws Exception {
Iterables.partition(actionRequests::iterator, maxBulkRequestSize);

chunks.forEach(chunk -> {
Exception exception;

final BackOff backoff = backoffConfig.backoff();

do {
Copy link
Contributor

Choose a reason for hiding this comment

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

@soundofjw wdyt of remove this do { ... } while(true) in favor of a

while (BackOffUtils.next(Sleeper.DEFAULT, backoff)) {
 ...
}

removes the need for all of this

try {
  final long sleepTime = backoff.nextBackOffMillis();
  if (sleepTime == BackOff.STOP) {
    break;
  }
  Thread.sleep(sleepTime);
} catch (InterruptedException | IOException e1) {
  LOG.error("Interrupt during backoff", e1);
  break;
}

Copy link
Author

Choose a reason for hiding this comment

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

Your proposal will introduce a sleep at the beginning of every attempt:

  public static boolean next(Sleeper sleeper, BackOff backOff)
      throws InterruptedException, IOException {
    long backOffTime = backOff.nextBackOffMillis();
    if (backOffTime == BackOff.STOP) {
      return false;
    }
    sleeper.sleep(backOffTime);
    return true;
  }```

Copy link
Contributor

Choose a reason for hiding this comment

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

Yeah sorry for poor explanation, yeah correct... what I was trying to say is that I think we can refactor this block by decoupling the logic a little bit and at the same time trying to leverage that util for the retry logic ... I think we can get a better semantic and readability.

try {
final BulkRequest bulkRequest = new BulkRequest().add(chunk).refresh(false);
Copy link
Contributor

Choose a reason for hiding this comment

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

@soundofjw no need for .refresh(false) it's already the default.

Copy link
Author

Choose a reason for hiding this comment

The 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 -1, we're getting refreshes anyway (after a certain point, probably hitting the maximum amount of docs that can hang in the bulk queue) - it's really hard to know why this is happening. But to your point, changing this had no effect.

Copy link
Contributor

@regadas regadas Feb 22, 2019

Choose a reason for hiding this comment

The 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()) {
throw new BulkExecutionException(bulkItemResponse);
} else {
exception = null;
break;
}
} catch (Exception e) {
exception = e;

LOG.error(
"ElasticsearchWriter: Failed to bulk save chunk of " +
Objects.toString(chunk.size()),
exception);

// Backoff
try {
final long sleepTime = backoff.nextBackOffMillis();
if (sleepTime == BackOff.STOP) {
break;
}
Thread.sleep(sleepTime);
} catch (InterruptedException | IOException e1) {
LOG.error("Interrupt during backoff", e1);
break;
}
}
} while (true);

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);
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -57,6 +57,8 @@ final case class ElasticsearchIO[T](esOptions: ElasticsearchOptions) extends Sci
.withFlushInterval(params.flushInterval)
.withNumOfShard(shards)
.withMaxBulkRequestSize(params.maxBulkRequestSize)
.withMaxRetries(params.maxRetries)
.withRetryPause(params.retryPause)
.withError(new beam.ThrowingConsumer[BulkExecutionException] {
override def accept(t: BulkExecutionException): Unit =
params.errorFn(t)
Expand All @@ -75,12 +77,16 @@ object ElasticsearchIO {
private[elasticsearch] val DefaultFlushInterval = Duration.standardSeconds(1)
private[elasticsearch] val DefaultNumShards = 0
private[elasticsearch] val DefaultMaxBulkRequestSize = 3000
private[elasticsearch] val DefaultMaxRetries = 3
private[elasticsearch] val DefaultRetryPause = 5
}

final case class WriteParam[T] private (
f: T => Iterable[ActionRequest[_]],
errorFn: BulkExecutionException => Unit = WriteParam.DefaultErrorFn,
flushInterval: Duration = WriteParam.DefaultFlushInterval,
numOfShards: Long = WriteParam.DefaultNumShards,
maxBulkRequestSize: Int = WriteParam.DefaultMaxBulkRequestSize)
maxBulkRequestSize: Int = WriteParam.DefaultMaxBulkRequestSize,
maxRetries: Int = WriteParam.DefaultMaxRetries,
retryPause: Int = WriteParam.DefaultRetryPause)
}