|
21 | 21 | import org.elasticsearch.client.Client;
|
22 | 22 | import org.elasticsearch.common.Nullable;
|
23 | 23 | import org.elasticsearch.common.logging.LoggerMessageFormat;
|
| 24 | +import org.elasticsearch.index.mapper.MapperParsingException; |
24 | 25 | import org.elasticsearch.xpack.core.ClientHelper;
|
25 | 26 | import org.elasticsearch.xpack.core.indexing.IndexerState;
|
26 | 27 | import org.elasticsearch.xpack.core.transform.transforms.TransformCheckpoint;
|
|
36 | 37 | import org.elasticsearch.xpack.transform.notifications.TransformAuditor;
|
37 | 38 | import org.elasticsearch.xpack.transform.persistence.SeqNoPrimaryTermAndIndex;
|
38 | 39 | import org.elasticsearch.xpack.transform.persistence.TransformConfigManager;
|
| 40 | +import org.elasticsearch.xpack.transform.utils.ExceptionRootCauseFinder; |
39 | 41 |
|
| 42 | +import java.util.LinkedHashMap; |
40 | 43 | import java.util.Map;
|
| 44 | +import java.util.Map.Entry; |
41 | 45 | import java.util.concurrent.Executor;
|
42 | 46 | import java.util.concurrent.atomic.AtomicBoolean;
|
43 | 47 | import java.util.concurrent.atomic.AtomicReference;
|
@@ -154,31 +158,63 @@ protected void doNextBulk(BulkRequest request, ActionListener<BulkResponse> next
|
154 | 158 | ActionListener.wrap(bulkResponse -> {
|
155 | 159 | if (bulkResponse.hasFailures()) {
|
156 | 160 | 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 | + |
157 | 164 | for (BulkItemResponse item : bulkResponse.getItems()) {
|
158 | 165 | if (item.isFailed()) {
|
| 166 | + deduplicatedFailures.putIfAbsent(item.getFailure().getCause().getClass().getSimpleName(), item); |
159 | 167 | failureCount++;
|
160 | 168 | }
|
161 |
| - // TODO gather information on irrecoverable failures and update isIrrecoverableFailure |
162 | 169 | }
|
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: {}", |
167 | 201 | 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 | + ) |
172 | 216 | );
|
173 |
| - auditBulkFailures = false; |
174 | 217 | }
|
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 |
| - ); |
182 | 218 | } else {
|
183 | 219 | auditBulkFailures = true;
|
184 | 220 | nextPhase.onResponse(bulkResponse);
|
@@ -320,11 +356,27 @@ SeqNoPrimaryTermAndIndex getSeqNoPrimaryTermAndIndex() {
|
320 | 356 | return seqNoPrimaryTermAndIndex.get();
|
321 | 357 | }
|
322 | 358 |
|
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("]"); |
327 | 367 | }
|
| 368 | + String failureMessage = failureMessageBuilder.toString(); |
| 369 | + return failureMessage; |
328 | 370 | }
|
329 | 371 |
|
| 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 | + } |
330 | 382 | }
|
0 commit comments