Skip to content

Commit a87610e

Browse files
authored
[7.x] [ML][Transform] reset failure count when a transform aggregation page is handled successfully (#76355) (#76365)
* [ML][Transform] reset failure count when a transform aggregation page is handled successfully (#76355) Failure count should not only be reset at checkpoints. Checkpoints could have many pages of data. Consequently, we should reset the failure count once we handle a single composite aggregation page. This way, the transform won't mark itself as failed erroneously when it has actually succeeded searches + indexing results within the same checkpoint. closes #76074 * fixing compilation
1 parent 28d6c89 commit a87610e

File tree

3 files changed

+185
-69
lines changed

3 files changed

+185
-69
lines changed

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

Lines changed: 63 additions & 62 deletions
Original file line numberDiff line numberDiff line change
@@ -143,71 +143,72 @@ protected void doNextBulk(BulkRequest request, ActionListener<BulkResponse> next
143143
client,
144144
BulkAction.INSTANCE,
145145
request,
146-
ActionListener.wrap(bulkResponse -> {
147-
if (bulkResponse.hasFailures()) {
148-
int failureCount = 0;
149-
// dedup the failures by the type of the exception, as they most likely have the same cause
150-
Map<String, BulkItemResponse> deduplicatedFailures = new LinkedHashMap<>();
151-
152-
for (BulkItemResponse item : bulkResponse.getItems()) {
153-
if (item.isFailed()) {
154-
deduplicatedFailures.putIfAbsent(item.getFailure().getCause().getClass().getSimpleName(), item);
155-
failureCount++;
156-
}
157-
}
158-
159-
// note: bulk failures are audited/logged in {@link TransformIndexer#handleFailure(Exception)}
160-
161-
// This calls AsyncTwoPhaseIndexer#finishWithIndexingFailure
162-
// Determine whether the failure is irrecoverable (transform should go into failed state) or not (transform increments
163-
// the indexing failure counter
164-
// and possibly retries)
165-
Throwable irrecoverableException = ExceptionRootCauseFinder.getFirstIrrecoverableExceptionFromBulkResponses(
166-
deduplicatedFailures.values()
167-
);
168-
if (irrecoverableException == null) {
169-
String failureMessage = getBulkIndexDetailedFailureMessage("Significant failures: ", deduplicatedFailures);
170-
logger.debug("[{}] Bulk index experienced [{}] failures. {}", getJobId(), failureCount, failureMessage);
171-
172-
Exception firstException = deduplicatedFailures.values().iterator().next().getFailure().getCause();
173-
nextPhase.onFailure(
174-
new BulkIndexingException(
175-
"Bulk index experienced [{}] failures. {}",
176-
firstException,
177-
false,
178-
failureCount,
179-
failureMessage
180-
)
181-
);
182-
} else {
183-
deduplicatedFailures.remove(irrecoverableException.getClass().getSimpleName());
184-
String failureMessage = getBulkIndexDetailedFailureMessage("Other failures: ", deduplicatedFailures);
185-
irrecoverableException = decorateBulkIndexException(irrecoverableException);
146+
ActionListener.wrap(bulkResponse -> handleBulkResponse(bulkResponse, nextPhase), nextPhase::onFailure)
147+
);
148+
}
186149

187-
logger.debug(
188-
"[{}] Bulk index experienced [{}] failures and at least 1 irrecoverable [{}]. {}",
189-
getJobId(),
190-
failureCount,
191-
ExceptionRootCauseFinder.getDetailedMessage(irrecoverableException),
192-
failureMessage
193-
);
150+
protected void handleBulkResponse(BulkResponse bulkResponse, ActionListener<BulkResponse> nextPhase) {
151+
if (bulkResponse.hasFailures() == false) {
152+
// We don't know the of failures that have occurred (searching, processing, indexing, etc.),
153+
// but if we search, process and bulk index then we have
154+
// successfully processed an entire page of the transform and should reset the counter, even if we are in the middle
155+
// of a checkpoint
156+
context.resetReasonAndFailureCounter();
157+
nextPhase.onResponse(bulkResponse);
158+
return;
159+
}
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+
164+
for (BulkItemResponse item : bulkResponse.getItems()) {
165+
if (item.isFailed()) {
166+
deduplicatedFailures.putIfAbsent(item.getFailure().getCause().getClass().getSimpleName(), item);
167+
failureCount++;
168+
}
169+
}
194170

195-
nextPhase.onFailure(
196-
new BulkIndexingException(
197-
"Bulk index experienced [{}] failures and at least 1 irrecoverable [{}]. {}",
198-
irrecoverableException,
199-
true,
200-
failureCount,
201-
ExceptionRootCauseFinder.getDetailedMessage(irrecoverableException),
202-
failureMessage
203-
)
204-
);
205-
}
206-
} else {
207-
nextPhase.onResponse(bulkResponse);
208-
}
209-
}, nextPhase::onFailure)
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+
Throwable irrecoverableException = ExceptionRootCauseFinder.getFirstIrrecoverableExceptionFromBulkResponses(
178+
deduplicatedFailures.values()
210179
);
180+
if (irrecoverableException == null) {
181+
String failureMessage = getBulkIndexDetailedFailureMessage("Significant failures: ", deduplicatedFailures);
182+
logger.debug("[{}] Bulk index experienced [{}] failures. {}", getJobId(), failureCount, failureMessage);
183+
184+
Exception firstException = deduplicatedFailures.values().iterator().next().getFailure().getCause();
185+
nextPhase.onFailure(
186+
new BulkIndexingException("Bulk index experienced [{}] failures. {}", firstException, false, failureCount, failureMessage)
187+
);
188+
} else {
189+
deduplicatedFailures.remove(irrecoverableException.getClass().getSimpleName());
190+
String failureMessage = getBulkIndexDetailedFailureMessage("Other failures: ", deduplicatedFailures);
191+
irrecoverableException = decorateBulkIndexException(irrecoverableException);
192+
193+
logger.debug(
194+
"[{}] Bulk index experienced [{}] failures and at least 1 irrecoverable [{}]. {}",
195+
getJobId(),
196+
failureCount,
197+
ExceptionRootCauseFinder.getDetailedMessage(irrecoverableException),
198+
failureMessage
199+
);
200+
201+
nextPhase.onFailure(
202+
new BulkIndexingException(
203+
"Bulk index experienced [{}] failures and at least 1 irrecoverable [{}]. {}",
204+
irrecoverableException,
205+
true,
206+
failureCount,
207+
ExceptionRootCauseFinder.getDetailedMessage(irrecoverableException),
208+
failureMessage
209+
)
210+
);
211+
}
211212
}
212213

213214
@Override

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

Lines changed: 1 addition & 1 deletion
Original file line numberDiff line numberDiff line change
@@ -144,7 +144,7 @@ public TransformIndexer(
144144
this.checkpointProvider = ExceptionsHelper.requireNonNull(checkpointProvider, "checkpointProvider");
145145
this.auditor = transformServices.getAuditor();
146146
this.transformConfig = ExceptionsHelper.requireNonNull(transformConfig, "transformConfig");
147-
this.progress = progress != null ? progress : new TransformProgress();
147+
this.progress = transformProgress != null ? transformProgress : new TransformProgress();
148148
this.lastCheckpoint = ExceptionsHelper.requireNonNull(lastCheckpoint, "lastCheckpoint");
149149
this.nextCheckpoint = ExceptionsHelper.requireNonNull(nextCheckpoint, "nextCheckpoint");
150150
this.context = ExceptionsHelper.requireNonNull(context, "context");

x-pack/plugin/transform/src/test/java/org/elasticsearch/xpack/transform/transforms/TransformIndexerFailureHandlingTests.java

Lines changed: 121 additions & 6 deletions
Original file line numberDiff line numberDiff line change
@@ -53,6 +53,7 @@
5353
import org.elasticsearch.xpack.transform.notifications.MockTransformAuditor;
5454
import org.elasticsearch.xpack.transform.notifications.TransformAuditor;
5555
import org.elasticsearch.xpack.transform.persistence.IndexBasedTransformConfigManager;
56+
import org.elasticsearch.xpack.transform.persistence.SeqNoPrimaryTermAndIndex;
5657
import org.junit.After;
5758
import org.junit.Before;
5859

@@ -63,6 +64,7 @@
6364
import java.util.concurrent.CountDownLatch;
6465
import java.util.concurrent.TimeUnit;
6566
import java.util.concurrent.atomic.AtomicBoolean;
67+
import java.util.concurrent.atomic.AtomicInteger;
6668
import java.util.concurrent.atomic.AtomicReference;
6769
import java.util.function.Consumer;
6870
import java.util.function.Function;
@@ -90,7 +92,7 @@ public class TransformIndexerFailureHandlingTests extends ESTestCase {
9092
private Client client;
9193
private ThreadPool threadPool;
9294

93-
class MockedTransformIndexer extends TransformIndexer {
95+
static class MockedTransformIndexer extends ClientTransformIndexer {
9496

9597
private final Function<SearchRequest, SearchResponse> searchFunction;
9698
private final Function<BulkRequest, BulkResponse> bulkFunction;
@@ -126,14 +128,17 @@ class MockedTransformIndexer extends TransformIndexer {
126128
mock(SchedulerEngine.class)
127129
),
128130
checkpointProvider,
129-
transformConfig,
130131
initialState,
131132
initialPosition,
133+
mock(Client.class),
132134
jobStats,
135+
transformConfig,
133136
/* TransformProgress */ null,
134137
TransformCheckpoint.EMPTY,
135138
TransformCheckpoint.EMPTY,
136-
context
139+
new SeqNoPrimaryTermAndIndex(1, 1, "foo"),
140+
context,
141+
false
137142
);
138143
this.searchFunction = searchFunction;
139144
this.bulkFunction = bulkFunction;
@@ -188,7 +193,7 @@ protected void doNextBulk(BulkRequest request, ActionListener<BulkResponse> next
188193

189194
try {
190195
BulkResponse response = bulkFunction.apply(request);
191-
nextPhase.onResponse(response);
196+
super.handleBulkResponse(response, nextPhase);
192197
} catch (Exception e) {
193198
nextPhase.onFailure(e);
194199
}
@@ -253,7 +258,7 @@ void doGetInitialProgress(SearchRequest request, ActionListener<SearchResponse>
253258
}
254259

255260
@Override
256-
void doDeleteByQuery(DeleteByQueryRequest deleteByQueryRequest, ActionListener<BulkByScrollResponse> responseListener) {
261+
protected void doDeleteByQuery(DeleteByQueryRequest deleteByQueryRequest, ActionListener<BulkByScrollResponse> responseListener) {
257262
try {
258263
BulkByScrollResponse response = deleteByQueryFunction.apply(deleteByQueryRequest);
259264
responseListener.onResponse(response);
@@ -263,7 +268,7 @@ void doDeleteByQuery(DeleteByQueryRequest deleteByQueryRequest, ActionListener<B
263268
}
264269

265270
@Override
266-
void refreshDestinationIndex(ActionListener<RefreshResponse> responseListener) {
271+
protected void refreshDestinationIndex(ActionListener<RefreshResponse> responseListener) {
267272
responseListener.onResponse(new RefreshResponse(1, 1, 0, Collections.emptyList()));
268273
}
269274

@@ -705,6 +710,116 @@ public void testRetentionPolicyDeleteByQueryThrowsTemporaryProblem() throws Exce
705710
assertEquals(1, context.getFailureCount());
706711
}
707712

713+
public void testFailureCounterIsResetOnSuccess() throws Exception {
714+
String transformId = randomAlphaOfLength(10);
715+
TransformConfig config = new TransformConfig(
716+
transformId,
717+
randomSourceConfig(),
718+
randomDestConfig(),
719+
null,
720+
null,
721+
null,
722+
randomPivotConfig(),
723+
null,
724+
randomBoolean() ? null : randomAlphaOfLengthBetween(1, 1000),
725+
null,
726+
null,
727+
null,
728+
null
729+
);
730+
731+
final SearchResponse searchResponse = new SearchResponse(
732+
new InternalSearchResponse(
733+
new SearchHits(new SearchHit[] { new SearchHit(1) }, new TotalHits(1L, TotalHits.Relation.EQUAL_TO), 1.0f),
734+
// Simulate completely null aggs
735+
null,
736+
new Suggest(Collections.emptyList()),
737+
new SearchProfileShardResults(Collections.emptyMap()),
738+
false,
739+
false,
740+
1
741+
),
742+
"",
743+
1,
744+
1,
745+
0,
746+
0,
747+
ShardSearchFailure.EMPTY_ARRAY,
748+
SearchResponse.Clusters.EMPTY
749+
);
750+
751+
AtomicReference<IndexerState> state = new AtomicReference<>(IndexerState.STOPPED);
752+
Function<SearchRequest, SearchResponse> searchFunction = new Function<SearchRequest, SearchResponse>() {
753+
final AtomicInteger calls = new AtomicInteger(0);
754+
755+
@Override
756+
public SearchResponse apply(SearchRequest searchRequest) {
757+
int call = calls.getAndIncrement();
758+
if (call == 0) {
759+
throw new SearchPhaseExecutionException(
760+
"query",
761+
"Partial shards failure",
762+
new ShardSearchFailure[] { new ShardSearchFailure(new Exception()) }
763+
);
764+
}
765+
return searchResponse;
766+
}
767+
};
768+
769+
Function<BulkRequest, BulkResponse> bulkFunction = request -> new BulkResponse(new BulkItemResponse[0], 1);
770+
771+
final AtomicBoolean failIndexerCalled = new AtomicBoolean(false);
772+
final AtomicReference<String> failureMessage = new AtomicReference<>();
773+
Consumer<String> failureConsumer = message -> {
774+
failIndexerCalled.compareAndSet(false, true);
775+
failureMessage.compareAndSet(null, message);
776+
};
777+
778+
MockTransformAuditor auditor = MockTransformAuditor.createMockAuditor();
779+
TransformContext.Listener contextListener = mock(TransformContext.Listener.class);
780+
TransformContext context = new TransformContext(TransformTaskState.STARTED, "", 0, contextListener);
781+
782+
MockedTransformIndexer indexer = createMockIndexer(
783+
config,
784+
state,
785+
searchFunction,
786+
bulkFunction,
787+
null,
788+
failureConsumer,
789+
threadPool,
790+
ThreadPool.Names.GENERIC,
791+
auditor,
792+
context
793+
);
794+
795+
final CountDownLatch latch = indexer.newLatch(1);
796+
797+
indexer.start();
798+
assertThat(indexer.getState(), equalTo(IndexerState.STARTED));
799+
assertTrue(indexer.maybeTriggerAsyncJob(System.currentTimeMillis()));
800+
assertThat(indexer.getState(), equalTo(IndexerState.INDEXING));
801+
802+
latch.countDown();
803+
assertBusy(() -> assertThat(indexer.getState(), equalTo(IndexerState.STARTED)), 10, TimeUnit.SECONDS);
804+
assertFalse(failIndexerCalled.get());
805+
assertThat(indexer.getState(), equalTo(IndexerState.STARTED));
806+
assertEquals(1, context.getFailureCount());
807+
808+
final CountDownLatch secondLatch = indexer.newLatch(1);
809+
810+
indexer.start();
811+
assertThat(indexer.getState(), equalTo(IndexerState.STARTED));
812+
assertTrue(indexer.maybeTriggerAsyncJob(System.currentTimeMillis()));
813+
assertThat(indexer.getState(), equalTo(IndexerState.INDEXING));
814+
815+
secondLatch.countDown();
816+
assertBusy(() -> assertThat(indexer.getState(), equalTo(IndexerState.STARTED)), 10, TimeUnit.SECONDS);
817+
assertFalse(failIndexerCalled.get());
818+
assertThat(indexer.getState(), equalTo(IndexerState.STARTED));
819+
auditor.assertAllExpectationsMatched();
820+
assertEquals(0, context.getFailureCount());
821+
}
822+
708823
private MockedTransformIndexer createMockIndexer(
709824
TransformConfig config,
710825
AtomicReference<IndexerState> state,

0 commit comments

Comments
 (0)