diff --git a/elasticsearch/src/main/java/org/elasticsearch/xpack/prelert/PrelertPlugin.java b/elasticsearch/src/main/java/org/elasticsearch/xpack/prelert/PrelertPlugin.java index d8ec69e2199..cf0b3220f56 100644 --- a/elasticsearch/src/main/java/org/elasticsearch/xpack/prelert/PrelertPlugin.java +++ b/elasticsearch/src/main/java/org/elasticsearch/xpack/prelert/PrelertPlugin.java @@ -26,9 +26,12 @@ import org.elasticsearch.threadpool.ExecutorBuilder; import org.elasticsearch.threadpool.FixedExecutorBuilder; import org.elasticsearch.threadpool.ThreadPool; import org.elasticsearch.watcher.ResourceWatcherService; +import org.elasticsearch.xpack.prelert.action.CloseJobAction; import org.elasticsearch.xpack.prelert.action.DeleteJobAction; import org.elasticsearch.xpack.prelert.action.DeleteListAction; import org.elasticsearch.xpack.prelert.action.DeleteModelSnapshotAction; +import org.elasticsearch.xpack.prelert.action.DeleteSchedulerAction; +import org.elasticsearch.xpack.prelert.action.FlushJobAction; import org.elasticsearch.xpack.prelert.action.GetBucketsAction; import org.elasticsearch.xpack.prelert.action.GetCategoriesDefinitionAction; import org.elasticsearch.xpack.prelert.action.GetInfluencersAction; @@ -37,17 +40,16 @@ import org.elasticsearch.xpack.prelert.action.GetListAction; import org.elasticsearch.xpack.prelert.action.GetModelSnapshotsAction; import org.elasticsearch.xpack.prelert.action.GetRecordsAction; import org.elasticsearch.xpack.prelert.action.JobDataAction; -import org.elasticsearch.xpack.prelert.action.CloseJobAction; -import org.elasticsearch.xpack.prelert.action.FlushJobAction; +import org.elasticsearch.xpack.prelert.action.OpenJobAction; import org.elasticsearch.xpack.prelert.action.PutJobAction; import org.elasticsearch.xpack.prelert.action.PutListAction; import org.elasticsearch.xpack.prelert.action.PutModelSnapshotDescriptionAction; -import org.elasticsearch.xpack.prelert.action.OpenJobAction; +import org.elasticsearch.xpack.prelert.action.PutSchedulerAction; import org.elasticsearch.xpack.prelert.action.RevertModelSnapshotAction; -import org.elasticsearch.xpack.prelert.action.StartJobSchedulerAction; -import org.elasticsearch.xpack.prelert.action.StopJobSchedulerAction; -import org.elasticsearch.xpack.prelert.action.UpdateJobSchedulerStatusAction; +import org.elasticsearch.xpack.prelert.action.StartSchedulerAction; +import org.elasticsearch.xpack.prelert.action.StopSchedulerAction; import org.elasticsearch.xpack.prelert.action.UpdateJobStatusAction; +import org.elasticsearch.xpack.prelert.action.UpdateSchedulerStatusAction; import org.elasticsearch.xpack.prelert.action.ValidateDetectorAction; import org.elasticsearch.xpack.prelert.action.ValidateTransformAction; import org.elasticsearch.xpack.prelert.action.ValidateTransformsAction; @@ -58,8 +60,8 @@ import org.elasticsearch.xpack.prelert.job.metadata.JobAllocator; import org.elasticsearch.xpack.prelert.job.metadata.JobLifeCycleService; import org.elasticsearch.xpack.prelert.job.metadata.PrelertInitializationService; import org.elasticsearch.xpack.prelert.job.metadata.PrelertMetadata; -import org.elasticsearch.xpack.prelert.job.persistence.JobDataDeleterFactory; import org.elasticsearch.xpack.prelert.job.persistence.JobDataCountsPersister; +import org.elasticsearch.xpack.prelert.job.persistence.JobDataDeleterFactory; import org.elasticsearch.xpack.prelert.job.persistence.JobProvider; import org.elasticsearch.xpack.prelert.job.persistence.JobResultsPersister; import org.elasticsearch.xpack.prelert.job.process.NativeController; @@ -75,15 +77,14 @@ import org.elasticsearch.xpack.prelert.job.process.normalizer.NormalizerProcessF import org.elasticsearch.xpack.prelert.job.scheduler.http.HttpDataExtractorFactory; import org.elasticsearch.xpack.prelert.job.status.StatusReporter; import org.elasticsearch.xpack.prelert.job.usage.UsageReporter; -import org.elasticsearch.xpack.prelert.rest.job.RestJobDataAction; import org.elasticsearch.xpack.prelert.rest.job.RestCloseJobAction; -import org.elasticsearch.xpack.prelert.rest.job.RestFlushJobAction; -import org.elasticsearch.xpack.prelert.rest.list.RestDeleteListAction; -import org.elasticsearch.xpack.prelert.rest.results.RestGetInfluencersAction; import org.elasticsearch.xpack.prelert.rest.job.RestDeleteJobAction; +import org.elasticsearch.xpack.prelert.rest.job.RestFlushJobAction; import org.elasticsearch.xpack.prelert.rest.job.RestGetJobsAction; -import org.elasticsearch.xpack.prelert.rest.job.RestPutJobAction; +import org.elasticsearch.xpack.prelert.rest.job.RestJobDataAction; import org.elasticsearch.xpack.prelert.rest.job.RestOpenJobAction; +import org.elasticsearch.xpack.prelert.rest.job.RestPutJobAction; +import org.elasticsearch.xpack.prelert.rest.list.RestDeleteListAction; import org.elasticsearch.xpack.prelert.rest.list.RestGetListAction; import org.elasticsearch.xpack.prelert.rest.list.RestPutListAction; import org.elasticsearch.xpack.prelert.rest.modelsnapshots.RestDeleteModelSnapshotAction; @@ -92,9 +93,12 @@ import org.elasticsearch.xpack.prelert.rest.modelsnapshots.RestPutModelSnapshotD import org.elasticsearch.xpack.prelert.rest.modelsnapshots.RestRevertModelSnapshotAction; import org.elasticsearch.xpack.prelert.rest.results.RestGetBucketsAction; import org.elasticsearch.xpack.prelert.rest.results.RestGetCategoriesAction; +import org.elasticsearch.xpack.prelert.rest.results.RestGetInfluencersAction; import org.elasticsearch.xpack.prelert.rest.results.RestGetRecordsAction; -import org.elasticsearch.xpack.prelert.rest.schedulers.RestStartJobSchedulerAction; -import org.elasticsearch.xpack.prelert.rest.schedulers.RestStopJobSchedulerAction; +import org.elasticsearch.xpack.prelert.rest.schedulers.RestDeleteSchedulerAction; +import org.elasticsearch.xpack.prelert.rest.schedulers.RestPutSchedulerAction; +import org.elasticsearch.xpack.prelert.rest.schedulers.RestStartSchedulerAction; +import org.elasticsearch.xpack.prelert.rest.schedulers.RestStopSchedulerAction; import org.elasticsearch.xpack.prelert.rest.validate.RestValidateDetectorAction; import org.elasticsearch.xpack.prelert.rest.validate.RestValidateTransformAction; import org.elasticsearch.xpack.prelert.rest.validate.RestValidateTransformsAction; @@ -217,8 +221,10 @@ public class PrelertPlugin extends Plugin implements ActionPlugin { RestGetModelSnapshotsAction.class, RestRevertModelSnapshotAction.class, RestPutModelSnapshotDescriptionAction.class, - RestStartJobSchedulerAction.class, - RestStopJobSchedulerAction.class, + RestPutSchedulerAction.class, + RestDeleteSchedulerAction.class, + RestStartSchedulerAction.class, + RestStopSchedulerAction.class, RestDeleteModelSnapshotAction.class ); } @@ -231,7 +237,7 @@ public class PrelertPlugin extends Plugin implements ActionPlugin { new ActionHandler<>(DeleteJobAction.INSTANCE, DeleteJobAction.TransportAction.class), new ActionHandler<>(OpenJobAction.INSTANCE, OpenJobAction.TransportAction.class), new ActionHandler<>(UpdateJobStatusAction.INSTANCE, UpdateJobStatusAction.TransportAction.class), - new ActionHandler<>(UpdateJobSchedulerStatusAction.INSTANCE, UpdateJobSchedulerStatusAction.TransportAction.class), + new ActionHandler<>(UpdateSchedulerStatusAction.INSTANCE, UpdateSchedulerStatusAction.TransportAction.class), new ActionHandler<>(GetListAction.INSTANCE, GetListAction.TransportAction.class), new ActionHandler<>(PutListAction.INSTANCE, PutListAction.TransportAction.class), new ActionHandler<>(DeleteListAction.INSTANCE, DeleteListAction.TransportAction.class), @@ -248,8 +254,10 @@ public class PrelertPlugin extends Plugin implements ActionPlugin { new ActionHandler<>(GetModelSnapshotsAction.INSTANCE, GetModelSnapshotsAction.TransportAction.class), new ActionHandler<>(RevertModelSnapshotAction.INSTANCE, RevertModelSnapshotAction.TransportAction.class), new ActionHandler<>(PutModelSnapshotDescriptionAction.INSTANCE, PutModelSnapshotDescriptionAction.TransportAction.class), - new ActionHandler<>(StartJobSchedulerAction.INSTANCE, StartJobSchedulerAction.TransportAction.class), - new ActionHandler<>(StopJobSchedulerAction.INSTANCE, StopJobSchedulerAction.TransportAction.class), + new ActionHandler<>(PutSchedulerAction.INSTANCE, PutSchedulerAction.TransportAction.class), + new ActionHandler<>(DeleteSchedulerAction.INSTANCE, DeleteSchedulerAction.TransportAction.class), + new ActionHandler<>(StartSchedulerAction.INSTANCE, StartSchedulerAction.TransportAction.class), + new ActionHandler<>(StopSchedulerAction.INSTANCE, StopSchedulerAction.TransportAction.class), new ActionHandler<>(DeleteModelSnapshotAction.INSTANCE, DeleteModelSnapshotAction.TransportAction.class) ); } diff --git a/elasticsearch/src/main/java/org/elasticsearch/xpack/prelert/action/DeleteSchedulerAction.java b/elasticsearch/src/main/java/org/elasticsearch/xpack/prelert/action/DeleteSchedulerAction.java new file mode 100644 index 00000000000..8b59d47c27d --- /dev/null +++ b/elasticsearch/src/main/java/org/elasticsearch/xpack/prelert/action/DeleteSchedulerAction.java @@ -0,0 +1,187 @@ +/* + * 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.prelert.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.AckedClusterStateUpdateTask; +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.metadata.MetaData; +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.common.xcontent.ToXContent; +import org.elasticsearch.common.xcontent.XContentBuilder; +import org.elasticsearch.threadpool.ThreadPool; +import org.elasticsearch.transport.TransportService; +import org.elasticsearch.xpack.prelert.job.SchedulerConfig; +import org.elasticsearch.xpack.prelert.job.metadata.PrelertMetadata; +import org.elasticsearch.xpack.prelert.utils.ExceptionsHelper; + +import java.io.IOException; +import java.util.Objects; + +public class DeleteSchedulerAction extends Action { + + public static final DeleteSchedulerAction INSTANCE = new DeleteSchedulerAction(); + public static final String NAME = "cluster:admin/prelert/scheduler/delete"; + + private DeleteSchedulerAction() { + 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 implements ToXContent { + + private String schedulerId; + + public Request(String schedulerId) { + this.schedulerId = ExceptionsHelper.requireNonNull(schedulerId, SchedulerConfig.ID.getPreferredName()); + } + + Request() { + } + + public String getSchedulerId() { + return schedulerId; + } + + @Override + public ActionRequestValidationException validate() { + return null; + } + + @Override + public void readFrom(StreamInput in) throws IOException { + super.readFrom(in); + schedulerId = in.readString(); + } + + @Override + public void writeTo(StreamOutput out) throws IOException { + super.writeTo(out); + out.writeString(schedulerId); + } + + @Override + public XContentBuilder toXContent(XContentBuilder builder, Params params) throws IOException { + builder.field(SchedulerConfig.ID.getPreferredName(), schedulerId); + return builder; + } + + @Override + public boolean equals(Object o) { + if (this == o) return true; + if (o == null || getClass() != o.getClass()) return false; + Request request = (Request) o; + return Objects.equals(schedulerId, request.schedulerId); + } + + @Override + public int hashCode() { + return Objects.hash(schedulerId); + } + } + + public static class RequestBuilder extends MasterNodeOperationRequestBuilder { + + public RequestBuilder(ElasticsearchClient client, DeleteSchedulerAction action) { + super(client, action, new Request()); + } + } + + public static class Response extends AcknowledgedResponse { + + private Response() { + } + + private Response(boolean acknowledged) { + super(acknowledged); + } + + @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 { + + @Inject + public TransportAction(Settings settings, TransportService transportService, ClusterService clusterService, + ThreadPool threadPool, ActionFilters actionFilters, IndexNameExpressionResolver indexNameExpressionResolver) { + super(settings, DeleteSchedulerAction.NAME, transportService, clusterService, threadPool, actionFilters, + indexNameExpressionResolver, Request::new); + } + + @Override + protected String executor() { + return ThreadPool.Names.SAME; + } + + @Override + protected Response newResponse() { + return new Response(); + } + + @Override + protected void masterOperation(Request request, ClusterState state, ActionListener listener) throws Exception { + clusterService.submitStateUpdateTask("delete-scheduler-" + request.getSchedulerId(), + new AckedClusterStateUpdateTask(request, listener) { + + @Override + protected Response newResponse(boolean acknowledged) { + return new Response(acknowledged); + } + + @Override + public ClusterState execute(ClusterState currentState) throws Exception { + PrelertMetadata currentMetadata = state.getMetaData().custom(PrelertMetadata.TYPE); + PrelertMetadata newMetadata = new PrelertMetadata.Builder(currentMetadata) + .removeScheduler(request.getSchedulerId()).build(); + return ClusterState.builder(state).metaData( + MetaData.builder(currentState.getMetaData()).putCustom(PrelertMetadata.TYPE, newMetadata).build()) + .build(); + } + }); + + } + + @Override + protected ClusterBlockException checkBlock(Request request, ClusterState state) { + return state.blocks().globalBlockedException(ClusterBlockLevel.METADATA_WRITE); + } + } +} diff --git a/elasticsearch/src/main/java/org/elasticsearch/xpack/prelert/action/PutSchedulerAction.java b/elasticsearch/src/main/java/org/elasticsearch/xpack/prelert/action/PutSchedulerAction.java new file mode 100644 index 00000000000..0753df8229f --- /dev/null +++ b/elasticsearch/src/main/java/org/elasticsearch/xpack/prelert/action/PutSchedulerAction.java @@ -0,0 +1,227 @@ +/* + * 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.prelert.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.AckedClusterStateUpdateTask; +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.metadata.MetaData; +import org.elasticsearch.cluster.service.ClusterService; +import org.elasticsearch.common.ParseFieldMatcherSupplier; +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.xcontent.ToXContent; +import org.elasticsearch.common.xcontent.XContentBuilder; +import org.elasticsearch.common.xcontent.XContentParser; +import org.elasticsearch.threadpool.ThreadPool; +import org.elasticsearch.transport.TransportService; +import org.elasticsearch.xpack.prelert.job.SchedulerConfig; +import org.elasticsearch.xpack.prelert.job.metadata.PrelertMetadata; + +import java.io.IOException; +import java.util.Objects; + +public class PutSchedulerAction extends Action { + + public static final PutSchedulerAction INSTANCE = new PutSchedulerAction(); + public static final String NAME = "cluster:admin/prelert/scheduler/put"; + + private PutSchedulerAction() { + 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 implements ToXContent { + + public static Request parseRequest(String schedulerId, XContentParser parser, ParseFieldMatcherSupplier matcherSupplier) { + SchedulerConfig.Builder scheduler = SchedulerConfig.PARSER.apply(parser, matcherSupplier); + scheduler.setId(schedulerId); + return new Request(scheduler.build()); + } + + private SchedulerConfig scheduler; + + public Request(SchedulerConfig scheduler) { + this.scheduler = scheduler; + } + + Request() { + } + + public SchedulerConfig getScheduler() { + return scheduler; + } + + @Override + public ActionRequestValidationException validate() { + return null; + } + + @Override + public void readFrom(StreamInput in) throws IOException { + super.readFrom(in); + scheduler = new SchedulerConfig(in); + } + + @Override + public void writeTo(StreamOutput out) throws IOException { + super.writeTo(out); + scheduler.writeTo(out); + } + + @Override + public XContentBuilder toXContent(XContentBuilder builder, Params params) throws IOException { + scheduler.toXContent(builder, params); + return builder; + } + + @Override + public boolean equals(Object o) { + if (this == o) return true; + if (o == null || getClass() != o.getClass()) return false; + Request request = (Request) o; + return Objects.equals(scheduler, request.scheduler); + } + + @Override + public int hashCode() { + return Objects.hash(scheduler); + } + } + + public static class RequestBuilder extends MasterNodeOperationRequestBuilder { + + public RequestBuilder(ElasticsearchClient client, PutSchedulerAction action) { + super(client, action, new Request()); + } + } + + public static class Response extends AcknowledgedResponse implements ToXContent { + + private SchedulerConfig scheduler; + + public Response(boolean acked, SchedulerConfig scheduler) { + super(acked); + this.scheduler = scheduler; + } + + Response() { + } + + public SchedulerConfig getResponse() { + return scheduler; + } + + @Override + public void readFrom(StreamInput in) throws IOException { + super.readFrom(in); + readAcknowledged(in); + scheduler = new SchedulerConfig(in); + } + + @Override + public void writeTo(StreamOutput out) throws IOException { + super.writeTo(out); + writeAcknowledged(out); + scheduler.writeTo(out); + } + + @Override + public XContentBuilder toXContent(XContentBuilder builder, Params params) throws IOException { + scheduler.doXContentBody(builder, params); + 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 Objects.equals(scheduler, response.scheduler); + } + + @Override + public int hashCode() { + return Objects.hash(scheduler); + } + } + + public static class TransportAction extends TransportMasterNodeAction { + + @Inject + public TransportAction(Settings settings, TransportService transportService, ClusterService clusterService, + ThreadPool threadPool, ActionFilters actionFilters, IndexNameExpressionResolver indexNameExpressionResolver) { + super(settings, PutSchedulerAction.NAME, transportService, clusterService, threadPool, actionFilters, + indexNameExpressionResolver, Request::new); + } + + @Override + protected String executor() { + return ThreadPool.Names.SAME; + } + + @Override + protected Response newResponse() { + return new Response(); + } + + @Override + protected void masterOperation(Request request, ClusterState state, ActionListener listener) throws Exception { + clusterService.submitStateUpdateTask("put-scheduler-" + request.getScheduler().getId(), + new AckedClusterStateUpdateTask(request, listener) { + + @Override + protected Response newResponse(boolean acknowledged) { + if (acknowledged) { + logger.info("Created scheduler [{}]", request.getScheduler().getId()); + } + return new Response(acknowledged, request.getScheduler()); + } + + @Override + public ClusterState execute(ClusterState currentState) throws Exception { + return putScheduler(request, currentState); + } + }); + } + + private ClusterState putScheduler(Request request, ClusterState clusterState) { + PrelertMetadata currentMetadata = clusterState.getMetaData().custom(PrelertMetadata.TYPE); + PrelertMetadata newMetadata = new PrelertMetadata.Builder(currentMetadata) + .putScheduler(request.getScheduler()).build(); + return ClusterState.builder(clusterState).metaData( + MetaData.builder(clusterState.getMetaData()).putCustom(PrelertMetadata.TYPE, newMetadata).build()) + .build(); + } + + @Override + protected ClusterBlockException checkBlock(Request request, ClusterState state) { + return state.blocks().globalBlockedException(ClusterBlockLevel.METADATA_WRITE); + } + } +} diff --git a/elasticsearch/src/main/java/org/elasticsearch/xpack/prelert/action/StartJobSchedulerAction.java b/elasticsearch/src/main/java/org/elasticsearch/xpack/prelert/action/StartSchedulerAction.java similarity index 78% rename from elasticsearch/src/main/java/org/elasticsearch/xpack/prelert/action/StartJobSchedulerAction.java rename to elasticsearch/src/main/java/org/elasticsearch/xpack/prelert/action/StartSchedulerAction.java index aaefd2cdbb9..66007d2cb86 100644 --- a/elasticsearch/src/main/java/org/elasticsearch/xpack/prelert/action/StartJobSchedulerAction.java +++ b/elasticsearch/src/main/java/org/elasticsearch/xpack/prelert/action/StartSchedulerAction.java @@ -30,23 +30,23 @@ import org.elasticsearch.tasks.Task; import org.elasticsearch.tasks.TaskId; import org.elasticsearch.threadpool.ThreadPool; import org.elasticsearch.transport.TransportService; -import org.elasticsearch.xpack.prelert.job.Job; +import org.elasticsearch.xpack.prelert.job.SchedulerConfig; import org.elasticsearch.xpack.prelert.job.scheduler.ScheduledJobRunner; import org.elasticsearch.xpack.prelert.utils.ExceptionsHelper; import java.io.IOException; import java.util.Objects; -public class StartJobSchedulerAction -extends Action { +public class StartSchedulerAction + extends Action { public static final ParseField START_TIME = new ParseField("start"); public static final ParseField END_TIME = new ParseField("end"); - public static final StartJobSchedulerAction INSTANCE = new StartJobSchedulerAction(); - public static final String NAME = "cluster:admin/prelert/job/scheduler/run"; + public static final StartSchedulerAction INSTANCE = new StartSchedulerAction(); + public static final String NAME = "cluster:admin/prelert/scheduler/start"; - private StartJobSchedulerAction() { + private StartSchedulerAction() { super(NAME); } @@ -65,33 +65,33 @@ extends Action PARSER = new ObjectParser<>(NAME, Request::new); static { - PARSER.declareString((request, jobId) -> request.jobId = jobId, Job.ID); + PARSER.declareString((request, schedulerId) -> request.schedulerId = schedulerId, SchedulerConfig.ID); PARSER.declareLong((request, startTime) -> request.startTime = startTime, START_TIME); PARSER.declareLong(Request::setEndTime, END_TIME); } - public static Request parseRequest(String jobId, XContentParser parser, ParseFieldMatcherSupplier parseFieldMatcherSupplier) { + public static Request parseRequest(String schedulerId, XContentParser parser, ParseFieldMatcherSupplier parseFieldMatcherSupplier) { Request request = PARSER.apply(parser, parseFieldMatcherSupplier); - if (jobId != null) { - request.jobId = jobId; + if (schedulerId != null) { + request.schedulerId = schedulerId; } return request; } - private String jobId; + private String schedulerId; private long startTime; private Long endTime; - public Request(String jobId, long startTime) { - this.jobId = ExceptionsHelper.requireNonNull(jobId, Job.ID.getPreferredName()); + public Request(String schedulerId, long startTime) { + this.schedulerId = ExceptionsHelper.requireNonNull(schedulerId, SchedulerConfig.ID.getPreferredName()); this.startTime = startTime; } Request() { } - public String getJobId() { - return jobId; + public String getSchedulerId() { + return schedulerId; } public long getStartTime() { @@ -113,13 +113,13 @@ extends Action { - public RequestBuilder(ElasticsearchClient client, StartJobSchedulerAction action) { + public RequestBuilder(ElasticsearchClient client, StartSchedulerAction action) { super(client, action, new Request()); } } @@ -182,8 +182,8 @@ extends Action listener) { SchedulerTask schedulerTask = (SchedulerTask) task; - scheduledJobRunner.run(request.getJobId(), request.getStartTime(), request.getEndTime(), schedulerTask, (error) -> { + scheduledJobRunner.run(request.getSchedulerId(), request.getStartTime(), request.getEndTime(), schedulerTask, (error) -> { if (error != null) { listener.onFailure(error); } else { diff --git a/elasticsearch/src/main/java/org/elasticsearch/xpack/prelert/action/StopJobSchedulerAction.java b/elasticsearch/src/main/java/org/elasticsearch/xpack/prelert/action/StopSchedulerAction.java similarity index 82% rename from elasticsearch/src/main/java/org/elasticsearch/xpack/prelert/action/StopJobSchedulerAction.java rename to elasticsearch/src/main/java/org/elasticsearch/xpack/prelert/action/StopSchedulerAction.java index 9e7dae3ea1e..f3c7dd4f5ab 100644 --- a/elasticsearch/src/main/java/org/elasticsearch/xpack/prelert/action/StopJobSchedulerAction.java +++ b/elasticsearch/src/main/java/org/elasticsearch/xpack/prelert/action/StopSchedulerAction.java @@ -29,20 +29,19 @@ import org.elasticsearch.common.settings.Settings; import org.elasticsearch.tasks.TaskInfo; import org.elasticsearch.threadpool.ThreadPool; import org.elasticsearch.transport.TransportService; -import org.elasticsearch.xpack.prelert.job.Job; -import org.elasticsearch.xpack.prelert.job.manager.JobManager; +import org.elasticsearch.xpack.prelert.job.SchedulerConfig; import org.elasticsearch.xpack.prelert.utils.ExceptionsHelper; import java.io.IOException; import java.util.Objects; -public class StopJobSchedulerAction -extends Action { +public class StopSchedulerAction + extends Action { - public static final StopJobSchedulerAction INSTANCE = new StopJobSchedulerAction(); - public static final String NAME = "cluster:admin/prelert/job/scheduler/stop"; + public static final StopSchedulerAction INSTANCE = new StopSchedulerAction(); + public static final String NAME = "cluster:admin/prelert/scheduler/stop"; - private StopJobSchedulerAction() { + private StopSchedulerAction() { super(NAME); } @@ -58,17 +57,17 @@ extends Action { - public RequestBuilder(ElasticsearchClient client, StopJobSchedulerAction action) { + public RequestBuilder(ElasticsearchClient client, StopSchedulerAction action) { super(client, action, new Request()); } } @@ -139,7 +138,7 @@ extends Action listener) { - String jobId = request.getJobId(); + String schedulerId = request.getSchedulerId(); ListTasksRequest listTasksRequest = new ListTasksRequest(); - listTasksRequest.setActions(StartJobSchedulerAction.NAME); + listTasksRequest.setActions(StartSchedulerAction.NAME); listTasksRequest.setDetailed(true); listTasksAction.execute(listTasksRequest, new ActionListener() { @Override public void onResponse(ListTasksResponse listTasksResponse) { - String expectedJobDescription = "job-scheduler-" + jobId; + String expectedJobDescription = "scheduler-" + schedulerId; for (TaskInfo taskInfo : listTasksResponse.getTasks()) { if (expectedJobDescription.equals(taskInfo.getDescription())) { CancelTasksRequest cancelTasksRequest = new CancelTasksRequest(); @@ -173,7 +172,7 @@ extends Action { +public class UpdateSchedulerStatusAction extends Action { - public static final UpdateJobSchedulerStatusAction INSTANCE = new UpdateJobSchedulerStatusAction(); - public static final String NAME = "cluster:admin/prelert/job/scheduler/status/update"; + public static final UpdateSchedulerStatusAction INSTANCE = new UpdateSchedulerStatusAction(); + public static final String NAME = "cluster:admin/prelert/scheduler/status/update"; - private UpdateJobSchedulerStatusAction() { + private UpdateSchedulerStatusAction() { super(NAME); } @@ -55,22 +55,22 @@ public class UpdateJobSchedulerStatusAction extends Action { - private String jobId; + private String schedulerId; private SchedulerStatus schedulerStatus; - public Request(String jobId, SchedulerStatus schedulerStatus) { - this.jobId = ExceptionsHelper.requireNonNull(jobId, Job.ID.getPreferredName()); + public Request(String schedulerId, SchedulerStatus schedulerStatus) { + this.schedulerId = ExceptionsHelper.requireNonNull(schedulerId, SchedulerConfig.ID.getPreferredName()); this.schedulerStatus = ExceptionsHelper.requireNonNull(schedulerStatus, "status"); } Request() {} - public String getJobId() { - return jobId; + public String getSchedulerId() { + return schedulerId; } - public void setJobId(String jobId) { - this.jobId = jobId; + public void setSchedulerId(String schedulerId) { + this.schedulerId = schedulerId; } public SchedulerStatus getSchedulerStatus() { @@ -89,20 +89,20 @@ public class UpdateJobSchedulerStatusAction extends Action implements Writeable, ToXContent }, LAST_DATA_TIME, ValueType.VALUE); PARSER.declareObject(Builder::setAnalysisConfig, AnalysisConfig.PARSER, ANALYSIS_CONFIG); PARSER.declareObject(Builder::setAnalysisLimits, AnalysisLimits.PARSER, ANALYSIS_LIMITS); - PARSER.declareObject(Builder::setSchedulerConfig, SchedulerConfig.PARSER, SCHEDULER_CONFIG); PARSER.declareObject(Builder::setDataDescription, DataDescription.PARSER, DATA_DESCRIPTION); PARSER.declareObjectArray(Builder::setTransforms, TransformConfig.PARSER, TRANSFORMS); PARSER.declareObject(Builder::setModelDebugConfig, ModelDebugConfig.PARSER, MODEL_DEBUG_CONFIG); @@ -135,7 +133,6 @@ public class Job extends AbstractDiffable implements Writeable, ToXContent private final long timeout; private final AnalysisConfig analysisConfig; private final AnalysisLimits analysisLimits; - private final SchedulerConfig schedulerConfig; private final DataDescription dataDescription; private final List transforms; private final ModelDebugConfig modelDebugConfig; @@ -148,10 +145,9 @@ public class Job extends AbstractDiffable implements Writeable, ToXContent private final String modelSnapshotId; public Job(String jobId, String description, Date createTime, Date finishedTime, Date lastDataTime, long timeout, - AnalysisConfig analysisConfig, AnalysisLimits analysisLimits, SchedulerConfig schedulerConfig, - DataDescription dataDescription, List transforms, - ModelDebugConfig modelDebugConfig, IgnoreDowntime ignoreDowntime, Long renormalizationWindowDays, - Long backgroundPersistInterval, Long modelSnapshotRetentionDays, Long resultsRetentionDays, + AnalysisConfig analysisConfig, AnalysisLimits analysisLimits, DataDescription dataDescription, + List transforms, ModelDebugConfig modelDebugConfig, IgnoreDowntime ignoreDowntime, + Long renormalizationWindowDays, Long backgroundPersistInterval, Long modelSnapshotRetentionDays, Long resultsRetentionDays, Map customSettings, String modelSnapshotId) { this.jobId = jobId; this.description = description; @@ -161,7 +157,6 @@ public class Job extends AbstractDiffable implements Writeable, ToXContent this.timeout = timeout; this.analysisConfig = analysisConfig; this.analysisLimits = analysisLimits; - this.schedulerConfig = schedulerConfig; this.dataDescription = dataDescription; this.transforms = transforms; this.modelDebugConfig = modelDebugConfig; @@ -183,7 +178,6 @@ public class Job extends AbstractDiffable implements Writeable, ToXContent timeout = in.readVLong(); analysisConfig = new AnalysisConfig(in); analysisLimits = in.readOptionalWriteable(AnalysisLimits::new); - schedulerConfig = in.readOptionalWriteable(SchedulerConfig::new); dataDescription = in.readOptionalWriteable(DataDescription::new); transforms = in.readList(TransformConfig::new); modelDebugConfig = in.readOptionalWriteable(ModelDebugConfig::new); @@ -290,10 +284,6 @@ public class Job extends AbstractDiffable implements Writeable, ToXContent return ignoreDowntime; } - public SchedulerConfig getSchedulerConfig() { - return schedulerConfig; - } - public ModelDebugConfig getModelDebugConfig() { return modelDebugConfig; } @@ -405,7 +395,6 @@ public class Job extends AbstractDiffable implements Writeable, ToXContent out.writeVLong(timeout); analysisConfig.writeTo(out); out.writeOptionalWriteable(analysisLimits); - out.writeOptionalWriteable(schedulerConfig); out.writeOptionalWriteable(dataDescription); out.writeList(transforms); out.writeOptionalWriteable(modelDebugConfig); @@ -441,9 +430,6 @@ public class Job extends AbstractDiffable implements Writeable, ToXContent if (analysisLimits != null) { builder.field(ANALYSIS_LIMITS.getPreferredName(), analysisLimits, params); } - if (schedulerConfig != null) { - builder.field(SCHEDULER_CONFIG.getPreferredName(), schedulerConfig, params); - } if (dataDescription != null) { builder.field(DATA_DESCRIPTION.getPreferredName(), dataDescription, params); } @@ -490,8 +476,10 @@ public class Job extends AbstractDiffable implements Writeable, ToXContent Job that = (Job) other; return Objects.equals(this.jobId, that.jobId) && Objects.equals(this.description, that.description) && Objects.equals(this.createTime, that.createTime) - && Objects.equals(this.finishedTime, that.finishedTime) && Objects.equals(this.lastDataTime, that.lastDataTime) - && (this.timeout == that.timeout) && Objects.equals(this.analysisConfig, that.analysisConfig) + && Objects.equals(this.finishedTime, that.finishedTime) + && Objects.equals(this.lastDataTime, that.lastDataTime) + && (this.timeout == that.timeout) + && Objects.equals(this.analysisConfig, that.analysisConfig) && Objects.equals(this.analysisLimits, that.analysisLimits) && Objects.equals(this.dataDescription, that.dataDescription) && Objects.equals(this.modelDebugConfig, that.modelDebugConfig) && Objects.equals(this.transforms, that.transforms) && Objects.equals(this.ignoreDowntime, that.ignoreDowntime) @@ -528,10 +516,6 @@ public class Job extends AbstractDiffable implements Writeable, ToXContent public static class Builder { - /** - * Valid jobId characters. Note that '.' is allowed but not documented. - */ - private static final Pattern VALID_JOB_ID_CHAR_PATTERN = Pattern.compile("[a-z0-9_\\-\\.]+"); public static final int MAX_JOB_ID_LENGTH = 64; public static final long MIN_BACKGROUND_PERSIST_INTERVAL = 3600; public static final long DEFAULT_TIMEOUT = 600; @@ -541,7 +525,6 @@ public class Job extends AbstractDiffable implements Writeable, ToXContent private AnalysisConfig analysisConfig; private AnalysisLimits analysisLimits; - private SchedulerConfig schedulerConfig; private List transforms = new ArrayList<>(); private DataDescription dataDescription; private Date createTime; @@ -568,7 +551,6 @@ public class Job extends AbstractDiffable implements Writeable, ToXContent this.id = job.getId(); this.description = job.getDescription(); this.analysisConfig = job.getAnalysisConfig(); - this.schedulerConfig = job.getSchedulerConfig(); this.transforms = job.getTransforms(); this.dataDescription = job.getDataDescription(); this.createTime = job.getCreateTime(); @@ -617,10 +599,6 @@ public class Job extends AbstractDiffable implements Writeable, ToXContent this.analysisLimits = analysisLimits; } - public void setSchedulerConfig(SchedulerConfig.Builder config) { - schedulerConfig = config.build(); - } - public void setTimeout(Long timeout) { this.timeout = timeout; } @@ -686,25 +664,6 @@ public class Job extends AbstractDiffable implements Writeable, ToXContent throw new IllegalArgumentException(Messages.getMessage(Messages.JOB_CONFIG_MISSING_ANALYSISCONFIG)); } - if (schedulerConfig != null) { - if (analysisConfig.getBucketSpan() == null) { - throw new IllegalArgumentException(Messages.getMessage(Messages.JOB_CONFIG_SCHEDULER_REQUIRES_BUCKET_SPAN)); - } - if (analysisConfig.getLatency() != null && analysisConfig.getLatency() > 0) { - throw new IllegalArgumentException( - Messages.getMessage(Messages.JOB_CONFIG_SCHEDULER_ELASTICSEARCH_DOES_NOT_SUPPORT_LATENCY)); - } - if (schedulerConfig.getAggregationsOrAggs() != null - && !SchedulerConfig.DOC_COUNT.equals(analysisConfig.getSummaryCountFieldName())) { - throw new IllegalArgumentException( - Messages.getMessage(Messages.JOB_CONFIG_SCHEDULER_AGGREGATIONS_REQUIRES_SUMMARY_COUNT_FIELD, - SchedulerConfig.DOC_COUNT)); - } - if (dataDescription == null || dataDescription.getFormat() != DataDescription.DataFormat.ELASTICSEARCH) { - throw new IllegalArgumentException( - Messages.getMessage(Messages.JOB_CONFIG_SCHEDULER_ELASTICSEARCH_REQUIRES_DATAFORMAT_ELASTICSEARCH)); - } - } if (transforms != null && transforms.isEmpty() == false) { TransformConfigsVerifier.verify(transforms); checkTransformOutputIsUsed(); @@ -745,12 +704,12 @@ public class Job extends AbstractDiffable implements Writeable, ToXContent if (id.length() > MAX_JOB_ID_LENGTH) { throw new IllegalArgumentException(Messages.getMessage(Messages.JOB_CONFIG_ID_TOO_LONG, MAX_JOB_ID_LENGTH)); } - if (!VALID_JOB_ID_CHAR_PATTERN.matcher(id).matches()) { + if (!PrelertStrings.isValidId(id)) { throw new IllegalArgumentException(Messages.getMessage(Messages.JOB_CONFIG_INVALID_JOBID_CHARS)); } return new Job( id, description, createTime, finishedTime, lastDataTime, timeout, analysisConfig, analysisLimits, - schedulerConfig, dataDescription, transforms, modelDebugConfig, ignoreDowntime, renormalizationWindowDays, + dataDescription, transforms, modelDebugConfig, ignoreDowntime, renormalizationWindowDays, backgroundPersistInterval, modelSnapshotRetentionDays, resultsRetentionDays, customSettings, modelSnapshotId ); } diff --git a/elasticsearch/src/main/java/org/elasticsearch/xpack/prelert/job/SchedulerConfig.java b/elasticsearch/src/main/java/org/elasticsearch/xpack/prelert/job/SchedulerConfig.java index a777511316b..f7ea7af7107 100644 --- a/elasticsearch/src/main/java/org/elasticsearch/xpack/prelert/job/SchedulerConfig.java +++ b/elasticsearch/src/main/java/org/elasticsearch/xpack/prelert/job/SchedulerConfig.java @@ -12,9 +12,11 @@ import org.elasticsearch.common.ParseFieldMatcherSupplier; 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.ConstructingObjectParser; +import org.elasticsearch.common.xcontent.ObjectParser; import org.elasticsearch.common.xcontent.XContentBuilder; import org.elasticsearch.xpack.prelert.job.messages.Messages; +import org.elasticsearch.xpack.prelert.utils.ExceptionsHelper; +import org.elasticsearch.xpack.prelert.utils.PrelertStrings; import java.io.IOException; import java.util.ArrayList; @@ -47,6 +49,7 @@ public class SchedulerConfig extends ToXContentToBytes implements Writeable { */ public static final String DOC_COUNT = "doc_count"; + public static final ParseField ID = new ParseField("scheduler_id"); public static final ParseField QUERY_DELAY = new ParseField("query_delay"); public static final ParseField FREQUENCY = new ParseField("frequency"); public static final ParseField INDEXES = new ParseField("indexes"); @@ -58,13 +61,13 @@ public class SchedulerConfig extends ToXContentToBytes implements Writeable { public static final ParseField AGGS = new ParseField("aggs"); public static final ParseField SCRIPT_FIELDS = new ParseField("script_fields"); - @SuppressWarnings("unchecked") - public static final ConstructingObjectParser PARSER = - new ConstructingObjectParser<>("schedule_config", a -> new SchedulerConfig.Builder((List) a[0], (List) a[1])); + public static final ObjectParser PARSER = new ObjectParser<>("schedule_config", Builder::new); static { - PARSER.declareStringArray(ConstructingObjectParser.constructorArg(), INDEXES); - PARSER.declareStringArray(ConstructingObjectParser.constructorArg(), TYPES); + PARSER.declareString(Builder::setId, ID); + PARSER.declareString(Builder::setJobId, Job.ID); + PARSER.declareStringArray(Builder::setIndexes, INDEXES); + PARSER.declareStringArray(Builder::setTypes, TYPES); PARSER.declareLong(Builder::setQueryDelay, QUERY_DELAY); PARSER.declareLong(Builder::setFrequency, FREQUENCY); PARSER.declareObject(Builder::setQuery, (p, c) -> { @@ -99,6 +102,9 @@ public class SchedulerConfig extends ToXContentToBytes implements Writeable { PARSER.declareInt(Builder::setScrollSize, SCROLL_SIZE); } + private final String id; + private final String jobId; + /** * The delay in seconds before starting to query a period of time */ @@ -120,9 +126,11 @@ public class SchedulerConfig extends ToXContentToBytes implements Writeable { private final Boolean retrieveWholeSource; private final Integer scrollSize; - private SchedulerConfig(Long queryDelay, Long frequency, List indexes, List types, Map query, - Map aggregations, Map aggs, Map scriptFields, - Boolean retrieveWholeSource, Integer scrollSize) { + private SchedulerConfig(String id, String jobId, Long queryDelay, Long frequency, List indexes, List types, + Map query, Map aggregations, Map aggs, + Map scriptFields, Boolean retrieveWholeSource, Integer scrollSize) { + this.id = id; + this.jobId = jobId; this.queryDelay = queryDelay; this.frequency = frequency; this.indexes = indexes; @@ -136,6 +144,8 @@ public class SchedulerConfig extends ToXContentToBytes implements Writeable { } public SchedulerConfig(StreamInput in) throws IOException { + this.id = in.readString(); + this.jobId = in.readString(); this.queryDelay = in.readOptionalLong(); this.frequency = in.readOptionalLong(); if (in.readBoolean()) { @@ -172,12 +182,20 @@ public class SchedulerConfig extends ToXContentToBytes implements Writeable { this.scrollSize = in.readOptionalVInt(); } + public String getId() { + return id; + } + + public String getJobId() { + return jobId; + } + public Long getQueryDelay() { - return this.queryDelay; + return queryDelay; } public Long getFrequency() { - return this.frequency; + return frequency; } /** @@ -316,6 +334,8 @@ public class SchedulerConfig extends ToXContentToBytes implements Writeable { @Override public void writeTo(StreamOutput out) throws IOException { + out.writeString(id); + out.writeString(jobId); out.writeOptionalLong(queryDelay); out.writeOptionalLong(frequency); if (indexes != null) { @@ -361,6 +381,14 @@ public class SchedulerConfig extends ToXContentToBytes implements Writeable { @Override public XContentBuilder toXContent(XContentBuilder builder, Params params) throws IOException { builder.startObject(); + doXContentBody(builder, params); + builder.endObject(); + return builder; + } + + public XContentBuilder doXContentBody(XContentBuilder builder, Params params) throws IOException { + builder.field(ID.getPreferredName(), id); + builder.field(Job.ID.getPreferredName(), jobId); if (queryDelay != null) { builder.field(QUERY_DELAY.getPreferredName(), queryDelay); } @@ -391,7 +419,6 @@ public class SchedulerConfig extends ToXContentToBytes implements Writeable { if (scrollSize != null) { builder.field(SCROLL_SIZE.getPreferredName(), scrollSize); } - builder.endObject(); return builder; } @@ -412,7 +439,9 @@ public class SchedulerConfig extends ToXContentToBytes implements Writeable { SchedulerConfig that = (SchedulerConfig) other; - return Objects.equals(this.frequency, that.frequency) + return Objects.equals(this.id, that.id) + && Objects.equals(this.jobId, that.jobId) + && Objects.equals(this.frequency, that.frequency) && Objects.equals(this.queryDelay, that.queryDelay) && Objects.equals(this.indexes, that.indexes) && Objects.equals(this.types, that.types) && Objects.equals(this.query, that.query) @@ -423,8 +452,8 @@ public class SchedulerConfig extends ToXContentToBytes implements Writeable { @Override public int hashCode() { - return Objects.hash(frequency, queryDelay, indexes, types, query, retrieveWholeSource, scrollSize, getAggregationsOrAggs(), - scriptFields); + return Objects.hash(id, jobId, frequency, queryDelay, indexes, types, query, retrieveWholeSource, scrollSize, + getAggregationsOrAggs(), scriptFields); } public static class Builder { @@ -437,6 +466,8 @@ public class SchedulerConfig extends ToXContentToBytes implements Writeable { */ private static final String MATCH_ALL_ES_QUERY = "match_all"; + private String id; + private String jobId; private Long queryDelay; private Long frequency; private List indexes = Collections.emptyList(); @@ -451,10 +482,7 @@ public class SchedulerConfig extends ToXContentToBytes implements Writeable { private Boolean retrieveWholeSource; private Integer scrollSize; - public Builder(List indexes, List types) { - this.indexes = Objects.requireNonNull(indexes); - this.types = Objects.requireNonNull(types); - + public Builder() { Map query = new HashMap<>(); query.put(MATCH_ALL_ES_QUERY, new HashMap()); setQuery(query); @@ -463,7 +491,15 @@ public class SchedulerConfig extends ToXContentToBytes implements Writeable { setScrollSize(DEFAULT_SCROLL_SIZE); } + public Builder(String id, String jobId) { + this(); + this.id = ExceptionsHelper.requireNonNull(id, ID.getPreferredName()); + this.jobId = ExceptionsHelper.requireNonNull(jobId, Job.ID.getPreferredName()); + } + public Builder(SchedulerConfig config) { + this.id = config.id; + this.jobId = config.jobId; this.queryDelay = config.queryDelay; this.frequency = config.frequency; this.indexes = config.indexes; @@ -476,9 +512,25 @@ public class SchedulerConfig extends ToXContentToBytes implements Writeable { this.scrollSize = config.scrollSize; } + public void setId(String schedulerId) { + id = ExceptionsHelper.requireNonNull(schedulerId, ID.getPreferredName()); + } + + public void setJobId(String jobId) { + this.jobId = ExceptionsHelper.requireNonNull(jobId, Job.ID.getPreferredName()); + } + + public void setIndexes(List indexes) { + this.indexes = ExceptionsHelper.requireNonNull(indexes, INDEXES.getPreferredName()); + } + + public void setTypes(List types) { + this.types = ExceptionsHelper.requireNonNull(types, TYPES.getPreferredName()); + } + public void setQueryDelay(long queryDelay) { if (queryDelay < 0) { - String msg = Messages.getMessage(Messages.JOB_CONFIG_SCHEDULER_INVALID_OPTION_VALUE, + String msg = Messages.getMessage(Messages.SCHEDULER_CONFIG_INVALID_OPTION_VALUE, SchedulerConfig.QUERY_DELAY.getPreferredName(), queryDelay); throw new IllegalArgumentException(msg); } @@ -487,7 +539,7 @@ public class SchedulerConfig extends ToXContentToBytes implements Writeable { public void setFrequency(long frequency) { if (frequency <= 0) { - String msg = Messages.getMessage(Messages.JOB_CONFIG_SCHEDULER_INVALID_OPTION_VALUE, + String msg = Messages.getMessage(Messages.SCHEDULER_CONFIG_INVALID_OPTION_VALUE, SchedulerConfig.FREQUENCY.getPreferredName(), frequency); throw new IllegalArgumentException(msg); } @@ -520,64 +572,19 @@ public class SchedulerConfig extends ToXContentToBytes implements Writeable { public void setScrollSize(int scrollSize) { if (scrollSize < 0) { - String msg = Messages.getMessage(Messages.JOB_CONFIG_SCHEDULER_INVALID_OPTION_VALUE, + String msg = Messages.getMessage(Messages.SCHEDULER_CONFIG_INVALID_OPTION_VALUE, SchedulerConfig.SCROLL_SIZE.getPreferredName(), scrollSize); throw new IllegalArgumentException(msg); } this.scrollSize = scrollSize; } - public Long getQueryDelay() { - return queryDelay; - } - - public Long getFrequency() { - return frequency; - } - - public List getIndexes() { - return indexes; - } - - public List getTypes() { - return types; - } - - public Map getQuery() { - return query; - } - - public Map getAggregations() { - return aggregations; - } - - public Map getAggs() { - return aggs; - } - - /** - * Convenience method to get either aggregations or aggs. - * - * @return The aggregations (whether initially specified in aggregations - * or aggs), or null if neither are set. - */ - public Map getAggregationsOrAggs() { - return (this.aggregations != null) ? this.aggregations : this.aggs; - } - - public Map getScriptFields() { - return scriptFields; - } - - public Boolean getRetrieveWholeSource() { - return retrieveWholeSource; - } - - public Integer getScrollSize() { - return scrollSize; - } - public SchedulerConfig build() { + ExceptionsHelper.requireNonNull(id, ID.getPreferredName()); + ExceptionsHelper.requireNonNull(jobId, Job.ID.getPreferredName()); + if (!PrelertStrings.isValidId(id)) { + throw new IllegalArgumentException(ID.getPreferredName() + " [" + id + "] contains invalid characters"); + } if (indexes == null || indexes.isEmpty() || indexes.contains(null) || indexes.contains("")) { throw invalidOptionValue(INDEXES.getPreferredName(), indexes); } @@ -585,20 +592,20 @@ public class SchedulerConfig extends ToXContentToBytes implements Writeable { throw invalidOptionValue(TYPES.getPreferredName(), types); } if (aggregations != null && aggs != null) { - String msg = Messages.getMessage(Messages.JOB_CONFIG_SCHEDULER_MULTIPLE_AGGREGATIONS); + String msg = Messages.getMessage(Messages.SCHEDULER_CONFIG_MULTIPLE_AGGREGATIONS); throw new IllegalArgumentException(msg); } if (Boolean.TRUE.equals(retrieveWholeSource)) { if (scriptFields != null) { - throw notSupportedValue(SCRIPT_FIELDS, Messages.JOB_CONFIG_SCHEDULER_FIELD_NOT_SUPPORTED); + throw notSupportedValue(SCRIPT_FIELDS, Messages.SCHEDULER_CONFIG_FIELD_NOT_SUPPORTED); } } - return new SchedulerConfig(queryDelay, frequency, indexes, types, query, aggregations, aggs, scriptFields, retrieveWholeSource, - scrollSize); + return new SchedulerConfig(id, jobId, queryDelay, frequency, indexes, types, query, aggregations, aggs, scriptFields, + retrieveWholeSource, scrollSize); } private static ElasticsearchException invalidOptionValue(String fieldName, Object value) { - String msg = Messages.getMessage(Messages.JOB_CONFIG_SCHEDULER_INVALID_OPTION_VALUE, fieldName, value); + String msg = Messages.getMessage(Messages.SCHEDULER_CONFIG_INVALID_OPTION_VALUE, fieldName, value); throw new IllegalArgumentException(msg); } diff --git a/elasticsearch/src/main/java/org/elasticsearch/xpack/prelert/job/extraction/DataExtractorFactory.java b/elasticsearch/src/main/java/org/elasticsearch/xpack/prelert/job/extraction/DataExtractorFactory.java index 96988d5211a..95b7dc7a742 100644 --- a/elasticsearch/src/main/java/org/elasticsearch/xpack/prelert/job/extraction/DataExtractorFactory.java +++ b/elasticsearch/src/main/java/org/elasticsearch/xpack/prelert/job/extraction/DataExtractorFactory.java @@ -6,7 +6,8 @@ package org.elasticsearch.xpack.prelert.job.extraction; import org.elasticsearch.xpack.prelert.job.Job; +import org.elasticsearch.xpack.prelert.job.SchedulerConfig; public interface DataExtractorFactory { - DataExtractor newExtractor(Job job); + DataExtractor newExtractor(SchedulerConfig schedulerConfig, Job job); } diff --git a/elasticsearch/src/main/java/org/elasticsearch/xpack/prelert/job/manager/JobManager.java b/elasticsearch/src/main/java/org/elasticsearch/xpack/prelert/job/manager/JobManager.java index 3e6ba434714..a0eb136ae1b 100644 --- a/elasticsearch/src/main/java/org/elasticsearch/xpack/prelert/job/manager/JobManager.java +++ b/elasticsearch/src/main/java/org/elasticsearch/xpack/prelert/job/manager/JobManager.java @@ -16,20 +16,21 @@ import org.elasticsearch.common.component.AbstractComponent; import org.elasticsearch.common.logging.Loggers; import org.elasticsearch.common.settings.Settings; import org.elasticsearch.xpack.prelert.action.DeleteJobAction; -import org.elasticsearch.xpack.prelert.action.PutJobAction; import org.elasticsearch.xpack.prelert.action.OpenJobAction; +import org.elasticsearch.xpack.prelert.action.PutJobAction; import org.elasticsearch.xpack.prelert.action.RevertModelSnapshotAction; -import org.elasticsearch.xpack.prelert.action.UpdateJobSchedulerStatusAction; import org.elasticsearch.xpack.prelert.action.UpdateJobStatusAction; +import org.elasticsearch.xpack.prelert.action.UpdateSchedulerStatusAction; import org.elasticsearch.xpack.prelert.job.IgnoreDowntime; import org.elasticsearch.xpack.prelert.job.Job; -import org.elasticsearch.xpack.prelert.job.SchedulerStatus; import org.elasticsearch.xpack.prelert.job.JobStatus; import org.elasticsearch.xpack.prelert.job.ModelSnapshot; +import org.elasticsearch.xpack.prelert.job.SchedulerStatus; import org.elasticsearch.xpack.prelert.job.audit.Auditor; import org.elasticsearch.xpack.prelert.job.messages.Messages; import org.elasticsearch.xpack.prelert.job.metadata.Allocation; import org.elasticsearch.xpack.prelert.job.metadata.PrelertMetadata; +import org.elasticsearch.xpack.prelert.job.metadata.Scheduler; import org.elasticsearch.xpack.prelert.job.persistence.JobProvider; import org.elasticsearch.xpack.prelert.job.persistence.JobResultsPersister; import org.elasticsearch.xpack.prelert.job.persistence.QueryPage; @@ -251,32 +252,31 @@ public class JobManager extends AbstractComponent { ClusterState removeJobFromClusterState(String jobId, ClusterState currentState) { PrelertMetadata.Builder builder = createPrelertMetadataBuilder(currentState); builder.removeJob(jobId); - return buildNewClusterState(currentState, builder); } public Optional getSchedulerStatus(String jobId) { PrelertMetadata prelertMetadata = clusterService.state().metaData().custom(PrelertMetadata.TYPE); - return Optional.ofNullable(prelertMetadata.getSchedulerStatuses().get(jobId)); + return prelertMetadata.getSchedulerStatusByJobId(jobId); } - public void updateSchedulerStatus(UpdateJobSchedulerStatusAction.Request request, - ActionListener actionListener) { - String jobId = request.getJobId(); + public void updateSchedulerStatus(UpdateSchedulerStatusAction.Request request, + ActionListener actionListener) { + String schedulerId = request.getSchedulerId(); SchedulerStatus newStatus = request.getSchedulerStatus(); - clusterService.submitStateUpdateTask("update-scheduler-status-job-" + jobId, - new AckedClusterStateUpdateTask(request, actionListener) { + clusterService.submitStateUpdateTask("update-scheduler-status-" + schedulerId, + new AckedClusterStateUpdateTask(request, actionListener) { @Override public ClusterState execute(ClusterState currentState) throws Exception { PrelertMetadata.Builder builder = createPrelertMetadataBuilder(currentState); - builder.updateSchedulerStatus(jobId, newStatus); + builder.updateSchedulerStatus(schedulerId, newStatus); return buildNewClusterState(currentState, builder); } @Override - protected UpdateJobSchedulerStatusAction.Response newResponse(boolean acknowledged) { - return new UpdateJobSchedulerStatusAction.Response(acknowledged); + protected UpdateSchedulerStatusAction.Response newResponse(boolean acknowledged) { + return new UpdateSchedulerStatusAction.Response(acknowledged); } }); } diff --git a/elasticsearch/src/main/java/org/elasticsearch/xpack/prelert/job/messages/Messages.java b/elasticsearch/src/main/java/org/elasticsearch/xpack/prelert/job/messages/Messages.java index 5797ac0d35d..049069abb2a 100644 --- a/elasticsearch/src/main/java/org/elasticsearch/xpack/prelert/job/messages/Messages.java +++ b/elasticsearch/src/main/java/org/elasticsearch/xpack/prelert/job/messages/Messages.java @@ -62,7 +62,6 @@ public final class Messages { public static final String SYSTEM_AUDIT_SHUTDOWN = "system.audit.shutdown"; public static final String JOB_CANNOT_DELETE_WHILE_RUNNING = "job.cannot.delete.while.running"; - public static final String JOB_CANNOT_DELETE_WHILE_SCHEDULER_RUNS = "job.cannot.delete.while.scheduler.runs"; public static final String JOB_CANNOT_PAUSE = "job.cannot.pause"; public static final String JOB_CANNOT_RESUME = "job.cannot.resume"; @@ -180,17 +179,6 @@ public final class Messages { public static final String JOB_CONFIG_TRANSFORM_UNKNOWN_TYPE = "job.config.transform.unknown.type"; public static final String JOB_CONFIG_UNKNOWN_FUNCTION = "job.config.unknown.function"; - public static final String JOB_CONFIG_SCHEDULER_FIELD_NOT_SUPPORTED = "job.config.scheduler.field.not.supported"; - public static final String JOB_CONFIG_SCHEDULER_INVALID_OPTION_VALUE = "job.config.scheduler.invalid.option.value"; - public static final String JOB_CONFIG_SCHEDULER_REQUIRES_BUCKET_SPAN = "job.config.scheduler.requires.bucket.span"; - public static final String JOB_CONFIG_SCHEDULER_ELASTICSEARCH_DOES_NOT_SUPPORT_LATENCY = "job.config.scheduler.elasticsearch.does.not." - + "support.latency"; - public static final String JOB_CONFIG_SCHEDULER_AGGREGATIONS_REQUIRES_SUMMARY_COUNT_FIELD = "job.config.scheduler.aggregations." - + "requires.summary.count.field"; - public static final String JOB_CONFIG_SCHEDULER_ELASTICSEARCH_REQUIRES_DATAFORMAT_ELASTICSEARCH = "job.config.scheduler.elasticsearch." - + "requires.dataformat.elasticsearch"; - public static final String JOB_CONFIG_SCHEDULER_MULTIPLE_AGGREGATIONS = "job.config.scheduler.multiple.aggregations"; - public static final String JOB_DATA_CONCURRENT_USE_CLOSE = "job.data.concurrent.use.close"; public static final String JOB_DATA_CONCURRENT_USE_FLUSH = "job.data.concurrent.use.flush"; public static final String JOB_DATA_CONCURRENT_USE_PAUSE = "job.data.concurrent.use.pause"; @@ -199,16 +187,22 @@ public final class Messages { public static final String JOB_DATA_CONCURRENT_USE_UPDATE = "job.data.concurrent.use.update"; public static final String JOB_DATA_CONCURRENT_USE_UPLOAD = "job.data.concurrent.use.upload"; - public static final String JOB_SCHEDULER_CANNOT_START = "job.scheduler.cannot.start"; - public static final String JOB_SCHEDULER_CANNOT_STOP_IN_CURRENT_STATE = "job.scheduler.cannot.stop.in.current.state"; - public static final String JOB_SCHEDULER_CANNOT_UPDATE_IN_CURRENT_STATE = "job.scheduler.cannot.update.in.current.state"; - public static final String JOB_SCHEDULER_FAILED_TO_STOP = "job.scheduler.failed.to.stop"; - public static final String JOB_SCHEDULER_NO_SUCH_SCHEDULED_JOB = "job.scheduler.no.such.scheduled.job"; - public static final String JOB_SCHEDULER_STATUS_STARTED = "job.scheduler.status.started"; - public static final String JOB_SCHEDULER_STATUS_STOPPING = "job.scheduler.status.stopping"; - public static final String JOB_SCHEDULER_STATUS_STOPPED = "job.scheduler.status.stopped"; - public static final String JOB_SCHEDULER_STATUS_UPDATING = "job.scheduler.status.updating"; - public static final String JOB_SCHEDULER_STATUS_DELETING = "job.scheduler.status.deleting"; + public static final String SCHEDULER_CONFIG_FIELD_NOT_SUPPORTED = "scheduler.config.field.not.supported"; + public static final String SCHEDULER_CONFIG_INVALID_OPTION_VALUE = "scheduler.config.invalid.option.value"; + public static final String SCHEDULER_CONFIG_MULTIPLE_AGGREGATIONS = "scheduler.config.multiple.aggregations"; + + public static final String SCHEDULER_DOES_NOT_SUPPORT_JOB_WITH_LATENCY = "scheduler.does.not.support.job.with.latency"; + public static final String SCHEDULER_AGGREGATIONS_REQUIRES_JOB_WITH_SUMMARY_COUNT_FIELD = + "scheduler.aggregations.requires.job.with.summary.count.field"; + public static final String SCHEDULER_REQUIRES_JOB_WITH_DATAFORMAT_ELASTICSEARCH = + "scheduler.requires.job.with.dataformat.elasticsearch"; + + public static final String SCHEDULER_CANNOT_START = "scheduler.cannot.start"; + public static final String SCHEDULER_CANNOT_STOP_IN_CURRENT_STATE = "scheduler.cannot.stop.in.current.state"; + public static final String SCHEDULER_CANNOT_UPDATE_IN_CURRENT_STATE = "scheduler.cannot.update.in.current.state"; + public static final String SCHEDULER_CANNOT_DELETE_IN_CURRENT_STATE = "scheduler.cannot.delete.in.current.state"; + public static final String SCHEDULER_FAILED_TO_STOP = "scheduler.failed.to.stop"; + public static final String SCHEDULER_NOT_FOUND = "scheduler.not.found"; public static final String JOB_MISSING_QUANTILES = "job.missing.quantiles"; public static final String JOB_UNKNOWN_ID = "job.unknown.id"; diff --git a/elasticsearch/src/main/java/org/elasticsearch/xpack/prelert/job/metadata/PrelertMetadata.java b/elasticsearch/src/main/java/org/elasticsearch/xpack/prelert/job/metadata/PrelertMetadata.java index 661587b299c..8ef4c4420a8 100644 --- a/elasticsearch/src/main/java/org/elasticsearch/xpack/prelert/job/metadata/PrelertMetadata.java +++ b/elasticsearch/src/main/java/org/elasticsearch/xpack/prelert/job/metadata/PrelertMetadata.java @@ -5,8 +5,8 @@ */ package org.elasticsearch.xpack.prelert.job.metadata; +import org.elasticsearch.ResourceAlreadyExistsException; import org.elasticsearch.ResourceNotFoundException; -import org.elasticsearch.cluster.AbstractDiffable; import org.elasticsearch.cluster.Diff; import org.elasticsearch.cluster.DiffableUtils; import org.elasticsearch.cluster.metadata.MetaData; @@ -18,12 +18,15 @@ 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.ObjectParser; +import org.elasticsearch.common.xcontent.ToXContent; import org.elasticsearch.common.xcontent.XContentBuilder; import org.elasticsearch.common.xcontent.XContentParser; import org.elasticsearch.xpack.prelert.job.Job; -import org.elasticsearch.xpack.prelert.job.SchedulerStatus; import org.elasticsearch.xpack.prelert.job.JobStatus; +import org.elasticsearch.xpack.prelert.job.SchedulerConfig; +import org.elasticsearch.xpack.prelert.job.SchedulerStatus; import org.elasticsearch.xpack.prelert.job.messages.Messages; +import org.elasticsearch.xpack.prelert.job.scheduler.ScheduledJobValidator; import org.elasticsearch.xpack.prelert.utils.ExceptionsHelper; import java.io.IOException; @@ -33,14 +36,15 @@ import java.util.Date; import java.util.EnumSet; import java.util.Map; import java.util.Objects; +import java.util.Optional; import java.util.SortedMap; import java.util.TreeMap; -import java.util.stream.Collectors; public class PrelertMetadata 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 SCHEDULERS_FIELD = new ParseField("schedulers"); public static final String TYPE = "prelert"; public static final PrelertMetadata PROTO = new PrelertMetadata(Collections.emptySortedMap(), Collections.emptySortedMap(), @@ -52,17 +56,18 @@ public class PrelertMetadata implements MetaData.Custom { static { PRELERT_METADATA_PARSER.declareObjectArray(Builder::putJobs, (p, c) -> Job.PARSER.apply(p, c).build(), JOBS_FIELD); PRELERT_METADATA_PARSER.declareObjectArray(Builder::putAllocations, Allocation.PARSER, ALLOCATIONS_FIELD); + PRELERT_METADATA_PARSER.declareObjectArray(Builder::putSchedulers, Scheduler.PARSER, SCHEDULERS_FIELD); } private final SortedMap jobs; private final SortedMap allocations; - private final SortedMap schedulerStatuses; + private final SortedMap schedulers; private PrelertMetadata(SortedMap jobs, SortedMap allocations, - SortedMap schedulerStatuses) { + SortedMap schedulers) { this.jobs = Collections.unmodifiableSortedMap(jobs); this.allocations = Collections.unmodifiableSortedMap(allocations); - this.schedulerStatuses = Collections.unmodifiableSortedMap(schedulerStatuses); + this.schedulers = Collections.unmodifiableSortedMap(schedulers); } public Map getJobs() { @@ -73,8 +78,21 @@ public class PrelertMetadata implements MetaData.Custom { return allocations; } - public SortedMap getSchedulerStatuses() { - return schedulerStatuses; + public SortedMap getSchedulers() { + return schedulers; + } + + public Scheduler getScheduler(String schedulerId) { + return schedulers.get(schedulerId); + } + + public Optional getSchedulerByJobId(String jobId) { + return schedulers.values().stream().filter(s -> s.getJobId().equals(jobId)).findFirst(); + } + + public Optional getSchedulerStatusByJobId(String jobId) { + Optional scheduler = getSchedulerByJobId(jobId); + return Optional.ofNullable(scheduler.isPresent() ? scheduler.get().getStatus() : null); } @Override @@ -117,27 +135,23 @@ public class PrelertMetadata implements MetaData.Custom { allocations.put(in.readString(), Allocation.PROTO.readFrom(in)); } size = in.readVInt(); - TreeMap schedulerStatuses = new TreeMap<>(); + TreeMap schedulers = new TreeMap<>(); for (int i = 0; i < size; i++) { - schedulerStatuses.put(in.readString(), SchedulerStatus.fromStream(in)); + schedulers.put(in.readString(), new Scheduler(in)); } - return new PrelertMetadata(jobs, allocations, schedulerStatuses); + return new PrelertMetadata(jobs, allocations, schedulers); } @Override public void writeTo(StreamOutput out) throws IOException { - out.writeVInt(jobs.size()); - for (Map.Entry entry : jobs.entrySet()) { - out.writeString(entry.getKey()); - entry.getValue().writeTo(out); - } - out.writeVInt(allocations.size()); - for (Map.Entry entry : allocations.entrySet()) { - out.writeString(entry.getKey()); - entry.getValue().writeTo(out); - } - out.writeVInt(schedulerStatuses.size()); - for (Map.Entry entry : schedulerStatuses.entrySet()) { + writeMap(jobs, out); + writeMap(allocations, out); + writeMap(schedulers, out); + } + + private static void writeMap(Map map, StreamOutput out) throws IOException { + out.writeVInt(map.size()); + for (Map.Entry entry : map.entrySet()) { out.writeString(entry.getKey()); entry.getValue().writeTo(out); } @@ -145,100 +159,52 @@ public class PrelertMetadata implements MetaData.Custom { @Override public XContentBuilder toXContent(XContentBuilder builder, Params params) throws IOException { - builder.startArray(JOBS_FIELD.getPreferredName()); - for (Job job : jobs.values()) { - builder.value(job); - } - builder.endArray(); - builder.startArray(ALLOCATIONS_FIELD.getPreferredName()); - for (Map.Entry entry : allocations.entrySet()) { - builder.value(entry.getValue()); - } - builder.endArray(); + mapValuesToXContent(JOBS_FIELD, jobs, builder, params); + mapValuesToXContent(ALLOCATIONS_FIELD, allocations, builder, params); + mapValuesToXContent(SCHEDULERS_FIELD, schedulers, builder, params); return builder; } + private static void mapValuesToXContent(ParseField field, Map map, XContentBuilder builder, + Params params) throws IOException { + builder.startArray(field.getPreferredName()); + for (Map.Entry entry : map.entrySet()) { + entry.getValue().toXContent(builder, params); + } + builder.endArray(); + } + static class PrelertMetadataDiff implements Diff { final Diff> jobs; final Diff> allocations; - final Diff> schedulerStatuses; + final Diff> schedulers; PrelertMetadataDiff(PrelertMetadata before, PrelertMetadata after) { this.jobs = DiffableUtils.diff(before.jobs, after.jobs, DiffableUtils.getStringKeySerializer()); this.allocations = DiffableUtils.diff(before.allocations, after.allocations, DiffableUtils.getStringKeySerializer()); - this.schedulerStatuses = DiffableUtils.diff( - toSchedulerDiff(before.schedulerStatuses), - toSchedulerDiff(after.schedulerStatuses), - DiffableUtils.getStringKeySerializer()); + this.schedulers = DiffableUtils.diff(before.schedulers, after.schedulers, DiffableUtils.getStringKeySerializer()); } PrelertMetadataDiff(StreamInput in) throws IOException { jobs = DiffableUtils.readJdkMapDiff(in, DiffableUtils.getStringKeySerializer(), Job.PROTO); allocations = DiffableUtils.readJdkMapDiff(in, DiffableUtils.getStringKeySerializer(), Allocation.PROTO); - schedulerStatuses = DiffableUtils.readJdkMapDiff(in, DiffableUtils.getStringKeySerializer(), SchedulerStatusDiff.PROTO); + schedulers = DiffableUtils.readJdkMapDiff(in, DiffableUtils.getStringKeySerializer(), Scheduler.PROTO); } @Override public MetaData.Custom apply(MetaData.Custom part) { TreeMap newJobs = new TreeMap<>(jobs.apply(((PrelertMetadata) part).jobs)); TreeMap newAllocations = new TreeMap<>(allocations.apply(((PrelertMetadata) part).allocations)); - - Map newSchedulerStatuses = - schedulerStatuses.apply(toSchedulerDiff((((PrelertMetadata) part)).schedulerStatuses)); - return new PrelertMetadata(newJobs, newAllocations, new TreeMap<>(toSchedulerStatusMap(newSchedulerStatuses))); + TreeMap newSchedulers = new TreeMap<>(schedulers.apply(((PrelertMetadata) part).schedulers)); + return new PrelertMetadata(newJobs, newAllocations, newSchedulers); } @Override public void writeTo(StreamOutput out) throws IOException { jobs.writeTo(out); allocations.writeTo(out); - schedulerStatuses.writeTo(out); - } - - private static Map toSchedulerDiff(Map from) { - return from.entrySet().stream() - .collect(Collectors.toMap(Map.Entry::getKey, (entry) -> new SchedulerStatusDiff(entry.getValue()))); - } - - private static Map toSchedulerStatusMap(Map from) { - return from.entrySet().stream() - .collect(Collectors.toMap(Map.Entry::getKey, (entry) -> entry.getValue().status)); - } - - // SchedulerStatus is enum and that can't extend from anything - static class SchedulerStatusDiff extends AbstractDiffable implements Writeable { - - static SchedulerStatusDiff PROTO = new SchedulerStatusDiff(null); - - private final SchedulerStatus status; - - SchedulerStatusDiff(SchedulerStatus status) { - this.status = status; - } - - @Override - public SchedulerStatusDiff readFrom(StreamInput in) throws IOException { - return new SchedulerStatusDiff(SchedulerStatus.fromStream(in)); - } - - @Override - public void writeTo(StreamOutput out) throws IOException { - status.writeTo(out); - } - - @Override - public boolean equals(Object o) { - if (this == o) return true; - if (o == null || getClass() != o.getClass()) return false; - SchedulerStatusDiff that = (SchedulerStatusDiff) o; - return status == that.status; - } - - @Override - public int hashCode() { - return Objects.hash(status); - } + schedulers.writeTo(out); } } @@ -251,30 +217,30 @@ public class PrelertMetadata implements MetaData.Custom { PrelertMetadata that = (PrelertMetadata) o; return Objects.equals(jobs, that.jobs) && Objects.equals(allocations, that.allocations) && - Objects.equals(schedulerStatuses, that.schedulerStatuses); + Objects.equals(schedulers, that.schedulers); } @Override public int hashCode() { - return Objects.hash(jobs, allocations, schedulerStatuses); + return Objects.hash(jobs, allocations, schedulers); } public static class Builder { private TreeMap jobs; private TreeMap allocations; - private TreeMap schedulerStatuses; + private TreeMap schedulers; public Builder() { this.jobs = new TreeMap<>(); this.allocations = new TreeMap<>(); - this.schedulerStatuses = new TreeMap<>(); + this.schedulers = new TreeMap<>(); } public Builder(PrelertMetadata previous) { jobs = new TreeMap<>(previous.jobs); allocations = new TreeMap<>(previous.allocations); - schedulerStatuses = new TreeMap<>(previous.schedulerStatuses); + schedulers = new TreeMap<>(previous.schedulers); } public Builder putJob(Job job, boolean overwrite) { @@ -289,9 +255,6 @@ public class PrelertMetadata implements MetaData.Custom { builder.setStatus(JobStatus.CLOSED); allocations.put(job.getId(), builder.build()); } - if (job.getSchedulerConfig() != null) { - schedulerStatuses.put(job.getId(), SchedulerStatus.STOPPED); - } return this; } @@ -299,6 +262,13 @@ public class PrelertMetadata implements MetaData.Custom { if (jobs.remove(jobId) == null) { throw new ResourceNotFoundException("job [" + jobId + "] does not exist"); } + + Optional scheduler = getSchedulerByJobId(jobId); + if (scheduler.isPresent()) { + throw ExceptionsHelper.conflictStatusException("Cannot delete job [" + jobId + "] while scheduler [" + + scheduler.get().getId() + "] refers to it"); + } + Allocation previousAllocation = this.allocations.remove(jobId); if (previousAllocation != null) { if (!previousAllocation.getStatus().isAnyOf(JobStatus.CLOSED, JobStatus.FAILED)) { @@ -306,16 +276,50 @@ public class PrelertMetadata implements MetaData.Custom { Messages.JOB_CANNOT_DELETE_WHILE_RUNNING, jobId, previousAllocation.getStatus())); } } - SchedulerStatus previousStatus = this.schedulerStatuses.remove(jobId); - if (previousStatus != null) { - if (previousStatus != SchedulerStatus.STOPPED) { - String message = Messages.getMessage(Messages.JOB_CANNOT_DELETE_WHILE_SCHEDULER_RUNS, jobId); - throw ExceptionsHelper.conflictStatusException(message); - } - } + return this; } + public Builder putScheduler(SchedulerConfig schedulerConfig) { + if (schedulers.containsKey(schedulerConfig.getId())) { + throw new ResourceAlreadyExistsException("A scheduler with id [" + schedulerConfig.getId() + "] already exists"); + } + String jobId = schedulerConfig.getJobId(); + Job job = jobs.get(jobId); + if (job == null) { + throw ExceptionsHelper.missingJobException(jobId); + } + Optional existingScheduler = getSchedulerByJobId(jobId); + if (existingScheduler.isPresent()) { + throw ExceptionsHelper.conflictStatusException("A scheduler [" + existingScheduler.get().getId() + + "] already exists for job [" + jobId + "]"); + } + ScheduledJobValidator.validate(schedulerConfig, job); + return putScheduler(new Scheduler(schedulerConfig, SchedulerStatus.STOPPED)); + } + + private Builder putScheduler(Scheduler scheduler) { + schedulers.put(scheduler.getId(), scheduler); + return this; + } + + public Builder removeScheduler(String schedulerId) { + Scheduler scheduler = schedulers.get(schedulerId); + if (scheduler == null) { + throw new ResourceNotFoundException(Messages.getMessage(Messages.SCHEDULER_NOT_FOUND, schedulerId)); + } + if (scheduler.getStatus() != SchedulerStatus.STOPPED) { + String msg = Messages.getMessage(Messages.SCHEDULER_CANNOT_DELETE_IN_CURRENT_STATE, schedulerId, scheduler.getStatus()); + throw ExceptionsHelper.conflictStatusException(msg); + } + schedulers.remove(schedulerId); + return this; + } + + private Optional getSchedulerByJobId(String jobId) { + return schedulers.values().stream().filter(s -> s.getJobId().equals(jobId)).findFirst(); + } + // only for parsing private Builder putAllocations(Collection allocations) { for (Allocation.Builder allocationBuilder : allocations) { @@ -332,8 +336,15 @@ public class PrelertMetadata implements MetaData.Custom { return this; } + private Builder putSchedulers(Collection schedulers) { + for (Scheduler scheduler : schedulers) { + putScheduler(scheduler); + } + return this; + } + public PrelertMetadata build() { - return new PrelertMetadata(jobs, allocations, schedulerStatuses); + return new PrelertMetadata(jobs, allocations, schedulers); } public Builder assignToNode(String jobId, String nodeId) { @@ -385,29 +396,30 @@ public class PrelertMetadata implements MetaData.Custom { return this; } - public Builder updateSchedulerStatus(String jobId, SchedulerStatus newStatus) { - SchedulerStatus currentStatus = schedulerStatuses.get(jobId); - if (currentStatus == null) { - throw new IllegalArgumentException(Messages.getMessage(Messages.JOB_SCHEDULER_NO_SUCH_SCHEDULED_JOB, jobId)); + public Builder updateSchedulerStatus(String schedulerId, SchedulerStatus newStatus) { + Scheduler scheduler = schedulers.get(schedulerId); + if (scheduler == null) { + throw new ResourceNotFoundException(Messages.getMessage(Messages.SCHEDULER_NOT_FOUND, schedulerId)); } + SchedulerStatus currentStatus = scheduler.getStatus(); switch (newStatus) { case STARTED: if (currentStatus != SchedulerStatus.STOPPED) { - String msg = Messages.getMessage(Messages.JOB_SCHEDULER_CANNOT_START, jobId, newStatus); + String msg = Messages.getMessage(Messages.SCHEDULER_CANNOT_START, schedulerId, newStatus); throw ExceptionsHelper.conflictStatusException(msg); } break; case STOPPED: if (currentStatus != SchedulerStatus.STARTED) { - String msg = Messages.getMessage(Messages.JOB_SCHEDULER_CANNOT_STOP_IN_CURRENT_STATE, jobId, newStatus); + String msg = Messages.getMessage(Messages.SCHEDULER_CANNOT_STOP_IN_CURRENT_STATE, schedulerId, newStatus); throw ExceptionsHelper.conflictStatusException(msg); } break; default: - throw new IllegalArgumentException("[" + jobId + "] invalid requested job scheduler status [" + newStatus + "]"); + throw new IllegalArgumentException("[" + schedulerId + "] requested invalid scheduler status [" + newStatus + "]"); } - schedulerStatuses.put(jobId, newStatus); + schedulers.put(schedulerId, new Scheduler(scheduler.getConfig(), newStatus)); return this; } } diff --git a/elasticsearch/src/main/java/org/elasticsearch/xpack/prelert/job/metadata/Scheduler.java b/elasticsearch/src/main/java/org/elasticsearch/xpack/prelert/job/metadata/Scheduler.java new file mode 100644 index 00000000000..bc176f7a89a --- /dev/null +++ b/elasticsearch/src/main/java/org/elasticsearch/xpack/prelert/job/metadata/Scheduler.java @@ -0,0 +1,101 @@ +/* + * 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.prelert.job.metadata; + +import org.elasticsearch.cluster.AbstractDiffable; +import org.elasticsearch.common.ParseField; +import org.elasticsearch.common.ParseFieldMatcherSupplier; +import org.elasticsearch.common.io.stream.StreamInput; +import org.elasticsearch.common.io.stream.StreamOutput; +import org.elasticsearch.common.xcontent.ConstructingObjectParser; +import org.elasticsearch.common.xcontent.ObjectParser; +import org.elasticsearch.common.xcontent.ToXContent; +import org.elasticsearch.common.xcontent.XContentBuilder; +import org.elasticsearch.xpack.prelert.job.SchedulerConfig; +import org.elasticsearch.xpack.prelert.job.SchedulerStatus; + +import java.io.IOException; +import java.util.Objects; + +public class Scheduler extends AbstractDiffable implements ToXContent { + + private static final ParseField CONFIG_FIELD = new ParseField("config"); + private static final ParseField STATUS_FIELD = new ParseField("status"); + + static final Scheduler PROTO = new Scheduler(null, null); + + static final ConstructingObjectParser PARSER = new ConstructingObjectParser<>("scheduler", + a -> new Scheduler((SchedulerConfig) a[0], (SchedulerStatus) a[1])); + + static { + PARSER.declareObject(ConstructingObjectParser.constructorArg(), SchedulerConfig.PARSER, CONFIG_FIELD); + PARSER.declareField(ConstructingObjectParser.constructorArg(), (p, c) -> SchedulerStatus.fromString(p.text()), STATUS_FIELD, + ObjectParser.ValueType.STRING); + } + + private final SchedulerConfig config; + private final SchedulerStatus status; + + public Scheduler(SchedulerConfig config, SchedulerStatus status) { + this.config = config; + this.status = status; + } + + public Scheduler(StreamInput in) throws IOException { + this.config = new SchedulerConfig(in); + this.status = SchedulerStatus.fromStream(in); + } + + public String getId() { + return config.getId(); + } + + public String getJobId() { + return config.getJobId(); + } + + public SchedulerConfig getConfig() { + return config; + } + + public SchedulerStatus getStatus() { + return status; + } + + @Override + public Scheduler readFrom(StreamInput in) throws IOException { + return new Scheduler(in); + } + + @Override + public void writeTo(StreamOutput out) throws IOException { + config.writeTo(out); + status.writeTo(out); + } + + @Override + public XContentBuilder toXContent(XContentBuilder builder, Params params) throws IOException { + builder.startObject(); + builder.field(CONFIG_FIELD.getPreferredName(), config); + builder.field(STATUS_FIELD.getPreferredName(), status); + builder.endObject(); + return builder; + } + + @Override + public boolean equals(Object o) { + if (this == o) return true; + if (o == null || getClass() != o.getClass()) return false; + Scheduler that = (Scheduler) o; + return Objects.equals(config, that.config) && + Objects.equals(status, that.status); + } + + @Override + public int hashCode() { + return Objects.hash(config, status); + } +} diff --git a/elasticsearch/src/main/java/org/elasticsearch/xpack/prelert/job/process/autodetect/AutodetectCommunicator.java b/elasticsearch/src/main/java/org/elasticsearch/xpack/prelert/job/process/autodetect/AutodetectCommunicator.java index d7d27d2deaf..01399fa954a 100644 --- a/elasticsearch/src/main/java/org/elasticsearch/xpack/prelert/job/process/autodetect/AutodetectCommunicator.java +++ b/elasticsearch/src/main/java/org/elasticsearch/xpack/prelert/job/process/autodetect/AutodetectCommunicator.java @@ -70,7 +70,7 @@ public class AutodetectCommunicator implements Closeable { private DataToProcessWriter createProcessWriter(Job job, AutodetectProcess process, StatusReporter statusReporter) { return DataToProcessWriterFactory.create(true, process, job.getDataDescription(), job.getAnalysisConfig(), - job.getSchedulerConfig(), new TransformConfigs(job.getTransforms()) , statusReporter, LOGGER); + new TransformConfigs(job.getTransforms()) , statusReporter, LOGGER); } public DataCounts writeToJob(InputStream inputStream, DataLoadParams params, Supplier cancelled) throws IOException { diff --git a/elasticsearch/src/main/java/org/elasticsearch/xpack/prelert/job/process/autodetect/writer/DataToProcessWriterFactory.java b/elasticsearch/src/main/java/org/elasticsearch/xpack/prelert/job/process/autodetect/writer/DataToProcessWriterFactory.java index d2d66cdee4c..ce6b0a26301 100644 --- a/elasticsearch/src/main/java/org/elasticsearch/xpack/prelert/job/process/autodetect/writer/DataToProcessWriterFactory.java +++ b/elasticsearch/src/main/java/org/elasticsearch/xpack/prelert/job/process/autodetect/writer/DataToProcessWriterFactory.java @@ -33,14 +33,13 @@ public final class DataToProcessWriterFactory { * format is JSON or otherwise a {@link CsvDataToProcessWriter} */ public static DataToProcessWriter create(boolean includeControlField, AutodetectProcess autodetectProcess, - DataDescription dataDescription, AnalysisConfig analysisConfig, - SchedulerConfig schedulerConfig, TransformConfigs transforms, - StatusReporter statusReporter, Logger logger) { + DataDescription dataDescription, AnalysisConfig analysisConfig, + TransformConfigs transforms, StatusReporter statusReporter, Logger logger) { switch (dataDescription.getFormat()) { case JSON: case ELASTICSEARCH: return new JsonDataToProcessWriter(includeControlField, autodetectProcess, dataDescription, analysisConfig, - schedulerConfig, transforms, statusReporter, logger); + transforms, statusReporter, logger); case DELIMITED: return new CsvDataToProcessWriter(includeControlField, autodetectProcess, dataDescription, analysisConfig, transforms, statusReporter, logger); diff --git a/elasticsearch/src/main/java/org/elasticsearch/xpack/prelert/job/process/autodetect/writer/JsonDataToProcessWriter.java b/elasticsearch/src/main/java/org/elasticsearch/xpack/prelert/job/process/autodetect/writer/JsonDataToProcessWriter.java index e31c09804ec..be2937a0ba3 100644 --- a/elasticsearch/src/main/java/org/elasticsearch/xpack/prelert/job/process/autodetect/writer/JsonDataToProcessWriter.java +++ b/elasticsearch/src/main/java/org/elasticsearch/xpack/prelert/job/process/autodetect/writer/JsonDataToProcessWriter.java @@ -5,28 +5,24 @@ */ package org.elasticsearch.xpack.prelert.job.process.autodetect.writer; -import java.io.IOException; -import java.io.InputStream; -import java.util.Arrays; -import java.util.Collection; -import java.util.Collections; -import java.util.List; -import java.util.Map; -import java.util.function.Supplier; - -import org.apache.logging.log4j.Logger; - import com.fasterxml.jackson.core.JsonFactory; import com.fasterxml.jackson.core.JsonParser; +import org.apache.logging.log4j.Logger; import org.elasticsearch.xpack.prelert.job.AnalysisConfig; import org.elasticsearch.xpack.prelert.job.DataCounts; import org.elasticsearch.xpack.prelert.job.DataDescription; import org.elasticsearch.xpack.prelert.job.DataDescription.DataFormat; -import org.elasticsearch.xpack.prelert.job.SchedulerConfig; import org.elasticsearch.xpack.prelert.job.process.autodetect.AutodetectProcess; import org.elasticsearch.xpack.prelert.job.status.StatusReporter; import org.elasticsearch.xpack.prelert.job.transform.TransformConfigs; +import java.io.IOException; +import java.io.InputStream; +import java.util.Arrays; +import java.util.Collection; +import java.util.Map; +import java.util.function.Supplier; + /** * A writer for transforming and piping JSON data from an * inputstream to outputstream. @@ -38,18 +34,10 @@ import org.elasticsearch.xpack.prelert.job.transform.TransformConfigs; class JsonDataToProcessWriter extends AbstractDataToProcessWriter { private static final String ELASTICSEARCH_SOURCE_FIELD = "_source"; - /** - * Scheduler config. May be null. - */ - private SchedulerConfig schedulerConfig; - - public JsonDataToProcessWriter(boolean includeControlField, AutodetectProcess autodetectProcess, - DataDescription dataDescription, AnalysisConfig analysisConfig, - SchedulerConfig schedulerConfig, TransformConfigs transforms, - StatusReporter statusReporter, Logger logger) { - super(includeControlField, autodetectProcess, dataDescription, analysisConfig, transforms, - statusReporter, logger); - this.schedulerConfig = schedulerConfig; + public JsonDataToProcessWriter(boolean includeControlField, AutodetectProcess autodetectProcess, DataDescription dataDescription, + AnalysisConfig analysisConfig, TransformConfigs transforms, StatusReporter statusReporter, + Logger logger) { + super(includeControlField, autodetectProcess, dataDescription, analysisConfig, transforms, statusReporter, logger); } /** @@ -85,7 +73,7 @@ class JsonDataToProcessWriter extends AbstractDataToProcessWriter { // We never expect to get the control field boolean[] gotFields = new boolean[analysisFields.size()]; - JsonRecordReader recordReader = makeRecordReader(parser); + JsonRecordReader recordReader = new SimpleJsonRecordReader(parser, inFieldIndexes, getRecordHoldingField(), logger); long inputFieldCount = recordReader.read(input, gotFields); while (inputFieldCount >= 0) { Arrays.fill(record, ""); @@ -110,25 +98,11 @@ class JsonDataToProcessWriter extends AbstractDataToProcessWriter { private String getRecordHoldingField() { if (dataDescription.getFormat().equals(DataFormat.ELASTICSEARCH)) { - if (schedulerConfig != null) { - if (schedulerConfig.getAggregationsOrAggs() != null) { - return SchedulerConfig.AGGREGATIONS.getPreferredName(); - } - } return ELASTICSEARCH_SOURCE_FIELD; } return ""; } - // TODO norelease: Feels like this is checked in the wrong place. The fact that there is a different format, should - // be specified to this class and this class shouldn't know about the existence of SchedulerConfig - private JsonRecordReader makeRecordReader(JsonParser parser) { - List nestingOrder = (schedulerConfig != null) ? - schedulerConfig.buildAggregatedFieldList() : Collections.emptyList(); - return nestingOrder.isEmpty() ? new SimpleJsonRecordReader(parser, inFieldIndexes, getRecordHoldingField(), logger) - : new AggregatedJsonRecordReader(parser, inFieldIndexes, getRecordHoldingField(), logger, nestingOrder); - } - /** * Don't enforce the check that all the fields are present in JSON docs. * Always returns true diff --git a/elasticsearch/src/main/java/org/elasticsearch/xpack/prelert/job/scheduler/ScheduledJobRunner.java b/elasticsearch/src/main/java/org/elasticsearch/xpack/prelert/job/scheduler/ScheduledJobRunner.java index d261d534999..98cd35426ca 100644 --- a/elasticsearch/src/main/java/org/elasticsearch/xpack/prelert/job/scheduler/ScheduledJobRunner.java +++ b/elasticsearch/src/main/java/org/elasticsearch/xpack/prelert/job/scheduler/ScheduledJobRunner.java @@ -17,18 +17,20 @@ import org.elasticsearch.common.util.concurrent.FutureUtils; import org.elasticsearch.rest.RestStatus; import org.elasticsearch.threadpool.ThreadPool; import org.elasticsearch.xpack.prelert.PrelertPlugin; -import org.elasticsearch.xpack.prelert.action.StartJobSchedulerAction; -import org.elasticsearch.xpack.prelert.action.UpdateJobSchedulerStatusAction; +import org.elasticsearch.xpack.prelert.action.StartSchedulerAction; +import org.elasticsearch.xpack.prelert.action.UpdateSchedulerStatusAction; import org.elasticsearch.xpack.prelert.job.DataCounts; import org.elasticsearch.xpack.prelert.job.Job; -import org.elasticsearch.xpack.prelert.job.SchedulerStatus; import org.elasticsearch.xpack.prelert.job.JobStatus; +import org.elasticsearch.xpack.prelert.job.SchedulerStatus; import org.elasticsearch.xpack.prelert.job.audit.Auditor; import org.elasticsearch.xpack.prelert.job.config.DefaultFrequency; import org.elasticsearch.xpack.prelert.job.extraction.DataExtractor; import org.elasticsearch.xpack.prelert.job.extraction.DataExtractorFactory; +import org.elasticsearch.xpack.prelert.job.messages.Messages; import org.elasticsearch.xpack.prelert.job.metadata.Allocation; import org.elasticsearch.xpack.prelert.job.metadata.PrelertMetadata; +import org.elasticsearch.xpack.prelert.job.metadata.Scheduler; import org.elasticsearch.xpack.prelert.job.persistence.BucketsQueryBuilder; import org.elasticsearch.xpack.prelert.job.persistence.JobProvider; import org.elasticsearch.xpack.prelert.job.persistence.QueryPage; @@ -61,14 +63,16 @@ public class ScheduledJobRunner extends AbstractComponent { this.currentTimeSupplier = Objects.requireNonNull(currentTimeSupplier); } - public void run(String jobId, long startTime, Long endTime, StartJobSchedulerAction.SchedulerTask task, Consumer handler) { + public void run(String schedulerId, long startTime, Long endTime, StartSchedulerAction.SchedulerTask task, + Consumer handler) { PrelertMetadata prelertMetadata = clusterService.state().metaData().custom(PrelertMetadata.TYPE); - validate(jobId, prelertMetadata); + validate(schedulerId, prelertMetadata); - setJobSchedulerStatus(jobId, SchedulerStatus.STARTED, error -> { - logger.info("[{}] Starting scheduler", jobId); - Job job = prelertMetadata.getJobs().get(jobId); - Holder holder = createJobScheduler(job, task, handler); + setJobSchedulerStatus(schedulerId, SchedulerStatus.STARTED, error -> { + Scheduler scheduler = prelertMetadata.getScheduler(schedulerId); + logger.info("Starting scheduler [{}] for job [{}]", schedulerId, scheduler.getJobId()); + Job job = prelertMetadata.getJobs().get(scheduler.getJobId()); + Holder holder = createJobScheduler(scheduler, job, handler); task.setHolder(holder); holder.future = threadPool.executor(PrelertPlugin.SCHEDULER_THREAD_POOL_NAME).submit(() -> { try { @@ -87,7 +91,7 @@ public class ScheduledJobRunner extends AbstractComponent { holder.stop(); } } catch (Exception e) { - logger.error("Failed lookback import for job[" + job.getId() + "]", e); + logger.error("Failed lookback import for job [" + job.getId() + "]", e); holder.stop(); } holder.problemTracker.finishReport(); @@ -129,39 +133,40 @@ public class ScheduledJobRunner extends AbstractComponent { } } - public static void validate(String jobId, PrelertMetadata prelertMetadata) { - Job job = prelertMetadata.getJobs().get(jobId); + public static void validate(String schedulerId, PrelertMetadata prelertMetadata) { + Scheduler scheduler = prelertMetadata.getScheduler(schedulerId); + if (scheduler == null) { + throw new ResourceNotFoundException(Messages.getMessage(Messages.SCHEDULER_NOT_FOUND, schedulerId)); + } + Job job = prelertMetadata.getJobs().get(scheduler.getJobId()); if (job == null) { - throw ExceptionsHelper.missingJobException(jobId); + throw ExceptionsHelper.missingJobException(scheduler.getJobId()); } - if (job.getSchedulerConfig() == null) { - throw new IllegalArgumentException("job [" + job.getId() + "] is not a scheduled job"); - } - - Allocation allocation = prelertMetadata.getAllocations().get(jobId); + Allocation allocation = prelertMetadata.getAllocations().get(scheduler.getJobId()); if (allocation.getStatus() != JobStatus.OPENED) { throw new ElasticsearchStatusException("cannot start scheduler, expected job status [{}], but got [{}]", RestStatus.CONFLICT, JobStatus.OPENED, allocation.getStatus()); } - SchedulerStatus status = prelertMetadata.getSchedulerStatuses().get(jobId); + SchedulerStatus status = scheduler.getStatus(); if (status != SchedulerStatus.STOPPED) { throw new ElasticsearchStatusException("scheduler already started, expected scheduler status [{}], but got [{}]", RestStatus.CONFLICT, SchedulerStatus.STOPPED, status); } + ScheduledJobValidator.validate(scheduler.getConfig(), job); } - private Holder createJobScheduler(Job job, StartJobSchedulerAction.SchedulerTask task, Consumer handler) { + private Holder createJobScheduler(Scheduler scheduler, Job job, Consumer handler) { Auditor auditor = jobProvider.audit(job.getId()); - Duration frequency = getFrequencyOrDefault(job); - Duration queryDelay = Duration.ofSeconds(job.getSchedulerConfig().getQueryDelay()); - DataExtractor dataExtractor = dataExtractorFactory.newExtractor(job); + Duration frequency = getFrequencyOrDefault(scheduler, job); + Duration queryDelay = Duration.ofSeconds(scheduler.getConfig().getQueryDelay()); + DataExtractor dataExtractor = dataExtractorFactory.newExtractor(scheduler.getConfig(), job); ScheduledJob scheduledJob = new ScheduledJob(job.getId(), frequency.toMillis(), queryDelay.toMillis(), dataExtractor, client, auditor, currentTimeSupplier, getLatestFinalBucketEndTimeMs(job), getLatestRecordTimestamp(job.getId())); - return new Holder(job, scheduledJob, new ProblemTracker(() -> auditor), handler); + return new Holder(scheduler, scheduledJob, new ProblemTracker(() -> auditor), handler); } private long getLatestFinalBucketEndTimeMs(Job job) { @@ -193,8 +198,8 @@ public class ScheduledJobRunner extends AbstractComponent { return latestRecordTimeMs; } - private static Duration getFrequencyOrDefault(Job job) { - Long frequency = job.getSchedulerConfig().getFrequency(); + private static Duration getFrequencyOrDefault(Scheduler scheduler, Job job) { + Long frequency = scheduler.getConfig().getFrequency(); Long bucketSpan = job.getAnalysisConfig().getBucketSpan(); return frequency == null ? DefaultFrequency.ofBucketSpan(bucketSpan) : Duration.ofSeconds(frequency); } @@ -203,22 +208,22 @@ public class ScheduledJobRunner extends AbstractComponent { return new TimeValue(Math.max(1, next - currentTimeSupplier.get())); } - private void setJobSchedulerStatus(String jobId, SchedulerStatus status, Consumer supplier) { - UpdateJobSchedulerStatusAction.Request request = new UpdateJobSchedulerStatusAction.Request(jobId, status); - client.execute(UpdateJobSchedulerStatusAction.INSTANCE, request, new ActionListener() { + private void setJobSchedulerStatus(String schedulerId, SchedulerStatus status, Consumer supplier) { + UpdateSchedulerStatusAction.Request request = new UpdateSchedulerStatusAction.Request(schedulerId, status); + client.execute(UpdateSchedulerStatusAction.INSTANCE, request, new ActionListener() { @Override - public void onResponse(UpdateJobSchedulerStatusAction.Response response) { + public void onResponse(UpdateSchedulerStatusAction.Response response) { if (response.isAcknowledged()) { - logger.debug("successfully set job scheduler status to [{}] for job [{}]", status, jobId); + logger.debug("successfully set scheduler [{}] status to [{}]", schedulerId, status); } else { - logger.info("set job scheduler status to [{}] for job [{}], but was not acknowledged", status, jobId); + logger.info("set scheduler [{}] status to [{}], but was not acknowledged", schedulerId, status); } supplier.accept(null); } @Override public void onFailure(Exception e) { - logger.error("could not set job scheduler status to [" + status + "] for job [" + jobId +"]", e); + logger.error("could not set scheduler [" + schedulerId + "] status to [" + status + "]", e); supplier.accept(e); } }); @@ -226,14 +231,14 @@ public class ScheduledJobRunner extends AbstractComponent { public class Holder { - private final String jobId; + private final Scheduler scheduler; private final ScheduledJob scheduledJob; private final ProblemTracker problemTracker; private final Consumer handler; volatile Future future; - private Holder(Job job, ScheduledJob scheduledJob, ProblemTracker problemTracker, Consumer handler) { - this.jobId = job.getId(); + private Holder(Scheduler scheduler, ScheduledJob scheduledJob, ProblemTracker problemTracker, Consumer handler) { + this.scheduler = scheduler; this.scheduledJob = scheduledJob; this.problemTracker = problemTracker; this.handler = handler; @@ -244,10 +249,10 @@ public class ScheduledJobRunner extends AbstractComponent { } public void stop() { - logger.info("Stopping scheduler for job [{}]", jobId); + logger.info("Stopping scheduler [{}] for job [{}]", scheduler.getId(), scheduler.getJobId()); scheduledJob.stop(); FutureUtils.cancel(future); - setJobSchedulerStatus(jobId, SchedulerStatus.STOPPED, error -> handler.accept(null)); + setJobSchedulerStatus(scheduler.getId(), SchedulerStatus.STOPPED, error -> handler.accept(null)); } } diff --git a/elasticsearch/src/main/java/org/elasticsearch/xpack/prelert/job/scheduler/ScheduledJobValidator.java b/elasticsearch/src/main/java/org/elasticsearch/xpack/prelert/job/scheduler/ScheduledJobValidator.java new file mode 100644 index 00000000000..1208335decb --- /dev/null +++ b/elasticsearch/src/main/java/org/elasticsearch/xpack/prelert/job/scheduler/ScheduledJobValidator.java @@ -0,0 +1,38 @@ +/* + * 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.prelert.job.scheduler; + +import org.elasticsearch.xpack.prelert.job.AnalysisConfig; +import org.elasticsearch.xpack.prelert.job.DataDescription; +import org.elasticsearch.xpack.prelert.job.Job; +import org.elasticsearch.xpack.prelert.job.SchedulerConfig; +import org.elasticsearch.xpack.prelert.job.messages.Messages; + +public final class ScheduledJobValidator { + + private ScheduledJobValidator() {} + + /** + * Validates a schedulerConfig in relation to the job it refers to + * @param schedulerConfig the scheduler config + * @param job the job + */ + public static void validate(SchedulerConfig schedulerConfig, Job job) { + AnalysisConfig analysisConfig = job.getAnalysisConfig(); + if (analysisConfig.getLatency() != null && analysisConfig.getLatency() > 0) { + throw new IllegalArgumentException(Messages.getMessage(Messages.SCHEDULER_DOES_NOT_SUPPORT_JOB_WITH_LATENCY)); + } + if (schedulerConfig.getAggregationsOrAggs() != null + && !SchedulerConfig.DOC_COUNT.equals(analysisConfig.getSummaryCountFieldName())) { + throw new IllegalArgumentException( + Messages.getMessage(Messages.SCHEDULER_AGGREGATIONS_REQUIRES_JOB_WITH_SUMMARY_COUNT_FIELD, SchedulerConfig.DOC_COUNT)); + } + DataDescription dataDescription = job.getDataDescription(); + if (dataDescription == null || dataDescription.getFormat() != DataDescription.DataFormat.ELASTICSEARCH) { + throw new IllegalArgumentException(Messages.getMessage(Messages.SCHEDULER_REQUIRES_JOB_WITH_DATAFORMAT_ELASTICSEARCH)); + } + } +} diff --git a/elasticsearch/src/main/java/org/elasticsearch/xpack/prelert/job/scheduler/http/HttpDataExtractorFactory.java b/elasticsearch/src/main/java/org/elasticsearch/xpack/prelert/job/scheduler/http/HttpDataExtractorFactory.java index e1471bca9ec..5b4c0b4f63d 100644 --- a/elasticsearch/src/main/java/org/elasticsearch/xpack/prelert/job/scheduler/http/HttpDataExtractorFactory.java +++ b/elasticsearch/src/main/java/org/elasticsearch/xpack/prelert/job/scheduler/http/HttpDataExtractorFactory.java @@ -33,9 +33,8 @@ public class HttpDataExtractorFactory implements DataExtractorFactory { } @Override - public DataExtractor newExtractor(Job job) { + public DataExtractor newExtractor(SchedulerConfig schedulerConfig, Job job) { String timeField = job.getDataDescription().getTimeField(); - SchedulerConfig schedulerConfig = job.getSchedulerConfig(); ElasticsearchQueryBuilder queryBuilder = new ElasticsearchQueryBuilder( stringifyElasticsearchQuery(schedulerConfig.getQuery()), stringifyElasticsearchAggregations(schedulerConfig.getAggregations(), schedulerConfig.getAggs()), diff --git a/elasticsearch/src/main/java/org/elasticsearch/xpack/prelert/rest/schedulers/RestDeleteSchedulerAction.java b/elasticsearch/src/main/java/org/elasticsearch/xpack/prelert/rest/schedulers/RestDeleteSchedulerAction.java new file mode 100644 index 00000000000..32d2efe8727 --- /dev/null +++ b/elasticsearch/src/main/java/org/elasticsearch/xpack/prelert/rest/schedulers/RestDeleteSchedulerAction.java @@ -0,0 +1,41 @@ +/* + * 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.prelert.rest.schedulers; + +import org.elasticsearch.client.node.NodeClient; +import org.elasticsearch.common.inject.Inject; +import org.elasticsearch.common.settings.Settings; +import org.elasticsearch.rest.BaseRestHandler; +import org.elasticsearch.rest.RestController; +import org.elasticsearch.rest.RestRequest; +import org.elasticsearch.rest.action.AcknowledgedRestListener; +import org.elasticsearch.xpack.prelert.PrelertPlugin; +import org.elasticsearch.xpack.prelert.action.DeleteSchedulerAction; +import org.elasticsearch.xpack.prelert.job.SchedulerConfig; + +import java.io.IOException; + +public class RestDeleteSchedulerAction extends BaseRestHandler { + + private final DeleteSchedulerAction.TransportAction transportDeleteSchedulerAction; + + @Inject + public RestDeleteSchedulerAction(Settings settings, RestController controller, + DeleteSchedulerAction.TransportAction transportDeleteSchedulerAction) { + super(settings); + this.transportDeleteSchedulerAction = transportDeleteSchedulerAction; + controller.registerHandler(RestRequest.Method.DELETE, PrelertPlugin.BASE_PATH + "schedulers/{" + + SchedulerConfig.ID.getPreferredName() + "}", this); + } + + @Override + protected RestChannelConsumer prepareRequest(RestRequest restRequest, NodeClient client) throws IOException { + String schedulerId = restRequest.param(SchedulerConfig.ID.getPreferredName()); + DeleteSchedulerAction.Request deleteSchedulerRequest = new DeleteSchedulerAction.Request(schedulerId); + return channel -> transportDeleteSchedulerAction.execute(deleteSchedulerRequest, new AcknowledgedRestListener<>(channel)); + } + +} diff --git a/elasticsearch/src/main/java/org/elasticsearch/xpack/prelert/rest/schedulers/RestPutSchedulerAction.java b/elasticsearch/src/main/java/org/elasticsearch/xpack/prelert/rest/schedulers/RestPutSchedulerAction.java new file mode 100644 index 00000000000..f5a47dfacb2 --- /dev/null +++ b/elasticsearch/src/main/java/org/elasticsearch/xpack/prelert/rest/schedulers/RestPutSchedulerAction.java @@ -0,0 +1,45 @@ +/* + * 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.prelert.rest.schedulers; + +import org.elasticsearch.client.node.NodeClient; +import org.elasticsearch.common.inject.Inject; +import org.elasticsearch.common.settings.Settings; +import org.elasticsearch.common.xcontent.XContentFactory; +import org.elasticsearch.common.xcontent.XContentParser; +import org.elasticsearch.rest.BaseRestHandler; +import org.elasticsearch.rest.RestController; +import org.elasticsearch.rest.RestRequest; +import org.elasticsearch.rest.action.RestToXContentListener; +import org.elasticsearch.xpack.prelert.PrelertPlugin; +import org.elasticsearch.xpack.prelert.action.PutSchedulerAction; +import org.elasticsearch.xpack.prelert.job.SchedulerConfig; + +import java.io.IOException; + +public class RestPutSchedulerAction extends BaseRestHandler { + + private final PutSchedulerAction.TransportAction transportPutSchedulerAction; + + @Inject + public RestPutSchedulerAction(Settings settings, RestController controller, + PutSchedulerAction.TransportAction transportPutSchedulerAction) { + super(settings); + this.transportPutSchedulerAction = transportPutSchedulerAction; + controller.registerHandler(RestRequest.Method.PUT, PrelertPlugin.BASE_PATH + "schedulers/{" + + SchedulerConfig.ID.getPreferredName() + "}", this); + } + + @Override + protected RestChannelConsumer prepareRequest(RestRequest restRequest, NodeClient client) throws IOException { + String schedulerId = restRequest.param(SchedulerConfig.ID.getPreferredName()); + XContentParser parser = XContentFactory.xContent(restRequest.content()).createParser(restRequest.content()); + PutSchedulerAction.Request putSchedulerRequest = PutSchedulerAction.Request.parseRequest(schedulerId, parser, + () -> parseFieldMatcher); + return channel -> transportPutSchedulerAction.execute(putSchedulerRequest, new RestToXContentListener<>(channel)); + } + +} diff --git a/elasticsearch/src/main/java/org/elasticsearch/xpack/prelert/rest/schedulers/RestStartJobSchedulerAction.java b/elasticsearch/src/main/java/org/elasticsearch/xpack/prelert/rest/schedulers/RestStartSchedulerAction.java similarity index 72% rename from elasticsearch/src/main/java/org/elasticsearch/xpack/prelert/rest/schedulers/RestStartJobSchedulerAction.java rename to elasticsearch/src/main/java/org/elasticsearch/xpack/prelert/rest/schedulers/RestStartSchedulerAction.java index f81089b3997..af7219a540a 100644 --- a/elasticsearch/src/main/java/org/elasticsearch/xpack/prelert/rest/schedulers/RestStartJobSchedulerAction.java +++ b/elasticsearch/src/main/java/org/elasticsearch/xpack/prelert/rest/schedulers/RestStartSchedulerAction.java @@ -23,55 +23,55 @@ import org.elasticsearch.rest.RestStatus; import org.elasticsearch.tasks.LoggingTaskListener; import org.elasticsearch.tasks.Task; import org.elasticsearch.xpack.prelert.PrelertPlugin; -import org.elasticsearch.xpack.prelert.action.StartJobSchedulerAction; -import org.elasticsearch.xpack.prelert.job.Job; +import org.elasticsearch.xpack.prelert.action.StartSchedulerAction; +import org.elasticsearch.xpack.prelert.job.SchedulerConfig; import org.elasticsearch.xpack.prelert.job.messages.Messages; import org.elasticsearch.xpack.prelert.job.metadata.PrelertMetadata; import org.elasticsearch.xpack.prelert.job.scheduler.ScheduledJobRunner; import java.io.IOException; -public class RestStartJobSchedulerAction extends BaseRestHandler { +public class RestStartSchedulerAction extends BaseRestHandler { private static final String DEFAULT_START = "0"; private final ClusterService clusterService; @Inject - public RestStartJobSchedulerAction(Settings settings, RestController controller, ClusterService clusterService) { + public RestStartSchedulerAction(Settings settings, RestController controller, ClusterService clusterService) { super(settings); this.clusterService = clusterService; controller.registerHandler(RestRequest.Method.POST, - PrelertPlugin.BASE_PATH + "schedulers/{" + Job.ID.getPreferredName() + "}/_start", this); + PrelertPlugin.BASE_PATH + "schedulers/{" + SchedulerConfig.ID.getPreferredName() + "}/_start", this); } @Override protected RestChannelConsumer prepareRequest(RestRequest restRequest, NodeClient client) throws IOException { - String jobId = restRequest.param(Job.ID.getPreferredName()); + String schedulerId = restRequest.param(SchedulerConfig.ID.getPreferredName()); // This validation happens also in ScheduledJobRunner, 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. PrelertMetadata prelertMetadata = clusterService.state().metaData().custom(PrelertMetadata.TYPE); - ScheduledJobRunner.validate(jobId, prelertMetadata); + ScheduledJobRunner.validate(schedulerId, prelertMetadata); - StartJobSchedulerAction.Request jobSchedulerRequest; + StartSchedulerAction.Request jobSchedulerRequest; if (restRequest.hasContentOrSourceParam()) { BytesReference bodyBytes = restRequest.contentOrSourceParam(); XContentParser parser = XContentFactory.xContent(bodyBytes).createParser(bodyBytes); - jobSchedulerRequest = StartJobSchedulerAction.Request.parseRequest(jobId, parser, () -> parseFieldMatcher); + jobSchedulerRequest = StartSchedulerAction.Request.parseRequest(schedulerId, parser, () -> parseFieldMatcher); } else { - long startTimeMillis = parseDateOrThrow(restRequest.param(StartJobSchedulerAction.START_TIME.getPreferredName(), - DEFAULT_START), StartJobSchedulerAction.START_TIME.getPreferredName()); + long startTimeMillis = parseDateOrThrow(restRequest.param(StartSchedulerAction.START_TIME.getPreferredName(), + DEFAULT_START), StartSchedulerAction.START_TIME.getPreferredName()); Long endTimeMillis = null; - if (restRequest.hasParam(StartJobSchedulerAction.END_TIME.getPreferredName())) { - endTimeMillis = parseDateOrThrow(restRequest.param(StartJobSchedulerAction.END_TIME.getPreferredName()), - StartJobSchedulerAction.END_TIME.getPreferredName()); + if (restRequest.hasParam(StartSchedulerAction.END_TIME.getPreferredName())) { + endTimeMillis = parseDateOrThrow(restRequest.param(StartSchedulerAction.END_TIME.getPreferredName()), + StartSchedulerAction.END_TIME.getPreferredName()); } - jobSchedulerRequest = new StartJobSchedulerAction.Request(jobId, startTimeMillis); + jobSchedulerRequest = new StartSchedulerAction.Request(schedulerId, startTimeMillis); jobSchedulerRequest.setEndTime(endTimeMillis); } - return sendTask(client.executeLocally(StartJobSchedulerAction.INSTANCE, jobSchedulerRequest, LoggingTaskListener.instance())); + return sendTask(client.executeLocally(StartSchedulerAction.INSTANCE, jobSchedulerRequest, LoggingTaskListener.instance())); } private RestChannelConsumer sendTask(Task task) throws IOException { diff --git a/elasticsearch/src/main/java/org/elasticsearch/xpack/prelert/rest/schedulers/RestStopJobSchedulerAction.java b/elasticsearch/src/main/java/org/elasticsearch/xpack/prelert/rest/schedulers/RestStopSchedulerAction.java similarity index 63% rename from elasticsearch/src/main/java/org/elasticsearch/xpack/prelert/rest/schedulers/RestStopJobSchedulerAction.java rename to elasticsearch/src/main/java/org/elasticsearch/xpack/prelert/rest/schedulers/RestStopSchedulerAction.java index 0726eee7815..4f9c85ba739 100644 --- a/elasticsearch/src/main/java/org/elasticsearch/xpack/prelert/rest/schedulers/RestStopJobSchedulerAction.java +++ b/elasticsearch/src/main/java/org/elasticsearch/xpack/prelert/rest/schedulers/RestStopSchedulerAction.java @@ -13,28 +13,28 @@ import org.elasticsearch.rest.RestController; import org.elasticsearch.rest.RestRequest; import org.elasticsearch.rest.action.AcknowledgedRestListener; import org.elasticsearch.xpack.prelert.PrelertPlugin; -import org.elasticsearch.xpack.prelert.action.StopJobSchedulerAction; -import org.elasticsearch.xpack.prelert.job.Job; +import org.elasticsearch.xpack.prelert.action.StopSchedulerAction; +import org.elasticsearch.xpack.prelert.job.SchedulerConfig; import java.io.IOException; -public class RestStopJobSchedulerAction extends BaseRestHandler { +public class RestStopSchedulerAction extends BaseRestHandler { - private final StopJobSchedulerAction.TransportAction transportJobSchedulerAction; + private final StopSchedulerAction.TransportAction transportJobSchedulerAction; @Inject - public RestStopJobSchedulerAction(Settings settings, RestController controller, - StopJobSchedulerAction.TransportAction transportJobSchedulerAction) { + public RestStopSchedulerAction(Settings settings, RestController controller, + StopSchedulerAction.TransportAction transportJobSchedulerAction) { super(settings); this.transportJobSchedulerAction = transportJobSchedulerAction; controller.registerHandler(RestRequest.Method.POST, PrelertPlugin.BASE_PATH + "schedulers/{" - + Job.ID.getPreferredName() + "}/_stop", this); + + SchedulerConfig.ID.getPreferredName() + "}/_stop", this); } @Override protected RestChannelConsumer prepareRequest(RestRequest restRequest, NodeClient client) throws IOException { - StopJobSchedulerAction.Request jobSchedulerRequest = new StopJobSchedulerAction.Request( - restRequest.param(Job.ID.getPreferredName())); + StopSchedulerAction.Request jobSchedulerRequest = new StopSchedulerAction.Request( + restRequest.param(SchedulerConfig.ID.getPreferredName())); return channel -> transportJobSchedulerAction.execute(jobSchedulerRequest, new AcknowledgedRestListener<>(channel)); } } diff --git a/elasticsearch/src/main/java/org/elasticsearch/xpack/prelert/utils/PrelertStrings.java b/elasticsearch/src/main/java/org/elasticsearch/xpack/prelert/utils/PrelertStrings.java index 513425d692b..169664a714f 100644 --- a/elasticsearch/src/main/java/org/elasticsearch/xpack/prelert/utils/PrelertStrings.java +++ b/elasticsearch/src/main/java/org/elasticsearch/xpack/prelert/utils/PrelertStrings.java @@ -12,8 +12,14 @@ import java.util.regex.Pattern; * with one of the myriad String utility classes out there. */ public final class PrelertStrings { + private static final Pattern NEEDS_QUOTING = Pattern.compile("\\W"); + /** + * Valid user entered id characters. Note that '.' is allowed but not documented. + */ + private static final Pattern VALID_ID_CHAR_PATTERN = Pattern.compile("[a-z0-9_\\-\\.]+"); + private PrelertStrings() { } @@ -45,4 +51,8 @@ public final class PrelertStrings { quoted.append('\"'); return quoted.toString(); } + + public static boolean isValidId(String id) { + return VALID_ID_CHAR_PATTERN.matcher(id).matches(); + } } diff --git a/elasticsearch/src/main/resources/org/elasticsearch/xpack/prelert/job/messages/prelert_messages.properties b/elasticsearch/src/main/resources/org/elasticsearch/xpack/prelert/job/messages/prelert_messages.properties index 06fe0a129b9..fd54c1be62b 100644 --- a/elasticsearch/src/main/resources/org/elasticsearch/xpack/prelert/job/messages/prelert_messages.properties +++ b/elasticsearch/src/main/resources/org/elasticsearch/xpack/prelert/job/messages/prelert_messages.properties @@ -44,7 +44,6 @@ system.audit.started = System started system.audit.shutdown = System shut down job.cannot.delete.while.running = Cannot delete job ''{0}'' while it is {1} -job.cannot.delete.while.scheduler.runs = Cannot delete job ''{0}'' while the scheduler is running job.cannot.pause = Cannot pause job ''{0}'' while its status is {1} job.cannot.resume = Cannot resume job ''{0}'' while its status is {1} @@ -136,13 +135,14 @@ job.config.transform.outputs.unused = None of the outputs of transform ''{0}'' a job.config.transform.output.name.used.more.than.once = Transform output name ''{0}'' is used more than once job.config.transform.unknown.type = Unknown TransformType ''{0}'' job.config.unknown.function = Unknown function ''{0}'' -job.config.scheduler.field.not.supported = Scheduler configuration field {0} not supported -job.config.scheduler.invalid.option.value = Invalid {0} value ''{1}'' in scheduler configuration -job.config.scheduler.requires.bucket.span = A job configured with scheduler requires that bucket_span is specified -job.config.scheduler.elasticsearch.does.not.support.latency = A job configured with an Elasticsearch scheduler cannot support latency -job.config.scheduler.aggregations.requires.summary.count.field = A scheduler job with aggregations must have summary_count_field_name ''{1}'' -job.config.scheduler.elasticsearch.requires.dataformat.elasticsearch = A job configured with an Elasticsearch scheduler must have dataFormat ''ELASTICSEARCH'' -job.config.scheduler.multiple.aggregations = Both aggregations and aggs were specified - please just specify one + +scheduler.config.field.not.supported = Scheduler configuration field {0} not supported +scheduler.config.invalid.option.value = Invalid {0} value ''{1}'' in scheduler configuration +scheduler.config.multiple.aggregations = Both aggregations and aggs were specified - please just specify one + +scheduler.does.not.support.job.with.latency = A job configured with scheduler cannot support latency +scheduler.aggregations.requires.job.with.summary.count.field = A job configured with a scheduler with aggregations must have summary_count_field_name ''{0}'' +scheduler.requires.job.with.dataformat.elasticsearch = A job configured with a scheduler must have dataFormat ''ELASTICSEARCH'' job.data.concurrent.use.close = Cannot close job {0} while another connection {2}is {1} the job job.data.concurrent.use.flush = Cannot flush job {0} while another connection {2}is {1} the job @@ -155,17 +155,12 @@ job.data.concurrent.use.upload = Cannot write to job {0} while another connectio job.missing.quantiles = Cannot read persisted quantiles for job ''{0}'' job.unknown.id = No known job with id ''{0}'' -job.scheduler.cannot.start = Cannot start scheduler for job ''{0}'' while its status is {1} -job.scheduler.cannot.stop.in.current.state = Cannot stop scheduler for job ''{0}'' while its status is {1} -job.scheduler.cannot.update.in.current.state = Cannot update scheduler for job ''{0}'' while its status is {1} -job.scheduler.failed.to.stop = Failed to stop scheduler -job.scheduler.no.such.scheduled.job = There is no job ''{0}'' with a scheduler configured -job.scheduler.status.started = started -job.scheduler.status.stopping = stopping -job.scheduler.status.stopped = stopped -job.scheduler.status.updating = updating -job.scheduler.status.deleting = deleting - +scheduler.cannot.start = Cannot start scheduler [{0}] while its status is {1} +scheduler.cannot.stop.in.current.state = Cannot stop scheduler [{0}] while its status is {1} +scheduler.cannot.update.in.current.state = Cannot update scheduler [{0}] while its status is {1} +scheduler.cannot.delete.in.current.state = Cannot delete scheduler [{0}] while its status is {1} +scheduler.failed.to.stop = Failed to stop scheduler +scheduler.not.found = No scheduler with id [{0}] exists json.job.config.mapping.error = JSON mapping error reading the job configuration json.job.config.parse.error = JSON parse error reading the job configuration diff --git a/elasticsearch/src/test/java/org/elasticsearch/xpack/prelert/action/DeleteSchedulerRequestTests.java b/elasticsearch/src/test/java/org/elasticsearch/xpack/prelert/action/DeleteSchedulerRequestTests.java new file mode 100644 index 00000000000..9adf5964132 --- /dev/null +++ b/elasticsearch/src/test/java/org/elasticsearch/xpack/prelert/action/DeleteSchedulerRequestTests.java @@ -0,0 +1,22 @@ +/* + * 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.prelert.action; + +import org.elasticsearch.xpack.prelert.action.DeleteSchedulerAction.Request; +import org.elasticsearch.xpack.prelert.support.AbstractStreamableTestCase; + +public class DeleteSchedulerRequestTests extends AbstractStreamableTestCase { + + @Override + protected Request createTestInstance() { + return new Request(randomAsciiOfLengthBetween(1, 20)); + } + + @Override + protected Request createBlankInstance() { + return new Request(); + } +} \ No newline at end of file diff --git a/elasticsearch/src/test/java/org/elasticsearch/xpack/prelert/action/GetJobActionResponseTests.java b/elasticsearch/src/test/java/org/elasticsearch/xpack/prelert/action/GetJobActionResponseTests.java index 5d8ffde7d9e..4ac1fa42f1f 100644 --- a/elasticsearch/src/test/java/org/elasticsearch/xpack/prelert/action/GetJobActionResponseTests.java +++ b/elasticsearch/src/test/java/org/elasticsearch/xpack/prelert/action/GetJobActionResponseTests.java @@ -13,11 +13,10 @@ import org.elasticsearch.xpack.prelert.job.DataDescription; import org.elasticsearch.xpack.prelert.job.Detector; import org.elasticsearch.xpack.prelert.job.IgnoreDowntime; import org.elasticsearch.xpack.prelert.job.Job; -import org.elasticsearch.xpack.prelert.job.SchedulerStatus; import org.elasticsearch.xpack.prelert.job.JobStatus; import org.elasticsearch.xpack.prelert.job.ModelDebugConfig; import org.elasticsearch.xpack.prelert.job.ModelSizeStats; -import org.elasticsearch.xpack.prelert.job.SchedulerConfig; +import org.elasticsearch.xpack.prelert.job.SchedulerStatus; import org.elasticsearch.xpack.prelert.job.persistence.QueryPage; import org.elasticsearch.xpack.prelert.job.transform.TransformConfig; import org.elasticsearch.xpack.prelert.job.transform.TransformType; @@ -25,7 +24,6 @@ import org.elasticsearch.xpack.prelert.support.AbstractStreamableTestCase; import org.joda.time.DateTime; import java.util.ArrayList; -import java.util.Arrays; import java.util.Collections; import java.util.Date; import java.util.EnumSet; @@ -50,7 +48,6 @@ public class GetJobActionResponseTests extends AbstractStreamableTestCase transformConfigList = new ArrayList<>(numTransformers); @@ -67,7 +64,7 @@ public class GetJobActionResponseTests extends AbstractStreamableTestCase { + + private String schedulerId; + + @Before + public void setUpSchedulerId() { + schedulerId = SchedulerConfigTests.randomValidSchedulerId(); + } + + @Override + protected Request createTestInstance() { + SchedulerConfig.Builder schedulerConfig = new SchedulerConfig.Builder(schedulerId, randomAsciiOfLength(10)); + schedulerConfig.setIndexes(Arrays.asList(randomAsciiOfLength(10))); + schedulerConfig.setTypes(Arrays.asList(randomAsciiOfLength(10))); + return new Request(schedulerConfig.build()); + } + + @Override + protected Request createBlankInstance() { + return new Request(); + } + + @Override + protected Request parseInstance(XContentParser parser, ParseFieldMatcher matcher) { + return Request.parseRequest(schedulerId, parser, () -> matcher); + } + +} diff --git a/elasticsearch/src/test/java/org/elasticsearch/xpack/prelert/action/PutSchedulerActionResponseTests.java b/elasticsearch/src/test/java/org/elasticsearch/xpack/prelert/action/PutSchedulerActionResponseTests.java new file mode 100644 index 00000000000..b097fec98c6 --- /dev/null +++ b/elasticsearch/src/test/java/org/elasticsearch/xpack/prelert/action/PutSchedulerActionResponseTests.java @@ -0,0 +1,31 @@ +/* + * 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.prelert.action; + +import org.elasticsearch.xpack.prelert.action.PutSchedulerAction.Response; +import org.elasticsearch.xpack.prelert.job.SchedulerConfig; +import org.elasticsearch.xpack.prelert.job.SchedulerConfigTests; +import org.elasticsearch.xpack.prelert.support.AbstractStreamableTestCase; + +import java.util.Arrays; + +public class PutSchedulerActionResponseTests extends AbstractStreamableTestCase { + + @Override + protected Response createTestInstance() { + SchedulerConfig.Builder schedulerConfig = new SchedulerConfig.Builder( + SchedulerConfigTests.randomValidSchedulerId(), randomAsciiOfLength(10)); + schedulerConfig.setIndexes(Arrays.asList(randomAsciiOfLength(10))); + schedulerConfig.setTypes(Arrays.asList(randomAsciiOfLength(10))); + return new Response(randomBoolean(), schedulerConfig.build()); + } + + @Override + protected Response createBlankInstance() { + return new Response(); + } + +} diff --git a/elasticsearch/src/test/java/org/elasticsearch/xpack/prelert/action/ScheduledJobsIT.java b/elasticsearch/src/test/java/org/elasticsearch/xpack/prelert/action/ScheduledJobsIT.java index 96a8b393eb0..ad0e04d66e4 100644 --- a/elasticsearch/src/test/java/org/elasticsearch/xpack/prelert/action/ScheduledJobsIT.java +++ b/elasticsearch/src/test/java/org/elasticsearch/xpack/prelert/action/ScheduledJobsIT.java @@ -23,14 +23,14 @@ import org.elasticsearch.xpack.prelert.job.DataCounts; import org.elasticsearch.xpack.prelert.job.DataDescription; import org.elasticsearch.xpack.prelert.job.Detector; import org.elasticsearch.xpack.prelert.job.Job; -import org.elasticsearch.xpack.prelert.job.SchedulerStatus; import org.elasticsearch.xpack.prelert.job.SchedulerConfig; +import org.elasticsearch.xpack.prelert.job.SchedulerStatus; import org.elasticsearch.xpack.prelert.job.metadata.PrelertMetadata; +import org.elasticsearch.xpack.prelert.job.metadata.Scheduler; import org.elasticsearch.xpack.prelert.job.persistence.JobResultsPersister; import org.junit.After; import java.io.IOException; -import java.net.InetSocketAddress; import java.util.Collection; import java.util.Collections; import java.util.Map; @@ -75,16 +75,21 @@ public class ScheduledJobsIT extends ESIntegTestCase { OpenJobAction.Response openJobResponse = client().execute(OpenJobAction.INSTANCE, new OpenJobAction.Request(job.getId())).get(); assertTrue(openJobResponse.isAcknowledged()); - StartJobSchedulerAction.Request startSchedulerRequest = new StartJobSchedulerAction.Request("_job_id", 0L); + SchedulerConfig schedulerConfig = createScheduler(job.getId() + "-scheduler", job.getId()); + PutSchedulerAction.Request putSchedulerRequest = new PutSchedulerAction.Request(schedulerConfig); + PutSchedulerAction.Response putSchedulerResponse = client().execute(PutSchedulerAction.INSTANCE, putSchedulerRequest).get(); + assertTrue(putSchedulerResponse.isAcknowledged()); + + StartSchedulerAction.Request startSchedulerRequest = new StartSchedulerAction.Request(schedulerConfig.getId(), 0L); startSchedulerRequest.setEndTime(now); - client().execute(StartJobSchedulerAction.INSTANCE, startSchedulerRequest).get(); + client().execute(StartSchedulerAction.INSTANCE, startSchedulerRequest).get(); assertBusy(() -> { DataCounts dataCounts = getDataCounts("_job_id"); assertThat(dataCounts.getInputRecordCount(), equalTo(numDocs)); PrelertMetadata prelertMetadata = client().admin().cluster().prepareState().all().get() .getState().metaData().custom(PrelertMetadata.TYPE); - assertThat(prelertMetadata.getSchedulerStatuses().get("_job_id"), equalTo(SchedulerStatus.STOPPED)); + assertThat(prelertMetadata.getSchedulerStatusByJobId("_job_id").get(), equalTo(SchedulerStatus.STOPPED)); }); } @@ -104,11 +109,16 @@ public class ScheduledJobsIT extends ESIntegTestCase { OpenJobAction.Response openJobResponse = client().execute(OpenJobAction.INSTANCE, new OpenJobAction.Request(job.getId())).get(); assertTrue(openJobResponse.isAcknowledged()); + SchedulerConfig schedulerConfig = createScheduler(job.getId() + "-scheduler", job.getId()); + PutSchedulerAction.Request putSchedulerRequest = new PutSchedulerAction.Request(schedulerConfig); + PutSchedulerAction.Response putSchedulerResponse = client().execute(PutSchedulerAction.INSTANCE, putSchedulerRequest).get(); + assertTrue(putSchedulerResponse.isAcknowledged()); + AtomicReference errorHolder = new AtomicReference<>(); Thread t = new Thread(() -> { try { - StartJobSchedulerAction.Request startSchedulerRequest = new StartJobSchedulerAction.Request("_job_id", 0L); - client().execute(StartJobSchedulerAction.INSTANCE, startSchedulerRequest).get(); + StartSchedulerAction.Request startSchedulerRequest = new StartSchedulerAction.Request(schedulerConfig.getId(), 0L); + client().execute(StartSchedulerAction.INSTANCE, startSchedulerRequest).get(); } catch (Exception | AssertionError e) { errorHolder.set(e); } @@ -127,13 +137,13 @@ public class ScheduledJobsIT extends ESIntegTestCase { assertThat(dataCounts.getInputRecordCount(), equalTo(numDocs1 + numDocs2)); }, 30, TimeUnit.SECONDS); - StopJobSchedulerAction.Request stopSchedulerRequest = new StopJobSchedulerAction.Request("_job_id"); - StopJobSchedulerAction.Response stopJobResponse = client().execute(StopJobSchedulerAction.INSTANCE, stopSchedulerRequest).get(); + StopSchedulerAction.Request stopSchedulerRequest = new StopSchedulerAction.Request(schedulerConfig.getId()); + StopSchedulerAction.Response stopJobResponse = client().execute(StopSchedulerAction.INSTANCE, stopSchedulerRequest).get(); assertTrue(stopJobResponse.isAcknowledged()); assertBusy(() -> { PrelertMetadata prelertMetadata = client().admin().cluster().prepareState().all().get() .getState().metaData().custom(PrelertMetadata.TYPE); - assertThat(prelertMetadata.getSchedulerStatuses().get("_job_id"), equalTo(SchedulerStatus.STOPPED)); + assertThat(prelertMetadata.getSchedulerStatusByJobId("_job_id").get(), equalTo(SchedulerStatus.STOPPED)); }); assertThat(errorHolder.get(), nullValue()); } @@ -156,12 +166,6 @@ public class ScheduledJobsIT extends ESIntegTestCase { } private Job.Builder createJob() { - SchedulerConfig.Builder scheduler = new SchedulerConfig.Builder(Collections.singletonList("data"), - Collections.singletonList("type")); - scheduler.setQueryDelay(1); - scheduler.setFrequency(2); - InetSocketAddress address = cluster().httpAddresses()[0]; - DataDescription.Builder dataDescription = new DataDescription.Builder(); dataDescription.setFormat(DataDescription.DataFormat.ELASTICSEARCH); dataDescription.setTimeFormat(DataDescription.EPOCH_MS); @@ -170,7 +174,6 @@ public class ScheduledJobsIT extends ESIntegTestCase { AnalysisConfig.Builder analysisConfig = new AnalysisConfig.Builder(Collections.singletonList(d.build())); Job.Builder builder = new Job.Builder(); - builder.setSchedulerConfig(scheduler); builder.setId("_job_id"); builder.setAnalysisConfig(analysisConfig); @@ -178,6 +181,15 @@ public class ScheduledJobsIT extends ESIntegTestCase { return builder; } + private SchedulerConfig createScheduler(String schedulerId, String jobId) { + SchedulerConfig.Builder builder = new SchedulerConfig.Builder(schedulerId, jobId); + builder.setQueryDelay(1); + builder.setFrequency(2); + builder.setIndexes(Collections.singletonList("data")); + builder.setTypes(Collections.singletonList("type")); + return builder.build(); + } + private DataCounts getDataCounts(String jobId) { GetResponse getResponse = client().prepareGet(JobResultsPersister.getJobIndexName(jobId), DataCounts.TYPE.getPreferredName(), jobId + "-data-counts").get(); @@ -193,13 +205,19 @@ public class ScheduledJobsIT extends ESIntegTestCase { } public static void clearPrelertMetadata(Client client) throws Exception { + deleteAllSchedulers(client); + deleteAllJobs(client); + } + + private static void deleteAllSchedulers(Client client) throws Exception { MetaData metaData = client.admin().cluster().prepareState().get().getState().getMetaData(); PrelertMetadata prelertMetadata = metaData.custom(PrelertMetadata.TYPE); - for (Map.Entry entry : prelertMetadata.getJobs().entrySet()) { - String jobId = entry.getKey(); + for (Scheduler scheduler : prelertMetadata.getSchedulers().values()) { + String schedulerId = scheduler.getId(); + String jobId = scheduler.getJobId(); try { - StopJobSchedulerAction.Response response = - client.execute(StopJobSchedulerAction.INSTANCE, new StopJobSchedulerAction.Request(jobId)).get(); + StopSchedulerAction.Response response = + client.execute(StopSchedulerAction.INSTANCE, new StopSchedulerAction.Request(schedulerId)).get(); assertTrue(response.isAcknowledged()); assertBusy(() -> { GetJobsAction.Response r = null; @@ -216,6 +234,21 @@ public class ScheduledJobsIT extends ESIntegTestCase { } catch (Exception e) { // ignore } + try { + DeleteSchedulerAction.Response response = + client.execute(DeleteSchedulerAction.INSTANCE, new DeleteSchedulerAction.Request(schedulerId)).get(); + assertTrue(response.isAcknowledged()); + } catch (Exception e) { + // ignore + } + } + } + + public static void deleteAllJobs(Client client) throws Exception { + MetaData metaData = client.admin().cluster().prepareState().get().getState().getMetaData(); + PrelertMetadata prelertMetadata = metaData.custom(PrelertMetadata.TYPE); + for (Map.Entry entry : prelertMetadata.getJobs().entrySet()) { + String jobId = entry.getKey(); try { CloseJobAction.Response response = client.execute(CloseJobAction.INSTANCE, new CloseJobAction.Request(jobId)).get(); @@ -228,5 +261,4 @@ public class ScheduledJobsIT extends ESIntegTestCase { assertTrue(response.isAcknowledged()); } } - } diff --git a/elasticsearch/src/test/java/org/elasticsearch/xpack/prelert/action/StartJobSchedulerActionRequestTests.java b/elasticsearch/src/test/java/org/elasticsearch/xpack/prelert/action/StartJobSchedulerActionRequestTests.java index b3577697579..17278eb4141 100644 --- a/elasticsearch/src/test/java/org/elasticsearch/xpack/prelert/action/StartJobSchedulerActionRequestTests.java +++ b/elasticsearch/src/test/java/org/elasticsearch/xpack/prelert/action/StartJobSchedulerActionRequestTests.java @@ -7,10 +7,10 @@ package org.elasticsearch.xpack.prelert.action; import org.elasticsearch.common.ParseFieldMatcher; import org.elasticsearch.common.xcontent.XContentParser; -import org.elasticsearch.xpack.prelert.action.StartJobSchedulerAction.Request; +import org.elasticsearch.xpack.prelert.action.StartSchedulerAction.Request; import org.elasticsearch.xpack.prelert.support.AbstractStreamableXContentTestCase; -public class StartJobSchedulerActionRequestTests extends AbstractStreamableXContentTestCase { +public class StartJobSchedulerActionRequestTests extends AbstractStreamableXContentTestCase { @Override protected Request createTestInstance() { diff --git a/elasticsearch/src/test/java/org/elasticsearch/xpack/prelert/action/StopJobSchedulerActionRequestTests.java b/elasticsearch/src/test/java/org/elasticsearch/xpack/prelert/action/StopJobSchedulerActionRequestTests.java index 49d33af1b93..af101802dd0 100644 --- a/elasticsearch/src/test/java/org/elasticsearch/xpack/prelert/action/StopJobSchedulerActionRequestTests.java +++ b/elasticsearch/src/test/java/org/elasticsearch/xpack/prelert/action/StopJobSchedulerActionRequestTests.java @@ -5,10 +5,10 @@ */ package org.elasticsearch.xpack.prelert.action; -import org.elasticsearch.xpack.prelert.action.StopJobSchedulerAction.Request; +import org.elasticsearch.xpack.prelert.action.StopSchedulerAction.Request; import org.elasticsearch.xpack.prelert.support.AbstractStreamableTestCase; -public class StopJobSchedulerActionRequestTests extends AbstractStreamableTestCase { +public class StopJobSchedulerActionRequestTests extends AbstractStreamableTestCase { @Override protected Request createTestInstance() { diff --git a/elasticsearch/src/test/java/org/elasticsearch/xpack/prelert/action/UpdateJobSchedulerStatusRequestTests.java b/elasticsearch/src/test/java/org/elasticsearch/xpack/prelert/action/UpdateJobSchedulerStatusRequestTests.java index f27eed7b651..51e63464cbd 100644 --- a/elasticsearch/src/test/java/org/elasticsearch/xpack/prelert/action/UpdateJobSchedulerStatusRequestTests.java +++ b/elasticsearch/src/test/java/org/elasticsearch/xpack/prelert/action/UpdateJobSchedulerStatusRequestTests.java @@ -5,7 +5,7 @@ */ package org.elasticsearch.xpack.prelert.action; -import org.elasticsearch.xpack.prelert.action.UpdateJobSchedulerStatusAction.Request; +import org.elasticsearch.xpack.prelert.action.UpdateSchedulerStatusAction.Request; import org.elasticsearch.xpack.prelert.job.SchedulerStatus; import org.elasticsearch.xpack.prelert.support.AbstractStreamableTestCase; diff --git a/elasticsearch/src/test/java/org/elasticsearch/xpack/prelert/integration/ScheduledJobIT.java b/elasticsearch/src/test/java/org/elasticsearch/xpack/prelert/integration/ScheduledJobIT.java index 019355b3ad7..59b60aceebb 100644 --- a/elasticsearch/src/test/java/org/elasticsearch/xpack/prelert/integration/ScheduledJobIT.java +++ b/elasticsearch/src/test/java/org/elasticsearch/xpack/prelert/integration/ScheduledJobIT.java @@ -5,7 +5,6 @@ */ package org.elasticsearch.xpack.prelert.integration; -import org.apache.http.HttpHost; import org.apache.http.entity.StringEntity; import org.apache.logging.log4j.Logger; import org.elasticsearch.client.Response; @@ -34,11 +33,13 @@ public class ScheduledJobIT extends ESRestTestCase { public void testStartJobScheduler_GivenLookbackOnly() throws Exception { String jobId = "_id2"; createAirlineDataIndex(); - createScheduledJob(jobId); + createJob(jobId); + String schedulerId = "_sched1"; + createScheduler(schedulerId, jobId); openJob(client(), jobId); Response startSchedulerRequest = client().performRequest("post", - PrelertPlugin.BASE_PATH + "schedulers/" + jobId + "/_start?start=2016-06-01T00:00:00Z&end=2016-06-02T00:00:00Z"); + PrelertPlugin.BASE_PATH + "schedulers/" + schedulerId + "/_start?start=2016-06-01T00:00:00Z&end=2016-06-02T00:00:00Z"); assertThat(startSchedulerRequest.getStatusLine().getStatusCode(), equalTo(200)); assertThat(responseEntityToString(startSchedulerRequest), containsString("{\"task\":\"")); assertBusy(() -> { @@ -56,11 +57,13 @@ public class ScheduledJobIT extends ESRestTestCase { public void testStartJobScheduler_GivenRealtime() throws Exception { String jobId = "_id3"; createAirlineDataIndex(); - createScheduledJob(jobId); + createJob(jobId); + String schedulerId = "_sched1"; + createScheduler(schedulerId, jobId); openJob(client(), jobId); Response response = client().performRequest("post", - PrelertPlugin.BASE_PATH + "schedulers/" + jobId + "/_start?start=2016-06-01T00:00:00Z"); + PrelertPlugin.BASE_PATH + "schedulers/" + schedulerId + "/_start?start=2016-06-01T00:00:00Z"); assertThat(response.getStatusLine().getStatusCode(), equalTo(200)); assertThat(responseEntityToString(response), containsString("{\"task\":\"")); assertBusy(() -> { @@ -79,14 +82,20 @@ public class ScheduledJobIT extends ESRestTestCase { () -> client().performRequest("delete", PrelertPlugin.BASE_PATH + "anomaly_detectors/" + jobId)); response = e.getResponse(); assertThat(response.getStatusLine().getStatusCode(), equalTo(409)); - assertThat(responseEntityToString(response), containsString("Cannot delete job '" + jobId + "' while it is OPENED")); + assertThat(responseEntityToString(response), containsString("Cannot delete job [" + jobId + "] while scheduler [" + schedulerId + + "] refers to it")); - response = client().performRequest("post", PrelertPlugin.BASE_PATH + "schedulers/" + jobId + "/_stop"); + response = client().performRequest("post", PrelertPlugin.BASE_PATH + "schedulers/" + schedulerId + "/_stop"); assertThat(response.getStatusLine().getStatusCode(), equalTo(200)); assertThat(responseEntityToString(response), equalTo("{\"acknowledged\":true}")); waitForSchedulerStoppedState(client(), jobId); client().performRequest("POST", "/_xpack/prelert/anomaly_detectors/" + jobId + "/_close"); + + response = client().performRequest("delete", PrelertPlugin.BASE_PATH + "schedulers/" + schedulerId); + assertThat(response.getStatusLine().getStatusCode(), equalTo(200)); + assertThat(responseEntityToString(response), equalTo("{\"acknowledged\":true}")); + response = client().performRequest("delete", PrelertPlugin.BASE_PATH + "anomaly_detectors/" + jobId); assertThat(response.getStatusLine().getStatusCode(), equalTo(200)); assertThat(responseEntityToString(response), equalTo("{\"acknowledged\":true}")); @@ -106,20 +115,24 @@ public class ScheduledJobIT extends ESRestTestCase { client().performRequest("post", "airline-data/_refresh"); } - private Response createScheduledJob(String id) throws Exception { - HttpHost httpHost = getClusterHosts().get(0); - logger.info("Http host = " + httpHost.toURI()); + private Response createJob(String id) throws Exception { String job = "{\n" + " \"job_id\":\"" + id + "\",\n" + " \"description\":\"Analysis of response time by airline\",\n" + " \"analysis_config\" : {\n" + " \"bucket_span\":3600,\n" + " \"detectors\" :[{\"function\":\"mean\",\"field_name\":\"responsetime\",\"by_field_name\":\"airline\"}]\n" + " },\n" + " \"data_description\" : {\n" + " \"format\":\"ELASTICSEARCH\",\n" - + " \"time_field\":\"time\",\n" + " \"time_format\":\"yyyy-MM-dd'T'HH:mm:ssX\"\n" + " },\n" - + " \"scheduler_config\" : {\n" + " \"indexes\":[\"airline-data\"],\n" - + " \"types\":[\"response\"],\n" + " \"retrieve_whole_source\":true\n" + " }\n" + "}"; + + " \"time_field\":\"time\",\n" + " \"time_format\":\"yyyy-MM-dd'T'HH:mm:ssX\"\n" + " }\n" + + "}"; return client().performRequest("put", PrelertPlugin.BASE_PATH + "anomaly_detectors", Collections.emptyMap(), new StringEntity(job)); } + private Response createScheduler(String schedulerId, String jobId) throws IOException { + String schedulerConfig = "{" + "\"job_id\": \"" + jobId + "\",\n" + "\"indexes\":[\"airline-data\"],\n" + + "\"types\":[\"response\"],\n" + "\"retrieve_whole_source\":true\n" + "}"; + return client().performRequest("put", PrelertPlugin.BASE_PATH + "schedulers/" + schedulerId, Collections.emptyMap(), + new StringEntity(schedulerConfig)); + } + private static String responseEntityToString(Response response) throws Exception { try (BufferedReader reader = new BufferedReader(new InputStreamReader(response.getEntity().getContent(), StandardCharsets.UTF_8))) { return reader.lines().collect(Collectors.joining("\n")); @@ -133,7 +146,7 @@ public class ScheduledJobIT extends ESRestTestCase { Response getJobResponse = client.performRequest("get", PrelertPlugin.BASE_PATH + "anomaly_detectors/" + jobId + "/_stats", Collections.singletonMap("metric", "scheduler_status")); - assertThat(responseEntityToString(getJobResponse), containsString("\"status\":\"STOPPED\"")); + assertThat(responseEntityToString(getJobResponse), containsString("\"scheduler_status\":\"STOPPED\"")); } catch (Exception e) { throw new RuntimeException(e); } @@ -151,6 +164,35 @@ public class ScheduledJobIT extends ESRestTestCase { } public static void clearPrelertMetadata(RestClient client) throws IOException { + deleteAllSchedulers(client); + deleteAllJobs(client); + } + + @SuppressWarnings("unchecked") + private static void deleteAllSchedulers(RestClient client) throws IOException { + Map clusterStateAsMap = entityAsMap(client.performRequest("GET", "/_cluster/state", + Collections.singletonMap("filter_path", "metadata.prelert.schedulers"))); + List> schedulers = + (List>) XContentMapValues.extractValue("metadata.prelert.schedulers", clusterStateAsMap); + if (schedulers == null) { + return; + } + + for (Map scheduler : schedulers) { + Map schedulerMap = (Map) scheduler.get("config"); + String schedulerId = (String) schedulerMap.get("scheduler_id"); + String jobId = (String) schedulerMap.get("job_id"); + try { + client.performRequest("POST", "/_xpack/prelert/schedulers/" + schedulerId + "/_stop"); + waitForSchedulerStoppedState(client, jobId); + } catch (Exception e) { + // ignore + } + client.performRequest("DELETE", "/_xpack/prelert/schedulers/" + schedulerId); + } + } + + private static void deleteAllJobs(RestClient client) throws IOException { Map clusterStateAsMap = entityAsMap(client.performRequest("GET", "/_cluster/state", Collections.singletonMap("filter_path", "metadata.prelert.jobs"))); @SuppressWarnings("unchecked") @@ -162,12 +204,6 @@ public class ScheduledJobIT extends ESRestTestCase { for (Map jobConfig : jobConfigs) { String jobId = (String) jobConfig.get("job_id"); - try { - client.performRequest("POST", "/_xpack/prelert/schedulers/" + jobId + "/_stop"); - waitForSchedulerStoppedState(client, jobId); - } catch (Exception e) { - // ignore - } try { Response response = client.performRequest("POST", "/_xpack/prelert/anomaly_detectors/" + jobId + "/_close"); assertThat(response.getStatusLine().getStatusCode(), equalTo(200)); diff --git a/elasticsearch/src/test/java/org/elasticsearch/xpack/prelert/job/JobTests.java b/elasticsearch/src/test/java/org/elasticsearch/xpack/prelert/job/JobTests.java index 118a80f4ed7..807858908ab 100644 --- a/elasticsearch/src/test/java/org/elasticsearch/xpack/prelert/job/JobTests.java +++ b/elasticsearch/src/test/java/org/elasticsearch/xpack/prelert/job/JobTests.java @@ -8,7 +8,6 @@ package org.elasticsearch.xpack.prelert.job; import com.carrotsearch.randomizedtesting.generators.CodepointSetGenerator; import org.elasticsearch.common.ParseFieldMatcher; import org.elasticsearch.common.io.stream.Writeable; -import org.elasticsearch.common.xcontent.XContentFactory; import org.elasticsearch.common.xcontent.XContentParser; import org.elasticsearch.test.ESTestCase; import org.elasticsearch.xpack.prelert.job.condition.Condition; @@ -18,7 +17,6 @@ import org.elasticsearch.xpack.prelert.job.transform.TransformConfig; import org.elasticsearch.xpack.prelert.job.transform.TransformType; import org.elasticsearch.xpack.prelert.support.AbstractSerializingTestCase; -import java.io.IOException; import java.util.ArrayList; import java.util.Arrays; import java.util.Collections; @@ -63,7 +61,6 @@ public class JobTests extends AbstractSerializingTestCase { assertNull(job.getBackgroundPersistInterval()); assertNull(job.getModelSnapshotRetentionDays()); assertNull(job.getResultsRetentionDays()); - assertNull(job.getSchedulerConfig()); assertEquals(Collections.emptyList(), job.getTransforms()); assertNotNull(job.allFields()); assertFalse(job.allFields().isEmpty()); @@ -79,19 +76,6 @@ public class JobTests extends AbstractSerializingTestCase { assertEquals(IgnoreDowntime.ONCE, job.getIgnoreDowntime()); } - public void testConstructor_GivenJobConfigurationWithScheduler_ShouldFillDefaults() { - Job.Builder builder = new Job.Builder("foo"); - DataDescription.Builder dataDescriptionBuilder = new DataDescription.Builder(); - dataDescriptionBuilder.setFormat(DataDescription.DataFormat.ELASTICSEARCH); - builder.setDataDescription(dataDescriptionBuilder); - Detector.Builder detectorBuilder = new Detector.Builder(); - detectorBuilder.setFunction("count"); - builder.setAnalysisConfig(new AnalysisConfig.Builder(Arrays.asList(detectorBuilder.build()))); - builder.setSchedulerConfig(new SchedulerConfig.Builder(Arrays.asList("my_index"), Arrays.asList("my_type"))); - - assertEquals(60L, builder.build().getSchedulerConfig().getQueryDelay().longValue()); - } - public void testEquals_noId() { expectThrows(IllegalArgumentException.class, () -> buildJobBuilder("").build(true)); assertNotNull(buildJobBuilder(null).build(true).getId()); // test auto id generation @@ -420,96 +404,6 @@ public class JobTests extends AbstractSerializingTestCase { assertEquals(errorMessage, e.getMessage()); } - public void testVerify_GivenElasticsearchSchedulerAndNonZeroLatency() { - String errorMessage = Messages.getMessage(Messages.JOB_CONFIG_SCHEDULER_ELASTICSEARCH_DOES_NOT_SUPPORT_LATENCY); - SchedulerConfig.Builder schedulerConfig = createValidElasticsearchSchedulerConfig(); - Job.Builder builder = buildJobBuilder("foo"); - builder.setSchedulerConfig(schedulerConfig); - AnalysisConfig.Builder ac = createAnalysisConfig(); - ac.setBucketSpan(1800L); - ac.setLatency(3600L); - builder.setAnalysisConfig(ac); - IllegalArgumentException e = ESTestCase.expectThrows(IllegalArgumentException.class, builder::build); - - assertEquals(errorMessage, e.getMessage()); - } - - public void testVerify_GivenElasticsearchSchedulerAndZeroLatency() { - SchedulerConfig.Builder schedulerConfig = createValidElasticsearchSchedulerConfig(); - Job.Builder builder = buildJobBuilder("foo"); - builder.setSchedulerConfig(schedulerConfig); - DataDescription.Builder dataDescription = new DataDescription.Builder(); - dataDescription.setFormat(DataDescription.DataFormat.ELASTICSEARCH); - builder.setDataDescription(dataDescription); - AnalysisConfig.Builder ac = createAnalysisConfig(); - ac.setBucketSpan(1800L); - ac.setLatency(0L); - builder.setAnalysisConfig(ac); - builder.build(); - } - - public void testVerify_GivenElasticsearchSchedulerAndNoLatency() { - SchedulerConfig.Builder schedulerConfig = createValidElasticsearchSchedulerConfig(); - Job.Builder builder = buildJobBuilder("foo"); - builder.setSchedulerConfig(schedulerConfig); - DataDescription.Builder dataDescription = new DataDescription.Builder(); - dataDescription.setFormat(DataDescription.DataFormat.ELASTICSEARCH); - builder.setDataDescription(dataDescription); - AnalysisConfig.Builder ac = createAnalysisConfig(); - ac.setBatchSpan(1800L); - ac.setBucketSpan(100L); - builder.setAnalysisConfig(ac); - builder.build(); - } - - public void testVerify_GivenElasticsearchSchedulerWithAggsAndCorrectSummaryCountField() throws IOException { - SchedulerConfig.Builder schedulerConfig = createValidElasticsearchSchedulerConfigWithAggs(); - Job.Builder builder = buildJobBuilder("foo"); - builder.setSchedulerConfig(schedulerConfig); - DataDescription.Builder dataDescription = new DataDescription.Builder(); - dataDescription.setFormat(DataDescription.DataFormat.ELASTICSEARCH); - builder.setDataDescription(dataDescription); - AnalysisConfig.Builder ac = createAnalysisConfig(); - ac.setBucketSpan(1800L); - ac.setSummaryCountFieldName("doc_count"); - builder.setAnalysisConfig(ac); - builder.build(); - } - - public void testVerify_GivenElasticsearchSchedulerWithAggsAndNoSummaryCountField() throws IOException { - String errorMessage = Messages.getMessage(Messages.JOB_CONFIG_SCHEDULER_AGGREGATIONS_REQUIRES_SUMMARY_COUNT_FIELD, - SchedulerConfig.DOC_COUNT); - SchedulerConfig.Builder schedulerConfig = createValidElasticsearchSchedulerConfigWithAggs(); - Job.Builder builder = buildJobBuilder("foo"); - builder.setSchedulerConfig(schedulerConfig); - DataDescription.Builder dataDescription = new DataDescription.Builder(); - dataDescription.setFormat(DataDescription.DataFormat.ELASTICSEARCH); - builder.setDataDescription(dataDescription); - AnalysisConfig.Builder ac = createAnalysisConfig(); - ac.setBucketSpan(1800L); - builder.setAnalysisConfig(ac); - IllegalArgumentException e = ESTestCase.expectThrows(IllegalArgumentException.class, builder::build); - - assertEquals(errorMessage, e.getMessage()); - } - - public void testVerify_GivenElasticsearchSchedulerWithAggsAndWrongSummaryCountField() throws IOException { - String errorMessage = Messages.getMessage( - Messages.JOB_CONFIG_SCHEDULER_AGGREGATIONS_REQUIRES_SUMMARY_COUNT_FIELD, SchedulerConfig.DOC_COUNT); - SchedulerConfig.Builder schedulerConfig = createValidElasticsearchSchedulerConfigWithAggs(); - Job.Builder builder = buildJobBuilder("foo"); - builder.setSchedulerConfig(schedulerConfig); - DataDescription.Builder dataDescription = new DataDescription.Builder(); - dataDescription.setFormat(DataDescription.DataFormat.ELASTICSEARCH); - builder.setDataDescription(dataDescription); - AnalysisConfig.Builder ac = createAnalysisConfig(); - ac.setBucketSpan(1800L); - ac.setSummaryCountFieldName("wrong"); - builder.setAnalysisConfig(ac); - IllegalArgumentException e = ESTestCase.expectThrows(IllegalArgumentException.class, builder::build); - assertEquals(errorMessage, e.getMessage()); - } - public static Job.Builder buildJobBuilder(String id) { Job.Builder builder = new Job.Builder(id); builder.setCreateTime(new Date()); @@ -520,42 +414,6 @@ public class JobTests extends AbstractSerializingTestCase { return builder; } - private static SchedulerConfig.Builder createValidElasticsearchSchedulerConfig() { - return new SchedulerConfig.Builder(Arrays.asList("myIndex"), Arrays.asList("myType")); - } - - private static SchedulerConfig.Builder createValidElasticsearchSchedulerConfigWithAggs() - throws IOException { - SchedulerConfig.Builder schedulerConfig = createValidElasticsearchSchedulerConfig(); - String aggStr = - "{" + - "\"buckets\" : {" + - "\"histogram\" : {" + - "\"field\" : \"time\"," + - "\"interval\" : 3600000" + - "}," + - "\"aggs\" : {" + - "\"byField\" : {" + - "\"terms\" : {" + - "\"field\" : \"airline\"," + - "\"size\" : 0" + - "}," + - "\"aggs\" : {" + - "\"stats\" : {" + - "\"stats\" : {" + - "\"field\" : \"responsetime\"" + - "}" + - "}" + - "}" + - "}" + - "}" + - "} " + - "}"; - XContentParser parser = XContentFactory.xContent(aggStr).createParser(aggStr); - schedulerConfig.setAggs(parser.map()); - return schedulerConfig; - } - public static String randomValidJobId() { CodepointSetGenerator generator = new CodepointSetGenerator("abcdefghijklmnopqrstuvwxyz".toCharArray()); return generator.ofCodePointsLength(random(), 10, 10); @@ -592,13 +450,6 @@ public class JobTests extends AbstractSerializingTestCase { if (randomBoolean()) { builder.setDataDescription(new DataDescription.Builder()); } - if (randomBoolean()) { - SchedulerConfig.Builder schedulerConfig = new SchedulerConfig.Builder(randomStringList(1, 10), randomStringList(1, 10)); - builder.setSchedulerConfig(schedulerConfig); - DataDescription.Builder dataDescriptionBuilder = new DataDescription.Builder(); - dataDescriptionBuilder.setFormat(DataDescription.DataFormat.ELASTICSEARCH); - builder.setDataDescription(dataDescriptionBuilder); - } String[] outputs; TransformType[] transformTypes ; AnalysisConfig ac = analysisConfig.build(); diff --git a/elasticsearch/src/test/java/org/elasticsearch/xpack/prelert/job/SchedulerConfigTests.java b/elasticsearch/src/test/java/org/elasticsearch/xpack/prelert/job/SchedulerConfigTests.java index 538e8c3eeb6..7cb5e891d9a 100644 --- a/elasticsearch/src/test/java/org/elasticsearch/xpack/prelert/job/SchedulerConfigTests.java +++ b/elasticsearch/src/test/java/org/elasticsearch/xpack/prelert/job/SchedulerConfigTests.java @@ -5,6 +5,7 @@ */ package org.elasticsearch.xpack.prelert.job; +import com.carrotsearch.randomizedtesting.generators.CodepointSetGenerator; import org.apache.logging.log4j.Logger; import org.elasticsearch.common.ParseFieldMatcher; import org.elasticsearch.common.io.stream.Writeable; @@ -27,7 +28,9 @@ public class SchedulerConfigTests extends AbstractSerializingTestCase ParseFieldMatcher.STRICT).build(); - assertNotNull(jobConfig); - - SchedulerConfig.Builder schedulerConfig = new SchedulerConfig.Builder(jobConfig.getSchedulerConfig()); + XContentParser parser = XContentFactory.xContent(schedulerConfigStr).createParser(schedulerConfigStr); + SchedulerConfig schedulerConfig = SchedulerConfig.PARSER.apply(parser, () -> ParseFieldMatcher.STRICT).build(); assertNotNull(schedulerConfig); Map query = schedulerConfig.getQuery(); @@ -102,7 +98,9 @@ public class SchedulerConfigTests extends AbstractSerializingTestCase ParseFieldMatcher.STRICT).build(); - assertNotNull(jobConfig); - - SchedulerConfig schedulerConfig = jobConfig.getSchedulerConfig(); + XContentParser parser = XContentFactory.xContent(configStr).createParser(configStr); + SchedulerConfig schedulerConfig = SchedulerConfig.PARSER.apply(parser, () -> ParseFieldMatcher.STRICT).build(); assertNotNull(schedulerConfig); Map aggs = schedulerConfig.getAggregationsOrAggs(); @@ -145,56 +136,23 @@ public class SchedulerConfigTests extends AbstractSerializingTestCase()); - originalSchedulerConfig.setQueryDelay(30L); - originalSchedulerConfig.setRetrieveWholeSource(true); - originalSchedulerConfig.setScrollSize(2000); - - SchedulerConfig.Builder defaultedSchedulerConfig = new SchedulerConfig.Builder(Arrays.asList("index"), Arrays.asList("type")); - defaultedSchedulerConfig.setQuery(new HashMap<>()); - defaultedSchedulerConfig.setQueryDelay(30L); - defaultedSchedulerConfig.setRetrieveWholeSource(true); - defaultedSchedulerConfig.setScrollSize(2000); - - assertEquals(originalSchedulerConfig.build(), defaultedSchedulerConfig.build()); - } - - public void testFillDefaults_GivenDataSourceIsElasticsearchAndDefaultsAreApplied() { - SchedulerConfig.Builder expectedSchedulerConfig = new SchedulerConfig.Builder(Arrays.asList("index"), Arrays.asList("type")); + public void testFillDefaults() { + SchedulerConfig.Builder expectedSchedulerConfig = new SchedulerConfig.Builder("scheduler1", "job1"); + expectedSchedulerConfig.setIndexes(Arrays.asList("index")); + expectedSchedulerConfig.setTypes(Arrays.asList("type")); Map defaultQuery = new HashMap<>(); defaultQuery.put("match_all", new HashMap()); expectedSchedulerConfig.setQuery(defaultQuery); expectedSchedulerConfig.setQueryDelay(60L); expectedSchedulerConfig.setRetrieveWholeSource(false); expectedSchedulerConfig.setScrollSize(1000); - SchedulerConfig.Builder defaultedSchedulerConfig = new SchedulerConfig.Builder(Arrays.asList("index"), Arrays.asList("type")); + SchedulerConfig.Builder defaultedSchedulerConfig = new SchedulerConfig.Builder("scheduler1", "job1"); + defaultedSchedulerConfig.setIndexes(Arrays.asList("index")); + defaultedSchedulerConfig.setTypes(Arrays.asList("type")); + assertEquals(expectedSchedulerConfig.build(), defaultedSchedulerConfig.build()); } - public void testEquals_GivenDifferentClass() { - SchedulerConfig.Builder builder = new SchedulerConfig.Builder(Arrays.asList("index"), Arrays.asList("type")); - assertFalse(builder.build().equals("a string")); - } - - public void testEquals_GivenSameRef() { - SchedulerConfig.Builder builder = new SchedulerConfig.Builder(Arrays.asList("index"), Arrays.asList("type")); - SchedulerConfig schedulerConfig = builder.build(); - assertTrue(schedulerConfig.equals(schedulerConfig)); - } - - public void testEquals_GivenEqual() { - SchedulerConfig.Builder b1 = createFullyPopulated(); - SchedulerConfig.Builder b2 = createFullyPopulated(); - - SchedulerConfig sc1 = b1.build(); - SchedulerConfig sc2 = b2.build(); - assertTrue(sc1.equals(sc2)); - assertTrue(sc2.equals(sc1)); - assertEquals(sc1.hashCode(), sc2.hashCode()); - } - public void testEquals_GivenDifferentQueryDelay() { SchedulerConfig.Builder b1 = createFullyPopulated(); SchedulerConfig.Builder b2 = createFullyPopulated(); @@ -229,18 +187,18 @@ public class SchedulerConfigTests extends AbstractSerializingTestCase query = new HashMap<>(); @@ -282,8 +242,10 @@ public class SchedulerConfigTests extends AbstractSerializingTestCase()), conf.build().getQuery()); } - public void testCheckValidElasticsearch_GivenScriptFieldsNotWholeSource() throws IOException { - SchedulerConfig.Builder conf = new SchedulerConfig.Builder(Arrays.asList("myindex"), Arrays.asList("mytype")); + public void testCheckValid_GivenScriptFieldsNotWholeSource() throws IOException { + SchedulerConfig.Builder conf = new SchedulerConfig.Builder("scheduler1", "job1"); + conf.setIndexes(Arrays.asList("myindex")); + conf.setTypes(Arrays.asList("mytype")); String json = "{ \"twiceresponsetime\" : { \"script\" : { \"lang\" : \"expression\", " + "\"inline\" : \"doc['responsetime'].value * 2\" } } }"; XContentParser parser = XContentFactory.xContent(json).createParser(json); @@ -307,8 +273,10 @@ public class SchedulerConfigTests extends AbstractSerializingTestCase new SchedulerConfig.Builder(null, Arrays.asList("mytype"))); + public void testCheckValid_GivenNullIndexes() throws IOException { + SchedulerConfig.Builder conf = new SchedulerConfig.Builder("scheduler1", "job1"); + expectThrows(IllegalArgumentException.class, () -> conf.setIndexes(null)); } - public void testCheckValidElasticsearch_GivenEmptyIndexes() throws IOException { - SchedulerConfig.Builder conf = new SchedulerConfig.Builder(Collections.emptyList(), Arrays.asList("mytype")); + public void testCheckValid_GivenEmptyIndexes() throws IOException { + SchedulerConfig.Builder conf = new SchedulerConfig.Builder("scheduler1", "job1"); + conf.setIndexes(Collections.emptyList()); IllegalArgumentException e = ESTestCase.expectThrows(IllegalArgumentException.class, conf::build); - assertEquals(Messages.getMessage(Messages.JOB_CONFIG_SCHEDULER_INVALID_OPTION_VALUE, "indexes", "[]"), e.getMessage()); + assertEquals(Messages.getMessage(Messages.SCHEDULER_CONFIG_INVALID_OPTION_VALUE, "indexes", "[]"), e.getMessage()); } - public void testCheckValidElasticsearch_GivenIndexesContainsOnlyNulls() throws IOException { + public void testCheckValid_GivenIndexesContainsOnlyNulls() throws IOException { List indexes = new ArrayList<>(); indexes.add(null); indexes.add(null); - SchedulerConfig.Builder conf = new SchedulerConfig.Builder(indexes, Arrays.asList("mytype")); + SchedulerConfig.Builder conf = new SchedulerConfig.Builder("scheduler1", "job1"); + conf.setIndexes(indexes); IllegalArgumentException e = ESTestCase.expectThrows(IllegalArgumentException.class, conf::build); - assertEquals(Messages.getMessage(Messages.JOB_CONFIG_SCHEDULER_INVALID_OPTION_VALUE, "indexes", "[null, null]"), e.getMessage()); + assertEquals(Messages.getMessage(Messages.SCHEDULER_CONFIG_INVALID_OPTION_VALUE, "indexes", "[null, null]"), e.getMessage()); } - public void testCheckValidElasticsearch_GivenIndexesContainsOnlyEmptyStrings() throws IOException { + public void testCheckValid_GivenIndexesContainsOnlyEmptyStrings() throws IOException { List indexes = new ArrayList<>(); indexes.add(""); indexes.add(""); - SchedulerConfig.Builder conf = new SchedulerConfig.Builder(indexes, Arrays.asList("mytype")); + SchedulerConfig.Builder conf = new SchedulerConfig.Builder("scheduler1", "job1"); + conf.setIndexes(indexes); IllegalArgumentException e = ESTestCase.expectThrows(IllegalArgumentException.class, conf::build); - assertEquals(Messages.getMessage(Messages.JOB_CONFIG_SCHEDULER_INVALID_OPTION_VALUE, "indexes", "[, ]"), e.getMessage()); + assertEquals(Messages.getMessage(Messages.SCHEDULER_CONFIG_INVALID_OPTION_VALUE, "indexes", "[, ]"), e.getMessage()); } - public void testCheckValidElasticsearch_GivenNegativeQueryDelay() throws IOException { - SchedulerConfig.Builder conf = new SchedulerConfig.Builder(Arrays.asList("myindex"), Arrays.asList("mytype")); + public void testCheckValid_GivenNegativeQueryDelay() throws IOException { + SchedulerConfig.Builder conf = new SchedulerConfig.Builder("scheduler1", "job1"); IllegalArgumentException e = ESTestCase.expectThrows(IllegalArgumentException.class, () -> conf.setQueryDelay(-10L)); - assertEquals(Messages.getMessage(Messages.JOB_CONFIG_SCHEDULER_INVALID_OPTION_VALUE, "query_delay", -10L), e.getMessage()); + assertEquals(Messages.getMessage(Messages.SCHEDULER_CONFIG_INVALID_OPTION_VALUE, "query_delay", -10L), e.getMessage()); } - public void testCheckValidElasticsearch_GivenZeroFrequency() throws IOException { - SchedulerConfig.Builder conf = new SchedulerConfig.Builder(Arrays.asList("myindex"), Arrays.asList("mytype")); + public void testCheckValid_GivenZeroFrequency() throws IOException { + SchedulerConfig.Builder conf = new SchedulerConfig.Builder("scheduler1", "job1"); IllegalArgumentException e = ESTestCase.expectThrows(IllegalArgumentException.class, () -> conf.setFrequency(0L)); - assertEquals(Messages.getMessage(Messages.JOB_CONFIG_SCHEDULER_INVALID_OPTION_VALUE, "frequency", 0L), e.getMessage()); + assertEquals(Messages.getMessage(Messages.SCHEDULER_CONFIG_INVALID_OPTION_VALUE, "frequency", 0L), e.getMessage()); } - public void testCheckValidElasticsearch_GivenNegativeFrequency() throws IOException { - SchedulerConfig.Builder conf = new SchedulerConfig.Builder(Arrays.asList("myindex"), Arrays.asList("mytype")); + public void testCheckValid_GivenNegativeFrequency() throws IOException { + SchedulerConfig.Builder conf = new SchedulerConfig.Builder("scheduler1", "job1"); IllegalArgumentException e = ESTestCase.expectThrows(IllegalArgumentException.class, () -> conf.setFrequency(-600L)); - assertEquals(Messages.getMessage(Messages.JOB_CONFIG_SCHEDULER_INVALID_OPTION_VALUE, "frequency", -600L), e.getMessage()); + assertEquals(Messages.getMessage(Messages.SCHEDULER_CONFIG_INVALID_OPTION_VALUE, "frequency", -600L), e.getMessage()); } - public void testCheckValidElasticsearch_GivenNegativeScrollSize() throws IOException { - SchedulerConfig.Builder conf = new SchedulerConfig.Builder(Arrays.asList("myindex"), Arrays.asList("mytype")); + public void testCheckValid_GivenNegativeScrollSize() throws IOException { + SchedulerConfig.Builder conf = new SchedulerConfig.Builder("scheduler1", "job1"); IllegalArgumentException e = ESTestCase.expectThrows(IllegalArgumentException.class, () -> conf.setScrollSize(-1000)); - assertEquals(Messages.getMessage(Messages.JOB_CONFIG_SCHEDULER_INVALID_OPTION_VALUE, "scroll_size", -1000L), e.getMessage()); + assertEquals(Messages.getMessage(Messages.SCHEDULER_CONFIG_INVALID_OPTION_VALUE, "scroll_size", -1000L), e.getMessage()); } - public void testCheckValidElasticsearch_GivenBothAggregationsAndAggsAreSet() { - SchedulerConfig.Builder conf = new SchedulerConfig.Builder(Arrays.asList("myindex"), Arrays.asList("mytype")); + public void testCheckValid_GivenBothAggregationsAndAggsAreSet() { + SchedulerConfig.Builder conf = new SchedulerConfig.Builder("scheduler1", "job1"); + conf.setIndexes(Arrays.asList("myindex")); + conf.setTypes(Arrays.asList("mytype")); conf.setScrollSize(1000); Map aggs = new HashMap<>(); conf.setAggregations(aggs); conf.setAggs(aggs); IllegalArgumentException e = ESTestCase.expectThrows(IllegalArgumentException.class, conf::build); - assertEquals(Messages.getMessage(Messages.JOB_CONFIG_SCHEDULER_MULTIPLE_AGGREGATIONS), e.getMessage()); + assertEquals(Messages.getMessage(Messages.SCHEDULER_CONFIG_MULTIPLE_AGGREGATIONS), e.getMessage()); + } + + public static String randomValidSchedulerId() { + CodepointSetGenerator generator = new CodepointSetGenerator("abcdefghijklmnopqrstuvwxyz".toCharArray()); + return generator.ofCodePointsLength(random(), 10, 10); } } diff --git a/elasticsearch/src/test/java/org/elasticsearch/xpack/prelert/job/metadata/PrelertMetadataTests.java b/elasticsearch/src/test/java/org/elasticsearch/xpack/prelert/job/metadata/PrelertMetadataTests.java index 0f9276f3cd3..b4296d5f09e 100644 --- a/elasticsearch/src/test/java/org/elasticsearch/xpack/prelert/job/metadata/PrelertMetadataTests.java +++ b/elasticsearch/src/test/java/org/elasticsearch/xpack/prelert/job/metadata/PrelertMetadataTests.java @@ -16,9 +16,11 @@ 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.prelert.job.DataDescription; import org.elasticsearch.xpack.prelert.job.Job; import org.elasticsearch.xpack.prelert.job.JobStatus; import org.elasticsearch.xpack.prelert.job.JobTests; +import org.elasticsearch.xpack.prelert.job.SchedulerConfig; import org.elasticsearch.xpack.prelert.job.SchedulerStatus; import org.elasticsearch.xpack.prelert.support.AbstractSerializingTestCase; @@ -26,6 +28,7 @@ import java.io.IOException; import static org.elasticsearch.xpack.prelert.job.JobTests.buildJobBuilder; import static org.elasticsearch.xpack.prelert.job.scheduler.ScheduledJobRunnerTests.createScheduledJob; +import static org.elasticsearch.xpack.prelert.job.scheduler.ScheduledJobRunnerTests.createSchedulerConfig; import static org.hamcrest.Matchers.equalTo; import static org.hamcrest.Matchers.notNullValue; import static org.hamcrest.Matchers.nullValue; @@ -92,10 +95,10 @@ public class PrelertMetadataTests extends AbstractSerializingTestCase builder2.removeJob("1")); assertThat(e.status(), equalTo(RestStatus.CONFLICT)); } + public void testRemoveJob_failSchedulerRefersToJob() { + Job job1 = createScheduledJob().build(); + SchedulerConfig schedulerConfig1 = createSchedulerConfig("scheduler1", job1.getId()).build(); + PrelertMetadata.Builder builder = new PrelertMetadata.Builder(); + builder.putJob(job1, false); + builder.putScheduler(schedulerConfig1); + + ElasticsearchStatusException e = expectThrows(ElasticsearchStatusException.class, () -> builder.removeJob(job1.getId())); + assertThat(e.status(), equalTo(RestStatus.CONFLICT)); + String expectedMsg = "Cannot delete job [" + job1.getId() + "] while scheduler [" + schedulerConfig1.getId() + "] refers to it"; + assertThat(e.getMessage(), equalTo(expectedMsg)); + } + public void testRemoveJob_failBecauseJobDoesNotExist() { PrelertMetadata.Builder builder1 = new PrelertMetadata.Builder(); expectThrows(ResourceNotFoundException.class, () -> builder1.removeJob("1")); } - public void testCrudScheduledJob() { + public void testCrudScheduler() { Job job1 = createScheduledJob().build(); + SchedulerConfig schedulerConfig1 = createSchedulerConfig("scheduler1", job1.getId()).build(); PrelertMetadata.Builder builder = new PrelertMetadata.Builder(); builder.putJob(job1, false); + builder.putScheduler(schedulerConfig1); PrelertMetadata result = builder.build(); assertThat(result.getJobs().get("foo"), sameInstance(job1)); assertThat(result.getAllocations().get("foo").getStatus(), equalTo(JobStatus.CLOSED)); - assertThat(result.getSchedulerStatuses().get("foo"), equalTo(SchedulerStatus.STOPPED)); + assertThat(result.getSchedulers().get("scheduler1").getConfig(), sameInstance(schedulerConfig1)); + assertThat(result.getSchedulers().get("scheduler1").getStatus(), equalTo(SchedulerStatus.STOPPED)); builder = new PrelertMetadata.Builder(result); - builder.removeJob("foo"); + builder.removeScheduler("scheduler1"); result = builder.build(); - assertThat(result.getJobs().get("foo"), nullValue()); - assertThat(result.getAllocations().get("foo"), nullValue()); - assertThat(result.getSchedulerStatuses().get("foo"), nullValue()); + assertThat(result.getJobs().get("foo"), sameInstance(job1)); + assertThat(result.getAllocations().get("foo").getStatus(), equalTo(JobStatus.CLOSED)); + assertThat(result.getSchedulers().get("scheduler1"), nullValue()); } - public void testDeletedScheduledJob_failBecauseSchedulerStarted() { + public void testPutScheduler_failBecauseJobDoesNotExist() { + SchedulerConfig schedulerConfig1 = createSchedulerConfig("scheduler1", "missing-job").build(); + PrelertMetadata.Builder builder = new PrelertMetadata.Builder(); + + expectThrows(ResourceNotFoundException.class, () -> builder.putScheduler(schedulerConfig1)); + } + + public void testPutScheduler_failBecauseSchedulerIdIsAlreadyTaken() { Job job1 = createScheduledJob().build(); + SchedulerConfig schedulerConfig1 = createSchedulerConfig("scheduler1", job1.getId()).build(); PrelertMetadata.Builder builder = new PrelertMetadata.Builder(); builder.putJob(job1, false); + builder.putScheduler(schedulerConfig1); + + expectThrows(ResourceAlreadyExistsException.class, () -> builder.putScheduler(schedulerConfig1)); + } + + public void testPutScheduler_failBecauseJobAlreadyHasScheduler() { + Job job1 = createScheduledJob().build(); + SchedulerConfig schedulerConfig1 = createSchedulerConfig("scheduler1", job1.getId()).build(); + SchedulerConfig schedulerConfig2 = createSchedulerConfig("scheduler2", job1.getId()).build(); + PrelertMetadata.Builder builder = new PrelertMetadata.Builder(); + builder.putJob(job1, false); + builder.putScheduler(schedulerConfig1); + + ElasticsearchStatusException e = expectThrows(ElasticsearchStatusException.class, () -> builder.putScheduler(schedulerConfig2)); + assertThat(e.status(), equalTo(RestStatus.CONFLICT)); + } + + public void testPutScheduler_failBecauseJobIsNotCompatibleForScheduler() { + Job.Builder job1 = createScheduledJob(); + DataDescription.Builder dataDescription = new DataDescription.Builder(); + dataDescription.setFormat(DataDescription.DataFormat.DELIMITED); + job1.setDataDescription(dataDescription); + SchedulerConfig schedulerConfig1 = createSchedulerConfig("scheduler1", job1.getId()).build(); + PrelertMetadata.Builder builder = new PrelertMetadata.Builder(); + builder.putJob(job1.build(), false); + + expectThrows(IllegalArgumentException.class, () -> builder.putScheduler(schedulerConfig1)); + } + + public void testRemoveScheduler_failBecauseSchedulerStarted() { + Job job1 = createScheduledJob().build(); + SchedulerConfig schedulerConfig1 = createSchedulerConfig("scheduler1", job1.getId()).build(); + PrelertMetadata.Builder builder = new PrelertMetadata.Builder(); + builder.putJob(job1, false); + builder.putScheduler(schedulerConfig1); builder.updateStatus("foo", JobStatus.OPENING, null); builder.updateStatus("foo", JobStatus.OPENED, null); - builder.updateSchedulerStatus("foo", SchedulerStatus.STARTED); + builder.updateSchedulerStatus("scheduler1", SchedulerStatus.STARTED); PrelertMetadata result = builder.build(); assertThat(result.getJobs().get("foo"), sameInstance(job1)); assertThat(result.getAllocations().get("foo").getStatus(), equalTo(JobStatus.OPENED)); - assertThat(result.getSchedulerStatuses().get("foo"), equalTo(SchedulerStatus.STARTED)); + assertThat(result.getSchedulers().get("scheduler1").getConfig(), sameInstance(schedulerConfig1)); + assertThat(result.getSchedulers().get("scheduler1").getStatus(), equalTo(SchedulerStatus.STARTED)); PrelertMetadata.Builder builder2 = new PrelertMetadata.Builder(result); - ElasticsearchStatusException e = expectThrows(ElasticsearchStatusException.class, () -> builder2.removeJob("foo")); + ElasticsearchStatusException e = expectThrows(ElasticsearchStatusException.class, () -> builder2.removeScheduler("scheduler1")); assertThat(e.status(), equalTo(RestStatus.CONFLICT)); } diff --git a/elasticsearch/src/test/java/org/elasticsearch/xpack/prelert/job/process/autodetect/writer/DataToProcessWriterFactoryTests.java b/elasticsearch/src/test/java/org/elasticsearch/xpack/prelert/job/process/autodetect/writer/DataToProcessWriterFactoryTests.java index f3413b10ff3..1cca583678f 100644 --- a/elasticsearch/src/test/java/org/elasticsearch/xpack/prelert/job/process/autodetect/writer/DataToProcessWriterFactoryTests.java +++ b/elasticsearch/src/test/java/org/elasticsearch/xpack/prelert/job/process/autodetect/writer/DataToProcessWriterFactoryTests.java @@ -5,19 +5,17 @@ */ package org.elasticsearch.xpack.prelert.job.process.autodetect.writer; -import static org.mockito.Mockito.mock; - import org.apache.logging.log4j.Logger; import org.elasticsearch.test.ESTestCase; - import org.elasticsearch.xpack.prelert.job.AnalysisConfig; import org.elasticsearch.xpack.prelert.job.DataDescription; import org.elasticsearch.xpack.prelert.job.DataDescription.DataFormat; -import org.elasticsearch.xpack.prelert.job.SchedulerConfig; import org.elasticsearch.xpack.prelert.job.process.autodetect.AutodetectProcess; import org.elasticsearch.xpack.prelert.job.status.StatusReporter; import org.elasticsearch.xpack.prelert.job.transform.TransformConfigs; +import static org.mockito.Mockito.mock; + public class DataToProcessWriterFactoryTests extends ESTestCase { public void testCreate_GivenDataFormatIsJson() { DataDescription.Builder dataDescription = new DataDescription.Builder(); @@ -49,7 +47,6 @@ public class DataToProcessWriterFactoryTests extends ESTestCase { private static DataToProcessWriter createWriter(DataDescription dataDescription) { return DataToProcessWriterFactory.create(true, mock(AutodetectProcess.class), dataDescription, - mock(AnalysisConfig.class), mock(SchedulerConfig.class), mock(TransformConfigs.class), - mock(StatusReporter.class), mock(Logger.class)); + mock(AnalysisConfig.class), mock(TransformConfigs.class), mock(StatusReporter.class), mock(Logger.class)); } } diff --git a/elasticsearch/src/test/java/org/elasticsearch/xpack/prelert/job/process/autodetect/writer/DataWithTransformsToProcessWriterTests.java b/elasticsearch/src/test/java/org/elasticsearch/xpack/prelert/job/process/autodetect/writer/DataWithTransformsToProcessWriterTests.java index 112b02f482b..728690afdf6 100644 --- a/elasticsearch/src/test/java/org/elasticsearch/xpack/prelert/job/process/autodetect/writer/DataWithTransformsToProcessWriterTests.java +++ b/elasticsearch/src/test/java/org/elasticsearch/xpack/prelert/job/process/autodetect/writer/DataWithTransformsToProcessWriterTests.java @@ -5,9 +5,21 @@ */ package org.elasticsearch.xpack.prelert.job.process.autodetect.writer; -import static org.mockito.Matchers.any; -import static org.mockito.Mockito.doAnswer; -import static org.mockito.Mockito.verify; +import org.apache.logging.log4j.Logger; +import org.elasticsearch.test.ESTestCase; +import org.elasticsearch.xpack.prelert.job.AnalysisConfig; +import org.elasticsearch.xpack.prelert.job.DataDescription; +import org.elasticsearch.xpack.prelert.job.DataDescription.DataFormat; +import org.elasticsearch.xpack.prelert.job.Detector; +import org.elasticsearch.xpack.prelert.job.process.autodetect.AutodetectProcess; +import org.elasticsearch.xpack.prelert.job.status.StatusReporter; +import org.elasticsearch.xpack.prelert.job.transform.TransformConfig; +import org.elasticsearch.xpack.prelert.job.transform.TransformConfigs; +import org.elasticsearch.xpack.prelert.job.transform.TransformType; +import org.junit.Before; +import org.mockito.Mockito; +import org.mockito.invocation.InvocationOnMock; +import org.mockito.stubbing.Answer; import java.io.ByteArrayInputStream; import java.io.IOException; @@ -17,22 +29,9 @@ import java.util.ArrayList; import java.util.Arrays; import java.util.List; -import org.apache.logging.log4j.Logger; -import org.elasticsearch.test.ESTestCase; -import org.elasticsearch.xpack.prelert.job.process.autodetect.AutodetectProcess; -import org.junit.Before; -import org.mockito.Mockito; -import org.mockito.invocation.InvocationOnMock; -import org.mockito.stubbing.Answer; - -import org.elasticsearch.xpack.prelert.job.AnalysisConfig; -import org.elasticsearch.xpack.prelert.job.DataDescription; -import org.elasticsearch.xpack.prelert.job.DataDescription.DataFormat; -import org.elasticsearch.xpack.prelert.job.Detector; -import org.elasticsearch.xpack.prelert.job.status.StatusReporter; -import org.elasticsearch.xpack.prelert.job.transform.TransformConfig; -import org.elasticsearch.xpack.prelert.job.transform.TransformConfigs; -import org.elasticsearch.xpack.prelert.job.transform.TransformType; +import static org.mockito.Matchers.any; +import static org.mockito.Mockito.doAnswer; +import static org.mockito.Mockito.verify; public class DataWithTransformsToProcessWriterTests extends ESTestCase { private AutodetectProcess autodetectProcess; @@ -129,7 +128,7 @@ public class DataWithTransformsToProcessWriterTests extends ESTestCase { if (doCsv) { return new CsvDataToProcessWriter(true, autodetectProcess, dd.build(), ac, tcs, statusReporter, logger); } else { - return new JsonDataToProcessWriter(true, autodetectProcess, dd.build(), ac, null, tcs, statusReporter, logger); + return new JsonDataToProcessWriter(true, autodetectProcess, dd.build(), ac, tcs, statusReporter, logger); } } diff --git a/elasticsearch/src/test/java/org/elasticsearch/xpack/prelert/job/process/autodetect/writer/JsonDataToProcessWriterTests.java b/elasticsearch/src/test/java/org/elasticsearch/xpack/prelert/job/process/autodetect/writer/JsonDataToProcessWriterTests.java index e54679296bb..08a63e6a356 100644 --- a/elasticsearch/src/test/java/org/elasticsearch/xpack/prelert/job/process/autodetect/writer/JsonDataToProcessWriterTests.java +++ b/elasticsearch/src/test/java/org/elasticsearch/xpack/prelert/job/process/autodetect/writer/JsonDataToProcessWriterTests.java @@ -5,13 +5,23 @@ */ package org.elasticsearch.xpack.prelert.job.process.autodetect.writer; -import static org.mockito.Matchers.any; -import static org.mockito.Matchers.anyLong; -import static org.mockito.Mockito.atLeastOnce; -import static org.mockito.Mockito.doAnswer; -import static org.mockito.Mockito.never; -import static org.mockito.Mockito.times; -import static org.mockito.Mockito.verify; +import org.apache.logging.log4j.Logger; +import org.elasticsearch.ElasticsearchParseException; +import org.elasticsearch.tasks.TaskCancelledException; +import org.elasticsearch.test.ESTestCase; +import org.elasticsearch.xpack.prelert.job.AnalysisConfig; +import org.elasticsearch.xpack.prelert.job.DataDescription; +import org.elasticsearch.xpack.prelert.job.DataDescription.DataFormat; +import org.elasticsearch.xpack.prelert.job.Detector; +import org.elasticsearch.xpack.prelert.job.process.autodetect.AutodetectProcess; +import org.elasticsearch.xpack.prelert.job.status.StatusReporter; +import org.elasticsearch.xpack.prelert.job.transform.TransformConfig; +import org.elasticsearch.xpack.prelert.job.transform.TransformConfigs; +import org.elasticsearch.xpack.prelert.job.transform.TransformType; +import org.junit.Before; +import org.mockito.Mockito; +import org.mockito.invocation.InvocationOnMock; +import org.mockito.stubbing.Answer; import java.io.ByteArrayInputStream; import java.io.IOException; @@ -23,24 +33,13 @@ import java.util.List; import java.util.concurrent.atomic.AtomicBoolean; import java.util.concurrent.atomic.AtomicReference; -import org.apache.logging.log4j.Logger; -import org.elasticsearch.ElasticsearchParseException; -import org.elasticsearch.tasks.TaskCancelledException; -import org.elasticsearch.test.ESTestCase; -import org.elasticsearch.xpack.prelert.job.process.autodetect.AutodetectProcess; -import org.junit.Before; -import org.mockito.Mockito; -import org.mockito.invocation.InvocationOnMock; -import org.mockito.stubbing.Answer; - -import org.elasticsearch.xpack.prelert.job.AnalysisConfig; -import org.elasticsearch.xpack.prelert.job.DataDescription; -import org.elasticsearch.xpack.prelert.job.DataDescription.DataFormat; -import org.elasticsearch.xpack.prelert.job.Detector; -import org.elasticsearch.xpack.prelert.job.status.StatusReporter; -import org.elasticsearch.xpack.prelert.job.transform.TransformConfig; -import org.elasticsearch.xpack.prelert.job.transform.TransformConfigs; -import org.elasticsearch.xpack.prelert.job.transform.TransformType; +import static org.mockito.Matchers.any; +import static org.mockito.Matchers.anyLong; +import static org.mockito.Mockito.atLeastOnce; +import static org.mockito.Mockito.doAnswer; +import static org.mockito.Mockito.never; +import static org.mockito.Mockito.times; +import static org.mockito.Mockito.verify; public class JsonDataToProcessWriterTests extends ESTestCase { @@ -390,9 +389,8 @@ public class JsonDataToProcessWriterTests extends ESTestCase { } private JsonDataToProcessWriter createWriter() { - return new JsonDataToProcessWriter(true, autodetectProcess, dataDescription.build(), - analysisConfig, null, new TransformConfigs(transforms), - statusReporter, logger); + return new JsonDataToProcessWriter(true, autodetectProcess, dataDescription.build(), analysisConfig, + new TransformConfigs(transforms), statusReporter, logger); } private void assertWrittenRecordsEqualTo(List expectedRecords) { diff --git a/elasticsearch/src/test/java/org/elasticsearch/xpack/prelert/job/scheduler/ScheduledJobRunnerTests.java b/elasticsearch/src/test/java/org/elasticsearch/xpack/prelert/job/scheduler/ScheduledJobRunnerTests.java index 869767643cb..48780d3b050 100644 --- a/elasticsearch/src/test/java/org/elasticsearch/xpack/prelert/job/scheduler/ScheduledJobRunnerTests.java +++ b/elasticsearch/src/test/java/org/elasticsearch/xpack/prelert/job/scheduler/ScheduledJobRunnerTests.java @@ -6,6 +6,7 @@ package org.elasticsearch.xpack.prelert.job.scheduler; import org.elasticsearch.ElasticsearchStatusException; +import org.elasticsearch.ResourceNotFoundException; import org.elasticsearch.action.ActionFuture; import org.elasticsearch.action.ActionListener; import org.elasticsearch.client.Client; @@ -19,18 +20,17 @@ import org.elasticsearch.threadpool.ThreadPool; import org.elasticsearch.xpack.prelert.PrelertPlugin; import org.elasticsearch.xpack.prelert.action.FlushJobAction; import org.elasticsearch.xpack.prelert.action.JobDataAction; -import org.elasticsearch.xpack.prelert.action.StartJobSchedulerAction; -import org.elasticsearch.xpack.prelert.action.UpdateJobSchedulerStatusAction; +import org.elasticsearch.xpack.prelert.action.StartSchedulerAction; +import org.elasticsearch.xpack.prelert.action.UpdateSchedulerStatusAction; import org.elasticsearch.xpack.prelert.job.AnalysisConfig; import org.elasticsearch.xpack.prelert.job.DataCounts; import org.elasticsearch.xpack.prelert.job.DataDescription; import org.elasticsearch.xpack.prelert.job.Detector; import org.elasticsearch.xpack.prelert.job.Job; -import org.elasticsearch.xpack.prelert.job.SchedulerStatus; import org.elasticsearch.xpack.prelert.job.JobStatus; import org.elasticsearch.xpack.prelert.job.SchedulerConfig; +import org.elasticsearch.xpack.prelert.job.SchedulerStatus; import org.elasticsearch.xpack.prelert.job.audit.Auditor; -import org.elasticsearch.xpack.prelert.job.data.DataProcessor; import org.elasticsearch.xpack.prelert.job.extraction.DataExtractor; import org.elasticsearch.xpack.prelert.job.extraction.DataExtractorFactory; import org.elasticsearch.xpack.prelert.job.metadata.PrelertMetadata; @@ -40,7 +40,6 @@ import org.elasticsearch.xpack.prelert.job.persistence.QueryPage; import org.junit.Before; import java.io.ByteArrayInputStream; -import java.io.IOException; import java.io.InputStream; import java.nio.charset.Charset; import java.util.Arrays; @@ -49,9 +48,8 @@ import java.util.Optional; import java.util.concurrent.ExecutorService; import java.util.function.Consumer; -import static org.elasticsearch.xpack.prelert.action.UpdateJobSchedulerStatusAction.INSTANCE; -import static org.elasticsearch.xpack.prelert.action.UpdateJobSchedulerStatusAction.Request; -import static org.elasticsearch.xpack.prelert.job.JobTests.buildJobBuilder; +import static org.elasticsearch.xpack.prelert.action.UpdateSchedulerStatusAction.INSTANCE; +import static org.elasticsearch.xpack.prelert.action.UpdateSchedulerStatusAction.Request; import static org.hamcrest.Matchers.equalTo; import static org.mockito.Matchers.any; import static org.mockito.Matchers.anyString; @@ -85,9 +83,9 @@ public class ScheduledJobRunnerTests extends ESTestCase { doAnswer(invocation -> { @SuppressWarnings("unchecked") ActionListener actionListener = (ActionListener) invocation.getArguments()[2]; - actionListener.onResponse(new UpdateJobSchedulerStatusAction.Response()); + actionListener.onResponse(new UpdateSchedulerStatusAction.Response()); return null; - }).when(client).execute(same(UpdateJobSchedulerStatusAction.INSTANCE), any(), any()); + }).when(client).execute(same(UpdateSchedulerStatusAction.INSTANCE), any(), any()); JobProvider jobProvider = mock(JobProvider.class); when(jobProvider.dataCounts(anyString())).thenReturn(new DataCounts("foo")); @@ -112,10 +110,13 @@ public class ScheduledJobRunnerTests extends ESTestCase { } public void testStart_GivenNewlyCreatedJobLoopBack() throws Exception { - Job.Builder builder = createScheduledJob(); + Job.Builder jobBuilder = createScheduledJob(); + SchedulerConfig schedulerConfig = createSchedulerConfig("scheduler1", "foo").build(); DataCounts dataCounts = new DataCounts("foo", 1, 0, 0, 0, 0, 0, 0, new Date(0), new Date(0)); - Job job = builder.build(); - PrelertMetadata prelertMetadata = new PrelertMetadata.Builder().putJob(job, false) + Job job = jobBuilder.build(); + PrelertMetadata prelertMetadata = new PrelertMetadata.Builder() + .putJob(job, false) + .putScheduler(schedulerConfig) .updateStatus("foo", JobStatus.OPENED, null) .build(); when(clusterService.state()).thenReturn(ClusterState.builder(new ClusterName("_name")) @@ -123,29 +124,32 @@ public class ScheduledJobRunnerTests extends ESTestCase { .build()); DataExtractor dataExtractor = mock(DataExtractor.class); - when(dataExtractorFactory.newExtractor(job)).thenReturn(dataExtractor); + when(dataExtractorFactory.newExtractor(schedulerConfig, job)).thenReturn(dataExtractor); when(dataExtractor.hasNext()).thenReturn(true).thenReturn(false); InputStream in = new ByteArrayInputStream("".getBytes(Charset.forName("utf-8"))); when(dataExtractor.next()).thenReturn(Optional.of(in)); when(jobDataFuture.get()).thenReturn(new JobDataAction.Response(dataCounts)); Consumer handler = mockConsumer(); - StartJobSchedulerAction.SchedulerTask task = mock(StartJobSchedulerAction.SchedulerTask.class); - scheduledJobRunner.run("foo", 0L, 60000L, task, handler); + StartSchedulerAction.SchedulerTask task = mock(StartSchedulerAction.SchedulerTask.class); + scheduledJobRunner.run("scheduler1", 0L, 60000L, task, handler); verify(dataExtractor).newSearch(eq(0L), eq(60000L), any()); verify(threadPool, times(1)).executor(PrelertPlugin.SCHEDULER_THREAD_POOL_NAME); verify(threadPool, never()).schedule(any(), any(), any()); verify(client).execute(same(JobDataAction.INSTANCE), eq(new JobDataAction.Request("foo"))); verify(client).execute(same(FlushJobAction.INSTANCE), any()); - verify(client).execute(same(INSTANCE), eq(new Request("foo", SchedulerStatus.STARTED)), any()); - verify(client).execute(same(INSTANCE), eq(new Request("foo", SchedulerStatus.STOPPED)), any()); + verify(client).execute(same(INSTANCE), eq(new Request("scheduler1", SchedulerStatus.STARTED)), any()); + verify(client).execute(same(INSTANCE), eq(new Request("scheduler1", SchedulerStatus.STOPPED)), any()); } public void testStart_GivenNewlyCreatedJobLoopBackAndRealtime() throws Exception { - Job.Builder builder = createScheduledJob(); + Job.Builder jobBuilder = createScheduledJob(); + SchedulerConfig schedulerConfig = createSchedulerConfig("scheduler1", "foo").build(); DataCounts dataCounts = new DataCounts("foo", 1, 0, 0, 0, 0, 0, 0, new Date(0), new Date(0)); - Job job = builder.build(); - PrelertMetadata prelertMetadata = new PrelertMetadata.Builder().putJob(job, false) + Job job = jobBuilder.build(); + PrelertMetadata prelertMetadata = new PrelertMetadata.Builder() + .putJob(job, false) + .putScheduler(schedulerConfig) .updateStatus("foo", JobStatus.OPENED, null) .build(); when(clusterService.state()).thenReturn(ClusterState.builder(new ClusterName("_name")) @@ -153,21 +157,21 @@ public class ScheduledJobRunnerTests extends ESTestCase { .build()); DataExtractor dataExtractor = mock(DataExtractor.class); - when(dataExtractorFactory.newExtractor(job)).thenReturn(dataExtractor); + when(dataExtractorFactory.newExtractor(schedulerConfig, job)).thenReturn(dataExtractor); when(dataExtractor.hasNext()).thenReturn(true).thenReturn(false); InputStream in = new ByteArrayInputStream("".getBytes(Charset.forName("utf-8"))); when(dataExtractor.next()).thenReturn(Optional.of(in)); when(jobDataFuture.get()).thenReturn(new JobDataAction.Response(dataCounts)); Consumer handler = mockConsumer(); boolean cancelled = randomBoolean(); - StartJobSchedulerAction.SchedulerTask task = new StartJobSchedulerAction.SchedulerTask(1, "type", "action", null, "foo"); - scheduledJobRunner.run("foo", 0L, null, task, handler); + StartSchedulerAction.SchedulerTask task = new StartSchedulerAction.SchedulerTask(1, "type", "action", null, "scheduler1"); + scheduledJobRunner.run("scheduler1", 0L, null, task, handler); verify(dataExtractor).newSearch(eq(0L), eq(60000L), any()); verify(threadPool, times(1)).executor(PrelertPlugin.SCHEDULER_THREAD_POOL_NAME); if (cancelled) { task.stop(); - verify(client).execute(same(INSTANCE), eq(new Request("foo", SchedulerStatus.STOPPED)), any()); + verify(client).execute(same(INSTANCE), eq(new Request("scheduler1", SchedulerStatus.STOPPED)), any()); } else { verify(client).execute(same(JobDataAction.INSTANCE), eq(new JobDataAction.Request("foo"))); verify(client).execute(same(FlushJobAction.INSTANCE), any()); @@ -175,16 +179,20 @@ public class ScheduledJobRunnerTests extends ESTestCase { } } + public static SchedulerConfig.Builder createSchedulerConfig(String schedulerId, String jobId) { + SchedulerConfig.Builder schedulerConfig = new SchedulerConfig.Builder(schedulerId, jobId); + schedulerConfig.setIndexes(Arrays.asList("myIndex")); + schedulerConfig.setTypes(Arrays.asList("myType")); + return schedulerConfig; + } + public static Job.Builder createScheduledJob() { AnalysisConfig.Builder acBuilder = new AnalysisConfig.Builder(Arrays.asList(new Detector.Builder("metric", "field").build())); acBuilder.setBucketSpan(3600L); acBuilder.setDetectors(Arrays.asList(new Detector.Builder("metric", "field").build())); - SchedulerConfig.Builder schedulerConfig = new SchedulerConfig.Builder(Arrays.asList("myIndex"), Arrays.asList("myType")); - Job.Builder builder = new Job.Builder("foo"); builder.setAnalysisConfig(acBuilder); - builder.setSchedulerConfig(schedulerConfig); DataDescription.Builder dataDescription = new DataDescription.Builder(); dataDescription.setFormat(DataDescription.DataFormat.ELASTICSEARCH); builder.setDataDescription(dataDescription); @@ -192,22 +200,25 @@ public class ScheduledJobRunnerTests extends ESTestCase { } public void testValidate() { - Job job1 = buildJobBuilder("foo").build(); - PrelertMetadata prelertMetadata1 = new PrelertMetadata.Builder().putJob(job1, false).build(); - Exception e = expectThrows(IllegalArgumentException.class, () -> ScheduledJobRunner.validate("foo", prelertMetadata1)); - assertThat(e.getMessage(), equalTo("job [foo] is not a scheduled job")); + Job job1 = createScheduledJob().build(); + PrelertMetadata prelertMetadata1 = new PrelertMetadata.Builder() + .putJob(job1, false) + .build(); + Exception e = expectThrows(ResourceNotFoundException.class, () -> ScheduledJobRunner.validate("some-scheduler", prelertMetadata1)); + assertThat(e.getMessage(), equalTo("No scheduler with id [some-scheduler] exists")); - Job job2 = createScheduledJob().build(); - PrelertMetadata prelertMetadata2 = new PrelertMetadata.Builder().putJob(job2, false).build(); - e = expectThrows(ElasticsearchStatusException.class, () -> ScheduledJobRunner.validate("foo", prelertMetadata2)); + SchedulerConfig schedulerConfig1 = createSchedulerConfig("foo-scheduler", "foo").build(); + PrelertMetadata prelertMetadata2 = new PrelertMetadata.Builder(prelertMetadata1) + .putScheduler(schedulerConfig1) + .build(); + e = expectThrows(ElasticsearchStatusException.class, () -> ScheduledJobRunner.validate("foo-scheduler", prelertMetadata2)); assertThat(e.getMessage(), equalTo("cannot start scheduler, expected job status [OPENED], but got [CLOSED]")); - Job job3 = createScheduledJob().build(); - PrelertMetadata prelertMetadata3 = new PrelertMetadata.Builder().putJob(job3, false) + PrelertMetadata prelertMetadata3 = new PrelertMetadata.Builder(prelertMetadata2) .updateStatus("foo", JobStatus.OPENED, null) - .updateSchedulerStatus("foo", SchedulerStatus.STARTED) + .updateSchedulerStatus("foo-scheduler", SchedulerStatus.STARTED) .build(); - e = expectThrows(ElasticsearchStatusException.class, () -> ScheduledJobRunner.validate("foo", prelertMetadata3)); + e = expectThrows(ElasticsearchStatusException.class, () -> ScheduledJobRunner.validate("foo-scheduler", prelertMetadata3)); assertThat(e.getMessage(), equalTo("scheduler already started, expected scheduler status [STOPPED], but got [STARTED]")); } diff --git a/elasticsearch/src/test/java/org/elasticsearch/xpack/prelert/job/scheduler/ScheduledJobValidatorTests.java b/elasticsearch/src/test/java/org/elasticsearch/xpack/prelert/job/scheduler/ScheduledJobValidatorTests.java new file mode 100644 index 00000000000..184993a6c4c --- /dev/null +++ b/elasticsearch/src/test/java/org/elasticsearch/xpack/prelert/job/scheduler/ScheduledJobValidatorTests.java @@ -0,0 +1,180 @@ +/* + * 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.prelert.job.scheduler; + +import org.elasticsearch.common.xcontent.XContentFactory; +import org.elasticsearch.common.xcontent.XContentParser; +import org.elasticsearch.test.ESTestCase; +import org.elasticsearch.xpack.prelert.job.AnalysisConfig; +import org.elasticsearch.xpack.prelert.job.DataDescription; +import org.elasticsearch.xpack.prelert.job.Detector; +import org.elasticsearch.xpack.prelert.job.Job; +import org.elasticsearch.xpack.prelert.job.SchedulerConfig; +import org.elasticsearch.xpack.prelert.job.messages.Messages; + +import java.io.IOException; +import java.util.Arrays; +import java.util.Collections; +import java.util.Date; + +public class ScheduledJobValidatorTests extends ESTestCase { + + public void testValidate_GivenNonZeroLatency() { + String errorMessage = Messages.getMessage(Messages.SCHEDULER_DOES_NOT_SUPPORT_JOB_WITH_LATENCY); + Job.Builder builder = buildJobBuilder("foo"); + AnalysisConfig.Builder ac = createAnalysisConfig(); + ac.setBucketSpan(1800L); + ac.setLatency(3600L); + builder.setAnalysisConfig(ac); + Job job = builder.build(); + SchedulerConfig schedulerConfig = createValidSchedulerConfig().build(); + + IllegalArgumentException e = ESTestCase.expectThrows(IllegalArgumentException.class, + () -> ScheduledJobValidator.validate(schedulerConfig, job)); + + assertEquals(errorMessage, e.getMessage()); + } + + public void testVerify_GivenZeroLatency() { + Job.Builder builder = buildJobBuilder("foo"); + DataDescription.Builder dataDescription = new DataDescription.Builder(); + dataDescription.setFormat(DataDescription.DataFormat.ELASTICSEARCH); + builder.setDataDescription(dataDescription); + AnalysisConfig.Builder ac = createAnalysisConfig(); + ac.setBucketSpan(1800L); + ac.setLatency(0L); + builder.setAnalysisConfig(ac); + Job job = builder.build(); + SchedulerConfig schedulerConfig = createValidSchedulerConfig().build(); + + ScheduledJobValidator.validate(schedulerConfig, job); + } + + public void testVerify_GivenNoLatency() { + Job.Builder builder = buildJobBuilder("foo"); + DataDescription.Builder dataDescription = new DataDescription.Builder(); + dataDescription.setFormat(DataDescription.DataFormat.ELASTICSEARCH); + builder.setDataDescription(dataDescription); + AnalysisConfig.Builder ac = createAnalysisConfig(); + ac.setBatchSpan(1800L); + ac.setBucketSpan(100L); + builder.setAnalysisConfig(ac); + Job job = builder.build(); + SchedulerConfig schedulerConfig = createValidSchedulerConfig().build(); + + ScheduledJobValidator.validate(schedulerConfig, job); + } + + public void testVerify_GivenAggsAndCorrectSummaryCountField() throws IOException { + Job.Builder builder = buildJobBuilder("foo"); + DataDescription.Builder dataDescription = new DataDescription.Builder(); + dataDescription.setFormat(DataDescription.DataFormat.ELASTICSEARCH); + builder.setDataDescription(dataDescription); + AnalysisConfig.Builder ac = createAnalysisConfig(); + ac.setBucketSpan(1800L); + ac.setSummaryCountFieldName("doc_count"); + builder.setAnalysisConfig(ac); + Job job = builder.build(); + SchedulerConfig schedulerConfig = createValidSchedulerConfigWithAggs().build(); + + ScheduledJobValidator.validate(schedulerConfig, job); + } + + public void testVerify_GivenAggsAndNoSummaryCountField() throws IOException { + String errorMessage = Messages.getMessage(Messages.SCHEDULER_AGGREGATIONS_REQUIRES_JOB_WITH_SUMMARY_COUNT_FIELD, + SchedulerConfig.DOC_COUNT); + Job.Builder builder = buildJobBuilder("foo"); + DataDescription.Builder dataDescription = new DataDescription.Builder(); + dataDescription.setFormat(DataDescription.DataFormat.ELASTICSEARCH); + builder.setDataDescription(dataDescription); + AnalysisConfig.Builder ac = createAnalysisConfig(); + ac.setBucketSpan(1800L); + builder.setAnalysisConfig(ac); + Job job = builder.build(); + SchedulerConfig schedulerConfig = createValidSchedulerConfigWithAggs().build(); + + IllegalArgumentException e = ESTestCase.expectThrows(IllegalArgumentException.class, + () -> ScheduledJobValidator.validate(schedulerConfig, job)); + + assertEquals(errorMessage, e.getMessage()); + } + + public void testVerify_GivenAggsAndWrongSummaryCountField() throws IOException { + String errorMessage = Messages.getMessage( + Messages.SCHEDULER_AGGREGATIONS_REQUIRES_JOB_WITH_SUMMARY_COUNT_FIELD, SchedulerConfig.DOC_COUNT); + Job.Builder builder = buildJobBuilder("foo"); + DataDescription.Builder dataDescription = new DataDescription.Builder(); + dataDescription.setFormat(DataDescription.DataFormat.ELASTICSEARCH); + builder.setDataDescription(dataDescription); + AnalysisConfig.Builder ac = createAnalysisConfig(); + ac.setBucketSpan(1800L); + ac.setSummaryCountFieldName("wrong"); + builder.setAnalysisConfig(ac); + Job job = builder.build(); + SchedulerConfig schedulerConfig = createValidSchedulerConfigWithAggs().build(); + + IllegalArgumentException e = ESTestCase.expectThrows(IllegalArgumentException.class, + () -> ScheduledJobValidator.validate(schedulerConfig, job)); + + assertEquals(errorMessage, e.getMessage()); + } + + public static Job.Builder buildJobBuilder(String id) { + Job.Builder builder = new Job.Builder(id); + builder.setCreateTime(new Date()); + AnalysisConfig.Builder ac = createAnalysisConfig(); + DataDescription.Builder dc = new DataDescription.Builder(); + builder.setAnalysisConfig(ac); + builder.setDataDescription(dc); + return builder; + } + + public static AnalysisConfig.Builder createAnalysisConfig() { + Detector.Builder d1 = new Detector.Builder("info_content", "domain"); + d1.setOverFieldName("client"); + Detector.Builder d2 = new Detector.Builder("min", "field"); + AnalysisConfig.Builder ac = new AnalysisConfig.Builder(Arrays.asList(d1.build(), d2.build())); + return ac; + } + + private static SchedulerConfig.Builder createValidSchedulerConfigWithAggs() throws IOException { + SchedulerConfig.Builder schedulerConfig = createValidSchedulerConfig(); + String aggStr = + "{" + + "\"buckets\" : {" + + "\"histogram\" : {" + + "\"field\" : \"time\"," + + "\"interval\" : 3600000" + + "}," + + "\"aggs\" : {" + + "\"byField\" : {" + + "\"terms\" : {" + + "\"field\" : \"airline\"," + + "\"size\" : 0" + + "}," + + "\"aggs\" : {" + + "\"stats\" : {" + + "\"stats\" : {" + + "\"field\" : \"responsetime\"" + + "}" + + "}" + + "}" + + "}" + + "}" + + "} " + + "}"; + XContentParser parser = XContentFactory.xContent(aggStr).createParser(aggStr); + schedulerConfig.setAggs(parser.map()); + return schedulerConfig; + } + + private static SchedulerConfig.Builder createValidSchedulerConfig() { + SchedulerConfig.Builder builder = new SchedulerConfig.Builder("my-scheduler", "my-job"); + builder.setIndexes(Collections.singletonList("myIndex")); + builder.setTypes(Collections.singletonList("myType")); + return builder; + } +} \ No newline at end of file diff --git a/elasticsearch/src/test/java/org/elasticsearch/xpack/prelert/rest/schedulers/RestStartJobSchedulerActionTests.java b/elasticsearch/src/test/java/org/elasticsearch/xpack/prelert/rest/schedulers/RestStartJobSchedulerActionTests.java index 87faac6829a..3c35a288312 100644 --- a/elasticsearch/src/test/java/org/elasticsearch/xpack/prelert/rest/schedulers/RestStartJobSchedulerActionTests.java +++ b/elasticsearch/src/test/java/org/elasticsearch/xpack/prelert/rest/schedulers/RestStartJobSchedulerActionTests.java @@ -18,6 +18,7 @@ import org.elasticsearch.test.ESTestCase; import org.elasticsearch.test.rest.FakeRestRequest; import org.elasticsearch.xpack.prelert.job.Job; import org.elasticsearch.xpack.prelert.job.JobStatus; +import org.elasticsearch.xpack.prelert.job.SchedulerConfig; import org.elasticsearch.xpack.prelert.job.metadata.PrelertMetadata; import org.elasticsearch.xpack.prelert.job.scheduler.ScheduledJobRunnerTests; @@ -32,18 +33,20 @@ public class RestStartJobSchedulerActionTests extends ESTestCase { public void testPrepareRequest() throws Exception { ClusterService clusterService = mock(ClusterService.class); Job.Builder job = ScheduledJobRunnerTests.createScheduledJob(); + SchedulerConfig schedulerConfig = ScheduledJobRunnerTests.createSchedulerConfig("foo-scheduler", "foo").build(); PrelertMetadata prelertMetadata = new PrelertMetadata.Builder() .putJob(job.build(), false) + .putScheduler(schedulerConfig) .updateStatus("foo", JobStatus.OPENED, null) .build(); when(clusterService.state()).thenReturn(ClusterState.builder(new ClusterName("_name")) .metaData(MetaData.builder().putCustom(PrelertMetadata.TYPE, prelertMetadata)) .build()); - RestStartJobSchedulerAction action = new RestStartJobSchedulerAction(Settings.EMPTY, mock(RestController.class), clusterService); + RestStartSchedulerAction action = new RestStartSchedulerAction(Settings.EMPTY, mock(RestController.class), clusterService); Map params = new HashMap<>(); params.put("start", "not-a-date"); - params.put("job_id", "foo"); + params.put("scheduler_id", "foo-scheduler"); RestRequest restRequest1 = new FakeRestRequest.Builder().withParams(params).build(); ElasticsearchParseException e = expectThrows(ElasticsearchParseException.class, () -> action.prepareRequest(restRequest1, mock(NodeClient.class))); @@ -52,7 +55,7 @@ public class RestStartJobSchedulerActionTests extends ESTestCase { params = new HashMap<>(); params.put("end", "not-a-date"); - params.put("job_id", "foo"); + params.put("scheduler_id", "foo-scheduler"); RestRequest restRequest2 = new FakeRestRequest.Builder().withParams(params).build(); e = expectThrows(ElasticsearchParseException.class, () -> action.prepareRequest(restRequest2, mock(NodeClient.class))); assertEquals("Query param 'end' with value 'not-a-date' cannot be parsed as a date or converted to a number (epoch).", @@ -60,11 +63,11 @@ public class RestStartJobSchedulerActionTests extends ESTestCase { } public void testParseDateOrThrow() { - assertEquals(0L, RestStartJobSchedulerAction.parseDateOrThrow("0", "start")); - assertEquals(0L, RestStartJobSchedulerAction.parseDateOrThrow("1970-01-01T00:00:00Z", "start")); + assertEquals(0L, RestStartSchedulerAction.parseDateOrThrow("0", "start")); + assertEquals(0L, RestStartSchedulerAction.parseDateOrThrow("1970-01-01T00:00:00Z", "start")); Exception e = expectThrows(ElasticsearchParseException.class, - () -> RestStartJobSchedulerAction.parseDateOrThrow("not-a-date", "start")); + () -> RestStartSchedulerAction.parseDateOrThrow("not-a-date", "start")); assertEquals("Query param 'start' with value 'not-a-date' cannot be parsed as a date or converted to a number (epoch).", e.getMessage()); } diff --git a/elasticsearch/src/test/resources/rest-api-spec/api/xpack.prelert.delete_scheduler.json b/elasticsearch/src/test/resources/rest-api-spec/api/xpack.prelert.delete_scheduler.json new file mode 100644 index 00000000000..755431458e3 --- /dev/null +++ b/elasticsearch/src/test/resources/rest-api-spec/api/xpack.prelert.delete_scheduler.json @@ -0,0 +1,17 @@ +{ + "xpack.prelert.delete_scheduler": { + "methods": [ "DELETE" ], + "url": { + "path": "/_xpack/prelert/schedulers/{scheduler_id}", + "paths": [ "/_xpack/prelert/schedulers/{scheduler_id}" ], + "parts": { + "scheduler_id": { + "type": "string", + "required": true, + "description": "The ID of the scheduler to delete" + } + } + }, + "body": null + } +} diff --git a/elasticsearch/src/test/resources/rest-api-spec/api/xpack.prelert.put_scheduler.json b/elasticsearch/src/test/resources/rest-api-spec/api/xpack.prelert.put_scheduler.json new file mode 100644 index 00000000000..4ad0e62a767 --- /dev/null +++ b/elasticsearch/src/test/resources/rest-api-spec/api/xpack.prelert.put_scheduler.json @@ -0,0 +1,20 @@ +{ + "xpack.prelert.put_scheduler": { + "methods": [ "PUT" ], + "url": { + "path": "/_xpack/prelert/schedulers/{scheduler_id}", + "paths": [ "/_xpack/prelert/schedulers/{scheduler_id}" ], + "parts": { + "scheduler_id": { + "type": "string", + "required": true, + "description": "The ID of the scheduler to create" + } + } + }, + "body": { + "description" : "The scheduler config", + "required" : true + } + } +} diff --git a/elasticsearch/src/test/resources/rest-api-spec/api/xpack.prelert.start_job_scheduler.json b/elasticsearch/src/test/resources/rest-api-spec/api/xpack.prelert.start_scheduler.json similarity index 70% rename from elasticsearch/src/test/resources/rest-api-spec/api/xpack.prelert.start_job_scheduler.json rename to elasticsearch/src/test/resources/rest-api-spec/api/xpack.prelert.start_scheduler.json index 4c68d376629..bc88352cddb 100644 --- a/elasticsearch/src/test/resources/rest-api-spec/api/xpack.prelert.start_job_scheduler.json +++ b/elasticsearch/src/test/resources/rest-api-spec/api/xpack.prelert.start_scheduler.json @@ -1,14 +1,14 @@ { - "xpack.prelert.start_job_scheduler": { + "xpack.prelert.start_scheduler": { "methods": [ "POST" ], "url": { - "path": "/_xpack/prelert/schedulers/{job_id}/_start", - "paths": [ "/_xpack/prelert/schedulers/{job_id}/_start" ], + "path": "/_xpack/prelert/schedulers/{scheduler_id}/_start", + "paths": [ "/_xpack/prelert/schedulers/{scheduler_id}/_start" ], "parts": { - "job_id": { + "scheduler_id": { "type": "string", "required": true, - "description": "The ID of the job whose scheduler to start" + "description": "The ID of the scheduler to start" } }, "params": { diff --git a/elasticsearch/src/test/resources/rest-api-spec/api/xpack.prelert.stop_job_scheduler.json b/elasticsearch/src/test/resources/rest-api-spec/api/xpack.prelert.stop_job_scheduler.json deleted file mode 100644 index 091c60123a7..00000000000 --- a/elasticsearch/src/test/resources/rest-api-spec/api/xpack.prelert.stop_job_scheduler.json +++ /dev/null @@ -1,17 +0,0 @@ -{ - "xpack.prelert.stop_job_scheduler": { - "methods": [ "POST" ], - "url": { - "path": "/_xpack/prelert/schedulers/{job_id}/_stop", - "paths": [ "/_xpack/prelert/schedulers/{job_id}/_stop" ], - "parts": { - "job_id": { - "type": "string", - "required": true, - "description": "The ID of the job whose scheduler to stop" - } - } - }, - "body": null - } -} diff --git a/elasticsearch/src/test/resources/rest-api-spec/api/xpack.prelert.stop_scheduler.json b/elasticsearch/src/test/resources/rest-api-spec/api/xpack.prelert.stop_scheduler.json new file mode 100644 index 00000000000..9def2a7c055 --- /dev/null +++ b/elasticsearch/src/test/resources/rest-api-spec/api/xpack.prelert.stop_scheduler.json @@ -0,0 +1,17 @@ +{ + "xpack.prelert.stop_scheduler": { + "methods": [ "POST" ], + "url": { + "path": "/_xpack/prelert/schedulers/{scheduler_id}/_stop", + "paths": [ "/_xpack/prelert/schedulers/{scheduler_id}/_stop" ], + "parts": { + "scheduler_id": { + "type": "string", + "required": true, + "description": "The ID of the scheduler to stop" + } + } + }, + "body": null + } +} diff --git a/elasticsearch/src/test/resources/rest-api-spec/test/jobs_crud.yaml b/elasticsearch/src/test/resources/rest-api-spec/test/jobs_crud.yaml index 10d57d63339..e95ecad14ff 100644 --- a/elasticsearch/src/test/resources/rest-api-spec/test/jobs_crud.yaml +++ b/elasticsearch/src/test/resources/rest-api-spec/test/jobs_crud.yaml @@ -101,6 +101,42 @@ } } +--- +"Test delete job that is referred by a scheduler": + - do: + xpack.prelert.put_job: + body: > + { + "job_id":"scheduler-job", + "description":"Analysis of response time by airline", + "analysis_config" : { + "bucket_span":3600, + "detectors" :[{"function":"metric","field_name":"responsetime","by_field_name":"airline"}] + }, + "data_description" : { + "format":"ELASTICSEARCH", + "time_field":"time", + "time_format":"yyyy-MM-dd HH:mm:ssX" + } + } + - match: { job_id: "scheduler-job" } + + - do: + xpack.prelert.put_scheduler: + scheduler_id: test-scheduler-1 + body: > + { + "job_id":"scheduler-job", + "indexes":["index-foo"], + "types":["type-bar"] + } + - match: { scheduler_id: "test-scheduler-1" } + + - do: + catch: /Cannot delete job \[scheduler-job\] while scheduler \[test-scheduler-1\] refers to it/ + xpack.prelert.delete_job: + job_id: scheduler-job + --- "Test jobid + from and/or size": - do: diff --git a/elasticsearch/src/test/resources/rest-api-spec/test/jobs_get_stats.yaml b/elasticsearch/src/test/resources/rest-api-spec/test/jobs_get_stats.yaml index 95d9f3c9260..8b0c3a75065 100644 --- a/elasticsearch/src/test/resources/rest-api-spec/test/jobs_get_stats.yaml +++ b/elasticsearch/src/test/resources/rest-api-spec/test/jobs_get_stats.yaml @@ -34,16 +34,22 @@ setup: "format" : "ELASTICSEARCH", "time_field":"time", "time_format":"yyyy-MM-dd'T'HH:mm:ssX" - }, - "scheduler_config": { - "indexes":["farequote"], - "types":["response"] } } - do: xpack.prelert.open_job: job_id: scheduled-job + - do: + xpack.prelert.put_scheduler: + scheduler_id: scheduler-1 + body: > + { + "job_id":"scheduled-job", + "indexes":["farequote"], + "types":["response"] + } + - do: index: index: prelertresults-job-stats-test diff --git a/elasticsearch/src/test/resources/rest-api-spec/test/schedulers_crud.yaml b/elasticsearch/src/test/resources/rest-api-spec/test/schedulers_crud.yaml new file mode 100644 index 00000000000..f0e350bef2b --- /dev/null +++ b/elasticsearch/src/test/resources/rest-api-spec/test/schedulers_crud.yaml @@ -0,0 +1,130 @@ +setup: + - do: + xpack.prelert.put_job: + body: > + { + "job_id":"job-1", + "analysis_config" : { + "bucket_span":3600, + "detectors" :[{"function":"count"}] + }, + "data_description" : { + "format":"ELASTICSEARCH", + "time_field":"time", + "time_format":"epoch" + } + } + + - do: + xpack.prelert.put_job: + body: > + { + "job_id":"job-2", + "analysis_config" : { + "bucket_span":3600, + "detectors" :[{"function":"count"}] + }, + "data_description" : { + "format":"ELASTICSEARCH", + "time_field":"time", + "time_format":"epoch" + } + } +--- +"Test put scheduler referring to missing job_id": + - do: + catch: /resource_not_found_exception/ + xpack.prelert.put_scheduler: + scheduler_id: test-scheduler-1 + body: > + { + "job_id":"a-missing-job", + "indexes":["index-foo"], + "types":["type-bar"] + } + +--- +"Test put scheduler referring to existing job_id": + - do: + xpack.prelert.put_scheduler: + scheduler_id: test-scheduler-1 + body: > + { + "job_id":"job-1", + "indexes":["index-foo"], + "types":["type-bar"] + } + - match: { scheduler_id: "test-scheduler-1" } + +--- +"Test put scheduler whose id is already taken": + - do: + xpack.prelert.put_scheduler: + scheduler_id: test-scheduler-1 + body: > + { + "job_id":"job-1", + "indexes":["index-foo"], + "types":["type-bar"] + } + - match: { scheduler_id: "test-scheduler-1" } + + - do: + catch: /resource_already_exists_exception/ + xpack.prelert.put_scheduler: + scheduler_id: test-scheduler-1 + body: > + { + "job_id":"job-2", + "indexes":["index-foo"], + "types":["type-bar"] + } + +--- +"Test put scheduler with job_id that is already used by another scheduler": + - do: + xpack.prelert.put_scheduler: + scheduler_id: test-scheduler-1 + body: > + { + "job_id":"job-1", + "indexes":["index-foo"], + "types":["type-bar"] + } + - match: { scheduler_id: "test-scheduler-1" } + + - do: + catch: /A scheduler \[test-scheduler-1\] already exists for job \[job-1\]/ + xpack.prelert.put_scheduler: + scheduler_id: test-scheduler-2 + body: > + { + "job_id":"job-1", + "indexes":["index-foo"], + "types":["type-bar"] + } + +--- +"Test delete scheduler with missing id": + - do: + catch: /resource_not_found_exception/ + xpack.prelert.delete_scheduler: + scheduler_id: a-missing-scheduler + +--- +"Test delete scheduler": + - do: + xpack.prelert.put_scheduler: + scheduler_id: test-scheduler-1 + body: > + { + "job_id":"job-1", + "indexes":["index-foo"], + "types":["type-bar"] + } + - match: { scheduler_id: "test-scheduler-1" } + + - do: + xpack.prelert.delete_scheduler: + scheduler_id: test-scheduler-1 + - match: { acknowledged: true } diff --git a/elasticsearch/src/test/resources/rest-api-spec/test/start_stop_scheduler.yaml b/elasticsearch/src/test/resources/rest-api-spec/test/start_stop_scheduler.yaml index 7f152789d45..45668e174f7 100644 --- a/elasticsearch/src/test/resources/rest-api-spec/test/start_stop_scheduler.yaml +++ b/elasticsearch/src/test/resources/rest-api-spec/test/start_stop_scheduler.yaml @@ -1,20 +1,4 @@ setup: - - do: - xpack.prelert.put_job: - body: > - { - "job_id":"non-scheduled-job", - "description":"Analysis of response time by airline", - "analysis_config" : { - "bucket_span":3600, - "detectors" :[{"function":"metric","field_name":"responsetime","by_field_name":"airline"}] - }, - "data_description" : { - "format":"JSON", - "time_field":"time", - "time_format":"epoch" - } - } - do: xpack.prelert.put_job: body: > @@ -29,49 +13,50 @@ setup: "format":"ELASTICSEARCH", "time_field":"time", "time_format":"epoch" - }, - "scheduler_config" : { - "indexes" : "airline-data", - "types" : "response" } } - + - do: + xpack.prelert.put_scheduler: + scheduler_id: scheduler-1 + body: > + { + "job_id":"scheduled-job", + "indexes":"airline-data", + "types":"response" + } --- "Test start and stop scheduler happy path": - do: xpack.prelert.open_job: job_id: "scheduled-job" - do: - xpack.prelert.start_job_scheduler: - "job_id": "scheduled-job" + xpack.prelert.start_scheduler: + "scheduler_id": "scheduler-1" "start": 0 - do: - xpack.prelert.stop_job_scheduler: - "job_id": "scheduled-job" + xpack.prelert.stop_scheduler: + "scheduler_id": "scheduler-1" --- -"Test start non existing job": +"Test start non existing scheduler": - do: catch: missing - xpack.prelert.start_job_scheduler: - "job_id": "non-existing-job" - "start": 0 - ---- -"Test start non scheduled job": - - do: - catch: request - xpack.prelert.start_job_scheduler: - "job_id": "non-scheduled-job" + xpack.prelert.start_scheduler: + "scheduler_id": "non-existing-scheduler" "start": 0 --- "Test start scheduled job, but not open": - do: catch: conflict - xpack.prelert.start_job_scheduler: - "job_id": "scheduled-job" + xpack.prelert.start_scheduler: + "scheduler_id": "scheduler-1" + "start": 0 + - do: + catch: /cannot start scheduler, expected job status \[OPENED\], but got \[CLOSED\]/ + xpack.prelert.start_scheduler: + "scheduler_id": "scheduler-1" "start": 0 --- @@ -80,11 +65,17 @@ setup: xpack.prelert.open_job: job_id: "scheduled-job" - do: - xpack.prelert.start_job_scheduler: - "job_id": "scheduled-job" + xpack.prelert.start_scheduler: + "scheduler_id": "scheduler-1" "start": 0 - do: catch: conflict - xpack.prelert.start_job_scheduler: - "job_id": "scheduled-job" + xpack.prelert.start_scheduler: + "scheduler_id": "scheduler-1" + "start": 0 + + - do: + catch: /scheduler already started, expected scheduler status \[STOPPED\], but got \[STARTED\]/ + xpack.prelert.start_scheduler: + "scheduler_id": "scheduler-1" "start": 0