Skip to content

Commit d95bca8

Browse files
authored
[ML] Audit job open failures and stop any corresponding datafeed (#80665)
The anomaly detection code contained an assumption dating back to 2016 that if a job failed then its datafeed would notice and stop itself. That works if the job fails on a node after it has successfully started up. But it doesn't work if the job fails during the startup sequence. If the job is being started for the first time then the datafeed won't be running, so there's no problem, but if the job fails when it's being reassigned to a new node then it breaks down, because the datafeed is started by not assigned to any node at that instant. This PR addresses this by making the job force-stop its own datafeed if it fails during its startup sequence and the datafeed is started. Fixes #48934 Additionally, auditing of job failures during the startup sequence is moved so that it happens for all failure scenarios instead of just one. Fixes #80621
1 parent 8b2019a commit d95bca8

File tree

4 files changed

+173
-39
lines changed

4 files changed

+173
-39
lines changed

x-pack/plugin/ml/src/internalClusterTest/java/org/elasticsearch/xpack/ml/integration/MlDistributedFailureIT.java

Lines changed: 102 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -63,6 +63,7 @@
6363
import org.elasticsearch.xpack.core.ml.job.process.autodetect.state.ModelSnapshot;
6464
import org.elasticsearch.xpack.core.ml.notifications.NotificationsIndex;
6565
import org.elasticsearch.xpack.ml.MachineLearning;
66+
import org.elasticsearch.xpack.ml.job.persistence.JobResultsPersister;
6667
import org.elasticsearch.xpack.ml.job.process.autodetect.BlackHoleAutodetectProcess;
6768
import org.elasticsearch.xpack.ml.support.BaseMlIntegTestCase;
6869
import org.junit.After;
@@ -542,7 +543,108 @@ public void testClusterWithTwoMlNodes_RunsDatafeed_GivenOriginalNodeGoesDown() t
542543
assertThat(dataCounts.getProcessedRecordCount(), equalTo(numDocs));
543544
assertThat(dataCounts.getOutOfOrderTimeStampCount(), equalTo(0L));
544545
});
546+
}
547+
548+
public void testClusterWithTwoMlNodes_StopsDatafeed_GivenJobFailsOnReassign() throws Exception {
549+
internalCluster().ensureAtMostNumDataNodes(0);
550+
logger.info("Starting dedicated master node...");
551+
internalCluster().startMasterOnlyNode();
552+
logger.info("Starting ml and data node...");
553+
internalCluster().startNode(onlyRoles(Set.of(DiscoveryNodeRole.DATA_ROLE, DiscoveryNodeRole.ML_ROLE)));
554+
logger.info("Starting another ml and data node...");
555+
internalCluster().startNode(onlyRoles(Set.of(DiscoveryNodeRole.DATA_ROLE, DiscoveryNodeRole.ML_ROLE)));
556+
ensureStableCluster();
557+
558+
// index some datafeed data
559+
client().admin().indices().prepareCreate("data").setMapping("time", "type=date").get();
560+
long numDocs = 80000;
561+
long now = System.currentTimeMillis();
562+
long weekAgo = now - 604800000;
563+
long twoWeeksAgo = weekAgo - 604800000;
564+
indexDocs(logger, "data", numDocs, twoWeeksAgo, weekAgo);
565+
566+
String jobId = "test-node-goes-down-while-running-job";
567+
String datafeedId = jobId + "-datafeed";
568+
569+
Job.Builder job = createScheduledJob(jobId);
570+
PutJobAction.Request putJobRequest = new PutJobAction.Request(job);
571+
client().execute(PutJobAction.INSTANCE, putJobRequest).actionGet();
572+
573+
DatafeedConfig config = createDatafeed(datafeedId, job.getId(), Collections.singletonList("data"), TimeValue.timeValueHours(1));
574+
PutDatafeedAction.Request putDatafeedRequest = new PutDatafeedAction.Request(config);
575+
client().execute(PutDatafeedAction.INSTANCE, putDatafeedRequest).actionGet();
576+
577+
client().execute(OpenJobAction.INSTANCE, new OpenJobAction.Request(job.getId()));
578+
579+
assertBusy(() -> {
580+
GetJobsStatsAction.Response statsResponse = client().execute(
581+
GetJobsStatsAction.INSTANCE,
582+
new GetJobsStatsAction.Request(job.getId())
583+
).actionGet();
584+
assertEquals(JobState.OPENED, statsResponse.getResponse().results().get(0).getState());
585+
}, 30, TimeUnit.SECONDS);
586+
587+
DiscoveryNode nodeRunningJob = client().execute(GetJobsStatsAction.INSTANCE, new GetJobsStatsAction.Request(job.getId()))
588+
.actionGet()
589+
.getResponse()
590+
.results()
591+
.get(0)
592+
.getNode();
593+
594+
setMlIndicesDelayedNodeLeftTimeoutToZero();
595+
596+
StartDatafeedAction.Request startDatafeedRequest = new StartDatafeedAction.Request(config.getId(), 0L);
597+
client().execute(StartDatafeedAction.INSTANCE, startDatafeedRequest).get();
598+
599+
waitForJobToHaveProcessedAtLeast(jobId, 1000);
600+
601+
// The datafeed should be started
602+
assertBusy(() -> {
603+
GetDatafeedsStatsAction.Response statsResponse = client().execute(
604+
GetDatafeedsStatsAction.INSTANCE,
605+
new GetDatafeedsStatsAction.Request(config.getId())
606+
).actionGet();
607+
assertEquals(DatafeedState.STARTED, statsResponse.getResponse().results().get(0).getDatafeedState());
608+
}, 30, TimeUnit.SECONDS);
609+
610+
// Create a problem that will make the job fail when it restarts on a different node
611+
String snapshotId = "123";
612+
ModelSnapshot modelSnapshot = new ModelSnapshot.Builder(jobId).setSnapshotId(snapshotId).setTimestamp(new Date()).build();
613+
JobResultsPersister jobResultsPersister = internalCluster().getInstance(
614+
JobResultsPersister.class,
615+
internalCluster().getMasterName()
616+
);
617+
jobResultsPersister.persistModelSnapshot(modelSnapshot, WriteRequest.RefreshPolicy.IMMEDIATE, () -> true);
618+
UpdateJobAction.Request updateJobRequest = UpdateJobAction.Request.internal(
619+
jobId,
620+
new JobUpdate.Builder(jobId).setModelSnapshotId(snapshotId).build()
621+
);
622+
client().execute(UpdateJobAction.INSTANCE, updateJobRequest).actionGet();
623+
refresh(AnomalyDetectorsIndex.resultsWriteAlias(jobId));
624+
625+
// Make the job move to a different node
626+
internalCluster().stopNode(nodeRunningJob.getName());
627+
628+
// Wait for the job to fail during reassignment
629+
assertBusy(() -> {
630+
GetJobsStatsAction.Response statsResponse = client().execute(
631+
GetJobsStatsAction.INSTANCE,
632+
new GetJobsStatsAction.Request(job.getId())
633+
).actionGet();
634+
assertEquals(JobState.FAILED, statsResponse.getResponse().results().get(0).getState());
635+
}, 30, TimeUnit.SECONDS);
636+
637+
// The datafeed should then be stopped
638+
assertBusy(() -> {
639+
GetDatafeedsStatsAction.Response statsResponse = client().execute(
640+
GetDatafeedsStatsAction.INSTANCE,
641+
new GetDatafeedsStatsAction.Request(config.getId())
642+
).actionGet();
643+
assertEquals(DatafeedState.STOPPED, statsResponse.getResponse().results().get(0).getDatafeedState());
644+
}, 30, TimeUnit.SECONDS);
545645

646+
// Force close the failed job to clean up
647+
client().execute(CloseJobAction.INSTANCE, new CloseJobAction.Request(jobId).setForce(true)).actionGet();
546648
}
547649

548650
private void setupJobWithoutDatafeed(String jobId, ByteSizeValue modelMemoryLimit) throws Exception {

x-pack/plugin/ml/src/main/java/org/elasticsearch/xpack/ml/datafeed/DatafeedNodeSelector.java

Lines changed: 1 addition & 1 deletion
Original file line numberDiff line numberDiff line change
@@ -145,7 +145,7 @@ private AssignmentFailure checkAssignment() {
145145
}
146146

147147
if (jobState.isAnyOf(JobState.OPENING, JobState.OPENED) == false) {
148-
// lets try again later when the job has been opened:
148+
// let's try again later when the job has been opened:
149149
String reason = "cannot start datafeed ["
150150
+ datafeedId
151151
+ "], because the job's ["

x-pack/plugin/ml/src/main/java/org/elasticsearch/xpack/ml/job/process/autodetect/AutodetectProcessManager.java

Lines changed: 7 additions & 15 deletions
Original file line numberDiff line numberDiff line change
@@ -404,13 +404,7 @@ public void writeUpdateProcessMessage(JobTask jobTask, UpdateParams updateParams
404404
// Step 3. Set scheduled events on message and write update process message
405405
ActionListener<QueryPage<ScheduledEvent>> eventsListener = ActionListener.wrap(events -> {
406406
updateProcessMessage.setScheduledEvents(events == null ? null : events.results());
407-
communicator.writeUpdateProcessMessage(updateProcessMessage.build(), (aVoid, e) -> {
408-
if (e == null) {
409-
handler.accept(null);
410-
} else {
411-
handler.accept(e);
412-
}
413-
});
407+
communicator.writeUpdateProcessMessage(updateProcessMessage.build(), (aVoid, e) -> handler.accept(e));
414408
}, handler);
415409

416410
// Step 2. Set the filters on the message and get scheduled events
@@ -545,20 +539,18 @@ public void openJob(
545539

546540
// Start the process
547541
ActionListener<Boolean> stateAliasHandler = ActionListener.wrap(
548-
r -> {
549-
jobManager.getJob(
550-
jobId,
551-
ActionListener.wrap(job -> startProcess(jobTask, job, closeHandler), e -> closeHandler.accept(e, true))
552-
);
553-
},
542+
r -> jobManager.getJob(
543+
jobId,
544+
ActionListener.wrap(job -> startProcess(jobTask, job, closeHandler), e -> closeHandler.accept(e, true))
545+
),
554546
e -> {
555547
if (ExceptionsHelper.unwrapCause(e) instanceof InvalidAliasNameException) {
556548
String msg = "Detected a problem with your setup of machine learning, the state index alias ["
557549
+ AnomalyDetectorsIndex.jobStateIndexWriteAlias()
558550
+ "] exists as index but must be an alias.";
559551
logger.error(new ParameterizedMessage("[{}] {}", jobId, msg), e);
560-
auditor.error(jobId, msg);
561-
setJobState(jobTask, JobState.FAILED, msg, e2 -> closeHandler.accept(e, true));
552+
// The close handler is responsible for auditing this and setting the job state to failed
553+
closeHandler.accept(new IllegalStateException(msg, e), true);
562554
} else {
563555
closeHandler.accept(e, true);
564556
}

x-pack/plugin/ml/src/main/java/org/elasticsearch/xpack/ml/job/task/OpenJobPersistentTasksExecutor.java

Lines changed: 63 additions & 23 deletions
Original file line numberDiff line numberDiff line change
@@ -39,6 +39,7 @@
3939
import org.elasticsearch.xpack.core.ml.action.OpenJobAction;
4040
import org.elasticsearch.xpack.core.ml.action.ResetJobAction;
4141
import org.elasticsearch.xpack.core.ml.action.RevertModelSnapshotAction;
42+
import org.elasticsearch.xpack.core.ml.action.StopDatafeedAction;
4243
import org.elasticsearch.xpack.core.ml.job.config.Blocked;
4344
import org.elasticsearch.xpack.core.ml.job.config.Job;
4445
import org.elasticsearch.xpack.core.ml.job.config.JobState;
@@ -298,8 +299,8 @@ private void runJob(JobTask jobTask, JobState jobState, OpenJobAction.JobParams
298299
return;
299300
}
300301

301-
ActionListener<Boolean> hasRunningDatafeedTaskListener = ActionListener.wrap(hasRunningDatafeed -> {
302-
if (hasRunningDatafeed && isMasterNodeVersionOnOrAfter(MIN_MASTER_NODE_VERSION_FOR_REVERTING_TO_CURRENT_SNAPSHOT)) {
302+
ActionListener<String> getRunningDatafeedListener = ActionListener.wrap(runningDatafeedId -> {
303+
if (runningDatafeedId != null && isMasterNodeVersionOnOrAfter(MIN_MASTER_NODE_VERSION_FOR_REVERTING_TO_CURRENT_SNAPSHOT)) {
303304

304305
// This job has a running datafeed attached to it.
305306
// In order to prevent gaps in the model we revert to the current snapshot deleting intervening results.
@@ -319,45 +320,84 @@ private void runJob(JobTask jobTask, JobState jobState, OpenJobAction.JobParams
319320
}
320321
});
321322

322-
hasRunningDatafeedTask(jobTask.getJobId(), hasRunningDatafeedTaskListener);
323+
getRunningDatafeed(jobTask.getJobId(), getRunningDatafeedListener);
323324
}
324325

325326
private void failTask(JobTask jobTask, String reason) {
327+
String jobId = jobTask.getJobId();
328+
auditor.error(jobId, reason);
326329
JobTaskState failedState = new JobTaskState(JobState.FAILED, jobTask.getAllocationId(), reason);
327-
jobTask.updatePersistentTaskState(
328-
failedState,
329-
ActionListener.wrap(
330-
r -> logger.debug(() -> new ParameterizedMessage("[{}] updated task state to failed", jobTask.getJobId())),
331-
e -> {
332-
logger.error(
333-
new ParameterizedMessage(
334-
"[{}] error while setting task state to failed; marking task as failed",
335-
jobTask.getJobId()
336-
),
337-
e
338-
);
339-
jobTask.markAsFailed(e);
340-
}
341-
)
342-
);
330+
jobTask.updatePersistentTaskState(failedState, ActionListener.wrap(r -> {
331+
logger.debug("[{}] updated task state to failed", jobId);
332+
stopAssociatedDatafeedForFailedJob(jobId);
333+
}, e -> {
334+
logger.error(new ParameterizedMessage("[{}] error while setting task state to failed; marking task as failed", jobId), e);
335+
jobTask.markAsFailed(e);
336+
stopAssociatedDatafeedForFailedJob(jobId);
337+
}));
338+
}
339+
340+
private void stopAssociatedDatafeedForFailedJob(String jobId) {
341+
342+
if (autodetectProcessManager.isNodeDying()) {
343+
// The node shutdown caught us at a bad time, and we cannot stop the datafeed
344+
return;
345+
}
346+
347+
ActionListener<String> getRunningDatafeedListener = ActionListener.wrap(runningDatafeedId -> {
348+
if (runningDatafeedId == null) {
349+
return;
350+
}
351+
StopDatafeedAction.Request request = new StopDatafeedAction.Request(runningDatafeedId);
352+
request.setForce(true);
353+
executeAsyncWithOrigin(
354+
client,
355+
ML_ORIGIN,
356+
StopDatafeedAction.INSTANCE,
357+
request,
358+
ActionListener.wrap(
359+
// StopDatafeedAction will audit the stopping of the datafeed if it succeeds so we don't need to do that here
360+
r -> logger.info("[{}] stopped associated datafeed [{}] after job failure", jobId, runningDatafeedId),
361+
e -> {
362+
if (autodetectProcessManager.isNodeDying() == false) {
363+
logger.error(
364+
new ParameterizedMessage(
365+
"[{}] failed to stop associated datafeed [{}] after job failure",
366+
jobId,
367+
runningDatafeedId
368+
),
369+
e
370+
);
371+
auditor.error(jobId, "failed to stop associated datafeed after job failure");
372+
}
373+
}
374+
)
375+
);
376+
}, e -> {
377+
if (autodetectProcessManager.isNodeDying() == false) {
378+
logger.error(new ParameterizedMessage("[{}] failed to search for associated datafeed", jobId), e);
379+
}
380+
});
381+
382+
getRunningDatafeed(jobId, getRunningDatafeedListener);
343383
}
344384

345385
private boolean isMasterNodeVersionOnOrAfter(Version version) {
346386
return clusterState.nodes().getMasterNode().getVersion().onOrAfter(version);
347387
}
348388

349-
private void hasRunningDatafeedTask(String jobId, ActionListener<Boolean> listener) {
389+
private void getRunningDatafeed(String jobId, ActionListener<String> listener) {
350390
ActionListener<Set<String>> datafeedListener = ActionListener.wrap(datafeeds -> {
351391
assert datafeeds.size() <= 1;
352392
if (datafeeds.isEmpty()) {
353-
listener.onResponse(false);
393+
listener.onResponse(null);
354394
return;
355395
}
356396

357397
String datafeedId = datafeeds.iterator().next();
358398
PersistentTasksCustomMetadata tasks = clusterState.getMetadata().custom(PersistentTasksCustomMetadata.TYPE);
359399
PersistentTasksCustomMetadata.PersistentTask<?> datafeedTask = MlTasks.getDatafeedTask(datafeedId, tasks);
360-
listener.onResponse(datafeedTask != null);
400+
listener.onResponse(datafeedTask != null ? datafeedId : null);
361401
}, listener::onFailure);
362402

363403
datafeedConfigProvider.findDatafeedIdsForJobIds(Collections.singleton(jobId), datafeedListener);
@@ -504,7 +544,7 @@ private void openJob(JobTask jobTask) {
504544
}
505545
} else if (autodetectProcessManager.isNodeDying() == false) {
506546
logger.error(new ParameterizedMessage("[{}] failed to open job", jobTask.getJobId()), e2);
507-
failTask(jobTask, "failed to open job");
547+
failTask(jobTask, "failed to open job: " + e2.getMessage());
508548
}
509549
});
510550
}

0 commit comments

Comments
 (0)