Skip to content

Commit 052dbb4

Browse files
Optimize error handling after lazy rollovers (#111572)
This commit improves the performance of the error-handling process after a lazy rollover or an index creation failed.
1 parent f0339ed commit 052dbb4

File tree

7 files changed

+164
-255
lines changed

7 files changed

+164
-255
lines changed

server/src/main/java/org/elasticsearch/action/bulk/BulkOperation.java

Lines changed: 0 additions & 20 deletions
Original file line numberDiff line numberDiff line change
@@ -87,7 +87,6 @@ final class BulkOperation extends ActionRunnable<BulkResponse> {
8787
private final ConcurrentLinkedQueue<BulkItemRequest> failureStoreRedirects = new ConcurrentLinkedQueue<>();
8888
private final long startTimeNanos;
8989
private final ClusterStateObserver observer;
90-
private final Map<String, IndexNotFoundException> indicesThatCannotBeCreated;
9190
private final Executor executor;
9291
private final LongSupplier relativeTimeProvider;
9392
private final FailureStoreDocumentConverter failureStoreDocumentConverter;
@@ -107,7 +106,6 @@ final class BulkOperation extends ActionRunnable<BulkResponse> {
107106
BulkRequest bulkRequest,
108107
NodeClient client,
109108
AtomicArray<BulkItemResponse> responses,
110-
Map<String, IndexNotFoundException> indicesThatCannotBeCreated,
111109
IndexNameExpressionResolver indexNameExpressionResolver,
112110
LongSupplier relativeTimeProvider,
113111
long startTimeNanos,
@@ -122,7 +120,6 @@ final class BulkOperation extends ActionRunnable<BulkResponse> {
122120
bulkRequest,
123121
client,
124122
responses,
125-
indicesThatCannotBeCreated,
126123
indexNameExpressionResolver,
127124
relativeTimeProvider,
128125
startTimeNanos,
@@ -141,7 +138,6 @@ final class BulkOperation extends ActionRunnable<BulkResponse> {
141138
BulkRequest bulkRequest,
142139
NodeClient client,
143140
AtomicArray<BulkItemResponse> responses,
144-
Map<String, IndexNotFoundException> indicesThatCannotBeCreated,
145141
IndexNameExpressionResolver indexNameExpressionResolver,
146142
LongSupplier relativeTimeProvider,
147143
long startTimeNanos,
@@ -158,7 +154,6 @@ final class BulkOperation extends ActionRunnable<BulkResponse> {
158154
this.bulkRequest = bulkRequest;
159155
this.listener = listener;
160156
this.startTimeNanos = startTimeNanos;
161-
this.indicesThatCannotBeCreated = indicesThatCannotBeCreated;
162157
this.executor = executor;
163158
this.relativeTimeProvider = relativeTimeProvider;
164159
this.indexNameExpressionResolver = indexNameExpressionResolver;
@@ -298,9 +293,6 @@ private Map<ShardId, List<BulkItemRequest>> groupRequestsByShards(
298293
if (addFailureIfRequiresAliasAndAliasIsMissing(docWriteRequest, bulkItemRequest.id(), metadata)) {
299294
continue;
300295
}
301-
if (addFailureIfIndexCannotBeCreated(docWriteRequest, bulkItemRequest.id())) {
302-
continue;
303-
}
304296
if (addFailureIfRequiresDataStreamAndNoParentDataStream(docWriteRequest, bulkItemRequest.id(), metadata)) {
305297
continue;
306298
}
@@ -759,18 +751,6 @@ private boolean addFailureIfIndexIsClosed(DocWriteRequest<?> request, Index conc
759751
return false;
760752
}
761753

762-
private boolean addFailureIfIndexCannotBeCreated(DocWriteRequest<?> request, int idx) {
763-
IndexNotFoundException cannotCreate = indicesThatCannotBeCreated.get(request.index());
764-
if (cannotCreate != null) {
765-
var failureStoreStatus = isFailureStoreRequest(request)
766-
? IndexDocFailureStoreStatus.FAILED
767-
: IndexDocFailureStoreStatus.NOT_APPLICABLE_OR_UNKNOWN;
768-
addFailureAndDiscardRequest(request, idx, request.index(), cannotCreate, failureStoreStatus);
769-
return true;
770-
}
771-
return false;
772-
}
773-
774754
private static boolean isFailureStoreRequest(DocWriteRequest<?> request) {
775755
return request instanceof IndexRequest ir && ir.isWriteToFailureStore();
776756
}

server/src/main/java/org/elasticsearch/action/bulk/TransportBulkAction.java

Lines changed: 63 additions & 50 deletions
Original file line numberDiff line numberDiff line change
@@ -43,7 +43,6 @@
4343
import org.elasticsearch.common.io.stream.Writeable;
4444
import org.elasticsearch.common.util.concurrent.AtomicArray;
4545
import org.elasticsearch.features.FeatureService;
46-
import org.elasticsearch.index.IndexNotFoundException;
4746
import org.elasticsearch.index.IndexingPressure;
4847
import org.elasticsearch.index.VersionType;
4948
import org.elasticsearch.indices.SystemIndices;
@@ -60,6 +59,7 @@
6059
import java.util.Objects;
6160
import java.util.Set;
6261
import java.util.SortedMap;
62+
import java.util.concurrent.ConcurrentHashMap;
6363
import java.util.concurrent.Executor;
6464
import java.util.function.Function;
6565
import java.util.function.LongSupplier;
@@ -351,29 +351,36 @@ protected void createMissingIndicesAndIndexData(
351351
final AtomicArray<BulkItemResponse> responses = new AtomicArray<>(bulkRequest.requests.size());
352352
// Optimizing when there are no prerequisite actions
353353
if (indicesToAutoCreate.isEmpty() && dataStreamsToBeRolledOver.isEmpty() && failureStoresToBeRolledOver.isEmpty()) {
354-
executeBulk(task, bulkRequest, startTimeNanos, listener, executor, responses, Map.of());
354+
executeBulk(task, bulkRequest, startTimeNanos, listener, executor, responses);
355355
return;
356356
}
357-
final Map<String, IndexNotFoundException> indicesThatCannotBeCreated = new HashMap<>();
357+
Map<String, Exception> indicesExceptions = new ConcurrentHashMap<>();
358+
Map<String, Exception> dataStreamExceptions = new ConcurrentHashMap<>();
359+
Map<String, Exception> failureStoreExceptions = new ConcurrentHashMap<>();
358360
Runnable executeBulkRunnable = () -> executor.execute(new ActionRunnable<>(listener) {
359361
@Override
360362
protected void doRun() {
361-
executeBulk(task, bulkRequest, startTimeNanos, listener, executor, responses, indicesThatCannotBeCreated);
363+
failRequestsWhenPrerequisiteActionFailed(
364+
indicesExceptions,
365+
dataStreamExceptions,
366+
failureStoreExceptions,
367+
bulkRequest,
368+
responses
369+
);
370+
executeBulk(task, bulkRequest, startTimeNanos, listener, executor, responses);
362371
}
363372
});
364373
try (RefCountingRunnable refs = new RefCountingRunnable(executeBulkRunnable)) {
365-
createIndices(bulkRequest, indicesToAutoCreate, indicesThatCannotBeCreated, responses, refs);
366-
rollOverDataStreams(bulkRequest, dataStreamsToBeRolledOver, false, responses, refs);
367-
rollOverDataStreams(bulkRequest, failureStoresToBeRolledOver, true, responses, refs);
374+
createIndices(indicesToAutoCreate, refs, indicesExceptions);
375+
rollOverDataStreams(bulkRequest, dataStreamsToBeRolledOver, false, refs, dataStreamExceptions);
376+
rollOverDataStreams(bulkRequest, failureStoresToBeRolledOver, true, refs, failureStoreExceptions);
368377
}
369378
}
370379

371380
private void createIndices(
372-
BulkRequest bulkRequest,
373381
Map<String, CreateIndexRequest> indicesToAutoCreate,
374-
Map<String, IndexNotFoundException> indicesThatCannotBeCreated,
375-
AtomicArray<BulkItemResponse> responses,
376-
RefCountingRunnable refs
382+
RefCountingRunnable refs,
383+
final Map<String, Exception> indicesExceptions
377384
) {
378385
for (Map.Entry<String, CreateIndexRequest> indexEntry : indicesToAutoCreate.entrySet()) {
379386
final String index = indexEntry.getKey();
@@ -384,25 +391,26 @@ public void onResponse(CreateIndexResponse createIndexResponse) {}
384391
@Override
385392
public void onFailure(Exception e) {
386393
final Throwable cause = ExceptionsHelper.unwrapCause(e);
387-
if (cause instanceof IndexNotFoundException indexNotFoundException) {
388-
synchronized (indicesThatCannotBeCreated) {
389-
indicesThatCannotBeCreated.put(index, indexNotFoundException);
390-
}
391-
} else if ((cause instanceof ResourceAlreadyExistsException) == false) {
394+
if ((cause instanceof ResourceAlreadyExistsException) == false) {
392395
// fail all requests involving this index, if create didn't work
393-
failRequestsWhenPrerequisiteActionFailed(index, bulkRequest, responses, e);
396+
indicesExceptions.put(index, e);
394397
}
395398
}
396399
}, refs.acquire()));
397400
}
398401
}
399402

403+
// Separate method to allow for overriding in tests.
404+
void createIndex(CreateIndexRequest createIndexRequest, ActionListener<CreateIndexResponse> listener) {
405+
client.execute(AutoCreateAction.INSTANCE, createIndexRequest, listener);
406+
}
407+
400408
private void rollOverDataStreams(
401409
BulkRequest bulkRequest,
402410
Set<String> dataStreamsToBeRolledOver,
403411
boolean targetFailureStore,
404-
AtomicArray<BulkItemResponse> responses,
405-
RefCountingRunnable refs
412+
RefCountingRunnable refs,
413+
Map<String, Exception> dataStreamExceptions
406414
) {
407415
for (String dataStream : dataStreamsToBeRolledOver) {
408416
RolloverRequest rolloverRequest = new RolloverRequest(dataStream, null);
@@ -416,7 +424,7 @@ private void rollOverDataStreams(
416424
}
417425
// We are executing a lazy rollover because it is an action specialised for this situation, when we want an
418426
// unconditional and performant rollover.
419-
rolloverClient.execute(LazyRolloverAction.INSTANCE, rolloverRequest, ActionListener.releaseAfter(new ActionListener<>() {
427+
rollOver(rolloverRequest, ActionListener.releaseAfter(new ActionListener<>() {
420428

421429
@Override
422430
public void onResponse(RolloverResponse result) {
@@ -431,26 +439,52 @@ public void onResponse(RolloverResponse result) {
431439

432440
@Override
433441
public void onFailure(Exception e) {
434-
failRequestsWhenPrerequisiteActionFailed(dataStream, bulkRequest, responses, e);
442+
dataStreamExceptions.put(dataStream, e);
435443
}
436444
}, refs.acquire()));
437445
}
438446
}
439447

448+
// Separate method to allow for overriding in tests.
449+
void rollOver(RolloverRequest rolloverRequest, ActionListener<RolloverResponse> listener) {
450+
rolloverClient.execute(LazyRolloverAction.INSTANCE, rolloverRequest, listener);
451+
}
452+
440453
/**
441-
* Fails all requests involving this index or data stream because the prerequisite action failed too.
454+
* Mark all the requests for which the prerequisite action failed (i.e. index creation or data stream/failure store rollover) as failed.
442455
*/
443-
private static void failRequestsWhenPrerequisiteActionFailed(
444-
String target,
456+
private void failRequestsWhenPrerequisiteActionFailed(
457+
Map<String, Exception> indicesExceptions,
458+
Map<String, Exception> dataStreamExceptions,
459+
Map<String, Exception> failureStoreExceptions,
445460
BulkRequest bulkRequest,
446-
AtomicArray<BulkItemResponse> responses,
447-
Exception error
461+
AtomicArray<BulkItemResponse> responses
448462
) {
463+
if (indicesExceptions.isEmpty() && dataStreamExceptions.isEmpty() && failureStoreExceptions.isEmpty()) {
464+
return;
465+
}
449466
for (int i = 0; i < bulkRequest.requests.size(); i++) {
450467
DocWriteRequest<?> request = bulkRequest.requests.get(i);
451-
if (request != null && setResponseFailureIfIndexMatches(responses, i, request, target, error)) {
452-
bulkRequest.requests.set(i, null);
468+
if (request == null) {
469+
continue;
453470
}
471+
var exception = indicesExceptions.get(request.index());
472+
if (exception == null) {
473+
if (request instanceof IndexRequest indexRequest && indexRequest.isWriteToFailureStore()) {
474+
exception = failureStoreExceptions.get(request.index());
475+
} else {
476+
exception = dataStreamExceptions.get(request.index());
477+
}
478+
}
479+
if (exception == null) {
480+
continue;
481+
}
482+
var failureStoreStatus = request instanceof IndexRequest ir && ir.isWriteToFailureStore()
483+
? IndexDocFailureStoreStatus.FAILED
484+
: IndexDocFailureStoreStatus.NOT_APPLICABLE_OR_UNKNOWN;
485+
var failure = new BulkItemResponse.Failure(request.index(), request.id(), exception, failureStoreStatus);
486+
responses.set(i, BulkItemResponse.failure(i, request.opType(), failure));
487+
bulkRequest.requests.set(i, null);
454488
}
455489
}
456490

@@ -532,33 +566,13 @@ private static boolean isSystemIndex(SortedMap<String, IndexAbstraction> indices
532566
}
533567
}
534568

535-
void createIndex(CreateIndexRequest createIndexRequest, ActionListener<CreateIndexResponse> listener) {
536-
client.execute(AutoCreateAction.INSTANCE, createIndexRequest, listener);
537-
}
538-
539-
private static boolean setResponseFailureIfIndexMatches(
540-
AtomicArray<BulkItemResponse> responses,
541-
int idx,
542-
DocWriteRequest<?> request,
543-
String index,
544-
Exception e
545-
) {
546-
if (index.equals(request.index())) {
547-
BulkItemResponse.Failure failure = new BulkItemResponse.Failure(request.index(), request.id(), e);
548-
responses.set(idx, BulkItemResponse.failure(idx, request.opType(), failure));
549-
return true;
550-
}
551-
return false;
552-
}
553-
554569
void executeBulk(
555570
Task task,
556571
BulkRequest bulkRequest,
557572
long startTimeNanos,
558573
ActionListener<BulkResponse> listener,
559574
Executor executor,
560-
AtomicArray<BulkItemResponse> responses,
561-
Map<String, IndexNotFoundException> indicesThatCannotBeCreated
575+
AtomicArray<BulkItemResponse> responses
562576
) {
563577
new BulkOperation(
564578
task,
@@ -568,7 +582,6 @@ void executeBulk(
568582
bulkRequest,
569583
client,
570584
responses,
571-
indicesThatCannotBeCreated,
572585
indexNameExpressionResolver,
573586
relativeTimeNanosProvider,
574587
startTimeNanos,

server/src/test/java/org/elasticsearch/action/bulk/BulkOperationTests.java

Lines changed: 0 additions & 6 deletions
Original file line numberDiff line numberDiff line change
@@ -42,7 +42,6 @@
4242
import org.elasticsearch.common.util.concurrent.AtomicArray;
4343
import org.elasticsearch.common.util.concurrent.EsExecutors;
4444
import org.elasticsearch.common.util.concurrent.ThreadContext;
45-
import org.elasticsearch.index.IndexNotFoundException;
4645
import org.elasticsearch.index.IndexVersion;
4746
import org.elasticsearch.index.mapper.MapperException;
4847
import org.elasticsearch.index.shard.ShardId;
@@ -1022,7 +1021,6 @@ private BulkOperation newBulkOperation(NodeClient client, BulkRequest request, A
10221021
client,
10231022
request,
10241023
new AtomicArray<>(request.numberOfActions()),
1025-
Map.of(),
10261024
mockObserver(DEFAULT_STATE),
10271025
listener,
10281026
new FailureStoreDocumentConverter()
@@ -1040,7 +1038,6 @@ private BulkOperation newBulkOperation(
10401038
client,
10411039
request,
10421040
new AtomicArray<>(request.numberOfActions()),
1043-
Map.of(),
10441041
mockObserver(DEFAULT_STATE),
10451042
listener,
10461043
failureStoreDocumentConverter
@@ -1059,7 +1056,6 @@ private BulkOperation newBulkOperation(
10591056
client,
10601057
request,
10611058
new AtomicArray<>(request.numberOfActions()),
1062-
Map.of(),
10631059
observer,
10641060
listener,
10651061
new FailureStoreDocumentConverter()
@@ -1071,7 +1067,6 @@ private BulkOperation newBulkOperation(
10711067
NodeClient client,
10721068
BulkRequest request,
10731069
AtomicArray<BulkItemResponse> existingResponses,
1074-
Map<String, IndexNotFoundException> indicesThatCanNotBeCreated,
10751070
ClusterStateObserver observer,
10761071
ActionListener<BulkResponse> listener,
10771072
FailureStoreDocumentConverter failureStoreDocumentConverter
@@ -1100,7 +1095,6 @@ private BulkOperation newBulkOperation(
11001095
request,
11011096
client,
11021097
existingResponses,
1103-
indicesThatCanNotBeCreated,
11041098
indexNameExpressionResolver,
11051099
() -> endTime,
11061100
timeZero,

0 commit comments

Comments
 (0)