Skip to content

Commit d2b17bf

Browse files
author
Hendrik Muhs
committed
check bulk indexing error for permanent problems
fixes elastic#50122
1 parent a6fa577 commit d2b17bf

File tree

7 files changed

+159
-23
lines changed

7 files changed

+159
-23
lines changed

x-pack/plugin/core/src/main/java/org/elasticsearch/xpack/core/transform/TransformMessages.java

+2
Original file line numberDiff line numberDiff line change
@@ -79,6 +79,8 @@ public class TransformMessages {
7979
+ "please simplify job or increase heap size on data nodes.";
8080
public static final String LOG_TRANSFORM_PIVOT_SCRIPT_ERROR =
8181
"Failed to execute script with error: [{0}], stack trace: {1}";
82+
public static final String LOG_TRANSFORM_PIVOT_IRRECOVERABLE_BULK_INDEXING_ERROR =
83+
"Failed to index documents into destination index due to permanent error: [{0}]";
8284

8385
public static final String FAILED_TO_PARSE_TRANSFORM_CHECKPOINTS =
8486
"Failed to parse transform checkpoints for [{0}]";
Original file line numberDiff line numberDiff line change
@@ -0,0 +1,31 @@
1+
/*
2+
* Copyright Elasticsearch B.V. and/or licensed to Elasticsearch B.V. under one
3+
* or more contributor license agreements. Licensed under the Elastic License;
4+
* you may not use this file except in compliance with the Elastic License.
5+
*/
6+
7+
package org.elasticsearch.xpack.transform.transforms;
8+
9+
import org.elasticsearch.ElasticsearchException;
10+
11+
// Wrapper for indexing failures thrown internally in the transform indexer
12+
class BulkIndexingException extends ElasticsearchException {
13+
private final boolean irrecoverable;
14+
15+
/**
16+
* Create a BulkIndexingException
17+
*
18+
* @param msg The message
19+
* @param cause The most important cause of the bulk indexing failure
20+
* @param irrecoverable whether this is a permanent or irrecoverable error (controls retry)
21+
* @param args arguments for formating the message
22+
*/
23+
BulkIndexingException(String msg, Throwable cause, boolean irrecoverable, Object... args) {
24+
super(msg, cause, args);
25+
this.irrecoverable = irrecoverable;
26+
}
27+
28+
public boolean isIrrecoverable() {
29+
return irrecoverable;
30+
}
31+
}

x-pack/plugin/transform/src/main/java/org/elasticsearch/xpack/transform/transforms/ClientTransformIndexer.java

+73-21
Original file line numberDiff line numberDiff line change
@@ -21,6 +21,7 @@
2121
import org.elasticsearch.client.Client;
2222
import org.elasticsearch.common.Nullable;
2323
import org.elasticsearch.common.logging.LoggerMessageFormat;
24+
import org.elasticsearch.index.mapper.MapperParsingException;
2425
import org.elasticsearch.xpack.core.ClientHelper;
2526
import org.elasticsearch.xpack.core.indexing.IndexerState;
2627
import org.elasticsearch.xpack.core.transform.transforms.TransformCheckpoint;
@@ -36,8 +37,11 @@
3637
import org.elasticsearch.xpack.transform.notifications.TransformAuditor;
3738
import org.elasticsearch.xpack.transform.persistence.SeqNoPrimaryTermAndIndex;
3839
import org.elasticsearch.xpack.transform.persistence.TransformConfigManager;
40+
import org.elasticsearch.xpack.transform.utils.ExceptionRootCauseFinder;
3941

42+
import java.util.LinkedHashMap;
4043
import java.util.Map;
44+
import java.util.Map.Entry;
4145
import java.util.concurrent.Executor;
4246
import java.util.concurrent.atomic.AtomicBoolean;
4347
import java.util.concurrent.atomic.AtomicReference;
@@ -154,31 +158,63 @@ protected void doNextBulk(BulkRequest request, ActionListener<BulkResponse> next
154158
ActionListener.wrap(bulkResponse -> {
155159
if (bulkResponse.hasFailures()) {
156160
int failureCount = 0;
161+
// dedup the failures by the type of the exception, as they most likely have the same cause
162+
Map<String, BulkItemResponse> deduplicatedFailures = new LinkedHashMap<>();
163+
157164
for (BulkItemResponse item : bulkResponse.getItems()) {
158165
if (item.isFailed()) {
166+
deduplicatedFailures.putIfAbsent(item.getFailure().getCause().getClass().getSimpleName(), item);
159167
failureCount++;
160168
}
161-
// TODO gather information on irrecoverable failures and update isIrrecoverableFailure
162169
}
163-
if (auditBulkFailures) {
164-
String failureMessage = bulkResponse.buildFailureMessage();
165-
logger.debug("[{}] Bulk index failure encountered: {}", getJobId(), failureMessage);
166-
auditor.warning(
170+
171+
// note: bulk failures are audited/logged in {@link TransformIndexer#handleFailure(Exception)}
172+
173+
// This calls AsyncTwoPhaseIndexer#finishWithIndexingFailure
174+
// Determine whether the failure is irrecoverable (transform should go into failed state) or not (transform increments
175+
// the indexing failure counter
176+
// and possibly retries)
177+
Exception irrecoverableException = ExceptionRootCauseFinder.getFirstIrrecoverableExceptionFromBulkResponses(
178+
deduplicatedFailures.values()
179+
);
180+
if (irrecoverableException == null) {
181+
String failureMessage = getBulkIndexDetailedFailureMessage(deduplicatedFailures);
182+
logger.debug("[{}] Bulk index experienced [{}] failures. Significant falures: {}", getJobId(), failureMessage);
183+
184+
Exception firstException = deduplicatedFailures.values().iterator().next().getFailure().getCause();
185+
nextPhase.onFailure(
186+
new BulkIndexingException(
187+
"Bulk index experienced [{}] failures. Significant falures: {}",
188+
firstException,
189+
false,
190+
failureCount,
191+
failureMessage
192+
)
193+
);
194+
} else {
195+
deduplicatedFailures.remove(irrecoverableException.getClass().getSimpleName());
196+
String failureMessage = getBulkIndexDetailedFailureMessage(deduplicatedFailures);
197+
irrecoverableException = decorateBulkIndexException(irrecoverableException);
198+
199+
logger.debug(
200+
"[{}] Bulk index experienced [{}] failures and at least 1 irrecoverable [{}]. Other failures: {}",
167201
getJobId(),
168-
"Experienced at least ["
169-
+ failureCount
170-
+ "] bulk index failures. See the logs of the node running the transform for details. "
171-
+ failureMessage
202+
failureCount,
203+
ExceptionRootCauseFinder.getDetailedMessage(irrecoverableException),
204+
failureMessage
205+
);
206+
207+
nextPhase.onFailure(
208+
new BulkIndexingException(
209+
"Bulk index experienced [{}] failures and at least 1 irrecoverable [{}]. Other failures: {}",
210+
irrecoverableException,
211+
true,
212+
failureCount,
213+
ExceptionRootCauseFinder.getDetailedMessage(irrecoverableException),
214+
failureMessage
215+
)
172216
);
173-
auditBulkFailures = false;
174217
}
175-
// This calls AsyncTwoPhaseIndexer#finishWithIndexingFailure
176-
// It increments the indexing failure, and then calls the `onFailure` logic
177-
nextPhase.onFailure(
178-
new BulkIndexingException(
179-
"Bulk index experienced failures. " + "See the logs of the node running the transform for details."
180-
)
181-
);
182218
} else {
183219
auditBulkFailures = true;
184220
nextPhase.onResponse(bulkResponse);
@@ -320,11 +356,27 @@ SeqNoPrimaryTermAndIndex getSeqNoPrimaryTermAndIndex() {
320356
return seqNoPrimaryTermAndIndex.get();
321357
}
322358

323-
// Considered a recoverable indexing failure
324-
private static class BulkIndexingException extends ElasticsearchException {
325-
BulkIndexingException(String msg, Object... args) {
326-
super(msg, args);
359+
private static String getBulkIndexDetailedFailureMessage(Map<String, BulkItemResponse> failures) {
360+
StringBuilder failureMessageBuilder = new StringBuilder();
361+
for (Entry<String, BulkItemResponse> failure : failures.entrySet()) {
362+
failureMessageBuilder.append("\n[")
363+
.append(failure.getKey())
364+
.append("] message [")
365+
.append(failure.getValue().getFailureMessage())
366+
.append("]");
327367
}
368+
String failureMessage = failureMessageBuilder.toString();
369+
return failureMessage;
328370
}
329371

372+
private static Exception decorateBulkIndexException(Exception irrecoverableException) {
373+
if (irrecoverableException instanceof MapperParsingException) {
374+
return new TransformException(
375+
"Destination index mappings are incompatible with the transform configuration.",
376+
irrecoverableException
377+
);
378+
}
379+
380+
return irrecoverableException;
381+
}
330382
}
Original file line numberDiff line numberDiff line change
@@ -0,0 +1,15 @@
1+
/*
2+
* Copyright Elasticsearch B.V. and/or licensed to Elasticsearch B.V. under one
3+
* or more contributor license agreements. Licensed under the Elastic License;
4+
* you may not use this file except in compliance with the Elastic License.
5+
*/
6+
7+
package org.elasticsearch.xpack.transform.transforms;
8+
9+
import org.elasticsearch.ElasticsearchException;
10+
11+
class TransformException extends ElasticsearchException {
12+
TransformException(String msg, Throwable cause, Object... args) {
13+
super(msg, cause, args);
14+
}
15+
}

x-pack/plugin/transform/src/main/java/org/elasticsearch/xpack/transform/transforms/TransformIndexer.java

+16-2
Original file line numberDiff line numberDiff line change
@@ -480,6 +480,8 @@ synchronized void handleFailure(Exception e) {
480480
} else if (unwrappedException instanceof ScriptException) {
481481
handleScriptException((ScriptException) unwrappedException);
482482
// irrecoverable error without special handling
483+
} else if (unwrappedException instanceof BulkIndexingException && ((BulkIndexingException) unwrappedException).isIrrecoverable()) {
484+
handleIrrecoverableBulkIndexingException((BulkIndexingException) unwrappedException);
483485
} else if (unwrappedException instanceof IndexNotFoundException
484486
|| unwrappedException instanceof AggregationResultUtils.AggregationExtractionException
485487
|| unwrappedException instanceof TransformConfigReloadingException) {
@@ -834,9 +836,21 @@ private void handleScriptException(ScriptException scriptException) {
834836
failIndexer(message);
835837
}
836838

839+
/**
840+
* Handle script exception case. This is error is irrecoverable.
841+
*
842+
* @param bulkIndexingException ScriptException thrown
843+
*/
844+
private void handleIrrecoverableBulkIndexingException(BulkIndexingException bulkIndexingException) {
845+
String message = TransformMessages.getMessage(
846+
TransformMessages.LOG_TRANSFORM_PIVOT_IRRECOVERABLE_BULK_INDEXING_ERROR,
847+
bulkIndexingException.getDetailedMessage()
848+
);
849+
failIndexer(message);
850+
}
851+
837852
protected void failIndexer(String failureMessage) {
838-
logger.error("[{}] transform has failed; experienced: [{}].", getJobId(), failureMessage);
839-
auditor.error(getJobId(), failureMessage);
853+
// note: logging and audit is done as part of context.markAsFailed
840854
context.markAsFailed(failureMessage);
841855
}
842856

x-pack/plugin/transform/src/main/java/org/elasticsearch/xpack/transform/transforms/TransformTask.java

+2
Original file line numberDiff line numberDiff line change
@@ -455,6 +455,8 @@ public synchronized void fail(String reason, ActionListener<Void> listener) {
455455
listener.onResponse(null);
456456
return;
457457
}
458+
459+
logger.error("[{}] transform has failed; experienced: [{}].", transform.getId(), reason);
458460
auditor.error(transform.getId(), reason);
459461
// We should not keep retrying. Either the task will be stopped, or started
460462
// If it is started again, it is registered again.

x-pack/plugin/transform/src/main/java/org/elasticsearch/xpack/transform/utils/ExceptionRootCauseFinder.java

+20
Original file line numberDiff line numberDiff line change
@@ -7,8 +7,12 @@
77
package org.elasticsearch.xpack.transform.utils;
88

99
import org.elasticsearch.ElasticsearchException;
10+
import org.elasticsearch.action.bulk.BulkItemResponse;
1011
import org.elasticsearch.action.search.SearchPhaseExecutionException;
1112
import org.elasticsearch.action.search.ShardSearchFailure;
13+
import org.elasticsearch.index.mapper.MapperParsingException;
14+
15+
import java.util.Collection;
1216

1317
/**
1418
* Set of static utils to find the cause of a search exception.
@@ -53,6 +57,22 @@ public static String getDetailedMessage(Throwable t) {
5357
return t.getMessage();
5458
}
5559

60+
/**
61+
* Return the first irrecoverableException from a collection of bulk responses if there are any.
62+
*
63+
* @param failures a collection of bulk item responses
64+
* @return The first exception considered irrecoverable if there are any, null if no irrecoverable exception found
65+
*/
66+
public static Exception getFirstIrrecoverableExceptionFromBulkResponses(Collection<BulkItemResponse> failures) {
67+
for (BulkItemResponse failure : failures) {
68+
if (failure.getFailure().getCause() instanceof MapperParsingException) {
69+
return failure.getFailure().getCause();
70+
}
71+
}
72+
73+
return null;
74+
}
75+
5676
private ExceptionRootCauseFinder() {}
5777

5878
}

0 commit comments

Comments
 (0)