|
|
@@ -57,7 +57,6 @@ import org.elasticsearch.cluster.service.ClusterService;
|
|
|
import org.elasticsearch.common.inject.Inject;
|
|
|
import org.elasticsearch.common.lease.Releasable;
|
|
|
import org.elasticsearch.common.unit.TimeValue;
|
|
|
-import org.elasticsearch.common.util.concurrent.AbstractRunnable;
|
|
|
import org.elasticsearch.common.util.concurrent.AtomicArray;
|
|
|
import org.elasticsearch.index.Index;
|
|
|
import org.elasticsearch.index.IndexNotFoundException;
|
|
|
@@ -253,8 +252,13 @@ public class TransportBulkAction extends HandledTransportAction<BulkRequest, Bul
|
|
|
@Override
|
|
|
public void onResponse(CreateIndexResponse result) {
|
|
|
if (counter.decrementAndGet() == 0) {
|
|
|
- threadPool.executor(executorName).execute(
|
|
|
- () -> executeBulk(task, bulkRequest, startTime, listener, responses, indicesThatCannotBeCreated));
|
|
|
+ threadPool.executor(executorName).execute(new ActionRunnable<>(listener) {
|
|
|
+
|
|
|
+ @Override
|
|
|
+ protected void doRun() {
|
|
|
+ executeBulk(task, bulkRequest, startTime, listener, responses, indicesThatCannotBeCreated);
|
|
|
+ }
|
|
|
+ });
|
|
|
}
|
|
|
}
|
|
|
|
|
|
@@ -274,11 +278,22 @@ public class TransportBulkAction extends HandledTransportAction<BulkRequest, Bul
|
|
|
}
|
|
|
}
|
|
|
if (counter.decrementAndGet() == 0) {
|
|
|
- threadPool.executor(executorName).execute(() -> executeBulk(task, bulkRequest, startTime,
|
|
|
- ActionListener.wrap(listener::onResponse, inner -> {
|
|
|
- inner.addSuppressed(e);
|
|
|
- listener.onFailure(inner);
|
|
|
- }), responses, indicesThatCannotBeCreated));
|
|
|
+ final ActionListener<BulkResponse> wrappedListener = ActionListener.wrap(listener::onResponse, inner -> {
|
|
|
+ inner.addSuppressed(e);
|
|
|
+ listener.onFailure(inner);
|
|
|
+ });
|
|
|
+ threadPool.executor(executorName).execute(new ActionRunnable<>(wrappedListener) {
|
|
|
+ @Override
|
|
|
+ protected void doRun() {
|
|
|
+ executeBulk(task, bulkRequest, startTime, wrappedListener, responses, indicesThatCannotBeCreated);
|
|
|
+ }
|
|
|
+
|
|
|
+ @Override
|
|
|
+ public void onRejection(Exception rejectedException) {
|
|
|
+ rejectedException.addSuppressed(e);
|
|
|
+ super.onRejection(rejectedException);
|
|
|
+ }
|
|
|
+ });
|
|
|
}
|
|
|
}
|
|
|
});
|
|
|
@@ -698,14 +713,9 @@ public class TransportBulkAction extends HandledTransportAction<BulkRequest, Bul
|
|
|
assert Thread.currentThread().getName().contains(executorName);
|
|
|
doInternalExecute(task, bulkRequest, executorName, actionListener);
|
|
|
} else {
|
|
|
- threadPool.executor(executorName).execute(new AbstractRunnable() {
|
|
|
- @Override
|
|
|
- public void onFailure(Exception e) {
|
|
|
- listener.onFailure(e);
|
|
|
- }
|
|
|
-
|
|
|
+ threadPool.executor(executorName).execute(new ActionRunnable<>(listener) {
|
|
|
@Override
|
|
|
- protected void doRun() throws Exception {
|
|
|
+ protected void doRun() {
|
|
|
doInternalExecute(task, bulkRequest, executorName, actionListener);
|
|
|
}
|
|
|
|