mirror of
https://github.com/honeymoose/OpenSearch.git
synced 2025-02-22 21:05:23 +00:00
Decouple BulkProcessor from client implementation (#23373)
This commit modifies the BulkProcessor to be decoupled from the client implementation. Instead it just takes a BiConsumer<BulkRequest, ActionListener<BulkResponse>> that executes the BulkRequest.
This commit is contained in:
parent
0257a7b97a
commit
5b1fbe5e6c
@ -26,6 +26,7 @@ import org.elasticsearch.ElasticsearchStatusException;
|
||||
import org.elasticsearch.action.DocWriteRequest;
|
||||
import org.elasticsearch.action.DocWriteResponse;
|
||||
import org.elasticsearch.action.bulk.BulkItemResponse;
|
||||
import org.elasticsearch.action.bulk.BulkProcessor;
|
||||
import org.elasticsearch.action.bulk.BulkRequest;
|
||||
import org.elasticsearch.action.bulk.BulkResponse;
|
||||
import org.elasticsearch.action.delete.DeleteRequest;
|
||||
@ -38,6 +39,9 @@ import org.elasticsearch.action.update.UpdateRequest;
|
||||
import org.elasticsearch.action.update.UpdateResponse;
|
||||
import org.elasticsearch.common.Strings;
|
||||
import org.elasticsearch.common.bytes.BytesReference;
|
||||
import org.elasticsearch.common.settings.Settings;
|
||||
import org.elasticsearch.common.unit.ByteSizeUnit;
|
||||
import org.elasticsearch.common.unit.ByteSizeValue;
|
||||
import org.elasticsearch.common.xcontent.XContentBuilder;
|
||||
import org.elasticsearch.common.xcontent.XContentType;
|
||||
import org.elasticsearch.index.VersionType;
|
||||
@ -46,10 +50,13 @@ import org.elasticsearch.rest.RestStatus;
|
||||
import org.elasticsearch.script.Script;
|
||||
import org.elasticsearch.script.ScriptType;
|
||||
import org.elasticsearch.search.fetch.subphase.FetchSourceContext;
|
||||
import org.elasticsearch.threadpool.ThreadPool;
|
||||
|
||||
import java.io.IOException;
|
||||
import java.util.Collections;
|
||||
import java.util.Map;
|
||||
import java.util.concurrent.TimeUnit;
|
||||
import java.util.concurrent.atomic.AtomicReference;
|
||||
|
||||
import static java.util.Collections.singletonMap;
|
||||
|
||||
@ -577,6 +584,102 @@ public class CrudIT extends ESRestHighLevelClientTestCase {
|
||||
assertTrue(bulkResponse.getTookInMillis() > 0);
|
||||
assertEquals(nbItems, bulkResponse.getItems().length);
|
||||
|
||||
validateBulkResponses(nbItems, errors, bulkResponse, bulkRequest);
|
||||
}
|
||||
|
||||
public void testBulkProcessorIntegration() throws IOException, InterruptedException {
|
||||
int nbItems = randomIntBetween(10, 100);
|
||||
boolean[] errors = new boolean[nbItems];
|
||||
|
||||
XContentType xContentType = randomFrom(XContentType.JSON, XContentType.SMILE);
|
||||
|
||||
AtomicReference<BulkResponse> responseRef = new AtomicReference<>();
|
||||
AtomicReference<BulkRequest> requestRef = new AtomicReference<>();
|
||||
AtomicReference<Throwable> error = new AtomicReference<>();
|
||||
|
||||
BulkProcessor.Listener listener = new BulkProcessor.Listener() {
|
||||
@Override
|
||||
public void beforeBulk(long executionId, BulkRequest request) {
|
||||
|
||||
}
|
||||
|
||||
@Override
|
||||
public void afterBulk(long executionId, BulkRequest request, BulkResponse response) {
|
||||
responseRef.set(response);
|
||||
requestRef.set(request);
|
||||
}
|
||||
|
||||
@Override
|
||||
public void afterBulk(long executionId, BulkRequest request, Throwable failure) {
|
||||
error.set(failure);
|
||||
}
|
||||
};
|
||||
|
||||
ThreadPool threadPool = new ThreadPool(Settings.builder().put("node.name", getClass().getName()).build());
|
||||
try(BulkProcessor processor = new BulkProcessor.Builder(highLevelClient()::bulkAsync, listener, threadPool)
|
||||
.setConcurrentRequests(0)
|
||||
.setBulkSize(new ByteSizeValue(5, ByteSizeUnit.GB))
|
||||
.setBulkActions(nbItems + 1)
|
||||
.build()) {
|
||||
for (int i = 0; i < nbItems; i++) {
|
||||
String id = String.valueOf(i);
|
||||
boolean erroneous = randomBoolean();
|
||||
errors[i] = erroneous;
|
||||
|
||||
DocWriteRequest.OpType opType = randomFrom(DocWriteRequest.OpType.values());
|
||||
if (opType == DocWriteRequest.OpType.DELETE) {
|
||||
if (erroneous == false) {
|
||||
assertEquals(RestStatus.CREATED,
|
||||
highLevelClient().index(new IndexRequest("index", "test", id).source("field", -1)).status());
|
||||
}
|
||||
DeleteRequest deleteRequest = new DeleteRequest("index", "test", id);
|
||||
processor.add(deleteRequest);
|
||||
|
||||
} else {
|
||||
if (opType == DocWriteRequest.OpType.INDEX) {
|
||||
IndexRequest indexRequest = new IndexRequest("index", "test", id).source(xContentType, "id", i);
|
||||
if (erroneous) {
|
||||
indexRequest.version(12L);
|
||||
}
|
||||
processor.add(indexRequest);
|
||||
|
||||
} else if (opType == DocWriteRequest.OpType.CREATE) {
|
||||
IndexRequest createRequest = new IndexRequest("index", "test", id).source(xContentType, "id", i).create(true);
|
||||
if (erroneous) {
|
||||
assertEquals(RestStatus.CREATED, highLevelClient().index(createRequest).status());
|
||||
}
|
||||
processor.add(createRequest);
|
||||
|
||||
} else if (opType == DocWriteRequest.OpType.UPDATE) {
|
||||
UpdateRequest updateRequest = new UpdateRequest("index", "test", id)
|
||||
.doc(new IndexRequest().source(xContentType, "id", i));
|
||||
if (erroneous == false) {
|
||||
assertEquals(RestStatus.CREATED,
|
||||
highLevelClient().index(new IndexRequest("index", "test", id).source("field", -1)).status());
|
||||
}
|
||||
processor.add(updateRequest);
|
||||
}
|
||||
}
|
||||
}
|
||||
assertNull(responseRef.get());
|
||||
assertNull(requestRef.get());
|
||||
}
|
||||
|
||||
|
||||
BulkResponse bulkResponse = responseRef.get();
|
||||
BulkRequest bulkRequest = requestRef.get();
|
||||
|
||||
assertEquals(RestStatus.OK, bulkResponse.status());
|
||||
assertTrue(bulkResponse.getTookInMillis() > 0);
|
||||
assertEquals(nbItems, bulkResponse.getItems().length);
|
||||
assertNull(error.get());
|
||||
|
||||
validateBulkResponses(nbItems, errors, bulkResponse, bulkRequest);
|
||||
|
||||
terminate(threadPool);
|
||||
}
|
||||
|
||||
private void validateBulkResponses(int nbItems, boolean[] errors, BulkResponse bulkResponse, BulkRequest bulkRequest) {
|
||||
for (int i = 0; i < nbItems; i++) {
|
||||
BulkItemResponse bulkItemResponse = bulkResponse.getItems()[i];
|
||||
|
||||
|
@ -19,6 +19,7 @@
|
||||
|
||||
package org.elasticsearch.action.bulk;
|
||||
|
||||
import org.elasticsearch.action.ActionListener;
|
||||
import org.elasticsearch.action.DocWriteRequest;
|
||||
import org.elasticsearch.action.delete.DeleteRequest;
|
||||
import org.elasticsearch.action.index.IndexRequest;
|
||||
@ -28,17 +29,14 @@ import org.elasticsearch.common.bytes.BytesReference;
|
||||
import org.elasticsearch.common.unit.ByteSizeUnit;
|
||||
import org.elasticsearch.common.unit.ByteSizeValue;
|
||||
import org.elasticsearch.common.unit.TimeValue;
|
||||
import org.elasticsearch.common.util.concurrent.EsExecutors;
|
||||
import org.elasticsearch.common.util.concurrent.FutureUtils;
|
||||
import org.elasticsearch.common.xcontent.XContentType;
|
||||
import org.elasticsearch.threadpool.ThreadPool;
|
||||
|
||||
import java.io.Closeable;
|
||||
import java.util.Objects;
|
||||
import java.util.concurrent.Executors;
|
||||
import java.util.concurrent.ScheduledFuture;
|
||||
import java.util.concurrent.ScheduledThreadPoolExecutor;
|
||||
import java.util.concurrent.TimeUnit;
|
||||
import java.util.concurrent.atomic.AtomicLong;
|
||||
import java.util.function.BiConsumer;
|
||||
|
||||
/**
|
||||
* A bulk processor is a thread safe bulk processing class, allowing to easily set when to "flush" a new bulk request
|
||||
@ -66,7 +64,7 @@ public class BulkProcessor implements Closeable {
|
||||
|
||||
/**
|
||||
* Callback after a failed execution of bulk request.
|
||||
*
|
||||
* <p>
|
||||
* Note that in case an instance of <code>InterruptedException</code> is passed, which means that request processing has been
|
||||
* cancelled externally, the thread's interruption status has been restored prior to calling this method.
|
||||
*/
|
||||
@ -78,10 +76,10 @@ public class BulkProcessor implements Closeable {
|
||||
*/
|
||||
public static class Builder {
|
||||
|
||||
private final Client client;
|
||||
private final BiConsumer<BulkRequest, ActionListener<BulkResponse>> consumer;
|
||||
private final Listener listener;
|
||||
private final ThreadPool threadPool;
|
||||
|
||||
private String name;
|
||||
private int concurrentRequests = 1;
|
||||
private int bulkActions = 1000;
|
||||
private ByteSizeValue bulkSize = new ByteSizeValue(5, ByteSizeUnit.MB);
|
||||
@ -92,17 +90,10 @@ public class BulkProcessor implements Closeable {
|
||||
* Creates a builder of bulk processor with the client to use and the listener that will be used
|
||||
* to be notified on the completion of bulk requests.
|
||||
*/
|
||||
public Builder(Client client, Listener listener) {
|
||||
this.client = client;
|
||||
public Builder(BiConsumer<BulkRequest, ActionListener<BulkResponse>> consumer, Listener listener, ThreadPool threadPool) {
|
||||
this.consumer = consumer;
|
||||
this.listener = listener;
|
||||
}
|
||||
|
||||
/**
|
||||
* Sets an optional name to identify this bulk processor.
|
||||
*/
|
||||
public Builder setName(String name) {
|
||||
this.name = name;
|
||||
return this;
|
||||
this.threadPool = threadPool;
|
||||
}
|
||||
|
||||
/**
|
||||
@ -164,7 +155,7 @@ public class BulkProcessor implements Closeable {
|
||||
* Builds a new bulk processor.
|
||||
*/
|
||||
public BulkProcessor build() {
|
||||
return new BulkProcessor(client, backoffPolicy, listener, name, concurrentRequests, bulkActions, bulkSize, flushInterval);
|
||||
return new BulkProcessor(consumer, backoffPolicy, listener, concurrentRequests, bulkActions, bulkSize, flushInterval, threadPool);
|
||||
}
|
||||
}
|
||||
|
||||
@ -172,15 +163,13 @@ public class BulkProcessor implements Closeable {
|
||||
Objects.requireNonNull(client, "client");
|
||||
Objects.requireNonNull(listener, "listener");
|
||||
|
||||
return new Builder(client, listener);
|
||||
return new Builder(client::bulk, listener, client.threadPool());
|
||||
}
|
||||
|
||||
private final int bulkActions;
|
||||
private final long bulkSize;
|
||||
|
||||
|
||||
private final ScheduledThreadPoolExecutor scheduler;
|
||||
private final ScheduledFuture<?> scheduledFuture;
|
||||
private final ThreadPool.Cancellable cancellableFlushTask;
|
||||
|
||||
private final AtomicLong executionIdGen = new AtomicLong();
|
||||
|
||||
@ -189,22 +178,21 @@ public class BulkProcessor implements Closeable {
|
||||
|
||||
private volatile boolean closed = false;
|
||||
|
||||
BulkProcessor(Client client, BackoffPolicy backoffPolicy, Listener listener, @Nullable String name, int concurrentRequests, int bulkActions, ByteSizeValue bulkSize, @Nullable TimeValue flushInterval) {
|
||||
BulkProcessor(BiConsumer<BulkRequest, ActionListener<BulkResponse>> consumer, BackoffPolicy backoffPolicy, Listener listener,
|
||||
int concurrentRequests, int bulkActions, ByteSizeValue bulkSize, @Nullable TimeValue flushInterval,
|
||||
ThreadPool threadPool) {
|
||||
this.bulkActions = bulkActions;
|
||||
this.bulkSize = bulkSize.getBytes();
|
||||
|
||||
this.bulkRequest = new BulkRequest();
|
||||
this.bulkRequestHandler = (concurrentRequests == 0) ? BulkRequestHandler.syncHandler(client, backoffPolicy, listener) : BulkRequestHandler.asyncHandler(client, backoffPolicy, listener, concurrentRequests);
|
||||
|
||||
if (flushInterval != null) {
|
||||
this.scheduler = (ScheduledThreadPoolExecutor) Executors.newScheduledThreadPool(1, EsExecutors.daemonThreadFactory(client.settings(), (name != null ? "[" + name + "]" : "") + "bulk_processor"));
|
||||
this.scheduler.setExecuteExistingDelayedTasksAfterShutdownPolicy(false);
|
||||
this.scheduler.setContinueExistingPeriodicTasksAfterShutdownPolicy(false);
|
||||
this.scheduledFuture = this.scheduler.scheduleWithFixedDelay(new Flush(), flushInterval.millis(), flushInterval.millis(), TimeUnit.MILLISECONDS);
|
||||
if (concurrentRequests == 0) {
|
||||
this.bulkRequestHandler = BulkRequestHandler.syncHandler(consumer, backoffPolicy, listener, threadPool);
|
||||
} else {
|
||||
this.scheduler = null;
|
||||
this.scheduledFuture = null;
|
||||
this.bulkRequestHandler = BulkRequestHandler.asyncHandler(consumer, backoffPolicy, listener, threadPool, concurrentRequests);
|
||||
}
|
||||
|
||||
// Start period flushing task after everything is setup
|
||||
this.cancellableFlushTask = startFlushTask(flushInterval, threadPool);
|
||||
}
|
||||
|
||||
/**
|
||||
@ -214,20 +202,20 @@ public class BulkProcessor implements Closeable {
|
||||
public void close() {
|
||||
try {
|
||||
awaitClose(0, TimeUnit.NANOSECONDS);
|
||||
} catch(InterruptedException exc) {
|
||||
} catch (InterruptedException exc) {
|
||||
Thread.currentThread().interrupt();
|
||||
}
|
||||
}
|
||||
|
||||
/**
|
||||
* Closes the processor. If flushing by time is enabled, then it's shutdown. Any remaining bulk actions are flushed.
|
||||
*
|
||||
* <p>
|
||||
* If concurrent requests are not enabled, returns {@code true} immediately.
|
||||
* If concurrent requests are enabled, waits for up to the specified timeout for all bulk requests to complete then returns {@code true},
|
||||
* If the specified waiting time elapses before all bulk requests complete, {@code false} is returned.
|
||||
*
|
||||
* @param timeout The maximum time to wait for the bulk requests to complete
|
||||
* @param unit The time unit of the {@code timeout} argument
|
||||
* @param unit The time unit of the {@code timeout} argument
|
||||
* @return {@code true} if all bulk requests completed and {@code false} if the waiting time elapsed before all the bulk requests completed
|
||||
* @throws InterruptedException If the current thread is interrupted
|
||||
*/
|
||||
@ -236,10 +224,9 @@ public class BulkProcessor implements Closeable {
|
||||
return true;
|
||||
}
|
||||
closed = true;
|
||||
if (this.scheduledFuture != null) {
|
||||
FutureUtils.cancel(this.scheduledFuture);
|
||||
this.scheduler.shutdown();
|
||||
}
|
||||
|
||||
this.cancellableFlushTask.cancel();
|
||||
|
||||
if (bulkRequest.numberOfActions() > 0) {
|
||||
execute();
|
||||
}
|
||||
@ -301,12 +288,28 @@ public class BulkProcessor implements Closeable {
|
||||
* Adds the data from the bytes to be processed by the bulk processor
|
||||
*/
|
||||
public synchronized BulkProcessor add(BytesReference data, @Nullable String defaultIndex, @Nullable String defaultType,
|
||||
@Nullable String defaultPipeline, @Nullable Object payload, XContentType xContentType) throws Exception {
|
||||
@Nullable String defaultPipeline, @Nullable Object payload, XContentType xContentType) throws Exception {
|
||||
bulkRequest.add(data, defaultIndex, defaultType, null, null, null, defaultPipeline, payload, true, xContentType);
|
||||
executeIfNeeded();
|
||||
return this;
|
||||
}
|
||||
|
||||
private ThreadPool.Cancellable startFlushTask(TimeValue flushInterval, ThreadPool threadPool) {
|
||||
if (flushInterval == null) {
|
||||
return new ThreadPool.Cancellable() {
|
||||
@Override
|
||||
public void cancel() {}
|
||||
|
||||
@Override
|
||||
public boolean isCancelled() {
|
||||
return true;
|
||||
}
|
||||
};
|
||||
}
|
||||
|
||||
return threadPool.scheduleWithFixedDelay(new Flush(), flushInterval, ThreadPool.Names.GENERIC);
|
||||
}
|
||||
|
||||
private void executeIfNeeded() {
|
||||
ensureOpen();
|
||||
if (!isOverTheLimit()) {
|
||||
|
@ -22,23 +22,27 @@ import org.apache.logging.log4j.Logger;
|
||||
import org.apache.logging.log4j.message.ParameterizedMessage;
|
||||
import org.apache.logging.log4j.util.Supplier;
|
||||
import org.elasticsearch.action.ActionListener;
|
||||
import org.elasticsearch.client.Client;
|
||||
import org.elasticsearch.common.logging.Loggers;
|
||||
import org.elasticsearch.common.settings.Settings;
|
||||
import org.elasticsearch.common.util.concurrent.EsRejectedExecutionException;
|
||||
import org.elasticsearch.threadpool.ThreadPool;
|
||||
|
||||
import java.util.concurrent.Semaphore;
|
||||
import java.util.concurrent.TimeUnit;
|
||||
import java.util.function.BiConsumer;
|
||||
|
||||
/**
|
||||
* Abstracts the low-level details of bulk request handling
|
||||
*/
|
||||
abstract class BulkRequestHandler {
|
||||
protected final Logger logger;
|
||||
protected final Client client;
|
||||
protected final BiConsumer<BulkRequest, ActionListener<BulkResponse>> consumer;
|
||||
protected final ThreadPool threadPool;
|
||||
|
||||
protected BulkRequestHandler(Client client) {
|
||||
this.client = client;
|
||||
this.logger = Loggers.getLogger(getClass(), client.settings());
|
||||
protected BulkRequestHandler(BiConsumer<BulkRequest, ActionListener<BulkResponse>> consumer, ThreadPool threadPool) {
|
||||
this.logger = Loggers.getLogger(getClass());
|
||||
this.consumer = consumer;
|
||||
this.threadPool = threadPool;
|
||||
}
|
||||
|
||||
|
||||
@ -47,20 +51,25 @@ abstract class BulkRequestHandler {
|
||||
public abstract boolean awaitClose(long timeout, TimeUnit unit) throws InterruptedException;
|
||||
|
||||
|
||||
public static BulkRequestHandler syncHandler(Client client, BackoffPolicy backoffPolicy, BulkProcessor.Listener listener) {
|
||||
return new SyncBulkRequestHandler(client, backoffPolicy, listener);
|
||||
public static BulkRequestHandler syncHandler(BiConsumer<BulkRequest, ActionListener<BulkResponse>> consumer,
|
||||
BackoffPolicy backoffPolicy, BulkProcessor.Listener listener,
|
||||
ThreadPool threadPool) {
|
||||
return new SyncBulkRequestHandler(consumer, backoffPolicy, listener, threadPool);
|
||||
}
|
||||
|
||||
public static BulkRequestHandler asyncHandler(Client client, BackoffPolicy backoffPolicy, BulkProcessor.Listener listener, int concurrentRequests) {
|
||||
return new AsyncBulkRequestHandler(client, backoffPolicy, listener, concurrentRequests);
|
||||
public static BulkRequestHandler asyncHandler(BiConsumer<BulkRequest, ActionListener<BulkResponse>> consumer,
|
||||
BackoffPolicy backoffPolicy, BulkProcessor.Listener listener,
|
||||
ThreadPool threadPool, int concurrentRequests) {
|
||||
return new AsyncBulkRequestHandler(consumer, backoffPolicy, listener, threadPool, concurrentRequests);
|
||||
}
|
||||
|
||||
private static class SyncBulkRequestHandler extends BulkRequestHandler {
|
||||
private final BulkProcessor.Listener listener;
|
||||
private final BackoffPolicy backoffPolicy;
|
||||
|
||||
SyncBulkRequestHandler(Client client, BackoffPolicy backoffPolicy, BulkProcessor.Listener listener) {
|
||||
super(client);
|
||||
SyncBulkRequestHandler(BiConsumer<BulkRequest, ActionListener<BulkResponse>> consumer, BackoffPolicy backoffPolicy,
|
||||
BulkProcessor.Listener listener, ThreadPool threadPool) {
|
||||
super(consumer, threadPool);
|
||||
this.backoffPolicy = backoffPolicy;
|
||||
this.listener = listener;
|
||||
}
|
||||
@ -71,9 +80,10 @@ abstract class BulkRequestHandler {
|
||||
try {
|
||||
listener.beforeBulk(executionId, bulkRequest);
|
||||
BulkResponse bulkResponse = Retry
|
||||
.on(EsRejectedExecutionException.class)
|
||||
.policy(backoffPolicy)
|
||||
.withSyncBackoff(client, bulkRequest);
|
||||
.on(EsRejectedExecutionException.class)
|
||||
.policy(backoffPolicy)
|
||||
.using(threadPool)
|
||||
.withSyncBackoff(consumer, bulkRequest, Settings.EMPTY);
|
||||
afterCalled = true;
|
||||
listener.afterBulk(executionId, bulkRequest, bulkResponse);
|
||||
} catch (InterruptedException e) {
|
||||
@ -103,8 +113,10 @@ abstract class BulkRequestHandler {
|
||||
private final Semaphore semaphore;
|
||||
private final int concurrentRequests;
|
||||
|
||||
private AsyncBulkRequestHandler(Client client, BackoffPolicy backoffPolicy, BulkProcessor.Listener listener, int concurrentRequests) {
|
||||
super(client);
|
||||
private AsyncBulkRequestHandler(BiConsumer<BulkRequest, ActionListener<BulkResponse>> consumer, BackoffPolicy backoffPolicy,
|
||||
BulkProcessor.Listener listener, ThreadPool threadPool,
|
||||
int concurrentRequests) {
|
||||
super(consumer, threadPool);
|
||||
this.backoffPolicy = backoffPolicy;
|
||||
assert concurrentRequests > 0;
|
||||
this.listener = listener;
|
||||
@ -121,26 +133,27 @@ abstract class BulkRequestHandler {
|
||||
semaphore.acquire();
|
||||
acquired = true;
|
||||
Retry.on(EsRejectedExecutionException.class)
|
||||
.policy(backoffPolicy)
|
||||
.withAsyncBackoff(client, bulkRequest, new ActionListener<BulkResponse>() {
|
||||
@Override
|
||||
public void onResponse(BulkResponse response) {
|
||||
try {
|
||||
listener.afterBulk(executionId, bulkRequest, response);
|
||||
} finally {
|
||||
semaphore.release();
|
||||
}
|
||||
.policy(backoffPolicy)
|
||||
.using(threadPool)
|
||||
.withAsyncBackoff(consumer, bulkRequest, new ActionListener<BulkResponse>() {
|
||||
@Override
|
||||
public void onResponse(BulkResponse response) {
|
||||
try {
|
||||
listener.afterBulk(executionId, bulkRequest, response);
|
||||
} finally {
|
||||
semaphore.release();
|
||||
}
|
||||
}
|
||||
|
||||
@Override
|
||||
public void onFailure(Exception e) {
|
||||
try {
|
||||
listener.afterBulk(executionId, bulkRequest, e);
|
||||
} finally {
|
||||
semaphore.release();
|
||||
}
|
||||
@Override
|
||||
public void onFailure(Exception e) {
|
||||
try {
|
||||
listener.afterBulk(executionId, bulkRequest, e);
|
||||
} finally {
|
||||
semaphore.release();
|
||||
}
|
||||
});
|
||||
}
|
||||
}, Settings.EMPTY);
|
||||
bulkRequestSetupSuccessful = true;
|
||||
} catch (InterruptedException e) {
|
||||
Thread.currentThread().interrupt();
|
||||
|
@ -20,19 +20,25 @@ package org.elasticsearch.action.bulk;
|
||||
|
||||
import org.apache.logging.log4j.Logger;
|
||||
import org.elasticsearch.ExceptionsHelper;
|
||||
import org.elasticsearch.action.ActionFuture;
|
||||
import org.elasticsearch.action.ActionListener;
|
||||
import org.elasticsearch.action.support.PlainActionFuture;
|
||||
import org.elasticsearch.client.Client;
|
||||
import org.elasticsearch.common.logging.Loggers;
|
||||
import org.elasticsearch.common.settings.Settings;
|
||||
import org.elasticsearch.common.unit.TimeValue;
|
||||
import org.elasticsearch.common.util.concurrent.EsExecutors;
|
||||
import org.elasticsearch.common.util.concurrent.FutureUtils;
|
||||
import org.elasticsearch.threadpool.ThreadPool;
|
||||
|
||||
import java.util.ArrayList;
|
||||
import java.util.Iterator;
|
||||
import java.util.List;
|
||||
import java.util.concurrent.Executors;
|
||||
import java.util.concurrent.ScheduledExecutorService;
|
||||
import java.util.concurrent.ScheduledFuture;
|
||||
import java.util.concurrent.ScheduledThreadPoolExecutor;
|
||||
import java.util.concurrent.TimeUnit;
|
||||
import java.util.function.BiConsumer;
|
||||
import java.util.function.BiFunction;
|
||||
import java.util.function.Predicate;
|
||||
|
||||
/**
|
||||
@ -42,11 +48,16 @@ public class Retry {
|
||||
private final Class<? extends Throwable> retryOnThrowable;
|
||||
|
||||
private BackoffPolicy backoffPolicy;
|
||||
private ThreadPool threadPool;
|
||||
|
||||
public static Retry on(Class<? extends Throwable> retryOnThrowable) {
|
||||
return new Retry(retryOnThrowable);
|
||||
}
|
||||
|
||||
Retry(Class<? extends Throwable> retryOnThrowable) {
|
||||
this.retryOnThrowable = retryOnThrowable;
|
||||
}
|
||||
|
||||
/**
|
||||
* @param backoffPolicy The backoff policy that defines how long and how often to wait for retries.
|
||||
*/
|
||||
@ -55,42 +66,48 @@ public class Retry {
|
||||
return this;
|
||||
}
|
||||
|
||||
Retry(Class<? extends Throwable> retryOnThrowable) {
|
||||
this.retryOnThrowable = retryOnThrowable;
|
||||
}
|
||||
|
||||
/**
|
||||
* Invokes #bulk(BulkRequest, ActionListener) on the provided client. Backs off on the provided exception and delegates results to the
|
||||
* provided listener.
|
||||
*
|
||||
* @param client Client invoking the bulk request.
|
||||
* @param bulkRequest The bulk request that should be executed.
|
||||
* @param listener A listener that is invoked when the bulk request finishes or completes with an exception. The listener is not
|
||||
* @param threadPool The threadPool that will be used to schedule retries.
|
||||
*/
|
||||
public void withAsyncBackoff(Client client, BulkRequest bulkRequest, ActionListener<BulkResponse> listener) {
|
||||
AsyncRetryHandler r = new AsyncRetryHandler(retryOnThrowable, backoffPolicy, client, listener);
|
||||
r.execute(bulkRequest);
|
||||
|
||||
public Retry using(ThreadPool threadPool) {
|
||||
this.threadPool = threadPool;
|
||||
return this;
|
||||
}
|
||||
|
||||
/**
|
||||
* Invokes #bulk(BulkRequest) on the provided client. Backs off on the provided exception.
|
||||
*
|
||||
* @param client Client invoking the bulk request.
|
||||
* Invokes #apply(BulkRequest, ActionListener). Backs off on the provided exception and delegates results to the
|
||||
* provided listener. Retries will be attempted using the provided schedule function
|
||||
* @param consumer The consumer to which apply the request and listener
|
||||
* @param bulkRequest The bulk request that should be executed.
|
||||
* @param listener A listener that is invoked when the bulk request finishes or completes with an exception. The listener is not
|
||||
* @param settings settings
|
||||
*/
|
||||
public void withAsyncBackoff(BiConsumer<BulkRequest, ActionListener<BulkResponse>> consumer, BulkRequest bulkRequest, ActionListener<BulkResponse> listener, Settings settings) {
|
||||
RetryHandler r = new RetryHandler(retryOnThrowable, backoffPolicy, consumer, listener, settings, threadPool);
|
||||
r.execute(bulkRequest);
|
||||
}
|
||||
|
||||
/**
|
||||
* Invokes #apply(BulkRequest, ActionListener). Backs off on the provided exception. Retries will be attempted using
|
||||
* the provided schedule function.
|
||||
*
|
||||
* @param consumer The consumer to which apply the request and listener
|
||||
* @param bulkRequest The bulk request that should be executed.
|
||||
* @param settings settings
|
||||
* @return the bulk response as returned by the client.
|
||||
* @throws Exception Any exception thrown by the callable.
|
||||
*/
|
||||
public BulkResponse withSyncBackoff(Client client, BulkRequest bulkRequest) throws Exception {
|
||||
return SyncRetryHandler
|
||||
.create(retryOnThrowable, backoffPolicy, client)
|
||||
.executeBlocking(bulkRequest)
|
||||
.actionGet();
|
||||
public BulkResponse withSyncBackoff(BiConsumer<BulkRequest, ActionListener<BulkResponse>> consumer, BulkRequest bulkRequest, Settings settings) throws Exception {
|
||||
PlainActionFuture<BulkResponse> actionFuture = PlainActionFuture.newFuture();
|
||||
RetryHandler r = new RetryHandler(retryOnThrowable, backoffPolicy, consumer, actionFuture, settings, threadPool);
|
||||
r.execute(bulkRequest);
|
||||
return actionFuture.actionGet();
|
||||
}
|
||||
|
||||
static class AbstractRetryHandler implements ActionListener<BulkResponse> {
|
||||
static class RetryHandler implements ActionListener<BulkResponse> {
|
||||
private final Logger logger;
|
||||
private final Client client;
|
||||
private final ThreadPool threadPool;
|
||||
private final BiConsumer<BulkRequest, ActionListener<BulkResponse>> consumer;
|
||||
private final ActionListener<BulkResponse> listener;
|
||||
private final Iterator<TimeValue> backoff;
|
||||
private final Class<? extends Throwable> retryOnThrowable;
|
||||
@ -102,12 +119,15 @@ public class Retry {
|
||||
private volatile BulkRequest currentBulkRequest;
|
||||
private volatile ScheduledFuture<?> scheduledRequestFuture;
|
||||
|
||||
AbstractRetryHandler(Class<? extends Throwable> retryOnThrowable, BackoffPolicy backoffPolicy, Client client, ActionListener<BulkResponse> listener) {
|
||||
RetryHandler(Class<? extends Throwable> retryOnThrowable, BackoffPolicy backoffPolicy,
|
||||
BiConsumer<BulkRequest, ActionListener<BulkResponse>> consumer, ActionListener<BulkResponse> listener,
|
||||
Settings settings, ThreadPool threadPool) {
|
||||
this.retryOnThrowable = retryOnThrowable;
|
||||
this.backoff = backoffPolicy.iterator();
|
||||
this.client = client;
|
||||
this.consumer = consumer;
|
||||
this.listener = listener;
|
||||
this.logger = Loggers.getLogger(getClass(), client.settings());
|
||||
this.logger = Loggers.getLogger(getClass(), settings);
|
||||
this.threadPool = threadPool;
|
||||
// in contrast to System.currentTimeMillis(), nanoTime() uses a monotonic clock under the hood
|
||||
this.startTimestampNanos = System.nanoTime();
|
||||
}
|
||||
@ -142,9 +162,8 @@ public class Retry {
|
||||
assert backoff.hasNext();
|
||||
TimeValue next = backoff.next();
|
||||
logger.trace("Retry of bulk request scheduled in {} ms.", next.millis());
|
||||
Runnable retry = () -> this.execute(bulkRequestForRetry);
|
||||
retry = client.threadPool().getThreadContext().preserveContext(retry);
|
||||
scheduledRequestFuture = client.threadPool().schedule(next, ThreadPool.Names.SAME, retry);
|
||||
Runnable command = threadPool.getThreadContext().preserveContext(() -> this.execute(bulkRequestForRetry));
|
||||
scheduledRequestFuture = threadPool.schedule(next, ThreadPool.Names.SAME, command);
|
||||
}
|
||||
|
||||
private BulkRequest createBulkRequestForRetry(BulkResponse bulkItemResponses) {
|
||||
@ -208,32 +227,7 @@ public class Retry {
|
||||
|
||||
public void execute(BulkRequest bulkRequest) {
|
||||
this.currentBulkRequest = bulkRequest;
|
||||
client.bulk(bulkRequest, this);
|
||||
}
|
||||
}
|
||||
|
||||
static class AsyncRetryHandler extends AbstractRetryHandler {
|
||||
AsyncRetryHandler(Class<? extends Throwable> retryOnThrowable, BackoffPolicy backoffPolicy, Client client, ActionListener<BulkResponse> listener) {
|
||||
super(retryOnThrowable, backoffPolicy, client, listener);
|
||||
}
|
||||
}
|
||||
|
||||
static class SyncRetryHandler extends AbstractRetryHandler {
|
||||
private final PlainActionFuture<BulkResponse> actionFuture;
|
||||
|
||||
public static SyncRetryHandler create(Class<? extends Throwable> retryOnThrowable, BackoffPolicy backoffPolicy, Client client) {
|
||||
PlainActionFuture<BulkResponse> actionFuture = PlainActionFuture.newFuture();
|
||||
return new SyncRetryHandler(retryOnThrowable, backoffPolicy, client, actionFuture);
|
||||
}
|
||||
|
||||
SyncRetryHandler(Class<? extends Throwable> retryOnThrowable, BackoffPolicy backoffPolicy, Client client, PlainActionFuture<BulkResponse> actionFuture) {
|
||||
super(retryOnThrowable, backoffPolicy, client, actionFuture);
|
||||
this.actionFuture = actionFuture;
|
||||
}
|
||||
|
||||
public ActionFuture<BulkResponse> executeBlocking(BulkRequest bulkRequest) {
|
||||
super.execute(bulkRequest);
|
||||
return actionFuture;
|
||||
consumer.accept(bulkRequest, this);
|
||||
}
|
||||
}
|
||||
}
|
||||
|
@ -36,6 +36,7 @@ import org.elasticsearch.action.delete.DeleteRequest;
|
||||
import org.elasticsearch.action.index.IndexRequest;
|
||||
import org.elasticsearch.client.ParentTaskAssigningClient;
|
||||
import org.elasticsearch.cluster.ClusterState;
|
||||
import org.elasticsearch.common.settings.Settings;
|
||||
import org.elasticsearch.common.unit.ByteSizeValue;
|
||||
import org.elasticsearch.common.unit.TimeValue;
|
||||
import org.elasticsearch.common.util.concurrent.AbstractRunnable;
|
||||
@ -74,7 +75,6 @@ import java.util.function.BiFunction;
|
||||
import static java.lang.Math.max;
|
||||
import static java.lang.Math.min;
|
||||
import static java.util.Collections.emptyList;
|
||||
import static java.util.Collections.emptyMap;
|
||||
import static java.util.Collections.unmodifiableList;
|
||||
import static org.elasticsearch.action.bulk.BackoffPolicy.exponentialBackoff;
|
||||
import static org.elasticsearch.action.bulk.byscroll.AbstractBulkByScrollRequest.SIZE_ALL_MATCHES;
|
||||
@ -106,6 +106,7 @@ public abstract class AbstractAsyncBulkByScrollAction<Request extends AbstractBu
|
||||
private final ActionListener<BulkByScrollResponse> listener;
|
||||
private final Retry bulkRetry;
|
||||
private final ScrollableHitSource scrollSource;
|
||||
private final Settings settings;
|
||||
|
||||
/**
|
||||
* This BiFunction is used to apply various changes depending of the Reindex action and the search hit,
|
||||
@ -114,19 +115,26 @@ public abstract class AbstractAsyncBulkByScrollAction<Request extends AbstractBu
|
||||
*/
|
||||
private final BiFunction<RequestWrapper<?>, ScrollableHitSource.Hit, RequestWrapper<?>> scriptApplier;
|
||||
|
||||
public AbstractAsyncBulkByScrollAction(WorkingBulkByScrollTask task, Logger logger, ParentTaskAssigningClient client,
|
||||
ThreadPool threadPool, Request mainRequest, ScriptService scriptService, ClusterState clusterState,
|
||||
ActionListener<BulkByScrollResponse> listener) {
|
||||
this(task, logger, client, threadPool, mainRequest, scriptService, clusterState, listener, client.settings());
|
||||
}
|
||||
|
||||
public AbstractAsyncBulkByScrollAction(WorkingBulkByScrollTask task, Logger logger, ParentTaskAssigningClient client,
|
||||
ThreadPool threadPool, Request mainRequest, ScriptService scriptService, ClusterState clusterState,
|
||||
ActionListener<BulkByScrollResponse> listener) {
|
||||
ActionListener<BulkByScrollResponse> listener, Settings settings) {
|
||||
this.task = task;
|
||||
this.logger = logger;
|
||||
this.client = client;
|
||||
this.settings = settings;
|
||||
this.threadPool = threadPool;
|
||||
this.scriptService = scriptService;
|
||||
this.clusterState = clusterState;
|
||||
this.mainRequest = mainRequest;
|
||||
this.listener = listener;
|
||||
BackoffPolicy backoffPolicy = buildBackoffPolicy();
|
||||
bulkRetry = Retry.on(EsRejectedExecutionException.class).policy(BackoffPolicy.wrap(backoffPolicy, task::countBulkRetry));
|
||||
bulkRetry = Retry.on(EsRejectedExecutionException.class).policy(BackoffPolicy.wrap(backoffPolicy, task::countBulkRetry)).using(threadPool);
|
||||
scrollSource = buildScrollableResultSource(backoffPolicy);
|
||||
scriptApplier = Objects.requireNonNull(buildScriptApplier(), "script applier must not be null");
|
||||
/*
|
||||
@ -329,7 +337,7 @@ public abstract class AbstractAsyncBulkByScrollAction<Request extends AbstractBu
|
||||
finishHim(null);
|
||||
return;
|
||||
}
|
||||
bulkRetry.withAsyncBackoff(client, request, new ActionListener<BulkResponse>() {
|
||||
bulkRetry.withAsyncBackoff(client::bulk, request, new ActionListener<BulkResponse>() {
|
||||
@Override
|
||||
public void onResponse(BulkResponse response) {
|
||||
onBulkResponse(thisBatchStartTime, response);
|
||||
@ -339,7 +347,7 @@ public abstract class AbstractAsyncBulkByScrollAction<Request extends AbstractBu
|
||||
public void onFailure(Exception e) {
|
||||
finishHim(e);
|
||||
}
|
||||
});
|
||||
}, settings);
|
||||
}
|
||||
|
||||
/**
|
||||
|
@ -62,7 +62,7 @@ public class BulkProcessorIT extends ESIntegTestCase {
|
||||
BulkProcessorTestListener listener = new BulkProcessorTestListener(latch);
|
||||
|
||||
int numDocs = randomIntBetween(10, 100);
|
||||
try (BulkProcessor processor = BulkProcessor.builder(client(), listener).setName("foo")
|
||||
try (BulkProcessor processor = BulkProcessor.builder(client(), listener)
|
||||
//let's make sure that the bulk action limit trips, one single execution will index all the documents
|
||||
.setConcurrentRequests(randomIntBetween(0, 1)).setBulkActions(numDocs)
|
||||
.setFlushInterval(TimeValue.timeValueHours(24)).setBulkSize(new ByteSizeValue(1, ByteSizeUnit.GB))
|
||||
@ -86,7 +86,7 @@ public class BulkProcessorIT extends ESIntegTestCase {
|
||||
|
||||
int numDocs = randomIntBetween(10, 100);
|
||||
|
||||
try (BulkProcessor processor = BulkProcessor.builder(client(), listener).setName("foo")
|
||||
try (BulkProcessor processor = BulkProcessor.builder(client(), listener)
|
||||
//let's make sure that this bulk won't be automatically flushed
|
||||
.setConcurrentRequests(randomIntBetween(0, 10)).setBulkActions(numDocs + randomIntBetween(1, 100))
|
||||
.setFlushInterval(TimeValue.timeValueHours(24)).setBulkSize(new ByteSizeValue(1, ByteSizeUnit.GB)).build()) {
|
||||
@ -203,7 +203,7 @@ public class BulkProcessorIT extends ESIntegTestCase {
|
||||
BulkProcessorTestListener listener = new BulkProcessorTestListener();
|
||||
|
||||
int numDocs = randomIntBetween(10, 100);
|
||||
BulkProcessor processor = BulkProcessor.builder(client(), listener).setName("foo")
|
||||
BulkProcessor processor = BulkProcessor.builder(client(), listener)
|
||||
//let's make sure that the bulk action limit trips, one single execution will index all the documents
|
||||
.setConcurrentRequests(randomIntBetween(0, 1)).setBulkActions(numDocs)
|
||||
.setFlushInterval(TimeValue.timeValueHours(24)).setBulkSize(new ByteSizeValue(randomIntBetween(1, 10),
|
||||
|
@ -59,7 +59,6 @@ public class BulkProcessorRetryIT extends ESIntegTestCase {
|
||||
.build();
|
||||
}
|
||||
|
||||
|
||||
public void testBulkRejectionLoadWithoutBackoff() throws Throwable {
|
||||
boolean rejectedExecutionExpected = true;
|
||||
executeBulkRejectionLoad(BackoffPolicy.noBackoff(), rejectedExecutionExpected);
|
||||
|
@ -87,7 +87,8 @@ public class RetryTests extends ESTestCase {
|
||||
BulkResponse response = Retry
|
||||
.on(EsRejectedExecutionException.class)
|
||||
.policy(backoff)
|
||||
.withSyncBackoff(bulkClient, bulkRequest);
|
||||
.using(bulkClient.threadPool())
|
||||
.withSyncBackoff(bulkClient::bulk, bulkRequest, bulkClient.settings());
|
||||
|
||||
assertFalse(response.hasFailures());
|
||||
assertThat(response.getItems().length, equalTo(bulkRequest.numberOfActions()));
|
||||
@ -100,7 +101,8 @@ public class RetryTests extends ESTestCase {
|
||||
BulkResponse response = Retry
|
||||
.on(EsRejectedExecutionException.class)
|
||||
.policy(backoff)
|
||||
.withSyncBackoff(bulkClient, bulkRequest);
|
||||
.using(bulkClient.threadPool())
|
||||
.withSyncBackoff(bulkClient::bulk, bulkRequest, bulkClient.settings());
|
||||
|
||||
assertTrue(response.hasFailures());
|
||||
assertThat(response.getItems().length, equalTo(bulkRequest.numberOfActions()));
|
||||
@ -113,7 +115,8 @@ public class RetryTests extends ESTestCase {
|
||||
BulkRequest bulkRequest = createBulkRequest();
|
||||
Retry.on(EsRejectedExecutionException.class)
|
||||
.policy(backoff)
|
||||
.withAsyncBackoff(bulkClient, bulkRequest, listener);
|
||||
.using(bulkClient.threadPool())
|
||||
.withAsyncBackoff(bulkClient::bulk, bulkRequest, listener, bulkClient.settings());
|
||||
|
||||
listener.awaitCallbacksCalled();
|
||||
listener.assertOnResponseCalled();
|
||||
@ -129,7 +132,8 @@ public class RetryTests extends ESTestCase {
|
||||
BulkRequest bulkRequest = createBulkRequest();
|
||||
Retry.on(EsRejectedExecutionException.class)
|
||||
.policy(backoff)
|
||||
.withAsyncBackoff(bulkClient, bulkRequest, listener);
|
||||
.using(bulkClient.threadPool())
|
||||
.withAsyncBackoff(bulkClient::bulk, bulkRequest, listener, bulkClient.settings());
|
||||
|
||||
listener.awaitCallbacksCalled();
|
||||
|
||||
|
@ -660,7 +660,7 @@ public class AsyncBulkByScrollActionTests extends ESTestCase {
|
||||
private class DummyAsyncBulkByScrollAction extends AbstractAsyncBulkByScrollAction<DummyAbstractBulkByScrollRequest> {
|
||||
DummyAsyncBulkByScrollAction() {
|
||||
super(testTask, AsyncBulkByScrollActionTests.this.logger, new ParentTaskAssigningClient(client, localNode, testTask),
|
||||
client.threadPool(), testRequest, null, null, listener);
|
||||
client.threadPool(), testRequest, null, null, listener, Settings.EMPTY);
|
||||
}
|
||||
|
||||
@Override
|
||||
|
@ -251,10 +251,16 @@ public class TransportReindexAction extends HandledTransportAction<ReindexReques
|
||||
*/
|
||||
private List<Thread> createdThreads = emptyList();
|
||||
|
||||
AsyncIndexBySearchAction(WorkingBulkByScrollTask task, Logger logger, ParentTaskAssigningClient client,
|
||||
ThreadPool threadPool, ReindexRequest request, ScriptService scriptService, ClusterState clusterState,
|
||||
ActionListener<BulkByScrollResponse> listener) {
|
||||
this(task, logger, client, threadPool, request, scriptService, clusterState, listener, client.settings());
|
||||
}
|
||||
|
||||
AsyncIndexBySearchAction(WorkingBulkByScrollTask task, Logger logger, ParentTaskAssigningClient client,
|
||||
ThreadPool threadPool, ReindexRequest request, ScriptService scriptService, ClusterState clusterState,
|
||||
ActionListener<BulkByScrollResponse> listener) {
|
||||
super(task, logger, client, threadPool, request, scriptService, clusterState, listener);
|
||||
ActionListener<BulkByScrollResponse> listener, Settings settings) {
|
||||
super(task, logger, client, threadPool, request, scriptService, clusterState, listener, settings);
|
||||
}
|
||||
|
||||
@Override
|
||||
|
@ -90,10 +90,16 @@ public class TransportUpdateByQueryAction extends HandledTransportAction<UpdateB
|
||||
* Simple implementation of update-by-query using scrolling and bulk.
|
||||
*/
|
||||
static class AsyncIndexBySearchAction extends AbstractAsyncBulkByScrollAction<UpdateByQueryRequest> {
|
||||
AsyncIndexBySearchAction(WorkingBulkByScrollTask task, Logger logger, ParentTaskAssigningClient client,
|
||||
ThreadPool threadPool, UpdateByQueryRequest request, ScriptService scriptService, ClusterState clusterState,
|
||||
ActionListener<BulkByScrollResponse> listener) {
|
||||
this(task, logger, client, threadPool, request, scriptService, clusterState, listener, client.settings());
|
||||
}
|
||||
|
||||
AsyncIndexBySearchAction(WorkingBulkByScrollTask task, Logger logger, ParentTaskAssigningClient client,
|
||||
ThreadPool threadPool, UpdateByQueryRequest request, ScriptService scriptService, ClusterState clusterState,
|
||||
ActionListener<BulkByScrollResponse> listener) {
|
||||
super(task, logger, client, threadPool, request, scriptService, clusterState, listener);
|
||||
ActionListener<BulkByScrollResponse> listener, Settings settings) {
|
||||
super(task, logger, client, threadPool, request, scriptService, clusterState, listener, settings);
|
||||
}
|
||||
|
||||
@Override
|
||||
|
@ -25,6 +25,7 @@ import org.elasticsearch.action.bulk.byscroll.BulkByScrollResponse;
|
||||
import org.elasticsearch.action.bulk.byscroll.ScrollableHitSource.Hit;
|
||||
import org.elasticsearch.action.index.IndexRequest;
|
||||
import org.elasticsearch.action.search.SearchRequest;
|
||||
import org.elasticsearch.common.settings.Settings;
|
||||
|
||||
/**
|
||||
* Index-by-search test for ttl, timestamp, and routing.
|
||||
@ -81,7 +82,7 @@ public class ReindexMetadataTests extends AbstractAsyncBulkByScrollActionMetadat
|
||||
private class TestAction extends TransportReindexAction.AsyncIndexBySearchAction {
|
||||
TestAction() {
|
||||
super(ReindexMetadataTests.this.task, ReindexMetadataTests.this.logger, null, ReindexMetadataTests.this.threadPool, request(),
|
||||
null, null, listener());
|
||||
null, null, listener(), Settings.EMPTY);
|
||||
}
|
||||
|
||||
public ReindexRequest mainRequest() {
|
||||
|
@ -23,6 +23,7 @@ import org.elasticsearch.action.bulk.byscroll.BulkByScrollResponse;
|
||||
import org.elasticsearch.action.index.IndexRequest;
|
||||
import org.elasticsearch.action.search.SearchRequest;
|
||||
import org.elasticsearch.common.lucene.uid.Versions;
|
||||
import org.elasticsearch.common.settings.Settings;
|
||||
import org.elasticsearch.script.ScriptService;
|
||||
|
||||
import java.util.Map;
|
||||
@ -112,6 +113,6 @@ public class ReindexScriptTests extends AbstractAsyncBulkByScrollActionScriptTes
|
||||
@Override
|
||||
protected TransportReindexAction.AsyncIndexBySearchAction action(ScriptService scriptService, ReindexRequest request) {
|
||||
return new TransportReindexAction.AsyncIndexBySearchAction(task, logger, null, threadPool, request, scriptService, null,
|
||||
listener());
|
||||
listener(), Settings.EMPTY);
|
||||
}
|
||||
}
|
||||
|
@ -76,8 +76,8 @@ public class RetryTests extends ESSingleNodeTestCase {
|
||||
for (int i = 0; i < DOC_COUNT; i++) {
|
||||
bulk.add(client().prepareIndex("source", "test").setSource("foo", "bar " + i));
|
||||
}
|
||||
Retry retry = Retry.on(EsRejectedExecutionException.class).policy(BackoffPolicy.exponentialBackoff());
|
||||
BulkResponse response = retry.withSyncBackoff(client(), bulk.request());
|
||||
Retry retry = Retry.on(EsRejectedExecutionException.class).policy(BackoffPolicy.exponentialBackoff()).using(client().threadPool());
|
||||
BulkResponse response = retry.withSyncBackoff(client()::bulk, bulk.request(), client().settings());
|
||||
assertFalse(response.buildFailureMessage(), response.hasFailures());
|
||||
client().admin().indices().prepareRefresh("source").get();
|
||||
}
|
||||
|
@ -25,6 +25,7 @@ import org.elasticsearch.action.bulk.byscroll.BulkByScrollResponse;
|
||||
import org.elasticsearch.action.bulk.byscroll.ScrollableHitSource.Hit;
|
||||
import org.elasticsearch.action.index.IndexRequest;
|
||||
import org.elasticsearch.action.search.SearchRequest;
|
||||
import org.elasticsearch.common.settings.Settings;
|
||||
|
||||
public class UpdateByQueryMetadataTests
|
||||
extends AbstractAsyncBulkByScrollActionMetadataTestCase<UpdateByQueryRequest, BulkByScrollResponse> {
|
||||
@ -47,7 +48,8 @@ public class UpdateByQueryMetadataTests
|
||||
private class TestAction extends TransportUpdateByQueryAction.AsyncIndexBySearchAction {
|
||||
TestAction() {
|
||||
super(UpdateByQueryMetadataTests.this.task, UpdateByQueryMetadataTests.this.logger, null,
|
||||
UpdateByQueryMetadataTests.this.threadPool, request(), null, null, listener());
|
||||
UpdateByQueryMetadataTests.this.threadPool, request(), null, null, listener(),
|
||||
Settings.EMPTY);
|
||||
}
|
||||
|
||||
@Override
|
||||
|
@ -21,6 +21,7 @@ package org.elasticsearch.index.reindex;
|
||||
|
||||
import org.elasticsearch.action.bulk.byscroll.BulkByScrollResponse;
|
||||
import org.elasticsearch.action.search.SearchRequest;
|
||||
import org.elasticsearch.common.settings.Settings;
|
||||
import org.elasticsearch.script.ScriptService;
|
||||
|
||||
import java.util.Date;
|
||||
@ -56,6 +57,6 @@ public class UpdateByQueryWithScriptTests
|
||||
@Override
|
||||
protected TransportUpdateByQueryAction.AsyncIndexBySearchAction action(ScriptService scriptService, UpdateByQueryRequest request) {
|
||||
return new TransportUpdateByQueryAction.AsyncIndexBySearchAction(task, logger, null, threadPool, request, scriptService, null,
|
||||
listener());
|
||||
listener(), Settings.EMPTY);
|
||||
}
|
||||
}
|
||||
|
Loading…
x
Reference in New Issue
Block a user