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:
parent
af2486b834
commit
c6763489d5
|
@ -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),
|
||||
|
|
|
@ -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();
|
||||
}
|
||||
}
|
||||
|
||||
|
|
|
@ -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());
|
||||
|
||||
|
|
|
@ -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();
|
||||
|
|
|
@ -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");
|
||||
}
|
||||
|
||||
}
|
||||
}
|
|
@ -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);
|
||||
}
|
||||
}
|
||||
}
|
||||
|
|
|
@ -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);
|
||||
}
|
||||
});
|
||||
}
|
||||
|
||||
}
|
||||
|
|
|
@ -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));
|
||||
}
|
||||
|
||||
|
|
|
@ -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);
|
||||
}
|
||||
|
|
|
@ -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);
|
||||
}
|
||||
|
||||
|
|
|
@ -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);
|
||||
}
|
||||
}
|
||||
}
|
||||
|
|
|
@ -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);
|
||||
}
|
||||
}
|
||||
}
|
|
@ -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) {
|
||||
|
|
|
@ -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);
|
||||
|
|
|
@ -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.
|
||||
|
|
|
@ -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);
|
||||
}
|
||||
}
|
||||
}
|
||||
|
|
|
@ -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}
|
||||
*/
|
||||
|
|
|
@ -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);
|
||||
}
|
||||
|
||||
}
|
||||
}
|
|
@ -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;
|
||||
}
|
||||
}
|
||||
|
||||
|
|
|
@ -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);
|
||||
|
|
|
@ -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)) {
|
||||
|
|
|
@ -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) {
|
||||
|
|
|
@ -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
|
||||
|
|
|
@ -173,7 +173,6 @@ public class AutoDetectResultProcessor {
|
|||
completionLatch.await();
|
||||
} catch (InterruptedException e) {
|
||||
Thread.currentThread().interrupt();
|
||||
throw new RuntimeException(e);
|
||||
}
|
||||
}
|
||||
|
||||
|
|
|
@ -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;
|
||||
|
|
|
@ -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));
|
||||
}
|
||||
|
|
|
@ -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);
|
||||
}
|
||||
});
|
||||
};
|
||||
}
|
||||
}
|
||||
|
|
|
@ -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;
|
||||
}
|
||||
|
||||
}
|
||||
|
||||
|
|
|
@ -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);
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
|
|
|
@ -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);
|
||||
}
|
||||
|
|
|
@ -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;
|
||||
|
|
|
@ -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) {
|
||||
|
|
|
@ -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);
|
||||
}
|
||||
}
|
|
@ -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());
|
||||
}
|
||||
|
||||
}
|
|
@ -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();
|
||||
}
|
||||
}
|
||||
|
|
|
@ -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);
|
||||
}
|
||||
|
|
|
@ -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);
|
||||
}
|
||||
}
|
|
@ -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();
|
||||
}
|
||||
}
|
|
@ -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());
|
||||
}
|
||||
|
||||
}
|
|
@ -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]"));
|
||||
}
|
||||
|
||||
|
|
|
@ -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();
|
||||
}
|
||||
}
|
|
@ -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))
|
||||
|
|
|
@ -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 {
|
||||
|
|
|
@ -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);
|
||||
}
|
||||
|
||||
}
|
|
@ -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 {
|
||||
|
|
|
@ -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();
|
||||
}
|
||||
}
|
||||
|
|
|
@ -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);
|
||||
}
|
||||
|
|
|
@ -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");
|
||||
}
|
||||
|
||||
}
|
||||
|
|
|
@ -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++) {
|
||||
|
|
|
@ -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());
|
||||
}
|
||||
}
|
|
@ -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"));
|
||||
}
|
||||
}
|
||||
|
|
|
@ -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 {
|
||||
|
|
|
@ -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 {
|
||||
|
|
|
@ -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;
|
||||
}
|
||||
|
|
|
@ -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());
|
||||
}
|
||||
}
|
||||
|
||||
}
|
|
@ -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);
|
||||
|
|
|
@ -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),
|
||||
|
|
|
@ -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) {
|
||||
|
|
|
@ -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
|
||||
|
|
|
@ -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"
|
||||
}
|
||||
|
|
|
@ -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"
|
||||
}
|
||||
|
|
|
@ -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
|
||||
|
|
Loading…
Reference in New Issue