Moved job lifecycle over to persistent tasks

Also replaced the DELETING status from JobState with a boolean flag on Job. The state of a job is now stored inside a persistent task in cluster state. Jobs that aren't running don't have a persistent task, so I moved that notion of being deleted to the job config itself.

Original commit: elastic/x-pack-elasticsearch@b0ed82124d
This commit is contained in:
Martijn van Groningen 2017-02-08 21:31:06 +01:00
parent af2486b834
commit c6763489d5
62 changed files with 1294 additions and 1509 deletions

View File

@ -14,6 +14,7 @@ import org.elasticsearch.cluster.NamedDiff;
import org.elasticsearch.cluster.metadata.IndexNameExpressionResolver;
import org.elasticsearch.cluster.metadata.MetaData;
import org.elasticsearch.cluster.node.DiscoveryNodes;
import org.elasticsearch.cluster.routing.UnassignedInfo;
import org.elasticsearch.cluster.service.ClusterService;
import org.elasticsearch.common.ParseField;
import org.elasticsearch.common.io.stream.NamedWriteableRegistry;
@ -23,6 +24,7 @@ import org.elasticsearch.common.settings.Setting;
import org.elasticsearch.common.settings.Setting.Property;
import org.elasticsearch.common.settings.Settings;
import org.elasticsearch.common.settings.SettingsFilter;
import org.elasticsearch.common.unit.TimeValue;
import org.elasticsearch.common.xcontent.NamedXContentRegistry;
import org.elasticsearch.env.Environment;
import org.elasticsearch.plugins.ActionPlugin;
@ -30,6 +32,7 @@ import org.elasticsearch.plugins.Plugin;
import org.elasticsearch.rest.RestController;
import org.elasticsearch.rest.RestHandler;
import org.elasticsearch.script.ScriptService;
import org.elasticsearch.tasks.Task;
import org.elasticsearch.threadpool.ExecutorBuilder;
import org.elasticsearch.threadpool.FixedExecutorBuilder;
import org.elasticsearch.threadpool.ThreadPool;
@ -50,7 +53,6 @@ import org.elasticsearch.xpack.ml.action.GetJobsAction;
import org.elasticsearch.xpack.ml.action.GetJobsStatsAction;
import org.elasticsearch.xpack.ml.action.GetModelSnapshotsAction;
import org.elasticsearch.xpack.ml.action.GetRecordsAction;
import org.elasticsearch.xpack.ml.action.InternalOpenJobAction;
import org.elasticsearch.xpack.ml.action.MlDeleteByQueryAction;
import org.elasticsearch.xpack.ml.action.OpenJobAction;
import org.elasticsearch.xpack.ml.action.PostDataAction;
@ -60,7 +62,6 @@ import org.elasticsearch.xpack.ml.action.PutJobAction;
import org.elasticsearch.xpack.ml.action.RevertModelSnapshotAction;
import org.elasticsearch.xpack.ml.action.StartDatafeedAction;
import org.elasticsearch.xpack.ml.action.StopDatafeedAction;
import org.elasticsearch.xpack.ml.action.UpdateJobStateAction;
import org.elasticsearch.xpack.ml.action.UpdateJobAction;
import org.elasticsearch.xpack.ml.action.UpdateModelSnapshotAction;
import org.elasticsearch.xpack.ml.action.UpdateProcessAction;
@ -68,6 +69,7 @@ import org.elasticsearch.xpack.ml.action.ValidateDetectorAction;
import org.elasticsearch.xpack.ml.action.ValidateJobConfigAction;
import org.elasticsearch.xpack.ml.datafeed.DatafeedJobRunner;
import org.elasticsearch.xpack.ml.job.JobManager;
import org.elasticsearch.xpack.ml.job.config.JobState;
import org.elasticsearch.xpack.ml.job.metadata.MlInitializationService;
import org.elasticsearch.xpack.ml.job.metadata.MlMetadata;
import org.elasticsearch.xpack.ml.job.persistence.JobDataCountsPersister;
@ -179,11 +181,13 @@ public class MlPlugin extends Plugin implements ActionPlugin {
return Arrays.asList(
new NamedWriteableRegistry.Entry(MetaData.Custom.class, "ml", MlMetadata::new),
new NamedWriteableRegistry.Entry(NamedDiff.class, "ml", MlMetadata.MlMetadataDiff::new),
new NamedWriteableRegistry.Entry(PersistentActionCoordinator.Status.class,
PersistentActionCoordinator.Status.NAME, PersistentActionCoordinator.Status::new),
new NamedWriteableRegistry.Entry(Task.Status.class, PersistentActionCoordinator.Status.NAME,
PersistentActionCoordinator.Status::new),
new NamedWriteableRegistry.Entry(ClusterState.Custom.class, PersistentTasksInProgress.TYPE, PersistentTasksInProgress::new),
new NamedWriteableRegistry.Entry(NamedDiff.class, PersistentTasksInProgress.TYPE, PersistentTasksInProgress::readDiffFrom),
new NamedWriteableRegistry.Entry(PersistentActionRequest.class, StartDatafeedAction.NAME, StartDatafeedAction.Request::new)
new NamedWriteableRegistry.Entry(PersistentActionRequest.class, StartDatafeedAction.NAME, StartDatafeedAction.Request::new),
new NamedWriteableRegistry.Entry(PersistentActionRequest.class, OpenJobAction.NAME, OpenJobAction.Request::new),
new NamedWriteableRegistry.Entry(Task.Status.class, JobState.NAME, JobState::fromStream)
);
}
@ -204,8 +208,15 @@ public class MlPlugin extends Plugin implements ActionPlugin {
if (false == enabled) {
return emptyList();
}
// Whether we are using native process is a good way to detect whether we are in dev / test mode:
TimeValue delayedNodeTimeOutSetting;
if (USE_NATIVE_PROCESS_OPTION.get(settings)) {
delayedNodeTimeOutSetting = UnassignedInfo.INDEX_DELAYED_NODE_LEFT_TIMEOUT_SETTING.get(settings);
} else {
delayedNodeTimeOutSetting = TimeValue.timeValueNanos(0);
}
JobResultsPersister jobResultsPersister = new JobResultsPersister(settings, client);
JobProvider jobProvider = new JobProvider(client, 0);
JobProvider jobProvider = new JobProvider(client, 1, delayedNodeTimeOutSetting);
JobDataCountsPersister jobDataCountsPersister = new JobDataCountsPersister(settings, client);
JobManager jobManager = new JobManager(settings, jobProvider, jobResultsPersister, clusterService);
@ -233,7 +244,7 @@ public class MlPlugin extends Plugin implements ActionPlugin {
jobResultsPersister, jobDataCountsPersister, autodetectProcessFactory, normalizerFactory);
DatafeedJobRunner datafeedJobRunner = new DatafeedJobRunner(threadPool, client, clusterService, jobProvider,
System::currentTimeMillis);
PersistentActionService persistentActionService = new PersistentActionService(Settings.EMPTY, clusterService, client);
PersistentActionService persistentActionService = new PersistentActionService(Settings.EMPTY, threadPool, clusterService, client);
PersistentActionRegistry persistentActionRegistry = new PersistentActionRegistry(Settings.EMPTY);
return Arrays.asList(
@ -301,8 +312,6 @@ public class MlPlugin extends Plugin implements ActionPlugin {
new ActionHandler<>(UpdateJobAction.INSTANCE, UpdateJobAction.TransportAction.class),
new ActionHandler<>(DeleteJobAction.INSTANCE, DeleteJobAction.TransportAction.class),
new ActionHandler<>(OpenJobAction.INSTANCE, OpenJobAction.TransportAction.class),
new ActionHandler<>(InternalOpenJobAction.INSTANCE, InternalOpenJobAction.TransportAction.class),
new ActionHandler<>(UpdateJobStateAction.INSTANCE, UpdateJobStateAction.TransportAction.class),
new ActionHandler<>(GetFiltersAction.INSTANCE, GetFiltersAction.TransportAction.class),
new ActionHandler<>(PutFilterAction.INSTANCE, PutFilterAction.TransportAction.class),
new ActionHandler<>(DeleteFilterAction.INSTANCE, DeleteFilterAction.TransportAction.class),

View File

@ -9,7 +9,6 @@ import org.elasticsearch.ElasticsearchStatusException;
import org.elasticsearch.ResourceNotFoundException;
import org.elasticsearch.action.Action;
import org.elasticsearch.action.ActionListener;
import org.elasticsearch.action.ActionRequest;
import org.elasticsearch.action.ActionRequestBuilder;
import org.elasticsearch.action.ActionRequestValidationException;
import org.elasticsearch.action.ActionResponse;
@ -18,30 +17,45 @@ import org.elasticsearch.action.admin.cluster.node.tasks.cancel.TransportCancelT
import org.elasticsearch.action.admin.cluster.node.tasks.list.ListTasksRequest;
import org.elasticsearch.action.admin.cluster.node.tasks.list.TransportListTasksAction;
import org.elasticsearch.action.support.ActionFilters;
import org.elasticsearch.action.support.HandledTransportAction;
import org.elasticsearch.action.support.master.MasterNodeRequest;
import org.elasticsearch.action.support.master.TransportMasterNodeAction;
import org.elasticsearch.client.ElasticsearchClient;
import org.elasticsearch.cluster.ClusterState;
import org.elasticsearch.cluster.ClusterStateUpdateTask;
import org.elasticsearch.cluster.block.ClusterBlockException;
import org.elasticsearch.cluster.block.ClusterBlockLevel;
import org.elasticsearch.cluster.metadata.IndexNameExpressionResolver;
import org.elasticsearch.cluster.metadata.MetaData;
import org.elasticsearch.cluster.service.ClusterService;
import org.elasticsearch.common.ParseField;
import org.elasticsearch.common.inject.Inject;
import org.elasticsearch.common.io.stream.StreamInput;
import org.elasticsearch.common.io.stream.StreamOutput;
import org.elasticsearch.common.settings.Settings;
import org.elasticsearch.common.unit.TimeValue;
import org.elasticsearch.common.xcontent.ObjectParser;
import org.elasticsearch.common.xcontent.ToXContent;
import org.elasticsearch.common.xcontent.ToXContentObject;
import org.elasticsearch.common.xcontent.XContentBuilder;
import org.elasticsearch.common.xcontent.XContentParser;
import org.elasticsearch.rest.RestStatus;
import org.elasticsearch.tasks.TaskInfo;
import org.elasticsearch.threadpool.ThreadPool;
import org.elasticsearch.transport.TransportService;
import org.elasticsearch.xpack.ml.job.config.Job;
import org.elasticsearch.xpack.ml.job.config.JobState;
import org.elasticsearch.xpack.ml.job.metadata.Allocation;
import org.elasticsearch.xpack.ml.job.metadata.MlMetadata;
import org.elasticsearch.xpack.ml.utils.ExceptionsHelper;
import org.elasticsearch.xpack.ml.utils.JobStateObserver;
import org.elasticsearch.xpack.persistent.PersistentTaskClusterService;
import org.elasticsearch.xpack.persistent.PersistentTasksInProgress;
import org.elasticsearch.xpack.persistent.PersistentTasksInProgress.PersistentTaskInProgress;
import java.io.IOException;
import java.util.Date;
import java.util.HashMap;
import java.util.Map;
import java.util.Objects;
import java.util.function.Predicate;
public class CloseJobAction extends Action<CloseJobAction.Request, CloseJobAction.Response, CloseJobAction.RequestBuilder> {
@ -62,10 +76,27 @@ public class CloseJobAction extends Action<CloseJobAction.Request, CloseJobActio
return new Response();
}
public static class Request extends ActionRequest {
public static class Request extends MasterNodeRequest<Request> implements ToXContent {
public static final ParseField TIMEOUT = new ParseField("timeout");
public static ObjectParser<Request, Void> PARSER = new ObjectParser<>(NAME, Request::new);
static {
PARSER.declareString(Request::setJobId, Job.ID);
PARSER.declareString((request, val) ->
request.setTimeout(TimeValue.parseTimeValue(val, TIMEOUT.getPreferredName())), TIMEOUT);
}
public static Request parseRequest(String jobId, XContentParser parser) {
Request request = PARSER.apply(parser, null);
if (jobId != null) {
request.jobId = jobId;
}
return request;
}
private String jobId;
private TimeValue closeTimeout = TimeValue.timeValueMinutes(20);
private TimeValue timeout = TimeValue.timeValueMinutes(20);
Request() {}
@ -77,12 +108,16 @@ public class CloseJobAction extends Action<CloseJobAction.Request, CloseJobActio
return jobId;
}
public TimeValue getCloseTimeout() {
return closeTimeout;
public void setJobId(String jobId) {
this.jobId = jobId;
}
public void setCloseTimeout(TimeValue closeTimeout) {
this.closeTimeout = closeTimeout;
public TimeValue getTimeout() {
return timeout;
}
public void setTimeout(TimeValue timeout) {
this.timeout = timeout;
}
@Override
@ -94,19 +129,28 @@ public class CloseJobAction extends Action<CloseJobAction.Request, CloseJobActio
public void readFrom(StreamInput in) throws IOException {
super.readFrom(in);
jobId = in.readString();
closeTimeout = new TimeValue(in);
timeout = new TimeValue(in);
}
@Override
public void writeTo(StreamOutput out) throws IOException {
super.writeTo(out);
out.writeString(jobId);
closeTimeout.writeTo(out);
timeout.writeTo(out);
}
@Override
public XContentBuilder toXContent(XContentBuilder builder, Params params) throws IOException {
builder.startObject();
builder.field(Job.ID.getPreferredName(), jobId);
builder.field(TIMEOUT.getPreferredName(), timeout.getStringRep());
builder.endObject();
return builder;
}
@Override
public int hashCode() {
return Objects.hash(jobId, closeTimeout);
return Objects.hash(jobId, timeout);
}
@Override
@ -119,7 +163,7 @@ public class CloseJobAction extends Action<CloseJobAction.Request, CloseJobActio
}
Request other = (Request) obj;
return Objects.equals(jobId, other.jobId) &&
Objects.equals(closeTimeout, other.closeTimeout);
Objects.equals(timeout, other.timeout);
}
}
@ -179,71 +223,135 @@ public class CloseJobAction extends Action<CloseJobAction.Request, CloseJobActio
}
}
public static class TransportAction extends HandledTransportAction<Request, Response> {
public static class TransportAction extends TransportMasterNodeAction<Request, Response> {
private final ClusterService clusterService;
private final JobStateObserver jobStateObserver;
private final TransportListTasksAction listTasksAction;
private final TransportCancelTasksAction cancelTasksAction;
private final PersistentTaskClusterService persistentTaskClusterService;
@Inject
public TransportAction(Settings settings, TransportService transportService, ThreadPool threadPool,
ActionFilters actionFilters, IndexNameExpressionResolver indexNameExpressionResolver,
ClusterService clusterService, TransportCancelTasksAction cancelTasksAction,
TransportListTasksAction listTasksAction) {
super(settings, CloseJobAction.NAME, threadPool, transportService, actionFilters, indexNameExpressionResolver, Request::new);
ClusterService clusterService, TransportListTasksAction listTasksAction,
TransportCancelTasksAction cancelTasksAction, PersistentTaskClusterService persistentTaskClusterService) {
super(settings, CloseJobAction.NAME, transportService, clusterService, threadPool, actionFilters,
indexNameExpressionResolver, Request::new);
this.clusterService = clusterService;
this.jobStateObserver = new JobStateObserver(threadPool, clusterService);
this.cancelTasksAction = cancelTasksAction;
this.listTasksAction = listTasksAction;
this.cancelTasksAction = cancelTasksAction;
this.persistentTaskClusterService = persistentTaskClusterService;
}
@Override
protected void doExecute(Request request, ActionListener<Response> listener) {
MlMetadata mlMetadata = clusterService.state().metaData().custom(MlMetadata.TYPE);
validate(request.jobId, mlMetadata);
protected String executor() {
return ThreadPool.Names.SAME;
}
@Override
protected Response newResponse() {
return new Response();
}
@Override
protected void masterOperation(Request request, ClusterState state, ActionListener<Response> listener) throws Exception {
PersistentTaskInProgress<?> task = validateAndFindTask(request.getJobId(), state);
clusterService.submitStateUpdateTask("closing job [" + request.getJobId() + "]", new ClusterStateUpdateTask() {
@Override
public ClusterState execute(ClusterState currentState) throws Exception {
return moveJobToClosingState(request.getJobId(), currentState);
}
@Override
public void onFailure(String source, Exception e) {
listener.onFailure(e);
}
@Override
public void clusterStateProcessed(String source, ClusterState oldState, ClusterState newState) {
ListTasksRequest listTasksRequest = new ListTasksRequest();
listTasksRequest.setActions(InternalOpenJobAction.NAME);
listTasksRequest.setDetailed(true);
listTasksRequest.setActions(OpenJobAction.NAME + "[c]");
listTasksAction.execute(listTasksRequest, ActionListener.wrap(listTasksResponse -> {
String expectedJobDescription = "job-" + request.jobId;
String expectedDescription = "job-" + request.getJobId();
for (TaskInfo taskInfo : listTasksResponse.getTasks()) {
if (expectedJobDescription.equals(taskInfo.getDescription())) {
if (expectedDescription.equals(taskInfo.getDescription())) {
CancelTasksRequest cancelTasksRequest = new CancelTasksRequest();
cancelTasksRequest.setTaskId(taskInfo.getTaskId());
cancelTasksAction.execute(cancelTasksRequest, ActionListener.wrap(
cancelTasksResponse -> {
jobStateObserver.waitForState(request.jobId, request.closeTimeout, JobState.CLOSED,
e -> {
if (e != null) {
listener.onFailure(e);
} else {
listener.onResponse(new CloseJobAction.Response(true));
}
}
);
},
listener::onFailure)
cancelTasksAction.execute(cancelTasksRequest, ActionListener.wrap(cancelTaskResponse -> {
persistentTaskClusterService.completeOrRestartPersistentTask(task.getId(), null,
ActionListener.wrap(
empty -> listener.onResponse(new CloseJobAction.Response(true)),
listener::onFailure
)
);
}, listener::onFailure));
return;
}
}
listener.onFailure(new ResourceNotFoundException("No job [" + request.jobId + "] running"));
listener.onFailure(new ResourceNotFoundException("task not found for job [" + request.getJobId() + "]"));
}, listener::onFailure));
}
});
}
static void validate(String jobId, MlMetadata mlMetadata) {
Allocation allocation = mlMetadata.getAllocations().get(jobId);
if (allocation == null) {
@Override
protected ClusterBlockException checkBlock(Request request, ClusterState state) {
return state.blocks().globalBlockedException(ClusterBlockLevel.METADATA_WRITE);
}
}
static PersistentTaskInProgress<?> validateAndFindTask(String jobId, ClusterState state) {
MlMetadata mlMetadata = state.metaData().custom(MlMetadata.TYPE);
if (mlMetadata.getJobs().containsKey(jobId) == false) {
throw ExceptionsHelper.missingJobException(jobId);
}
if (allocation.getState() != JobState.OPENED) {
throw new ElasticsearchStatusException("job not opened, expected job state [{}], but got [{}]",
RestStatus.CONFLICT, JobState.OPENED, allocation.getState());
PersistentTasksInProgress tasks = state.custom(PersistentTasksInProgress.TYPE);
if (tasks != null) {
Predicate<PersistentTaskInProgress<?>> p = t -> {
OpenJobAction.Request storedRequest = (OpenJobAction.Request) t.getRequest();
return storedRequest.getJobId().equals(jobId);
};
for (PersistentTaskInProgress<?> task : tasks.findTasks(OpenJobAction.NAME, p)) {
OpenJobAction.Request storedRequest = (OpenJobAction.Request) task.getRequest();
if (storedRequest.getJobId().equals(jobId)) {
JobState jobState = (JobState) task.getStatus();
if (jobState != JobState.OPENED) {
throw new ElasticsearchStatusException("cannot close job, expected job state [{}], but got [{}]",
RestStatus.CONFLICT, JobState.OPENED, jobState);
}
return task;
}
}
}
throw new ElasticsearchStatusException("cannot close job, expected job state [{}], but got [{}]",
RestStatus.CONFLICT, JobState.OPENED, JobState.CLOSED);
}
static ClusterState moveJobToClosingState(String jobId, ClusterState currentState) {
PersistentTaskInProgress<?> task = validateAndFindTask(jobId, currentState);
PersistentTasksInProgress currentTasks = currentState.custom(PersistentTasksInProgress.TYPE);
Map<Long, PersistentTaskInProgress<?>> updatedTasks = new HashMap<>(currentTasks.taskMap());
for (PersistentTaskInProgress<?> taskInProgress : currentTasks.tasks()) {
if (taskInProgress.getId() == task.getId()) {
updatedTasks.put(taskInProgress.getId(), new PersistentTaskInProgress<>(taskInProgress, JobState.CLOSING));
}
}
PersistentTasksInProgress newTasks = new PersistentTasksInProgress(currentTasks.getCurrentId(), updatedTasks);
MlMetadata mlMetadata = currentState.metaData().custom(MlMetadata.TYPE);
Job.Builder jobBuilder = new Job.Builder(mlMetadata.getJobs().get(jobId));
jobBuilder.setFinishedTime(new Date());
MlMetadata.Builder mlMetadataBuilder = new MlMetadata.Builder(mlMetadata);
mlMetadataBuilder.putJob(jobBuilder.build(), true);
ClusterState.Builder builder = ClusterState.builder(currentState);
return builder
.putCustom(PersistentTasksInProgress.TYPE, newTasks)
.metaData(new MetaData.Builder(currentState.metaData())
.putCustom(MlMetadata.TYPE, mlMetadataBuilder.build()))
.build();
}
}

View File

@ -8,6 +8,7 @@ package org.elasticsearch.xpack.ml.action;
import org.elasticsearch.action.Action;
import org.elasticsearch.action.ActionListener;
import org.elasticsearch.action.ActionRequestBuilder;
import org.elasticsearch.action.admin.cluster.node.tasks.list.TransportListTasksAction;
import org.elasticsearch.action.support.ActionFilters;
import org.elasticsearch.action.support.tasks.BaseTasksResponse;
import org.elasticsearch.client.ElasticsearchClient;
@ -27,9 +28,9 @@ import org.elasticsearch.common.xcontent.XContentParser;
import org.elasticsearch.threadpool.ThreadPool;
import org.elasticsearch.transport.TransportService;
import org.elasticsearch.xpack.ml.MlPlugin;
import org.elasticsearch.xpack.ml.job.JobManager;
import org.elasticsearch.xpack.ml.job.config.Job;
import org.elasticsearch.xpack.ml.job.process.autodetect.AutodetectProcessManager;
import org.elasticsearch.xpack.ml.job.JobManager;
import org.elasticsearch.xpack.ml.job.process.autodetect.params.InterimResultsParams;
import org.elasticsearch.xpack.ml.job.process.autodetect.params.TimeRange;
import org.elasticsearch.xpack.ml.utils.ExceptionsHelper;
@ -242,15 +243,15 @@ public class FlushJobAction extends Action<FlushJobAction.Request, FlushJobActio
}
}
public static class TransportAction extends TransportJobTaskAction<InternalOpenJobAction.JobTask, Request, Response> {
public static class TransportAction extends TransportJobTaskAction<OpenJobAction.JobTask, Request, Response> {
@Inject
public TransportAction(Settings settings, TransportService transportService, ThreadPool threadPool, ClusterService clusterService,
ActionFilters actionFilters, IndexNameExpressionResolver indexNameExpressionResolver,
AutodetectProcessManager processManager, JobManager jobManager) {
AutodetectProcessManager processManager, JobManager jobManager, TransportListTasksAction listTasksAction) {
super(settings, FlushJobAction.NAME, threadPool, clusterService, transportService, actionFilters,
indexNameExpressionResolver, FlushJobAction.Request::new, FlushJobAction.Response::new, MlPlugin.THREAD_POOL_NAME,
jobManager, processManager, Request::getJobId);
jobManager, processManager, Request::getJobId, listTasksAction);
}
@Override
@ -261,7 +262,7 @@ public class FlushJobAction extends Action<FlushJobAction.Request, FlushJobActio
}
@Override
protected void taskOperation(Request request, InternalOpenJobAction.JobTask task,
protected void taskOperation(Request request, OpenJobAction.JobTask task,
ActionListener<FlushJobAction.Response> listener) {
jobManager.getJobOrThrowIfUnknown(request.getJobId());

View File

@ -42,6 +42,7 @@ import org.elasticsearch.xpack.ml.job.process.autodetect.AutodetectProcessManage
import org.elasticsearch.xpack.ml.job.process.autodetect.state.DataCounts;
import org.elasticsearch.xpack.ml.job.process.autodetect.state.ModelSizeStats;
import org.elasticsearch.xpack.ml.utils.ExceptionsHelper;
import org.elasticsearch.xpack.persistent.PersistentTasksInProgress;
import java.io.IOException;
import java.util.ArrayList;
@ -97,7 +98,7 @@ public class GetJobsStatsAction extends Action<GetJobsStatsAction.Request, GetJo
@Override
public boolean match(Task task) {
return jobId.equals(Job.ALL) || InternalOpenJobAction.JobTask.match(task, jobId);
return jobId.equals(Job.ALL) || OpenJobAction.JobTask.match(task, jobId);
}
@Override
@ -290,7 +291,7 @@ public class GetJobsStatsAction extends Action<GetJobsStatsAction.Request, GetJo
}
}
public static class TransportAction extends TransportTasksAction<InternalOpenJobAction.JobTask, Request, Response,
public static class TransportAction extends TransportTasksAction<OpenJobAction.JobTask, Request, Response,
QueryPage<Response.JobStats>> {
private final ClusterService clusterService;
@ -342,13 +343,13 @@ public class GetJobsStatsAction extends Action<GetJobsStatsAction.Request, GetJo
}
@Override
protected void taskOperation(Request request, InternalOpenJobAction.JobTask task,
protected void taskOperation(Request request, OpenJobAction.JobTask task,
ActionListener<QueryPage<Response.JobStats>> listener) {
logger.debug("Get stats for job '{}'", request.getJobId());
MlMetadata mlMetadata = clusterService.state().metaData().custom(MlMetadata.TYPE);
PersistentTasksInProgress tasks = clusterService.state().custom(PersistentTasksInProgress.TYPE);
Optional<Tuple<DataCounts, ModelSizeStats>> stats = processManager.getStatistics(request.getJobId());
if (stats.isPresent()) {
JobState jobState = mlMetadata.getAllocations().get(request.jobId).getState();
JobState jobState = MlMetadata.getJobState(request.jobId, tasks);
Response.JobStats jobStats = new Response.JobStats(request.jobId, stats.get().v1(), stats.get().v2(), jobState);
listener.onResponse(new QueryPage<>(Collections.singletonList(jobStats), 1, Job.RESULTS_FIELD));
} else {
@ -365,14 +366,14 @@ public class GetJobsStatsAction extends Action<GetJobsStatsAction.Request, GetJo
return;
}
MlMetadata mlMetadata = clusterService.state().metaData().custom(MlMetadata.TYPE);
AtomicInteger counter = new AtomicInteger(jobIds.size());
AtomicArray<Response.JobStats> jobStats = new AtomicArray<>(jobIds.size());
PersistentTasksInProgress tasks = clusterService.state().custom(PersistentTasksInProgress.TYPE);
for (int i = 0; i < jobIds.size(); i++) {
int slot = i;
String jobId = jobIds.get(i);
gatherDataCountsAndModelSizeStats(jobId, (dataCounts, modelSizeStats) -> {
JobState jobState = mlMetadata.getAllocations().get(jobId).getState();
JobState jobState = MlMetadata.getJobState(request.jobId, tasks);
jobStats.set(slot, new Response.JobStats(jobId, dataCounts, modelSizeStats, jobState));
if (counter.decrementAndGet() == 0) {
List<Response.JobStats> results = response.getResponse().results();

View File

@ -1,135 +0,0 @@
/*
* Copyright Elasticsearch B.V. and/or licensed to Elasticsearch B.V. under one
* or more contributor license agreements. Licensed under the Elastic License;
* you may not use this file except in compliance with the Elastic License.
*/
package org.elasticsearch.xpack.ml.action;
import org.elasticsearch.action.Action;
import org.elasticsearch.action.ActionListener;
import org.elasticsearch.action.ActionRequestBuilder;
import org.elasticsearch.action.ActionResponse;
import org.elasticsearch.action.support.ActionFilters;
import org.elasticsearch.action.support.HandledTransportAction;
import org.elasticsearch.client.ElasticsearchClient;
import org.elasticsearch.cluster.metadata.IndexNameExpressionResolver;
import org.elasticsearch.common.inject.Inject;
import org.elasticsearch.common.settings.Settings;
import org.elasticsearch.tasks.CancellableTask;
import org.elasticsearch.tasks.Task;
import org.elasticsearch.tasks.TaskId;
import org.elasticsearch.threadpool.ThreadPool;
import org.elasticsearch.transport.TransportService;
import org.elasticsearch.xpack.ml.job.config.JobState;
import org.elasticsearch.xpack.ml.job.process.autodetect.AutodetectProcessManager;
public class InternalOpenJobAction extends Action<InternalOpenJobAction.Request, InternalOpenJobAction.Response,
InternalOpenJobAction.RequestBuilder> {
public static final InternalOpenJobAction INSTANCE = new InternalOpenJobAction();
public static final String NAME = "cluster:admin/ml/anomaly_detectors/internal_open";
private InternalOpenJobAction() {
super(NAME);
}
@Override
public RequestBuilder newRequestBuilder(ElasticsearchClient client) {
return new RequestBuilder(client, this);
}
@Override
public Response newResponse() {
return new Response();
}
public static class Request extends OpenJobAction.Request {
public Request(String jobId) {
super(jobId);
}
Request() {
super();
}
@Override
public Task createTask(long id, String type, String action, TaskId parentTaskId) {
return new JobTask(getJobId(), id, type, action, parentTaskId);
}
}
static class RequestBuilder extends ActionRequestBuilder<Request, Response, RequestBuilder> {
RequestBuilder(ElasticsearchClient client, InternalOpenJobAction action) {
super(client, action, new Request());
}
}
public static class Response extends ActionResponse {
Response() {}
}
public static class JobTask extends CancellableTask {
private volatile Runnable cancelHandler;
JobTask(String jobId, long id, String type, String action, TaskId parentTask) {
super(id, type, action, "job-" + jobId, parentTask);
}
@Override
public boolean shouldCancelChildrenOnCancellation() {
return true;
}
@Override
protected void onCancelled() {
cancelHandler.run();
}
static boolean match(Task task, String expectedJobId) {
String expectedDescription = "job-" + expectedJobId;
return task instanceof JobTask && expectedDescription.equals(task.getDescription());
}
}
public static class TransportAction extends HandledTransportAction<Request, Response> {
private final AutodetectProcessManager autodetectProcessManager;
@Inject
public TransportAction(Settings settings, TransportService transportService, ThreadPool threadPool,
ActionFilters actionFilters, IndexNameExpressionResolver indexNameExpressionResolver,
AutodetectProcessManager autodetectProcessManager) {
super(settings, InternalOpenJobAction.NAME, threadPool, transportService, actionFilters, indexNameExpressionResolver,
Request::new);
this.autodetectProcessManager = autodetectProcessManager;
}
@Override
protected void doExecute(Task task, Request request, ActionListener<Response> listener) {
JobTask jobTask = (JobTask) task;
autodetectProcessManager.setJobState(request.getJobId(), JobState.OPENING, aVoid -> {
jobTask.cancelHandler = () -> autodetectProcessManager.closeJob(request.getJobId());
autodetectProcessManager.openJob(request.getJobId(), request.isIgnoreDowntime(), e -> {
if (e == null) {
listener.onResponse(new Response());
} else {
listener.onFailure(e);
}
});
}, listener::onFailure);
}
@Override
protected void doExecute(Request request, ActionListener<Response> listener) {
throw new IllegalStateException("shouldn't get invoked");
}
}
}

View File

@ -5,38 +5,55 @@
*/
package org.elasticsearch.xpack.ml.action;
import org.elasticsearch.ElasticsearchStatusException;
import org.elasticsearch.action.Action;
import org.elasticsearch.action.ActionListener;
import org.elasticsearch.action.ActionRequest;
import org.elasticsearch.action.ActionRequestBuilder;
import org.elasticsearch.action.ActionRequestValidationException;
import org.elasticsearch.action.ActionResponse;
import org.elasticsearch.action.admin.cluster.node.tasks.cancel.CancelTasksRequest;
import org.elasticsearch.action.support.ActionFilters;
import org.elasticsearch.action.support.HandledTransportAction;
import org.elasticsearch.client.ElasticsearchClient;
import org.elasticsearch.cluster.ClusterState;
import org.elasticsearch.cluster.metadata.IndexNameExpressionResolver;
import org.elasticsearch.cluster.node.DiscoveryNodes;
import org.elasticsearch.cluster.service.ClusterService;
import org.elasticsearch.common.ParseField;
import org.elasticsearch.common.Strings;
import org.elasticsearch.common.inject.Inject;
import org.elasticsearch.common.inject.internal.Nullable;
import org.elasticsearch.common.io.stream.StreamInput;
import org.elasticsearch.common.io.stream.StreamOutput;
import org.elasticsearch.common.settings.Settings;
import org.elasticsearch.common.unit.TimeValue;
import org.elasticsearch.common.xcontent.ToXContentObject;
import org.elasticsearch.common.xcontent.ObjectParser;
import org.elasticsearch.common.xcontent.XContentBuilder;
import org.elasticsearch.tasks.LoggingTaskListener;
import org.elasticsearch.common.xcontent.XContentParser;
import org.elasticsearch.rest.RestStatus;
import org.elasticsearch.tasks.Task;
import org.elasticsearch.tasks.TaskId;
import org.elasticsearch.threadpool.ThreadPool;
import org.elasticsearch.transport.TransportResponse;
import org.elasticsearch.transport.TransportService;
import org.elasticsearch.xpack.ml.job.config.Job;
import org.elasticsearch.xpack.ml.job.config.JobState;
import org.elasticsearch.xpack.ml.job.metadata.MlMetadata;
import org.elasticsearch.xpack.ml.job.process.autodetect.AutodetectProcessManager;
import org.elasticsearch.xpack.ml.utils.ExceptionsHelper;
import org.elasticsearch.xpack.ml.utils.JobStateObserver;
import org.elasticsearch.xpack.persistent.PersistentActionRegistry;
import org.elasticsearch.xpack.persistent.PersistentActionRequest;
import org.elasticsearch.xpack.persistent.PersistentActionResponse;
import org.elasticsearch.xpack.persistent.PersistentActionService;
import org.elasticsearch.xpack.persistent.PersistentTask;
import org.elasticsearch.xpack.persistent.PersistentTasksInProgress;
import org.elasticsearch.xpack.persistent.PersistentTasksInProgress.PersistentTaskInProgress;
import org.elasticsearch.xpack.persistent.TransportPersistentAction;
import java.io.IOException;
import java.util.Objects;
import java.util.function.Consumer;
public class OpenJobAction extends Action<OpenJobAction.Request, OpenJobAction.Response, OpenJobAction.RequestBuilder> {
public class OpenJobAction extends Action<OpenJobAction.Request, PersistentActionResponse, OpenJobAction.RequestBuilder> {
public static final OpenJobAction INSTANCE = new OpenJobAction();
public static final String NAME = "cluster:admin/ml/anomaly_detectors/open";
@ -51,22 +68,43 @@ public class OpenJobAction extends Action<OpenJobAction.Request, OpenJobAction.R
}
@Override
public Response newResponse() {
return new Response();
public PersistentActionResponse newResponse() {
return new PersistentActionResponse();
}
public static class Request extends ActionRequest {
public static class Request extends PersistentActionRequest {
public static final ParseField IGNORE_DOWNTIME = new ParseField("ignore_downtime");
public static final ParseField TIMEOUT = new ParseField("timeout");
public static ObjectParser<Request, Void> PARSER = new ObjectParser<>(NAME, Request::new);
static {
PARSER.declareString(Request::setJobId, Job.ID);
PARSER.declareBoolean(Request::setIgnoreDowntime, IGNORE_DOWNTIME);
PARSER.declareString((request, val) ->
request.setTimeout(TimeValue.parseTimeValue(val, TIMEOUT.getPreferredName())), TIMEOUT);
}
public static Request parseRequest(String jobId, XContentParser parser) {
Request request = PARSER.apply(parser, null);
if (jobId != null) {
request.jobId = jobId;
}
return request;
}
private String jobId;
private boolean ignoreDowntime;
private TimeValue openTimeout = TimeValue.timeValueSeconds(20);
private TimeValue timeout = TimeValue.timeValueSeconds(20);
public Request(String jobId) {
this.jobId = ExceptionsHelper.requireNonNull(jobId, Job.ID.getPreferredName());
}
public Request(StreamInput in) throws IOException {
readFrom(in);
}
Request() {}
public String getJobId() {
@ -85,12 +123,17 @@ public class OpenJobAction extends Action<OpenJobAction.Request, OpenJobAction.R
this.ignoreDowntime = ignoreDowntime;
}
public TimeValue getOpenTimeout() {
return openTimeout;
public TimeValue getTimeout() {
return timeout;
}
public void setOpenTimeout(TimeValue openTimeout) {
this.openTimeout = openTimeout;
public void setTimeout(TimeValue timeout) {
this.timeout = timeout;
}
@Override
public Task createTask(long id, String type, String action, TaskId parentTaskId) {
return new JobTask(getJobId(), id, type, action, parentTaskId);
}
@Override
@ -103,7 +146,7 @@ public class OpenJobAction extends Action<OpenJobAction.Request, OpenJobAction.R
super.readFrom(in);
jobId = in.readString();
ignoreDowntime = in.readBoolean();
openTimeout = TimeValue.timeValueMillis(in.readVLong());
timeout = TimeValue.timeValueMillis(in.readVLong());
}
@Override
@ -111,12 +154,27 @@ public class OpenJobAction extends Action<OpenJobAction.Request, OpenJobAction.R
super.writeTo(out);
out.writeString(jobId);
out.writeBoolean(ignoreDowntime);
out.writeVLong(openTimeout.millis());
out.writeVLong(timeout.millis());
}
@Override
public XContentBuilder toXContent(XContentBuilder builder, Params params) throws IOException {
builder.startObject();
builder.field(Job.ID.getPreferredName(), jobId);
builder.field(IGNORE_DOWNTIME.getPreferredName(), ignoreDowntime);
builder.field(TIMEOUT.getPreferredName(), timeout.getStringRep());
builder.endObject();
return builder;
}
@Override
public String getWriteableName() {
return NAME;
}
@Override
public int hashCode() {
return Objects.hash(jobId, ignoreDowntime, openTimeout);
return Objects.hash(jobId, ignoreDowntime, timeout);
}
@Override
@ -130,107 +188,131 @@ public class OpenJobAction extends Action<OpenJobAction.Request, OpenJobAction.R
OpenJobAction.Request other = (OpenJobAction.Request) obj;
return Objects.equals(jobId, other.jobId) &&
Objects.equals(ignoreDowntime, other.ignoreDowntime) &&
Objects.equals(openTimeout, other.openTimeout);
Objects.equals(timeout, other.timeout);
}
@Override
public String toString() {
return Strings.toString(this);
}
}
static class RequestBuilder extends ActionRequestBuilder<Request, Response, RequestBuilder> {
public static class JobTask extends PersistentTask {
private volatile Consumer<String> cancelHandler;
JobTask(String jobId, long id, String type, String action, TaskId parentTask) {
super(id, type, action, "job-" + jobId, parentTask);
}
@Override
protected void onCancelled() {
String reason = CancelTasksRequest.DEFAULT_REASON.equals(getReasonCancelled()) ? null : getReasonCancelled();
cancelHandler.accept(reason);
}
static boolean match(Task task, String expectedJobId) {
String expectedDescription = "job-" + expectedJobId;
return task instanceof JobTask && expectedDescription.equals(task.getDescription());
}
}
static class RequestBuilder extends ActionRequestBuilder<Request, PersistentActionResponse, RequestBuilder> {
RequestBuilder(ElasticsearchClient client, OpenJobAction action) {
super(client, action, new Request());
}
}
public static class Response extends ActionResponse implements ToXContentObject {
private boolean opened;
Response() {}
Response(boolean opened) {
this.opened = opened;
}
public boolean isOpened() {
return opened;
}
@Override
public void readFrom(StreamInput in) throws IOException {
super.readFrom(in);
opened = in.readBoolean();
}
@Override
public void writeTo(StreamOutput out) throws IOException {
super.writeTo(out);
out.writeBoolean(opened);
}
@Override
public XContentBuilder toXContent(XContentBuilder builder, Params params) throws IOException {
builder.startObject();
builder.field("opened", opened);
builder.endObject();
return builder;
}
@Override
public boolean equals(Object o) {
if (this == o) return true;
if (o == null || getClass() != o.getClass()) return false;
Response response = (Response) o;
return opened == response.opened;
}
@Override
public int hashCode() {
return Objects.hash(opened);
}
}
public static class TransportAction extends HandledTransportAction<Request, Response> {
public static class TransportAction extends TransportPersistentAction<Request> {
private final JobStateObserver observer;
private final ClusterService clusterService;
private final InternalOpenJobAction.TransportAction internalOpenJobAction;
private final AutodetectProcessManager autodetectProcessManager;
@Inject
public TransportAction(Settings settings, TransportService transportService, ThreadPool threadPool,
PersistentActionService persistentActionService, PersistentActionRegistry persistentActionRegistry,
ActionFilters actionFilters, IndexNameExpressionResolver indexNameExpressionResolver,
ClusterService clusterService, InternalOpenJobAction.TransportAction internalOpenJobAction) {
super(settings, OpenJobAction.NAME, threadPool, transportService, actionFilters, indexNameExpressionResolver, Request::new);
this.clusterService = clusterService;
ClusterService clusterService, AutodetectProcessManager autodetectProcessManager) {
super(settings, OpenJobAction.NAME, false, threadPool, transportService, persistentActionService,
persistentActionRegistry, actionFilters, indexNameExpressionResolver, Request::new, ThreadPool.Names.MANAGEMENT);
this.autodetectProcessManager = autodetectProcessManager;
this.observer = new JobStateObserver(threadPool, clusterService);
this.internalOpenJobAction = internalOpenJobAction;
}
@Override
protected void doExecute(Request request, ActionListener<Response> listener) {
// This validation happens also in InternalOpenJobAction, the reason we do it here too is that if it fails there
// we are unable to provide the user immediate feedback. We would create the task and the validation would fail
// in the background, whereas now the validation failure is part of the response being returned.
MlMetadata mlMetadata = clusterService.state().metaData().custom(MlMetadata.TYPE);
validate(mlMetadata, request.getJobId());
protected void doExecute(Request request, ActionListener<PersistentActionResponse> listener) {
ActionListener<PersistentActionResponse> finalListener =
ActionListener.wrap(response -> waitForJobStarted(request, response, listener), listener::onFailure);
super.doExecute(request, finalListener);
}
InternalOpenJobAction.Request internalRequest = new InternalOpenJobAction.Request(request.jobId);
internalOpenJobAction.execute(internalRequest, LoggingTaskListener.instance());
observer.waitForState(request.getJobId(), request.openTimeout, JobState.OPENED, e -> {
void waitForJobStarted(Request request, PersistentActionResponse response, ActionListener<PersistentActionResponse> listener) {
observer.waitForState(request.getJobId(), request.timeout, JobState.OPENED, e -> {
if (e != null) {
listener.onFailure(e);
} else {
listener.onResponse(new Response(true));
listener.onResponse(response);
}
});
}
@Override
public void validate(Request request, ClusterState clusterState) {
MlMetadata mlMetadata = clusterState.metaData().custom(MlMetadata.TYPE);
PersistentTasksInProgress tasks = clusterState.custom(PersistentTasksInProgress.TYPE);
OpenJobAction.validate(request.getJobId(), mlMetadata, tasks, clusterState.nodes());
}
@Override
protected void nodeOperation(PersistentTask task, Request request, ActionListener<TransportResponse.Empty> listener) {
autodetectProcessManager.setJobState(task.getPersistentTaskId(), JobState.OPENING, e1 -> {
if (e1 != null) {
listener.onFailure(e1);
return;
}
JobTask jobTask = (JobTask) task;
jobTask.cancelHandler = (reason) -> autodetectProcessManager.closeJob(request.getJobId(), reason);
autodetectProcessManager.openJob(request.getJobId(), task.getPersistentTaskId(), request.isIgnoreDowntime(), e2 -> {
if (e2 == null) {
listener.onResponse(new TransportResponse.Empty());
} else {
listener.onFailure(e2);
}
});
});
}
}
/**
* Fail fast before trying to update the job state on master node if the job doesn't exist or its state
* is not what it should be.
*/
public static void validate(MlMetadata mlMetadata, String jobId) {
MlMetadata.Builder builder = new MlMetadata.Builder(mlMetadata);
builder.updateState(jobId, JobState.OPENING, null);
static void validate(String jobId, MlMetadata mlMetadata, @Nullable PersistentTasksInProgress tasks, DiscoveryNodes nodes) {
Job job = mlMetadata.getJobs().get(jobId);
if (job == null) {
throw ExceptionsHelper.missingJobException(jobId);
}
if (job.isDeleted()) {
throw new ElasticsearchStatusException("Cannot open job [" + jobId + "] because it has been marked as deleted",
RestStatus.CONFLICT);
}
PersistentTaskInProgress<?> task = MlMetadata.getTask(jobId, tasks);
JobState jobState = MlMetadata.getJobState(jobId, tasks);
if (task != null && task.getExecutorNode() != null && jobState == JobState.OPENED) {
if (nodes.nodeExists(task.getExecutorNode()) == false) {
// The state is open and the node were running on no longer exists.
// We can skip the job state check below, because when the node
// disappeared we didn't have time to set the status to failed.
return;
}
}
if (jobState.isAnyOf(JobState.CLOSED, JobState.FAILED) == false) {
throw new ElasticsearchStatusException("[" + jobId + "] expected state [" + JobState.CLOSED
+ "] or [" + JobState.FAILED + "], but got [" + jobState +"]", RestStatus.CONFLICT);
}
}
}

View File

@ -8,6 +8,7 @@ package org.elasticsearch.xpack.ml.action;
import org.elasticsearch.action.Action;
import org.elasticsearch.action.ActionListener;
import org.elasticsearch.action.ActionRequestBuilder;
import org.elasticsearch.action.admin.cluster.node.tasks.list.TransportListTasksAction;
import org.elasticsearch.action.support.ActionFilters;
import org.elasticsearch.action.support.tasks.BaseTasksResponse;
import org.elasticsearch.client.ElasticsearchClient;
@ -26,12 +27,12 @@ import org.elasticsearch.rest.RestStatus;
import org.elasticsearch.threadpool.ThreadPool;
import org.elasticsearch.transport.TransportService;
import org.elasticsearch.xpack.ml.MlPlugin;
import org.elasticsearch.xpack.ml.job.process.autodetect.state.DataCounts;
import org.elasticsearch.xpack.ml.job.JobManager;
import org.elasticsearch.xpack.ml.job.config.DataDescription;
import org.elasticsearch.xpack.ml.job.process.autodetect.AutodetectProcessManager;
import org.elasticsearch.xpack.ml.job.JobManager;
import org.elasticsearch.xpack.ml.job.process.autodetect.params.DataLoadParams;
import org.elasticsearch.xpack.ml.job.process.autodetect.params.TimeRange;
import org.elasticsearch.xpack.ml.job.process.autodetect.state.DataCounts;
import java.io.IOException;
import java.util.Objects;
@ -218,14 +219,14 @@ public class PostDataAction extends Action<PostDataAction.Request, PostDataActio
}
public static class TransportAction extends TransportJobTaskAction<InternalOpenJobAction.JobTask, Request, Response> {
public static class TransportAction extends TransportJobTaskAction<OpenJobAction.JobTask, Request, Response> {
@Inject
public TransportAction(Settings settings, TransportService transportService, ThreadPool threadPool, ClusterService clusterService,
ActionFilters actionFilters, IndexNameExpressionResolver indexNameExpressionResolver,
JobManager jobManager, AutodetectProcessManager processManager) {
JobManager jobManager, AutodetectProcessManager processManager, TransportListTasksAction listTasksAction) {
super(settings, PostDataAction.NAME, threadPool, clusterService, transportService, actionFilters, indexNameExpressionResolver,
Request::new, Response::new, MlPlugin.THREAD_POOL_NAME, jobManager, processManager, Request::getJobId);
Request::new, Response::new, MlPlugin.THREAD_POOL_NAME, jobManager, processManager, Request::getJobId, listTasksAction);
}
@Override
@ -236,17 +237,15 @@ public class PostDataAction extends Action<PostDataAction.Request, PostDataActio
}
@Override
protected void taskOperation(Request request, InternalOpenJobAction.JobTask task, ActionListener<Response> listener) {
protected void taskOperation(Request request, OpenJobAction.JobTask task, ActionListener<Response> listener) {
TimeRange timeRange = TimeRange.builder().startTime(request.getResetStart()).endTime(request.getResetEnd()).build();
DataLoadParams params = new DataLoadParams(timeRange, Optional.ofNullable(request.getDataDescription()));
threadPool.executor(MlPlugin.THREAD_POOL_NAME).execute(() -> {
try {
DataCounts dataCounts = processManager.processData(request.getJobId(), request.content.streamInput(), params);
listener.onResponse(new Response(dataCounts));
} catch (Exception e) {
listener.onFailure(e);
}
});
}
}

View File

@ -40,7 +40,6 @@ import org.elasticsearch.xpack.ml.job.JobManager;
import org.elasticsearch.xpack.ml.job.config.Job;
import org.elasticsearch.xpack.ml.job.config.JobState;
import org.elasticsearch.xpack.ml.job.messages.Messages;
import org.elasticsearch.xpack.ml.job.metadata.Allocation;
import org.elasticsearch.xpack.ml.job.persistence.JobDataCountsPersister;
import org.elasticsearch.xpack.ml.job.persistence.JobDataDeleter;
import org.elasticsearch.xpack.ml.job.persistence.JobProvider;
@ -287,8 +286,8 @@ extends Action<RevertModelSnapshotAction.Request, RevertModelSnapshotAction.Resp
request.getSnapshotId(), request.getJobId(), request.getDeleteInterveningResults());
QueryPage<Job> job = jobManager.getJob(request.getJobId(), clusterService.state());
Allocation allocation = jobManager.getJobAllocation(request.getJobId());
if (job.count() > 0 && allocation.getState().equals(JobState.CLOSED) == false) {
JobState jobState = jobManager.getJobState(request.getJobId());
if (job.count() > 0 && jobState.equals(JobState.CLOSED) == false) {
throw ExceptionsHelper.conflictStatusException(Messages.getMessage(Messages.REST_JOB_NOT_CLOSED_REVERT));
}

View File

@ -35,7 +35,6 @@ import org.elasticsearch.xpack.ml.datafeed.DatafeedJobValidator;
import org.elasticsearch.xpack.ml.datafeed.DatafeedState;
import org.elasticsearch.xpack.ml.job.config.Job;
import org.elasticsearch.xpack.ml.job.config.JobState;
import org.elasticsearch.xpack.ml.job.metadata.Allocation;
import org.elasticsearch.xpack.ml.job.metadata.MlMetadata;
import org.elasticsearch.xpack.ml.utils.ExceptionsHelper;
import org.elasticsearch.xpack.persistent.PersistentActionRegistry;
@ -205,11 +204,6 @@ public class StartDatafeedAction
this.holder = holder;
}
@Override
public boolean shouldCancelChildrenOnCancellation() {
return true;
}
@Override
protected void onCancelled() {
stop();
@ -241,7 +235,8 @@ public class StartDatafeedAction
@Override
public void validate(Request request, ClusterState clusterState) {
MlMetadata mlMetadata = clusterState.metaData().custom(MlMetadata.TYPE);
StartDatafeedAction.validate(request.getDatafeedId(), mlMetadata);
PersistentTasksInProgress tasks = clusterState.custom(PersistentTasksInProgress.TYPE);
StartDatafeedAction.validate(request.getDatafeedId(), mlMetadata, tasks);
PersistentTasksInProgress persistentTasksInProgress = clusterState.custom(PersistentTasksInProgress.TYPE);
if (persistentTasksInProgress == null) {
return;
@ -273,7 +268,7 @@ public class StartDatafeedAction
}
public static void validate(String datafeedId, MlMetadata mlMetadata) {
public static void validate(String datafeedId, MlMetadata mlMetadata, PersistentTasksInProgress tasks) {
DatafeedConfig datafeed = mlMetadata.getDatafeed(datafeedId);
if (datafeed == null) {
throw ExceptionsHelper.missingDatafeedException(datafeedId);
@ -282,10 +277,10 @@ public class StartDatafeedAction
if (job == null) {
throw ExceptionsHelper.missingJobException(datafeed.getJobId());
}
Allocation allocation = mlMetadata.getAllocations().get(datafeed.getJobId());
if (allocation.getState() != JobState.OPENED) {
JobState jobState = MlMetadata.getJobState(datafeed.getJobId(), tasks);
if (jobState != JobState.OPENED) {
throw new ElasticsearchStatusException("cannot start datafeed, expected job state [{}], but got [{}]",
RestStatus.CONFLICT, JobState.OPENED, allocation.getState());
RestStatus.CONFLICT, JobState.OPENED, jobState);
}
DatafeedJobValidator.validate(datafeed, job);
}

View File

@ -146,14 +146,14 @@ public class StopDatafeedAction
MlMetadata mlMetadata = state.metaData().custom(MlMetadata.TYPE);
validate(datafeedId, mlMetadata);
PersistentTasksInProgress tasksInProgress = state.custom(PersistentTasksInProgress.TYPE);
if (tasksInProgress != null) {
for (PersistentTaskInProgress<?> taskInProgress : tasksInProgress.findTasks(StartDatafeedAction.NAME, p -> true)) {
StartDatafeedAction.Request storedRequest = (StartDatafeedAction.Request) taskInProgress.getRequest();
PersistentTasksInProgress tasks = state.custom(PersistentTasksInProgress.TYPE);
if (tasks != null) {
for (PersistentTaskInProgress<?> task : tasks.findTasks(StartDatafeedAction.NAME, p -> true)) {
StartDatafeedAction.Request storedRequest = (StartDatafeedAction.Request) task.getRequest();
if (storedRequest.getDatafeedId().equals(datafeedId)) {
RemovePersistentTaskAction.Request cancelTasksRequest = new RemovePersistentTaskAction.Request();
cancelTasksRequest.setTaskId(taskInProgress.getId());
removePersistentTaskAction.execute(cancelTasksRequest, listener);
RemovePersistentTaskAction.Request removeTaskRequest = new RemovePersistentTaskAction.Request();
removeTaskRequest.setTaskId(task.getId());
removePersistentTaskAction.execute(removeTaskRequest, listener);
return;
}
}
@ -164,6 +164,7 @@ public class StopDatafeedAction
@Override
protected ClusterBlockException checkBlock(Request request, ClusterState state) {
// Remove persistent action actually updates cs, here we just read it.
return state.blocks().globalBlockedException(ClusterBlockLevel.METADATA_READ);
}

View File

@ -7,8 +7,12 @@ package org.elasticsearch.xpack.ml.action;
import org.elasticsearch.ElasticsearchException;
import org.elasticsearch.ElasticsearchStatusException;
import org.elasticsearch.ResourceNotFoundException;
import org.elasticsearch.action.ActionListener;
import org.elasticsearch.action.FailedNodeException;
import org.elasticsearch.action.TaskOperationFailure;
import org.elasticsearch.action.admin.cluster.node.tasks.list.ListTasksRequest;
import org.elasticsearch.action.admin.cluster.node.tasks.list.TransportListTasksAction;
import org.elasticsearch.action.support.ActionFilters;
import org.elasticsearch.action.support.tasks.BaseTasksRequest;
import org.elasticsearch.action.support.tasks.BaseTasksResponse;
@ -21,13 +25,13 @@ import org.elasticsearch.common.io.stream.Writeable;
import org.elasticsearch.common.settings.Settings;
import org.elasticsearch.rest.RestStatus;
import org.elasticsearch.tasks.Task;
import org.elasticsearch.tasks.TaskInfo;
import org.elasticsearch.threadpool.ThreadPool;
import org.elasticsearch.transport.TransportService;
import org.elasticsearch.xpack.ml.job.JobManager;
import org.elasticsearch.xpack.ml.job.config.Job;
import org.elasticsearch.xpack.ml.job.config.JobState;
import org.elasticsearch.xpack.ml.job.process.autodetect.AutodetectProcessManager;
import org.elasticsearch.xpack.ml.job.JobManager;
import org.elasticsearch.xpack.ml.job.metadata.Allocation;
import org.elasticsearch.xpack.ml.utils.ExceptionsHelper;
import java.io.IOException;
@ -46,17 +50,50 @@ public abstract class TransportJobTaskAction<OperationTask extends Task, Request
protected final JobManager jobManager;
protected final AutodetectProcessManager processManager;
private final Function<Request, String> jobIdFromRequest;
private final TransportListTasksAction listTasksAction;
TransportJobTaskAction(Settings settings, String actionName, ThreadPool threadPool, ClusterService clusterService,
TransportService transportService, ActionFilters actionFilters,
IndexNameExpressionResolver indexNameExpressionResolver, Supplier<Request> requestSupplier,
Supplier<Response> responseSupplier, String nodeExecutor, JobManager jobManager,
AutodetectProcessManager processManager, Function<Request, String> jobIdFromRequest) {
AutodetectProcessManager processManager, Function<Request, String> jobIdFromRequest,
TransportListTasksAction listTasksAction) {
super(settings, actionName, threadPool, clusterService, transportService, actionFilters, indexNameExpressionResolver,
requestSupplier, responseSupplier, nodeExecutor);
this.jobManager = jobManager;
this.processManager = processManager;
this.jobIdFromRequest = jobIdFromRequest;
this.listTasksAction = listTasksAction;
}
@Override
protected void doExecute(Task task, Request request, ActionListener<Response> listener) {
// the same validation that exists in AutodetectProcessManager#processData(...) and flush(...) methods
// is required here too because if the job hasn't been opened yet then no task exist for it yet and then
// #taskOperation(...) method will not be invoked, returning an empty result to the client.
// This ensures that we return an understandable error:
String jobId = jobIdFromRequest.apply(request);
jobManager.getJobOrThrowIfUnknown(jobId);
JobState jobState = jobManager.getJobState(jobId);
if (jobState != JobState.OPENED) {
listener.onFailure( new ElasticsearchStatusException("job [" + jobId + "] state is [" + jobState +
"], but must be [" + JobState.OPENED + "] to perform requested action", RestStatus.CONFLICT));
} else {
ListTasksRequest listTasksRequest = new ListTasksRequest();
listTasksRequest.setDetailed(true);
listTasksRequest.setActions(OpenJobAction.NAME + "[c]");
listTasksAction.execute(listTasksRequest, ActionListener.wrap(listTasksResponse -> {
String expectedDescription = "job-" + jobId;
for (TaskInfo taskInfo : listTasksResponse.getTasks()) {
if (expectedDescription.equals(taskInfo.getDescription())) {
request.setTaskId(taskInfo.getTaskId());
super.doExecute(task, request, listener);
return;
}
}
listener.onFailure(new ResourceNotFoundException("task not found for job [" + jobId + "] " + listTasksResponse));
}, listener::onFailure));
}
}
@Override
@ -70,21 +107,9 @@ public abstract class TransportJobTaskAction<OperationTask extends Task, Request
throw new ElasticsearchException(taskOperationFailures.get(0).getCause());
} else if (failedNodeExceptions.isEmpty() == false) {
throw new ElasticsearchException(failedNodeExceptions.get(0).getCause());
} else {
// the same validation that exists in AutodetectProcessManager#processData(...) and flush(...) methods
// is required here too because if the job hasn't been opened yet then no task exist for it yet and then
// #taskOperation(...) method will not be invoked, returning an empty result to the client.
// This ensures that we return an understandable error:
String jobId = jobIdFromRequest.apply(request);
jobManager.getJobOrThrowIfUnknown(jobId);
Allocation allocation = jobManager.getJobAllocation(jobId);
if (allocation.getState() != JobState.OPENED) {
throw new ElasticsearchStatusException("job [" + jobId + "] state is [" + allocation.getState() +
"], but must be [" + JobState.OPENED + "] to perform requested action", RestStatus.CONFLICT);
} else {
throw new IllegalStateException("No errors or response");
}
}
} else {
if (tasks.size() > 1) {
throw new IllegalStateException("Expected one node level response, but got [" + tasks.size() + "]");
@ -127,7 +152,7 @@ public abstract class TransportJobTaskAction<OperationTask extends Task, Request
@Override
public boolean match(Task task) {
return InternalOpenJobAction.JobTask.match(task, jobId);
return OpenJobAction.JobTask.match(task, jobId);
}
}
}

View File

@ -1,194 +0,0 @@
/*
* Copyright Elasticsearch B.V. and/or licensed to Elasticsearch B.V. under one
* or more contributor license agreements. Licensed under the Elastic License;
* you may not use this file except in compliance with the Elastic License.
*/
package org.elasticsearch.xpack.ml.action;
import org.elasticsearch.action.Action;
import org.elasticsearch.action.ActionListener;
import org.elasticsearch.action.ActionRequestValidationException;
import org.elasticsearch.action.support.ActionFilters;
import org.elasticsearch.action.support.master.AcknowledgedRequest;
import org.elasticsearch.action.support.master.AcknowledgedResponse;
import org.elasticsearch.action.support.master.MasterNodeOperationRequestBuilder;
import org.elasticsearch.action.support.master.TransportMasterNodeAction;
import org.elasticsearch.client.ElasticsearchClient;
import org.elasticsearch.cluster.ClusterState;
import org.elasticsearch.cluster.block.ClusterBlockException;
import org.elasticsearch.cluster.block.ClusterBlockLevel;
import org.elasticsearch.cluster.metadata.IndexNameExpressionResolver;
import org.elasticsearch.cluster.service.ClusterService;
import org.elasticsearch.common.inject.Inject;
import org.elasticsearch.common.io.stream.StreamInput;
import org.elasticsearch.common.io.stream.StreamOutput;
import org.elasticsearch.common.settings.Settings;
import org.elasticsearch.threadpool.ThreadPool;
import org.elasticsearch.transport.TransportService;
import org.elasticsearch.xpack.ml.job.JobManager;
import org.elasticsearch.xpack.ml.job.config.Job;
import org.elasticsearch.xpack.ml.job.config.JobState;
import org.elasticsearch.xpack.ml.job.metadata.Allocation;
import org.elasticsearch.xpack.ml.utils.ExceptionsHelper;
import java.io.IOException;
import java.util.Objects;
public class UpdateJobStateAction
extends Action<UpdateJobStateAction.Request, UpdateJobStateAction.Response, UpdateJobStateAction.RequestBuilder> {
public static final UpdateJobStateAction INSTANCE = new UpdateJobStateAction();
public static final String NAME = "cluster:admin/ml/anomaly_detectors/state/update";
private UpdateJobStateAction() {
super(NAME);
}
@Override
public RequestBuilder newRequestBuilder(ElasticsearchClient client) {
return new RequestBuilder(client, this);
}
@Override
public Response newResponse() {
return new Response();
}
public static class Request extends AcknowledgedRequest<Request> {
private String jobId;
private JobState state;
private String reason;
public Request(String jobId, JobState state) {
this.jobId = ExceptionsHelper.requireNonNull(jobId, Job.ID.getPreferredName());
this.state = ExceptionsHelper.requireNonNull(state, Allocation.STATE.getPreferredName());
}
Request() {}
public String getJobId() {
return jobId;
}
public void setJobId(String jobId) {
this.jobId = jobId;
}
public JobState getState() {
return state;
}
public void setState(JobState state) {
this.state = state;
}
public String getReason() {
return reason;
}
public void setReason(String reason) {
this.reason = reason;
}
@Override
public ActionRequestValidationException validate() {
return null;
}
@Override
public void readFrom(StreamInput in) throws IOException {
super.readFrom(in);
jobId = in.readString();
state = JobState.fromStream(in);
reason = in.readOptionalString();
}
@Override
public void writeTo(StreamOutput out) throws IOException {
super.writeTo(out);
out.writeString(jobId);
state.writeTo(out);
out.writeOptionalString(reason);
}
@Override
public int hashCode() {
return Objects.hash(jobId, state);
}
@Override
public boolean equals(Object obj) {
if (this == obj) {
return true;
}
if (obj == null || obj.getClass() != getClass()) {
return false;
}
UpdateJobStateAction.Request other = (UpdateJobStateAction.Request) obj;
return Objects.equals(jobId, other.jobId) && Objects.equals(state, other.state);
}
}
static class RequestBuilder extends MasterNodeOperationRequestBuilder<Request, Response, RequestBuilder> {
RequestBuilder(ElasticsearchClient client, UpdateJobStateAction action) {
super(client, action, new Request());
}
}
public static class Response extends AcknowledgedResponse {
public Response(boolean acknowledged) {
super(acknowledged);
}
private Response() {}
@Override
public void readFrom(StreamInput in) throws IOException {
super.readFrom(in);
readAcknowledged(in);
}
@Override
public void writeTo(StreamOutput out) throws IOException {
super.writeTo(out);
writeAcknowledged(out);
}
}
public static class TransportAction extends TransportMasterNodeAction<Request, Response> {
private final JobManager jobManager;
@Inject
public TransportAction(Settings settings, TransportService transportService, ClusterService clusterService,
ThreadPool threadPool, ActionFilters actionFilters, IndexNameExpressionResolver indexNameExpressionResolver,
JobManager jobManager) {
super(settings, UpdateJobStateAction.NAME, transportService, clusterService, threadPool, actionFilters,
indexNameExpressionResolver, Request::new);
this.jobManager = jobManager;
}
@Override
protected String executor() {
return ThreadPool.Names.SAME;
}
@Override
protected Response newResponse() {
return new Response();
}
@Override
protected void masterOperation(Request request, ClusterState state, ActionListener<Response> listener) throws Exception {
jobManager.setJobState(request, listener);
}
@Override
protected ClusterBlockException checkBlock(Request request, ClusterState state) {
return state.blocks().globalBlockedException(ClusterBlockLevel.METADATA_WRITE);
}
}
}

View File

@ -8,6 +8,7 @@ package org.elasticsearch.xpack.ml.action;
import org.elasticsearch.action.Action;
import org.elasticsearch.action.ActionListener;
import org.elasticsearch.action.ActionRequestBuilder;
import org.elasticsearch.action.admin.cluster.node.tasks.list.TransportListTasksAction;
import org.elasticsearch.action.support.ActionFilters;
import org.elasticsearch.action.support.tasks.BaseTasksResponse;
import org.elasticsearch.client.ElasticsearchClient;
@ -176,14 +177,15 @@ public class UpdateProcessAction extends
}
}
public static class TransportAction extends TransportJobTaskAction<InternalOpenJobAction.JobTask, Request, Response> {
public static class TransportAction extends TransportJobTaskAction<OpenJobAction.JobTask, Request, Response> {
@Inject
public TransportAction(Settings settings, TransportService transportService, ThreadPool threadPool, ClusterService clusterService,
ActionFilters actionFilters, IndexNameExpressionResolver indexNameExpressionResolver,
JobManager jobManager, AutodetectProcessManager processManager) {
JobManager jobManager, AutodetectProcessManager processManager, TransportListTasksAction listTasksAction) {
super(settings, NAME, threadPool, clusterService, transportService, actionFilters, indexNameExpressionResolver,
Request::new, Response::new, MlPlugin.THREAD_POOL_NAME, jobManager, processManager, Request::getJobId);
Request::new, Response::new, MlPlugin.THREAD_POOL_NAME, jobManager, processManager, Request::getJobId,
listTasksAction);
}
@Override
@ -194,7 +196,7 @@ public class UpdateProcessAction extends
}
@Override
protected void taskOperation(Request request, InternalOpenJobAction.JobTask task, ActionListener<Response> listener) {
protected void taskOperation(Request request, OpenJobAction.JobTask task, ActionListener<Response> listener) {
threadPool.executor(MlPlugin.THREAD_POOL_NAME).execute(() -> {
try {
if (request.getModelDebugConfig() != null) {

View File

@ -7,15 +7,18 @@ package org.elasticsearch.xpack.ml.datafeed;
import org.elasticsearch.common.io.stream.StreamInput;
import org.elasticsearch.common.io.stream.StreamOutput;
import org.elasticsearch.common.io.stream.Writeable;
import org.elasticsearch.common.xcontent.XContentBuilder;
import org.elasticsearch.tasks.Task;
import java.io.IOException;
import java.util.Locale;
public enum DatafeedState implements Writeable {
public enum DatafeedState implements Task.Status {
STARTED, STOPPED;
public static final String NAME = "DatafeedState";
public static DatafeedState fromString(String name) {
return valueOf(name.trim().toUpperCase(Locale.ROOT));
}
@ -28,11 +31,22 @@ public enum DatafeedState implements Writeable {
return values()[ordinal];
}
@Override
public String getWriteableName() {
return NAME;
}
@Override
public void writeTo(StreamOutput out) throws IOException {
out.writeVInt(ordinal());
}
@Override
public XContentBuilder toXContent(XContentBuilder builder, Params params) throws IOException {
builder.value(this.toString().toLowerCase(Locale.ROOT));
return builder;
}
@Override
public String toString() {
return name().toLowerCase(Locale.ROOT);

View File

@ -6,11 +6,12 @@
package org.elasticsearch.xpack.ml.job;
import org.elasticsearch.ResourceAlreadyExistsException;
import org.elasticsearch.ResourceNotFoundException;
import org.elasticsearch.action.ActionListener;
import org.elasticsearch.client.Client;
import org.elasticsearch.cluster.AckedClusterStateUpdateTask;
import org.elasticsearch.cluster.ClusterChangedEvent;
import org.elasticsearch.cluster.ClusterState;
import org.elasticsearch.cluster.ClusterStateUpdateTask;
import org.elasticsearch.cluster.ack.AckedRequest;
import org.elasticsearch.cluster.metadata.MetaData;
import org.elasticsearch.cluster.service.ClusterService;
@ -20,14 +21,12 @@ import org.elasticsearch.common.settings.Settings;
import org.elasticsearch.xpack.ml.action.DeleteJobAction;
import org.elasticsearch.xpack.ml.action.PutJobAction;
import org.elasticsearch.xpack.ml.action.RevertModelSnapshotAction;
import org.elasticsearch.xpack.ml.action.UpdateJobStateAction;
import org.elasticsearch.xpack.ml.action.util.QueryPage;
import org.elasticsearch.xpack.ml.job.config.IgnoreDowntime;
import org.elasticsearch.xpack.ml.job.config.Job;
import org.elasticsearch.xpack.ml.job.config.JobState;
import org.elasticsearch.xpack.ml.job.config.JobUpdate;
import org.elasticsearch.xpack.ml.job.messages.Messages;
import org.elasticsearch.xpack.ml.job.metadata.Allocation;
import org.elasticsearch.xpack.ml.job.metadata.MlMetadata;
import org.elasticsearch.xpack.ml.job.persistence.AnomalyDetectorsIndex;
import org.elasticsearch.xpack.ml.job.persistence.JobProvider;
@ -37,6 +36,7 @@ import org.elasticsearch.xpack.ml.job.process.autodetect.state.ModelSnapshot;
import org.elasticsearch.xpack.ml.job.results.AnomalyRecord;
import org.elasticsearch.xpack.ml.notifications.Auditor;
import org.elasticsearch.xpack.ml.utils.ExceptionsHelper;
import org.elasticsearch.xpack.persistent.PersistentTasksInProgress;
import java.util.Collections;
import java.util.List;
@ -134,8 +134,9 @@ public class JobManager extends AbstractComponent {
return getJobOrThrowIfUnknown(clusterService.state(), jobId);
}
public Allocation getJobAllocation(String jobId) {
return getAllocation(clusterService.state(), jobId);
public JobState getJobState(String jobId) {
PersistentTasksInProgress tasks = clusterService.state().custom(PersistentTasksInProgress.TYPE);
return MlMetadata.getJobState(jobId, tasks);
}
/**
@ -260,45 +261,52 @@ public class JobManager extends AbstractComponent {
@Override
public ClusterState execute(ClusterState currentState) throws Exception {
return removeJobFromState(jobId, currentState);
MlMetadata.Builder builder = createMlMetadataBuilder(currentState);
builder.deleteJob(jobId, currentState.custom(PersistentTasksInProgress.TYPE));
return buildNewClusterState(currentState, builder);
}
});
// Step 1. When the job's status updates to DELETING, begin deleting the physical storage
// Step 1. When the job has been marked as deleted then begin deleting the physical storage
// -------
CheckedConsumer<UpdateJobStateAction.Response, Exception> updateHandler = response -> {
CheckedConsumer<Boolean, Exception> updateHandler = response -> {
// Successfully updated the status to DELETING, begin actually deleting
if (response.isAcknowledged()) {
logger.info("Job [" + jobId + "] set to [" + JobState.DELETING + "]");
if (response) {
logger.info("Job [" + jobId + "] is successfully marked as deleted");
} else {
logger.warn("Job [" + jobId + "] change to [" + JobState.DELETING + "] was not acknowledged.");
logger.warn("Job [" + jobId + "] marked as deleted wan't acknowledged");
}
// This task manages the physical deletion of the job (removing the results, then the index)
task.delete(jobId, indexName, client, deleteJobStateHandler::accept, actionListener::onFailure);
};
// Step 0. Kick off the chain of callbacks with the initial UpdateStatus call
// -------
UpdateJobStateAction.Request updateStateListener = new UpdateJobStateAction.Request(jobId, JobState.DELETING);
setJobState(updateStateListener, ActionListener.wrap(updateHandler, actionListener::onFailure));
}
ClusterState removeJobFromState(String jobId, ClusterState currentState) {
MlMetadata.Builder builder = createMlMetadataBuilder(currentState);
builder.deleteJob(jobId);
clusterService.submitStateUpdateTask("mark-job-as-deleted", new ClusterStateUpdateTask() {
@Override
public ClusterState execute(ClusterState currentState) throws Exception {
MlMetadata currentMlMetadata = currentState.metaData().custom(MlMetadata.TYPE);
PersistentTasksInProgress tasks = currentState.custom(PersistentTasksInProgress.TYPE);
MlMetadata.Builder builder = new MlMetadata.Builder(currentMlMetadata);
builder.markJobAsDeleted(jobId, tasks);
return buildNewClusterState(currentState, builder);
}
private Allocation getAllocation(ClusterState state, String jobId) {
MlMetadata mlMetadata = state.metaData().custom(MlMetadata.TYPE);
Allocation allocation = mlMetadata.getAllocations().get(jobId);
if (allocation == null) {
throw new ResourceNotFoundException("No allocation found for job with id [" + jobId + "]");
@Override
public void onFailure(String source, Exception e) {
actionListener.onFailure(e);
}
return allocation;
@Override
public void clusterStatePublished(ClusterChangedEvent clusterChangedEvent) {
try {
updateHandler.accept(true);
} catch (Exception e) {
actionListener.onFailure(e);
}
}
});
}
public Auditor audit(String jobId) {
@ -338,26 +346,6 @@ public class JobManager extends AbstractComponent {
});
}
public void setJobState(UpdateJobStateAction.Request request, ActionListener<UpdateJobStateAction.Response> actionListener) {
clusterService.submitStateUpdateTask("set-job-state-" + request.getState() + "-" + request.getJobId(),
new AckedClusterStateUpdateTask<UpdateJobStateAction.Response>(request, actionListener) {
@Override
public ClusterState execute(ClusterState currentState) throws Exception {
MlMetadata.Builder builder = new MlMetadata.Builder(currentState.metaData().custom(MlMetadata.TYPE));
builder.updateState(request.getJobId(), request.getState(), request.getReason());
return ClusterState.builder(currentState)
.metaData(MetaData.builder(currentState.metaData()).putCustom(MlMetadata.TYPE, builder.build()))
.build();
}
@Override
protected UpdateJobStateAction.Response newResponse(boolean acknowledged) {
return new UpdateJobStateAction.Response(acknowledged);
}
});
}
/**
* Update a persisted model snapshot metadata document to match the
* argument supplied.

View File

@ -61,6 +61,7 @@ public class Job extends AbstractDiffable<Job> implements Writeable, ToXContent
public static final ParseField RESULTS_RETENTION_DAYS = new ParseField("results_retention_days");
public static final ParseField MODEL_SNAPSHOT_ID = new ParseField("model_snapshot_id");
public static final ParseField INDEX_NAME = new ParseField("index_name");
public static final ParseField DELETED = new ParseField("deleted");
// Used for QueryPage
public static final ParseField RESULTS_FIELD = new ParseField("jobs");
@ -113,6 +114,7 @@ public class Job extends AbstractDiffable<Job> implements Writeable, ToXContent
PARSER.declareField(Builder::setCustomSettings, (p, c) -> p.map(), CUSTOM_SETTINGS, ValueType.OBJECT);
PARSER.declareStringOrNull(Builder::setModelSnapshotId, MODEL_SNAPSHOT_ID);
PARSER.declareString(Builder::setIndexName, INDEX_NAME);
PARSER.declareBoolean(Builder::setDeleted, DELETED);
}
private final String jobId;
@ -133,13 +135,13 @@ public class Job extends AbstractDiffable<Job> implements Writeable, ToXContent
private final Map<String, Object> customSettings;
private final String modelSnapshotId;
private final String indexName;
private final boolean deleted;
public Job(String jobId, String description, Date createTime, Date finishedTime, Date lastDataTime,
AnalysisConfig analysisConfig, AnalysisLimits analysisLimits, DataDescription dataDescription,
ModelDebugConfig modelDebugConfig, IgnoreDowntime ignoreDowntime,
Long renormalizationWindowDays, Long backgroundPersistInterval, Long modelSnapshotRetentionDays, Long resultsRetentionDays,
Map<String, Object> customSettings, String modelSnapshotId, String indexName) {
Map<String, Object> customSettings, String modelSnapshotId, String indexName, boolean deleted) {
if (analysisConfig == null) {
throw new IllegalArgumentException(Messages.getMessage(Messages.JOB_CONFIG_MISSING_ANALYSISCONFIG));
}
@ -179,6 +181,7 @@ public class Job extends AbstractDiffable<Job> implements Writeable, ToXContent
this.customSettings = customSettings;
this.modelSnapshotId = modelSnapshotId;
this.indexName = indexName;
this.deleted = deleted;
}
public Job(StreamInput in) throws IOException {
@ -199,6 +202,7 @@ public class Job extends AbstractDiffable<Job> implements Writeable, ToXContent
customSettings = in.readMap();
modelSnapshotId = in.readOptionalString();
indexName = in.readString();
deleted = in.readBoolean();
}
/**
@ -338,6 +342,10 @@ public class Job extends AbstractDiffable<Job> implements Writeable, ToXContent
return modelSnapshotId;
}
public boolean isDeleted() {
return deleted;
}
/**
* Get a list of all input data fields mentioned in the job configuration,
* namely analysis fields and the time field.
@ -395,6 +403,7 @@ public class Job extends AbstractDiffable<Job> implements Writeable, ToXContent
out.writeMap(customSettings);
out.writeOptionalString(modelSnapshotId);
out.writeString(indexName);
out.writeBoolean(deleted);
}
@Override
@ -453,6 +462,9 @@ public class Job extends AbstractDiffable<Job> implements Writeable, ToXContent
builder.field(MODEL_SNAPSHOT_ID.getPreferredName(), modelSnapshotId);
}
builder.field(INDEX_NAME.getPreferredName(), indexName);
if (params.paramAsBoolean("all", false)) {
builder.field(DELETED.getPreferredName(), deleted);
}
return builder;
}
@ -481,7 +493,8 @@ public class Job extends AbstractDiffable<Job> implements Writeable, ToXContent
&& Objects.equals(this.resultsRetentionDays, that.resultsRetentionDays)
&& Objects.equals(this.customSettings, that.customSettings)
&& Objects.equals(this.modelSnapshotId, that.modelSnapshotId)
&& Objects.equals(this.indexName, that.indexName);
&& Objects.equals(this.indexName, that.indexName)
&& Objects.equals(this.deleted, that.deleted);
}
@Override
@ -489,7 +502,7 @@ public class Job extends AbstractDiffable<Job> implements Writeable, ToXContent
return Objects.hash(jobId, description, createTime, finishedTime, lastDataTime, analysisConfig,
analysisLimits, dataDescription, modelDebugConfig, renormalizationWindowDays,
backgroundPersistInterval, modelSnapshotRetentionDays, resultsRetentionDays, ignoreDowntime, customSettings,
modelSnapshotId, indexName);
modelSnapshotId, indexName, deleted);
}
// Class alreadt extends from AbstractDiffable, so copied from ToXContentToBytes#toString()
@ -524,6 +537,7 @@ public class Job extends AbstractDiffable<Job> implements Writeable, ToXContent
private Map<String, Object> customSettings;
private String modelSnapshotId;
private String indexName;
private boolean deleted;
public Builder() {
}
@ -634,6 +648,10 @@ public class Job extends AbstractDiffable<Job> implements Writeable, ToXContent
this.indexName = indexName;
}
public void setDeleted(boolean deleted) {
this.deleted = deleted;
}
public Job build() {
return build(false, null);
}
@ -665,8 +683,7 @@ public class Job extends AbstractDiffable<Job> implements Writeable, ToXContent
id, description, createTime, finishedTime, lastDataTime, analysisConfig, analysisLimits,
dataDescription, modelDebugConfig, ignoreDowntime, renormalizationWindowDays,
backgroundPersistInterval, modelSnapshotRetentionDays, resultsRetentionDays, customSettings, modelSnapshotId,
indexName
);
indexName, deleted);
}
}
}

View File

@ -7,7 +7,8 @@ package org.elasticsearch.xpack.ml.job.config;
import org.elasticsearch.common.io.stream.StreamInput;
import org.elasticsearch.common.io.stream.StreamOutput;
import org.elasticsearch.common.io.stream.Writeable;
import org.elasticsearch.common.xcontent.XContentBuilder;
import org.elasticsearch.tasks.Task;
import java.io.IOException;
import java.util.Arrays;
@ -18,9 +19,11 @@ import java.util.Locale;
* When a job is created it is initialised in to the state closed
* i.e. it is not running.
*/
public enum JobState implements Writeable {
public enum JobState implements Task.Status {
CLOSING, CLOSED, OPENING, OPENED, FAILED, DELETING;
CLOSING, CLOSED, OPENING, OPENED, FAILED;
public static final String NAME = "JobState";
public static JobState fromString(String name) {
return valueOf(name.trim().toUpperCase(Locale.ROOT));
@ -34,11 +37,22 @@ public enum JobState implements Writeable {
return values()[ordinal];
}
@Override
public String getWriteableName() {
return NAME;
}
@Override
public void writeTo(StreamOutput out) throws IOException {
out.writeVInt(ordinal());
}
@Override
public XContentBuilder toXContent(XContentBuilder builder, Params params) throws IOException {
builder.value(this.toString().toLowerCase(Locale.ROOT));
return builder;
}
/**
* @return {@code true} if state matches any of the given {@code candidates}
*/

View File

@ -1,204 +0,0 @@
/*
* Copyright Elasticsearch B.V. and/or licensed to Elasticsearch B.V. under one
* or more contributor license agreements. Licensed under the Elastic License;
* you may not use this file except in compliance with the Elastic License.
*/
package org.elasticsearch.xpack.ml.job.metadata;
import org.elasticsearch.cluster.AbstractDiffable;
import org.elasticsearch.common.ParseField;
import org.elasticsearch.common.Strings;
import org.elasticsearch.common.io.stream.StreamInput;
import org.elasticsearch.common.io.stream.StreamOutput;
import org.elasticsearch.common.xcontent.ObjectParser;
import org.elasticsearch.common.xcontent.ToXContent;
import org.elasticsearch.common.xcontent.XContentBuilder;
import org.elasticsearch.xpack.ml.job.config.Job;
import org.elasticsearch.xpack.ml.job.config.JobState;
import java.io.IOException;
import java.util.Objects;
public class Allocation extends AbstractDiffable<Allocation> implements ToXContent {
private static final ParseField NODE_ID_FIELD = new ParseField("node_id");
private static final ParseField JOB_ID_FIELD = new ParseField("job_id");
private static final ParseField IGNORE_DOWNTIME_FIELD = new ParseField("ignore_downtime");
public static final ParseField STATE = new ParseField("state");
public static final ParseField STATE_REASON = new ParseField("state_reason");
static final ObjectParser<Builder, Void> PARSER = new ObjectParser<>("allocation", Builder::new);
static {
PARSER.declareString(Builder::setNodeId, NODE_ID_FIELD);
PARSER.declareString(Builder::setJobId, JOB_ID_FIELD);
PARSER.declareBoolean(Builder::setIgnoreDowntime, IGNORE_DOWNTIME_FIELD);
PARSER.declareField(Builder::setState, (p, c) -> JobState.fromString(p.text()), STATE, ObjectParser.ValueType.STRING);
PARSER.declareString(Builder::setStateReason, STATE_REASON);
}
private final String nodeId;
private final String jobId;
private final boolean ignoreDowntime;
private final JobState state;
private final String stateReason;
public Allocation(String nodeId, String jobId, boolean ignoreDowntime, JobState state, String stateReason) {
this.nodeId = nodeId;
this.jobId = jobId;
this.ignoreDowntime = ignoreDowntime;
this.state = state;
this.stateReason = stateReason;
}
public Allocation(StreamInput in) throws IOException {
this.nodeId = in.readOptionalString();
this.jobId = in.readString();
this.ignoreDowntime = in.readBoolean();
this.state = JobState.fromStream(in);
this.stateReason = in.readOptionalString();
}
public String getNodeId() {
return nodeId;
}
public String getJobId() {
return jobId;
}
/**
* @return Whether to ignore downtime at startup.
*
* When the job state is set to STARTED, to ignoreDowntime will be set to false.
*/
public boolean isIgnoreDowntime() {
return ignoreDowntime;
}
public JobState getState() {
return state;
}
public String getStateReason() {
return stateReason;
}
@Override
public void writeTo(StreamOutput out) throws IOException {
out.writeOptionalString(nodeId);
out.writeString(jobId);
out.writeBoolean(ignoreDowntime);
state.writeTo(out);
out.writeOptionalString(stateReason);
}
@Override
public XContentBuilder toXContent(XContentBuilder builder, Params params) throws IOException {
builder.startObject();
if (nodeId != null) {
builder.field(NODE_ID_FIELD.getPreferredName(), nodeId);
}
builder.field(JOB_ID_FIELD.getPreferredName(), jobId);
builder.field(IGNORE_DOWNTIME_FIELD.getPreferredName(), ignoreDowntime);
builder.field(STATE.getPreferredName(), state);
if (stateReason != null) {
builder.field(STATE_REASON.getPreferredName(), stateReason);
}
builder.endObject();
return builder;
}
@Override
public boolean equals(Object o) {
if (this == o) return true;
if (o == null || getClass() != o.getClass()) return false;
Allocation that = (Allocation) o;
return Objects.equals(nodeId, that.nodeId) &&
Objects.equals(jobId, that.jobId) &&
Objects.equals(ignoreDowntime, that.ignoreDowntime) &&
Objects.equals(state, that.state) &&
Objects.equals(stateReason, that.stateReason);
}
@Override
public int hashCode() {
return Objects.hash(nodeId, jobId, ignoreDowntime, state, stateReason);
}
// Class already extends from AbstractDiffable, so copied from ToXContentToBytes#toString()
@Override
public final String toString() {
return Strings.toString(this);
}
public static class Builder {
private String nodeId;
private String jobId;
private boolean ignoreDowntime;
private JobState state;
private String stateReason;
public Builder() {
}
public Builder(Job job) {
this.jobId = job.getId();
}
public Builder(Allocation allocation) {
this.nodeId = allocation.nodeId;
this.jobId = allocation.jobId;
this.ignoreDowntime = allocation.ignoreDowntime;
this.state = allocation.state;
this.stateReason = allocation.stateReason;
}
public void setNodeId(String nodeId) {
this.nodeId = nodeId;
}
public void setJobId(String jobId) {
this.jobId = jobId;
}
public void setIgnoreDowntime(boolean ignoreDownTime) {
this.ignoreDowntime = ignoreDownTime;
}
@SuppressWarnings("incomplete-switch")
public void setState(JobState newState) {
if (this.state != null) {
switch (newState) {
case CLOSING:
if (this.state != JobState.OPENED) {
throw new IllegalArgumentException("[" + jobId + "] expected state [" + JobState.OPENED
+ "], but got [" + state +"]");
}
break;
case OPENING:
if (this.state.isAnyOf(JobState.CLOSED, JobState.FAILED) == false) {
throw new IllegalArgumentException("[" + jobId + "] expected state [" + JobState.CLOSED
+ "] or [" + JobState.FAILED + "], but got [" + state +"]");
}
break;
case OPENED:
ignoreDowntime = false;
break;
}
}
this.state = newState;
}
public void setStateReason(String stateReason) {
this.stateReason = stateReason;
}
public Allocation build() {
return new Allocation(nodeId, jobId, ignoreDowntime, state, stateReason);
}
}
}

View File

@ -5,7 +5,6 @@
*/
package org.elasticsearch.xpack.ml.job.metadata;
import org.elasticsearch.ElasticsearchStatusException;
import org.elasticsearch.ResourceAlreadyExistsException;
import org.elasticsearch.ResourceNotFoundException;
import org.elasticsearch.cluster.AbstractDiffable;
@ -22,7 +21,7 @@ import org.elasticsearch.common.io.stream.Writeable;
import org.elasticsearch.common.xcontent.ObjectParser;
import org.elasticsearch.common.xcontent.ToXContent;
import org.elasticsearch.common.xcontent.XContentBuilder;
import org.elasticsearch.rest.RestStatus;
import org.elasticsearch.xpack.ml.action.OpenJobAction;
import org.elasticsearch.xpack.ml.action.StartDatafeedAction;
import org.elasticsearch.xpack.ml.datafeed.DatafeedConfig;
import org.elasticsearch.xpack.ml.datafeed.DatafeedJobValidator;
@ -37,7 +36,6 @@ import org.elasticsearch.xpack.persistent.PersistentTasksInProgress.PersistentTa
import java.io.IOException;
import java.util.Collection;
import java.util.Collections;
import java.util.Date;
import java.util.EnumSet;
import java.util.Map;
import java.util.Objects;
@ -49,30 +47,22 @@ import java.util.function.Predicate;
public class MlMetadata implements MetaData.Custom {
private static final ParseField JOBS_FIELD = new ParseField("jobs");
private static final ParseField ALLOCATIONS_FIELD = new ParseField("allocations");
private static final ParseField DATAFEEDS_FIELD = new ParseField("datafeeds");
public static final String TYPE = "ml";
public static final MlMetadata EMPTY_METADATA = new MlMetadata(Collections.emptySortedMap(),
Collections.emptySortedMap(), Collections.emptySortedMap());
public static final ObjectParser<Builder, Void> ML_METADATA_PARSER = new ObjectParser<>("ml_metadata",
Builder::new);
public static final MlMetadata EMPTY_METADATA = new MlMetadata(Collections.emptySortedMap(), Collections.emptySortedMap());
public static final ObjectParser<Builder, Void> ML_METADATA_PARSER = new ObjectParser<>("ml_metadata", Builder::new);
static {
ML_METADATA_PARSER.declareObjectArray(Builder::putJobs, (p, c) -> Job.PARSER.apply(p, c).build(), JOBS_FIELD);
ML_METADATA_PARSER.declareObjectArray(Builder::putAllocations, Allocation.PARSER, ALLOCATIONS_FIELD);
ML_METADATA_PARSER.declareObjectArray(Builder::putDatafeeds, (p, c) -> DatafeedConfig.PARSER.apply(p, c).build(), DATAFEEDS_FIELD);
}
private final SortedMap<String, Job> jobs;
private final SortedMap<String, Allocation> allocations;
private final SortedMap<String, DatafeedConfig> datafeeds;
private MlMetadata(SortedMap<String, Job> jobs, SortedMap<String, Allocation> allocations,
SortedMap<String, DatafeedConfig> datafeeds) {
private MlMetadata(SortedMap<String, Job> jobs, SortedMap<String, DatafeedConfig> datafeeds) {
this.jobs = Collections.unmodifiableSortedMap(jobs);
this.allocations = Collections.unmodifiableSortedMap(allocations);
this.datafeeds = Collections.unmodifiableSortedMap(datafeeds);
}
@ -80,10 +70,6 @@ public class MlMetadata implements MetaData.Custom {
return jobs;
}
public SortedMap<String, Allocation> getAllocations() {
return allocations;
}
public SortedMap<String, DatafeedConfig> getDatafeeds() {
return datafeeds;
}
@ -117,12 +103,6 @@ public class MlMetadata implements MetaData.Custom {
}
this.jobs = jobs;
size = in.readVInt();
TreeMap<String, Allocation> allocations = new TreeMap<>();
for (int i = 0; i < size; i++) {
allocations.put(in.readString(), new Allocation(in));
}
this.allocations = allocations;
size = in.readVInt();
TreeMap<String, DatafeedConfig> datafeeds = new TreeMap<>();
for (int i = 0; i < size; i++) {
datafeeds.put(in.readString(), new DatafeedConfig(in));
@ -133,7 +113,6 @@ public class MlMetadata implements MetaData.Custom {
@Override
public void writeTo(StreamOutput out) throws IOException {
writeMap(jobs, out);
writeMap(allocations, out);
writeMap(datafeeds, out);
}
@ -148,7 +127,6 @@ public class MlMetadata implements MetaData.Custom {
@Override
public XContentBuilder toXContent(XContentBuilder builder, Params params) throws IOException {
mapValuesToXContent(JOBS_FIELD, jobs, builder, params);
mapValuesToXContent(ALLOCATIONS_FIELD, allocations, builder, params);
mapValuesToXContent(DATAFEEDS_FIELD, datafeeds, builder, params);
return builder;
}
@ -165,20 +143,16 @@ public class MlMetadata implements MetaData.Custom {
public static class MlMetadataDiff implements NamedDiff<MetaData.Custom> {
final Diff<Map<String, Job>> jobs;
final Diff<Map<String, Allocation>> allocations;
final Diff<Map<String, DatafeedConfig>> datafeeds;
MlMetadataDiff(MlMetadata before, MlMetadata after) {
this.jobs = DiffableUtils.diff(before.jobs, after.jobs, DiffableUtils.getStringKeySerializer());
this.allocations = DiffableUtils.diff(before.allocations, after.allocations, DiffableUtils.getStringKeySerializer());
this.datafeeds = DiffableUtils.diff(before.datafeeds, after.datafeeds, DiffableUtils.getStringKeySerializer());
}
public MlMetadataDiff(StreamInput in) throws IOException {
this.jobs = DiffableUtils.readJdkMapDiff(in, DiffableUtils.getStringKeySerializer(), Job::new,
MlMetadataDiff::readJobDiffFrom);
this.allocations = DiffableUtils.readJdkMapDiff(in, DiffableUtils.getStringKeySerializer(), Allocation::new,
MlMetadataDiff::readAllocationDiffFrom);
this.datafeeds = DiffableUtils.readJdkMapDiff(in, DiffableUtils.getStringKeySerializer(), DatafeedConfig::new,
MlMetadataDiff::readSchedulerDiffFrom);
}
@ -186,15 +160,13 @@ public class MlMetadata implements MetaData.Custom {
@Override
public MetaData.Custom apply(MetaData.Custom part) {
TreeMap<String, Job> newJobs = new TreeMap<>(jobs.apply(((MlMetadata) part).jobs));
TreeMap<String, Allocation> newAllocations = new TreeMap<>(allocations.apply(((MlMetadata) part).allocations));
TreeMap<String, DatafeedConfig> newDatafeeds = new TreeMap<>(datafeeds.apply(((MlMetadata) part).datafeeds));
return new MlMetadata(newJobs, newAllocations, newDatafeeds);
return new MlMetadata(newJobs, newDatafeeds);
}
@Override
public void writeTo(StreamOutput out) throws IOException {
jobs.writeTo(out);
allocations.writeTo(out);
datafeeds.writeTo(out);
}
@ -207,10 +179,6 @@ public class MlMetadata implements MetaData.Custom {
return AbstractDiffable.readDiffFrom(Job::new, in);
}
static Diff<Allocation> readAllocationDiffFrom(StreamInput in) throws IOException {
return AbstractDiffable.readDiffFrom(Allocation::new, in);
}
static Diff<DatafeedConfig> readSchedulerDiffFrom(StreamInput in) throws IOException {
return AbstractDiffable.readDiffFrom(DatafeedConfig::new, in);
}
@ -224,7 +192,6 @@ public class MlMetadata implements MetaData.Custom {
return false;
MlMetadata that = (MlMetadata) o;
return Objects.equals(jobs, that.jobs) &&
Objects.equals(allocations, that.allocations) &&
Objects.equals(datafeeds, that.datafeeds);
}
@ -235,24 +202,21 @@ public class MlMetadata implements MetaData.Custom {
@Override
public int hashCode() {
return Objects.hash(jobs, allocations, datafeeds);
return Objects.hash(jobs, datafeeds);
}
public static class Builder {
private TreeMap<String, Job> jobs;
private TreeMap<String, Allocation> allocations;
private TreeMap<String, DatafeedConfig> datafeeds;
public Builder() {
this.jobs = new TreeMap<>();
this.allocations = new TreeMap<>();
this.datafeeds = new TreeMap<>();
}
public Builder(MlMetadata previous) {
jobs = new TreeMap<>(previous.jobs);
allocations = new TreeMap<>(previous.allocations);
datafeeds = new TreeMap<>(previous.datafeeds);
}
@ -261,39 +225,27 @@ public class MlMetadata implements MetaData.Custom {
throw ExceptionsHelper.jobAlreadyExists(job.getId());
}
this.jobs.put(job.getId(), job);
Allocation allocation = allocations.get(job.getId());
if (allocation == null) {
Allocation.Builder builder = new Allocation.Builder(job);
builder.setState(JobState.CLOSED);
allocations.put(job.getId(), builder.build());
}
return this;
}
public Builder deleteJob(String jobId) {
Job job = jobs.remove(jobId);
if (job == null) {
throw new ResourceNotFoundException("job [" + jobId + "] does not exist");
}
public Builder deleteJob(String jobId, PersistentTasksInProgress tasks) {
Optional<DatafeedConfig> datafeed = getDatafeedByJobId(jobId);
if (datafeed.isPresent()) {
throw ExceptionsHelper.conflictStatusException("Cannot delete job [" + jobId + "] while datafeed ["
+ datafeed.get().getId() + "] refers to it");
}
Allocation previousAllocation = this.allocations.remove(jobId);
if (previousAllocation != null) {
if (!previousAllocation.getState().equals(JobState.DELETING)) {
throw ExceptionsHelper.conflictStatusException("Cannot delete job [" + jobId + "] because it is in ["
+ previousAllocation.getState() + "] state. Must be in [" + JobState.DELETING + "] state.");
JobState jobState = MlMetadata.getJobState(jobId, tasks);
if (jobState.isAnyOf(JobState.CLOSED, JobState.FAILED) == false) {
throw ExceptionsHelper.conflictStatusException("Unexpected job state [" + jobState + "], expected [" +
JobState.CLOSED + "]");
}
} else {
throw new ResourceNotFoundException("No Cluster State found for job [" + jobId + "]");
Job job = jobs.remove(jobId);
if (job == null) {
throw new ResourceNotFoundException("job [" + jobId + "] does not exist");
}
if (job.isDeleted() == false) {
throw ExceptionsHelper.conflictStatusException("Cannot delete job [" + jobId + "] because it hasn't marked as deleted");
}
return this;
}
@ -341,15 +293,6 @@ public class MlMetadata implements MetaData.Custom {
return datafeeds.values().stream().filter(s -> s.getJobId().equals(jobId)).findFirst();
}
// only for parsing
private Builder putAllocations(Collection<Allocation.Builder> allocations) {
for (Allocation.Builder allocationBuilder : allocations) {
Allocation allocation = allocationBuilder.build();
this.allocations.put(allocation.getJobId(), allocation);
}
return this;
}
private Builder putJobs(Collection<Job> jobs) {
for (Job job : jobs) {
putJob(job, true);
@ -365,76 +308,55 @@ public class MlMetadata implements MetaData.Custom {
}
public MlMetadata build() {
return new MlMetadata(jobs, allocations, datafeeds);
return new MlMetadata(jobs, datafeeds);
}
public Builder assignToNode(String jobId, String nodeId) {
Allocation allocation = allocations.get(jobId);
if (allocation == null) {
throw new IllegalStateException("[" + jobId + "] no allocation to assign to node [" + nodeId + "]");
}
Allocation.Builder builder = new Allocation.Builder(allocation);
builder.setNodeId(nodeId);
allocations.put(jobId, builder.build());
return this;
}
public Builder updateState(String jobId, JobState jobState, @Nullable String reason) {
if (jobs.containsKey(jobId) == false) {
public void markJobAsDeleted(String jobId, PersistentTasksInProgress tasks) {
Job job = jobs.get(jobId);
if (job == null) {
throw ExceptionsHelper.missingJobException(jobId);
}
Allocation previous = allocations.get(jobId);
if (previous == null) {
throw new IllegalStateException("[" + jobId + "] no allocation exist to update the state to [" + jobState + "]");
if (job.isDeleted()) {
// Job still exists
return;
}
// Cannot update the state to DELETING if there are datafeeds attached
if (jobState.equals(JobState.DELETING)) {
Optional<DatafeedConfig> datafeed = getDatafeedByJobId(jobId);
if (datafeed.isPresent()) {
throw ExceptionsHelper.conflictStatusException("Cannot delete job [" + jobId + "] while datafeed ["
+ datafeed.get().getId() + "] refers to it");
}
JobState jobState = getJobState(jobId, tasks);
if (jobState.isAnyOf(JobState.CLOSED, JobState.FAILED) == false) {
throw ExceptionsHelper.conflictStatusException("Unexpected job state [" + jobState + "], expected [" +
JobState.CLOSED + "]");
}
Job.Builder jobBuilder = new Job.Builder(job);
jobBuilder.setDeleted(true);
putJob(jobBuilder.build(), true);
}
if (previous.getState().equals(JobState.DELETING)) {
// If we're already Deleting there's nothing to do
if (jobState.equals(JobState.DELETING)) {
return this;
}
// Once a job goes into Deleting, it cannot be changed
throw new ElasticsearchStatusException("Cannot change state of job [" + jobId + "] to [" + jobState + "] because " +
"it is currently in [" + JobState.DELETING + "] state.", RestStatus.CONFLICT);
public static PersistentTasksInProgress.PersistentTaskInProgress<?> getTask(String jobId, @Nullable PersistentTasksInProgress tasks) {
if (tasks != null) {
Predicate<PersistentTasksInProgress.PersistentTaskInProgress<?>> p = t -> {
OpenJobAction.Request storedRequest = (OpenJobAction.Request) t.getRequest();
return storedRequest.getJobId().equals(jobId);
};
for (PersistentTasksInProgress.PersistentTaskInProgress<?> task : tasks.findTasks(OpenJobAction.NAME, p)) {
return task;
}
Allocation.Builder builder = new Allocation.Builder(previous);
builder.setState(jobState);
if (reason != null) {
builder.setStateReason(reason);
}
if (previous.getState() != jobState && jobState == JobState.CLOSED) {
Job.Builder job = new Job.Builder(this.jobs.get(jobId));
job.setFinishedTime(new Date());
this.jobs.put(job.getId(), job.build());
}
allocations.put(jobId, builder.build());
return this;
return null;
}
public Builder setIgnoreDowntime(String jobId) {
if (jobs.containsKey(jobId) == false) {
throw ExceptionsHelper.missingJobException(jobId);
}
Allocation allocation = allocations.get(jobId);
if (allocation == null) {
throw new IllegalStateException("[" + jobId + "] no allocation to ignore downtime");
}
Allocation.Builder builder = new Allocation.Builder(allocation);
builder.setIgnoreDowntime(true);
allocations.put(jobId, builder.build());
return this;
public static JobState getJobState(String jobId, @Nullable PersistentTasksInProgress tasks) {
PersistentTasksInProgress.PersistentTaskInProgress<?> task = getTask(jobId, tasks);
if (task != null && task.getStatus() != null) {
return (JobState) task.getStatus();
} else {
// If we haven't opened a job than there will be no persistent task, which is the same as if the job was closed
return JobState.CLOSED;
}
}

View File

@ -23,10 +23,12 @@ import org.elasticsearch.action.search.SearchRequest;
import org.elasticsearch.action.search.SearchResponse;
import org.elasticsearch.client.Client;
import org.elasticsearch.cluster.metadata.IndexMetaData;
import org.elasticsearch.cluster.routing.UnassignedInfo;
import org.elasticsearch.common.Strings;
import org.elasticsearch.common.bytes.BytesReference;
import org.elasticsearch.common.logging.Loggers;
import org.elasticsearch.common.settings.Settings;
import org.elasticsearch.common.unit.TimeValue;
import org.elasticsearch.common.xcontent.NamedXContentRegistry;
import org.elasticsearch.common.xcontent.XContentBuilder;
import org.elasticsearch.common.xcontent.XContentFactory;
@ -116,10 +118,14 @@ public class JobProvider {
private final Client client;
private final int numberOfReplicas;
// Allows us in test mode to disable the delay of shard allocation, so that in tests we don't have to wait for
// for at least a minute for shards to get allocated.
private final TimeValue delayedNodeTimeOutSetting;
public JobProvider(Client client, int numberOfReplicas) {
public JobProvider(Client client, int numberOfReplicas, TimeValue delayedNodeTimeOutSetting) {
this.client = Objects.requireNonNull(client);
this.numberOfReplicas = numberOfReplicas;
this.delayedNodeTimeOutSetting = delayedNodeTimeOutSetting;
}
/**
@ -172,6 +178,7 @@ public class JobProvider {
// least possible burden on Elasticsearch
.put(IndexMetaData.SETTING_NUMBER_OF_SHARDS, 1)
.put(IndexMetaData.SETTING_NUMBER_OF_REPLICAS, numberOfReplicas)
.put(UnassignedInfo.INDEX_DELAYED_NODE_LEFT_TIMEOUT_SETTING.getKey(), delayedNodeTimeOutSetting)
// Sacrifice durability for performance: in the event of power
// failure we can lose the last 5 seconds of changes, but it's
// much faster
@ -200,6 +207,7 @@ public class JobProvider {
// least possible burden on Elasticsearch
.put(IndexMetaData.SETTING_NUMBER_OF_SHARDS, 1)
.put(IndexMetaData.SETTING_NUMBER_OF_REPLICAS, numberOfReplicas)
.put(UnassignedInfo.INDEX_DELAYED_NODE_LEFT_TIMEOUT_SETTING.getKey(), delayedNodeTimeOutSetting)
// Sacrifice durability for performance: in the event of power
// failure we can lose the last 5 seconds of changes, but it's
// much faster
@ -218,6 +226,7 @@ public class JobProvider {
// least possible burden on Elasticsearch
.put(IndexMetaData.SETTING_NUMBER_OF_SHARDS, 1)
.put(IndexMetaData.SETTING_NUMBER_OF_REPLICAS, numberOfReplicas)
.put(UnassignedInfo.INDEX_DELAYED_NODE_LEFT_TIMEOUT_SETTING.getKey(), delayedNodeTimeOutSetting)
// We need to allow fields not mentioned in the mappings to
// pick up default mappings and be used in queries
.put(MapperService.INDEX_MAPPER_DYNAMIC_SETTING.getKey(), true);

View File

@ -7,6 +7,7 @@ package org.elasticsearch.xpack.ml.job.process.autodetect;
import org.apache.logging.log4j.Logger;
import org.apache.logging.log4j.message.ParameterizedMessage;
import org.elasticsearch.ElasticsearchException;
import org.elasticsearch.ElasticsearchStatusException;
import org.elasticsearch.common.CheckedSupplier;
import org.elasticsearch.common.logging.Loggers;
@ -44,6 +45,7 @@ public class AutodetectCommunicator implements Closeable {
private static final Logger LOGGER = Loggers.getLogger(AutodetectCommunicator.class);
private static final Duration FLUSH_PROCESS_CHECK_FREQUENCY = Duration.ofSeconds(1);
private final long taskId;
private final Job job;
private final DataCountsReporter dataCountsReporter;
private final AutodetectProcess autodetectProcess;
@ -52,8 +54,9 @@ public class AutodetectCommunicator implements Closeable {
final AtomicReference<CountDownLatch> inUse = new AtomicReference<>();
public AutodetectCommunicator(Job job, AutodetectProcess process, DataCountsReporter dataCountsReporter,
public AutodetectCommunicator(long taskId, Job job, AutodetectProcess process, DataCountsReporter dataCountsReporter,
AutoDetectResultProcessor autoDetectResultProcessor, Consumer<Exception> handler) {
this.taskId = taskId;
this.job = job;
this.autodetectProcess = process;
this.dataCountsReporter = dataCountsReporter;
@ -86,11 +89,15 @@ public class AutodetectCommunicator implements Closeable {
@Override
public void close() throws IOException {
close(null);
}
public void close(String errorReason) throws IOException {
checkAndRun(() -> Messages.getMessage(Messages.JOB_DATA_CONCURRENT_USE_CLOSE, job.getId()), () -> {
dataCountsReporter.close();
autodetectProcess.close();
autoDetectResultProcessor.awaitCompletion();
handler.accept(null);
handler.accept(errorReason != null ? new ElasticsearchException(errorReason) : null);
return null;
}, true);
}
@ -162,6 +169,10 @@ public class AutodetectCommunicator implements Closeable {
return dataCountsReporter.runningTotalStats();
}
public long getTaskId() {
return taskId;
}
private <T> T checkAndRun(Supplier<String> errorMessage, CheckedSupplier<T, IOException> callback, boolean wait) throws IOException {
CountDownLatch latch = new CountDownLatch(1);
if (inUse.compareAndSet(null, latch)) {

View File

@ -9,6 +9,7 @@ import org.apache.lucene.util.IOUtils;
import org.elasticsearch.ElasticsearchStatusException;
import org.elasticsearch.action.ActionListener;
import org.elasticsearch.client.Client;
import org.elasticsearch.common.CheckedConsumer;
import org.elasticsearch.common.collect.Tuple;
import org.elasticsearch.common.component.AbstractComponent;
import org.elasticsearch.common.settings.Setting;
@ -17,14 +18,12 @@ import org.elasticsearch.common.util.concurrent.EsRejectedExecutionException;
import org.elasticsearch.rest.RestStatus;
import org.elasticsearch.threadpool.ThreadPool;
import org.elasticsearch.xpack.ml.MlPlugin;
import org.elasticsearch.xpack.ml.action.UpdateJobStateAction;
import org.elasticsearch.xpack.ml.job.JobManager;
import org.elasticsearch.xpack.ml.job.config.DetectionRule;
import org.elasticsearch.xpack.ml.job.config.Job;
import org.elasticsearch.xpack.ml.job.config.JobState;
import org.elasticsearch.xpack.ml.job.config.MlFilter;
import org.elasticsearch.xpack.ml.job.config.ModelDebugConfig;
import org.elasticsearch.xpack.ml.job.metadata.Allocation;
import org.elasticsearch.xpack.ml.job.persistence.JobDataCountsPersister;
import org.elasticsearch.xpack.ml.job.persistence.JobProvider;
import org.elasticsearch.xpack.ml.job.persistence.JobRenormalizedResultsPersister;
@ -42,6 +41,7 @@ import org.elasticsearch.xpack.ml.job.process.normalizer.Renormalizer;
import org.elasticsearch.xpack.ml.job.process.normalizer.ScoresUpdater;
import org.elasticsearch.xpack.ml.job.process.normalizer.ShortCircuitingRenormalizer;
import org.elasticsearch.xpack.ml.utils.ExceptionsHelper;
import org.elasticsearch.xpack.persistent.UpdatePersistentTaskStatusAction;
import java.io.IOException;
import java.io.InputStream;
@ -116,15 +116,15 @@ public class AutodetectProcessManager extends AbstractComponent {
* @return Count of records, fields, bytes, etc written
*/
public DataCounts processData(String jobId, InputStream input, DataLoadParams params) {
Allocation allocation = jobManager.getJobAllocation(jobId);
if (allocation.getState() != JobState.OPENED) {
throw new IllegalArgumentException("job [" + jobId + "] state is [" + allocation.getState() + "], but must be ["
JobState jobState = jobManager.getJobState(jobId);
if (jobState != JobState.OPENED) {
throw new IllegalArgumentException("job [" + jobId + "] state is [" + jobState + "], but must be ["
+ JobState.OPENED + "] for processing data");
}
AutodetectCommunicator communicator = autoDetectCommunicatorByJob.get(jobId);
if (communicator == null) {
throw new IllegalStateException("job [" + jobId + "] with state [" + allocation.getState() + "] hasn't been started");
throw new IllegalStateException("job [" + jobId + "] with state [" + jobState + "] hasn't been started");
}
try {
return communicator.writeToJob(input, params);
@ -188,21 +188,22 @@ public class AutodetectProcessManager extends AbstractComponent {
// TODO check for errors from autodetects
}
public void openJob(String jobId, boolean ignoreDowntime, Consumer<Exception> handler) {
public void openJob(String jobId, long taskId, boolean ignoreDowntime, Consumer<Exception> handler) {
gatherRequiredInformation(jobId, (dataCounts, modelSnapshot, quantiles, filters) -> {
autoDetectCommunicatorByJob.computeIfAbsent(jobId, id -> {
AutodetectCommunicator communicator =
create(id, dataCounts, modelSnapshot, quantiles, filters, ignoreDowntime, handler);
try {
AutodetectCommunicator communicator = autoDetectCommunicatorByJob.computeIfAbsent(jobId, id ->
create(id, taskId, dataCounts, modelSnapshot, quantiles, filters, ignoreDowntime, handler));
communicator.writeJobInputHeader();
} catch (IOException ioe) {
setJobState(taskId, jobId, JobState.OPENED);
} catch (Exception e1) {
if (e1 instanceof ElasticsearchStatusException) {
logger.info(e1.getMessage());
} else {
String msg = String.format(Locale.ROOT, "[%s] exception while opening job", jobId);
logger.error(msg);
throw ExceptionsHelper.serverError(msg, ioe);
logger.error(msg, e1);
}
setJobState(taskId, JobState.FAILED, e2 -> handler.accept(e1));
}
setJobState(jobId, JobState.OPENED);
return communicator;
});
}, handler);
}
@ -228,11 +229,11 @@ public class AutodetectProcessManager extends AbstractComponent {
}
AutodetectCommunicator create(String jobId, DataCounts dataCounts, ModelSnapshot modelSnapshot, Quantiles quantiles,
AutodetectCommunicator create(String jobId, long taskId, DataCounts dataCounts, ModelSnapshot modelSnapshot, Quantiles quantiles,
Set<MlFilter> filters, boolean ignoreDowntime, Consumer<Exception> handler) {
if (autoDetectCommunicatorByJob.size() == maxAllowedRunningJobs) {
throw new ElasticsearchStatusException("max running job capacity [" + maxAllowedRunningJobs + "] reached",
RestStatus.CONFLICT);
RestStatus.TOO_MANY_REQUESTS);
}
Job job = jobManager.getJobOrThrowIfUnknown(jobId);
@ -261,26 +262,26 @@ public class AutodetectProcessManager extends AbstractComponent {
}
throw e;
}
return new AutodetectCommunicator(job, process, dataCountsReporter, processor, handler);
return new AutodetectCommunicator(taskId, job, process, dataCountsReporter, processor, handler);
}
}
/**
* Stop the running job and mark it as finished.<br>
* @param jobId The job to stop
*
* @param errorReason If caused by failure, the reason for closing the job
*/
public void closeJob(String jobId) {
logger.debug("Closing job {}", jobId);
public void closeJob(String jobId, String errorReason) {
logger.debug("Attempting to close job [{}], because [{}]", jobId, errorReason);
AutodetectCommunicator communicator = autoDetectCommunicatorByJob.remove(jobId);
if (communicator == null) {
logger.debug("Cannot close: no active autodetect process for job {}", jobId);
return;
}
logger.info("Closing job [{}], because [{}]", jobId, errorReason);
try {
communicator.close();
setJobState(jobId, JobState.CLOSED);
communicator.close(errorReason);
} catch (Exception e) {
logger.warn("Exception closing stopped process input stream", e);
throw ExceptionsHelper.serverError("Exception closing stopped process input stream", e);
@ -303,11 +304,11 @@ public class AutodetectProcessManager extends AbstractComponent {
return Duration.between(communicator.getProcessStartTime(), ZonedDateTime.now());
}
private void setJobState(String jobId, JobState state) {
UpdateJobStateAction.Request request = new UpdateJobStateAction.Request(jobId, state);
client.execute(UpdateJobStateAction.INSTANCE, request, new ActionListener<UpdateJobStateAction.Response>() {
private void setJobState(long taskId, String jobId, JobState state) {
UpdatePersistentTaskStatusAction.Request request = new UpdatePersistentTaskStatusAction.Request(taskId, state);
client.execute(UpdatePersistentTaskStatusAction.INSTANCE, request, new ActionListener<UpdatePersistentTaskStatusAction.Response>() {
@Override
public void onResponse(UpdateJobStateAction.Response response) {
public void onResponse(UpdatePersistentTaskStatusAction.Response response) {
if (response.isAcknowledged()) {
logger.info("Successfully set job state to [{}] for job [{}]", state, jobId);
} else {
@ -322,9 +323,16 @@ public class AutodetectProcessManager extends AbstractComponent {
});
}
public void setJobState(String jobId, JobState state, Consumer<Void> handler, Consumer<Exception> errorHandler) {
UpdateJobStateAction.Request request = new UpdateJobStateAction.Request(jobId, state);
client.execute(UpdateJobStateAction.INSTANCE, request, ActionListener.wrap(r -> handler.accept(null), errorHandler));
public void setJobState(long taskId, JobState state, CheckedConsumer<Exception, IOException> handler) {
UpdatePersistentTaskStatusAction.Request request = new UpdatePersistentTaskStatusAction.Request(taskId, state);
client.execute(UpdatePersistentTaskStatusAction.INSTANCE, request,
ActionListener.wrap(r -> handler.accept(null), e -> {
try {
handler.accept(e);
} catch (IOException e1) {
logger.warn("Error while delagating exception [" + e.getMessage() + "]", e1);
}
}));
}
public Optional<Tuple<DataCounts, ModelSizeStats>> getStatistics(String jobId) {

View File

@ -5,8 +5,6 @@
*/
package org.elasticsearch.xpack.ml.job.process.autodetect;
import org.apache.logging.log4j.Logger;
import org.elasticsearch.common.logging.Loggers;
import org.elasticsearch.xpack.ml.job.config.DetectionRule;
import org.elasticsearch.xpack.ml.job.config.ModelDebugConfig;
import org.elasticsearch.xpack.ml.job.process.autodetect.output.FlushAcknowledgement;
@ -30,8 +28,8 @@ import java.util.concurrent.BlockingQueue;
*/
public class BlackHoleAutodetectProcess implements AutodetectProcess {
private static final Logger LOGGER = Loggers.getLogger(BlackHoleAutodetectProcess.class);
private static final String FLUSH_ID = "flush-1";
private static final AutodetectResult EMPTY = new AutodetectResult(null, null, null, null, null, null, null, null, null);
private final ZonedDateTime startTime;
@ -76,6 +74,7 @@ public class BlackHoleAutodetectProcess implements AutodetectProcess {
@Override
public void close() throws IOException {
results.add(EMPTY);
}
@Override
@ -89,10 +88,11 @@ public class BlackHoleAutodetectProcess implements AutodetectProcess {
public boolean hasNext() {
try {
result = results.take();
return result != EMPTY;
} catch (InterruptedException e) {
Thread.currentThread().interrupt();
return false;
}
return true;
}
@Override

View File

@ -173,7 +173,6 @@ public class AutoDetectResultProcessor {
completionLatch.await();
} catch (InterruptedException e) {
Thread.currentThread().interrupt();
throw new RuntimeException(e);
}
}

View File

@ -46,7 +46,7 @@ public class AutodetectResultsParser extends AbstractComponent {
}
}
private void consumeAndCloseStream(InputStream in) {
static void consumeAndCloseStream(InputStream in) {
try {
// read anything left in the stream before
// closing the stream otherwise if the process
@ -58,7 +58,7 @@ public class AutodetectResultsParser extends AbstractComponent {
}
in.close();
} catch (IOException e) {
logger.warn("Error closing result parser input stream", e);
throw new RuntimeException("Error closing result parser input stream", e);
}
}
@ -79,13 +79,16 @@ public class AutodetectResultsParser extends AbstractComponent {
try {
token = parser.nextToken();
} catch (IOException e) {
throw new ElasticsearchParseException(e.getMessage(), e);
logger.debug("io error while parsing", e);
consumeAndCloseStream(in);
return false;
}
if (token == XContentParser.Token.END_ARRAY) {
consumeAndCloseStream(in);
return false;
} else if (token != XContentParser.Token.START_OBJECT) {
logger.error("Expecting Json Field name token after the Start Object token");
consumeAndCloseStream(in);
throw new ElasticsearchParseException("unexpected token [" + token + "]");
}
return true;

View File

@ -29,8 +29,8 @@ public class RestCloseJobAction extends BaseRestHandler {
@Override
protected RestChannelConsumer prepareRequest(RestRequest restRequest, NodeClient client) throws IOException {
CloseJobAction.Request request = new CloseJobAction.Request(restRequest.param(Job.ID.getPreferredName()));
if (restRequest.hasParam("close_timeout")) {
request.setCloseTimeout(TimeValue.parseTimeValue(restRequest.param("close_timeout"), "close_timeout"));
if (restRequest.hasParam("timeout")) {
request.setTimeout(TimeValue.parseTimeValue(restRequest.param("timeout"), "timeout"));
}
return channel -> client.execute(CloseJobAction.INSTANCE, request, new RestToXContentListener<>(channel));
}

View File

@ -8,14 +8,18 @@ package org.elasticsearch.xpack.ml.rest.job;
import org.elasticsearch.client.node.NodeClient;
import org.elasticsearch.common.settings.Settings;
import org.elasticsearch.common.unit.TimeValue;
import org.elasticsearch.common.xcontent.XContentBuilder;
import org.elasticsearch.rest.BaseRestHandler;
import org.elasticsearch.rest.BytesRestResponse;
import org.elasticsearch.rest.RestController;
import org.elasticsearch.rest.RestRequest;
import org.elasticsearch.rest.action.RestToXContentListener;
import org.elasticsearch.rest.RestResponse;
import org.elasticsearch.rest.RestStatus;
import org.elasticsearch.rest.action.RestBuilderListener;
import org.elasticsearch.xpack.ml.MlPlugin;
import org.elasticsearch.xpack.ml.action.OpenJobAction;
import org.elasticsearch.xpack.ml.action.PostDataAction;
import org.elasticsearch.xpack.ml.job.config.Job;
import org.elasticsearch.xpack.persistent.PersistentActionResponse;
import java.io.IOException;
@ -29,14 +33,28 @@ public class RestOpenJobAction extends BaseRestHandler {
@Override
protected RestChannelConsumer prepareRequest(RestRequest restRequest, NodeClient client) throws IOException {
OpenJobAction.Request request = new OpenJobAction.Request(restRequest.param(Job.ID.getPreferredName()));
OpenJobAction.Request request;
if (restRequest.hasContentOrSourceParam()) {
request = OpenJobAction.Request.parseRequest(restRequest.param(Job.ID.getPreferredName()), restRequest.contentParser());
} else {
request = new OpenJobAction.Request(restRequest.param(Job.ID.getPreferredName()));
request.setIgnoreDowntime(restRequest.paramAsBoolean(OpenJobAction.Request.IGNORE_DOWNTIME.getPreferredName(), false));
if (restRequest.hasParam("open_timeout")) {
TimeValue openTimeout = restRequest.paramAsTime("open_timeout", TimeValue.timeValueSeconds(30));
request.setOpenTimeout(openTimeout);
if (restRequest.hasParam("timeout")) {
TimeValue openTimeout = restRequest.paramAsTime("timeout", TimeValue.timeValueSeconds(30));
request.setTimeout(openTimeout);
}
}
return channel -> {
client.execute(OpenJobAction.INSTANCE, request, new RestToXContentListener<>(channel));
client.execute(OpenJobAction.INSTANCE, request, new RestBuilderListener<PersistentActionResponse>(channel) {
@Override
public RestResponse buildResponse(PersistentActionResponse r, XContentBuilder builder) throws Exception {
builder.startObject();
builder.field("opened", true);
builder.endObject();
return new BytesRestResponse(RestStatus.OK, builder);
}
});
};
}
}

View File

@ -6,15 +6,17 @@
package org.elasticsearch.xpack.ml.utils;
import org.apache.logging.log4j.Logger;
import org.elasticsearch.ElasticsearchStatusException;
import org.elasticsearch.cluster.ClusterState;
import org.elasticsearch.cluster.ClusterStateObserver;
import org.elasticsearch.cluster.service.ClusterService;
import org.elasticsearch.common.logging.Loggers;
import org.elasticsearch.common.unit.TimeValue;
import org.elasticsearch.rest.RestStatus;
import org.elasticsearch.threadpool.ThreadPool;
import org.elasticsearch.xpack.ml.job.config.JobState;
import org.elasticsearch.xpack.ml.job.metadata.Allocation;
import org.elasticsearch.xpack.ml.job.metadata.MlMetadata;
import org.elasticsearch.xpack.persistent.PersistentTasksInProgress;
import java.util.function.Consumer;
import java.util.function.Predicate;
@ -38,8 +40,13 @@ public class JobStateObserver {
observer.waitForNextChange(new ClusterStateObserver.Listener() {
@Override
public void onNewClusterState(ClusterState state) {
if (jobStatePredicate.failed) {
handler.accept(new ElasticsearchStatusException("[" + jobId + "] expected state [" + JobState.OPENED +
"] but got [" + JobState.FAILED +"]", RestStatus.CONFLICT));
} else {
handler.accept(null);
}
}
@Override
public void onClusterServiceClose() {
@ -51,7 +58,12 @@ public class JobStateObserver {
@Override
public void onTimeout(TimeValue timeout) {
if (jobStatePredicate.test(clusterService.state())) {
if (jobStatePredicate.failed) {
handler.accept(new ElasticsearchStatusException("[" + jobId + "] expected state [" + JobState.OPENED +
"] but got [" + JobState.FAILED +"]", RestStatus.CONFLICT));
} else {
handler.accept(null);
}
} else {
Exception e = new IllegalArgumentException("Timeout expired while waiting for job state to change to ["
+ expectedState + "]");
@ -66,6 +78,8 @@ public class JobStateObserver {
private final String jobId;
private final JobState expectedState;
private volatile boolean failed;
JobStatePredicate(String jobId, JobState expectedState) {
this.jobId = jobId;
this.expectedState = expectedState;
@ -73,15 +87,15 @@ public class JobStateObserver {
@Override
public boolean test(ClusterState newState) {
MlMetadata metadata = newState.getMetaData().custom(MlMetadata.TYPE);
if (metadata != null) {
Allocation allocation = metadata.getAllocations().get(jobId);
if (allocation != null) {
return allocation.getState() == expectedState;
PersistentTasksInProgress tasks = newState.custom(PersistentTasksInProgress.TYPE);
JobState jobState = MlMetadata.getJobState(jobId, tasks);
if (jobState == JobState.FAILED) {
failed = true;
return true;
} else {
return jobState == expectedState;
}
}
return false;
}
}

View File

@ -8,6 +8,7 @@ package org.elasticsearch.xpack.persistent;
import org.apache.logging.log4j.message.ParameterizedMessage;
import org.apache.logging.log4j.util.Supplier;
import org.elasticsearch.action.ActionListener;
import org.elasticsearch.action.admin.cluster.node.tasks.cancel.CancelTasksRequest;
import org.elasticsearch.action.admin.cluster.node.tasks.cancel.CancelTasksResponse;
import org.elasticsearch.cluster.ClusterChangedEvent;
import org.elasticsearch.cluster.ClusterStateListener;
@ -19,11 +20,11 @@ import org.elasticsearch.common.io.stream.StreamInput;
import org.elasticsearch.common.io.stream.StreamOutput;
import org.elasticsearch.common.settings.Settings;
import org.elasticsearch.common.xcontent.XContentBuilder;
import org.elasticsearch.xpack.persistent.PersistentTasksInProgress.PersistentTaskInProgress;
import org.elasticsearch.tasks.Task;
import org.elasticsearch.tasks.TaskCancelledException;
import org.elasticsearch.tasks.TaskManager;
import org.elasticsearch.transport.TransportResponse.Empty;
import org.elasticsearch.xpack.persistent.PersistentTasksInProgress.PersistentTaskInProgress;
import java.io.IOException;
import java.util.HashMap;
@ -211,10 +212,14 @@ public class PersistentActionCoordinator extends AbstractComponent implements Cl
"cancelled task {} failed with an exception, cancellation reason [{}]",
task.getId(), task.getTask().getReasonCancelled()), e);
}
if (CancelTasksRequest.DEFAULT_REASON.equals(task.getTask().getReasonCancelled())) {
startCompletionNotification(task, null);
} else {
startCompletionNotification(task, e);
}
} else {
startCompletionNotification(task, e);
}
}
}

View File

@ -13,8 +13,10 @@ import org.elasticsearch.cluster.node.DiscoveryNode;
import org.elasticsearch.cluster.service.ClusterService;
import org.elasticsearch.common.component.AbstractComponent;
import org.elasticsearch.common.settings.Settings;
import org.elasticsearch.common.util.concurrent.AbstractRunnable;
import org.elasticsearch.tasks.Task;
import org.elasticsearch.tasks.TaskId;
import org.elasticsearch.threadpool.ThreadPool;
/**
* Service responsible for executing restartable actions that can survive disappearance of a coordinating and executor nodes.
@ -22,11 +24,13 @@ import org.elasticsearch.tasks.TaskId;
public class PersistentActionService extends AbstractComponent {
private final Client client;
private final ThreadPool threadPool;
private final ClusterService clusterService;
public PersistentActionService(Settings settings, ClusterService clusterService, Client client) {
public PersistentActionService(Settings settings, ThreadPool threadPool, ClusterService clusterService, Client client) {
super(settings);
this.client = client;
this.threadPool = threadPool;
this.clusterService = clusterService;
}
@ -43,8 +47,20 @@ public class PersistentActionService extends AbstractComponent {
public void sendCompletionNotification(long taskId, Exception failure,
ActionListener<CompletionPersistentTaskAction.Response> listener) {
CompletionPersistentTaskAction.Request restartRequest = new CompletionPersistentTaskAction.Request(taskId, failure);
// Need to fork otherwise: java.lang.AssertionError: should not be called by a cluster state applier.
// reason [the applied cluster state is not yet available])
try {
threadPool.executor(ThreadPool.Names.GENERIC).execute(new AbstractRunnable() {
@Override
public void onFailure(Exception e) {
listener.onFailure(e);
}
@Override
protected void doRun() throws Exception {
client.execute(CompletionPersistentTaskAction.INSTANCE, restartRequest, listener);
}
});
} catch (Exception e) {
listener.onFailure(e);
}

View File

@ -7,7 +7,6 @@ package org.elasticsearch.xpack.persistent;
import org.elasticsearch.common.inject.Provider;
import org.elasticsearch.tasks.CancellableTask;
import org.elasticsearch.tasks.Task;
import org.elasticsearch.tasks.TaskId;
/**
@ -27,6 +26,15 @@ public class PersistentTask extends CancellableTask {
return true;
}
// In case of persistent tasks we always need to return: `false`
// because in case of persistent task the parent task isn't a task in the task manager, but in cluster state.
// This instructs the task manager not to try to kill this persistent task when the task manager cannot find
// a fake parent node id "cluster" in the cluster state
@Override
public final boolean cancelOnParentLeaving() {
return false;
}
@Override
public Status getStatus() {
Provider<Status> statusProvider = this.statusProvider;

View File

@ -109,7 +109,7 @@ public final class PersistentTasksInProgress extends AbstractNamedDiffable<Clust
public PersistentTaskInProgress(PersistentTaskInProgress<Request> persistentTaskInProgress, String newExecutorNode) {
this(persistentTaskInProgress.id, persistentTaskInProgress.allocationId + 1L,
persistentTaskInProgress.action, persistentTaskInProgress.request, null, newExecutorNode);
persistentTaskInProgress.action, persistentTaskInProgress.request, persistentTaskInProgress.status, newExecutorNode);
}
public PersistentTaskInProgress(PersistentTaskInProgress<Request> persistentTaskInProgress, Status status) {

View File

@ -6,16 +6,17 @@
package org.elasticsearch.xpack.ml.action;
import org.elasticsearch.common.unit.TimeValue;
import org.elasticsearch.common.xcontent.XContentParser;
import org.elasticsearch.xpack.ml.action.CloseJobAction.Request;
import org.elasticsearch.xpack.ml.support.AbstractStreamableTestCase;
import org.elasticsearch.xpack.ml.support.AbstractStreamableXContentTestCase;
public class CloseJobActionRequestTests extends AbstractStreamableTestCase<Request> {
public class CloseJobActionRequestTests extends AbstractStreamableXContentTestCase<Request> {
@Override
protected Request createTestInstance() {
Request request = new Request(randomAsciiOfLengthBetween(1, 20));
if (randomBoolean()) {
request.setCloseTimeout(TimeValue.timeValueMillis(randomNonNegativeLong()));
request.setTimeout(TimeValue.timeValueMillis(randomNonNegativeLong()));
}
return request;
}
@ -24,4 +25,9 @@ public class CloseJobActionRequestTests extends AbstractStreamableTestCase<Reque
protected Request createBlankInstance() {
return new Request();
}
@Override
protected Request parseInstance(XContentParser parser) {
return Request.parseRequest(null, parser);
}
}

View File

@ -0,0 +1,73 @@
/*
* Copyright Elasticsearch B.V. and/or licensed to Elasticsearch B.V. under one
* or more contributor license agreements. Licensed under the Elastic License;
* you may not use this file except in compliance with the Elastic License.
*/
package org.elasticsearch.xpack.ml.action;
import org.elasticsearch.ElasticsearchStatusException;
import org.elasticsearch.ResourceNotFoundException;
import org.elasticsearch.cluster.ClusterName;
import org.elasticsearch.cluster.ClusterState;
import org.elasticsearch.cluster.metadata.MetaData;
import org.elasticsearch.test.ESTestCase;
import org.elasticsearch.xpack.ml.job.config.JobState;
import org.elasticsearch.xpack.ml.job.metadata.MlMetadata;
import org.elasticsearch.xpack.persistent.PersistentTasksInProgress;
import org.elasticsearch.xpack.persistent.PersistentTasksInProgress.PersistentTaskInProgress;
import java.util.Collections;
import static org.elasticsearch.xpack.ml.job.config.JobTests.buildJobBuilder;
public class CloseJobActionTests extends ESTestCase {
public void testMoveJobToClosingState() {
MlMetadata.Builder mlBuilder = new MlMetadata.Builder();
mlBuilder.putJob(buildJobBuilder("job_id").build(), false);
PersistentTaskInProgress<OpenJobAction.Request> task =
new PersistentTaskInProgress<>(1L, OpenJobAction.NAME, new OpenJobAction.Request("job_id"), null);
task = new PersistentTaskInProgress<>(task, JobState.OPENED);
ClusterState.Builder csBuilder = ClusterState.builder(new ClusterName("_name"))
.metaData(new MetaData.Builder().putCustom(MlMetadata.TYPE, mlBuilder.build()))
.putCustom(PersistentTasksInProgress.TYPE, new PersistentTasksInProgress(1L, Collections.singletonMap(1L, task)));
ClusterState result = CloseJobAction.moveJobToClosingState("job_id", csBuilder.build());
PersistentTasksInProgress actualTasks = result.custom(PersistentTasksInProgress.TYPE);
assertEquals(JobState.CLOSING, actualTasks.getTask(1L).getStatus());
MlMetadata actualMetadata = result.metaData().custom(MlMetadata.TYPE);
assertNotNull(actualMetadata.getJobs().get("job_id").getFinishedTime());
}
public void testMoveJobToClosingState_jobMissing() {
MlMetadata.Builder mlBuilder = new MlMetadata.Builder();
ClusterState.Builder csBuilder = ClusterState.builder(new ClusterName("_name"))
.metaData(new MetaData.Builder().putCustom(MlMetadata.TYPE, mlBuilder.build()))
.putCustom(PersistentTasksInProgress.TYPE, new PersistentTasksInProgress(1L, Collections.emptyMap()));
expectThrows(ResourceNotFoundException.class, () -> CloseJobAction.moveJobToClosingState("job_id", csBuilder.build()));
}
public void testMoveJobToClosingState_unexpectedJobState() {
MlMetadata.Builder mlBuilder = new MlMetadata.Builder();
mlBuilder.putJob(buildJobBuilder("job_id").build(), false);
PersistentTaskInProgress<OpenJobAction.Request> task =
new PersistentTaskInProgress<>(1L, OpenJobAction.NAME, new OpenJobAction.Request("job_id"), null);
task = new PersistentTaskInProgress<>(task, JobState.OPENING);
ClusterState.Builder csBuilder1 = ClusterState.builder(new ClusterName("_name"))
.metaData(new MetaData.Builder().putCustom(MlMetadata.TYPE, mlBuilder.build()))
.putCustom(PersistentTasksInProgress.TYPE, new PersistentTasksInProgress(1L, Collections.singletonMap(1L, task)));
ElasticsearchStatusException result =
expectThrows(ElasticsearchStatusException.class, () -> CloseJobAction.moveJobToClosingState("job_id", csBuilder1.build()));
assertEquals("cannot close job, expected job state [opened], but got [opening]", result.getMessage());
ClusterState.Builder csBuilder2 = ClusterState.builder(new ClusterName("_name"))
.metaData(new MetaData.Builder().putCustom(MlMetadata.TYPE, mlBuilder.build()))
.putCustom(PersistentTasksInProgress.TYPE, new PersistentTasksInProgress(1L, Collections.emptyMap()));
result = expectThrows(ElasticsearchStatusException.class, () -> CloseJobAction.moveJobToClosingState("job_id", csBuilder2.build()));
assertEquals("cannot close job, expected job state [opened], but got [closed]", result.getMessage());
}
}

View File

@ -5,7 +5,6 @@
*/
package org.elasticsearch.xpack.ml.action;
import org.elasticsearch.ExceptionsHelper;
import org.elasticsearch.action.admin.cluster.node.hotthreads.NodeHotThreads;
import org.elasticsearch.action.admin.cluster.node.hotthreads.NodesHotThreadsResponse;
import org.elasticsearch.action.bulk.BulkRequestBuilder;
@ -13,19 +12,9 @@ import org.elasticsearch.action.bulk.BulkResponse;
import org.elasticsearch.action.get.GetResponse;
import org.elasticsearch.action.index.IndexRequest;
import org.elasticsearch.action.support.WriteRequest;
import org.elasticsearch.client.Client;
import org.elasticsearch.cluster.metadata.MetaData;
import org.elasticsearch.common.settings.Settings;
import org.elasticsearch.common.util.concurrent.ThreadContext;
import org.elasticsearch.common.xcontent.NamedXContentRegistry;
import org.elasticsearch.common.xcontent.XContentHelper;
import org.elasticsearch.common.xcontent.XContentParser;
import org.elasticsearch.plugins.Plugin;
import org.elasticsearch.rest.RestStatus;
import org.elasticsearch.test.ESIntegTestCase;
import org.elasticsearch.test.SecurityIntegTestCase;
import org.elasticsearch.xpack.XPackPlugin;
import org.elasticsearch.xpack.ml.MlPlugin;
import org.elasticsearch.xpack.ml.datafeed.DatafeedConfig;
import org.elasticsearch.xpack.ml.datafeed.DatafeedState;
import org.elasticsearch.xpack.ml.job.config.AnalysisConfig;
@ -33,52 +22,32 @@ import org.elasticsearch.xpack.ml.job.config.DataDescription;
import org.elasticsearch.xpack.ml.job.config.Detector;
import org.elasticsearch.xpack.ml.job.config.Job;
import org.elasticsearch.xpack.ml.job.config.JobState;
import org.elasticsearch.xpack.ml.job.metadata.MlMetadata;
import org.elasticsearch.xpack.ml.job.persistence.AnomalyDetectorsIndex;
import org.elasticsearch.xpack.ml.job.process.autodetect.state.DataCounts;
import org.elasticsearch.xpack.security.authc.support.SecuredString;
import org.elasticsearch.xpack.ml.support.BaseMlIntegTestCase;
import org.elasticsearch.xpack.persistent.PersistentActionResponse;
import org.elasticsearch.xpack.persistent.RemovePersistentTaskAction;
import org.junit.After;
import org.junit.Before;
import java.io.IOException;
import java.util.Collection;
import java.util.Collections;
import java.util.List;
import java.util.Map;
import java.util.concurrent.ExecutionException;
import java.util.concurrent.TimeUnit;
import static org.elasticsearch.xpack.ml.integration.TooManyJobsIT.ensureClusterStateConsistencyWorkAround;
import static org.elasticsearch.xpack.security.authc.support.UsernamePasswordToken.basicAuthHeaderValue;
import static org.hamcrest.Matchers.equalTo;
import static org.hamcrest.Matchers.is;
@ESIntegTestCase.ClusterScope(numDataNodes = 1)
public class DatafeedJobsIT extends SecurityIntegTestCase {
public class DatafeedJobsIT extends BaseMlIntegTestCase {
@Override
protected Settings externalClusterClientSettings() {
return Settings.builder().put(super.externalClusterClientSettings()).put("transport.type", "security4")
.put(MlPlugin.ML_ENABLED.getKey(), true)
.put(ThreadContext.PREFIX + ".Authorization", basicAuthHeaderValue("elastic", new SecuredString("changeme".toCharArray())))
.build();
}
@Override
protected Collection<Class<? extends Plugin>> nodePlugins() {
return Collections.singleton(XPackPlugin.class);
}
@Override
protected Collection<Class<? extends Plugin>> transportClientPlugins() {
return nodePlugins();
@Before
public void startNode() {
internalCluster().ensureAtLeastNumDataNodes(1);
}
@After
public void clearMlMetadata() throws Exception {
clearMlMetadata(client());
public void stopNode() throws Exception {
cleanupWorkaround(1);
}
public void testLookbackOnly() throws Exception {
@ -97,11 +66,11 @@ public class DatafeedJobsIT extends SecurityIntegTestCase {
long numDocs2 = randomIntBetween(32, 2048);
indexDocs("data-2", numDocs2, oneWeekAgo, now);
Job.Builder job = createJob("lookback-job");
Job.Builder job = createScheduledJob("lookback-job");
PutJobAction.Request putJobRequest = new PutJobAction.Request(job.build(true, job.getId()));
PutJobAction.Response putJobResponse = client().execute(PutJobAction.INSTANCE, putJobRequest).get();
assertTrue(putJobResponse.isAcknowledged());
client().execute(InternalOpenJobAction.INSTANCE, new InternalOpenJobAction.Request(job.getId()));
client().execute(OpenJobAction.INSTANCE, new OpenJobAction.Request(job.getId()));
assertBusy(() -> {
GetJobsStatsAction.Response statsResponse =
client().execute(GetJobsStatsAction.INSTANCE, new GetJobsStatsAction.Request(job.getId())).actionGet();
@ -137,11 +106,11 @@ public class DatafeedJobsIT extends SecurityIntegTestCase {
long lastWeek = now - 604800000;
indexDocs("data", numDocs1, lastWeek, now);
Job.Builder job = createJob("realtime-job");
Job.Builder job = createScheduledJob("realtime-job");
PutJobAction.Request putJobRequest = new PutJobAction.Request(job.build(true, job.getId()));
PutJobAction.Response putJobResponse = client().execute(PutJobAction.INSTANCE, putJobRequest).get();
assertTrue(putJobResponse.isAcknowledged());
client().execute(InternalOpenJobAction.INSTANCE, new InternalOpenJobAction.Request(job.getId()));
client().execute(OpenJobAction.INSTANCE, new OpenJobAction.Request(job.getId()));
assertBusy(() -> {
GetJobsStatsAction.Response statsResponse =
client().execute(GetJobsStatsAction.INSTANCE, new GetJobsStatsAction.Request(job.getId())).actionGet();
@ -207,7 +176,7 @@ public class DatafeedJobsIT extends SecurityIntegTestCase {
logger.info("Indexed [{}] documents", numDocs);
}
private Job.Builder createJob(String jobId) {
private Job.Builder createScheduledJob(String jobId) {
DataDescription.Builder dataDescription = new DataDescription.Builder();
dataDescription.setFormat(DataDescription.DataFormat.JSON);
dataDescription.setTimeFormat("yyyy-MM-dd HH:mm:ss");
@ -246,61 +215,4 @@ public class DatafeedJobsIT extends SecurityIntegTestCase {
}
}
public static void clearMlMetadata(Client client) throws Exception {
deleteAllDatafeeds(client);
deleteAllJobs(client);
}
private static void deleteAllDatafeeds(Client client) throws Exception {
MetaData metaData = client.admin().cluster().prepareState().get().getState().getMetaData();
MlMetadata mlMetadata = metaData.custom(MlMetadata.TYPE);
for (DatafeedConfig datafeed : mlMetadata.getDatafeeds().values()) {
String datafeedId = datafeed.getId();
try {
RemovePersistentTaskAction.Response stopResponse =
client.execute(StopDatafeedAction.INSTANCE, new StopDatafeedAction.Request(datafeedId)).get();
assertTrue(stopResponse.isAcknowledged());
} catch (ExecutionException e) {
// CONFLICT is ok, as it means the datafeed has already stopped, which isn't an issue at all.
if (RestStatus.CONFLICT != ExceptionsHelper.status(e.getCause())) {
throw new RuntimeException(e);
}
}
assertBusy(() -> {
try {
GetDatafeedsStatsAction.Request request = new GetDatafeedsStatsAction.Request(datafeedId);
GetDatafeedsStatsAction.Response r = client.execute(GetDatafeedsStatsAction.INSTANCE, request).get();
assertThat(r.getResponse().results().get(0).getDatafeedState(), equalTo(DatafeedState.STOPPED));
} catch (InterruptedException | ExecutionException e) {
throw new RuntimeException(e);
}
});
DeleteDatafeedAction.Response deleteResponse =
client.execute(DeleteDatafeedAction.INSTANCE, new DeleteDatafeedAction.Request(datafeedId)).get();
assertTrue(deleteResponse.isAcknowledged());
}
}
public static void deleteAllJobs(Client client) throws Exception {
MetaData metaData = client.admin().cluster().prepareState().get().getState().getMetaData();
MlMetadata mlMetadata = metaData.custom(MlMetadata.TYPE);
for (Map.Entry<String, Job> entry : mlMetadata.getJobs().entrySet()) {
String jobId = entry.getKey();
try {
CloseJobAction.Response response =
client.execute(CloseJobAction.INSTANCE, new CloseJobAction.Request(jobId)).get();
assertTrue(response.isClosed());
} catch (Exception e) {
// ignore
}
DeleteJobAction.Response response =
client.execute(DeleteJobAction.INSTANCE, new DeleteJobAction.Request(jobId)).get();
assertTrue(response.isAcknowledged());
}
}
@Override
protected void ensureClusterStateConsistency() throws IOException {
ensureClusterStateConsistencyWorkAround();
}
}

View File

@ -6,6 +6,7 @@
package org.elasticsearch.xpack.ml.action;
import org.elasticsearch.xpack.ml.action.GetJobsAction.Response;
import org.elasticsearch.xpack.ml.action.util.QueryPage;
import org.elasticsearch.xpack.ml.job.config.AnalysisConfig;
import org.elasticsearch.xpack.ml.job.config.AnalysisLimits;
import org.elasticsearch.xpack.ml.job.config.DataDescription;
@ -13,7 +14,6 @@ import org.elasticsearch.xpack.ml.job.config.Detector;
import org.elasticsearch.xpack.ml.job.config.IgnoreDowntime;
import org.elasticsearch.xpack.ml.job.config.Job;
import org.elasticsearch.xpack.ml.job.config.ModelDebugConfig;
import org.elasticsearch.xpack.ml.action.util.QueryPage;
import org.elasticsearch.xpack.ml.support.AbstractStreamableTestCase;
import java.util.ArrayList;
@ -54,7 +54,7 @@ public class GetJobsActionResponseTests extends AbstractStreamableTestCase<GetJo
Job job = new Job(jobId, description, createTime, finishedTime, lastDataTime,
analysisConfig, analysisLimits, dataDescription,
modelDebugConfig, ignoreDowntime, normalizationWindowDays, backgroundPersistInterval,
modelSnapshotRetentionDays, resultsRetentionDays, customConfig, modelSnapshotId, indexName);
modelSnapshotRetentionDays, resultsRetentionDays, customConfig, modelSnapshotId, indexName, randomBoolean());
jobList.add(job);
}

View File

@ -6,16 +6,17 @@
package org.elasticsearch.xpack.ml.action;
import org.elasticsearch.common.unit.TimeValue;
import org.elasticsearch.common.xcontent.XContentParser;
import org.elasticsearch.xpack.ml.action.OpenJobAction.Request;
import org.elasticsearch.xpack.ml.support.AbstractStreamableTestCase;
import org.elasticsearch.xpack.ml.support.AbstractStreamableXContentTestCase;
public class OpenJobActionRequestTests extends AbstractStreamableTestCase<Request> {
public class OpenJobActionRequestTests extends AbstractStreamableXContentTestCase<Request> {
@Override
protected Request createTestInstance() {
Request request = new Request(randomAsciiOfLengthBetween(1, 20));
if (randomBoolean()) {
request.setOpenTimeout(TimeValue.timeValueMillis(randomNonNegativeLong()));
request.setTimeout(TimeValue.timeValueMillis(randomNonNegativeLong()));
}
return request;
}
@ -24,4 +25,9 @@ public class OpenJobActionRequestTests extends AbstractStreamableTestCase<Reques
protected Request createBlankInstance() {
return new Request();
}
@Override
protected Request parseInstance(XContentParser parser) {
return Request.parseRequest(null, parser);
}
}

View File

@ -1,22 +0,0 @@
/*
* Copyright Elasticsearch B.V. and/or licensed to Elasticsearch B.V. under one
* or more contributor license agreements. Licensed under the Elastic License;
* you may not use this file except in compliance with the Elastic License.
*/
package org.elasticsearch.xpack.ml.action;
import org.elasticsearch.xpack.ml.action.OpenJobAction.Response;
import org.elasticsearch.xpack.ml.support.AbstractStreamableTestCase;
public class OpenJobActionResponseTests extends AbstractStreamableTestCase<Response> {
@Override
protected Response createTestInstance() {
return new Response(randomBoolean());
}
@Override
protected Response createBlankInstance() {
return new Response();
}
}

View File

@ -0,0 +1,95 @@
/*
* Copyright Elasticsearch B.V. and/or licensed to Elasticsearch B.V. under one
* or more contributor license agreements. Licensed under the Elastic License;
* you may not use this file except in compliance with the Elastic License.
*/
package org.elasticsearch.xpack.ml.action;
import org.elasticsearch.ElasticsearchStatusException;
import org.elasticsearch.ResourceNotFoundException;
import org.elasticsearch.Version;
import org.elasticsearch.cluster.node.DiscoveryNode;
import org.elasticsearch.cluster.node.DiscoveryNodes;
import org.elasticsearch.common.transport.TransportAddress;
import org.elasticsearch.test.ESTestCase;
import org.elasticsearch.xpack.ml.job.config.Job;
import org.elasticsearch.xpack.ml.job.config.JobState;
import org.elasticsearch.xpack.ml.job.metadata.MlMetadata;
import org.elasticsearch.xpack.persistent.PersistentTasksInProgress;
import org.elasticsearch.xpack.persistent.PersistentTasksInProgress.PersistentTaskInProgress;
import java.net.InetAddress;
import java.util.Collections;
import static org.elasticsearch.xpack.ml.job.config.JobTests.buildJobBuilder;
public class OpenJobActionTests extends ESTestCase {
public void testValidate() {
MlMetadata.Builder mlBuilder = new MlMetadata.Builder();
mlBuilder.putJob(buildJobBuilder("job_id").build(), false);
DiscoveryNodes nodes = DiscoveryNodes.builder()
.add(new DiscoveryNode("_node_name", "_node_id", new TransportAddress(InetAddress.getLoopbackAddress(), 9300),
Collections.emptyMap(), Collections.emptySet(), Version.CURRENT))
.build();
PersistentTaskInProgress<OpenJobAction.Request> task =
new PersistentTaskInProgress<>(1L, OpenJobAction.NAME, new OpenJobAction.Request("job_id"), "_node_id");
task = new PersistentTaskInProgress<>(task, randomFrom(JobState.CLOSED, JobState.FAILED));
PersistentTasksInProgress tasks = new PersistentTasksInProgress(1L, Collections.singletonMap(1L, task));
OpenJobAction.validate("job_id", mlBuilder.build(), tasks, nodes);
OpenJobAction.validate("job_id", mlBuilder.build(), new PersistentTasksInProgress(1L, Collections.emptyMap()), nodes);
OpenJobAction.validate("job_id", mlBuilder.build(), null, nodes);
task = new PersistentTaskInProgress<>(1L, OpenJobAction.NAME, new OpenJobAction.Request("job_id"), "_other_node_id");
task = new PersistentTaskInProgress<>(task, JobState.OPENED);
tasks = new PersistentTasksInProgress(1L, Collections.singletonMap(1L, task));
OpenJobAction.validate("job_id", mlBuilder.build(), tasks, nodes);
}
public void testValidate_jobMissing() {
MlMetadata.Builder mlBuilder = new MlMetadata.Builder();
mlBuilder.putJob(buildJobBuilder("job_id1").build(), false);
expectThrows(ResourceNotFoundException.class, () -> OpenJobAction.validate("job_id2", mlBuilder.build(), null, null));
}
public void testValidate_jobMarkedAsDeleted() {
MlMetadata.Builder mlBuilder = new MlMetadata.Builder();
Job.Builder jobBuilder = buildJobBuilder("job_id");
jobBuilder.setDeleted(true);
mlBuilder.putJob(jobBuilder.build(), false);
Exception e = expectThrows(ElasticsearchStatusException.class,
() -> OpenJobAction.validate("job_id", mlBuilder.build(), null, null));
assertEquals("Cannot open job [job_id] because it has been marked as deleted", e.getMessage());
}
public void testValidate_unexpectedState() {
MlMetadata.Builder mlBuilder = new MlMetadata.Builder();
mlBuilder.putJob(buildJobBuilder("job_id").build(), false);
DiscoveryNodes nodes = DiscoveryNodes.builder()
.add(new DiscoveryNode("_node_name", "_node_id", new TransportAddress(InetAddress.getLoopbackAddress(), 9300),
Collections.emptyMap(), Collections.emptySet(), Version.CURRENT))
.build();
PersistentTaskInProgress<OpenJobAction.Request> task =
new PersistentTaskInProgress<>(1L, OpenJobAction.NAME, new OpenJobAction.Request("job_id"), "_node_id");
JobState jobState = randomFrom(JobState.OPENING, JobState.OPENED, JobState.CLOSING);
task = new PersistentTaskInProgress<>(task, jobState);
PersistentTasksInProgress tasks1 = new PersistentTasksInProgress(1L, Collections.singletonMap(1L, task));
Exception e = expectThrows(ElasticsearchStatusException.class,
() -> OpenJobAction.validate("job_id", mlBuilder.build(), tasks1, nodes));
assertEquals("[job_id] expected state [closed] or [failed], but got [" + jobState +"]", e.getMessage());
task = new PersistentTaskInProgress<>(1L, OpenJobAction.NAME, new OpenJobAction.Request("job_id"), "_other_node_id");
jobState = randomFrom(JobState.OPENING, JobState.CLOSING);
task = new PersistentTaskInProgress<>(task, jobState);
PersistentTasksInProgress tasks2 = new PersistentTasksInProgress(1L, Collections.singletonMap(1L, task));
e = expectThrows(ElasticsearchStatusException.class,
() -> OpenJobAction.validate("job_id", mlBuilder.build(), tasks2, nodes));
assertEquals("[job_id] expected state [closed] or [failed], but got [" + jobState +"]", e.getMessage());
}
}

View File

@ -14,6 +14,10 @@ import org.elasticsearch.xpack.ml.datafeed.DatafeedJobRunnerTests;
import org.elasticsearch.xpack.ml.job.config.Job;
import org.elasticsearch.xpack.ml.job.metadata.MlMetadata;
import org.elasticsearch.xpack.ml.support.AbstractStreamableXContentTestCase;
import org.elasticsearch.xpack.persistent.PersistentTasksInProgress;
import org.elasticsearch.xpack.persistent.PersistentTasksInProgress.PersistentTaskInProgress;
import java.util.Collections;
import static org.hamcrest.Matchers.equalTo;
@ -44,15 +48,19 @@ public class StartDatafeedActionRequestTests extends AbstractStreamableXContentT
.putJob(job1, false)
.build();
Exception e = expectThrows(ResourceNotFoundException.class,
() -> StartDatafeedAction.validate("some-datafeed", mlMetadata1));
() -> StartDatafeedAction.validate("some-datafeed", mlMetadata1,
new PersistentTasksInProgress(0L, Collections.emptyMap())));
assertThat(e.getMessage(), equalTo("No datafeed with id [some-datafeed] exists"));
PersistentTaskInProgress<OpenJobAction.Request> task =
new PersistentTaskInProgress<>(0L, OpenJobAction.NAME, new OpenJobAction.Request("foo"), null);
PersistentTasksInProgress tasks = new PersistentTasksInProgress(0L, Collections.singletonMap(0L, task));
DatafeedConfig datafeedConfig1 = DatafeedJobRunnerTests.createDatafeedConfig("foo-datafeed", "foo").build();
MlMetadata mlMetadata2 = new MlMetadata.Builder(mlMetadata1)
.putDatafeed(datafeedConfig1)
.build();
e = expectThrows(ElasticsearchStatusException.class,
() -> StartDatafeedAction.validate("foo-datafeed", mlMetadata2));
() -> StartDatafeedAction.validate("foo-datafeed", mlMetadata2, tasks));
assertThat(e.getMessage(), equalTo("cannot start datafeed, expected job state [opened], but got [closed]"));
}

View File

@ -1,23 +0,0 @@
/*
* Copyright Elasticsearch B.V. and/or licensed to Elasticsearch B.V. under one
* or more contributor license agreements. Licensed under the Elastic License;
* you may not use this file except in compliance with the Elastic License.
*/
package org.elasticsearch.xpack.ml.action;
import org.elasticsearch.xpack.ml.action.UpdateJobStateAction.Request;
import org.elasticsearch.xpack.ml.job.config.JobState;
import org.elasticsearch.xpack.ml.support.AbstractStreamableTestCase;
public class UpdateJobStateRequestTests extends AbstractStreamableTestCase<Request> {
@Override
protected Request createTestInstance() {
return new Request(randomAsciiOfLengthBetween(1, 20), randomFrom(JobState.values()));
}
@Override
protected Request createBlankInstance() {
return new Request();
}
}

View File

@ -30,7 +30,6 @@ import org.elasticsearch.xpack.ml.job.config.AnalysisConfig;
import org.elasticsearch.xpack.ml.job.config.DataDescription;
import org.elasticsearch.xpack.ml.job.config.Detector;
import org.elasticsearch.xpack.ml.job.config.Job;
import org.elasticsearch.xpack.ml.job.config.JobState;
import org.elasticsearch.xpack.ml.job.metadata.MlMetadata;
import org.elasticsearch.xpack.ml.job.persistence.JobProvider;
import org.elasticsearch.xpack.ml.job.process.autodetect.state.DataCounts;
@ -47,7 +46,6 @@ import java.util.concurrent.ExecutorService;
import java.util.function.Consumer;
import static org.hamcrest.Matchers.instanceOf;
import static org.hamcrest.Matchers.is;
import static org.mockito.Matchers.any;
import static org.mockito.Matchers.anyString;
import static org.mockito.Matchers.same;
@ -122,7 +120,6 @@ public class DatafeedJobRunnerTests extends ESTestCase {
MlMetadata mlMetadata = new MlMetadata.Builder()
.putJob(job, false)
.putDatafeed(datafeedConfig)
.updateState("foo", JobState.OPENED, null)
.build();
when(clusterService.state()).thenReturn(ClusterState.builder(new ClusterName("_name"))
.metaData(MetaData.builder().putCustom(MlMetadata.TYPE, mlMetadata))
@ -161,7 +158,6 @@ public class DatafeedJobRunnerTests extends ESTestCase {
MlMetadata mlMetadata = new MlMetadata.Builder()
.putJob(job, false)
.putDatafeed(datafeedConfig)
.updateState("foo", JobState.OPENED, null)
.build();
when(clusterService.state()).thenReturn(ClusterState.builder(new ClusterName("_name"))
.metaData(MetaData.builder().putCustom(MlMetadata.TYPE, mlMetadata))
@ -190,7 +186,6 @@ public class DatafeedJobRunnerTests extends ESTestCase {
MlMetadata mlMetadata = new MlMetadata.Builder()
.putJob(job, false)
.putDatafeed(datafeedConfig)
.updateState("foo", JobState.OPENED, null)
.build();
when(clusterService.state()).thenReturn(ClusterState.builder(new ClusterName("_name"))
.metaData(MetaData.builder().putCustom(MlMetadata.TYPE, mlMetadata))

View File

@ -8,6 +8,7 @@ package org.elasticsearch.xpack.ml.integration;
import org.elasticsearch.action.ActionListener;
import org.elasticsearch.common.io.stream.Writeable;
import org.elasticsearch.common.settings.Settings;
import org.elasticsearch.common.unit.TimeValue;
import org.elasticsearch.common.xcontent.ToXContent;
import org.elasticsearch.plugins.Plugin;
import org.elasticsearch.test.ESSingleNodeTestCase;
@ -42,8 +43,8 @@ import org.elasticsearch.xpack.ml.job.results.ModelDebugOutputTests;
import org.junit.Before;
import java.util.ArrayList;
import java.util.Collection;
import java.util.Arrays;
import java.util.Collection;
import java.util.Collections;
import java.util.Date;
import java.util.EnumSet;
@ -80,7 +81,7 @@ public class AutodetectResultProcessorIT extends ESSingleNodeTestCase {
public void createComponents() {
renormalizer = new NoOpRenormalizer();
jobResultsPersister = new JobResultsPersister(nodeSettings(), client());
jobProvider = new JobProvider(client(), 1);
jobProvider = new JobProvider(client(), 1, TimeValue.timeValueSeconds(1));
}
public void testProcessResults() throws Exception {

View File

@ -0,0 +1,54 @@
/*
* Copyright Elasticsearch B.V. and/or licensed to Elasticsearch B.V. under one
* or more contributor license agreements. Licensed under the Elastic License;
* you may not use this file except in compliance with the Elastic License.
*/
package org.elasticsearch.xpack.ml.integration;
import org.elasticsearch.xpack.ml.action.GetJobsStatsAction;
import org.elasticsearch.xpack.ml.action.OpenJobAction;
import org.elasticsearch.xpack.ml.action.PutJobAction;
import org.elasticsearch.xpack.ml.job.config.Job;
import org.elasticsearch.xpack.ml.job.config.JobState;
import org.elasticsearch.xpack.ml.support.BaseMlIntegTestCase;
public class BasicDistributedJobsIT extends BaseMlIntegTestCase {
public void testFailOverBasics() throws Exception {
internalCluster().ensureAtLeastNumDataNodes(4);
ensureStableCluster(4);
Job.Builder job = createJob("job_id");
PutJobAction.Request putJobRequest = new PutJobAction.Request(job.build(true, job.getId()));
PutJobAction.Response putJobResponse = client().execute(PutJobAction.INSTANCE, putJobRequest).get();
assertTrue(putJobResponse.isAcknowledged());
ensureGreen();
OpenJobAction.Request openJobRequest = new OpenJobAction.Request(job.getId());
client().execute(OpenJobAction.INSTANCE, openJobRequest).get();
assertBusy(() -> {
GetJobsStatsAction.Response statsResponse =
client().execute(GetJobsStatsAction.INSTANCE, new GetJobsStatsAction.Request(job.getId())).actionGet();
assertEquals(JobState.OPENED, statsResponse.getResponse().results().get(0).getState());
});
internalCluster().stopRandomDataNode();
ensureStableCluster(3);
ensureGreen();
assertBusy(() -> {
GetJobsStatsAction.Response statsResponse =
client().execute(GetJobsStatsAction.INSTANCE, new GetJobsStatsAction.Request(job.getId())).actionGet();
assertEquals(JobState.OPENED, statsResponse.getResponse().results().get(0).getState());
});
internalCluster().stopRandomDataNode();
ensureStableCluster(2);
ensureGreen();
assertBusy(() -> {
GetJobsStatsAction.Response statsResponse =
client().execute(GetJobsStatsAction.INSTANCE, new GetJobsStatsAction.Request(job.getId())).actionGet();
assertEquals(JobState.OPENED, statsResponse.getResponse().results().get(0).getState());
});
cleanupWorkaround(2);
}
}

View File

@ -414,7 +414,7 @@ public class DatafeedJobIT extends ESRestTestCase {
@After
public void clearMlState() throws Exception {
new MlRestTestStateCleaner(client(), this).clearMlMetadata();
new MlRestTestStateCleaner(logger, client(), this).clearMlMetadata();
}
private static class DatafeedBuilder {

View File

@ -361,6 +361,6 @@ public class MlJobIT extends ESRestTestCase {
@After
public void clearMlState() throws IOException {
new MlRestTestStateCleaner(client(), this).clearMlMetadata();
new MlRestTestStateCleaner(logger, client(), this).clearMlMetadata();
}
}

View File

@ -5,6 +5,7 @@
*/
package org.elasticsearch.xpack.ml.integration;
import org.apache.logging.log4j.Logger;
import org.elasticsearch.client.RestClient;
import org.elasticsearch.common.xcontent.support.XContentMapValues;
import org.elasticsearch.test.rest.ESRestTestCase;
@ -16,10 +17,12 @@ import java.util.Map;
public class MlRestTestStateCleaner {
private final Logger logger;
private final RestClient client;
private final ESRestTestCase testCase;
public MlRestTestStateCleaner(RestClient client, ESRestTestCase testCase) {
public MlRestTestStateCleaner(Logger logger, RestClient client, ESRestTestCase testCase) {
this.logger = logger;
this.client = client;
this.testCase = testCase;
}
@ -44,7 +47,11 @@ public class MlRestTestStateCleaner {
try {
client.performRequest("POST", "/_xpack/ml/datafeeds/" + datafeedId + "/_stop");
} catch (Exception e) {
// ignore
if (e.getMessage().contains("datafeed already stopped, expected datafeed state [started], but got [stopped]")) {
logger.debug("failed to stop datafeed [" + datafeedId + "]", e);
} else {
logger.warn("failed to stop datafeed [" + datafeedId + "]", e);
}
}
client.performRequest("DELETE", "/_xpack/ml/datafeeds/" + datafeedId);
}
@ -65,7 +72,11 @@ public class MlRestTestStateCleaner {
try {
client.performRequest("POST", "/_xpack/ml/anomaly_detectors/" + jobId + "/_close");
} catch (Exception e) {
// ignore
if (e.getMessage().contains("cannot close job, expected job state [opened], but got [closed]")) {
logger.debug("failed to close job [" + jobId + "]", e);
} else {
logger.warn("failed to close job [" + jobId + "]", e);
}
}
client.performRequest("DELETE", "/_xpack/ml/anomaly_detectors/" + jobId);
}

View File

@ -5,79 +5,42 @@
*/
package org.elasticsearch.xpack.ml.integration;
import org.elasticsearch.client.Client;
import org.elasticsearch.cluster.ClusterModule;
import org.elasticsearch.cluster.ClusterState;
import org.elasticsearch.cluster.metadata.MetaData;
import org.elasticsearch.common.io.stream.NamedWriteableRegistry;
import org.elasticsearch.ElasticsearchStatusException;
import org.elasticsearch.common.settings.Settings;
import org.elasticsearch.common.util.concurrent.ThreadContext;
import org.elasticsearch.plugins.Plugin;
import org.elasticsearch.search.SearchModule;
import org.elasticsearch.test.ESIntegTestCase;
import org.elasticsearch.test.SecurityIntegTestCase;
import org.elasticsearch.xpack.XPackPlugin;
import org.elasticsearch.xpack.ml.MlPlugin;
import org.elasticsearch.xpack.ml.action.DatafeedJobsIT;
import org.elasticsearch.xpack.ml.action.CloseJobAction;
import org.elasticsearch.xpack.ml.action.GetJobsStatsAction;
import org.elasticsearch.xpack.ml.action.OpenJobAction;
import org.elasticsearch.xpack.ml.action.PutJobAction;
import org.elasticsearch.xpack.ml.action.StartDatafeedAction;
import org.elasticsearch.xpack.ml.job.config.AnalysisConfig;
import org.elasticsearch.xpack.ml.job.config.DataDescription;
import org.elasticsearch.xpack.ml.job.config.Detector;
import org.elasticsearch.xpack.ml.job.config.Job;
import org.elasticsearch.xpack.ml.job.config.JobState;
import org.elasticsearch.xpack.ml.job.metadata.MlMetadata;
import org.elasticsearch.xpack.security.authc.support.SecuredString;
import org.elasticsearch.xpack.ml.job.process.autodetect.AutodetectProcessManager;
import org.elasticsearch.xpack.persistent.PersistentActionRequest;
import org.elasticsearch.xpack.persistent.PersistentTasksInProgress;
import org.junit.After;
import org.elasticsearch.xpack.ml.support.BaseMlIntegTestCase;
import org.elasticsearch.xpack.persistent.PersistentActionResponse;
import java.io.IOException;
import java.util.ArrayList;
import java.util.Collection;
import java.util.Collections;
import java.util.List;
import java.util.Map;
import java.util.concurrent.ExecutionException;
import static org.elasticsearch.test.XContentTestUtils.convertToMap;
import static org.elasticsearch.test.XContentTestUtils.differenceBetweenMapsIgnoringArrayOrder;
import static org.elasticsearch.xpack.security.authc.support.UsernamePasswordToken.basicAuthHeaderValue;
public class TooManyJobsIT extends BaseMlIntegTestCase {
@ESIntegTestCase.ClusterScope(numDataNodes = 1)
public class TooManyJobsIT extends SecurityIntegTestCase {
@Override
protected Settings externalClusterClientSettings() {
return Settings.builder().put(super.externalClusterClientSettings()).put("transport.type", "security4")
.put(MlPlugin.ML_ENABLED.getKey(), true)
.put(ThreadContext.PREFIX + ".Authorization",
basicAuthHeaderValue("elastic", new SecuredString("changeme".toCharArray())))
.build();
public void testSingleNode() throws Exception {
verifyMaxNumberOfJobsLimit(1, randomIntBetween(1, 32));
}
@Override
protected Collection<Class<? extends Plugin>> nodePlugins() {
return Collections.singleton(XPackPlugin.class);
public void testMultipleNodes() throws Exception {
verifyMaxNumberOfJobsLimit(3, randomIntBetween(1, 32));
}
@Override
protected Collection<Class<? extends Plugin>> transportClientPlugins() {
return nodePlugins();
private void verifyMaxNumberOfJobsLimit(int numNodes, int maxNumberOfJobsPerNode) throws Exception {
// clear all nodes, so that we can set max_running_jobs setting:
internalCluster().ensureAtMostNumDataNodes(0);
logger.info("[{}] is [{}]", AutodetectProcessManager.MAX_RUNNING_JOBS_PER_NODE.getKey(), maxNumberOfJobsPerNode);
for (int i = 0; i < numNodes; i++) {
internalCluster().startNode(Settings.builder()
.put(AutodetectProcessManager.MAX_RUNNING_JOBS_PER_NODE.getKey(), maxNumberOfJobsPerNode));
}
logger.info("Started [{}] nodes", numNodes);
@After
public void clearMlMetadata() throws Exception {
DatafeedJobsIT.clearMlMetadata(client());
}
public void testCannotStartTooManyAnalyticalProcesses() throws Exception {
int maxRunningJobsPerNode = AutodetectProcessManager.MAX_RUNNING_JOBS_PER_NODE.getDefault(Settings.EMPTY);
logger.info("[{}] is [{}]", AutodetectProcessManager.MAX_RUNNING_JOBS_PER_NODE.getKey(), maxRunningJobsPerNode);
for (int i = 1; i <= (maxRunningJobsPerNode + 1); i++) {
int clusterWideMaxNumberOfJobs = numNodes * maxNumberOfJobsPerNode;
for (int i = 1; i <= (clusterWideMaxNumberOfJobs + 1); i++) {
Job.Builder job = createJob(Integer.toString(i));
PutJobAction.Request putJobRequest = new PutJobAction.Request(job.build(true, job.getId()));
PutJobAction.Response putJobResponse = client().execute(PutJobAction.INSTANCE, putJobRequest).get();
@ -85,111 +48,34 @@ public class TooManyJobsIT extends SecurityIntegTestCase {
try {
OpenJobAction.Request openJobRequest = new OpenJobAction.Request(job.getId());
OpenJobAction.Response openJobResponse = client().execute(OpenJobAction.INSTANCE, openJobRequest).get();
assertTrue(openJobResponse.isOpened());
PersistentActionResponse openJobResponse = client().execute(OpenJobAction.INSTANCE, openJobRequest).get();
assertBusy(() -> {
GetJobsStatsAction.Response statsResponse =
client().execute(GetJobsStatsAction.INSTANCE, new GetJobsStatsAction.Request(job.getId())).actionGet();
assertEquals(statsResponse.getResponse().results().get(0).getState(), JobState.OPENED);
});
logger.info("Opened {}th job", i);
} catch (Exception e) {
Throwable cause = e.getCause().getCause();
if (IllegalArgumentException.class.equals(cause.getClass()) == false) {
logger.warn("Unexpected cause", e);
}
assertEquals(IllegalArgumentException.class, cause.getClass());
assertEquals("Timeout expired while waiting for job state to change to [opened]", cause.getMessage());
} catch (ExecutionException e) {
Exception cause = (Exception) e.getCause();
assertEquals(ElasticsearchStatusException.class, cause.getClass());
assertEquals("[" + i + "] expected state [" + JobState.OPENED + "] but got [" + JobState.FAILED +"]", cause.getMessage());
logger.info("good news everybody --> reached maximum number of allowed opened jobs, after trying to open the {}th job", i);
// now manually clean things up and see if we can succeed to run one new job
clearMlMetadata();
putJobResponse = client().execute(PutJobAction.INSTANCE, putJobRequest).get();
assertTrue(putJobResponse.isAcknowledged());
OpenJobAction.Response openJobResponse = client().execute(OpenJobAction.INSTANCE, new OpenJobAction.Request(job.getId()))
.get();
assertTrue(openJobResponse.isOpened());
// close the first job and check if the latest job gets opened:
CloseJobAction.Request closeRequest = new CloseJobAction.Request("1");
CloseJobAction.Response closeResponse = client().execute(CloseJobAction.INSTANCE, closeRequest).actionGet();
assertTrue(closeResponse.isClosed());
assertBusy(() -> {
GetJobsStatsAction.Response statsResponse =
client().execute(GetJobsStatsAction.INSTANCE, new GetJobsStatsAction.Request(job.getId())).actionGet();
assertEquals(statsResponse.getResponse().results().get(0).getState(), JobState.OPENED);
});
cleanupWorkaround(numNodes);
return;
}
}
fail("shouldn't be able to add more than [" + maxRunningJobsPerNode + "] jobs");
}
private Job.Builder createJob(String id) {
DataDescription.Builder dataDescription = new DataDescription.Builder();
dataDescription.setFormat(DataDescription.DataFormat.JSON);
dataDescription.setTimeFormat(DataDescription.EPOCH_MS);
Detector.Builder d = new Detector.Builder("count", null);
AnalysisConfig.Builder analysisConfig = new AnalysisConfig.Builder(Collections.singletonList(d.build()));
Job.Builder builder = new Job.Builder();
builder.setId(id);
builder.setAnalysisConfig(analysisConfig);
builder.setDataDescription(dataDescription);
return builder;
}
@Override
protected void ensureClusterStateConsistency() throws IOException {
ensureClusterStateConsistencyWorkAround();
}
// TODO: Fix in ES. In ESIntegTestCase we should get all NamedWriteableRegistry.Entry entries from ESIntegTestCase#nodePlugins()
public static void ensureClusterStateConsistencyWorkAround() throws IOException {
if (cluster() != null && cluster().size() > 0) {
List<NamedWriteableRegistry.Entry> namedWritables = new ArrayList<>(ClusterModule.getNamedWriteables());
SearchModule searchModule = new SearchModule(Settings.EMPTY, false, Collections.emptyList());
namedWritables.addAll(searchModule.getNamedWriteables());
namedWritables.add(new NamedWriteableRegistry.Entry(MetaData.Custom.class, "ml", MlMetadata::new));
namedWritables.add(new NamedWriteableRegistry.Entry(ClusterState.Custom.class, PersistentTasksInProgress.TYPE,
PersistentTasksInProgress::new));
namedWritables.add(new NamedWriteableRegistry.Entry(PersistentActionRequest.class, StartDatafeedAction.NAME,
StartDatafeedAction.Request::new));
final NamedWriteableRegistry namedWriteableRegistry = new NamedWriteableRegistry(namedWritables);
ClusterState masterClusterState = client().admin().cluster().prepareState().all().get().getState();
byte[] masterClusterStateBytes = ClusterState.Builder.toBytes(masterClusterState);
// remove local node reference
masterClusterState = ClusterState.Builder.fromBytes(masterClusterStateBytes, null, namedWriteableRegistry);
Map<String, Object> masterStateMap = convertToMap(masterClusterState);
int masterClusterStateSize = ClusterState.Builder.toBytes(masterClusterState).length;
String masterId = masterClusterState.nodes().getMasterNodeId();
for (Client client : cluster().getClients()) {
ClusterState localClusterState = client.admin().cluster().prepareState().all().setLocal(true).get().getState();
byte[] localClusterStateBytes = ClusterState.Builder.toBytes(localClusterState);
// remove local node reference
localClusterState = ClusterState.Builder.fromBytes(localClusterStateBytes, null, namedWriteableRegistry);
final Map<String, Object> localStateMap = convertToMap(localClusterState);
final int localClusterStateSize = ClusterState.Builder.toBytes(localClusterState).length;
// Check that the non-master node has the same version of the cluster state as the master and
// that the master node matches the master (otherwise there is no requirement for the cluster state to match)
if (masterClusterState.version() == localClusterState.version() &&
masterId.equals(localClusterState.nodes().getMasterNodeId())) {
try {
assertEquals("clusterstate UUID does not match", masterClusterState.stateUUID(),
localClusterState.stateUUID());
// We cannot compare serialization bytes since serialization order of maps is not guaranteed
// but we can compare serialization sizes - they should be the same
assertEquals("clusterstate size does not match", masterClusterStateSize, localClusterStateSize);
// Compare JSON serialization
assertNull("clusterstate JSON serialization does not match",
differenceBetweenMapsIgnoringArrayOrder(masterStateMap, localStateMap));
} catch (AssertionError error) {
fail("Cluster state from master:\n" + masterClusterState.toString() + "\nLocal cluster state:\n" +
localClusterState.toString());
throw error;
}
}
}
}
cleanupWorkaround(numNodes);
fail("shouldn't be able to add more than [" + clusterWideMaxNumberOfJobs + "] jobs");
}
}

View File

@ -22,18 +22,17 @@ import org.elasticsearch.index.Index;
import org.elasticsearch.test.ESTestCase;
import org.elasticsearch.xpack.ml.action.PutJobAction;
import org.elasticsearch.xpack.ml.action.util.QueryPage;
import org.elasticsearch.xpack.ml.notifications.Auditor;
import org.elasticsearch.xpack.ml.job.config.Job;
import org.elasticsearch.xpack.ml.job.metadata.MlMetadata;
import org.elasticsearch.xpack.ml.job.persistence.AnomalyDetectorsIndex;
import org.elasticsearch.xpack.ml.job.persistence.JobProvider;
import org.elasticsearch.xpack.ml.job.persistence.JobResultsPersister;
import org.elasticsearch.xpack.ml.notifications.Auditor;
import org.junit.Before;
import java.util.Arrays;
import java.util.HashSet;
import java.util.Set;
import java.util.function.BiConsumer;
import java.util.stream.Collectors;
import static org.elasticsearch.xpack.ml.job.config.JobTests.buildJobBuilder;
@ -43,8 +42,6 @@ import static org.mockito.Matchers.anyString;
import static org.mockito.Matchers.eq;
import static org.mockito.Mockito.doAnswer;
import static org.mockito.Mockito.mock;
import static org.mockito.Mockito.times;
import static org.mockito.Mockito.verify;
import static org.mockito.Mockito.when;
public class JobManagerTests extends ESTestCase {
@ -96,21 +93,6 @@ public class JobManagerTests extends ESTestCase {
assertEquals(job, jobManager.getJobOrThrowIfUnknown(cs, "foo"));
}
public void tesGetJobAllocation() {
JobManager jobManager = createJobManager();
Job job = buildJobBuilder("foo").build();
MlMetadata mlMetadata = new MlMetadata.Builder()
.putJob(job, false)
.assignToNode("foo", "nodeId")
.build();
ClusterState cs = ClusterState.builder(new ClusterName("_name"))
.metaData(MetaData.builder().putCustom(MlMetadata.TYPE, mlMetadata)).build();
when(clusterService.state()).thenReturn(cs);
assertEquals("nodeId", jobManager.getJobAllocation("foo").getNodeId());
expectThrows(ResourceNotFoundException.class, () -> jobManager.getJobAllocation("bar"));
}
public void testGetJob_GivenJobIdIsAll() {
MlMetadata.Builder mlMetadata = new MlMetadata.Builder();
for (int i = 0; i < 3; i++) {

View File

@ -1,43 +0,0 @@
/*
* Copyright Elasticsearch B.V. and/or licensed to Elasticsearch B.V. under one
* or more contributor license agreements. Licensed under the Elastic License;
* you may not use this file except in compliance with the Elastic License.
*/
package org.elasticsearch.xpack.ml.job.metadata;
import org.elasticsearch.common.io.stream.Writeable;
import org.elasticsearch.common.xcontent.XContentParser;
import org.elasticsearch.xpack.ml.job.config.JobState;
import org.elasticsearch.xpack.ml.support.AbstractSerializingTestCase;
public class AllocationTests extends AbstractSerializingTestCase<Allocation> {
@Override
protected Allocation createTestInstance() {
String nodeId = randomBoolean() ? randomAsciiOfLength(10) : null;
String jobId = randomAsciiOfLength(10);
boolean ignoreDowntime = randomBoolean();
JobState jobState = randomFrom(JobState.values());
String stateReason = randomBoolean() ? randomAsciiOfLength(10) : null;
return new Allocation(nodeId, jobId, ignoreDowntime, jobState, stateReason);
}
@Override
protected Writeable.Reader<Allocation> instanceReader() {
return Allocation::new;
}
@Override
protected Allocation parseInstance(XContentParser parser) {
return Allocation.PARSER.apply(parser, null).build();
}
public void testUnsetIgnoreDownTime() {
Allocation allocation = new Allocation("_node_id", "_job_id", true, JobState.OPENING, null);
assertTrue(allocation.isIgnoreDowntime());
Allocation.Builder builder = new Allocation.Builder(allocation);
builder.setState(JobState.OPENED);
allocation = builder.build();
assertFalse(allocation.isIgnoreDowntime());
}
}

View File

@ -15,6 +15,7 @@ import org.elasticsearch.common.xcontent.XContentFactory;
import org.elasticsearch.common.xcontent.XContentParser;
import org.elasticsearch.common.xcontent.XContentType;
import org.elasticsearch.rest.RestStatus;
import org.elasticsearch.xpack.ml.action.OpenJobAction;
import org.elasticsearch.xpack.ml.action.StartDatafeedAction;
import org.elasticsearch.xpack.ml.datafeed.DatafeedConfig;
import org.elasticsearch.xpack.ml.datafeed.DatafeedConfigTests;
@ -24,6 +25,7 @@ import org.elasticsearch.xpack.ml.job.config.JobState;
import org.elasticsearch.xpack.ml.job.config.JobTests;
import org.elasticsearch.xpack.ml.support.AbstractSerializingTestCase;
import org.elasticsearch.xpack.persistent.PersistentTasksInProgress;
import org.elasticsearch.xpack.persistent.PersistentTasksInProgress.PersistentTaskInProgress;
import java.io.IOException;
import java.util.Collections;
@ -32,7 +34,6 @@ import static org.elasticsearch.xpack.ml.datafeed.DatafeedJobRunnerTests.createD
import static org.elasticsearch.xpack.ml.datafeed.DatafeedJobRunnerTests.createDatafeedJob;
import static org.elasticsearch.xpack.ml.job.config.JobTests.buildJobBuilder;
import static org.hamcrest.Matchers.equalTo;
import static org.hamcrest.Matchers.notNullValue;
import static org.hamcrest.Matchers.nullValue;
import static org.hamcrest.Matchers.sameInstance;
@ -58,15 +59,6 @@ public class MlMetadataTests extends AbstractSerializingTestCase<MlMetadata> {
} else {
builder.putJob(job, false);
}
if (randomBoolean()) {
builder.updateState(job.getId(), JobState.OPENING, randomBoolean() ? "first reason" : null);
if (randomBoolean()) {
builder.updateState(job.getId(), JobState.OPENED, randomBoolean() ? "second reason" : null);
if (randomBoolean()) {
builder.updateState(job.getId(), JobState.CLOSING, randomBoolean() ? "third reason" : null);
}
}
}
}
return builder.build();
}
@ -105,10 +97,8 @@ public class MlMetadataTests extends AbstractSerializingTestCase<MlMetadata> {
MlMetadata result = builder.build();
assertThat(result.getJobs().get("1"), sameInstance(job1));
assertThat(result.getAllocations().get("1").getState(), equalTo(JobState.CLOSED));
assertThat(result.getDatafeeds().get("1"), nullValue());
assertThat(result.getJobs().get("2"), sameInstance(job2));
assertThat(result.getAllocations().get("2").getState(), equalTo(JobState.CLOSED));
assertThat(result.getDatafeeds().get("2"), nullValue());
builder = new MlMetadata.Builder(result);
@ -126,25 +116,23 @@ public class MlMetadataTests extends AbstractSerializingTestCase<MlMetadata> {
}
public void testRemoveJob() {
Job job1 = buildJobBuilder("1").build();
Job.Builder jobBuilder = buildJobBuilder("1");
jobBuilder.setDeleted(true);
Job job1 = jobBuilder.build();
MlMetadata.Builder builder = new MlMetadata.Builder();
builder.putJob(job1, false);
MlMetadata result = builder.build();
assertThat(result.getJobs().get("1"), sameInstance(job1));
assertThat(result.getAllocations().get("1").getState(), equalTo(JobState.CLOSED));
assertThat(result.getDatafeeds().get("1"), nullValue());
builder = new MlMetadata.Builder(result);
builder.updateState("1", JobState.DELETING, null);
assertThat(result.getJobs().get("1"), sameInstance(job1));
assertThat(result.getAllocations().get("1").getState(), equalTo(JobState.CLOSED));
assertThat(result.getDatafeeds().get("1"), nullValue());
builder.deleteJob("1");
builder.deleteJob("1", new PersistentTasksInProgress(0L, Collections.emptyMap()));
result = builder.build();
assertThat(result.getJobs().get("1"), nullValue());
assertThat(result.getAllocations().get("1"), nullValue());
assertThat(result.getDatafeeds().get("1"), nullValue());
}
@ -152,16 +140,19 @@ public class MlMetadataTests extends AbstractSerializingTestCase<MlMetadata> {
Job job1 = buildJobBuilder("1").build();
MlMetadata.Builder builder1 = new MlMetadata.Builder();
builder1.putJob(job1, false);
builder1.updateState("1", JobState.OPENING, null);
builder1.updateState("1", JobState.OPENED, null);
MlMetadata result = builder1.build();
assertThat(result.getJobs().get("1"), sameInstance(job1));
assertThat(result.getAllocations().get("1").getState(), equalTo(JobState.OPENED));
assertThat(result.getDatafeeds().get("1"), nullValue());
PersistentTaskInProgress<OpenJobAction.Request> task =
new PersistentTaskInProgress<>(
new PersistentTaskInProgress<>(0L, OpenJobAction.NAME, new OpenJobAction.Request("1"), null),
JobState.CLOSED
);
MlMetadata.Builder builder2 = new MlMetadata.Builder(result);
ElasticsearchStatusException e = expectThrows(ElasticsearchStatusException.class, () -> builder2.deleteJob("1"));
ElasticsearchStatusException e = expectThrows(ElasticsearchStatusException.class,
() -> builder2.deleteJob("1", new PersistentTasksInProgress(0L, Collections.singletonMap(0L, task))));
assertThat(e.status(), equalTo(RestStatus.CONFLICT));
}
@ -172,7 +163,8 @@ public class MlMetadataTests extends AbstractSerializingTestCase<MlMetadata> {
builder.putJob(job1, false);
builder.putDatafeed(datafeedConfig1);
ElasticsearchStatusException e = expectThrows(ElasticsearchStatusException.class, () -> builder.deleteJob(job1.getId()));
ElasticsearchStatusException e = expectThrows(ElasticsearchStatusException.class,
() -> builder.deleteJob(job1.getId(), new PersistentTasksInProgress(0L, Collections.emptyMap())));
assertThat(e.status(), equalTo(RestStatus.CONFLICT));
String expectedMsg = "Cannot delete job [" + job1.getId() + "] while datafeed [" + datafeedConfig1.getId() + "] refers to it";
assertThat(e.getMessage(), equalTo(expectedMsg));
@ -180,7 +172,8 @@ public class MlMetadataTests extends AbstractSerializingTestCase<MlMetadata> {
public void testRemoveJob_failBecauseJobDoesNotExist() {
MlMetadata.Builder builder1 = new MlMetadata.Builder();
expectThrows(ResourceNotFoundException.class, () -> builder1.deleteJob("1"));
expectThrows(ResourceNotFoundException.class,
() -> builder1.deleteJob("1", new PersistentTasksInProgress(0L, Collections.emptyMap())));
}
public void testCrudDatafeed() {
@ -192,14 +185,12 @@ public class MlMetadataTests extends AbstractSerializingTestCase<MlMetadata> {
MlMetadata result = builder.build();
assertThat(result.getJobs().get("foo"), sameInstance(job1));
assertThat(result.getAllocations().get("foo").getState(), equalTo(JobState.CLOSED));
assertThat(result.getDatafeeds().get("datafeed1"), sameInstance(datafeedConfig1));
builder = new MlMetadata.Builder(result);
builder.removeDatafeed("datafeed1", new PersistentTasksInProgress(0, Collections.emptyMap()));
result = builder.build();
assertThat(result.getJobs().get("foo"), sameInstance(job1));
assertThat(result.getAllocations().get("foo").getState(), equalTo(JobState.CLOSED));
assertThat(result.getDatafeeds().get("datafeed1"), nullValue());
}
@ -251,17 +242,14 @@ public class MlMetadataTests extends AbstractSerializingTestCase<MlMetadata> {
MlMetadata.Builder builder = new MlMetadata.Builder();
builder.putJob(job1, false);
builder.putDatafeed(datafeedConfig1);
builder.updateState("foo", JobState.OPENING, null);
builder.updateState("foo", JobState.OPENED, null);
MlMetadata result = builder.build();
assertThat(result.getJobs().get("foo"), sameInstance(job1));
assertThat(result.getAllocations().get("foo").getState(), equalTo(JobState.OPENED));
assertThat(result.getDatafeeds().get("datafeed1"), sameInstance(datafeedConfig1));
StartDatafeedAction.Request request = new StartDatafeedAction.Request("datafeed1", 0L);
PersistentTasksInProgress.PersistentTaskInProgress<StartDatafeedAction.Request> taskInProgress =
new PersistentTasksInProgress.PersistentTaskInProgress<>(0, StartDatafeedAction.NAME, request, null);
PersistentTaskInProgress<StartDatafeedAction.Request> taskInProgress =
new PersistentTaskInProgress<>(0, StartDatafeedAction.NAME, request, null);
PersistentTasksInProgress tasksInProgress =
new PersistentTasksInProgress(1, Collections.singletonMap(taskInProgress.getId(), taskInProgress));
@ -271,27 +259,4 @@ public class MlMetadataTests extends AbstractSerializingTestCase<MlMetadata> {
assertThat(e.status(), equalTo(RestStatus.CONFLICT));
}
public void testUpdateAllocation_setFinishedTime() {
MlMetadata.Builder builder = new MlMetadata.Builder();
builder.putJob(buildJobBuilder("my_job_id").build(), false);
builder.updateState("my_job_id", JobState.OPENING, null);
builder.updateState("my_job_id", JobState.OPENED, null);
MlMetadata mlMetadata = builder.build();
assertThat(mlMetadata.getJobs().get("my_job_id").getFinishedTime(), nullValue());
builder.updateState("my_job_id", JobState.CLOSED, null);
mlMetadata = builder.build();
assertThat(mlMetadata.getJobs().get("my_job_id").getFinishedTime(), notNullValue());
}
public void testUpdateState_failBecauseJobDoesNotExist() {
MlMetadata.Builder builder = new MlMetadata.Builder();
expectThrows(ResourceNotFoundException.class, () -> builder.updateState("missing-job", JobState.CLOSED, "for testting"));
}
public void testSetIgnoreDowntime_failBecauseJobDoesNotExist() {
MlMetadata.Builder builder = new MlMetadata.Builder();
expectThrows(ResourceNotFoundException.class, () -> builder.setIgnoreDowntime("missing-job"));
}
}

View File

@ -5,7 +5,6 @@
*/
package org.elasticsearch.xpack.ml.job.persistence;
import org.apache.lucene.util.LuceneTestCase;
import org.elasticsearch.ResourceNotFoundException;
import org.elasticsearch.action.ActionFuture;
import org.elasticsearch.action.ActionListener;
@ -18,6 +17,7 @@ import org.elasticsearch.action.search.SearchResponse;
import org.elasticsearch.client.Client;
import org.elasticsearch.common.settings.Settings;
import org.elasticsearch.common.text.Text;
import org.elasticsearch.common.unit.TimeValue;
import org.elasticsearch.common.xcontent.XContentFactory;
import org.elasticsearch.index.query.QueryBuilder;
import org.elasticsearch.search.SearchHit;
@ -27,14 +27,12 @@ import org.elasticsearch.test.ESTestCase;
import org.elasticsearch.xpack.ml.action.DeleteJobAction;
import org.elasticsearch.xpack.ml.action.util.QueryPage;
import org.elasticsearch.xpack.ml.job.config.AnalysisLimits;
import org.elasticsearch.xpack.ml.job.config.Job;
import org.elasticsearch.xpack.ml.job.persistence.InfluencersQueryBuilder.InfluencersQuery;
import org.elasticsearch.xpack.ml.job.process.autodetect.state.CategorizerState;
import org.elasticsearch.xpack.ml.job.process.autodetect.state.DataCounts;
import org.elasticsearch.xpack.ml.job.config.Job;
import org.elasticsearch.xpack.ml.job.process.autodetect.state.ModelSnapshot;
import org.elasticsearch.xpack.ml.job.process.autodetect.state.ModelState;
import org.elasticsearch.xpack.ml.notifications.AuditActivity;
import org.elasticsearch.xpack.ml.notifications.AuditMessage;
import org.elasticsearch.xpack.ml.job.persistence.InfluencersQueryBuilder.InfluencersQuery;
import org.elasticsearch.xpack.ml.job.process.autodetect.state.Quantiles;
import org.elasticsearch.xpack.ml.job.results.AnomalyRecord;
import org.elasticsearch.xpack.ml.job.results.Bucket;
@ -42,6 +40,8 @@ import org.elasticsearch.xpack.ml.job.results.CategoryDefinition;
import org.elasticsearch.xpack.ml.job.results.Influencer;
import org.elasticsearch.xpack.ml.job.results.PerPartitionMaxProbabilities;
import org.elasticsearch.xpack.ml.job.results.Result;
import org.elasticsearch.xpack.ml.notifications.AuditActivity;
import org.elasticsearch.xpack.ml.notifications.AuditMessage;
import org.elasticsearch.xpack.ml.notifications.Auditor;
import org.mockito.ArgumentCaptor;
@ -1150,7 +1150,7 @@ public class JobProviderTests extends ESTestCase {
}
private JobProvider createProvider(Client client) {
return new JobProvider(client, 0);
return new JobProvider(client, 0, TimeValue.timeValueSeconds(1));
}
private static GetResponse createGetResponse(boolean exists, Map<String, Object> source) throws IOException {

View File

@ -146,7 +146,8 @@ public class AutodetectCommunicatorTests extends ESTestCase {
return null;
}).when(executorService).execute(any(Runnable.class));
DataCountsReporter dataCountsReporter = mock(DataCountsReporter.class);
return new AutodetectCommunicator(createJobDetails(), autodetectProcess, dataCountsReporter, autoDetectResultProcessor, e -> {});
return new AutodetectCommunicator(0L, createJobDetails(), autodetectProcess, dataCountsReporter, autoDetectResultProcessor,
e -> {});
}
public void testWriteToJobInUse() throws IOException {

View File

@ -6,15 +6,14 @@
package org.elasticsearch.xpack.ml.job.process.autodetect;
import org.elasticsearch.ElasticsearchException;
import org.elasticsearch.ElasticsearchStatusException;
import org.elasticsearch.client.Client;
import org.elasticsearch.common.CheckedConsumer;
import org.elasticsearch.common.settings.Settings;
import org.elasticsearch.common.util.concurrent.EsExecutors;
import org.elasticsearch.common.util.concurrent.EsRejectedExecutionException;
import org.elasticsearch.test.ESTestCase;
import org.elasticsearch.threadpool.ThreadPool;
import org.elasticsearch.xpack.ml.MlPlugin;
import org.elasticsearch.xpack.ml.action.UpdateJobStateAction;
import org.elasticsearch.xpack.ml.action.util.QueryPage;
import org.elasticsearch.xpack.ml.job.JobManager;
import org.elasticsearch.xpack.ml.job.config.AnalysisConfig;
@ -25,7 +24,6 @@ import org.elasticsearch.xpack.ml.job.config.Job;
import org.elasticsearch.xpack.ml.job.config.JobState;
import org.elasticsearch.xpack.ml.job.config.MlFilter;
import org.elasticsearch.xpack.ml.job.config.ModelDebugConfig;
import org.elasticsearch.xpack.ml.job.metadata.Allocation;
import org.elasticsearch.xpack.ml.job.persistence.JobDataCountsPersister;
import org.elasticsearch.xpack.ml.job.persistence.JobProvider;
import org.elasticsearch.xpack.ml.job.persistence.JobResultsPersister;
@ -36,6 +34,7 @@ import org.elasticsearch.xpack.ml.job.process.autodetect.state.DataCounts;
import org.elasticsearch.xpack.ml.job.process.autodetect.state.ModelSnapshot;
import org.elasticsearch.xpack.ml.job.process.autodetect.state.Quantiles;
import org.elasticsearch.xpack.ml.job.process.normalizer.NormalizerFactory;
import org.elasticsearch.xpack.persistent.UpdatePersistentTaskStatusAction;
import org.junit.Before;
import org.mockito.Mockito;
@ -62,6 +61,7 @@ import static org.hamcrest.core.IsEqual.equalTo;
import static org.mockito.Matchers.any;
import static org.mockito.Matchers.anyBoolean;
import static org.mockito.Matchers.anyInt;
import static org.mockito.Matchers.anyLong;
import static org.mockito.Matchers.anyString;
import static org.mockito.Matchers.eq;
import static org.mockito.Mockito.mock;
@ -128,11 +128,11 @@ public class AutodetectProcessManagerTests extends ESTestCase {
when(jobManager.getJobOrThrowIfUnknown("foo")).thenReturn(createJobDetails("foo"));
AutodetectProcessManager manager = createManager(communicator, client);
manager.openJob("foo", false, e -> {});
manager.openJob("foo", 1L, false, e -> {});
assertEquals(1, manager.numberOfOpenJobs());
assertTrue(manager.jobHasActiveAutodetectProcess("foo"));
UpdateJobStateAction.Request expectedRequest = new UpdateJobStateAction.Request("foo", JobState.OPENED);
verify(client).execute(eq(UpdateJobStateAction.INSTANCE), eq(expectedRequest), any());
UpdatePersistentTaskStatusAction.Request expectedRequest = new UpdatePersistentTaskStatusAction.Request(1L, JobState.OPENED);
verify(client).execute(eq(UpdatePersistentTaskStatusAction.INSTANCE), eq(expectedRequest), any());
}
public void testOpenJob_exceedMaxNumJobs() {
@ -173,18 +173,26 @@ public class AutodetectProcessManagerTests extends ESTestCase {
consumer.accept(dataCounts, modelSnapshot, quantiles, filters);
return null;
}).when(manager).gatherRequiredInformation(any(), any(), any());
doAnswer(invocationOnMock -> {
@SuppressWarnings("unchecked")
CheckedConsumer<Exception, IOException> consumer = (CheckedConsumer<Exception, IOException>) invocationOnMock.getArguments()[2];
consumer.accept(null);
return null;
}).when(manager).setJobState(anyLong(), eq(JobState.FAILED), any());
manager.openJob("foo", false, e -> {});
manager.openJob("bar", false, e -> {});
manager.openJob("baz", false, e -> {});
manager.openJob("foo", 1L, false, e -> {});
manager.openJob("bar", 2L, false, e -> {});
manager.openJob("baz", 3L, false, e -> {});
assertEquals(3, manager.numberOfOpenJobs());
Exception e = expectThrows(ElasticsearchStatusException.class, () -> manager.openJob("foobar", false, e1 -> {}));
Exception[] holder = new Exception[1];
manager.openJob("foobar", 4L, false, e -> holder[0] = e);
Exception e = holder[0];
assertEquals("max running job capacity [3] reached", e.getMessage());
manager.closeJob("baz");
manager.closeJob("baz", null);
assertEquals(2, manager.numberOfOpenJobs());
manager.openJob("foobar", false, e1 -> {});
manager.openJob("foobar", 4L, false, e1 -> {});
assertEquals(3, manager.numberOfOpenJobs());
}
@ -194,7 +202,7 @@ public class AutodetectProcessManagerTests extends ESTestCase {
assertEquals(0, manager.numberOfOpenJobs());
DataLoadParams params = new DataLoadParams(TimeRange.builder().build(), Optional.empty());
manager.openJob("foo", false, e -> {});
manager.openJob("foo", 1L, false, e -> {});
manager.processData("foo", createInputStream(""), params);
assertEquals(1, manager.numberOfOpenJobs());
}
@ -207,7 +215,7 @@ public class AutodetectProcessManagerTests extends ESTestCase {
InputStream inputStream = createInputStream("");
doThrow(new IOException("blah")).when(communicator).writeToJob(inputStream, params);
manager.openJob("foo", false, e -> {});
manager.openJob("foo", 1L, false, e -> {});
ESTestCase.expectThrows(ElasticsearchException.class,
() -> manager.processData("foo", inputStream, params));
}
@ -217,12 +225,12 @@ public class AutodetectProcessManagerTests extends ESTestCase {
AutodetectProcessManager manager = createManager(communicator);
assertEquals(0, manager.numberOfOpenJobs());
manager.openJob("foo", false, e -> {});
manager.openJob("foo", 1L, false, e -> {});
manager.processData("foo", createInputStream(""), mock(DataLoadParams.class));
// job is created
assertEquals(1, manager.numberOfOpenJobs());
manager.closeJob("foo");
manager.closeJob("foo", null);
assertEquals(0, manager.numberOfOpenJobs());
}
@ -232,7 +240,7 @@ public class AutodetectProcessManagerTests extends ESTestCase {
DataLoadParams params = new DataLoadParams(TimeRange.builder().startTime("1000").endTime("2000").build(), Optional.empty());
InputStream inputStream = createInputStream("");
manager.openJob("foo", false, e -> {});
manager.openJob("foo", 1L, false, e -> {});
manager.processData("foo", inputStream, params);
verify(communicator).writeToJob(inputStream, params);
}
@ -242,7 +250,7 @@ public class AutodetectProcessManagerTests extends ESTestCase {
AutodetectProcessManager manager = createManager(communicator);
InputStream inputStream = createInputStream("");
manager.openJob("foo", false, e -> {});
manager.openJob("foo", 1L, false, e -> {});
manager.processData("foo", inputStream, mock(DataLoadParams.class));
InterimResultsParams params = InterimResultsParams.builder().build();
@ -283,7 +291,7 @@ public class AutodetectProcessManagerTests extends ESTestCase {
AutodetectProcessManager manager = createManager(communicator);
assertFalse(manager.jobHasActiveAutodetectProcess("foo"));
manager.openJob("foo", false, e -> {});
manager.openJob("foo", 1L, false, e -> {});
manager.processData("foo", createInputStream(""), mock(DataLoadParams.class));
assertTrue(manager.jobHasActiveAutodetectProcess("foo"));
@ -297,7 +305,7 @@ public class AutodetectProcessManagerTests extends ESTestCase {
givenAllocationWithState(JobState.OPENED);
InputStream inputStream = createInputStream("");
manager.openJob("foo", false, e -> {});
manager.openJob("foo", 1L, false, e -> {});
DataCounts dataCounts = manager.processData("foo", inputStream, mock(DataLoadParams.class));
assertThat(dataCounts, equalTo(new DataCounts("foo")));
@ -325,14 +333,12 @@ public class AutodetectProcessManagerTests extends ESTestCase {
jobResultsPersister, jobDataCountsPersister, autodetectProcessFactory, normalizerFactory);
expectThrows(EsRejectedExecutionException.class,
() -> manager.create("my_id", dataCounts, modelSnapshot, quantiles, filters, false, e -> {}));
() -> manager.create("my_id", 1L, dataCounts, modelSnapshot, quantiles, filters, false, e -> {}));
verify(autodetectProcess, times(1)).close();
}
private void givenAllocationWithState(JobState state) {
Allocation.Builder allocation = new Allocation.Builder();
allocation.setState(state);
when(jobManager.getJobAllocation("foo")).thenReturn(allocation.build());
when(jobManager.getJobState("foo")).thenReturn(state);
}
private AutodetectProcessManager createManager(AutodetectCommunicator communicator) {
@ -347,13 +353,13 @@ public class AutodetectProcessManagerTests extends ESTestCase {
jobResultsPersister, jobDataCountsPersister, autodetectProcessFactory, normalizerFactory);
manager = spy(manager);
doReturn(communicator).when(manager)
.create(any(), eq(dataCounts), eq(modelSnapshot), eq(quantiles), eq(filters), anyBoolean(), any());
.create(any(), anyLong(), eq(dataCounts), eq(modelSnapshot), eq(quantiles), eq(filters), anyBoolean(), any());
return manager;
}
private AutodetectProcessManager createManagerAndCallProcessData(AutodetectCommunicator communicator, String jobId) {
AutodetectProcessManager manager = createManager(communicator);
manager.openJob(jobId, false, e -> {});
manager.openJob(jobId, 1L, false, e -> {});
manager.processData(jobId, createInputStream(""), mock(DataLoadParams.class));
return manager;
}

View File

@ -0,0 +1,135 @@
/*
* Copyright Elasticsearch B.V. and/or licensed to Elasticsearch B.V. under one
* or more contributor license agreements. Licensed under the Elastic License;
* you may not use this file except in compliance with the Elastic License.
*/
package org.elasticsearch.xpack.ml.support;
import org.elasticsearch.ExceptionsHelper;
import org.elasticsearch.client.Client;
import org.elasticsearch.cluster.metadata.MetaData;
import org.elasticsearch.common.settings.Settings;
import org.elasticsearch.rest.RestStatus;
import org.elasticsearch.test.ESIntegTestCase;
import org.elasticsearch.test.SecurityIntegTestCase;
import org.elasticsearch.xpack.ml.MlPlugin;
import org.elasticsearch.xpack.ml.action.CloseJobAction;
import org.elasticsearch.xpack.ml.action.DeleteDatafeedAction;
import org.elasticsearch.xpack.ml.action.DeleteJobAction;
import org.elasticsearch.xpack.ml.action.GetDatafeedsStatsAction;
import org.elasticsearch.xpack.ml.action.StopDatafeedAction;
import org.elasticsearch.xpack.ml.datafeed.DatafeedConfig;
import org.elasticsearch.xpack.ml.datafeed.DatafeedState;
import org.elasticsearch.xpack.ml.job.config.AnalysisConfig;
import org.elasticsearch.xpack.ml.job.config.DataDescription;
import org.elasticsearch.xpack.ml.job.config.Detector;
import org.elasticsearch.xpack.ml.job.config.Job;
import org.elasticsearch.xpack.ml.job.metadata.MlMetadata;
import org.elasticsearch.xpack.persistent.RemovePersistentTaskAction;
import java.util.Collections;
import java.util.Map;
import java.util.concurrent.ExecutionException;
import static org.hamcrest.Matchers.equalTo;
/**
* A base class for testing datafeed and job lifecycle specifics.
*
* Note for other type of integration tests you should use the external test cluster created by the Gradle integTest task.
* For example tests extending this base class test with the non native autodetect process.
*/
@ESIntegTestCase.ClusterScope(scope = ESIntegTestCase.Scope.TEST, numDataNodes = 0, numClientNodes = 0,
transportClientRatio = 0, supportsDedicatedMasters = false)
public abstract class BaseMlIntegTestCase extends SecurityIntegTestCase {
@Override
protected boolean ignoreExternalCluster() {
return true;
}
@Override
protected Settings nodeSettings(int nodeOrdinal) {
Settings.Builder settings = Settings.builder().put(super.nodeSettings(nodeOrdinal));
settings.put(MlPlugin.USE_NATIVE_PROCESS_OPTION.getKey(), false);
settings.put(MlPlugin.ML_ENABLED.getKey(), true);
return settings.build();
}
protected Job.Builder createJob(String id) {
DataDescription.Builder dataDescription = new DataDescription.Builder();
dataDescription.setFormat(DataDescription.DataFormat.JSON);
dataDescription.setTimeFormat(DataDescription.EPOCH_MS);
Detector.Builder d = new Detector.Builder("count", null);
AnalysisConfig.Builder analysisConfig = new AnalysisConfig.Builder(Collections.singletonList(d.build()));
Job.Builder builder = new Job.Builder();
builder.setId(id);
builder.setAnalysisConfig(analysisConfig);
builder.setDataDescription(dataDescription);
return builder;
}
// Due to the fact that ml plugin creates the state, notifications and meta indices automatically
// when the test framework removes all indices then ml plugin adds them back. Causing validation to fail
// we should move to templates instead as that will fix the test problem
protected void cleanupWorkaround(int numNodes) throws Exception {
deleteAllDatafeeds(client());
deleteAllJobs(client());
for (int i = 0; i < numNodes; i++) {
internalCluster().stopRandomDataNode();
}
internalCluster().startNode(Settings.builder().put(MlPlugin.ML_ENABLED.getKey(), false));
}
private void deleteAllDatafeeds(Client client) throws Exception {
MetaData metaData = client.admin().cluster().prepareState().get().getState().getMetaData();
MlMetadata mlMetadata = metaData.custom(MlMetadata.TYPE);
for (DatafeedConfig datafeed : mlMetadata.getDatafeeds().values()) {
String datafeedId = datafeed.getId();
try {
RemovePersistentTaskAction.Response stopResponse =
client.execute(StopDatafeedAction.INSTANCE, new StopDatafeedAction.Request(datafeedId)).get();
assertTrue(stopResponse.isAcknowledged());
} catch (ExecutionException e) {
// CONFLICT is ok, as it means the datafeed has already stopped, which isn't an issue at all.
if (RestStatus.CONFLICT != ExceptionsHelper.status(e.getCause())) {
throw new RuntimeException(e);
}
}
assertBusy(() -> {
try {
GetDatafeedsStatsAction.Request request = new GetDatafeedsStatsAction.Request(datafeedId);
GetDatafeedsStatsAction.Response r = client.execute(GetDatafeedsStatsAction.INSTANCE, request).get();
assertThat(r.getResponse().results().get(0).getDatafeedState(), equalTo(DatafeedState.STOPPED));
} catch (InterruptedException | ExecutionException e) {
throw new RuntimeException(e);
}
});
DeleteDatafeedAction.Response deleteResponse =
client.execute(DeleteDatafeedAction.INSTANCE, new DeleteDatafeedAction.Request(datafeedId)).get();
assertTrue(deleteResponse.isAcknowledged());
}
}
private void deleteAllJobs(Client client) throws Exception {
MetaData metaData = client.admin().cluster().prepareState().get().getState().getMetaData();
MlMetadata mlMetadata = metaData.custom(MlMetadata.TYPE);
for (Map.Entry<String, Job> entry : mlMetadata.getJobs().entrySet()) {
String jobId = entry.getKey();
try {
CloseJobAction.Response response =
client.execute(CloseJobAction.INSTANCE, new CloseJobAction.Request(jobId)).get();
assertTrue(response.isClosed());
} catch (Exception e) {
logger.warn("Job [" + jobId + "] couldn't be closed", e);
}
DeleteJobAction.Response response =
client.execute(DeleteJobAction.INSTANCE, new DeleteJobAction.Request(jobId)).get();
assertTrue(response.isAcknowledged());
}
}
}

View File

@ -16,14 +16,14 @@ import org.elasticsearch.cluster.service.ClusterService;
import org.elasticsearch.common.UUIDs;
import org.elasticsearch.common.settings.ClusterSettings;
import org.elasticsearch.common.settings.Settings;
import org.elasticsearch.xpack.persistent.PersistentTasksInProgress.PersistentTaskInProgress;
import org.elasticsearch.xpack.persistent.CompletionPersistentTaskAction.Response;
import org.elasticsearch.xpack.persistent.TestPersistentActionPlugin.TestRequest;
import org.elasticsearch.tasks.Task;
import org.elasticsearch.tasks.TaskManager;
import org.elasticsearch.test.ESTestCase;
import org.elasticsearch.threadpool.ThreadPool;
import org.elasticsearch.transport.TransportResponse.Empty;
import org.elasticsearch.xpack.persistent.CompletionPersistentTaskAction.Response;
import org.elasticsearch.xpack.persistent.PersistentTasksInProgress.PersistentTaskInProgress;
import org.elasticsearch.xpack.persistent.TestPersistentActionPlugin.TestRequest;
import java.io.IOException;
import java.util.ArrayList;
@ -152,7 +152,7 @@ public class PersistentActionCoordinatorTests extends ESTestCase {
ClusterService clusterService = createClusterService();
AtomicLong capturedTaskId = new AtomicLong();
AtomicReference<ActionListener<CancelTasksResponse>> capturedListener = new AtomicReference<>();
PersistentActionService persistentActionService = new PersistentActionService(Settings.EMPTY, null, null) {
PersistentActionService persistentActionService = new PersistentActionService(Settings.EMPTY, null, null, null) {
@Override
public void sendCancellation(long taskId, ActionListener<CancelTasksResponse> listener) {
capturedTaskId.set(taskId);
@ -224,7 +224,8 @@ public class PersistentActionCoordinatorTests extends ESTestCase {
AtomicLong capturedTaskId = new AtomicLong(-1L);
AtomicReference<Exception> capturedException = new AtomicReference<>();
AtomicReference<ActionListener<Response>> capturedListener = new AtomicReference<>();
PersistentActionService persistentActionService = new PersistentActionService(Settings.EMPTY, clusterService, null) {
PersistentActionService persistentActionService =
new PersistentActionService(Settings.EMPTY, mock(ThreadPool.class), clusterService, null) {
@Override
public void sendCompletionNotification(long taskId, Exception failure, ActionListener<Response> listener) {
capturedTaskId.set(taskId);

View File

@ -42,7 +42,6 @@ import org.elasticsearch.tasks.Task;
import org.elasticsearch.tasks.TaskCancelledException;
import org.elasticsearch.tasks.TaskId;
import org.elasticsearch.threadpool.ThreadPool;
import org.elasticsearch.transport.TransportResponse;
import org.elasticsearch.transport.TransportResponse.Empty;
import org.elasticsearch.transport.TransportService;
import org.elasticsearch.watcher.ResourceWatcherService;
@ -57,7 +56,6 @@ import java.util.Objects;
import java.util.concurrent.CountDownLatch;
import java.util.concurrent.TimeUnit;
import java.util.concurrent.atomic.AtomicInteger;
import java.util.concurrent.atomic.AtomicLong;
import static java.util.Objects.requireNonNull;
import static org.elasticsearch.test.ESTestCase.awaitBusy;
@ -87,7 +85,7 @@ public class TestPersistentActionPlugin extends Plugin implements ActionPlugin {
ResourceWatcherService resourceWatcherService, ScriptService scriptService,
NamedXContentRegistry xContentRegistry) {
PersistentActionService persistentActionService = new PersistentActionService(Settings.EMPTY, clusterService, client);
PersistentActionService persistentActionService = new PersistentActionService(Settings.EMPTY, threadPool, clusterService, client);
PersistentActionRegistry persistentActionRegistry = new PersistentActionRegistry(Settings.EMPTY);
return Arrays.asList(
persistentActionService,
@ -100,7 +98,7 @@ public class TestPersistentActionPlugin extends Plugin implements ActionPlugin {
public List<NamedWriteableRegistry.Entry> getNamedWriteables() {
return Arrays.asList(
new NamedWriteableRegistry.Entry(PersistentActionRequest.class, TestPersistentAction.NAME, TestRequest::new),
new NamedWriteableRegistry.Entry(PersistentActionCoordinator.Status.class,
new NamedWriteableRegistry.Entry(Task.Status.class,
PersistentActionCoordinator.Status.NAME, PersistentActionCoordinator.Status::new),
new NamedWriteableRegistry.Entry(ClusterState.Custom.class, PersistentTasksInProgress.TYPE, PersistentTasksInProgress::new),
new NamedWriteableRegistry.Entry(NamedDiff.class, PersistentTasksInProgress.TYPE, PersistentTasksInProgress::readDiffFrom),

View File

@ -26,7 +26,7 @@ public class XPackRestIT extends XPackRestTestCase {
@After
public void clearMlState() throws IOException {
new MlRestTestStateCleaner(client(), this).clearMlMetadata();
new MlRestTestStateCleaner(logger, client(), this).clearMlMetadata();
}
public XPackRestIT(ClientYamlTestCandidate testCandidate) {

View File

@ -106,7 +106,6 @@ cluster:admin/ingest/pipeline/get
cluster:admin/ingest/pipeline/put
cluster:admin/ingest/pipeline/simulate
cluster:admin/ml/filters/get
cluster:admin/ml/anomaly_detectors/internal_open
cluster:admin/ml/anomaly_detectors/results/categories/get
cluster:admin/ml/anomaly_detectors/stats/get
cluster:admin/ml/anomaly_detectors/results/buckets/get
@ -133,7 +132,6 @@ cluster:admin/ml/datafeeds/stats/get
cluster:admin/ml/datafeeds/stop
cluster:admin/ml/datafeeds/start
cluster:admin/ml/anomaly_detectors/open
cluster:admin/ml/anomaly_detectors/state/update
cluster:admin/ml/job/update
indices:data/write/delete/mlbyquery
cluster:admin/ml/job/update/process

View File

@ -10,7 +10,7 @@
"required": true,
"description": "The name of the job to close"
},
"close_timeout": {
"timeout": {
"type": "time",
"description": "Controls the time to wait until a job has closed. Default to 30 minutes"
}

View File

@ -14,7 +14,7 @@
"type": "boolean",
"description": "Controls if gaps in data are treated as anomalous or as a maintenance window after a job re-start"
},
"open_timeout": {
"timeout": {
"type": "time",
"description": "Controls the time to wait until a job has opened. Default to 30 minutes"
}

View File

@ -88,6 +88,11 @@ setup:
{"airline":"AAL","responsetime":"132.2046","time":"1403481600"}
{"airline":"JZA","responsetime":"990.4628","time":"1403481600"}
- do:
xpack.ml.flush_job:
job_id: job-stats-test
- match: { flushed: true }
- do:
xpack.ml.close_job:
job_id: job-stats-test