Changed job lifecycle to be task oriented.
The job open api starts a task and ties that AutodetectCommunicator. The job close api is a sugar api, that uses the list and cancel task api to close a AutodetectCommunicator instance. The flush job and post data api redirect to the node holding the job task and then delegate the flush or data to the AutodetectCommunicator instance. Also: * Added basic multi node cluster test. * Fixed cluster state diffs bugs, forgot to mark ml metadata diffs as named writeable. * Moved waiting for open job logic into OpenJobAction.TransportAction and moved the logic that was original there to a new action named InternalOpenJobAction. Original commit: elastic/x-pack-elasticsearch@194a058dd2
This commit is contained in:
parent
f20f56e2e1
commit
9665368755
|
@ -9,10 +9,10 @@ import org.elasticsearch.ElasticsearchException;
|
||||||
import org.elasticsearch.action.ActionRequest;
|
import org.elasticsearch.action.ActionRequest;
|
||||||
import org.elasticsearch.action.ActionResponse;
|
import org.elasticsearch.action.ActionResponse;
|
||||||
import org.elasticsearch.client.Client;
|
import org.elasticsearch.client.Client;
|
||||||
|
import org.elasticsearch.cluster.NamedDiff;
|
||||||
import org.elasticsearch.cluster.metadata.MetaData;
|
import org.elasticsearch.cluster.metadata.MetaData;
|
||||||
import org.elasticsearch.cluster.service.ClusterService;
|
import org.elasticsearch.cluster.service.ClusterService;
|
||||||
import org.elasticsearch.common.ParseField;
|
import org.elasticsearch.common.ParseField;
|
||||||
import org.elasticsearch.common.component.LifecycleListener;
|
|
||||||
import org.elasticsearch.common.io.stream.NamedWriteableRegistry;
|
import org.elasticsearch.common.io.stream.NamedWriteableRegistry;
|
||||||
import org.elasticsearch.common.settings.Setting;
|
import org.elasticsearch.common.settings.Setting;
|
||||||
import org.elasticsearch.common.settings.Setting.Property;
|
import org.elasticsearch.common.settings.Setting.Property;
|
||||||
|
@ -44,6 +44,7 @@ import org.elasticsearch.xpack.ml.action.GetRecordsAction;
|
||||||
import org.elasticsearch.xpack.ml.action.GetDatafeedsAction;
|
import org.elasticsearch.xpack.ml.action.GetDatafeedsAction;
|
||||||
import org.elasticsearch.xpack.ml.action.GetDatafeedsStatsAction;
|
import org.elasticsearch.xpack.ml.action.GetDatafeedsStatsAction;
|
||||||
import org.elasticsearch.xpack.ml.action.InternalStartDatafeedAction;
|
import org.elasticsearch.xpack.ml.action.InternalStartDatafeedAction;
|
||||||
|
import org.elasticsearch.xpack.ml.action.InternalOpenJobAction;
|
||||||
import org.elasticsearch.xpack.ml.action.OpenJobAction;
|
import org.elasticsearch.xpack.ml.action.OpenJobAction;
|
||||||
import org.elasticsearch.xpack.ml.action.PostDataAction;
|
import org.elasticsearch.xpack.ml.action.PostDataAction;
|
||||||
import org.elasticsearch.xpack.ml.action.PutJobAction;
|
import org.elasticsearch.xpack.ml.action.PutJobAction;
|
||||||
|
@ -58,11 +59,8 @@ import org.elasticsearch.xpack.ml.action.UpdateDatafeedStatusAction;
|
||||||
import org.elasticsearch.xpack.ml.action.ValidateDetectorAction;
|
import org.elasticsearch.xpack.ml.action.ValidateDetectorAction;
|
||||||
import org.elasticsearch.xpack.ml.action.ValidateTransformAction;
|
import org.elasticsearch.xpack.ml.action.ValidateTransformAction;
|
||||||
import org.elasticsearch.xpack.ml.action.ValidateTransformsAction;
|
import org.elasticsearch.xpack.ml.action.ValidateTransformsAction;
|
||||||
import org.elasticsearch.xpack.ml.job.data.DataProcessor;
|
|
||||||
import org.elasticsearch.xpack.ml.job.manager.AutodetectProcessManager;
|
import org.elasticsearch.xpack.ml.job.manager.AutodetectProcessManager;
|
||||||
import org.elasticsearch.xpack.ml.job.manager.JobManager;
|
import org.elasticsearch.xpack.ml.job.manager.JobManager;
|
||||||
import org.elasticsearch.xpack.ml.job.metadata.JobAllocator;
|
|
||||||
import org.elasticsearch.xpack.ml.job.metadata.JobLifeCycleService;
|
|
||||||
import org.elasticsearch.xpack.ml.job.metadata.MlInitializationService;
|
import org.elasticsearch.xpack.ml.job.metadata.MlInitializationService;
|
||||||
import org.elasticsearch.xpack.ml.job.metadata.MlMetadata;
|
import org.elasticsearch.xpack.ml.job.metadata.MlMetadata;
|
||||||
import org.elasticsearch.xpack.ml.job.persistence.JobDataCountsPersister;
|
import org.elasticsearch.xpack.ml.job.persistence.JobDataCountsPersister;
|
||||||
|
@ -153,7 +151,10 @@ public class MlPlugin extends Plugin implements ActionPlugin {
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
public List<NamedWriteableRegistry.Entry> getNamedWriteables() {
|
public List<NamedWriteableRegistry.Entry> getNamedWriteables() {
|
||||||
return Collections.singletonList(new NamedWriteableRegistry.Entry(MetaData.Custom.class, "ml", MlMetadata::new));
|
return Arrays.asList(
|
||||||
|
new NamedWriteableRegistry.Entry(MetaData.Custom.class, "ml", MlMetadata::new),
|
||||||
|
new NamedWriteableRegistry.Entry(NamedDiff.class, "ml", MlMetadata.MlMetadataDiff::new)
|
||||||
|
);
|
||||||
}
|
}
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
|
@ -188,7 +189,8 @@ public class MlPlugin extends Plugin implements ActionPlugin {
|
||||||
throw new ElasticsearchException("Failed to create native process factories", e);
|
throw new ElasticsearchException("Failed to create native process factories", e);
|
||||||
}
|
}
|
||||||
} else {
|
} else {
|
||||||
autodetectProcessFactory = (jobDetails, ignoreDowntime, executorService) -> new BlackHoleAutodetectProcess();
|
autodetectProcessFactory = (jobDetails, modelSnapshot, quantiles, lists, ignoreDowntime, executorService) ->
|
||||||
|
new BlackHoleAutodetectProcess();
|
||||||
// factor of 1.0 makes renormalization a no-op
|
// factor of 1.0 makes renormalization a no-op
|
||||||
normalizerProcessFactory = (jobId, quantilesState, bucketSpan, perPartitionNormalization,
|
normalizerProcessFactory = (jobId, quantilesState, bucketSpan, perPartitionNormalization,
|
||||||
executorService) -> new MultiplyingNormalizerProcess(settings, 1.0);
|
executorService) -> new MultiplyingNormalizerProcess(settings, 1.0);
|
||||||
|
@ -196,31 +198,15 @@ public class MlPlugin extends Plugin implements ActionPlugin {
|
||||||
NormalizerFactory normalizerFactory = new NormalizerFactory(normalizerProcessFactory,
|
NormalizerFactory normalizerFactory = new NormalizerFactory(normalizerProcessFactory,
|
||||||
threadPool.executor(MlPlugin.THREAD_POOL_NAME));
|
threadPool.executor(MlPlugin.THREAD_POOL_NAME));
|
||||||
AutodetectResultsParser autodetectResultsParser = new AutodetectResultsParser(settings);
|
AutodetectResultsParser autodetectResultsParser = new AutodetectResultsParser(settings);
|
||||||
DataProcessor dataProcessor = new AutodetectProcessManager(settings, client, threadPool, jobManager, jobProvider,
|
AutodetectProcessManager dataProcessor = new AutodetectProcessManager(settings, client, threadPool, jobManager, jobProvider,
|
||||||
jobResultsPersister, jobRenormalizedResultsPersister, jobDataCountsPersister, autodetectResultsParser,
|
jobResultsPersister, jobRenormalizedResultsPersister, jobDataCountsPersister, autodetectResultsParser,
|
||||||
autodetectProcessFactory, normalizerFactory);
|
autodetectProcessFactory, normalizerFactory);
|
||||||
DatafeedJobRunner datafeedJobRunner = new DatafeedJobRunner(threadPool, client, clusterService, jobProvider,
|
DatafeedJobRunner datafeedJobRunner = new DatafeedJobRunner(threadPool, client, clusterService, jobProvider,
|
||||||
System::currentTimeMillis);
|
System::currentTimeMillis);
|
||||||
|
|
||||||
JobLifeCycleService jobLifeCycleService =
|
|
||||||
new JobLifeCycleService(settings, client, clusterService, dataProcessor, threadPool.generic());
|
|
||||||
// we hop on the lifecycle service of ResourceWatcherService, because
|
|
||||||
// that one is stopped before discovery is.
|
|
||||||
// (when discovery is stopped it will send a leave request to elected master node, which will then be removed
|
|
||||||
// from the cluster state, which then triggers other events)
|
|
||||||
resourceWatcherService.addLifecycleListener(new LifecycleListener() {
|
|
||||||
|
|
||||||
@Override
|
|
||||||
public void beforeStop() {
|
|
||||||
jobLifeCycleService.stop();
|
|
||||||
}
|
|
||||||
});
|
|
||||||
|
|
||||||
return Arrays.asList(
|
return Arrays.asList(
|
||||||
jobProvider,
|
jobProvider,
|
||||||
jobManager,
|
jobManager,
|
||||||
new JobAllocator(settings, clusterService, threadPool),
|
|
||||||
jobLifeCycleService,
|
|
||||||
new JobDataDeleterFactory(client), //NORELEASE: this should use Delete-by-query
|
new JobDataDeleterFactory(client), //NORELEASE: this should use Delete-by-query
|
||||||
dataProcessor,
|
dataProcessor,
|
||||||
new MlInitializationService(settings, threadPool, clusterService, jobProvider),
|
new MlInitializationService(settings, threadPool, clusterService, jobProvider),
|
||||||
|
@ -271,6 +257,7 @@ public class MlPlugin extends Plugin implements ActionPlugin {
|
||||||
new ActionHandler<>(PutJobAction.INSTANCE, PutJobAction.TransportAction.class),
|
new ActionHandler<>(PutJobAction.INSTANCE, PutJobAction.TransportAction.class),
|
||||||
new ActionHandler<>(DeleteJobAction.INSTANCE, DeleteJobAction.TransportAction.class),
|
new ActionHandler<>(DeleteJobAction.INSTANCE, DeleteJobAction.TransportAction.class),
|
||||||
new ActionHandler<>(OpenJobAction.INSTANCE, OpenJobAction.TransportAction.class),
|
new ActionHandler<>(OpenJobAction.INSTANCE, OpenJobAction.TransportAction.class),
|
||||||
|
new ActionHandler<>(InternalOpenJobAction.INSTANCE, InternalOpenJobAction.TransportAction.class),
|
||||||
new ActionHandler<>(UpdateJobStatusAction.INSTANCE, UpdateJobStatusAction.TransportAction.class),
|
new ActionHandler<>(UpdateJobStatusAction.INSTANCE, UpdateJobStatusAction.TransportAction.class),
|
||||||
new ActionHandler<>(UpdateDatafeedStatusAction.INSTANCE, UpdateDatafeedStatusAction.TransportAction.class),
|
new ActionHandler<>(UpdateDatafeedStatusAction.INSTANCE, UpdateDatafeedStatusAction.TransportAction.class),
|
||||||
new ActionHandler<>(GetListAction.INSTANCE, GetListAction.TransportAction.class),
|
new ActionHandler<>(GetListAction.INSTANCE, GetListAction.TransportAction.class),
|
||||||
|
|
|
@ -5,19 +5,21 @@
|
||||||
*/
|
*/
|
||||||
package org.elasticsearch.xpack.ml.action;
|
package org.elasticsearch.xpack.ml.action;
|
||||||
|
|
||||||
|
import org.elasticsearch.ElasticsearchStatusException;
|
||||||
|
import org.elasticsearch.ResourceNotFoundException;
|
||||||
import org.elasticsearch.action.Action;
|
import org.elasticsearch.action.Action;
|
||||||
import org.elasticsearch.action.ActionListener;
|
import org.elasticsearch.action.ActionListener;
|
||||||
|
import org.elasticsearch.action.ActionRequest;
|
||||||
|
import org.elasticsearch.action.ActionRequestBuilder;
|
||||||
import org.elasticsearch.action.ActionRequestValidationException;
|
import org.elasticsearch.action.ActionRequestValidationException;
|
||||||
|
import org.elasticsearch.action.ActionResponse;
|
||||||
|
import org.elasticsearch.action.admin.cluster.node.tasks.cancel.CancelTasksRequest;
|
||||||
|
import org.elasticsearch.action.admin.cluster.node.tasks.cancel.TransportCancelTasksAction;
|
||||||
|
import org.elasticsearch.action.admin.cluster.node.tasks.list.ListTasksRequest;
|
||||||
|
import org.elasticsearch.action.admin.cluster.node.tasks.list.TransportListTasksAction;
|
||||||
import org.elasticsearch.action.support.ActionFilters;
|
import org.elasticsearch.action.support.ActionFilters;
|
||||||
import org.elasticsearch.action.support.master.AcknowledgedRequest;
|
import org.elasticsearch.action.support.HandledTransportAction;
|
||||||
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.client.ElasticsearchClient;
|
||||||
import org.elasticsearch.cluster.ClusterState;
|
|
||||||
import org.elasticsearch.cluster.ClusterStateObserver;
|
|
||||||
import org.elasticsearch.cluster.block.ClusterBlockException;
|
|
||||||
import org.elasticsearch.cluster.block.ClusterBlockLevel;
|
|
||||||
import org.elasticsearch.cluster.metadata.IndexNameExpressionResolver;
|
import org.elasticsearch.cluster.metadata.IndexNameExpressionResolver;
|
||||||
import org.elasticsearch.cluster.service.ClusterService;
|
import org.elasticsearch.cluster.service.ClusterService;
|
||||||
import org.elasticsearch.common.inject.Inject;
|
import org.elasticsearch.common.inject.Inject;
|
||||||
|
@ -25,18 +27,21 @@ import org.elasticsearch.common.io.stream.StreamInput;
|
||||||
import org.elasticsearch.common.io.stream.StreamOutput;
|
import org.elasticsearch.common.io.stream.StreamOutput;
|
||||||
import org.elasticsearch.common.settings.Settings;
|
import org.elasticsearch.common.settings.Settings;
|
||||||
import org.elasticsearch.common.unit.TimeValue;
|
import org.elasticsearch.common.unit.TimeValue;
|
||||||
|
import org.elasticsearch.common.xcontent.ToXContentObject;
|
||||||
|
import org.elasticsearch.common.xcontent.XContentBuilder;
|
||||||
|
import org.elasticsearch.rest.RestStatus;
|
||||||
|
import org.elasticsearch.tasks.TaskInfo;
|
||||||
import org.elasticsearch.threadpool.ThreadPool;
|
import org.elasticsearch.threadpool.ThreadPool;
|
||||||
import org.elasticsearch.transport.TransportService;
|
import org.elasticsearch.transport.TransportService;
|
||||||
import org.elasticsearch.xpack.ml.job.Job;
|
import org.elasticsearch.xpack.ml.job.Job;
|
||||||
import org.elasticsearch.xpack.ml.job.JobStatus;
|
import org.elasticsearch.xpack.ml.job.JobStatus;
|
||||||
import org.elasticsearch.xpack.ml.job.manager.JobManager;
|
|
||||||
import org.elasticsearch.xpack.ml.job.metadata.Allocation;
|
import org.elasticsearch.xpack.ml.job.metadata.Allocation;
|
||||||
import org.elasticsearch.xpack.ml.job.metadata.MlMetadata;
|
import org.elasticsearch.xpack.ml.job.metadata.MlMetadata;
|
||||||
import org.elasticsearch.xpack.ml.utils.ExceptionsHelper;
|
import org.elasticsearch.xpack.ml.utils.ExceptionsHelper;
|
||||||
|
import org.elasticsearch.xpack.ml.utils.JobStatusObserver;
|
||||||
|
|
||||||
import java.io.IOException;
|
import java.io.IOException;
|
||||||
import java.util.Objects;
|
import java.util.Objects;
|
||||||
import java.util.function.Predicate;
|
|
||||||
|
|
||||||
public class CloseJobAction extends Action<CloseJobAction.Request, CloseJobAction.Response, CloseJobAction.RequestBuilder> {
|
public class CloseJobAction extends Action<CloseJobAction.Request, CloseJobAction.Response, CloseJobAction.RequestBuilder> {
|
||||||
|
|
||||||
|
@ -57,7 +62,7 @@ public class CloseJobAction extends Action<CloseJobAction.Request, CloseJobActio
|
||||||
return new Response();
|
return new Response();
|
||||||
}
|
}
|
||||||
|
|
||||||
public static class Request extends AcknowledgedRequest<Request> {
|
public static class Request extends ActionRequest {
|
||||||
|
|
||||||
private String jobId;
|
private String jobId;
|
||||||
private TimeValue closeTimeout = TimeValue.timeValueMinutes(30);
|
private TimeValue closeTimeout = TimeValue.timeValueMinutes(30);
|
||||||
|
@ -118,112 +123,127 @@ public class CloseJobAction extends Action<CloseJobAction.Request, CloseJobActio
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
static class RequestBuilder extends MasterNodeOperationRequestBuilder<Request, Response, RequestBuilder> {
|
static class RequestBuilder extends ActionRequestBuilder<Request, Response, RequestBuilder> {
|
||||||
|
|
||||||
public RequestBuilder(ElasticsearchClient client, CloseJobAction action) {
|
public RequestBuilder(ElasticsearchClient client, CloseJobAction action) {
|
||||||
super(client, action, new Request());
|
super(client, action, new Request());
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
public static class Response extends AcknowledgedResponse {
|
public static class Response extends ActionResponse implements ToXContentObject {
|
||||||
|
|
||||||
private Response() {
|
private boolean closed;
|
||||||
|
|
||||||
|
Response() {
|
||||||
}
|
}
|
||||||
|
|
||||||
private Response(boolean acknowledged) {
|
Response(boolean closed) {
|
||||||
super(acknowledged);
|
this.closed = closed;
|
||||||
|
}
|
||||||
|
|
||||||
|
public boolean isClosed() {
|
||||||
|
return closed;
|
||||||
}
|
}
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
public void readFrom(StreamInput in) throws IOException {
|
public void readFrom(StreamInput in) throws IOException {
|
||||||
super.readFrom(in);
|
super.readFrom(in);
|
||||||
readAcknowledged(in);
|
closed = in.readBoolean();
|
||||||
}
|
}
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
public void writeTo(StreamOutput out) throws IOException {
|
public void writeTo(StreamOutput out) throws IOException {
|
||||||
super.writeTo(out);
|
super.writeTo(out);
|
||||||
writeAcknowledged(out);
|
out.writeBoolean(closed);
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public XContentBuilder toXContent(XContentBuilder builder, Params params) throws IOException {
|
||||||
|
builder.startObject();
|
||||||
|
builder.field("closed", closed);
|
||||||
|
builder.endObject();
|
||||||
|
return builder;
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public boolean equals(Object o) {
|
||||||
|
if (this == o) return true;
|
||||||
|
if (o == null || getClass() != o.getClass()) return false;
|
||||||
|
Response response = (Response) o;
|
||||||
|
return closed == response.closed;
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public int hashCode() {
|
||||||
|
return Objects.hash(closed);
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
public static class TransportAction extends TransportMasterNodeAction<Request, Response> {
|
public static class TransportAction extends HandledTransportAction<Request, Response> {
|
||||||
|
|
||||||
private final JobManager jobManager;
|
private final ClusterService clusterService;
|
||||||
|
private final JobStatusObserver jobStatusObserver;
|
||||||
|
private final TransportListTasksAction listTasksAction;
|
||||||
|
private final TransportCancelTasksAction cancelTasksAction;
|
||||||
|
|
||||||
@Inject
|
@Inject
|
||||||
public TransportAction(Settings settings, TransportService transportService, ClusterService clusterService,
|
public TransportAction(Settings settings, TransportService transportService, ThreadPool threadPool,
|
||||||
ThreadPool threadPool, ActionFilters actionFilters, IndexNameExpressionResolver indexNameExpressionResolver,
|
ActionFilters actionFilters, IndexNameExpressionResolver indexNameExpressionResolver,
|
||||||
JobManager jobManager) {
|
ClusterService clusterService, TransportCancelTasksAction cancelTasksAction,
|
||||||
super(settings, CloseJobAction.NAME, transportService, clusterService, threadPool, actionFilters,
|
TransportListTasksAction listTasksAction) {
|
||||||
indexNameExpressionResolver, Request::new);
|
super(settings, CloseJobAction.NAME, threadPool, transportService, actionFilters, indexNameExpressionResolver, Request::new);
|
||||||
this.jobManager = jobManager;
|
this.clusterService = clusterService;
|
||||||
|
this.jobStatusObserver = new JobStatusObserver(threadPool, clusterService);
|
||||||
|
this.cancelTasksAction = cancelTasksAction;
|
||||||
|
this.listTasksAction = listTasksAction;
|
||||||
}
|
}
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
protected String executor() {
|
protected void doExecute(Request request, ActionListener<Response> listener) {
|
||||||
return ThreadPool.Names.SAME;
|
MlMetadata mlMetadata = clusterService.state().metaData().custom(MlMetadata.TYPE);
|
||||||
|
validate(request.jobId, mlMetadata);
|
||||||
|
|
||||||
|
ListTasksRequest listTasksRequest = new ListTasksRequest();
|
||||||
|
listTasksRequest.setActions(InternalOpenJobAction.NAME);
|
||||||
|
listTasksRequest.setDetailed(true);
|
||||||
|
listTasksAction.execute(listTasksRequest, ActionListener.wrap(listTasksResponse -> {
|
||||||
|
String expectedJobDescription = "job-" + request.jobId;
|
||||||
|
for (TaskInfo taskInfo : listTasksResponse.getTasks()) {
|
||||||
|
if (expectedJobDescription.equals(taskInfo.getDescription())) {
|
||||||
|
CancelTasksRequest cancelTasksRequest = new CancelTasksRequest();
|
||||||
|
cancelTasksRequest.setTaskId(taskInfo.getTaskId());
|
||||||
|
cancelTasksAction.execute(cancelTasksRequest, ActionListener.wrap(
|
||||||
|
cancelTasksResponse -> {
|
||||||
|
jobStatusObserver.waitForStatus(request.jobId, request.closeTimeout, JobStatus.CLOSED,
|
||||||
|
e -> {
|
||||||
|
if (e != null) {
|
||||||
|
listener.onFailure(e);
|
||||||
|
} else {
|
||||||
|
listener.onResponse(new CloseJobAction.Response(true));
|
||||||
|
}
|
||||||
|
}
|
||||||
|
);
|
||||||
|
},
|
||||||
|
listener::onFailure)
|
||||||
|
);
|
||||||
|
return;
|
||||||
|
}
|
||||||
|
}
|
||||||
|
listener.onFailure(new ResourceNotFoundException("No job [" + request.jobId + "] running"));
|
||||||
|
}, listener::onFailure));
|
||||||
}
|
}
|
||||||
|
|
||||||
@Override
|
static void validate(String jobId, MlMetadata mlMetadata) {
|
||||||
protected Response newResponse() {
|
Allocation allocation = mlMetadata.getAllocations().get(jobId);
|
||||||
return new Response();
|
if (allocation == null) {
|
||||||
|
throw ExceptionsHelper.missingJobException(jobId);
|
||||||
}
|
}
|
||||||
|
|
||||||
@Override
|
if (allocation.getStatus() != JobStatus.OPENED) {
|
||||||
protected void masterOperation(Request request, ClusterState state, ActionListener<Response> listener) throws Exception {
|
throw new ElasticsearchStatusException("job not opened, expected job status [{}], but got [{}]",
|
||||||
UpdateJobStatusAction.Request updateStatusRequest = new UpdateJobStatusAction.Request(request.getJobId(), JobStatus.CLOSING);
|
RestStatus.CONFLICT, JobStatus.OPENED, allocation.getStatus());
|
||||||
ActionListener<UpdateJobStatusAction.Response> delegateListener = ActionListener.wrap(
|
|
||||||
response -> respondWhenJobIsClosed(request.getJobId(), listener), listener::onFailure);
|
|
||||||
jobManager.setJobStatus(updateStatusRequest, delegateListener);
|
|
||||||
}
|
}
|
||||||
|
|
||||||
private void respondWhenJobIsClosed(String jobId, ActionListener<Response> listener) {
|
|
||||||
ClusterStateObserver observer = new ClusterStateObserver(clusterService, logger, threadPool.getThreadContext());
|
|
||||||
observer.waitForNextChange(new ClusterStateObserver.Listener() {
|
|
||||||
@Override
|
|
||||||
public void onNewClusterState(ClusterState state) {
|
|
||||||
listener.onResponse(new Response(true));
|
|
||||||
}
|
}
|
||||||
|
|
||||||
@Override
|
|
||||||
public void onClusterServiceClose() {
|
|
||||||
listener.onFailure(new IllegalStateException("Cluster service closed while waiting for job [" + jobId
|
|
||||||
+ "] status to change to [" + JobStatus.CLOSED + "]"));
|
|
||||||
}
|
|
||||||
|
|
||||||
@Override
|
|
||||||
public void onTimeout(TimeValue timeout) {
|
|
||||||
listener.onFailure(new IllegalStateException(
|
|
||||||
"Timeout expired while waiting for job [" + jobId + "] status to change to [" + JobStatus.CLOSED + "]"));
|
|
||||||
}
|
|
||||||
}, new JobClosedChangePredicate(jobId), TimeValue.timeValueMinutes(30));
|
|
||||||
}
|
|
||||||
|
|
||||||
private class JobClosedChangePredicate implements Predicate<ClusterState> {
|
|
||||||
|
|
||||||
private final String jobId;
|
|
||||||
|
|
||||||
JobClosedChangePredicate(String jobId) {
|
|
||||||
this.jobId = jobId;
|
|
||||||
}
|
|
||||||
|
|
||||||
@Override
|
|
||||||
public boolean test(ClusterState newState) {
|
|
||||||
MlMetadata metadata = newState.getMetaData().custom(MlMetadata.TYPE);
|
|
||||||
if (metadata != null) {
|
|
||||||
Allocation allocation = metadata.getAllocations().get(jobId);
|
|
||||||
return allocation != null && allocation.getStatus() == JobStatus.CLOSED;
|
|
||||||
}
|
|
||||||
return false;
|
|
||||||
}
|
|
||||||
|
|
||||||
}
|
|
||||||
@Override
|
|
||||||
protected ClusterBlockException checkBlock(Request request, ClusterState state) {
|
|
||||||
return state.blocks().globalBlockedException(ClusterBlockLevel.METADATA_WRITE);
|
|
||||||
}
|
|
||||||
|
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
|
|
|
@ -8,20 +8,20 @@ package org.elasticsearch.xpack.ml.action;
|
||||||
import org.elasticsearch.action.Action;
|
import org.elasticsearch.action.Action;
|
||||||
import org.elasticsearch.action.ActionListener;
|
import org.elasticsearch.action.ActionListener;
|
||||||
import org.elasticsearch.action.ActionRequestBuilder;
|
import org.elasticsearch.action.ActionRequestBuilder;
|
||||||
import org.elasticsearch.action.ActionRequestValidationException;
|
|
||||||
import org.elasticsearch.action.support.ActionFilters;
|
import org.elasticsearch.action.support.ActionFilters;
|
||||||
import org.elasticsearch.action.support.HandledTransportAction;
|
import org.elasticsearch.action.support.tasks.BaseTasksResponse;
|
||||||
import org.elasticsearch.action.support.master.AcknowledgedResponse;
|
|
||||||
import org.elasticsearch.action.support.master.MasterNodeRequest;
|
|
||||||
import org.elasticsearch.client.ElasticsearchClient;
|
import org.elasticsearch.client.ElasticsearchClient;
|
||||||
import org.elasticsearch.cluster.metadata.IndexNameExpressionResolver;
|
import org.elasticsearch.cluster.metadata.IndexNameExpressionResolver;
|
||||||
|
import org.elasticsearch.cluster.service.ClusterService;
|
||||||
import org.elasticsearch.common.ParseField;
|
import org.elasticsearch.common.ParseField;
|
||||||
import org.elasticsearch.common.inject.Inject;
|
import org.elasticsearch.common.inject.Inject;
|
||||||
import org.elasticsearch.common.io.stream.StreamInput;
|
import org.elasticsearch.common.io.stream.StreamInput;
|
||||||
import org.elasticsearch.common.io.stream.StreamOutput;
|
import org.elasticsearch.common.io.stream.StreamOutput;
|
||||||
|
import org.elasticsearch.common.io.stream.Writeable;
|
||||||
import org.elasticsearch.common.settings.Settings;
|
import org.elasticsearch.common.settings.Settings;
|
||||||
import org.elasticsearch.common.xcontent.ObjectParser;
|
import org.elasticsearch.common.xcontent.ObjectParser;
|
||||||
import org.elasticsearch.common.xcontent.ToXContent;
|
import org.elasticsearch.common.xcontent.ToXContent;
|
||||||
|
import org.elasticsearch.common.xcontent.ToXContentObject;
|
||||||
import org.elasticsearch.common.xcontent.XContentBuilder;
|
import org.elasticsearch.common.xcontent.XContentBuilder;
|
||||||
import org.elasticsearch.common.xcontent.XContentParser;
|
import org.elasticsearch.common.xcontent.XContentParser;
|
||||||
import org.elasticsearch.threadpool.ThreadPool;
|
import org.elasticsearch.threadpool.ThreadPool;
|
||||||
|
@ -56,7 +56,7 @@ public class FlushJobAction extends Action<FlushJobAction.Request, FlushJobActio
|
||||||
return new Response();
|
return new Response();
|
||||||
}
|
}
|
||||||
|
|
||||||
public static class Request extends MasterNodeRequest<Request> implements ToXContent {
|
public static class Request extends TransportJobTaskAction.JobTaskRequest<Request> implements ToXContent {
|
||||||
|
|
||||||
public static final ParseField CALC_INTERIM = new ParseField("calc_interim");
|
public static final ParseField CALC_INTERIM = new ParseField("calc_interim");
|
||||||
public static final ParseField START = new ParseField("start");
|
public static final ParseField START = new ParseField("start");
|
||||||
|
@ -81,7 +81,6 @@ public class FlushJobAction extends Action<FlushJobAction.Request, FlushJobActio
|
||||||
return request;
|
return request;
|
||||||
}
|
}
|
||||||
|
|
||||||
private String jobId;
|
|
||||||
private boolean calcInterim = false;
|
private boolean calcInterim = false;
|
||||||
private String start;
|
private String start;
|
||||||
private String end;
|
private String end;
|
||||||
|
@ -128,15 +127,9 @@ public class FlushJobAction extends Action<FlushJobAction.Request, FlushJobActio
|
||||||
this.advanceTime = advanceTime;
|
this.advanceTime = advanceTime;
|
||||||
}
|
}
|
||||||
|
|
||||||
@Override
|
|
||||||
public ActionRequestValidationException validate() {
|
|
||||||
return null;
|
|
||||||
}
|
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
public void readFrom(StreamInput in) throws IOException {
|
public void readFrom(StreamInput in) throws IOException {
|
||||||
super.readFrom(in);
|
super.readFrom(in);
|
||||||
jobId = in.readString();
|
|
||||||
calcInterim = in.readBoolean();
|
calcInterim = in.readBoolean();
|
||||||
start = in.readOptionalString();
|
start = in.readOptionalString();
|
||||||
end = in.readOptionalString();
|
end = in.readOptionalString();
|
||||||
|
@ -146,7 +139,6 @@ public class FlushJobAction extends Action<FlushJobAction.Request, FlushJobActio
|
||||||
@Override
|
@Override
|
||||||
public void writeTo(StreamOutput out) throws IOException {
|
public void writeTo(StreamOutput out) throws IOException {
|
||||||
super.writeTo(out);
|
super.writeTo(out);
|
||||||
out.writeString(jobId);
|
|
||||||
out.writeBoolean(calcInterim);
|
out.writeBoolean(calcInterim);
|
||||||
out.writeOptionalString(start);
|
out.writeOptionalString(start);
|
||||||
out.writeOptionalString(end);
|
out.writeOptionalString(end);
|
||||||
|
@ -200,48 +192,77 @@ public class FlushJobAction extends Action<FlushJobAction.Request, FlushJobActio
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
public static class Response extends AcknowledgedResponse {
|
public static class Response extends BaseTasksResponse implements Writeable, ToXContentObject {
|
||||||
|
|
||||||
private Response() {
|
private boolean flushed;
|
||||||
|
|
||||||
|
Response() {
|
||||||
}
|
}
|
||||||
|
|
||||||
private Response(boolean acknowledged) {
|
Response(boolean flushed) {
|
||||||
super(acknowledged);
|
super(null, null);
|
||||||
|
this.flushed = flushed;
|
||||||
|
}
|
||||||
|
|
||||||
|
public boolean isFlushed() {
|
||||||
|
return flushed;
|
||||||
}
|
}
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
public void readFrom(StreamInput in) throws IOException {
|
public void readFrom(StreamInput in) throws IOException {
|
||||||
super.readFrom(in);
|
super.readFrom(in);
|
||||||
readAcknowledged(in);
|
flushed = in.readBoolean();
|
||||||
}
|
}
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
public void writeTo(StreamOutput out) throws IOException {
|
public void writeTo(StreamOutput out) throws IOException {
|
||||||
super.writeTo(out);
|
super.writeTo(out);
|
||||||
writeAcknowledged(out);
|
out.writeBoolean(flushed);
|
||||||
}
|
|
||||||
}
|
|
||||||
|
|
||||||
public static class TransportAction extends HandledTransportAction<Request, Response> {
|
|
||||||
|
|
||||||
// NORELEASE This should be a master node operation that updates the job's state
|
|
||||||
private final AutodetectProcessManager processManager;
|
|
||||||
private final JobManager jobManager;
|
|
||||||
|
|
||||||
@Inject
|
|
||||||
public TransportAction(Settings settings, TransportService transportService, ThreadPool threadPool, ActionFilters actionFilters,
|
|
||||||
IndexNameExpressionResolver indexNameExpressionResolver, AutodetectProcessManager processManager, JobManager jobManager) {
|
|
||||||
super(settings, FlushJobAction.NAME, false, threadPool, transportService, actionFilters,
|
|
||||||
indexNameExpressionResolver, FlushJobAction.Request::new);
|
|
||||||
|
|
||||||
this.processManager = processManager;
|
|
||||||
this.jobManager = jobManager;
|
|
||||||
}
|
}
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
protected final void doExecute(FlushJobAction.Request request, ActionListener<FlushJobAction.Response> listener) {
|
public XContentBuilder toXContent(XContentBuilder builder, Params params) throws IOException {
|
||||||
threadPool.executor(MlPlugin.THREAD_POOL_NAME).execute(() -> {
|
builder.startObject();
|
||||||
try {
|
builder.field("flushed", flushed);
|
||||||
|
builder.endObject();
|
||||||
|
return builder;
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public boolean equals(Object o) {
|
||||||
|
if (this == o) return true;
|
||||||
|
if (o == null || getClass() != o.getClass()) return false;
|
||||||
|
Response response = (Response) o;
|
||||||
|
return flushed == response.flushed;
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public int hashCode() {
|
||||||
|
return Objects.hash(flushed);
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
public static class TransportAction extends TransportJobTaskAction<InternalOpenJobAction.JobTask, Request, Response> {
|
||||||
|
|
||||||
|
@Inject
|
||||||
|
public TransportAction(Settings settings, TransportService transportService, ThreadPool threadPool, ClusterService clusterService,
|
||||||
|
ActionFilters actionFilters, IndexNameExpressionResolver indexNameExpressionResolver,
|
||||||
|
AutodetectProcessManager processManager, JobManager jobManager) {
|
||||||
|
super(settings, FlushJobAction.NAME, threadPool, clusterService, transportService, actionFilters,
|
||||||
|
indexNameExpressionResolver, FlushJobAction.Request::new, FlushJobAction.Response::new, MlPlugin.THREAD_POOL_NAME,
|
||||||
|
jobManager, processManager, Request::getJobId);
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
protected FlushJobAction.Response readTaskResponse(StreamInput in) throws IOException {
|
||||||
|
Response response = new Response();
|
||||||
|
response.readFrom(in);
|
||||||
|
return response;
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
protected void taskOperation(Request request, InternalOpenJobAction.JobTask task,
|
||||||
|
ActionListener<FlushJobAction.Response> listener) {
|
||||||
jobManager.getJobOrThrowIfUnknown(request.getJobId());
|
jobManager.getJobOrThrowIfUnknown(request.getJobId());
|
||||||
|
|
||||||
InterimResultsParams.Builder paramsBuilder = InterimResultsParams.builder();
|
InterimResultsParams.Builder paramsBuilder = InterimResultsParams.builder();
|
||||||
|
@ -259,10 +280,6 @@ public class FlushJobAction extends Action<FlushJobAction.Request, FlushJobActio
|
||||||
paramsBuilder.forTimeRange(timeRangeBuilder.build());
|
paramsBuilder.forTimeRange(timeRangeBuilder.build());
|
||||||
processManager.flushJob(request.getJobId(), paramsBuilder.build());
|
processManager.flushJob(request.getJobId(), paramsBuilder.build());
|
||||||
listener.onResponse(new Response(true));
|
listener.onResponse(new Response(true));
|
||||||
} catch (Exception e) {
|
|
||||||
listener.onFailure(e);
|
|
||||||
}
|
|
||||||
});
|
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
|
@ -14,6 +14,7 @@ import org.elasticsearch.action.ActionResponse;
|
||||||
import org.elasticsearch.action.support.ActionFilters;
|
import org.elasticsearch.action.support.ActionFilters;
|
||||||
import org.elasticsearch.action.support.HandledTransportAction;
|
import org.elasticsearch.action.support.HandledTransportAction;
|
||||||
import org.elasticsearch.client.ElasticsearchClient;
|
import org.elasticsearch.client.ElasticsearchClient;
|
||||||
|
import org.elasticsearch.cluster.ClusterState;
|
||||||
import org.elasticsearch.cluster.metadata.IndexNameExpressionResolver;
|
import org.elasticsearch.cluster.metadata.IndexNameExpressionResolver;
|
||||||
import org.elasticsearch.cluster.service.ClusterService;
|
import org.elasticsearch.cluster.service.ClusterService;
|
||||||
import org.elasticsearch.common.Nullable;
|
import org.elasticsearch.common.Nullable;
|
||||||
|
@ -299,12 +300,14 @@ public class GetJobsStatsAction extends Action<GetJobsStatsAction.Request, GetJo
|
||||||
@Override
|
@Override
|
||||||
protected void doExecute(Request request, ActionListener<Response> listener) {
|
protected void doExecute(Request request, ActionListener<Response> listener) {
|
||||||
logger.debug("Get stats for job '{}'", request.getJobId());
|
logger.debug("Get stats for job '{}'", request.getJobId());
|
||||||
QueryPage<Job> jobs = jobManager.getJob(request.getJobId(), clusterService.state());
|
ClusterState clusterState = clusterService.state();
|
||||||
|
QueryPage<Job> jobs = jobManager.getJob(request.getJobId(), clusterState);
|
||||||
if (jobs.count() == 0) {
|
if (jobs.count() == 0) {
|
||||||
listener.onResponse(new GetJobsStatsAction.Response(new QueryPage<>(Collections.emptyList(), 0, Job.RESULTS_FIELD)));
|
listener.onResponse(new GetJobsStatsAction.Response(new QueryPage<>(Collections.emptyList(), 0, Job.RESULTS_FIELD)));
|
||||||
|
return;
|
||||||
}
|
}
|
||||||
MlMetadata mlMetadata = clusterService.state().metaData().custom(MlMetadata.TYPE);
|
|
||||||
|
|
||||||
|
MlMetadata mlMetadata = clusterState.metaData().custom(MlMetadata.TYPE);
|
||||||
AtomicInteger counter = new AtomicInteger(0);
|
AtomicInteger counter = new AtomicInteger(0);
|
||||||
AtomicArray<Response.JobStats> jobsStats = new AtomicArray<>(jobs.results().size());
|
AtomicArray<Response.JobStats> jobsStats = new AtomicArray<>(jobs.results().size());
|
||||||
for (int i = 0; i < jobs.results().size(); i++) {
|
for (int i = 0; i < jobs.results().size(); i++) {
|
||||||
|
|
|
@ -0,0 +1,135 @@
|
||||||
|
/*
|
||||||
|
* Copyright Elasticsearch B.V. and/or licensed to Elasticsearch B.V. under one
|
||||||
|
* or more contributor license agreements. Licensed under the Elastic License;
|
||||||
|
* you may not use this file except in compliance with the Elastic License.
|
||||||
|
*/
|
||||||
|
package org.elasticsearch.xpack.ml.action;
|
||||||
|
|
||||||
|
import org.elasticsearch.action.Action;
|
||||||
|
import org.elasticsearch.action.ActionListener;
|
||||||
|
import org.elasticsearch.action.ActionRequestBuilder;
|
||||||
|
import org.elasticsearch.action.ActionResponse;
|
||||||
|
import org.elasticsearch.action.support.ActionFilters;
|
||||||
|
import org.elasticsearch.action.support.HandledTransportAction;
|
||||||
|
import org.elasticsearch.client.ElasticsearchClient;
|
||||||
|
import org.elasticsearch.cluster.metadata.IndexNameExpressionResolver;
|
||||||
|
import org.elasticsearch.common.inject.Inject;
|
||||||
|
import org.elasticsearch.common.settings.Settings;
|
||||||
|
import org.elasticsearch.tasks.CancellableTask;
|
||||||
|
import org.elasticsearch.tasks.Task;
|
||||||
|
import org.elasticsearch.tasks.TaskId;
|
||||||
|
import org.elasticsearch.threadpool.ThreadPool;
|
||||||
|
import org.elasticsearch.transport.TransportService;
|
||||||
|
import org.elasticsearch.xpack.ml.job.JobStatus;
|
||||||
|
import org.elasticsearch.xpack.ml.job.manager.AutodetectProcessManager;
|
||||||
|
|
||||||
|
public class InternalOpenJobAction extends Action<InternalOpenJobAction.Request, InternalOpenJobAction.Response,
|
||||||
|
InternalOpenJobAction.RequestBuilder> {
|
||||||
|
|
||||||
|
public static final InternalOpenJobAction INSTANCE = new InternalOpenJobAction();
|
||||||
|
public static final String NAME = "cluster:admin/ml/job/internal_open";
|
||||||
|
|
||||||
|
private InternalOpenJobAction() {
|
||||||
|
super(NAME);
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public RequestBuilder newRequestBuilder(ElasticsearchClient client) {
|
||||||
|
return new RequestBuilder(client, this);
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public Response newResponse() {
|
||||||
|
return new Response();
|
||||||
|
}
|
||||||
|
|
||||||
|
public static class Request extends OpenJobAction.Request {
|
||||||
|
|
||||||
|
public Request(String jobId) {
|
||||||
|
super(jobId);
|
||||||
|
}
|
||||||
|
|
||||||
|
Request() {
|
||||||
|
super();
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public Task createTask(long id, String type, String action, TaskId parentTaskId) {
|
||||||
|
return new JobTask(getJobId(), id, type, action, parentTaskId);
|
||||||
|
}
|
||||||
|
|
||||||
|
}
|
||||||
|
|
||||||
|
static class RequestBuilder extends ActionRequestBuilder<Request, Response, RequestBuilder> {
|
||||||
|
|
||||||
|
public RequestBuilder(ElasticsearchClient client, InternalOpenJobAction action) {
|
||||||
|
super(client, action, new Request());
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
public static class Response extends ActionResponse {
|
||||||
|
|
||||||
|
Response() {}
|
||||||
|
|
||||||
|
}
|
||||||
|
|
||||||
|
public static class JobTask extends CancellableTask {
|
||||||
|
|
||||||
|
private volatile Runnable cancelHandler;
|
||||||
|
|
||||||
|
JobTask(String jobId, long id, String type, String action, TaskId parentTask) {
|
||||||
|
super(id, type, action, "job-" + jobId, parentTask);
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public boolean shouldCancelChildrenOnCancellation() {
|
||||||
|
return true;
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
protected void onCancelled() {
|
||||||
|
cancelHandler.run();
|
||||||
|
}
|
||||||
|
|
||||||
|
static boolean match(Task task, String expectedJobId) {
|
||||||
|
String expectedDescription = "job-" + expectedJobId;
|
||||||
|
return task instanceof JobTask && expectedDescription.equals(task.getDescription());
|
||||||
|
}
|
||||||
|
|
||||||
|
}
|
||||||
|
|
||||||
|
public static class TransportAction extends HandledTransportAction<Request, Response> {
|
||||||
|
|
||||||
|
private final AutodetectProcessManager autodetectProcessManager;
|
||||||
|
|
||||||
|
@Inject
|
||||||
|
public TransportAction(Settings settings, TransportService transportService, ThreadPool threadPool,
|
||||||
|
ActionFilters actionFilters, IndexNameExpressionResolver indexNameExpressionResolver,
|
||||||
|
AutodetectProcessManager autodetectProcessManager) {
|
||||||
|
super(settings, InternalOpenJobAction.NAME, threadPool, transportService, actionFilters, indexNameExpressionResolver,
|
||||||
|
Request::new);
|
||||||
|
this.autodetectProcessManager = autodetectProcessManager;
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
protected void doExecute(Task task, Request request, ActionListener<Response> listener) {
|
||||||
|
JobTask jobTask = (JobTask) task;
|
||||||
|
autodetectProcessManager.setJobStatus(request.getJobId(), JobStatus.OPENING, aVoid -> {
|
||||||
|
jobTask.cancelHandler = () -> autodetectProcessManager.closeJob(request.getJobId());
|
||||||
|
autodetectProcessManager.openJob(request.getJobId(), request.isIgnoreDowntime(), e -> {
|
||||||
|
if (e == null) {
|
||||||
|
listener.onResponse(new Response());
|
||||||
|
} else {
|
||||||
|
listener.onFailure(e);
|
||||||
|
}
|
||||||
|
});
|
||||||
|
}, listener::onFailure);
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
protected void doExecute(Request request, ActionListener<Response> listener) {
|
||||||
|
throw new IllegalStateException("shouldn't get invoked");
|
||||||
|
}
|
||||||
|
|
||||||
|
}
|
||||||
|
}
|
|
@ -5,41 +5,35 @@
|
||||||
*/
|
*/
|
||||||
package org.elasticsearch.xpack.ml.action;
|
package org.elasticsearch.xpack.ml.action;
|
||||||
|
|
||||||
import org.elasticsearch.ElasticsearchStatusException;
|
|
||||||
import org.elasticsearch.action.Action;
|
import org.elasticsearch.action.Action;
|
||||||
import org.elasticsearch.action.ActionListener;
|
import org.elasticsearch.action.ActionListener;
|
||||||
|
import org.elasticsearch.action.ActionRequest;
|
||||||
|
import org.elasticsearch.action.ActionRequestBuilder;
|
||||||
import org.elasticsearch.action.ActionRequestValidationException;
|
import org.elasticsearch.action.ActionRequestValidationException;
|
||||||
|
import org.elasticsearch.action.ActionResponse;
|
||||||
import org.elasticsearch.action.support.ActionFilters;
|
import org.elasticsearch.action.support.ActionFilters;
|
||||||
import org.elasticsearch.action.support.master.AcknowledgedRequest;
|
import org.elasticsearch.action.support.HandledTransportAction;
|
||||||
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.client.ElasticsearchClient;
|
||||||
import org.elasticsearch.cluster.ClusterState;
|
|
||||||
import org.elasticsearch.cluster.ClusterStateObserver;
|
|
||||||
import org.elasticsearch.cluster.block.ClusterBlockException;
|
|
||||||
import org.elasticsearch.cluster.block.ClusterBlockLevel;
|
|
||||||
import org.elasticsearch.cluster.metadata.IndexNameExpressionResolver;
|
import org.elasticsearch.cluster.metadata.IndexNameExpressionResolver;
|
||||||
import org.elasticsearch.cluster.service.ClusterService;
|
import org.elasticsearch.cluster.service.ClusterService;
|
||||||
import org.elasticsearch.common.ParseField;
|
|
||||||
import org.elasticsearch.common.inject.Inject;
|
import org.elasticsearch.common.inject.Inject;
|
||||||
import org.elasticsearch.common.io.stream.StreamInput;
|
import org.elasticsearch.common.io.stream.StreamInput;
|
||||||
import org.elasticsearch.common.io.stream.StreamOutput;
|
import org.elasticsearch.common.io.stream.StreamOutput;
|
||||||
import org.elasticsearch.common.settings.Settings;
|
import org.elasticsearch.common.settings.Settings;
|
||||||
import org.elasticsearch.common.unit.TimeValue;
|
import org.elasticsearch.common.unit.TimeValue;
|
||||||
import org.elasticsearch.rest.RestStatus;
|
import org.elasticsearch.common.xcontent.ToXContentObject;
|
||||||
|
import org.elasticsearch.common.xcontent.XContentBuilder;
|
||||||
|
import org.elasticsearch.tasks.LoggingTaskListener;
|
||||||
import org.elasticsearch.threadpool.ThreadPool;
|
import org.elasticsearch.threadpool.ThreadPool;
|
||||||
import org.elasticsearch.transport.TransportService;
|
import org.elasticsearch.transport.TransportService;
|
||||||
import org.elasticsearch.xpack.ml.job.Job;
|
import org.elasticsearch.xpack.ml.job.Job;
|
||||||
import org.elasticsearch.xpack.ml.job.JobStatus;
|
import org.elasticsearch.xpack.ml.job.JobStatus;
|
||||||
import org.elasticsearch.xpack.ml.job.manager.JobManager;
|
|
||||||
import org.elasticsearch.xpack.ml.job.metadata.Allocation;
|
|
||||||
import org.elasticsearch.xpack.ml.job.metadata.MlMetadata;
|
import org.elasticsearch.xpack.ml.job.metadata.MlMetadata;
|
||||||
import org.elasticsearch.xpack.ml.utils.ExceptionsHelper;
|
import org.elasticsearch.xpack.ml.utils.ExceptionsHelper;
|
||||||
|
import org.elasticsearch.xpack.ml.utils.JobStatusObserver;
|
||||||
|
|
||||||
import java.io.IOException;
|
import java.io.IOException;
|
||||||
import java.util.Objects;
|
import java.util.Objects;
|
||||||
import java.util.function.Predicate;
|
|
||||||
|
|
||||||
public class OpenJobAction extends Action<OpenJobAction.Request, OpenJobAction.Response, OpenJobAction.RequestBuilder> {
|
public class OpenJobAction extends Action<OpenJobAction.Request, OpenJobAction.Response, OpenJobAction.RequestBuilder> {
|
||||||
|
|
||||||
|
@ -60,13 +54,11 @@ public class OpenJobAction extends Action<OpenJobAction.Request, OpenJobAction.R
|
||||||
return new Response();
|
return new Response();
|
||||||
}
|
}
|
||||||
|
|
||||||
public static class Request extends AcknowledgedRequest<Request> {
|
public static class Request extends ActionRequest {
|
||||||
|
|
||||||
public static final ParseField OPEN_TIMEOUT = new ParseField("open_timeout");
|
|
||||||
|
|
||||||
private String jobId;
|
private String jobId;
|
||||||
private boolean ignoreDowntime;
|
private boolean ignoreDowntime;
|
||||||
private TimeValue openTimeout = TimeValue.timeValueMinutes(30);
|
private TimeValue openTimeout = TimeValue.timeValueSeconds(30);
|
||||||
|
|
||||||
public Request(String jobId) {
|
public Request(String jobId) {
|
||||||
this.jobId = ExceptionsHelper.requireNonNull(jobId, Job.ID.getPreferredName());
|
this.jobId = ExceptionsHelper.requireNonNull(jobId, Job.ID.getPreferredName());
|
||||||
|
@ -108,7 +100,7 @@ public class OpenJobAction extends Action<OpenJobAction.Request, OpenJobAction.R
|
||||||
super.readFrom(in);
|
super.readFrom(in);
|
||||||
jobId = in.readString();
|
jobId = in.readString();
|
||||||
ignoreDowntime = in.readBoolean();
|
ignoreDowntime = in.readBoolean();
|
||||||
openTimeout = new TimeValue(in);
|
openTimeout = TimeValue.timeValueMillis(in.readVLong());
|
||||||
}
|
}
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
|
@ -116,12 +108,12 @@ public class OpenJobAction extends Action<OpenJobAction.Request, OpenJobAction.R
|
||||||
super.writeTo(out);
|
super.writeTo(out);
|
||||||
out.writeString(jobId);
|
out.writeString(jobId);
|
||||||
out.writeBoolean(ignoreDowntime);
|
out.writeBoolean(ignoreDowntime);
|
||||||
openTimeout.writeTo(out);
|
out.writeVLong(openTimeout.millis());
|
||||||
}
|
}
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
public int hashCode() {
|
public int hashCode() {
|
||||||
return Objects.hash(jobId, ignoreDowntime, openTimeout);
|
return Objects.hash(jobId, ignoreDowntime);
|
||||||
}
|
}
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
|
@ -134,128 +126,108 @@ public class OpenJobAction extends Action<OpenJobAction.Request, OpenJobAction.R
|
||||||
}
|
}
|
||||||
OpenJobAction.Request other = (OpenJobAction.Request) obj;
|
OpenJobAction.Request other = (OpenJobAction.Request) obj;
|
||||||
return Objects.equals(jobId, other.jobId) &&
|
return Objects.equals(jobId, other.jobId) &&
|
||||||
Objects.equals(ignoreDowntime, other.ignoreDowntime) &&
|
Objects.equals(ignoreDowntime, other.ignoreDowntime);
|
||||||
Objects.equals(openTimeout, other.openTimeout);
|
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
static class RequestBuilder extends MasterNodeOperationRequestBuilder<Request, Response, RequestBuilder> {
|
static class RequestBuilder extends ActionRequestBuilder<Request, Response, RequestBuilder> {
|
||||||
|
|
||||||
public RequestBuilder(ElasticsearchClient client, OpenJobAction action) {
|
public RequestBuilder(ElasticsearchClient client, OpenJobAction action) {
|
||||||
super(client, action, new Request());
|
super(client, action, new Request());
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
public static class Response extends AcknowledgedResponse {
|
public static class Response extends ActionResponse implements ToXContentObject {
|
||||||
|
|
||||||
public Response(boolean acknowledged) {
|
private boolean opened;
|
||||||
super(acknowledged);
|
|
||||||
|
Response() {}
|
||||||
|
|
||||||
|
Response(boolean opened) {
|
||||||
|
this.opened = opened;
|
||||||
}
|
}
|
||||||
|
|
||||||
private Response() {}
|
public boolean isOpened() {
|
||||||
|
return opened;
|
||||||
|
}
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
public void readFrom(StreamInput in) throws IOException {
|
public void readFrom(StreamInput in) throws IOException {
|
||||||
super.readFrom(in);
|
super.readFrom(in);
|
||||||
readAcknowledged(in);
|
opened = in.readBoolean();
|
||||||
}
|
}
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
public void writeTo(StreamOutput out) throws IOException {
|
public void writeTo(StreamOutput out) throws IOException {
|
||||||
super.writeTo(out);
|
super.writeTo(out);
|
||||||
writeAcknowledged(out);
|
out.writeBoolean(opened);
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public XContentBuilder toXContent(XContentBuilder builder, Params params) throws IOException {
|
||||||
|
builder.startObject();
|
||||||
|
builder.field("opened", opened);
|
||||||
|
builder.endObject();
|
||||||
|
return builder;
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public boolean equals(Object o) {
|
||||||
|
if (this == o) return true;
|
||||||
|
if (o == null || getClass() != o.getClass()) return false;
|
||||||
|
Response response = (Response) o;
|
||||||
|
return opened == response.opened;
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public int hashCode() {
|
||||||
|
return Objects.hash(opened);
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
public static class TransportAction extends TransportMasterNodeAction<Request, Response> {
|
public static class TransportAction extends HandledTransportAction<Request, Response> {
|
||||||
|
|
||||||
private final JobManager jobManager;
|
private final JobStatusObserver observer;
|
||||||
|
private final ClusterService clusterService;
|
||||||
|
private final InternalOpenJobAction.TransportAction internalOpenJobAction;
|
||||||
|
|
||||||
@Inject
|
@Inject
|
||||||
public TransportAction(Settings settings, TransportService transportService, ClusterService clusterService,
|
public TransportAction(Settings settings, TransportService transportService, ThreadPool threadPool,
|
||||||
ThreadPool threadPool, ActionFilters actionFilters, IndexNameExpressionResolver indexNameExpressionResolver,
|
ActionFilters actionFilters, IndexNameExpressionResolver indexNameExpressionResolver,
|
||||||
JobManager jobManager) {
|
ClusterService clusterService, InternalOpenJobAction.TransportAction internalOpenJobAction) {
|
||||||
super(settings, OpenJobAction.NAME, transportService, clusterService, threadPool, actionFilters,
|
super(settings, OpenJobAction.NAME, threadPool, transportService, actionFilters, indexNameExpressionResolver, Request::new);
|
||||||
indexNameExpressionResolver, Request::new);
|
this.clusterService = clusterService;
|
||||||
this.jobManager = jobManager;
|
this.observer = new JobStatusObserver(threadPool, clusterService);
|
||||||
|
this.internalOpenJobAction = internalOpenJobAction;
|
||||||
}
|
}
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
protected String executor() {
|
protected void doExecute(Request request, ActionListener<Response> listener) {
|
||||||
return ThreadPool.Names.SAME;
|
// This validation happens also in InternalOpenJobAction, the reason we do it here too is that if it fails there
|
||||||
}
|
// we are unable to provide the user immediate feedback. We would create the task and the validation would fail
|
||||||
|
// in the background, whereas now the validation failure is part of the response being returned.
|
||||||
|
MlMetadata mlMetadata = clusterService.state().metaData().custom(MlMetadata.TYPE);
|
||||||
|
validate(mlMetadata, request.getJobId());
|
||||||
|
|
||||||
@Override
|
InternalOpenJobAction.Request internalRequest = new InternalOpenJobAction.Request(request.jobId);
|
||||||
protected Response newResponse() {
|
internalRequest.setIgnoreDowntime(internalRequest.isIgnoreDowntime());
|
||||||
return new Response();
|
internalOpenJobAction.execute(internalRequest, LoggingTaskListener.instance());
|
||||||
}
|
observer.waitForStatus(request.getJobId(), request.openTimeout, JobStatus.OPENED, e -> {
|
||||||
|
if (e != null) {
|
||||||
@Override
|
listener.onFailure(e);
|
||||||
protected void masterOperation(Request request, ClusterState state, ActionListener<Response> listener) throws Exception {
|
} else {
|
||||||
ActionListener<Response> delegateListener = ActionListener.wrap(response -> respondWhenJobIsOpened(request, listener),
|
|
||||||
listener::onFailure);
|
|
||||||
jobManager.openJob(request, delegateListener);
|
|
||||||
}
|
|
||||||
|
|
||||||
@Override
|
|
||||||
protected ClusterBlockException checkBlock(Request request, ClusterState state) {
|
|
||||||
return state.blocks().globalBlockedException(ClusterBlockLevel.METADATA_WRITE);
|
|
||||||
}
|
|
||||||
|
|
||||||
private void respondWhenJobIsOpened(Request request, ActionListener<Response> listener) {
|
|
||||||
ClusterStateObserver observer = new ClusterStateObserver(clusterService, logger, threadPool.getThreadContext());
|
|
||||||
observer.waitForNextChange(new ClusterStateObserver.Listener() {
|
|
||||||
@Override
|
|
||||||
public void onNewClusterState(ClusterState state) {
|
|
||||||
String jobId = request.getJobId();
|
|
||||||
MlMetadata metadata = state.getMetaData().custom(MlMetadata.TYPE);
|
|
||||||
Allocation allocation = metadata.getAllocations().get(jobId);
|
|
||||||
if (allocation != null) {
|
|
||||||
if (allocation.getStatus() == JobStatus.OPENED) {
|
|
||||||
listener.onResponse(new Response(true));
|
listener.onResponse(new Response(true));
|
||||||
} else {
|
|
||||||
String message = "[" + jobId + "] expected job status [" + JobStatus.OPENED + "], but got [" +
|
|
||||||
allocation.getStatus() + "], reason [" + allocation.getStatusReason() + "]";
|
|
||||||
listener.onFailure(new ElasticsearchStatusException(message, RestStatus.CONFLICT));
|
|
||||||
}
|
|
||||||
} else {
|
|
||||||
listener.onFailure(new IllegalStateException("no allocation for job [" + jobId + "]"));
|
|
||||||
}
|
}
|
||||||
|
});
|
||||||
}
|
}
|
||||||
|
|
||||||
@Override
|
/**
|
||||||
public void onClusterServiceClose() {
|
* Fail fast before trying to update the job status on master node if the job doesn't exist or its status
|
||||||
listener.onFailure(new IllegalStateException("Cluster service closed while waiting for job [" + request
|
* is not what it should be.
|
||||||
+ "] status to change to [" + JobStatus.OPENED + "]"));
|
*/
|
||||||
}
|
public static void validate(MlMetadata mlMetadata, String jobId) {
|
||||||
|
MlMetadata.Builder builder = new MlMetadata.Builder(mlMetadata);
|
||||||
@Override
|
builder.updateStatus(jobId, JobStatus.OPENING, null);
|
||||||
public void onTimeout(TimeValue timeout) {
|
|
||||||
listener.onFailure(new IllegalStateException(
|
|
||||||
"Timeout expired while waiting for job [" + request + "] status to change to [" + JobStatus.OPENED + "]"));
|
|
||||||
}
|
|
||||||
}, new JobOpenedChangePredicate(request.getJobId()), request.openTimeout);
|
|
||||||
}
|
|
||||||
|
|
||||||
private class JobOpenedChangePredicate implements Predicate<ClusterState> {
|
|
||||||
|
|
||||||
private final String jobId;
|
|
||||||
|
|
||||||
JobOpenedChangePredicate(String jobId) {
|
|
||||||
this.jobId = jobId;
|
|
||||||
}
|
|
||||||
|
|
||||||
@Override
|
|
||||||
public boolean test(ClusterState newState) {
|
|
||||||
MlMetadata metadata = newState.getMetaData().custom(MlMetadata.TYPE);
|
|
||||||
if (metadata != null) {
|
|
||||||
Allocation allocation = metadata.getAllocations().get(jobId);
|
|
||||||
if (allocation != null) {
|
|
||||||
return allocation.getStatus().isAnyOf(JobStatus.OPENED, JobStatus.FAILED);
|
|
||||||
}
|
|
||||||
}
|
|
||||||
return false;
|
|
||||||
}
|
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
|
@ -7,36 +7,31 @@ package org.elasticsearch.xpack.ml.action;
|
||||||
|
|
||||||
import org.elasticsearch.action.Action;
|
import org.elasticsearch.action.Action;
|
||||||
import org.elasticsearch.action.ActionListener;
|
import org.elasticsearch.action.ActionListener;
|
||||||
import org.elasticsearch.action.ActionRequest;
|
|
||||||
import org.elasticsearch.action.ActionRequestBuilder;
|
import org.elasticsearch.action.ActionRequestBuilder;
|
||||||
import org.elasticsearch.action.ActionRequestValidationException;
|
|
||||||
import org.elasticsearch.action.ActionResponse;
|
|
||||||
import org.elasticsearch.action.support.ActionFilters;
|
import org.elasticsearch.action.support.ActionFilters;
|
||||||
import org.elasticsearch.action.support.HandledTransportAction;
|
import org.elasticsearch.action.support.tasks.BaseTasksResponse;
|
||||||
import org.elasticsearch.client.ElasticsearchClient;
|
import org.elasticsearch.client.ElasticsearchClient;
|
||||||
import org.elasticsearch.cluster.metadata.IndexNameExpressionResolver;
|
import org.elasticsearch.cluster.metadata.IndexNameExpressionResolver;
|
||||||
|
import org.elasticsearch.cluster.service.ClusterService;
|
||||||
import org.elasticsearch.common.ParseField;
|
import org.elasticsearch.common.ParseField;
|
||||||
import org.elasticsearch.common.bytes.BytesReference;
|
import org.elasticsearch.common.bytes.BytesReference;
|
||||||
import org.elasticsearch.common.inject.Inject;
|
import org.elasticsearch.common.inject.Inject;
|
||||||
import org.elasticsearch.common.io.stream.StreamInput;
|
import org.elasticsearch.common.io.stream.StreamInput;
|
||||||
import org.elasticsearch.common.io.stream.StreamOutput;
|
import org.elasticsearch.common.io.stream.StreamOutput;
|
||||||
|
import org.elasticsearch.common.io.stream.Writeable;
|
||||||
import org.elasticsearch.common.settings.Settings;
|
import org.elasticsearch.common.settings.Settings;
|
||||||
import org.elasticsearch.common.xcontent.StatusToXContentObject;
|
import org.elasticsearch.common.xcontent.StatusToXContentObject;
|
||||||
import org.elasticsearch.common.xcontent.XContentBuilder;
|
import org.elasticsearch.common.xcontent.XContentBuilder;
|
||||||
import org.elasticsearch.rest.RestStatus;
|
import org.elasticsearch.rest.RestStatus;
|
||||||
import org.elasticsearch.tasks.CancellableTask;
|
|
||||||
import org.elasticsearch.tasks.Task;
|
|
||||||
import org.elasticsearch.tasks.TaskId;
|
|
||||||
import org.elasticsearch.threadpool.ThreadPool;
|
import org.elasticsearch.threadpool.ThreadPool;
|
||||||
import org.elasticsearch.transport.TransportService;
|
import org.elasticsearch.transport.TransportService;
|
||||||
import org.elasticsearch.xpack.ml.MlPlugin;
|
import org.elasticsearch.xpack.ml.MlPlugin;
|
||||||
import org.elasticsearch.xpack.ml.job.DataCounts;
|
import org.elasticsearch.xpack.ml.job.DataCounts;
|
||||||
import org.elasticsearch.xpack.ml.job.DataDescription;
|
import org.elasticsearch.xpack.ml.job.DataDescription;
|
||||||
import org.elasticsearch.xpack.ml.job.Job;
|
|
||||||
import org.elasticsearch.xpack.ml.job.manager.AutodetectProcessManager;
|
import org.elasticsearch.xpack.ml.job.manager.AutodetectProcessManager;
|
||||||
|
import org.elasticsearch.xpack.ml.job.manager.JobManager;
|
||||||
import org.elasticsearch.xpack.ml.job.process.autodetect.params.DataLoadParams;
|
import org.elasticsearch.xpack.ml.job.process.autodetect.params.DataLoadParams;
|
||||||
import org.elasticsearch.xpack.ml.job.process.autodetect.params.TimeRange;
|
import org.elasticsearch.xpack.ml.job.process.autodetect.params.TimeRange;
|
||||||
import org.elasticsearch.xpack.ml.utils.ExceptionsHelper;
|
|
||||||
|
|
||||||
import java.io.IOException;
|
import java.io.IOException;
|
||||||
import java.util.Objects;
|
import java.util.Objects;
|
||||||
|
@ -68,7 +63,7 @@ public class PostDataAction extends Action<PostDataAction.Request, PostDataActio
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
public static class Response extends ActionResponse implements StatusToXContentObject {
|
public static class Response extends BaseTasksResponse implements StatusToXContentObject, Writeable {
|
||||||
|
|
||||||
private DataCounts dataCounts;
|
private DataCounts dataCounts;
|
||||||
|
|
||||||
|
@ -80,6 +75,7 @@ public class PostDataAction extends Action<PostDataAction.Request, PostDataActio
|
||||||
}
|
}
|
||||||
|
|
||||||
public Response(DataCounts counts) {
|
public Response(DataCounts counts) {
|
||||||
|
super(null, null);
|
||||||
this.dataCounts = counts;
|
this.dataCounts = counts;
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@ -132,25 +128,12 @@ public class PostDataAction extends Action<PostDataAction.Request, PostDataActio
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
static class PostDataTask extends CancellableTask {
|
public static class Request extends TransportJobTaskAction.JobTaskRequest<Request> {
|
||||||
|
|
||||||
PostDataTask(long id, String type, String action, TaskId parentTaskId, String jobId) {
|
|
||||||
super(id, type, action, jobId + "_post_data", parentTaskId);
|
|
||||||
}
|
|
||||||
|
|
||||||
@Override
|
|
||||||
public boolean shouldCancelChildrenOnCancellation() {
|
|
||||||
return true;
|
|
||||||
}
|
|
||||||
}
|
|
||||||
|
|
||||||
public static class Request extends ActionRequest {
|
|
||||||
|
|
||||||
public static final ParseField IGNORE_DOWNTIME = new ParseField("ignore_downtime");
|
public static final ParseField IGNORE_DOWNTIME = new ParseField("ignore_downtime");
|
||||||
public static final ParseField RESET_START = new ParseField("reset_start");
|
public static final ParseField RESET_START = new ParseField("reset_start");
|
||||||
public static final ParseField RESET_END = new ParseField("reset_end");
|
public static final ParseField RESET_END = new ParseField("reset_end");
|
||||||
|
|
||||||
private String jobId;
|
|
||||||
private boolean ignoreDowntime = false;
|
private boolean ignoreDowntime = false;
|
||||||
private String resetStart = "";
|
private String resetStart = "";
|
||||||
private String resetEnd = "";
|
private String resetEnd = "";
|
||||||
|
@ -161,8 +144,7 @@ public class PostDataAction extends Action<PostDataAction.Request, PostDataActio
|
||||||
}
|
}
|
||||||
|
|
||||||
public Request(String jobId) {
|
public Request(String jobId) {
|
||||||
ExceptionsHelper.requireNonNull(jobId, Job.ID.getPreferredName());
|
super(jobId);
|
||||||
this.jobId = jobId;
|
|
||||||
}
|
}
|
||||||
|
|
||||||
public String getJobId() {
|
public String getJobId() {
|
||||||
|
@ -207,20 +189,9 @@ public class PostDataAction extends Action<PostDataAction.Request, PostDataActio
|
||||||
this.content = content;
|
this.content = content;
|
||||||
}
|
}
|
||||||
|
|
||||||
@Override
|
|
||||||
public Task createTask(long id, String type, String action, TaskId parentTaskId) {
|
|
||||||
return new PostDataTask(id, type, action, parentTaskId, jobId);
|
|
||||||
}
|
|
||||||
|
|
||||||
@Override
|
|
||||||
public ActionRequestValidationException validate() {
|
|
||||||
return null;
|
|
||||||
}
|
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
public void readFrom(StreamInput in) throws IOException {
|
public void readFrom(StreamInput in) throws IOException {
|
||||||
super.readFrom(in);
|
super.readFrom(in);
|
||||||
jobId = in.readString();
|
|
||||||
ignoreDowntime = in.readBoolean();
|
ignoreDowntime = in.readBoolean();
|
||||||
resetStart = in.readOptionalString();
|
resetStart = in.readOptionalString();
|
||||||
resetEnd = in.readOptionalString();
|
resetEnd = in.readOptionalString();
|
||||||
|
@ -231,7 +202,6 @@ public class PostDataAction extends Action<PostDataAction.Request, PostDataActio
|
||||||
@Override
|
@Override
|
||||||
public void writeTo(StreamOutput out) throws IOException {
|
public void writeTo(StreamOutput out) throws IOException {
|
||||||
super.writeTo(out);
|
super.writeTo(out);
|
||||||
out.writeString(jobId);
|
|
||||||
out.writeBoolean(ignoreDowntime);
|
out.writeBoolean(ignoreDowntime);
|
||||||
out.writeOptionalString(resetStart);
|
out.writeOptionalString(resetStart);
|
||||||
out.writeOptionalString(resetEnd);
|
out.writeOptionalString(resetEnd);
|
||||||
|
@ -265,28 +235,31 @@ public class PostDataAction extends Action<PostDataAction.Request, PostDataActio
|
||||||
}
|
}
|
||||||
|
|
||||||
|
|
||||||
public static class TransportAction extends HandledTransportAction<Request, Response> {
|
public static class TransportAction extends TransportJobTaskAction<InternalOpenJobAction.JobTask, Request, Response> {
|
||||||
|
|
||||||
private final AutodetectProcessManager processManager;
|
|
||||||
|
|
||||||
@Inject
|
@Inject
|
||||||
public TransportAction(Settings settings, TransportService transportService, ThreadPool threadPool, ActionFilters actionFilters,
|
public TransportAction(Settings settings, TransportService transportService, ThreadPool threadPool, ClusterService clusterService,
|
||||||
IndexNameExpressionResolver indexNameExpressionResolver, AutodetectProcessManager processManager) {
|
ActionFilters actionFilters, IndexNameExpressionResolver indexNameExpressionResolver,
|
||||||
super(settings, PostDataAction.NAME, false, threadPool, transportService, actionFilters,
|
JobManager jobManager, AutodetectProcessManager processManager) {
|
||||||
indexNameExpressionResolver, Request::new);
|
super(settings, PostDataAction.NAME, threadPool, clusterService, transportService, actionFilters, indexNameExpressionResolver,
|
||||||
this.processManager = processManager;
|
Request::new, Response::new, MlPlugin.THREAD_POOL_NAME, jobManager, processManager, Request::getJobId);
|
||||||
}
|
}
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
protected void doExecute(Task task, Request request, ActionListener<Response> listener) {
|
protected Response readTaskResponse(StreamInput in) throws IOException {
|
||||||
PostDataTask postDataTask = (PostDataTask) task;
|
Response response = new Response();
|
||||||
|
response.readFrom(in);
|
||||||
|
return response;
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
protected void taskOperation(Request request, InternalOpenJobAction.JobTask task, ActionListener<Response> listener) {
|
||||||
TimeRange timeRange = TimeRange.builder().startTime(request.getResetStart()).endTime(request.getResetEnd()).build();
|
TimeRange timeRange = TimeRange.builder().startTime(request.getResetStart()).endTime(request.getResetEnd()).build();
|
||||||
DataLoadParams params = new DataLoadParams(timeRange, request.isIgnoreDowntime(),
|
DataLoadParams params = new DataLoadParams(timeRange, request.isIgnoreDowntime(),
|
||||||
Optional.ofNullable(request.getDataDescription()));
|
Optional.ofNullable(request.getDataDescription()));
|
||||||
threadPool.executor(MlPlugin.THREAD_POOL_NAME).execute(() -> {
|
threadPool.executor(MlPlugin.THREAD_POOL_NAME).execute(() -> {
|
||||||
try {
|
try {
|
||||||
DataCounts dataCounts = processManager.processData(request.getJobId(), request.content.streamInput(), params,
|
DataCounts dataCounts = processManager.processData(request.getJobId(), request.content.streamInput(), params);
|
||||||
postDataTask::isCancelled);
|
|
||||||
listener.onResponse(new Response(dataCounts));
|
listener.onResponse(new Response(dataCounts));
|
||||||
} catch (Exception e) {
|
} catch (Exception e) {
|
||||||
listener.onFailure(e);
|
listener.onFailure(e);
|
||||||
|
@ -294,9 +267,5 @@ public class PostDataAction extends Action<PostDataAction.Request, PostDataActio
|
||||||
});
|
});
|
||||||
}
|
}
|
||||||
|
|
||||||
@Override
|
|
||||||
protected final void doExecute(Request request, ActionListener<Response> listener) {
|
|
||||||
throw new UnsupportedOperationException("the task parameter is required");
|
|
||||||
}
|
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
|
@ -0,0 +1,133 @@
|
||||||
|
/*
|
||||||
|
* Copyright Elasticsearch B.V. and/or licensed to Elasticsearch B.V. under one
|
||||||
|
* or more contributor license agreements. Licensed under the Elastic License;
|
||||||
|
* you may not use this file except in compliance with the Elastic License.
|
||||||
|
*/
|
||||||
|
package org.elasticsearch.xpack.ml.action;
|
||||||
|
|
||||||
|
import org.elasticsearch.ElasticsearchException;
|
||||||
|
import org.elasticsearch.ElasticsearchStatusException;
|
||||||
|
import org.elasticsearch.action.FailedNodeException;
|
||||||
|
import org.elasticsearch.action.TaskOperationFailure;
|
||||||
|
import org.elasticsearch.action.support.ActionFilters;
|
||||||
|
import org.elasticsearch.action.support.tasks.BaseTasksRequest;
|
||||||
|
import org.elasticsearch.action.support.tasks.BaseTasksResponse;
|
||||||
|
import org.elasticsearch.action.support.tasks.TransportTasksAction;
|
||||||
|
import org.elasticsearch.cluster.metadata.IndexNameExpressionResolver;
|
||||||
|
import org.elasticsearch.cluster.service.ClusterService;
|
||||||
|
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.settings.Settings;
|
||||||
|
import org.elasticsearch.rest.RestStatus;
|
||||||
|
import org.elasticsearch.tasks.Task;
|
||||||
|
import org.elasticsearch.threadpool.ThreadPool;
|
||||||
|
import org.elasticsearch.transport.TransportService;
|
||||||
|
import org.elasticsearch.xpack.ml.job.Job;
|
||||||
|
import org.elasticsearch.xpack.ml.job.JobStatus;
|
||||||
|
import org.elasticsearch.xpack.ml.job.manager.AutodetectProcessManager;
|
||||||
|
import org.elasticsearch.xpack.ml.job.manager.JobManager;
|
||||||
|
import org.elasticsearch.xpack.ml.job.metadata.Allocation;
|
||||||
|
import org.elasticsearch.xpack.ml.utils.ExceptionsHelper;
|
||||||
|
|
||||||
|
import java.io.IOException;
|
||||||
|
import java.util.List;
|
||||||
|
import java.util.function.Function;
|
||||||
|
import java.util.function.Supplier;
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Base class that redirects a request to a node where the job task is running.
|
||||||
|
*/
|
||||||
|
// TODO: Hacking around here with TransportTasksAction. Ideally we should have another base class in core that
|
||||||
|
// redirects to a single node only
|
||||||
|
public abstract class TransportJobTaskAction<OperationTask extends Task, Request extends TransportJobTaskAction.JobTaskRequest<Request>,
|
||||||
|
Response extends BaseTasksResponse & Writeable> extends TransportTasksAction<OperationTask, Request, Response, Response> {
|
||||||
|
|
||||||
|
protected final JobManager jobManager;
|
||||||
|
protected final AutodetectProcessManager processManager;
|
||||||
|
private final Function<Request, String> jobIdFromRequest;
|
||||||
|
|
||||||
|
TransportJobTaskAction(Settings settings, String actionName, ThreadPool threadPool, ClusterService clusterService,
|
||||||
|
TransportService transportService, ActionFilters actionFilters,
|
||||||
|
IndexNameExpressionResolver indexNameExpressionResolver, Supplier<Request> requestSupplier,
|
||||||
|
Supplier<Response> responseSupplier, String nodeExecutor, JobManager jobManager,
|
||||||
|
AutodetectProcessManager processManager, Function<Request, String> jobIdFromRequest) {
|
||||||
|
super(settings, actionName, threadPool, clusterService, transportService, actionFilters, indexNameExpressionResolver,
|
||||||
|
requestSupplier, responseSupplier, nodeExecutor);
|
||||||
|
this.jobManager = jobManager;
|
||||||
|
this.processManager = processManager;
|
||||||
|
this.jobIdFromRequest = jobIdFromRequest;
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
protected Response newResponse(Request request, List<Response> tasks, List<TaskOperationFailure> taskOperationFailures,
|
||||||
|
List<FailedNodeException> failedNodeExceptions) {
|
||||||
|
// no need to accumulate sub responses, since we only perform an operation on one task only
|
||||||
|
// not ideal, but throwing exceptions here works, because higher up the stack there is a try-catch block delegating to
|
||||||
|
// the actionlistener's onFailure
|
||||||
|
if (tasks.isEmpty()) {
|
||||||
|
if (taskOperationFailures.isEmpty() == false) {
|
||||||
|
throw new ElasticsearchException(taskOperationFailures.get(0).getCause());
|
||||||
|
} else if (failedNodeExceptions.isEmpty() == false) {
|
||||||
|
throw new ElasticsearchException(failedNodeExceptions.get(0).getCause());
|
||||||
|
} else {
|
||||||
|
// the same validation that exists in AutodetectProcessManager#processData(...) and flush(...) methods
|
||||||
|
// is required here too because if the job hasn't been opened yet then no task exist for it yet and then
|
||||||
|
// #taskOperation(...) method will not be invoked, returning an empty result to the client.
|
||||||
|
// This ensures that we return an understandable error:
|
||||||
|
String jobId = jobIdFromRequest.apply(request);
|
||||||
|
jobManager.getJobOrThrowIfUnknown(jobId);
|
||||||
|
Allocation allocation = jobManager.getJobAllocation(jobId);
|
||||||
|
if (allocation.getStatus() != JobStatus.OPENED) {
|
||||||
|
throw new ElasticsearchStatusException("job [" + jobId + "] status is [" + allocation.getStatus() +
|
||||||
|
"], but must be [" + JobStatus.OPENED + "] to perform requested action", RestStatus.CONFLICT);
|
||||||
|
} else {
|
||||||
|
throw new IllegalStateException("No errors or response");
|
||||||
|
}
|
||||||
|
}
|
||||||
|
} else {
|
||||||
|
if (tasks.size() > 1) {
|
||||||
|
throw new IllegalStateException("Expected one node level response, but got [" + tasks.size() + "]");
|
||||||
|
}
|
||||||
|
return tasks.get(0);
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
protected boolean accumulateExceptions() {
|
||||||
|
return false;
|
||||||
|
}
|
||||||
|
|
||||||
|
public static class JobTaskRequest<R extends JobTaskRequest<R>> extends BaseTasksRequest<R> {
|
||||||
|
|
||||||
|
String jobId;
|
||||||
|
|
||||||
|
JobTaskRequest() {
|
||||||
|
}
|
||||||
|
|
||||||
|
JobTaskRequest(String jobId) {
|
||||||
|
this.jobId = ExceptionsHelper.requireNonNull(jobId, Job.ID.getPreferredName());
|
||||||
|
}
|
||||||
|
|
||||||
|
public String getJobId() {
|
||||||
|
return jobId;
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public void readFrom(StreamInput in) throws IOException {
|
||||||
|
super.readFrom(in);
|
||||||
|
jobId = in.readString();
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public void writeTo(StreamOutput out) throws IOException {
|
||||||
|
super.writeTo(out);
|
||||||
|
out.writeString(jobId);
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public boolean match(Task task) {
|
||||||
|
return InternalOpenJobAction.JobTask.match(task, jobId);
|
||||||
|
}
|
||||||
|
}
|
||||||
|
}
|
|
@ -10,7 +10,7 @@ import org.elasticsearch.xpack.ml.job.process.autodetect.params.DataLoadParams;
|
||||||
import org.elasticsearch.xpack.ml.job.process.autodetect.params.InterimResultsParams;
|
import org.elasticsearch.xpack.ml.job.process.autodetect.params.InterimResultsParams;
|
||||||
|
|
||||||
import java.io.InputStream;
|
import java.io.InputStream;
|
||||||
import java.util.function.Supplier;
|
import java.util.function.Consumer;
|
||||||
|
|
||||||
public interface DataProcessor {
|
public interface DataProcessor {
|
||||||
|
|
||||||
|
@ -30,10 +30,9 @@ public interface DataProcessor {
|
||||||
* @param jobId the jobId
|
* @param jobId the jobId
|
||||||
* @param input Data input stream
|
* @param input Data input stream
|
||||||
* @param params Data processing parameters
|
* @param params Data processing parameters
|
||||||
* @param cancelled Whether the data processing has been cancelled
|
|
||||||
* @return Count of records, fields, bytes, etc written
|
* @return Count of records, fields, bytes, etc written
|
||||||
*/
|
*/
|
||||||
DataCounts processData(String jobId, InputStream input, DataLoadParams params, Supplier<Boolean> cancelled);
|
DataCounts processData(String jobId, InputStream input, DataLoadParams params);
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* Flush the running job, ensuring that the native process has had the
|
* Flush the running job, ensuring that the native process has had the
|
||||||
|
@ -46,7 +45,7 @@ public interface DataProcessor {
|
||||||
*/
|
*/
|
||||||
void flushJob(String jobId, InterimResultsParams interimResultsParams);
|
void flushJob(String jobId, InterimResultsParams interimResultsParams);
|
||||||
|
|
||||||
void openJob(String jobId, boolean ignoreDowntime);
|
void openJob(String jobId, boolean ignoreDowntime, Consumer<Exception> handler);
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* Stop the running job and mark it as finished.<br>
|
* Stop the running job and mark it as finished.<br>
|
||||||
|
|
|
@ -20,6 +20,7 @@ import org.elasticsearch.xpack.ml.job.DataCounts;
|
||||||
import org.elasticsearch.xpack.ml.job.Job;
|
import org.elasticsearch.xpack.ml.job.Job;
|
||||||
import org.elasticsearch.xpack.ml.job.JobStatus;
|
import org.elasticsearch.xpack.ml.job.JobStatus;
|
||||||
import org.elasticsearch.xpack.ml.job.ModelSizeStats;
|
import org.elasticsearch.xpack.ml.job.ModelSizeStats;
|
||||||
|
import org.elasticsearch.xpack.ml.job.ModelSnapshot;
|
||||||
import org.elasticsearch.xpack.ml.job.data.DataProcessor;
|
import org.elasticsearch.xpack.ml.job.data.DataProcessor;
|
||||||
import org.elasticsearch.xpack.ml.job.metadata.Allocation;
|
import org.elasticsearch.xpack.ml.job.metadata.Allocation;
|
||||||
import org.elasticsearch.xpack.ml.job.persistence.JobDataCountsPersister;
|
import org.elasticsearch.xpack.ml.job.persistence.JobDataCountsPersister;
|
||||||
|
@ -39,8 +40,10 @@ import org.elasticsearch.xpack.ml.job.process.normalizer.NormalizerFactory;
|
||||||
import org.elasticsearch.xpack.ml.job.process.normalizer.Renormalizer;
|
import org.elasticsearch.xpack.ml.job.process.normalizer.Renormalizer;
|
||||||
import org.elasticsearch.xpack.ml.job.process.normalizer.ScoresUpdater;
|
import org.elasticsearch.xpack.ml.job.process.normalizer.ScoresUpdater;
|
||||||
import org.elasticsearch.xpack.ml.job.process.normalizer.ShortCircuitingRenormalizer;
|
import org.elasticsearch.xpack.ml.job.process.normalizer.ShortCircuitingRenormalizer;
|
||||||
|
import org.elasticsearch.xpack.ml.job.quantiles.Quantiles;
|
||||||
import org.elasticsearch.xpack.ml.job.status.StatusReporter;
|
import org.elasticsearch.xpack.ml.job.status.StatusReporter;
|
||||||
import org.elasticsearch.xpack.ml.job.usage.UsageReporter;
|
import org.elasticsearch.xpack.ml.job.usage.UsageReporter;
|
||||||
|
import org.elasticsearch.xpack.ml.lists.ListDocument;
|
||||||
import org.elasticsearch.xpack.ml.utils.ExceptionsHelper;
|
import org.elasticsearch.xpack.ml.utils.ExceptionsHelper;
|
||||||
|
|
||||||
import java.io.IOException;
|
import java.io.IOException;
|
||||||
|
@ -49,13 +52,14 @@ import java.time.Duration;
|
||||||
import java.time.ZonedDateTime;
|
import java.time.ZonedDateTime;
|
||||||
import java.util.Locale;
|
import java.util.Locale;
|
||||||
import java.util.Optional;
|
import java.util.Optional;
|
||||||
|
import java.util.Set;
|
||||||
import java.util.concurrent.ConcurrentHashMap;
|
import java.util.concurrent.ConcurrentHashMap;
|
||||||
import java.util.concurrent.ConcurrentMap;
|
import java.util.concurrent.ConcurrentMap;
|
||||||
import java.util.concurrent.CountDownLatch;
|
import java.util.concurrent.CountDownLatch;
|
||||||
import java.util.concurrent.ExecutorService;
|
import java.util.concurrent.ExecutorService;
|
||||||
import java.util.concurrent.TimeoutException;
|
import java.util.concurrent.TimeoutException;
|
||||||
import java.util.concurrent.atomic.AtomicReference;
|
import java.util.concurrent.atomic.AtomicReference;
|
||||||
import java.util.function.Supplier;
|
import java.util.function.Consumer;
|
||||||
|
|
||||||
public class AutodetectProcessManager extends AbstractComponent implements DataProcessor {
|
public class AutodetectProcessManager extends AbstractComponent implements DataProcessor {
|
||||||
|
|
||||||
|
@ -107,7 +111,7 @@ public class AutodetectProcessManager extends AbstractComponent implements DataP
|
||||||
}
|
}
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
public DataCounts processData(String jobId, InputStream input, DataLoadParams params, Supplier<Boolean> cancelled) {
|
public DataCounts processData(String jobId, InputStream input, DataLoadParams params) {
|
||||||
Allocation allocation = jobManager.getJobAllocation(jobId);
|
Allocation allocation = jobManager.getJobAllocation(jobId);
|
||||||
if (allocation.getStatus() != JobStatus.OPENED) {
|
if (allocation.getStatus() != JobStatus.OPENED) {
|
||||||
throw new IllegalArgumentException("job [" + jobId + "] status is [" + allocation.getStatus() + "], but must be ["
|
throw new IllegalArgumentException("job [" + jobId + "] status is [" + allocation.getStatus() + "], but must be ["
|
||||||
|
@ -119,7 +123,7 @@ public class AutodetectProcessManager extends AbstractComponent implements DataP
|
||||||
throw new IllegalStateException("job [" + jobId + "] with status [" + allocation.getStatus() + "] hasn't been started");
|
throw new IllegalStateException("job [" + jobId + "] with status [" + allocation.getStatus() + "] hasn't been started");
|
||||||
}
|
}
|
||||||
try {
|
try {
|
||||||
return communicator.writeToJob(input, params, cancelled);
|
return communicator.writeToJob(input, params);
|
||||||
// TODO check for errors from autodetect
|
// TODO check for errors from autodetect
|
||||||
} catch (IOException e) {
|
} catch (IOException e) {
|
||||||
String msg = String.format(Locale.ROOT, "Exception writing to process for job %s", jobId);
|
String msg = String.format(Locale.ROOT, "Exception writing to process for job %s", jobId);
|
||||||
|
@ -163,9 +167,10 @@ public class AutodetectProcessManager extends AbstractComponent implements DataP
|
||||||
}
|
}
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
public void openJob(String jobId, boolean ignoreDowntime) {
|
public void openJob(String jobId, boolean ignoreDowntime, Consumer<Exception> handler) {
|
||||||
|
gatherRequiredInformation(jobId, (modelSnapshot, quantiles, lists) -> {
|
||||||
autoDetectCommunicatorByJob.computeIfAbsent(jobId, id -> {
|
autoDetectCommunicatorByJob.computeIfAbsent(jobId, id -> {
|
||||||
AutodetectCommunicator communicator = create(id, ignoreDowntime);
|
AutodetectCommunicator communicator = create(id, modelSnapshot, quantiles, lists, ignoreDowntime, handler);
|
||||||
try {
|
try {
|
||||||
communicator.writeJobInputHeader();
|
communicator.writeJobInputHeader();
|
||||||
} catch (IOException ioe) {
|
} catch (IOException ioe) {
|
||||||
|
@ -176,9 +181,28 @@ public class AutodetectProcessManager extends AbstractComponent implements DataP
|
||||||
setJobStatus(jobId, JobStatus.OPENED);
|
setJobStatus(jobId, JobStatus.OPENED);
|
||||||
return communicator;
|
return communicator;
|
||||||
});
|
});
|
||||||
|
}, handler);
|
||||||
}
|
}
|
||||||
|
|
||||||
AutodetectCommunicator create(String jobId, boolean ignoreDowntime) {
|
void gatherRequiredInformation(String jobId, TriConsumer handler, Consumer<Exception> errorHandler) {
|
||||||
|
Job job = jobManager.getJobOrThrowIfUnknown(jobId);
|
||||||
|
jobProvider.modelSnapshots(jobId, 0, 1, page -> {
|
||||||
|
ModelSnapshot modelSnapshot = page.results().isEmpty() ? null : page.results().get(1);
|
||||||
|
jobProvider.getQuantiles(jobId, quantiles -> {
|
||||||
|
String[] ids = job.getAnalysisConfig().extractReferencedLists().toArray(new String[0]);
|
||||||
|
jobProvider.getLists(listDocument -> handler.accept(modelSnapshot, quantiles, listDocument), errorHandler, ids);
|
||||||
|
}, errorHandler);
|
||||||
|
}, errorHandler);
|
||||||
|
}
|
||||||
|
|
||||||
|
interface TriConsumer {
|
||||||
|
|
||||||
|
void accept(ModelSnapshot modelSnapshot, Quantiles quantiles, Set<ListDocument> lists);
|
||||||
|
|
||||||
|
}
|
||||||
|
|
||||||
|
AutodetectCommunicator create(String jobId, ModelSnapshot modelSnapshot, Quantiles quantiles, Set<ListDocument> lists,
|
||||||
|
boolean ignoreDowntime, Consumer<Exception> handler) {
|
||||||
if (autoDetectCommunicatorByJob.size() == maxAllowedRunningJobs) {
|
if (autoDetectCommunicatorByJob.size() == maxAllowedRunningJobs) {
|
||||||
throw new ElasticsearchStatusException("max running job capacity [" + maxAllowedRunningJobs + "] reached",
|
throw new ElasticsearchStatusException("max running job capacity [" + maxAllowedRunningJobs + "] reached",
|
||||||
RestStatus.CONFLICT);
|
RestStatus.CONFLICT);
|
||||||
|
@ -200,8 +224,9 @@ public class AutodetectProcessManager extends AbstractComponent implements DataP
|
||||||
|
|
||||||
AutodetectProcess process = null;
|
AutodetectProcess process = null;
|
||||||
try {
|
try {
|
||||||
process = autodetectProcessFactory.createAutodetectProcess(job, ignoreDowntime, executorService);
|
process = autodetectProcessFactory.createAutodetectProcess(job, modelSnapshot, quantiles, lists,
|
||||||
return new AutodetectCommunicator(executorService, job, process, statusReporter, processor, stateProcessor);
|
ignoreDowntime, executorService);
|
||||||
|
return new AutodetectCommunicator(executorService, job, process, statusReporter, processor, stateProcessor, handler);
|
||||||
} catch (Exception e) {
|
} catch (Exception e) {
|
||||||
try {
|
try {
|
||||||
IOUtils.close(process);
|
IOUtils.close(process);
|
||||||
|
@ -274,7 +299,11 @@ public class AutodetectProcessManager extends AbstractComponent implements DataP
|
||||||
client.execute(UpdateJobStatusAction.INSTANCE, request, new ActionListener<UpdateJobStatusAction.Response>() {
|
client.execute(UpdateJobStatusAction.INSTANCE, request, new ActionListener<UpdateJobStatusAction.Response>() {
|
||||||
@Override
|
@Override
|
||||||
public void onResponse(UpdateJobStatusAction.Response response) {
|
public void onResponse(UpdateJobStatusAction.Response response) {
|
||||||
|
if (response.isAcknowledged()) {
|
||||||
logger.info("Successfully set job status to [{}] for job [{}]", status, jobId);
|
logger.info("Successfully set job status to [{}] for job [{}]", status, jobId);
|
||||||
|
} else {
|
||||||
|
logger.info("Changing job status to [{}] for job [{}] wasn't acked", status, jobId);
|
||||||
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
|
@ -284,6 +313,11 @@ public class AutodetectProcessManager extends AbstractComponent implements DataP
|
||||||
});
|
});
|
||||||
}
|
}
|
||||||
|
|
||||||
|
public void setJobStatus(String jobId, JobStatus status, Consumer<Void> handler, Consumer<Exception> errorHandler) {
|
||||||
|
UpdateJobStatusAction.Request request = new UpdateJobStatusAction.Request(jobId, status);
|
||||||
|
client.execute(UpdateJobStatusAction.INSTANCE, request, ActionListener.wrap(r -> handler.accept(null), errorHandler));
|
||||||
|
}
|
||||||
|
|
||||||
public Optional<ModelSizeStats> getModelSizeStats(String jobId) {
|
public Optional<ModelSizeStats> getModelSizeStats(String jobId) {
|
||||||
AutodetectCommunicator communicator = autoDetectCommunicatorByJob.get(jobId);
|
AutodetectCommunicator communicator = autoDetectCommunicatorByJob.get(jobId);
|
||||||
if (communicator == null) {
|
if (communicator == null) {
|
||||||
|
|
|
@ -5,7 +5,6 @@
|
||||||
*/
|
*/
|
||||||
package org.elasticsearch.xpack.ml.job.manager;
|
package org.elasticsearch.xpack.ml.job.manager;
|
||||||
|
|
||||||
import org.apache.logging.log4j.Logger;
|
|
||||||
import org.elasticsearch.ResourceAlreadyExistsException;
|
import org.elasticsearch.ResourceAlreadyExistsException;
|
||||||
import org.elasticsearch.ResourceNotFoundException;
|
import org.elasticsearch.ResourceNotFoundException;
|
||||||
import org.elasticsearch.action.ActionListener;
|
import org.elasticsearch.action.ActionListener;
|
||||||
|
@ -17,11 +16,9 @@ import org.elasticsearch.cluster.metadata.MetaData;
|
||||||
import org.elasticsearch.cluster.service.ClusterService;
|
import org.elasticsearch.cluster.service.ClusterService;
|
||||||
import org.elasticsearch.common.CheckedConsumer;
|
import org.elasticsearch.common.CheckedConsumer;
|
||||||
import org.elasticsearch.common.component.AbstractComponent;
|
import org.elasticsearch.common.component.AbstractComponent;
|
||||||
import org.elasticsearch.common.logging.Loggers;
|
|
||||||
import org.elasticsearch.common.settings.Settings;
|
import org.elasticsearch.common.settings.Settings;
|
||||||
import org.elasticsearch.index.IndexNotFoundException;
|
import org.elasticsearch.index.IndexNotFoundException;
|
||||||
import org.elasticsearch.xpack.ml.action.DeleteJobAction;
|
import org.elasticsearch.xpack.ml.action.DeleteJobAction;
|
||||||
import org.elasticsearch.xpack.ml.action.OpenJobAction;
|
|
||||||
import org.elasticsearch.xpack.ml.action.PutJobAction;
|
import org.elasticsearch.xpack.ml.action.PutJobAction;
|
||||||
import org.elasticsearch.xpack.ml.action.RevertModelSnapshotAction;
|
import org.elasticsearch.xpack.ml.action.RevertModelSnapshotAction;
|
||||||
import org.elasticsearch.xpack.ml.action.UpdateJobStatusAction;
|
import org.elasticsearch.xpack.ml.action.UpdateJobStatusAction;
|
||||||
|
@ -61,14 +58,13 @@ import java.util.stream.Collectors;
|
||||||
*/
|
*/
|
||||||
public class JobManager extends AbstractComponent {
|
public class JobManager extends AbstractComponent {
|
||||||
|
|
||||||
private static final Logger LOGGER = Loggers.getLogger(JobManager.class);
|
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* Field name in which to store the API version in the usage info
|
* Field name in which to store the API version in the usage info
|
||||||
*/
|
*/
|
||||||
public static final String APP_VER_FIELDNAME = "appVer";
|
public static final String APP_VER_FIELDNAME = "appVer";
|
||||||
|
|
||||||
public static final String DEFAULT_RECORD_SORT_FIELD = AnomalyRecord.PROBABILITY.getPreferredName();
|
public static final String DEFAULT_RECORD_SORT_FIELD = AnomalyRecord.PROBABILITY.getPreferredName();
|
||||||
|
|
||||||
private final JobProvider jobProvider;
|
private final JobProvider jobProvider;
|
||||||
private final ClusterService clusterService;
|
private final ClusterService clusterService;
|
||||||
private final JobResultsPersister jobResultsPersister;
|
private final JobResultsPersister jobResultsPersister;
|
||||||
|
@ -217,11 +213,9 @@ public class JobManager extends AbstractComponent {
|
||||||
|
|
||||||
|
|
||||||
public void deleteJob(Client client, DeleteJobAction.Request request, ActionListener<DeleteJobAction.Response> actionListener) {
|
public void deleteJob(Client client, DeleteJobAction.Request request, ActionListener<DeleteJobAction.Response> actionListener) {
|
||||||
|
|
||||||
String jobId = request.getJobId();
|
String jobId = request.getJobId();
|
||||||
String indexName = AnomalyDetectorsIndex.jobResultsIndexName(jobId);
|
String indexName = AnomalyDetectorsIndex.jobResultsIndexName(jobId);
|
||||||
LOGGER.debug("Deleting job '" + jobId + "'");
|
logger.debug("Deleting job '" + jobId + "'");
|
||||||
|
|
||||||
|
|
||||||
// Step 3. Listen for the Cluster State status change
|
// Step 3. Listen for the Cluster State status change
|
||||||
// Chain acknowledged status onto original actionListener
|
// Chain acknowledged status onto original actionListener
|
||||||
|
@ -372,28 +366,6 @@ public class JobManager extends AbstractComponent {
|
||||||
});
|
});
|
||||||
}
|
}
|
||||||
|
|
||||||
public void openJob(OpenJobAction.Request request, ActionListener<OpenJobAction.Response> actionListener) {
|
|
||||||
clusterService.submitStateUpdateTask("open-job-" + request.getJobId(),
|
|
||||||
new AckedClusterStateUpdateTask<OpenJobAction.Response>(request, actionListener) {
|
|
||||||
@Override
|
|
||||||
public ClusterState execute(ClusterState currentState) throws Exception {
|
|
||||||
MlMetadata.Builder builder = new MlMetadata.Builder(currentState.metaData().custom(MlMetadata.TYPE));
|
|
||||||
builder.updateStatus(request.getJobId(), JobStatus.OPENING, null);
|
|
||||||
if (request.isIgnoreDowntime()) {
|
|
||||||
builder.setIgnoreDowntime(request.getJobId());
|
|
||||||
}
|
|
||||||
return ClusterState.builder(currentState)
|
|
||||||
.metaData(MetaData.builder(currentState.metaData()).putCustom(MlMetadata.TYPE, builder.build()))
|
|
||||||
.build();
|
|
||||||
}
|
|
||||||
|
|
||||||
@Override
|
|
||||||
protected OpenJobAction.Response newResponse(boolean acknowledged) {
|
|
||||||
return new OpenJobAction.Response(acknowledged);
|
|
||||||
}
|
|
||||||
});
|
|
||||||
}
|
|
||||||
|
|
||||||
public void setJobStatus(UpdateJobStatusAction.Request request, ActionListener<UpdateJobStatusAction.Response> actionListener) {
|
public void setJobStatus(UpdateJobStatusAction.Request request, ActionListener<UpdateJobStatusAction.Response> actionListener) {
|
||||||
clusterService.submitStateUpdateTask("set-job-status-" + request.getStatus() + "-" + request.getJobId(),
|
clusterService.submitStateUpdateTask("set-job-status-" + request.getStatus() + "-" + request.getJobId(),
|
||||||
new AckedClusterStateUpdateTask<UpdateJobStatusAction.Response>(request, actionListener) {
|
new AckedClusterStateUpdateTask<UpdateJobStatusAction.Response>(request, actionListener) {
|
||||||
|
|
|
@ -1,94 +0,0 @@
|
||||||
/*
|
|
||||||
* Copyright Elasticsearch B.V. and/or licensed to Elasticsearch B.V. under one
|
|
||||||
* or more contributor license agreements. Licensed under the Elastic License;
|
|
||||||
* you may not use this file except in compliance with the Elastic License.
|
|
||||||
*/
|
|
||||||
package org.elasticsearch.xpack.ml.job.metadata;
|
|
||||||
|
|
||||||
import org.elasticsearch.cluster.ClusterChangedEvent;
|
|
||||||
import org.elasticsearch.cluster.ClusterState;
|
|
||||||
import org.elasticsearch.cluster.ClusterStateListener;
|
|
||||||
import org.elasticsearch.cluster.ClusterStateUpdateTask;
|
|
||||||
import org.elasticsearch.cluster.metadata.MetaData;
|
|
||||||
import org.elasticsearch.cluster.node.DiscoveryNode;
|
|
||||||
import org.elasticsearch.cluster.node.DiscoveryNodes;
|
|
||||||
import org.elasticsearch.cluster.service.ClusterService;
|
|
||||||
import org.elasticsearch.common.component.AbstractComponent;
|
|
||||||
import org.elasticsearch.common.settings.Settings;
|
|
||||||
import org.elasticsearch.threadpool.ThreadPool;
|
|
||||||
|
|
||||||
/**
|
|
||||||
* Runs only on the elected master node and decides to what nodes jobs should be allocated
|
|
||||||
*/
|
|
||||||
public class JobAllocator extends AbstractComponent implements ClusterStateListener {
|
|
||||||
|
|
||||||
private final ThreadPool threadPool;
|
|
||||||
private final ClusterService clusterService;
|
|
||||||
|
|
||||||
public JobAllocator(Settings settings, ClusterService clusterService, ThreadPool threadPool) {
|
|
||||||
super(settings);
|
|
||||||
this.threadPool = threadPool;
|
|
||||||
this.clusterService = clusterService;
|
|
||||||
clusterService.addListener(this);
|
|
||||||
}
|
|
||||||
|
|
||||||
ClusterState assignJobsToNodes(ClusterState current) {
|
|
||||||
if (shouldAllocate(current) == false) {
|
|
||||||
// returning same instance, so no cluster state update is performed
|
|
||||||
return current;
|
|
||||||
}
|
|
||||||
|
|
||||||
DiscoveryNodes nodes = current.getNodes();
|
|
||||||
if (nodes.getSize() != 1) {
|
|
||||||
throw new IllegalStateException("Current ml doesn't support multiple nodes");
|
|
||||||
}
|
|
||||||
|
|
||||||
// NORELEASE: Assumes ml always runs on a single node:
|
|
||||||
MlMetadata mlMetadata = current.getMetaData().custom(MlMetadata.TYPE);
|
|
||||||
MlMetadata.Builder builder = new MlMetadata.Builder(mlMetadata);
|
|
||||||
DiscoveryNode mlNode = nodes.getMasterNode(); // ml is now always master node
|
|
||||||
|
|
||||||
for (String jobId : mlMetadata.getAllocations().keySet()) {
|
|
||||||
builder.assignToNode(jobId, mlNode.getId());
|
|
||||||
}
|
|
||||||
|
|
||||||
return ClusterState.builder(current)
|
|
||||||
.metaData(MetaData.builder(current.metaData()).putCustom(MlMetadata.TYPE, builder.build()))
|
|
||||||
.build();
|
|
||||||
}
|
|
||||||
|
|
||||||
boolean shouldAllocate(ClusterState current) {
|
|
||||||
MlMetadata mlMetadata = current.getMetaData().custom(MlMetadata.TYPE);
|
|
||||||
if (mlMetadata == null) {
|
|
||||||
return false;
|
|
||||||
}
|
|
||||||
|
|
||||||
for (Allocation allocation : mlMetadata.getAllocations().values()) {
|
|
||||||
if (allocation.getNodeId() == null) {
|
|
||||||
return true;
|
|
||||||
}
|
|
||||||
}
|
|
||||||
return false;
|
|
||||||
}
|
|
||||||
|
|
||||||
@Override
|
|
||||||
public void clusterChanged(ClusterChangedEvent event) {
|
|
||||||
if (event.localNodeMaster()) {
|
|
||||||
if (shouldAllocate(event.state())) {
|
|
||||||
threadPool.executor(ThreadPool.Names.GENERIC).execute(() -> {
|
|
||||||
clusterService.submitStateUpdateTask("allocate_jobs", new ClusterStateUpdateTask() {
|
|
||||||
@Override
|
|
||||||
public ClusterState execute(ClusterState currentState) throws Exception {
|
|
||||||
return assignJobsToNodes(currentState);
|
|
||||||
}
|
|
||||||
|
|
||||||
@Override
|
|
||||||
public void onFailure(String source, Exception e) {
|
|
||||||
logger.error("failed to allocate jobs", e);
|
|
||||||
}
|
|
||||||
});
|
|
||||||
});
|
|
||||||
}
|
|
||||||
}
|
|
||||||
}
|
|
||||||
}
|
|
|
@ -1,145 +0,0 @@
|
||||||
/*
|
|
||||||
* Copyright Elasticsearch B.V. and/or licensed to Elasticsearch B.V. under one
|
|
||||||
* or more contributor license agreements. Licensed under the Elastic License;
|
|
||||||
* you may not use this file except in compliance with the Elastic License.
|
|
||||||
*/
|
|
||||||
package org.elasticsearch.xpack.ml.job.metadata;
|
|
||||||
|
|
||||||
import org.elasticsearch.action.ActionListener;
|
|
||||||
import org.elasticsearch.client.Client;
|
|
||||||
import org.elasticsearch.cluster.ClusterChangedEvent;
|
|
||||||
import org.elasticsearch.cluster.ClusterStateListener;
|
|
||||||
import org.elasticsearch.cluster.node.DiscoveryNode;
|
|
||||||
import org.elasticsearch.cluster.service.ClusterService;
|
|
||||||
import org.elasticsearch.common.component.AbstractComponent;
|
|
||||||
import org.elasticsearch.common.settings.Settings;
|
|
||||||
import org.elasticsearch.xpack.ml.action.UpdateJobStatusAction;
|
|
||||||
import org.elasticsearch.xpack.ml.job.JobStatus;
|
|
||||||
import org.elasticsearch.xpack.ml.job.data.DataProcessor;
|
|
||||||
|
|
||||||
import java.util.HashSet;
|
|
||||||
import java.util.Objects;
|
|
||||||
import java.util.Set;
|
|
||||||
import java.util.concurrent.Executor;
|
|
||||||
|
|
||||||
public class JobLifeCycleService extends AbstractComponent implements ClusterStateListener {
|
|
||||||
|
|
||||||
private final Client client;
|
|
||||||
private final DataProcessor dataProcessor;
|
|
||||||
private final Executor executor;
|
|
||||||
|
|
||||||
volatile boolean stopped = false;
|
|
||||||
volatile Set<String> localAssignedJobs = new HashSet<>();
|
|
||||||
|
|
||||||
public JobLifeCycleService(Settings settings, Client client, ClusterService clusterService, DataProcessor dataProcessor,
|
|
||||||
Executor executor) {
|
|
||||||
super(settings);
|
|
||||||
clusterService.addListener(this);
|
|
||||||
this.client = Objects.requireNonNull(client);
|
|
||||||
this.dataProcessor = Objects.requireNonNull(dataProcessor);
|
|
||||||
this.executor = Objects.requireNonNull(executor);
|
|
||||||
}
|
|
||||||
|
|
||||||
@Override
|
|
||||||
public void clusterChanged(ClusterChangedEvent event) {
|
|
||||||
MlMetadata mlMetadata = event.state().getMetaData().custom(MlMetadata.TYPE);
|
|
||||||
if (mlMetadata == null) {
|
|
||||||
logger.debug("Ml metadata not installed");
|
|
||||||
return;
|
|
||||||
}
|
|
||||||
|
|
||||||
if (stopped) {
|
|
||||||
logger.debug("no cluster state changes will be processed as the node has been stopped");
|
|
||||||
return;
|
|
||||||
}
|
|
||||||
|
|
||||||
// Single volatile read:
|
|
||||||
Set<String> localAssignedJobs = this.localAssignedJobs;
|
|
||||||
|
|
||||||
DiscoveryNode localNode = event.state().nodes().getLocalNode();
|
|
||||||
for (Allocation allocation : mlMetadata.getAllocations().values()) {
|
|
||||||
if (localNode.getId().equals(allocation.getNodeId())) {
|
|
||||||
if (localAssignedJobs.contains(allocation.getJobId()) == false) {
|
|
||||||
if (allocation.getStatus() == JobStatus.OPENING) {
|
|
||||||
startJob(allocation);
|
|
||||||
}
|
|
||||||
}
|
|
||||||
}
|
|
||||||
}
|
|
||||||
|
|
||||||
for (String localAllocatedJob : localAssignedJobs) {
|
|
||||||
Allocation allocation = mlMetadata.getAllocations().get(localAllocatedJob);
|
|
||||||
if (allocation != null) {
|
|
||||||
if (localNode.getId().equals(allocation.getNodeId()) && allocation.getStatus() == JobStatus.CLOSING) {
|
|
||||||
stopJob(localAllocatedJob);
|
|
||||||
}
|
|
||||||
} else {
|
|
||||||
stopJob(localAllocatedJob);
|
|
||||||
}
|
|
||||||
}
|
|
||||||
}
|
|
||||||
|
|
||||||
void startJob(Allocation allocation) {
|
|
||||||
logger.info("Starting job [" + allocation.getJobId() + "]");
|
|
||||||
executor.execute(() -> {
|
|
||||||
try {
|
|
||||||
dataProcessor.openJob(allocation.getJobId(), allocation.isIgnoreDowntime());
|
|
||||||
} catch (Exception e) {
|
|
||||||
logger.error("Failed to open job [" + allocation.getJobId() + "]", e);
|
|
||||||
updateJobStatus(allocation.getJobId(), JobStatus.FAILED, "failed to open, " + e.getMessage());
|
|
||||||
}
|
|
||||||
});
|
|
||||||
|
|
||||||
// update which jobs are now allocated locally
|
|
||||||
Set<String> newSet = new HashSet<>(localAssignedJobs);
|
|
||||||
newSet.add(allocation.getJobId());
|
|
||||||
localAssignedJobs = newSet;
|
|
||||||
}
|
|
||||||
|
|
||||||
void stopJob(String jobId) {
|
|
||||||
logger.info("Stopping job [" + jobId + "]");
|
|
||||||
executor.execute(() -> {
|
|
||||||
try {
|
|
||||||
dataProcessor.closeJob(jobId);
|
|
||||||
} catch (Exception e) {
|
|
||||||
logger.error("Failed to close job [" + jobId + "]", e);
|
|
||||||
updateJobStatus(jobId, JobStatus.FAILED, "failed to close, " + e.getMessage());
|
|
||||||
}
|
|
||||||
});
|
|
||||||
|
|
||||||
// update which jobs are now allocated locally
|
|
||||||
Set<String> newSet = new HashSet<>(localAssignedJobs);
|
|
||||||
newSet.remove(jobId);
|
|
||||||
localAssignedJobs = newSet;
|
|
||||||
}
|
|
||||||
|
|
||||||
public void stop() {
|
|
||||||
stopped = true;
|
|
||||||
Set<String> jobsToStop = this.localAssignedJobs;
|
|
||||||
for (String jobId : jobsToStop) {
|
|
||||||
try {
|
|
||||||
dataProcessor.closeJob(jobId);
|
|
||||||
} catch (Exception e) {
|
|
||||||
// in case of failure log it and continue with closing next job.
|
|
||||||
logger.error("Failed to close job [" + jobId + "] while stopping node", e);
|
|
||||||
}
|
|
||||||
|
|
||||||
}
|
|
||||||
}
|
|
||||||
|
|
||||||
private void updateJobStatus(String jobId, JobStatus status, String reason) {
|
|
||||||
UpdateJobStatusAction.Request request = new UpdateJobStatusAction.Request(jobId, status);
|
|
||||||
request.setReason(reason);
|
|
||||||
client.execute(UpdateJobStatusAction.INSTANCE, request, new ActionListener<UpdateJobStatusAction.Response>() {
|
|
||||||
@Override
|
|
||||||
public void onResponse(UpdateJobStatusAction.Response response) {
|
|
||||||
logger.info("Successfully set job status to [{}] for job [{}]", status, jobId);
|
|
||||||
}
|
|
||||||
|
|
||||||
@Override
|
|
||||||
public void onFailure(Exception e) {
|
|
||||||
logger.error("Could not set job status to [" + status + "] for job [" + jobId +"]", e);
|
|
||||||
}
|
|
||||||
});
|
|
||||||
}
|
|
||||||
}
|
|
|
@ -8,8 +8,10 @@ package org.elasticsearch.xpack.ml.job.metadata;
|
||||||
import org.elasticsearch.ElasticsearchStatusException;
|
import org.elasticsearch.ElasticsearchStatusException;
|
||||||
import org.elasticsearch.ResourceAlreadyExistsException;
|
import org.elasticsearch.ResourceAlreadyExistsException;
|
||||||
import org.elasticsearch.ResourceNotFoundException;
|
import org.elasticsearch.ResourceNotFoundException;
|
||||||
|
import org.elasticsearch.cluster.AbstractDiffable;
|
||||||
import org.elasticsearch.cluster.Diff;
|
import org.elasticsearch.cluster.Diff;
|
||||||
import org.elasticsearch.cluster.DiffableUtils;
|
import org.elasticsearch.cluster.DiffableUtils;
|
||||||
|
import org.elasticsearch.cluster.NamedDiff;
|
||||||
import org.elasticsearch.cluster.metadata.MetaData;
|
import org.elasticsearch.cluster.metadata.MetaData;
|
||||||
import org.elasticsearch.common.Nullable;
|
import org.elasticsearch.common.Nullable;
|
||||||
import org.elasticsearch.common.ParseField;
|
import org.elasticsearch.common.ParseField;
|
||||||
|
@ -19,6 +21,7 @@ import org.elasticsearch.common.io.stream.Writeable;
|
||||||
import org.elasticsearch.common.xcontent.ObjectParser;
|
import org.elasticsearch.common.xcontent.ObjectParser;
|
||||||
import org.elasticsearch.common.xcontent.ToXContent;
|
import org.elasticsearch.common.xcontent.ToXContent;
|
||||||
import org.elasticsearch.common.xcontent.XContentBuilder;
|
import org.elasticsearch.common.xcontent.XContentBuilder;
|
||||||
|
import org.elasticsearch.common.xcontent.XContentFactory;
|
||||||
import org.elasticsearch.rest.RestStatus;
|
import org.elasticsearch.rest.RestStatus;
|
||||||
import org.elasticsearch.xpack.ml.job.Job;
|
import org.elasticsearch.xpack.ml.job.Job;
|
||||||
import org.elasticsearch.xpack.ml.job.JobStatus;
|
import org.elasticsearch.xpack.ml.job.JobStatus;
|
||||||
|
@ -156,7 +159,7 @@ public class MlMetadata implements MetaData.Custom {
|
||||||
builder.endArray();
|
builder.endArray();
|
||||||
}
|
}
|
||||||
|
|
||||||
static class MlMetadataDiff implements Diff<MetaData.Custom> {
|
public static class MlMetadataDiff implements NamedDiff<MetaData.Custom> {
|
||||||
|
|
||||||
final Diff<Map<String, Job>> jobs;
|
final Diff<Map<String, Job>> jobs;
|
||||||
final Diff<Map<String, Allocation>> allocations;
|
final Diff<Map<String, Allocation>> allocations;
|
||||||
|
@ -168,6 +171,15 @@ public class MlMetadata implements MetaData.Custom {
|
||||||
this.datafeeds = DiffableUtils.diff(before.datafeeds, after.datafeeds, DiffableUtils.getStringKeySerializer());
|
this.datafeeds = DiffableUtils.diff(before.datafeeds, after.datafeeds, DiffableUtils.getStringKeySerializer());
|
||||||
}
|
}
|
||||||
|
|
||||||
|
public MlMetadataDiff(StreamInput in) throws IOException {
|
||||||
|
this.jobs = DiffableUtils.readJdkMapDiff(in, DiffableUtils.getStringKeySerializer(), Job::new,
|
||||||
|
MlMetadataDiff::readJobDiffFrom);
|
||||||
|
this.allocations = DiffableUtils.readJdkMapDiff(in, DiffableUtils.getStringKeySerializer(), Allocation::new,
|
||||||
|
MlMetadataDiff::readAllocationDiffFrom);
|
||||||
|
this.datafeeds = DiffableUtils.readJdkMapDiff(in, DiffableUtils.getStringKeySerializer(), Datafeed::new,
|
||||||
|
MlMetadataDiff::readSchedulerDiffFrom);
|
||||||
|
}
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
public MetaData.Custom apply(MetaData.Custom part) {
|
public MetaData.Custom apply(MetaData.Custom part) {
|
||||||
TreeMap<String, Job> newJobs = new TreeMap<>(jobs.apply(((MlMetadata) part).jobs));
|
TreeMap<String, Job> newJobs = new TreeMap<>(jobs.apply(((MlMetadata) part).jobs));
|
||||||
|
@ -182,6 +194,23 @@ public class MlMetadata implements MetaData.Custom {
|
||||||
allocations.writeTo(out);
|
allocations.writeTo(out);
|
||||||
datafeeds.writeTo(out);
|
datafeeds.writeTo(out);
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public String getWriteableName() {
|
||||||
|
return TYPE;
|
||||||
|
}
|
||||||
|
|
||||||
|
static Diff<Job> readJobDiffFrom(StreamInput in) throws IOException {
|
||||||
|
return AbstractDiffable.readDiffFrom(Job::new, in);
|
||||||
|
}
|
||||||
|
|
||||||
|
static Diff<Allocation> readAllocationDiffFrom(StreamInput in) throws IOException {
|
||||||
|
return AbstractDiffable.readDiffFrom(Allocation::new, in);
|
||||||
|
}
|
||||||
|
|
||||||
|
static Diff<Datafeed> readSchedulerDiffFrom(StreamInput in) throws IOException {
|
||||||
|
return AbstractDiffable.readDiffFrom(Datafeed::new, in);
|
||||||
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
|
@ -196,6 +225,21 @@ public class MlMetadata implements MetaData.Custom {
|
||||||
Objects.equals(datafeeds, that.datafeeds);
|
Objects.equals(datafeeds, that.datafeeds);
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public final String toString() {
|
||||||
|
try {
|
||||||
|
XContentBuilder builder = XContentFactory.jsonBuilder();
|
||||||
|
builder.prettyPrint();
|
||||||
|
builder.startObject();
|
||||||
|
toXContent(builder, EMPTY_PARAMS);
|
||||||
|
builder.endObject();
|
||||||
|
return builder.string();
|
||||||
|
} catch (Exception e) {
|
||||||
|
// So we have a stack trace logged somewhere
|
||||||
|
return "{ \"error\" : \"" + org.elasticsearch.ExceptionsHelper.detailedMessage(e) + "\"}";
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
public int hashCode() {
|
public int hashCode() {
|
||||||
return Objects.hash(jobs, allocations, datafeeds);
|
return Objects.hash(jobs, allocations, datafeeds);
|
||||||
|
|
|
@ -15,14 +15,14 @@ import org.elasticsearch.action.admin.indices.create.CreateIndexRequest;
|
||||||
import org.elasticsearch.action.admin.indices.delete.DeleteIndexRequest;
|
import org.elasticsearch.action.admin.indices.delete.DeleteIndexRequest;
|
||||||
import org.elasticsearch.action.get.GetRequest;
|
import org.elasticsearch.action.get.GetRequest;
|
||||||
import org.elasticsearch.action.get.GetResponse;
|
import org.elasticsearch.action.get.GetResponse;
|
||||||
|
import org.elasticsearch.action.get.MultiGetItemResponse;
|
||||||
|
import org.elasticsearch.action.get.MultiGetRequest;
|
||||||
import org.elasticsearch.action.search.MultiSearchRequest;
|
import org.elasticsearch.action.search.MultiSearchRequest;
|
||||||
import org.elasticsearch.action.search.MultiSearchResponse;
|
import org.elasticsearch.action.search.MultiSearchResponse;
|
||||||
import org.elasticsearch.action.search.SearchRequest;
|
import org.elasticsearch.action.search.SearchRequest;
|
||||||
import org.elasticsearch.action.search.SearchResponse;
|
import org.elasticsearch.action.search.SearchResponse;
|
||||||
import org.elasticsearch.action.support.PlainActionFuture;
|
|
||||||
import org.elasticsearch.client.Client;
|
import org.elasticsearch.client.Client;
|
||||||
import org.elasticsearch.cluster.metadata.IndexMetaData;
|
import org.elasticsearch.cluster.metadata.IndexMetaData;
|
||||||
import org.elasticsearch.common.CheckedConsumer;
|
|
||||||
import org.elasticsearch.common.Strings;
|
import org.elasticsearch.common.Strings;
|
||||||
import org.elasticsearch.common.bytes.BytesReference;
|
import org.elasticsearch.common.bytes.BytesReference;
|
||||||
import org.elasticsearch.common.logging.Loggers;
|
import org.elasticsearch.common.logging.Loggers;
|
||||||
|
@ -77,11 +77,13 @@ import java.util.ArrayList;
|
||||||
import java.util.Arrays;
|
import java.util.Arrays;
|
||||||
import java.util.Collection;
|
import java.util.Collection;
|
||||||
import java.util.Collections;
|
import java.util.Collections;
|
||||||
|
import java.util.HashSet;
|
||||||
import java.util.Iterator;
|
import java.util.Iterator;
|
||||||
import java.util.List;
|
import java.util.List;
|
||||||
import java.util.Locale;
|
import java.util.Locale;
|
||||||
import java.util.Objects;
|
import java.util.Objects;
|
||||||
import java.util.Optional;
|
import java.util.Optional;
|
||||||
|
import java.util.Set;
|
||||||
import java.util.concurrent.CountDownLatch;
|
import java.util.concurrent.CountDownLatch;
|
||||||
import java.util.concurrent.atomic.AtomicReference;
|
import java.util.concurrent.atomic.AtomicReference;
|
||||||
import java.util.function.BiConsumer;
|
import java.util.function.BiConsumer;
|
||||||
|
@ -287,13 +289,13 @@ public class JobProvider {
|
||||||
* @param jobId The job id
|
* @param jobId The job id
|
||||||
*/
|
*/
|
||||||
public void dataCounts(String jobId, Consumer<DataCounts> handler, Consumer<Exception> errorHandler) {
|
public void dataCounts(String jobId, Consumer<DataCounts> handler, Consumer<Exception> errorHandler) {
|
||||||
get(jobId, DataCounts.TYPE.getPreferredName(), DataCounts.documentId(jobId), handler, errorHandler,
|
String indexName = AnomalyDetectorsIndex.jobResultsIndexName(jobId);
|
||||||
|
get(jobId, indexName, DataCounts.TYPE.getPreferredName(), DataCounts.documentId(jobId), handler, errorHandler,
|
||||||
DataCounts.PARSER, () -> new DataCounts(jobId));
|
DataCounts.PARSER, () -> new DataCounts(jobId));
|
||||||
}
|
}
|
||||||
|
|
||||||
private <T, U> void get(String jobId, String type, String id, Consumer<T> handler, Consumer<Exception> errorHandler,
|
private <T, U> void get(String jobId, String indexName, String type, String id, Consumer<T> handler, Consumer<Exception> errorHandler,
|
||||||
BiFunction<XContentParser, U, T> objectParser, Supplier<T> notFoundSupplier) {
|
BiFunction<XContentParser, U, T> objectParser, Supplier<T> notFoundSupplier) {
|
||||||
String indexName = AnomalyDetectorsIndex.jobResultsIndexName(jobId);
|
|
||||||
GetRequest getRequest = new GetRequest(indexName, type, id);
|
GetRequest getRequest = new GetRequest(indexName, type, id);
|
||||||
client.get(getRequest, ActionListener.wrap(
|
client.get(getRequest, ActionListener.wrap(
|
||||||
response -> {
|
response -> {
|
||||||
|
@ -317,6 +319,38 @@ public class JobProvider {
|
||||||
}));
|
}));
|
||||||
}
|
}
|
||||||
|
|
||||||
|
private <T, U> void mget(String indexName, String type, String[] ids, Consumer<Set<T>> handler, Consumer<Exception> errorHandler,
|
||||||
|
BiFunction<XContentParser, U, T> objectParser) {
|
||||||
|
if (ids.length == 0) {
|
||||||
|
handler.accept(Collections.emptySet());
|
||||||
|
return;
|
||||||
|
}
|
||||||
|
|
||||||
|
MultiGetRequest multiGetRequest = new MultiGetRequest();
|
||||||
|
for (String id : ids) {
|
||||||
|
multiGetRequest.add(indexName, type, id);
|
||||||
|
}
|
||||||
|
client.multiGet(multiGetRequest, ActionListener.wrap(
|
||||||
|
mresponse -> {
|
||||||
|
Set<T> objects = new HashSet<>();
|
||||||
|
for (MultiGetItemResponse item : mresponse) {
|
||||||
|
GetResponse response = item.getResponse();
|
||||||
|
if (response.isExists()) {
|
||||||
|
BytesReference source = response.getSourceAsBytesRef();
|
||||||
|
try (XContentParser parser = XContentFactory.xContent(source)
|
||||||
|
.createParser(NamedXContentRegistry.EMPTY, source)) {
|
||||||
|
objects.add(objectParser.apply(parser, null));
|
||||||
|
} catch (IOException e) {
|
||||||
|
throw new ElasticsearchParseException("failed to parse " + type, e);
|
||||||
|
}
|
||||||
|
}
|
||||||
|
}
|
||||||
|
handler.accept(objects);
|
||||||
|
},
|
||||||
|
errorHandler)
|
||||||
|
);
|
||||||
|
}
|
||||||
|
|
||||||
private <T, U> Optional<T> getBlocking(String indexName, String type, String id, BiFunction<XContentParser, U, T> objectParser) {
|
private <T, U> Optional<T> getBlocking(String indexName, String type, String id, BiFunction<XContentParser, U, T> objectParser) {
|
||||||
GetRequest getRequest = new GetRequest(indexName, type, id);
|
GetRequest getRequest = new GetRequest(indexName, type, id);
|
||||||
try {
|
try {
|
||||||
|
@ -797,17 +831,14 @@ public class JobProvider {
|
||||||
/**
|
/**
|
||||||
* Get the persisted quantiles state for the job
|
* Get the persisted quantiles state for the job
|
||||||
*/
|
*/
|
||||||
public Optional<Quantiles> getQuantiles(String jobId) {
|
public void getQuantiles(String jobId, Consumer<Quantiles> handler, Consumer<Exception> errorHandler) {
|
||||||
String indexName = AnomalyDetectorsIndex.jobStateIndexName();
|
String indexName = AnomalyDetectorsIndex.jobStateIndexName();
|
||||||
String quantilesId = Quantiles.documentId(jobId);
|
String quantilesId = Quantiles.documentId(jobId);
|
||||||
LOGGER.trace("ES API CALL: get ID {} type {} from index {}", quantilesId, Quantiles.TYPE.getPreferredName(), indexName);
|
LOGGER.trace("ES API CALL: get ID {} type {} from index {}", quantilesId, Quantiles.TYPE.getPreferredName(), indexName);
|
||||||
|
get(jobId, indexName, Quantiles.TYPE.getPreferredName(), quantilesId, handler, errorHandler, Quantiles.PARSER, () -> {
|
||||||
Optional<Quantiles> quantiles = getBlocking(indexName, Quantiles.TYPE.getPreferredName(), quantilesId, Quantiles.PARSER);
|
LOGGER.info("There are currently no quantiles for job " + jobId);
|
||||||
if (quantiles.isPresent() && quantiles.get().getQuantileState() == null) {
|
return null;
|
||||||
LOGGER.error("Inconsistency - no " + Quantiles.QUANTILE_STATE + " field in quantiles for job " + jobId);
|
});
|
||||||
}
|
|
||||||
|
|
||||||
return quantiles;
|
|
||||||
}
|
}
|
||||||
|
|
||||||
/**
|
/**
|
||||||
|
@ -816,12 +847,10 @@ public class JobProvider {
|
||||||
* @param jobId the job id
|
* @param jobId the job id
|
||||||
* @param from number of snapshots to from
|
* @param from number of snapshots to from
|
||||||
* @param size number of snapshots to retrieve
|
* @param size number of snapshots to retrieve
|
||||||
* @return page of model snapshots
|
|
||||||
*/
|
*/
|
||||||
public QueryPage<ModelSnapshot> modelSnapshots(String jobId, int from, int size) {
|
public void modelSnapshots(String jobId, int from, int size, Consumer<QueryPage<ModelSnapshot>> handler,
|
||||||
PlainActionFuture<QueryPage<ModelSnapshot>> future = PlainActionFuture.newFuture();
|
Consumer<Exception> errorHandler) {
|
||||||
modelSnapshots(jobId, from, size, null, false, QueryBuilders.matchAllQuery(), future);
|
modelSnapshots(jobId, from, size, null, false, QueryBuilders.matchAllQuery(), handler, errorHandler);
|
||||||
return future.actionGet();
|
|
||||||
}
|
}
|
||||||
|
|
||||||
/**
|
/**
|
||||||
|
@ -846,7 +875,7 @@ public class JobProvider {
|
||||||
boolean sortDescending,
|
boolean sortDescending,
|
||||||
String snapshotId,
|
String snapshotId,
|
||||||
String description,
|
String description,
|
||||||
CheckedConsumer<QueryPage<ModelSnapshot>, Exception> handler,
|
Consumer<QueryPage<ModelSnapshot>> handler,
|
||||||
Consumer<Exception> errorHandler) {
|
Consumer<Exception> errorHandler) {
|
||||||
boolean haveId = snapshotId != null && !snapshotId.isEmpty();
|
boolean haveId = snapshotId != null && !snapshotId.isEmpty();
|
||||||
boolean haveDescription = description != null && !description.isEmpty();
|
boolean haveDescription = description != null && !description.isEmpty();
|
||||||
|
@ -866,7 +895,7 @@ public class JobProvider {
|
||||||
}
|
}
|
||||||
|
|
||||||
QueryBuilder qb = fb.timeRange(Bucket.TIMESTAMP.getPreferredName(), startEpochMs, endEpochMs).build();
|
QueryBuilder qb = fb.timeRange(Bucket.TIMESTAMP.getPreferredName(), startEpochMs, endEpochMs).build();
|
||||||
modelSnapshots(jobId, from, size, sortField, sortDescending, qb, ActionListener.wrap(handler, errorHandler));
|
modelSnapshots(jobId, from, size, sortField, sortDescending, qb, handler, errorHandler);
|
||||||
}
|
}
|
||||||
|
|
||||||
private void modelSnapshots(String jobId,
|
private void modelSnapshots(String jobId,
|
||||||
|
@ -875,7 +904,8 @@ public class JobProvider {
|
||||||
String sortField,
|
String sortField,
|
||||||
boolean sortDescending,
|
boolean sortDescending,
|
||||||
QueryBuilder qb,
|
QueryBuilder qb,
|
||||||
ActionListener<QueryPage<ModelSnapshot>> listener) {
|
Consumer<QueryPage<ModelSnapshot>> handler,
|
||||||
|
Consumer<Exception> errorHandler) {
|
||||||
if (Strings.isEmpty(sortField)) {
|
if (Strings.isEmpty(sortField)) {
|
||||||
sortField = ModelSnapshot.RESTORE_PRIORITY.getPreferredName();
|
sortField = ModelSnapshot.RESTORE_PRIORITY.getPreferredName();
|
||||||
}
|
}
|
||||||
|
@ -909,8 +939,8 @@ public class JobProvider {
|
||||||
|
|
||||||
QueryPage<ModelSnapshot> result =
|
QueryPage<ModelSnapshot> result =
|
||||||
new QueryPage<>(results, searchResponse.getHits().getTotalHits(), ModelSnapshot.RESULTS_FIELD);
|
new QueryPage<>(results, searchResponse.getHits().getTotalHits(), ModelSnapshot.RESULTS_FIELD);
|
||||||
listener.onResponse(result);
|
handler.accept(result);
|
||||||
}, listener::onFailure));
|
}, errorHandler));
|
||||||
}
|
}
|
||||||
|
|
||||||
/**
|
/**
|
||||||
|
@ -1016,7 +1046,8 @@ public class JobProvider {
|
||||||
LOGGER.trace("ES API CALL: get result type {} ID {} for job {}",
|
LOGGER.trace("ES API CALL: get result type {} ID {} for job {}",
|
||||||
ModelSizeStats.RESULT_TYPE_VALUE, ModelSizeStats.RESULT_TYPE_FIELD, jobId);
|
ModelSizeStats.RESULT_TYPE_VALUE, ModelSizeStats.RESULT_TYPE_FIELD, jobId);
|
||||||
|
|
||||||
get(jobId, Result.TYPE.getPreferredName(), ModelSizeStats.documentId(jobId),
|
String indexName = AnomalyDetectorsIndex.jobResultsIndexName(jobId);
|
||||||
|
get(jobId, indexName, Result.TYPE.getPreferredName(), ModelSizeStats.documentId(jobId),
|
||||||
handler, errorHandler, (parser, context) -> ModelSizeStats.PARSER.apply(parser, null).build(),
|
handler, errorHandler, (parser, context) -> ModelSizeStats.PARSER.apply(parser, null).build(),
|
||||||
() -> {
|
() -> {
|
||||||
LOGGER.warn("No memory usage details for job with id {}", jobId);
|
LOGGER.warn("No memory usage details for job with id {}", jobId);
|
||||||
|
@ -1027,11 +1058,10 @@ public class JobProvider {
|
||||||
/**
|
/**
|
||||||
* Retrieves the list with the given {@code listId} from the datastore.
|
* Retrieves the list with the given {@code listId} from the datastore.
|
||||||
*
|
*
|
||||||
* @param listId the id of the requested list
|
* @param ids the id of the requested list
|
||||||
* @return the matching list if it exists
|
|
||||||
*/
|
*/
|
||||||
public Optional<ListDocument> getList(String listId) {
|
public void getLists(Consumer<Set<ListDocument>> handler, Consumer<Exception> errorHandler, String... ids) {
|
||||||
return getBlocking(ML_INFO_INDEX, ListDocument.TYPE.getPreferredName(), listId, ListDocument.PARSER);
|
mget(ML_INFO_INDEX, ListDocument.TYPE.getPreferredName(), ids, handler, errorHandler, ListDocument.PARSER);
|
||||||
}
|
}
|
||||||
|
|
||||||
/**
|
/**
|
||||||
|
|
|
@ -20,7 +20,6 @@ import org.elasticsearch.xpack.ml.job.process.autodetect.writer.ModelDebugConfig
|
||||||
import org.elasticsearch.xpack.ml.job.quantiles.Quantiles;
|
import org.elasticsearch.xpack.ml.job.quantiles.Quantiles;
|
||||||
import org.elasticsearch.xpack.ml.lists.ListDocument;
|
import org.elasticsearch.xpack.ml.lists.ListDocument;
|
||||||
|
|
||||||
import java.io.FileNotFoundException;
|
|
||||||
import java.io.IOException;
|
import java.io.IOException;
|
||||||
import java.io.OutputStreamWriter;
|
import java.io.OutputStreamWriter;
|
||||||
import java.nio.charset.StandardCharsets;
|
import java.nio.charset.StandardCharsets;
|
||||||
|
@ -29,7 +28,6 @@ import java.nio.file.Path;
|
||||||
import java.util.HashSet;
|
import java.util.HashSet;
|
||||||
import java.util.List;
|
import java.util.List;
|
||||||
import java.util.Objects;
|
import java.util.Objects;
|
||||||
import java.util.Optional;
|
|
||||||
import java.util.Set;
|
import java.util.Set;
|
||||||
import java.util.concurrent.TimeoutException;
|
import java.util.concurrent.TimeoutException;
|
||||||
|
|
||||||
|
@ -47,7 +45,7 @@ public class AutodetectBuilder {
|
||||||
private Logger logger;
|
private Logger logger;
|
||||||
private boolean ignoreDowntime;
|
private boolean ignoreDowntime;
|
||||||
private Set<ListDocument> referencedLists;
|
private Set<ListDocument> referencedLists;
|
||||||
private Optional<Quantiles> quantiles;
|
private Quantiles quantiles;
|
||||||
private Environment env;
|
private Environment env;
|
||||||
private Settings settings;
|
private Settings settings;
|
||||||
private NativeController controller;
|
private NativeController controller;
|
||||||
|
@ -72,7 +70,6 @@ public class AutodetectBuilder {
|
||||||
this.logger = Objects.requireNonNull(logger);
|
this.logger = Objects.requireNonNull(logger);
|
||||||
ignoreDowntime = false;
|
ignoreDowntime = false;
|
||||||
referencedLists = new HashSet<>();
|
referencedLists = new HashSet<>();
|
||||||
quantiles = Optional.empty();
|
|
||||||
}
|
}
|
||||||
|
|
||||||
/**
|
/**
|
||||||
|
@ -94,9 +91,9 @@ public class AutodetectBuilder {
|
||||||
/**
|
/**
|
||||||
* Set quantiles to restore the normalizer state if any.
|
* Set quantiles to restore the normalizer state if any.
|
||||||
*
|
*
|
||||||
* @param quantiles the non-null quantiles
|
* @param quantiles the quantiles
|
||||||
*/
|
*/
|
||||||
public AutodetectBuilder quantiles(Optional<Quantiles> quantiles) {
|
public AutodetectBuilder quantiles(Quantiles quantiles) {
|
||||||
this.quantiles = quantiles;
|
this.quantiles = quantiles;
|
||||||
return this;
|
return this;
|
||||||
}
|
}
|
||||||
|
@ -157,8 +154,7 @@ public class AutodetectBuilder {
|
||||||
}
|
}
|
||||||
|
|
||||||
private void buildQuantiles(List<String> command) throws IOException {
|
private void buildQuantiles(List<String> command) throws IOException {
|
||||||
if (quantiles.isPresent() && !quantiles.get().getQuantileState().isEmpty()) {
|
if (quantiles != null && !quantiles.getQuantileState().isEmpty()) {
|
||||||
Quantiles quantiles = this.quantiles.get();
|
|
||||||
logger.info("Restoring quantiles for job '" + job.getId() + "'");
|
logger.info("Restoring quantiles for job '" + job.getId() + "'");
|
||||||
|
|
||||||
Path normalizersStateFilePath = ProcessCtrl.writeNormalizerInitState(
|
Path normalizersStateFilePath = ProcessCtrl.writeNormalizerInitState(
|
||||||
|
@ -170,7 +166,7 @@ public class AutodetectBuilder {
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
private void buildFieldConfig(List<String> command) throws IOException, FileNotFoundException {
|
private void buildFieldConfig(List<String> command) throws IOException {
|
||||||
if (job.getAnalysisConfig() != null) {
|
if (job.getAnalysisConfig() != null) {
|
||||||
// write to a temporary field config file
|
// write to a temporary field config file
|
||||||
Path fieldConfigFile = Files.createTempFile(env.tmpFile(), "fieldconfig", CONF_EXTENSION);
|
Path fieldConfigFile = Files.createTempFile(env.tmpFile(), "fieldconfig", CONF_EXTENSION);
|
||||||
|
|
|
@ -36,6 +36,7 @@ import java.util.Optional;
|
||||||
import java.util.concurrent.CountDownLatch;
|
import java.util.concurrent.CountDownLatch;
|
||||||
import java.util.concurrent.ExecutorService;
|
import java.util.concurrent.ExecutorService;
|
||||||
import java.util.concurrent.atomic.AtomicReference;
|
import java.util.concurrent.atomic.AtomicReference;
|
||||||
|
import java.util.function.Consumer;
|
||||||
import java.util.function.Supplier;
|
import java.util.function.Supplier;
|
||||||
|
|
||||||
public class AutodetectCommunicator implements Closeable {
|
public class AutodetectCommunicator implements Closeable {
|
||||||
|
@ -47,16 +48,18 @@ public class AutodetectCommunicator implements Closeable {
|
||||||
private final StatusReporter statusReporter;
|
private final StatusReporter statusReporter;
|
||||||
private final AutodetectProcess autodetectProcess;
|
private final AutodetectProcess autodetectProcess;
|
||||||
private final AutoDetectResultProcessor autoDetectResultProcessor;
|
private final AutoDetectResultProcessor autoDetectResultProcessor;
|
||||||
|
private final Consumer<Exception> handler;
|
||||||
|
|
||||||
final AtomicReference<CountDownLatch> inUse = new AtomicReference<>();
|
final AtomicReference<CountDownLatch> inUse = new AtomicReference<>();
|
||||||
|
|
||||||
public AutodetectCommunicator(ExecutorService autoDetectExecutor, Job job, AutodetectProcess process,
|
public AutodetectCommunicator(ExecutorService autoDetectExecutor, Job job, AutodetectProcess process,
|
||||||
StatusReporter statusReporter, AutoDetectResultProcessor autoDetectResultProcessor,
|
StatusReporter statusReporter, AutoDetectResultProcessor autoDetectResultProcessor,
|
||||||
StateProcessor stateProcessor) {
|
StateProcessor stateProcessor, Consumer<Exception> handler) {
|
||||||
this.job = job;
|
this.job = job;
|
||||||
this.autodetectProcess = process;
|
this.autodetectProcess = process;
|
||||||
this.statusReporter = statusReporter;
|
this.statusReporter = statusReporter;
|
||||||
this.autoDetectResultProcessor = autoDetectResultProcessor;
|
this.autoDetectResultProcessor = autoDetectResultProcessor;
|
||||||
|
this.handler = handler;
|
||||||
|
|
||||||
AnalysisConfig analysisConfig = job.getAnalysisConfig();
|
AnalysisConfig analysisConfig = job.getAnalysisConfig();
|
||||||
boolean usePerPartitionNormalization = analysisConfig.getUsePerPartitionNormalization();
|
boolean usePerPartitionNormalization = analysisConfig.getUsePerPartitionNormalization();
|
||||||
|
@ -77,7 +80,7 @@ public class AutodetectCommunicator implements Closeable {
|
||||||
job.getAnalysisConfig(), new TransformConfigs(job.getTransforms()) , statusReporter, LOGGER);
|
job.getAnalysisConfig(), new TransformConfigs(job.getTransforms()) , statusReporter, LOGGER);
|
||||||
}
|
}
|
||||||
|
|
||||||
public DataCounts writeToJob(InputStream inputStream, DataLoadParams params, Supplier<Boolean> cancelled) throws IOException {
|
public DataCounts writeToJob(InputStream inputStream, DataLoadParams params) throws IOException {
|
||||||
return checkAndRun(() -> Messages.getMessage(Messages.JOB_DATA_CONCURRENT_USE_UPLOAD, job.getId()), () -> {
|
return checkAndRun(() -> Messages.getMessage(Messages.JOB_DATA_CONCURRENT_USE_UPLOAD, job.getId()), () -> {
|
||||||
if (params.isResettingBuckets()) {
|
if (params.isResettingBuckets()) {
|
||||||
autodetectProcess.writeResetBucketsControlMessage(params);
|
autodetectProcess.writeResetBucketsControlMessage(params);
|
||||||
|
@ -85,7 +88,7 @@ public class AutodetectCommunicator implements Closeable {
|
||||||
CountingInputStream countingStream = new CountingInputStream(inputStream, statusReporter);
|
CountingInputStream countingStream = new CountingInputStream(inputStream, statusReporter);
|
||||||
|
|
||||||
DataToProcessWriter autoDetectWriter = createProcessWriter(params.getDataDescription());
|
DataToProcessWriter autoDetectWriter = createProcessWriter(params.getDataDescription());
|
||||||
DataCounts results = autoDetectWriter.write(countingStream, cancelled);
|
DataCounts results = autoDetectWriter.write(countingStream);
|
||||||
autoDetectWriter.flush();
|
autoDetectWriter.flush();
|
||||||
return results;
|
return results;
|
||||||
}, false);
|
}, false);
|
||||||
|
@ -97,6 +100,7 @@ public class AutodetectCommunicator implements Closeable {
|
||||||
statusReporter.close();
|
statusReporter.close();
|
||||||
autodetectProcess.close();
|
autodetectProcess.close();
|
||||||
autoDetectResultProcessor.awaitCompletion();
|
autoDetectResultProcessor.awaitCompletion();
|
||||||
|
handler.accept(null);
|
||||||
return null;
|
return null;
|
||||||
}, true);
|
}, true);
|
||||||
}
|
}
|
||||||
|
|
|
@ -6,20 +6,29 @@
|
||||||
package org.elasticsearch.xpack.ml.job.process.autodetect;
|
package org.elasticsearch.xpack.ml.job.process.autodetect;
|
||||||
|
|
||||||
import org.elasticsearch.xpack.ml.job.Job;
|
import org.elasticsearch.xpack.ml.job.Job;
|
||||||
|
import org.elasticsearch.xpack.ml.job.ModelSnapshot;
|
||||||
|
import org.elasticsearch.xpack.ml.job.quantiles.Quantiles;
|
||||||
|
import org.elasticsearch.xpack.ml.lists.ListDocument;
|
||||||
|
|
||||||
|
import java.util.Set;
|
||||||
import java.util.concurrent.ExecutorService;
|
import java.util.concurrent.ExecutorService;
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* Factory interface for creating implementations of {@link AutodetectProcess}
|
* Factory interface for creating implementations of {@link AutodetectProcess}
|
||||||
*/
|
*/
|
||||||
public interface AutodetectProcessFactory {
|
public interface AutodetectProcessFactory {
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* Create an implementation of {@link AutodetectProcess}
|
* Create an implementation of {@link AutodetectProcess}
|
||||||
*
|
*
|
||||||
* @param job Job configuration for the analysis process
|
* @param job Job configuration for the analysis process
|
||||||
|
* @param modelSnapshot The model snapshot to restore from
|
||||||
|
* @param quantiles The quantiles to push to the native process
|
||||||
|
* @param list The lists to push to the native process
|
||||||
* @param ignoreDowntime Should gaps in data be treated as anomalous or as a maintenance window after a job re-start
|
* @param ignoreDowntime Should gaps in data be treated as anomalous or as a maintenance window after a job re-start
|
||||||
* @param executorService Executor service used to start the async tasks a job needs to operate the analytical process
|
* @param executorService Executor service used to start the async tasks a job needs to operate the analytical process
|
||||||
* @return The process
|
* @return The process
|
||||||
*/
|
*/
|
||||||
AutodetectProcess createAutodetectProcess(Job job, boolean ignoreDowntime, ExecutorService executorService);
|
AutodetectProcess createAutodetectProcess(Job job, ModelSnapshot modelSnapshot, Quantiles quantiles, Set<ListDocument> list,
|
||||||
|
boolean ignoreDowntime, ExecutorService executorService);
|
||||||
}
|
}
|
||||||
|
|
|
@ -27,10 +27,8 @@ import java.io.OutputStream;
|
||||||
import java.nio.file.Path;
|
import java.nio.file.Path;
|
||||||
import java.time.Duration;
|
import java.time.Duration;
|
||||||
import java.util.ArrayList;
|
import java.util.ArrayList;
|
||||||
import java.util.HashSet;
|
|
||||||
import java.util.List;
|
import java.util.List;
|
||||||
import java.util.Objects;
|
import java.util.Objects;
|
||||||
import java.util.Optional;
|
|
||||||
import java.util.Set;
|
import java.util.Set;
|
||||||
import java.util.concurrent.ExecutorService;
|
import java.util.concurrent.ExecutorService;
|
||||||
import java.util.concurrent.TimeoutException;
|
import java.util.concurrent.TimeoutException;
|
||||||
|
@ -54,14 +52,12 @@ public class NativeAutodetectProcessFactory implements AutodetectProcessFactory
|
||||||
}
|
}
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
public AutodetectProcess createAutodetectProcess(Job job, boolean ignoreDowntime, ExecutorService executorService) {
|
public AutodetectProcess createAutodetectProcess(Job job, ModelSnapshot modelSnapshot, Quantiles quantiles, Set<ListDocument> list,
|
||||||
|
boolean ignoreDowntime, ExecutorService executorService) {
|
||||||
List<Path> filesToDelete = new ArrayList<>();
|
List<Path> filesToDelete = new ArrayList<>();
|
||||||
List<ModelSnapshot> modelSnapshots = jobProvider.modelSnapshots(job.getId(), 0, 1).results();
|
|
||||||
ModelSnapshot modelSnapshot = (modelSnapshots != null && !modelSnapshots.isEmpty()) ? modelSnapshots.get(0) : null;
|
|
||||||
|
|
||||||
ProcessPipes processPipes = new ProcessPipes(env, NAMED_PIPE_HELPER, ProcessCtrl.AUTODETECT, job.getId(),
|
ProcessPipes processPipes = new ProcessPipes(env, NAMED_PIPE_HELPER, ProcessCtrl.AUTODETECT, job.getId(),
|
||||||
true, false, true, true, modelSnapshot != null, !ProcessCtrl.DONT_PERSIST_MODEL_STATE_SETTING.get(settings));
|
true, false, true, true, modelSnapshot != null, !ProcessCtrl.DONT_PERSIST_MODEL_STATE_SETTING.get(settings));
|
||||||
createNativeProcess(job, processPipes, ignoreDowntime, filesToDelete);
|
createNativeProcess(job, quantiles, list, processPipes, ignoreDowntime, filesToDelete);
|
||||||
int numberOfAnalysisFields = job.getAnalysisConfig().analysisFields().size();
|
int numberOfAnalysisFields = job.getAnalysisConfig().analysisFields().size();
|
||||||
|
|
||||||
NativeAutodetectProcess autodetect = null;
|
NativeAutodetectProcess autodetect = null;
|
||||||
|
@ -91,16 +87,13 @@ public class NativeAutodetectProcessFactory implements AutodetectProcessFactory
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
private void createNativeProcess(Job job, ProcessPipes processPipes, boolean ignoreDowntime, List<Path> filesToDelete) {
|
private void createNativeProcess(Job job, Quantiles quantiles, Set<ListDocument> lists, ProcessPipes processPipes,
|
||||||
|
boolean ignoreDowntime, List<Path> filesToDelete) {
|
||||||
String jobId = job.getId();
|
|
||||||
Optional<Quantiles> quantiles = jobProvider.getQuantiles(jobId);
|
|
||||||
|
|
||||||
try {
|
try {
|
||||||
AutodetectBuilder autodetectBuilder = new AutodetectBuilder(job, filesToDelete, LOGGER, env,
|
AutodetectBuilder autodetectBuilder = new AutodetectBuilder(job, filesToDelete, LOGGER, env,
|
||||||
settings, nativeController, processPipes)
|
settings, nativeController, processPipes)
|
||||||
.ignoreDowntime(ignoreDowntime)
|
.ignoreDowntime(ignoreDowntime)
|
||||||
.referencedLists(resolveLists(job.getAnalysisConfig().extractReferencedLists()));
|
.referencedLists(lists);
|
||||||
|
|
||||||
// if state is null or empty it will be ignored
|
// if state is null or empty it will be ignored
|
||||||
// else it is used to restore the quantiles
|
// else it is used to restore the quantiles
|
||||||
|
@ -116,18 +109,5 @@ public class NativeAutodetectProcessFactory implements AutodetectProcessFactory
|
||||||
throw ExceptionsHelper.serverError(msg, e);
|
throw ExceptionsHelper.serverError(msg, e);
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
private Set<ListDocument> resolveLists(Set<String> listIds) {
|
|
||||||
Set<ListDocument> resolved = new HashSet<>();
|
|
||||||
for (String listId : listIds) {
|
|
||||||
Optional<ListDocument> list = jobProvider.getList(listId);
|
|
||||||
if (list.isPresent()) {
|
|
||||||
resolved.add(list.get());
|
|
||||||
} else {
|
|
||||||
LOGGER.warn("List '" + listId + "' could not be retrieved.");
|
|
||||||
}
|
|
||||||
}
|
|
||||||
return resolved;
|
|
||||||
}
|
|
||||||
}
|
}
|
||||||
|
|
||||||
|
|
|
@ -94,7 +94,7 @@ public abstract class AbstractDataToProcessWriter implements DataToProcessWriter
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* Create the transforms. This must be called before
|
* Create the transforms. This must be called before
|
||||||
* {@linkplain DataToProcessWriter#write(java.io.InputStream, java.util.function.Supplier)}
|
* {@linkplain DataToProcessWriter#write(java.io.InputStream)}
|
||||||
* even if no transforms are configured as it creates the
|
* even if no transforms are configured as it creates the
|
||||||
* date transform and sets up the field mappings.<br>
|
* date transform and sets up the field mappings.<br>
|
||||||
* <p>
|
* <p>
|
||||||
|
@ -212,7 +212,6 @@ public abstract class AbstractDataToProcessWriter implements DataToProcessWriter
|
||||||
* First all the transforms whose outputs the Date transform relies
|
* First all the transforms whose outputs the Date transform relies
|
||||||
* on are executed then the date transform then the remaining transforms.
|
* on are executed then the date transform then the remaining transforms.
|
||||||
*
|
*
|
||||||
* @param cancelled Determines whether the process writting has been cancelled
|
|
||||||
* @param input The record the transforms should read their input from. The contents should
|
* @param input The record the transforms should read their input from. The contents should
|
||||||
* align with the header parameter passed to {@linkplain #buildTransforms(String[])}
|
* align with the header parameter passed to {@linkplain #buildTransforms(String[])}
|
||||||
* @param output The record that will be written to the length encoded writer.
|
* @param output The record that will be written to the length encoded writer.
|
||||||
|
@ -220,12 +219,8 @@ public abstract class AbstractDataToProcessWriter implements DataToProcessWriter
|
||||||
* the size of the map returned by {@linkplain #outputFieldIndexes()}
|
* the size of the map returned by {@linkplain #outputFieldIndexes()}
|
||||||
* @param numberOfFieldsRead The total number read not just those included in the analysis
|
* @param numberOfFieldsRead The total number read not just those included in the analysis
|
||||||
*/
|
*/
|
||||||
protected boolean applyTransformsAndWrite(Supplier<Boolean> cancelled, String[] input, String[] output, long numberOfFieldsRead)
|
protected boolean applyTransformsAndWrite(String[] input, String[] output, long numberOfFieldsRead)
|
||||||
throws IOException {
|
throws IOException {
|
||||||
if (cancelled.get()) {
|
|
||||||
throw new TaskCancelledException("cancelled");
|
|
||||||
}
|
|
||||||
|
|
||||||
readWriteArea[TransformFactory.INPUT_ARRAY_INDEX] = input;
|
readWriteArea[TransformFactory.INPUT_ARRAY_INDEX] = input;
|
||||||
readWriteArea[TransformFactory.OUTPUT_ARRAY_INDEX] = output;
|
readWriteArea[TransformFactory.OUTPUT_ARRAY_INDEX] = output;
|
||||||
Arrays.fill(readWriteArea[TransformFactory.SCRATCH_ARRAY_INDEX], "");
|
Arrays.fill(readWriteArea[TransformFactory.SCRATCH_ARRAY_INDEX], "");
|
||||||
|
|
|
@ -24,7 +24,6 @@ import java.util.Collection;
|
||||||
import java.util.List;
|
import java.util.List;
|
||||||
import java.util.Locale;
|
import java.util.Locale;
|
||||||
import java.util.Map;
|
import java.util.Map;
|
||||||
import java.util.function.Supplier;
|
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* A writer for transforming and piping CSV data from an
|
* A writer for transforming and piping CSV data from an
|
||||||
|
@ -63,7 +62,7 @@ class CsvDataToProcessWriter extends AbstractDataToProcessWriter {
|
||||||
* header a exception is thrown
|
* header a exception is thrown
|
||||||
*/
|
*/
|
||||||
@Override
|
@Override
|
||||||
public DataCounts write(InputStream inputStream, Supplier<Boolean> cancelled) throws IOException {
|
public DataCounts write(InputStream inputStream) throws IOException {
|
||||||
CsvPreference csvPref = new CsvPreference.Builder(
|
CsvPreference csvPref = new CsvPreference.Builder(
|
||||||
dataDescription.getQuoteCharacter(),
|
dataDescription.getQuoteCharacter(),
|
||||||
dataDescription.getFieldDelimiter(),
|
dataDescription.getFieldDelimiter(),
|
||||||
|
@ -118,7 +117,7 @@ class CsvDataToProcessWriter extends AbstractDataToProcessWriter {
|
||||||
}
|
}
|
||||||
|
|
||||||
fillRecordFromLine(line, inputRecord);
|
fillRecordFromLine(line, inputRecord);
|
||||||
applyTransformsAndWrite(cancelled, inputRecord, record, inputFieldCount);
|
applyTransformsAndWrite(inputRecord, record, inputFieldCount);
|
||||||
}
|
}
|
||||||
|
|
||||||
// This function can throw
|
// This function can throw
|
||||||
|
|
|
@ -5,11 +5,10 @@
|
||||||
*/
|
*/
|
||||||
package org.elasticsearch.xpack.ml.job.process.autodetect.writer;
|
package org.elasticsearch.xpack.ml.job.process.autodetect.writer;
|
||||||
|
|
||||||
|
import org.elasticsearch.xpack.ml.job.DataCounts;
|
||||||
|
|
||||||
import java.io.IOException;
|
import java.io.IOException;
|
||||||
import java.io.InputStream;
|
import java.io.InputStream;
|
||||||
import java.util.function.Supplier;
|
|
||||||
|
|
||||||
import org.elasticsearch.xpack.ml.job.DataCounts;
|
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* A writer for transforming and piping data from an
|
* A writer for transforming and piping data from an
|
||||||
|
@ -33,7 +32,7 @@ public interface DataToProcessWriter {
|
||||||
*
|
*
|
||||||
* @return Counts of the records processed, bytes read etc
|
* @return Counts of the records processed, bytes read etc
|
||||||
*/
|
*/
|
||||||
DataCounts write(InputStream inputStream, Supplier<Boolean> cancelled) throws IOException;
|
DataCounts write(InputStream inputStream) throws IOException;
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* Flush the outputstream
|
* Flush the outputstream
|
||||||
|
|
|
@ -20,7 +20,6 @@ import java.io.InputStream;
|
||||||
import java.util.Arrays;
|
import java.util.Arrays;
|
||||||
import java.util.Collection;
|
import java.util.Collection;
|
||||||
import java.util.Map;
|
import java.util.Map;
|
||||||
import java.util.function.Supplier;
|
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* A writer for transforming and piping JSON data from an
|
* A writer for transforming and piping JSON data from an
|
||||||
|
@ -46,11 +45,11 @@ class JsonDataToProcessWriter extends AbstractDataToProcessWriter {
|
||||||
* timeField is missing from the JOSN inputIndex an exception is thrown
|
* timeField is missing from the JOSN inputIndex an exception is thrown
|
||||||
*/
|
*/
|
||||||
@Override
|
@Override
|
||||||
public DataCounts write(InputStream inputStream, Supplier<Boolean> cancelled) throws IOException {
|
public DataCounts write(InputStream inputStream) throws IOException {
|
||||||
statusReporter.startNewIncrementalCount();
|
statusReporter.startNewIncrementalCount();
|
||||||
|
|
||||||
try (JsonParser parser = new JsonFactory().createParser(inputStream)) {
|
try (JsonParser parser = new JsonFactory().createParser(inputStream)) {
|
||||||
writeJson(parser, cancelled);
|
writeJson(parser);
|
||||||
|
|
||||||
// this line can throw and will be propagated
|
// this line can throw and will be propagated
|
||||||
statusReporter.finishReporting();
|
statusReporter.finishReporting();
|
||||||
|
@ -59,7 +58,7 @@ class JsonDataToProcessWriter extends AbstractDataToProcessWriter {
|
||||||
return statusReporter.incrementalStats();
|
return statusReporter.incrementalStats();
|
||||||
}
|
}
|
||||||
|
|
||||||
private void writeJson(JsonParser parser, Supplier<Boolean> cancelled) throws IOException {
|
private void writeJson(JsonParser parser) throws IOException {
|
||||||
Collection<String> analysisFields = inputFields();
|
Collection<String> analysisFields = inputFields();
|
||||||
|
|
||||||
buildTransforms(analysisFields.toArray(new String[0]));
|
buildTransforms(analysisFields.toArray(new String[0]));
|
||||||
|
@ -88,7 +87,7 @@ class JsonDataToProcessWriter extends AbstractDataToProcessWriter {
|
||||||
record[inOut.outputIndex] = (field == null) ? "" : field;
|
record[inOut.outputIndex] = (field == null) ? "" : field;
|
||||||
}
|
}
|
||||||
|
|
||||||
applyTransformsAndWrite(cancelled, input, record, inputFieldCount);
|
applyTransformsAndWrite(input, record, inputFieldCount);
|
||||||
|
|
||||||
inputFieldCount = recordReader.read(input, gotFields);
|
inputFieldCount = recordReader.read(input, gotFields);
|
||||||
}
|
}
|
||||||
|
|
|
@ -5,6 +5,14 @@
|
||||||
*/
|
*/
|
||||||
package org.elasticsearch.xpack.ml.job.process.autodetect.writer;
|
package org.elasticsearch.xpack.ml.job.process.autodetect.writer;
|
||||||
|
|
||||||
|
import org.apache.logging.log4j.Logger;
|
||||||
|
import org.elasticsearch.xpack.ml.job.AnalysisConfig;
|
||||||
|
import org.elasticsearch.xpack.ml.job.DataCounts;
|
||||||
|
import org.elasticsearch.xpack.ml.job.DataDescription;
|
||||||
|
import org.elasticsearch.xpack.ml.job.process.autodetect.AutodetectProcess;
|
||||||
|
import org.elasticsearch.xpack.ml.job.status.StatusReporter;
|
||||||
|
import org.elasticsearch.xpack.ml.job.transform.TransformConfigs;
|
||||||
|
|
||||||
import java.io.BufferedReader;
|
import java.io.BufferedReader;
|
||||||
import java.io.IOException;
|
import java.io.IOException;
|
||||||
import java.io.InputStream;
|
import java.io.InputStream;
|
||||||
|
@ -13,16 +21,6 @@ import java.nio.charset.StandardCharsets;
|
||||||
import java.util.Arrays;
|
import java.util.Arrays;
|
||||||
import java.util.Collection;
|
import java.util.Collection;
|
||||||
import java.util.Map;
|
import java.util.Map;
|
||||||
import java.util.function.Supplier;
|
|
||||||
|
|
||||||
import org.apache.logging.log4j.Logger;
|
|
||||||
|
|
||||||
import org.elasticsearch.xpack.ml.job.AnalysisConfig;
|
|
||||||
import org.elasticsearch.xpack.ml.job.DataCounts;
|
|
||||||
import org.elasticsearch.xpack.ml.job.DataDescription;
|
|
||||||
import org.elasticsearch.xpack.ml.job.process.autodetect.AutodetectProcess;
|
|
||||||
import org.elasticsearch.xpack.ml.job.status.StatusReporter;
|
|
||||||
import org.elasticsearch.xpack.ml.job.transform.TransformConfigs;
|
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* This writer is used for reading inputIndex data that are unstructured and
|
* This writer is used for reading inputIndex data that are unstructured and
|
||||||
|
@ -44,7 +42,7 @@ public class SingleLineDataToProcessWriter extends AbstractDataToProcessWriter {
|
||||||
}
|
}
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
public DataCounts write(InputStream inputStream, Supplier<Boolean> cancelled) throws IOException {
|
public DataCounts write(InputStream inputStream) throws IOException {
|
||||||
statusReporter.startNewIncrementalCount();
|
statusReporter.startNewIncrementalCount();
|
||||||
|
|
||||||
try (BufferedReader bufferedReader = new BufferedReader(new InputStreamReader(inputStream, StandardCharsets.UTF_8))) {
|
try (BufferedReader bufferedReader = new BufferedReader(new InputStreamReader(inputStream, StandardCharsets.UTF_8))) {
|
||||||
|
@ -57,7 +55,7 @@ public class SingleLineDataToProcessWriter extends AbstractDataToProcessWriter {
|
||||||
for (String line = bufferedReader.readLine(); line != null;
|
for (String line = bufferedReader.readLine(); line != null;
|
||||||
line = bufferedReader.readLine()) {
|
line = bufferedReader.readLine()) {
|
||||||
Arrays.fill(record, "");
|
Arrays.fill(record, "");
|
||||||
applyTransformsAndWrite(cancelled, new String[]{line}, record, 1);
|
applyTransformsAndWrite(new String[]{line}, record, 1);
|
||||||
}
|
}
|
||||||
statusReporter.finishReporting();
|
statusReporter.finishReporting();
|
||||||
}
|
}
|
||||||
|
|
|
@ -12,7 +12,7 @@ import org.elasticsearch.common.unit.TimeValue;
|
||||||
import org.elasticsearch.rest.BaseRestHandler;
|
import org.elasticsearch.rest.BaseRestHandler;
|
||||||
import org.elasticsearch.rest.RestController;
|
import org.elasticsearch.rest.RestController;
|
||||||
import org.elasticsearch.rest.RestRequest;
|
import org.elasticsearch.rest.RestRequest;
|
||||||
import org.elasticsearch.rest.action.AcknowledgedRestListener;
|
import org.elasticsearch.rest.action.RestToXContentListener;
|
||||||
import org.elasticsearch.xpack.ml.MlPlugin;
|
import org.elasticsearch.xpack.ml.MlPlugin;
|
||||||
import org.elasticsearch.xpack.ml.action.CloseJobAction;
|
import org.elasticsearch.xpack.ml.action.CloseJobAction;
|
||||||
import org.elasticsearch.xpack.ml.job.Job;
|
import org.elasticsearch.xpack.ml.job.Job;
|
||||||
|
@ -34,6 +34,6 @@ public class RestCloseJobAction extends BaseRestHandler {
|
||||||
if (restRequest.hasParam("close_timeout")) {
|
if (restRequest.hasParam("close_timeout")) {
|
||||||
request.setCloseTimeout(TimeValue.parseTimeValue(restRequest.param("close_timeout"), "close_timeout"));
|
request.setCloseTimeout(TimeValue.parseTimeValue(restRequest.param("close_timeout"), "close_timeout"));
|
||||||
}
|
}
|
||||||
return channel -> client.execute(CloseJobAction.INSTANCE, request, new AcknowledgedRestListener<>(channel));
|
return channel -> client.execute(CloseJobAction.INSTANCE, request, new RestToXContentListener<>(channel));
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
|
@ -12,7 +12,7 @@ import org.elasticsearch.common.xcontent.XContentParser;
|
||||||
import org.elasticsearch.rest.BaseRestHandler;
|
import org.elasticsearch.rest.BaseRestHandler;
|
||||||
import org.elasticsearch.rest.RestController;
|
import org.elasticsearch.rest.RestController;
|
||||||
import org.elasticsearch.rest.RestRequest;
|
import org.elasticsearch.rest.RestRequest;
|
||||||
import org.elasticsearch.rest.action.AcknowledgedRestListener;
|
import org.elasticsearch.rest.action.RestToXContentListener;
|
||||||
import org.elasticsearch.xpack.ml.MlPlugin;
|
import org.elasticsearch.xpack.ml.MlPlugin;
|
||||||
import org.elasticsearch.xpack.ml.action.FlushJobAction;
|
import org.elasticsearch.xpack.ml.action.FlushJobAction;
|
||||||
import org.elasticsearch.xpack.ml.job.Job;
|
import org.elasticsearch.xpack.ml.job.Job;
|
||||||
|
@ -49,6 +49,6 @@ public class RestFlushJobAction extends BaseRestHandler {
|
||||||
request.setAdvanceTime(restRequest.param(FlushJobAction.Request.ADVANCE_TIME.getPreferredName(), DEFAULT_ADVANCE_TIME));
|
request.setAdvanceTime(restRequest.param(FlushJobAction.Request.ADVANCE_TIME.getPreferredName(), DEFAULT_ADVANCE_TIME));
|
||||||
}
|
}
|
||||||
|
|
||||||
return channel -> client.execute(FlushJobAction.INSTANCE, request, new AcknowledgedRestListener<>(channel));
|
return channel -> client.execute(FlushJobAction.INSTANCE, request, new RestToXContentListener<>(channel));
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
|
@ -12,7 +12,7 @@ import org.elasticsearch.common.unit.TimeValue;
|
||||||
import org.elasticsearch.rest.BaseRestHandler;
|
import org.elasticsearch.rest.BaseRestHandler;
|
||||||
import org.elasticsearch.rest.RestController;
|
import org.elasticsearch.rest.RestController;
|
||||||
import org.elasticsearch.rest.RestRequest;
|
import org.elasticsearch.rest.RestRequest;
|
||||||
import org.elasticsearch.rest.action.AcknowledgedRestListener;
|
import org.elasticsearch.rest.action.RestToXContentListener;
|
||||||
import org.elasticsearch.xpack.ml.MlPlugin;
|
import org.elasticsearch.xpack.ml.MlPlugin;
|
||||||
import org.elasticsearch.xpack.ml.action.OpenJobAction;
|
import org.elasticsearch.xpack.ml.action.OpenJobAction;
|
||||||
import org.elasticsearch.xpack.ml.action.PostDataAction;
|
import org.elasticsearch.xpack.ml.action.PostDataAction;
|
||||||
|
@ -33,11 +33,12 @@ public class RestOpenJobAction extends BaseRestHandler {
|
||||||
protected RestChannelConsumer prepareRequest(RestRequest restRequest, NodeClient client) throws IOException {
|
protected RestChannelConsumer prepareRequest(RestRequest restRequest, NodeClient client) throws IOException {
|
||||||
OpenJobAction.Request request = new OpenJobAction.Request(restRequest.param(Job.ID.getPreferredName()));
|
OpenJobAction.Request request = new OpenJobAction.Request(restRequest.param(Job.ID.getPreferredName()));
|
||||||
request.setIgnoreDowntime(restRequest.paramAsBoolean(PostDataAction.Request.IGNORE_DOWNTIME.getPreferredName(), false));
|
request.setIgnoreDowntime(restRequest.paramAsBoolean(PostDataAction.Request.IGNORE_DOWNTIME.getPreferredName(), false));
|
||||||
if (restRequest.hasParam(OpenJobAction.Request.OPEN_TIMEOUT.getPreferredName())) {
|
if (restRequest.hasParam("open_timeout")) {
|
||||||
request.setOpenTimeout(TimeValue.parseTimeValue(
|
TimeValue openTimeout = restRequest.paramAsTime("open_timeout", TimeValue.timeValueSeconds(30));
|
||||||
restRequest.param(OpenJobAction.Request.OPEN_TIMEOUT.getPreferredName()),
|
request.setOpenTimeout(openTimeout);
|
||||||
OpenJobAction.Request.OPEN_TIMEOUT.getPreferredName()));
|
|
||||||
}
|
}
|
||||||
return channel -> client.execute(OpenJobAction.INSTANCE, request, new AcknowledgedRestListener<>(channel));
|
return channel -> {
|
||||||
|
client.execute(OpenJobAction.INSTANCE, request, new RestToXContentListener<>(channel));
|
||||||
|
};
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
|
@ -53,15 +53,15 @@ public class DatafeedStatusObserver {
|
||||||
+ expectedStatus + "]");
|
+ expectedStatus + "]");
|
||||||
handler.accept(e);
|
handler.accept(e);
|
||||||
}
|
}
|
||||||
}, new DatafeedStoppedPredicate(datafeedId, expectedStatus), waitTimeout);
|
}, new DatafeedPredicate(datafeedId, expectedStatus), waitTimeout);
|
||||||
}
|
}
|
||||||
|
|
||||||
private static class DatafeedStoppedPredicate implements Predicate<ClusterState> {
|
private static class DatafeedPredicate implements Predicate<ClusterState> {
|
||||||
|
|
||||||
private final String datafeedId;
|
private final String datafeedId;
|
||||||
private final DatafeedStatus expectedStatus;
|
private final DatafeedStatus expectedStatus;
|
||||||
|
|
||||||
DatafeedStoppedPredicate(String datafeedId, DatafeedStatus expectedStatus) {
|
DatafeedPredicate(String datafeedId, DatafeedStatus expectedStatus) {
|
||||||
this.datafeedId = datafeedId;
|
this.datafeedId = datafeedId;
|
||||||
this.expectedStatus = expectedStatus;
|
this.expectedStatus = expectedStatus;
|
||||||
}
|
}
|
||||||
|
|
|
@ -0,0 +1,83 @@
|
||||||
|
/*
|
||||||
|
* Copyright Elasticsearch B.V. and/or licensed to Elasticsearch B.V. under one
|
||||||
|
* or more contributor license agreements. Licensed under the Elastic License;
|
||||||
|
* you may not use this file except in compliance with the Elastic License.
|
||||||
|
*/
|
||||||
|
package org.elasticsearch.xpack.ml.utils;
|
||||||
|
|
||||||
|
import org.apache.logging.log4j.Logger;
|
||||||
|
import org.elasticsearch.cluster.ClusterState;
|
||||||
|
import org.elasticsearch.cluster.ClusterStateObserver;
|
||||||
|
import org.elasticsearch.cluster.service.ClusterService;
|
||||||
|
import org.elasticsearch.common.logging.Loggers;
|
||||||
|
import org.elasticsearch.common.unit.TimeValue;
|
||||||
|
import org.elasticsearch.threadpool.ThreadPool;
|
||||||
|
import org.elasticsearch.xpack.ml.job.JobStatus;
|
||||||
|
import org.elasticsearch.xpack.ml.job.metadata.Allocation;
|
||||||
|
import org.elasticsearch.xpack.ml.job.metadata.MlMetadata;
|
||||||
|
|
||||||
|
import java.util.function.Consumer;
|
||||||
|
import java.util.function.Predicate;
|
||||||
|
|
||||||
|
public class JobStatusObserver {
|
||||||
|
|
||||||
|
private static final Logger LOGGER = Loggers.getLogger(JobStatusObserver.class);
|
||||||
|
|
||||||
|
private final ThreadPool threadPool;
|
||||||
|
private final ClusterService clusterService;
|
||||||
|
|
||||||
|
public JobStatusObserver(ThreadPool threadPool, ClusterService clusterService) {
|
||||||
|
this.threadPool = threadPool;
|
||||||
|
this.clusterService = clusterService;
|
||||||
|
}
|
||||||
|
|
||||||
|
public void waitForStatus(String jobId, TimeValue waitTimeout, JobStatus expectedStatus, Consumer<Exception> handler) {
|
||||||
|
ClusterStateObserver observer =
|
||||||
|
new ClusterStateObserver(clusterService, LOGGER, threadPool.getThreadContext());
|
||||||
|
observer.waitForNextChange(new ClusterStateObserver.Listener() {
|
||||||
|
@Override
|
||||||
|
public void onNewClusterState(ClusterState state) {
|
||||||
|
handler.accept(null);
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public void onClusterServiceClose() {
|
||||||
|
Exception e = new IllegalArgumentException("Cluster service closed while waiting for job status to change to ["
|
||||||
|
+ expectedStatus + "]");
|
||||||
|
handler.accept(new IllegalStateException(e));
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public void onTimeout(TimeValue timeout) {
|
||||||
|
Exception e = new IllegalArgumentException("Timeout expired while waiting for job status to change to ["
|
||||||
|
+ expectedStatus + "]");
|
||||||
|
handler.accept(e);
|
||||||
|
}
|
||||||
|
}, new JobStatusPredicate(jobId, expectedStatus), waitTimeout);
|
||||||
|
}
|
||||||
|
|
||||||
|
private static class JobStatusPredicate implements Predicate<ClusterState> {
|
||||||
|
|
||||||
|
private final String jobId;
|
||||||
|
private final JobStatus expectedStatus;
|
||||||
|
|
||||||
|
JobStatusPredicate(String jobId, JobStatus expectedStatus) {
|
||||||
|
this.jobId = jobId;
|
||||||
|
this.expectedStatus = expectedStatus;
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public boolean test(ClusterState newState) {
|
||||||
|
MlMetadata metadata = newState.getMetaData().custom(MlMetadata.TYPE);
|
||||||
|
if (metadata != null) {
|
||||||
|
Allocation allocation = metadata.getAllocations().get(jobId);
|
||||||
|
if (allocation != null) {
|
||||||
|
return allocation.getStatus() == expectedStatus;
|
||||||
|
}
|
||||||
|
}
|
||||||
|
return false;
|
||||||
|
}
|
||||||
|
|
||||||
|
}
|
||||||
|
|
||||||
|
}
|
|
@ -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.ml.action;
|
||||||
|
|
||||||
|
import org.elasticsearch.xpack.ml.action.CloseJobAction.Response;
|
||||||
|
import org.elasticsearch.xpack.ml.support.AbstractStreamableTestCase;
|
||||||
|
|
||||||
|
public class CloseJobActionResponseTests extends AbstractStreamableTestCase<Response> {
|
||||||
|
|
||||||
|
@Override
|
||||||
|
protected Response createTestInstance() {
|
||||||
|
return new Response(randomBoolean());
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
protected Response createBlankInstance() {
|
||||||
|
return new Response();
|
||||||
|
}
|
||||||
|
}
|
|
@ -25,6 +25,7 @@ import org.elasticsearch.xpack.ml.job.DataCounts;
|
||||||
import org.elasticsearch.xpack.ml.job.DataDescription;
|
import org.elasticsearch.xpack.ml.job.DataDescription;
|
||||||
import org.elasticsearch.xpack.ml.job.Detector;
|
import org.elasticsearch.xpack.ml.job.Detector;
|
||||||
import org.elasticsearch.xpack.ml.job.Job;
|
import org.elasticsearch.xpack.ml.job.Job;
|
||||||
|
import org.elasticsearch.xpack.ml.job.JobStatus;
|
||||||
import org.elasticsearch.xpack.ml.job.metadata.MlMetadata;
|
import org.elasticsearch.xpack.ml.job.metadata.MlMetadata;
|
||||||
import org.elasticsearch.xpack.ml.job.persistence.AnomalyDetectorsIndex;
|
import org.elasticsearch.xpack.ml.job.persistence.AnomalyDetectorsIndex;
|
||||||
import org.elasticsearch.xpack.ml.datafeed.Datafeed;
|
import org.elasticsearch.xpack.ml.datafeed.Datafeed;
|
||||||
|
@ -82,8 +83,12 @@ public class DatafeedJobsIT extends ESIntegTestCase {
|
||||||
PutJobAction.Request putJobRequest = new PutJobAction.Request(job.build(true, job.getId()));
|
PutJobAction.Request putJobRequest = new PutJobAction.Request(job.build(true, job.getId()));
|
||||||
PutJobAction.Response putJobResponse = client().execute(PutJobAction.INSTANCE, putJobRequest).get();
|
PutJobAction.Response putJobResponse = client().execute(PutJobAction.INSTANCE, putJobRequest).get();
|
||||||
assertTrue(putJobResponse.isAcknowledged());
|
assertTrue(putJobResponse.isAcknowledged());
|
||||||
OpenJobAction.Response openJobResponse = client().execute(OpenJobAction.INSTANCE, new OpenJobAction.Request(job.getId())).get();
|
client().execute(InternalOpenJobAction.INSTANCE, new InternalOpenJobAction.Request(job.getId()));
|
||||||
assertTrue(openJobResponse.isAcknowledged());
|
assertBusy(() -> {
|
||||||
|
GetJobsStatsAction.Response statsResponse =
|
||||||
|
client().execute(GetJobsStatsAction.INSTANCE, new GetJobsStatsAction.Request(job.getId())).actionGet();
|
||||||
|
assertEquals(statsResponse.getResponse().results().get(0).getStatus(), JobStatus.OPENED);
|
||||||
|
});
|
||||||
|
|
||||||
DatafeedConfig datafeedConfig = createDatafeed(job.getId() + "-datafeed", job.getId(), Collections.singletonList("data-*"));
|
DatafeedConfig datafeedConfig = createDatafeed(job.getId() + "-datafeed", job.getId(), Collections.singletonList("data-*"));
|
||||||
PutDatafeedAction.Request putDatafeedRequest = new PutDatafeedAction.Request(datafeedConfig);
|
PutDatafeedAction.Request putDatafeedRequest = new PutDatafeedAction.Request(datafeedConfig);
|
||||||
|
@ -119,8 +124,12 @@ public class DatafeedJobsIT extends ESIntegTestCase {
|
||||||
PutJobAction.Request putJobRequest = new PutJobAction.Request(job.build(true, job.getId()));
|
PutJobAction.Request putJobRequest = new PutJobAction.Request(job.build(true, job.getId()));
|
||||||
PutJobAction.Response putJobResponse = client().execute(PutJobAction.INSTANCE, putJobRequest).get();
|
PutJobAction.Response putJobResponse = client().execute(PutJobAction.INSTANCE, putJobRequest).get();
|
||||||
assertTrue(putJobResponse.isAcknowledged());
|
assertTrue(putJobResponse.isAcknowledged());
|
||||||
OpenJobAction.Response openJobResponse = client().execute(OpenJobAction.INSTANCE, new OpenJobAction.Request(job.getId())).get();
|
client().execute(InternalOpenJobAction.INSTANCE, new InternalOpenJobAction.Request(job.getId()));
|
||||||
assertTrue(openJobResponse.isAcknowledged());
|
assertBusy(() -> {
|
||||||
|
GetJobsStatsAction.Response statsResponse =
|
||||||
|
client().execute(GetJobsStatsAction.INSTANCE, new GetJobsStatsAction.Request(job.getId())).actionGet();
|
||||||
|
assertEquals(statsResponse.getResponse().results().get(0).getStatus(), JobStatus.OPENED);
|
||||||
|
});
|
||||||
|
|
||||||
DatafeedConfig datafeedConfig = createDatafeed(job.getId() + "-datafeed", job.getId(), Collections.singletonList("data"));
|
DatafeedConfig datafeedConfig = createDatafeed(job.getId() + "-datafeed", job.getId(), Collections.singletonList("data"));
|
||||||
PutDatafeedAction.Request putDatafeedRequest = new PutDatafeedAction.Request(datafeedConfig);
|
PutDatafeedAction.Request putDatafeedRequest = new PutDatafeedAction.Request(datafeedConfig);
|
||||||
|
@ -255,7 +264,7 @@ public class DatafeedJobsIT extends ESIntegTestCase {
|
||||||
try {
|
try {
|
||||||
CloseJobAction.Response response =
|
CloseJobAction.Response response =
|
||||||
client.execute(CloseJobAction.INSTANCE, new CloseJobAction.Request(jobId)).get();
|
client.execute(CloseJobAction.INSTANCE, new CloseJobAction.Request(jobId)).get();
|
||||||
assertTrue(response.isAcknowledged());
|
assertTrue(response.isClosed());
|
||||||
} catch (Exception e) {
|
} catch (Exception e) {
|
||||||
// ignore
|
// ignore
|
||||||
}
|
}
|
||||||
|
|
|
@ -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.ml.action;
|
||||||
|
|
||||||
|
import org.elasticsearch.xpack.ml.action.OpenJobAction.Response;
|
||||||
|
import org.elasticsearch.xpack.ml.support.AbstractStreamableTestCase;
|
||||||
|
|
||||||
|
public class OpenJobActionResponseTests extends AbstractStreamableTestCase<Response> {
|
||||||
|
|
||||||
|
@Override
|
||||||
|
protected Response createTestInstance() {
|
||||||
|
return new Response(randomBoolean());
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
protected Response createBlankInstance() {
|
||||||
|
return new Response();
|
||||||
|
}
|
||||||
|
}
|
|
@ -0,0 +1,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.ml.action;
|
||||||
|
|
||||||
|
import org.elasticsearch.xpack.ml.action.FlushJobAction.Response;
|
||||||
|
import org.elasticsearch.xpack.ml.support.AbstractStreamableTestCase;
|
||||||
|
|
||||||
|
public class PostDataFlushResponseTests extends AbstractStreamableTestCase<Response> {
|
||||||
|
|
||||||
|
@Override
|
||||||
|
protected Response createTestInstance() {
|
||||||
|
return new Response(randomBoolean());
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
protected Response createBlankInstance() {
|
||||||
|
return new Response();
|
||||||
|
}
|
||||||
|
}
|
|
@ -138,11 +138,11 @@ public class AutodetectResultProcessorIT extends ESSingleNodeTestCase {
|
||||||
ModelSizeStats persistedModelSizeStats = getModelSizeStats();
|
ModelSizeStats persistedModelSizeStats = getModelSizeStats();
|
||||||
assertEquals(modelSizeStats, persistedModelSizeStats);
|
assertEquals(modelSizeStats, persistedModelSizeStats);
|
||||||
|
|
||||||
QueryPage<ModelSnapshot> persistedModelSnapshot = jobProvider.modelSnapshots(JOB_ID, 0, 100);
|
QueryPage<ModelSnapshot> persistedModelSnapshot = getModelSnapshots();
|
||||||
assertEquals(1, persistedModelSnapshot.count());
|
assertEquals(1, persistedModelSnapshot.count());
|
||||||
assertEquals(modelSnapshot, persistedModelSnapshot.results().get(0));
|
assertEquals(modelSnapshot, persistedModelSnapshot.results().get(0));
|
||||||
|
|
||||||
Optional<Quantiles> persistedQuantiles = jobProvider.getQuantiles(JOB_ID);
|
Optional<Quantiles> persistedQuantiles = getQuantiles();
|
||||||
assertTrue(persistedQuantiles.isPresent());
|
assertTrue(persistedQuantiles.isPresent());
|
||||||
assertEquals(quantiles, persistedQuantiles.get());
|
assertEquals(quantiles, persistedQuantiles.get());
|
||||||
}
|
}
|
||||||
|
@ -515,6 +515,7 @@ public class AutodetectResultProcessorIT extends ESSingleNodeTestCase {
|
||||||
}
|
}
|
||||||
return resultHolder.get();
|
return resultHolder.get();
|
||||||
}
|
}
|
||||||
|
|
||||||
private QueryPage<AnomalyRecord> getRecords(RecordsQueryBuilder.RecordsQuery recordsQuery) throws Exception {
|
private QueryPage<AnomalyRecord> getRecords(RecordsQueryBuilder.RecordsQuery recordsQuery) throws Exception {
|
||||||
AtomicReference<Exception> errorHolder = new AtomicReference<>();
|
AtomicReference<Exception> errorHolder = new AtomicReference<>();
|
||||||
AtomicReference<QueryPage<AnomalyRecord>> resultHolder = new AtomicReference<>();
|
AtomicReference<QueryPage<AnomalyRecord>> resultHolder = new AtomicReference<>();
|
||||||
|
@ -532,4 +533,39 @@ public class AutodetectResultProcessorIT extends ESSingleNodeTestCase {
|
||||||
}
|
}
|
||||||
return resultHolder.get();
|
return resultHolder.get();
|
||||||
}
|
}
|
||||||
|
|
||||||
|
private QueryPage<ModelSnapshot> getModelSnapshots() throws Exception {
|
||||||
|
AtomicReference<Exception> errorHolder = new AtomicReference<>();
|
||||||
|
AtomicReference<QueryPage<ModelSnapshot>> resultHolder = new AtomicReference<>();
|
||||||
|
CountDownLatch latch = new CountDownLatch(1);
|
||||||
|
jobProvider.modelSnapshots(JOB_ID, 0, 100, page -> {
|
||||||
|
resultHolder.set(page);
|
||||||
|
latch.countDown();
|
||||||
|
}, e -> {
|
||||||
|
errorHolder.set(e);
|
||||||
|
latch.countDown();
|
||||||
|
});
|
||||||
|
latch.await();
|
||||||
|
if (errorHolder.get() != null) {
|
||||||
|
throw errorHolder.get();
|
||||||
|
}
|
||||||
|
return resultHolder.get();
|
||||||
|
}
|
||||||
|
private Optional<Quantiles> getQuantiles() throws Exception {
|
||||||
|
AtomicReference<Exception> errorHolder = new AtomicReference<>();
|
||||||
|
AtomicReference<Optional<Quantiles>> resultHolder = new AtomicReference<>();
|
||||||
|
CountDownLatch latch = new CountDownLatch(1);
|
||||||
|
jobProvider.getQuantiles(JOB_ID, q -> {
|
||||||
|
resultHolder.set(Optional.ofNullable(q));
|
||||||
|
latch.countDown();
|
||||||
|
}, e -> {
|
||||||
|
errorHolder.set(e);
|
||||||
|
latch.countDown();
|
||||||
|
});
|
||||||
|
latch.await();
|
||||||
|
if (errorHolder.get() != null) {
|
||||||
|
throw errorHolder.get();
|
||||||
|
}
|
||||||
|
return resultHolder.get();
|
||||||
|
}
|
||||||
}
|
}
|
||||||
|
|
|
@ -13,11 +13,11 @@ import org.elasticsearch.cluster.ClusterState;
|
||||||
import org.elasticsearch.cluster.metadata.MetaData;
|
import org.elasticsearch.cluster.metadata.MetaData;
|
||||||
import org.elasticsearch.common.io.stream.NamedWriteableRegistry;
|
import org.elasticsearch.common.io.stream.NamedWriteableRegistry;
|
||||||
import org.elasticsearch.common.settings.Settings;
|
import org.elasticsearch.common.settings.Settings;
|
||||||
import org.elasticsearch.common.unit.TimeValue;
|
|
||||||
import org.elasticsearch.plugins.Plugin;
|
import org.elasticsearch.plugins.Plugin;
|
||||||
import org.elasticsearch.rest.RestStatus;
|
import org.elasticsearch.rest.RestStatus;
|
||||||
import org.elasticsearch.test.ESIntegTestCase;
|
import org.elasticsearch.test.ESIntegTestCase;
|
||||||
import org.elasticsearch.xpack.ml.MlPlugin;
|
import org.elasticsearch.xpack.ml.MlPlugin;
|
||||||
|
import org.elasticsearch.xpack.ml.action.GetJobsStatsAction;
|
||||||
import org.elasticsearch.xpack.ml.action.OpenJobAction;
|
import org.elasticsearch.xpack.ml.action.OpenJobAction;
|
||||||
import org.elasticsearch.xpack.ml.action.PutJobAction;
|
import org.elasticsearch.xpack.ml.action.PutJobAction;
|
||||||
import org.elasticsearch.xpack.ml.action.DatafeedJobsIT;
|
import org.elasticsearch.xpack.ml.action.DatafeedJobsIT;
|
||||||
|
@ -25,6 +25,7 @@ import org.elasticsearch.xpack.ml.job.AnalysisConfig;
|
||||||
import org.elasticsearch.xpack.ml.job.DataDescription;
|
import org.elasticsearch.xpack.ml.job.DataDescription;
|
||||||
import org.elasticsearch.xpack.ml.job.Detector;
|
import org.elasticsearch.xpack.ml.job.Detector;
|
||||||
import org.elasticsearch.xpack.ml.job.Job;
|
import org.elasticsearch.xpack.ml.job.Job;
|
||||||
|
import org.elasticsearch.xpack.ml.job.JobStatus;
|
||||||
import org.elasticsearch.xpack.ml.job.manager.AutodetectProcessManager;
|
import org.elasticsearch.xpack.ml.job.manager.AutodetectProcessManager;
|
||||||
import org.elasticsearch.xpack.ml.job.metadata.MlMetadata;
|
import org.elasticsearch.xpack.ml.job.metadata.MlMetadata;
|
||||||
import org.junit.After;
|
import org.junit.After;
|
||||||
|
@ -68,20 +69,21 @@ public class TooManyJobsIT extends ESIntegTestCase {
|
||||||
|
|
||||||
try {
|
try {
|
||||||
OpenJobAction.Request openJobRequest = new OpenJobAction.Request(job.getId());
|
OpenJobAction.Request openJobRequest = new OpenJobAction.Request(job.getId());
|
||||||
openJobRequest.setOpenTimeout(TimeValue.timeValueSeconds(10));
|
OpenJobAction.Response openJobResponse = client().execute(OpenJobAction.INSTANCE, openJobRequest).get();
|
||||||
OpenJobAction.Response openJobResponse = client().execute(OpenJobAction.INSTANCE, openJobRequest)
|
assertTrue(openJobResponse.isOpened());
|
||||||
.get();
|
assertBusy(() -> {
|
||||||
assertTrue(openJobResponse.isAcknowledged());
|
GetJobsStatsAction.Response statsResponse =
|
||||||
|
client().execute(GetJobsStatsAction.INSTANCE, new GetJobsStatsAction.Request(job.getId())).actionGet();
|
||||||
|
assertEquals(statsResponse.getResponse().results().get(0).getStatus(), JobStatus.OPENED);
|
||||||
|
});
|
||||||
logger.info("Opened {}th job", i);
|
logger.info("Opened {}th job", i);
|
||||||
} catch (Exception e) {
|
} catch (Exception e) {
|
||||||
Throwable cause = ExceptionsHelper.unwrapCause(e.getCause());
|
Throwable cause = e.getCause().getCause();
|
||||||
if (ElasticsearchStatusException.class.equals(cause.getClass()) == false) {
|
if (IllegalArgumentException.class.equals(cause.getClass()) == false) {
|
||||||
logger.warn("Unexpected cause", e);
|
logger.warn("Unexpected cause", e);
|
||||||
}
|
}
|
||||||
assertEquals(ElasticsearchStatusException.class, cause.getClass());
|
assertEquals(IllegalArgumentException.class, cause.getClass());
|
||||||
assertEquals(RestStatus.CONFLICT, ((ElasticsearchStatusException) cause).status());
|
assertEquals("Timeout expired while waiting for job status to change to [OPENED]", cause.getMessage());
|
||||||
assertEquals("[" + (maxRunningJobsPerNode + 1) + "] expected job status [OPENED], but got [FAILED], reason " +
|
|
||||||
"[failed to open, max running job capacity [" + maxRunningJobsPerNode + "] reached]", cause.getMessage());
|
|
||||||
logger.info("good news everybody --> reached maximum number of allowed opened jobs, after trying to open the {}th job", i);
|
logger.info("good news everybody --> reached maximum number of allowed opened jobs, after trying to open the {}th job", i);
|
||||||
|
|
||||||
// now manually clean things up and see if we can succeed to run one new job
|
// now manually clean things up and see if we can succeed to run one new job
|
||||||
|
@ -90,7 +92,12 @@ public class TooManyJobsIT extends ESIntegTestCase {
|
||||||
assertTrue(putJobResponse.isAcknowledged());
|
assertTrue(putJobResponse.isAcknowledged());
|
||||||
OpenJobAction.Response openJobResponse = client().execute(OpenJobAction.INSTANCE, new OpenJobAction.Request(job.getId()))
|
OpenJobAction.Response openJobResponse = client().execute(OpenJobAction.INSTANCE, new OpenJobAction.Request(job.getId()))
|
||||||
.get();
|
.get();
|
||||||
assertTrue(openJobResponse.isAcknowledged());
|
assertTrue(openJobResponse.isOpened());
|
||||||
|
assertBusy(() -> {
|
||||||
|
GetJobsStatsAction.Response statsResponse =
|
||||||
|
client().execute(GetJobsStatsAction.INSTANCE, new GetJobsStatsAction.Request(job.getId())).actionGet();
|
||||||
|
assertEquals(statsResponse.getResponse().results().get(0).getStatus(), JobStatus.OPENED);
|
||||||
|
});
|
||||||
return;
|
return;
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
|
@ -20,6 +20,7 @@ import org.elasticsearch.xpack.ml.job.DataDescription;
|
||||||
import org.elasticsearch.xpack.ml.job.Detector;
|
import org.elasticsearch.xpack.ml.job.Detector;
|
||||||
import org.elasticsearch.xpack.ml.job.Job;
|
import org.elasticsearch.xpack.ml.job.Job;
|
||||||
import org.elasticsearch.xpack.ml.job.JobStatus;
|
import org.elasticsearch.xpack.ml.job.JobStatus;
|
||||||
|
import org.elasticsearch.xpack.ml.job.ModelSnapshot;
|
||||||
import org.elasticsearch.xpack.ml.job.data.DataProcessor;
|
import org.elasticsearch.xpack.ml.job.data.DataProcessor;
|
||||||
import org.elasticsearch.xpack.ml.job.metadata.Allocation;
|
import org.elasticsearch.xpack.ml.job.metadata.Allocation;
|
||||||
import org.elasticsearch.xpack.ml.job.persistence.JobDataCountsPersister;
|
import org.elasticsearch.xpack.ml.job.persistence.JobDataCountsPersister;
|
||||||
|
@ -34,7 +35,9 @@ import org.elasticsearch.xpack.ml.job.process.autodetect.params.DataLoadParams;
|
||||||
import org.elasticsearch.xpack.ml.job.process.autodetect.params.InterimResultsParams;
|
import org.elasticsearch.xpack.ml.job.process.autodetect.params.InterimResultsParams;
|
||||||
import org.elasticsearch.xpack.ml.job.process.autodetect.params.TimeRange;
|
import org.elasticsearch.xpack.ml.job.process.autodetect.params.TimeRange;
|
||||||
import org.elasticsearch.xpack.ml.job.process.normalizer.NormalizerFactory;
|
import org.elasticsearch.xpack.ml.job.process.normalizer.NormalizerFactory;
|
||||||
|
import org.elasticsearch.xpack.ml.job.quantiles.Quantiles;
|
||||||
import org.elasticsearch.xpack.ml.job.results.AutodetectResult;
|
import org.elasticsearch.xpack.ml.job.results.AutodetectResult;
|
||||||
|
import org.elasticsearch.xpack.ml.lists.ListDocument;
|
||||||
import org.junit.Before;
|
import org.junit.Before;
|
||||||
import org.mockito.Mockito;
|
import org.mockito.Mockito;
|
||||||
|
|
||||||
|
@ -43,14 +46,17 @@ import java.io.IOException;
|
||||||
import java.io.InputStream;
|
import java.io.InputStream;
|
||||||
import java.nio.charset.StandardCharsets;
|
import java.nio.charset.StandardCharsets;
|
||||||
import java.util.Collections;
|
import java.util.Collections;
|
||||||
|
import java.util.Date;
|
||||||
|
import java.util.HashSet;
|
||||||
import java.util.Iterator;
|
import java.util.Iterator;
|
||||||
|
import java.util.Set;
|
||||||
import java.util.Optional;
|
import java.util.Optional;
|
||||||
import java.util.concurrent.ExecutorService;
|
import java.util.concurrent.ExecutorService;
|
||||||
import java.util.function.Consumer;
|
import java.util.function.Consumer;
|
||||||
import java.util.function.Supplier;
|
|
||||||
import java.util.stream.Stream;
|
import java.util.stream.Stream;
|
||||||
|
|
||||||
import static org.elasticsearch.mock.orig.Mockito.doAnswer;
|
import static org.elasticsearch.mock.orig.Mockito.doAnswer;
|
||||||
|
import static org.elasticsearch.mock.orig.Mockito.doReturn;
|
||||||
import static org.elasticsearch.mock.orig.Mockito.doThrow;
|
import static org.elasticsearch.mock.orig.Mockito.doThrow;
|
||||||
import static org.elasticsearch.mock.orig.Mockito.times;
|
import static org.elasticsearch.mock.orig.Mockito.times;
|
||||||
import static org.elasticsearch.mock.orig.Mockito.verify;
|
import static org.elasticsearch.mock.orig.Mockito.verify;
|
||||||
|
@ -60,12 +66,11 @@ import static org.mockito.Matchers.any;
|
||||||
import static org.mockito.Matchers.anyBoolean;
|
import static org.mockito.Matchers.anyBoolean;
|
||||||
import static org.mockito.Matchers.anyString;
|
import static org.mockito.Matchers.anyString;
|
||||||
import static org.mockito.Matchers.eq;
|
import static org.mockito.Matchers.eq;
|
||||||
import static org.mockito.Mockito.doReturn;
|
|
||||||
import static org.mockito.Mockito.mock;
|
import static org.mockito.Mockito.mock;
|
||||||
import static org.mockito.Mockito.spy;
|
import static org.mockito.Mockito.spy;
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* Calling the {@link DataProcessor#processData(String, InputStream, DataLoadParams, java.util.function.Supplier)}
|
* Calling the {@link DataProcessor#processData(String, InputStream, DataLoadParams)}
|
||||||
* method causes an AutodetectCommunicator to be created on demand. Most of these tests have to
|
* method causes an AutodetectCommunicator to be created on demand. Most of these tests have to
|
||||||
* do that before they can assert other things
|
* do that before they can assert other things
|
||||||
*/
|
*/
|
||||||
|
@ -95,7 +100,7 @@ public class AutodetectProcessManagerTests extends ESTestCase {
|
||||||
when(jobManager.getJobOrThrowIfUnknown("foo")).thenReturn(createJobDetails("foo"));
|
when(jobManager.getJobOrThrowIfUnknown("foo")).thenReturn(createJobDetails("foo"));
|
||||||
AutodetectProcessManager manager = createManager(communicator, client);
|
AutodetectProcessManager manager = createManager(communicator, client);
|
||||||
|
|
||||||
manager.openJob("foo", false);
|
manager.openJob("foo", false, e -> {});
|
||||||
assertEquals(1, manager.numberOfOpenJobs());
|
assertEquals(1, manager.numberOfOpenJobs());
|
||||||
assertTrue(manager.jobHasActiveAutodetectProcess("foo"));
|
assertTrue(manager.jobHasActiveAutodetectProcess("foo"));
|
||||||
UpdateJobStatusAction.Request expectedRequest = new UpdateJobStatusAction.Request("foo", JobStatus.OPENED);
|
UpdateJobStatusAction.Request expectedRequest = new UpdateJobStatusAction.Request("foo", JobStatus.OPENED);
|
||||||
|
@ -137,24 +142,33 @@ public class AutodetectProcessManagerTests extends ESTestCase {
|
||||||
AutodetectProcess autodetectProcess = mock(AutodetectProcess.class);
|
AutodetectProcess autodetectProcess = mock(AutodetectProcess.class);
|
||||||
when(autodetectProcess.isProcessAlive()).thenReturn(true);
|
when(autodetectProcess.isProcessAlive()).thenReturn(true);
|
||||||
when(autodetectProcess.getPersistStream()).thenReturn(new ByteArrayInputStream(new byte[0]));
|
when(autodetectProcess.getPersistStream()).thenReturn(new ByteArrayInputStream(new byte[0]));
|
||||||
AutodetectProcessFactory autodetectProcessFactory = (j, i, e) -> autodetectProcess;
|
AutodetectProcessFactory autodetectProcessFactory = (j, modelSnapshot, quantiles, lists, i, e) -> autodetectProcess;
|
||||||
Settings.Builder settings = Settings.builder();
|
Settings.Builder settings = Settings.builder();
|
||||||
settings.put(AutodetectProcessManager.MAX_RUNNING_JOBS_PER_NODE.getKey(), 3);
|
settings.put(AutodetectProcessManager.MAX_RUNNING_JOBS_PER_NODE.getKey(), 3);
|
||||||
AutodetectProcessManager manager = new AutodetectProcessManager(settings.build(), client, threadPool, jobManager, jobProvider,
|
AutodetectProcessManager manager = spy(new AutodetectProcessManager(settings.build(), client, threadPool, jobManager, jobProvider,
|
||||||
jobResultsPersister, jobRenormalizedResultsPersister, jobDataCountsPersister, parser, autodetectProcessFactory,
|
jobResultsPersister, jobRenormalizedResultsPersister, jobDataCountsPersister, parser, autodetectProcessFactory,
|
||||||
normalizerFactory);
|
normalizerFactory));
|
||||||
|
|
||||||
manager.openJob("foo", false);
|
ModelSnapshot modelSnapshot = new ModelSnapshot("foo");
|
||||||
manager.openJob("bar", false);
|
Quantiles quantiles = new Quantiles("foo", new Date(), "state");
|
||||||
manager.openJob("baz", false);
|
Set<ListDocument> lists = new HashSet<>();
|
||||||
|
doAnswer(invocationOnMock -> {
|
||||||
|
AutodetectProcessManager.TriConsumer consumer = (AutodetectProcessManager.TriConsumer) invocationOnMock.getArguments()[1];
|
||||||
|
consumer.accept(modelSnapshot, quantiles, lists);
|
||||||
|
return null;
|
||||||
|
}).when(manager).gatherRequiredInformation(any(), any(), any());
|
||||||
|
|
||||||
|
manager.openJob("foo", false, e -> {});
|
||||||
|
manager.openJob("bar", false, e -> {});
|
||||||
|
manager.openJob("baz", false, e -> {});
|
||||||
assertEquals(3, manager.numberOfOpenJobs());
|
assertEquals(3, manager.numberOfOpenJobs());
|
||||||
|
|
||||||
Exception e = expectThrows(ElasticsearchStatusException.class, () -> manager.openJob("foobar", false));
|
Exception e = expectThrows(ElasticsearchStatusException.class, () -> manager.openJob("foobar", false, e1 -> {}));
|
||||||
assertEquals("max running job capacity [3] reached", e.getMessage());
|
assertEquals("max running job capacity [3] reached", e.getMessage());
|
||||||
|
|
||||||
manager.closeJob("baz");
|
manager.closeJob("baz");
|
||||||
assertEquals(2, manager.numberOfOpenJobs());
|
assertEquals(2, manager.numberOfOpenJobs());
|
||||||
manager.openJob("foobar", false);
|
manager.openJob("foobar", false, e1 -> {});
|
||||||
assertEquals(3, manager.numberOfOpenJobs());
|
assertEquals(3, manager.numberOfOpenJobs());
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@ -164,8 +178,8 @@ public class AutodetectProcessManagerTests extends ESTestCase {
|
||||||
assertEquals(0, manager.numberOfOpenJobs());
|
assertEquals(0, manager.numberOfOpenJobs());
|
||||||
|
|
||||||
DataLoadParams params = new DataLoadParams(TimeRange.builder().build(), false, Optional.empty());
|
DataLoadParams params = new DataLoadParams(TimeRange.builder().build(), false, Optional.empty());
|
||||||
manager.openJob("foo", false);
|
manager.openJob("foo", false, e -> {});
|
||||||
manager.processData("foo", createInputStream(""), params, () -> false);
|
manager.processData("foo", createInputStream(""), params);
|
||||||
assertEquals(1, manager.numberOfOpenJobs());
|
assertEquals(1, manager.numberOfOpenJobs());
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@ -175,12 +189,11 @@ public class AutodetectProcessManagerTests extends ESTestCase {
|
||||||
|
|
||||||
DataLoadParams params = mock(DataLoadParams.class);
|
DataLoadParams params = mock(DataLoadParams.class);
|
||||||
InputStream inputStream = createInputStream("");
|
InputStream inputStream = createInputStream("");
|
||||||
Supplier<Boolean> cancellable = () -> false;
|
doThrow(new IOException("blah")).when(communicator).writeToJob(inputStream, params);
|
||||||
doThrow(new IOException("blah")).when(communicator).writeToJob(inputStream, params, cancellable);
|
|
||||||
|
|
||||||
manager.openJob("foo", false);
|
manager.openJob("foo", false, e -> {});
|
||||||
ESTestCase.expectThrows(ElasticsearchException.class,
|
ESTestCase.expectThrows(ElasticsearchException.class,
|
||||||
() -> manager.processData("foo", inputStream, params, cancellable));
|
() -> manager.processData("foo", inputStream, params));
|
||||||
}
|
}
|
||||||
|
|
||||||
public void testCloseJob() {
|
public void testCloseJob() {
|
||||||
|
@ -189,8 +202,8 @@ public class AutodetectProcessManagerTests extends ESTestCase {
|
||||||
AutodetectProcessManager manager = createManager(communicator);
|
AutodetectProcessManager manager = createManager(communicator);
|
||||||
assertEquals(0, manager.numberOfOpenJobs());
|
assertEquals(0, manager.numberOfOpenJobs());
|
||||||
|
|
||||||
manager.openJob("foo", false);
|
manager.openJob("foo", false, e -> {});
|
||||||
manager.processData("foo", createInputStream(""), mock(DataLoadParams.class), () -> false);
|
manager.processData("foo", createInputStream(""), mock(DataLoadParams.class));
|
||||||
|
|
||||||
// job is created
|
// job is created
|
||||||
assertEquals(1, manager.numberOfOpenJobs());
|
assertEquals(1, manager.numberOfOpenJobs());
|
||||||
|
@ -202,12 +215,11 @@ public class AutodetectProcessManagerTests extends ESTestCase {
|
||||||
AutodetectCommunicator communicator = mock(AutodetectCommunicator.class);
|
AutodetectCommunicator communicator = mock(AutodetectCommunicator.class);
|
||||||
AutodetectProcessManager manager = createManager(communicator);
|
AutodetectProcessManager manager = createManager(communicator);
|
||||||
|
|
||||||
Supplier<Boolean> cancellable = () -> false;
|
|
||||||
DataLoadParams params = new DataLoadParams(TimeRange.builder().startTime("1000").endTime("2000").build(), true, Optional.empty());
|
DataLoadParams params = new DataLoadParams(TimeRange.builder().startTime("1000").endTime("2000").build(), true, Optional.empty());
|
||||||
InputStream inputStream = createInputStream("");
|
InputStream inputStream = createInputStream("");
|
||||||
manager.openJob("foo", false);
|
manager.openJob("foo", false, e -> {});
|
||||||
manager.processData("foo", inputStream, params, cancellable);
|
manager.processData("foo", inputStream, params);
|
||||||
verify(communicator).writeToJob(inputStream, params, cancellable);
|
verify(communicator).writeToJob(inputStream, params);
|
||||||
}
|
}
|
||||||
|
|
||||||
public void testFlush() throws IOException {
|
public void testFlush() throws IOException {
|
||||||
|
@ -216,8 +228,8 @@ public class AutodetectProcessManagerTests extends ESTestCase {
|
||||||
when(jobManager.getJobOrThrowIfUnknown("foo")).thenReturn(createJobDetails("foo"));
|
when(jobManager.getJobOrThrowIfUnknown("foo")).thenReturn(createJobDetails("foo"));
|
||||||
|
|
||||||
InputStream inputStream = createInputStream("");
|
InputStream inputStream = createInputStream("");
|
||||||
manager.openJob("foo", false);
|
manager.openJob("foo", false, e -> {});
|
||||||
manager.processData("foo", inputStream, mock(DataLoadParams.class), () -> false);
|
manager.processData("foo", inputStream, mock(DataLoadParams.class));
|
||||||
|
|
||||||
InterimResultsParams params = InterimResultsParams.builder().build();
|
InterimResultsParams params = InterimResultsParams.builder().build();
|
||||||
manager.flushJob("foo", params);
|
manager.flushJob("foo", params);
|
||||||
|
@ -248,8 +260,8 @@ public class AutodetectProcessManagerTests extends ESTestCase {
|
||||||
AutodetectProcessManager manager = createManager(communicator);
|
AutodetectProcessManager manager = createManager(communicator);
|
||||||
assertFalse(manager.jobHasActiveAutodetectProcess("foo"));
|
assertFalse(manager.jobHasActiveAutodetectProcess("foo"));
|
||||||
|
|
||||||
manager.openJob("foo", false);
|
manager.openJob("foo", false, e -> {});
|
||||||
manager.processData("foo", createInputStream(""), mock(DataLoadParams.class), () -> false);
|
manager.processData("foo", createInputStream(""), mock(DataLoadParams.class));
|
||||||
|
|
||||||
assertTrue(manager.jobHasActiveAutodetectProcess("foo"));
|
assertTrue(manager.jobHasActiveAutodetectProcess("foo"));
|
||||||
assertFalse(manager.jobHasActiveAutodetectProcess("bar"));
|
assertFalse(manager.jobHasActiveAutodetectProcess("bar"));
|
||||||
|
@ -257,7 +269,7 @@ public class AutodetectProcessManagerTests extends ESTestCase {
|
||||||
|
|
||||||
public void testProcessData_GivenStatusNotStarted() throws IOException {
|
public void testProcessData_GivenStatusNotStarted() throws IOException {
|
||||||
AutodetectCommunicator communicator = mock(AutodetectCommunicator.class);
|
AutodetectCommunicator communicator = mock(AutodetectCommunicator.class);
|
||||||
when(communicator.writeToJob(any(), any(), any())).thenReturn(new DataCounts("foo"));
|
when(communicator.writeToJob(any(), any())).thenReturn(new DataCounts("foo"));
|
||||||
AutodetectProcessManager manager = createManager(communicator);
|
AutodetectProcessManager manager = createManager(communicator);
|
||||||
|
|
||||||
Job job = createJobDetails("foo");
|
Job job = createJobDetails("foo");
|
||||||
|
@ -266,8 +278,8 @@ public class AutodetectProcessManagerTests extends ESTestCase {
|
||||||
givenAllocationWithStatus(JobStatus.OPENED);
|
givenAllocationWithStatus(JobStatus.OPENED);
|
||||||
|
|
||||||
InputStream inputStream = createInputStream("");
|
InputStream inputStream = createInputStream("");
|
||||||
manager.openJob("foo", false);
|
manager.openJob("foo", false, e -> {});
|
||||||
DataCounts dataCounts = manager.processData("foo", inputStream, mock(DataLoadParams.class), () -> false);
|
DataCounts dataCounts = manager.processData("foo", inputStream, mock(DataLoadParams.class));
|
||||||
|
|
||||||
assertThat(dataCounts, equalTo(new DataCounts("foo")));
|
assertThat(dataCounts, equalTo(new DataCounts("foo")));
|
||||||
}
|
}
|
||||||
|
@ -289,12 +301,20 @@ public class AutodetectProcessManagerTests extends ESTestCase {
|
||||||
|
|
||||||
AutodetectResultsParser parser = mock(AutodetectResultsParser.class);
|
AutodetectResultsParser parser = mock(AutodetectResultsParser.class);
|
||||||
AutodetectProcess autodetectProcess = mock(AutodetectProcess.class);
|
AutodetectProcess autodetectProcess = mock(AutodetectProcess.class);
|
||||||
AutodetectProcessFactory autodetectProcessFactory = (j, i, e) -> autodetectProcess;
|
AutodetectProcessFactory autodetectProcessFactory = (j, modelSnapshot, quantiles, lists, i, e) -> autodetectProcess;
|
||||||
AutodetectProcessManager manager = new AutodetectProcessManager(Settings.EMPTY, client, threadPool, jobManager, jobProvider,
|
AutodetectProcessManager manager = spy(new AutodetectProcessManager(Settings.EMPTY, client, threadPool, jobManager, jobProvider,
|
||||||
jobResultsPersister, jobRenormalizedResultsPersister, jobDataCountsPersister, parser, autodetectProcessFactory,
|
jobResultsPersister, jobRenormalizedResultsPersister, jobDataCountsPersister, parser, autodetectProcessFactory,
|
||||||
normalizerFactory);
|
normalizerFactory));
|
||||||
|
ModelSnapshot modelSnapshot = new ModelSnapshot("foo");
|
||||||
|
Quantiles quantiles = new Quantiles("foo", new Date(), "state");
|
||||||
|
Set<ListDocument> lists = new HashSet<>();
|
||||||
|
doAnswer(invocationOnMock -> {
|
||||||
|
AutodetectProcessManager.TriConsumer consumer = (AutodetectProcessManager.TriConsumer) invocationOnMock.getArguments()[1];
|
||||||
|
consumer.accept(modelSnapshot, quantiles, lists);
|
||||||
|
return null;
|
||||||
|
}).when(manager).gatherRequiredInformation(any(), any(), any());
|
||||||
|
|
||||||
expectThrows(EsRejectedExecutionException.class, () -> manager.create("my_id", false));
|
expectThrows(EsRejectedExecutionException.class, () -> manager.create("my_id", modelSnapshot, quantiles, lists, false, e -> {}));
|
||||||
verify(autodetectProcess, times(1)).close();
|
verify(autodetectProcess, times(1)).close();
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@ -317,14 +337,22 @@ public class AutodetectProcessManagerTests extends ESTestCase {
|
||||||
jobResultsPersister, jobRenormalizedResultsPersister, jobDataCountsPersister, parser, autodetectProcessFactory,
|
jobResultsPersister, jobRenormalizedResultsPersister, jobDataCountsPersister, parser, autodetectProcessFactory,
|
||||||
normalizerFactory);
|
normalizerFactory);
|
||||||
manager = spy(manager);
|
manager = spy(manager);
|
||||||
doReturn(communicator).when(manager).create(any(), anyBoolean());
|
ModelSnapshot modelSnapshot = new ModelSnapshot("foo");
|
||||||
|
Quantiles quantiles = new Quantiles("foo", new Date(), "state");
|
||||||
|
Set<ListDocument> lists = new HashSet<>();
|
||||||
|
doAnswer(invocationOnMock -> {
|
||||||
|
AutodetectProcessManager.TriConsumer consumer = (AutodetectProcessManager.TriConsumer) invocationOnMock.getArguments()[1];
|
||||||
|
consumer.accept(modelSnapshot, quantiles, lists);
|
||||||
|
return null;
|
||||||
|
}).when(manager).gatherRequiredInformation(any(), any(), any());
|
||||||
|
doReturn(communicator).when(manager).create(any(), eq(modelSnapshot), eq(quantiles), eq(lists), anyBoolean(), any());
|
||||||
return manager;
|
return manager;
|
||||||
}
|
}
|
||||||
|
|
||||||
private AutodetectProcessManager createManagerAndCallProcessData(AutodetectCommunicator communicator, String jobId) {
|
private AutodetectProcessManager createManagerAndCallProcessData(AutodetectCommunicator communicator, String jobId) {
|
||||||
AutodetectProcessManager manager = createManager(communicator);
|
AutodetectProcessManager manager = createManager(communicator);
|
||||||
manager.openJob(jobId, false);
|
manager.openJob(jobId, false, e -> {});
|
||||||
manager.processData(jobId, createInputStream(""), mock(DataLoadParams.class), () -> false);
|
manager.processData(jobId, createInputStream(""), mock(DataLoadParams.class));
|
||||||
return manager;
|
return manager;
|
||||||
}
|
}
|
||||||
|
|
||||||
|
|
|
@ -1,182 +0,0 @@
|
||||||
/*
|
|
||||||
* Copyright Elasticsearch B.V. and/or licensed to Elasticsearch B.V. under one
|
|
||||||
* or more contributor license agreements. Licensed under the Elastic License;
|
|
||||||
* you may not use this file except in compliance with the Elastic License.
|
|
||||||
*/
|
|
||||||
package org.elasticsearch.xpack.ml.job.metadata;
|
|
||||||
|
|
||||||
import org.elasticsearch.Version;
|
|
||||||
import org.elasticsearch.cluster.ClusterChangedEvent;
|
|
||||||
import org.elasticsearch.cluster.ClusterName;
|
|
||||||
import org.elasticsearch.cluster.ClusterState;
|
|
||||||
import org.elasticsearch.cluster.metadata.MetaData;
|
|
||||||
import org.elasticsearch.cluster.node.DiscoveryNode;
|
|
||||||
import org.elasticsearch.cluster.node.DiscoveryNodes;
|
|
||||||
import org.elasticsearch.cluster.service.ClusterService;
|
|
||||||
import org.elasticsearch.common.settings.Settings;
|
|
||||||
import org.elasticsearch.common.transport.TransportAddress;
|
|
||||||
import org.elasticsearch.test.ESTestCase;
|
|
||||||
import org.elasticsearch.threadpool.ThreadPool;
|
|
||||||
import org.elasticsearch.xpack.ml.job.JobStatus;
|
|
||||||
import org.junit.Before;
|
|
||||||
|
|
||||||
import java.net.InetAddress;
|
|
||||||
import java.util.concurrent.ExecutorService;
|
|
||||||
|
|
||||||
import static org.mockito.Mockito.doAnswer;
|
|
||||||
import static org.mockito.Mockito.mock;
|
|
||||||
import static org.mockito.Mockito.never;
|
|
||||||
import static org.mockito.Mockito.times;
|
|
||||||
import static org.mockito.Mockito.verify;
|
|
||||||
import static org.mockito.Mockito.when;
|
|
||||||
import static org.elasticsearch.xpack.ml.job.JobTests.buildJobBuilder;
|
|
||||||
import static org.mockito.Matchers.any;
|
|
||||||
|
|
||||||
public class JobAllocatorTests extends ESTestCase {
|
|
||||||
|
|
||||||
private ClusterService clusterService;
|
|
||||||
private ThreadPool threadPool;
|
|
||||||
private JobAllocator jobAllocator;
|
|
||||||
|
|
||||||
@Before
|
|
||||||
public void instantiateJobAllocator() {
|
|
||||||
clusterService = mock(ClusterService.class);
|
|
||||||
threadPool = mock(ThreadPool.class);
|
|
||||||
jobAllocator = new JobAllocator(Settings.EMPTY, clusterService, threadPool);
|
|
||||||
}
|
|
||||||
|
|
||||||
public void testShouldAllocate() {
|
|
||||||
ClusterState cs = ClusterState.builder(new ClusterName("_cluster_name")).metaData(MetaData.builder()
|
|
||||||
.putCustom(MlMetadata.TYPE, new MlMetadata.Builder().build()))
|
|
||||||
.build();
|
|
||||||
assertFalse("No jobs, so nothing to allocate", jobAllocator.shouldAllocate(cs));
|
|
||||||
|
|
||||||
MlMetadata.Builder pmBuilder = new MlMetadata.Builder(cs.metaData().custom(MlMetadata.TYPE));
|
|
||||||
pmBuilder.putJob((buildJobBuilder("my_job_id").build()), false);
|
|
||||||
cs = ClusterState.builder(cs).metaData(MetaData.builder()
|
|
||||||
.putCustom(MlMetadata.TYPE, pmBuilder.build()))
|
|
||||||
.build();
|
|
||||||
assertTrue("A unassigned job, so we should allocate", jobAllocator.shouldAllocate(cs));
|
|
||||||
|
|
||||||
pmBuilder = new MlMetadata.Builder(cs.metaData().custom(MlMetadata.TYPE));
|
|
||||||
pmBuilder.assignToNode("my_job_id", "_node_id");
|
|
||||||
cs = ClusterState.builder(cs).metaData(MetaData.builder()
|
|
||||||
.putCustom(MlMetadata.TYPE, pmBuilder.build()))
|
|
||||||
.build();
|
|
||||||
assertFalse("Job is allocate, so nothing to allocate", jobAllocator.shouldAllocate(cs));
|
|
||||||
}
|
|
||||||
|
|
||||||
public void testAssignJobsToNodes() throws Exception {
|
|
||||||
MlMetadata.Builder pmBuilder = new MlMetadata.Builder();
|
|
||||||
pmBuilder.putJob(buildJobBuilder("my_job_id").build(), false);
|
|
||||||
ClusterState cs1 = ClusterState.builder(new ClusterName("_cluster_name")).metaData(MetaData.builder()
|
|
||||||
.putCustom(MlMetadata.TYPE, pmBuilder.build()))
|
|
||||||
.nodes(DiscoveryNodes.builder()
|
|
||||||
.add(new DiscoveryNode("_node_id", new TransportAddress(InetAddress.getLoopbackAddress(), 9200), Version.CURRENT))
|
|
||||||
.masterNodeId("_node_id"))
|
|
||||||
.build();
|
|
||||||
ClusterState result1 = jobAllocator.assignJobsToNodes(cs1);
|
|
||||||
MlMetadata pm = result1.metaData().custom(MlMetadata.TYPE);
|
|
||||||
assertEquals("my_job_id must be allocated to _node_id", pm.getAllocations().get("my_job_id").getNodeId(), "_node_id");
|
|
||||||
|
|
||||||
ClusterState result2 = jobAllocator.assignJobsToNodes(result1);
|
|
||||||
assertSame("job has been allocated, same instance must be returned", result1, result2);
|
|
||||||
|
|
||||||
ClusterState cs2 = ClusterState.builder(new ClusterName("_cluster_name")).metaData(MetaData.builder()
|
|
||||||
.putCustom(MlMetadata.TYPE, pmBuilder.build()))
|
|
||||||
.nodes(
|
|
||||||
DiscoveryNodes.builder()
|
|
||||||
.add(new DiscoveryNode("_node_id1", new TransportAddress(InetAddress.getLoopbackAddress(), 9200), Version.CURRENT))
|
|
||||||
.add(new DiscoveryNode("_node_id2", new TransportAddress(InetAddress.getLoopbackAddress(), 9201), Version.CURRENT))
|
|
||||||
.masterNodeId("_node_id1")
|
|
||||||
)
|
|
||||||
.build();
|
|
||||||
// should fail, ml only support single node for now
|
|
||||||
expectThrows(IllegalStateException.class, () -> jobAllocator.assignJobsToNodes(cs2));
|
|
||||||
|
|
||||||
ClusterState cs3 = ClusterState.builder(new ClusterName("_cluster_name")).metaData(MetaData.builder()
|
|
||||||
.putCustom(MlMetadata.TYPE, pmBuilder.build()))
|
|
||||||
.build();
|
|
||||||
// we need to have at least one node
|
|
||||||
expectThrows(IllegalStateException.class, () -> jobAllocator.assignJobsToNodes(cs3));
|
|
||||||
|
|
||||||
pmBuilder = new MlMetadata.Builder(result1.getMetaData().custom(MlMetadata.TYPE));
|
|
||||||
pmBuilder.updateStatus("my_job_id", JobStatus.DELETING, null);
|
|
||||||
pmBuilder.deleteJob("my_job_id");
|
|
||||||
ClusterState cs4 = ClusterState.builder(new ClusterName("_cluster_name")).metaData(MetaData.builder()
|
|
||||||
.putCustom(MlMetadata.TYPE, pmBuilder.build()))
|
|
||||||
.nodes(DiscoveryNodes.builder()
|
|
||||||
.add(new DiscoveryNode("_node_id", new TransportAddress(InetAddress.getLoopbackAddress(), 9200), Version.CURRENT))
|
|
||||||
.masterNodeId("_node_id"))
|
|
||||||
.build();
|
|
||||||
ClusterState result3 = jobAllocator.assignJobsToNodes(cs4);
|
|
||||||
pm = result3.metaData().custom(MlMetadata.TYPE);
|
|
||||||
assertNull("my_job_id must be unallocated, because job has been removed", pm.getAllocations().get("my_job_id"));
|
|
||||||
}
|
|
||||||
|
|
||||||
public void testClusterChanged_onlyAllocateIfMasterAndHaveUnAllocatedJobs() throws Exception {
|
|
||||||
ExecutorService executorService = mock(ExecutorService.class);
|
|
||||||
doAnswer(invocation -> {
|
|
||||||
((Runnable) invocation.getArguments()[0]).run();
|
|
||||||
return null;
|
|
||||||
}).when(executorService).execute(any(Runnable.class));
|
|
||||||
when(threadPool.executor(ThreadPool.Names.GENERIC)).thenReturn(executorService);
|
|
||||||
|
|
||||||
|
|
||||||
ClusterState cs = ClusterState.builder(new ClusterName("_name"))
|
|
||||||
.metaData(MetaData.builder().putCustom(MlMetadata.TYPE, new MlMetadata.Builder().build()))
|
|
||||||
.nodes(DiscoveryNodes.builder()
|
|
||||||
.add(new DiscoveryNode("_id", new TransportAddress(InetAddress.getLoopbackAddress(), 9200), Version.CURRENT))
|
|
||||||
.localNodeId("_id")
|
|
||||||
)
|
|
||||||
.build();
|
|
||||||
jobAllocator.clusterChanged(new ClusterChangedEvent("_source", cs, cs));
|
|
||||||
verify(threadPool, never()).executor(ThreadPool.Names.GENERIC);
|
|
||||||
verify(clusterService, never()).submitStateUpdateTask(any(), any());
|
|
||||||
|
|
||||||
// make node master
|
|
||||||
cs = ClusterState.builder(new ClusterName("_name"))
|
|
||||||
.metaData(MetaData.builder().putCustom(MlMetadata.TYPE, new MlMetadata.Builder().build()))
|
|
||||||
.nodes(DiscoveryNodes.builder()
|
|
||||||
.add(new DiscoveryNode("_id", new TransportAddress(InetAddress.getLoopbackAddress(), 9200), Version.CURRENT))
|
|
||||||
.masterNodeId("_id")
|
|
||||||
.localNodeId("_id")
|
|
||||||
)
|
|
||||||
.build();
|
|
||||||
jobAllocator.clusterChanged(new ClusterChangedEvent("_source", cs, cs));
|
|
||||||
verify(threadPool, never()).executor(ThreadPool.Names.GENERIC);
|
|
||||||
verify(clusterService, never()).submitStateUpdateTask(any(), any());
|
|
||||||
|
|
||||||
// add an allocated job
|
|
||||||
MlMetadata.Builder pmBuilder = new MlMetadata.Builder();
|
|
||||||
pmBuilder.putJob(buildJobBuilder("my_job_id").build(), false);
|
|
||||||
pmBuilder.assignToNode("my_job_id", "_node_id");
|
|
||||||
cs = ClusterState.builder(new ClusterName("_name"))
|
|
||||||
.nodes(DiscoveryNodes.builder()
|
|
||||||
.add(new DiscoveryNode("_id", new TransportAddress(InetAddress.getLoopbackAddress(), 9200), Version.CURRENT))
|
|
||||||
.masterNodeId("_id")
|
|
||||||
.localNodeId("_id")
|
|
||||||
)
|
|
||||||
.metaData(MetaData.builder().putCustom(MlMetadata.TYPE, pmBuilder.build()))
|
|
||||||
.build();
|
|
||||||
jobAllocator.clusterChanged(new ClusterChangedEvent("_source", cs, cs));
|
|
||||||
verify(threadPool, never()).executor(ThreadPool.Names.GENERIC);
|
|
||||||
verify(clusterService, never()).submitStateUpdateTask(any(), any());
|
|
||||||
|
|
||||||
// make job not allocated
|
|
||||||
pmBuilder = new MlMetadata.Builder();
|
|
||||||
pmBuilder.putJob(buildJobBuilder("my_job_id").build(), false);
|
|
||||||
cs = ClusterState.builder(new ClusterName("_name"))
|
|
||||||
.nodes(DiscoveryNodes.builder()
|
|
||||||
.add(new DiscoveryNode("_id", new TransportAddress(InetAddress.getLoopbackAddress(), 9200), Version.CURRENT))
|
|
||||||
.masterNodeId("_id")
|
|
||||||
.localNodeId("_id")
|
|
||||||
)
|
|
||||||
.metaData(MetaData.builder().putCustom(MlMetadata.TYPE, pmBuilder.build()))
|
|
||||||
.build();
|
|
||||||
jobAllocator.clusterChanged(new ClusterChangedEvent("_source", cs, cs));
|
|
||||||
verify(threadPool, times(1)).executor(ThreadPool.Names.GENERIC);
|
|
||||||
verify(clusterService, times(1)).submitStateUpdateTask(any(), any());
|
|
||||||
}
|
|
||||||
|
|
||||||
}
|
|
|
@ -1,223 +0,0 @@
|
||||||
/*
|
|
||||||
* Copyright Elasticsearch B.V. and/or licensed to Elasticsearch B.V. under one
|
|
||||||
* or more contributor license agreements. Licensed under the Elastic License;
|
|
||||||
* you may not use this file except in compliance with the Elastic License.
|
|
||||||
*/
|
|
||||||
package org.elasticsearch.xpack.ml.job.metadata;
|
|
||||||
|
|
||||||
import org.elasticsearch.ElasticsearchStatusException;
|
|
||||||
import org.elasticsearch.Version;
|
|
||||||
import org.elasticsearch.client.Client;
|
|
||||||
import org.elasticsearch.cluster.ClusterChangedEvent;
|
|
||||||
import org.elasticsearch.cluster.ClusterName;
|
|
||||||
import org.elasticsearch.cluster.ClusterState;
|
|
||||||
import org.elasticsearch.cluster.metadata.MetaData;
|
|
||||||
import org.elasticsearch.cluster.node.DiscoveryNode;
|
|
||||||
import org.elasticsearch.cluster.node.DiscoveryNodes;
|
|
||||||
import org.elasticsearch.cluster.service.ClusterService;
|
|
||||||
import org.elasticsearch.common.settings.Settings;
|
|
||||||
import org.elasticsearch.common.transport.TransportAddress;
|
|
||||||
import org.elasticsearch.test.ESTestCase;
|
|
||||||
import org.elasticsearch.xpack.ml.action.UpdateJobStatusAction;
|
|
||||||
import org.elasticsearch.xpack.ml.job.JobStatus;
|
|
||||||
import org.elasticsearch.xpack.ml.job.data.DataProcessor;
|
|
||||||
import org.junit.Before;
|
|
||||||
|
|
||||||
import java.net.InetAddress;
|
|
||||||
|
|
||||||
import static org.elasticsearch.xpack.ml.job.JobTests.buildJobBuilder;
|
|
||||||
import static org.mockito.Matchers.any;
|
|
||||||
import static org.mockito.Matchers.eq;
|
|
||||||
import static org.mockito.Mockito.doThrow;
|
|
||||||
import static org.mockito.Mockito.mock;
|
|
||||||
import static org.mockito.Mockito.times;
|
|
||||||
import static org.mockito.Mockito.verify;
|
|
||||||
import static org.mockito.Mockito.verifyNoMoreInteractions;
|
|
||||||
|
|
||||||
public class JobLifeCycleServiceTests extends ESTestCase {
|
|
||||||
|
|
||||||
private DataProcessor dataProcessor;
|
|
||||||
private Client client;
|
|
||||||
private JobLifeCycleService jobLifeCycleService;
|
|
||||||
|
|
||||||
@Before
|
|
||||||
public void instantiateJobAllocator() {
|
|
||||||
ClusterService clusterService = mock(ClusterService.class);
|
|
||||||
dataProcessor = mock(DataProcessor.class);
|
|
||||||
client = mock(Client.class);
|
|
||||||
jobLifeCycleService = new JobLifeCycleService(Settings.EMPTY, client, clusterService, dataProcessor, Runnable::run);
|
|
||||||
}
|
|
||||||
|
|
||||||
public void testStartStop() {
|
|
||||||
Allocation.Builder allocation = new Allocation.Builder();
|
|
||||||
allocation.setJobId("my_job_id");
|
|
||||||
jobLifeCycleService.startJob(allocation.build());
|
|
||||||
assertTrue(jobLifeCycleService.localAssignedJobs.contains("my_job_id"));
|
|
||||||
verify(dataProcessor).openJob("my_job_id", false);
|
|
||||||
|
|
||||||
jobLifeCycleService.stopJob("my_job_id");
|
|
||||||
assertTrue(jobLifeCycleService.localAssignedJobs.isEmpty());
|
|
||||||
verify(dataProcessor).closeJob("my_job_id");
|
|
||||||
}
|
|
||||||
|
|
||||||
public void testClusterChanged_startJob() throws Exception {
|
|
||||||
MlMetadata.Builder pmBuilder = new MlMetadata.Builder();
|
|
||||||
pmBuilder.putJob(buildJobBuilder("my_job_id").build(), false);
|
|
||||||
ClusterState cs1 = ClusterState.builder(new ClusterName("_cluster_name")).metaData(MetaData.builder()
|
|
||||||
.putCustom(MlMetadata.TYPE, pmBuilder.build()))
|
|
||||||
.nodes(DiscoveryNodes.builder()
|
|
||||||
.add(new DiscoveryNode("_node_id", new TransportAddress(InetAddress.getLoopbackAddress(), 9200), Version.CURRENT))
|
|
||||||
.localNodeId("_node_id"))
|
|
||||||
.build();
|
|
||||||
jobLifeCycleService.clusterChanged(new ClusterChangedEvent("_source", cs1, cs1));
|
|
||||||
assertFalse("not allocated to a node", jobLifeCycleService.localAssignedJobs.contains("my_job_id"));
|
|
||||||
|
|
||||||
pmBuilder = new MlMetadata.Builder();
|
|
||||||
pmBuilder.putJob(buildJobBuilder("my_job_id").build(), false);
|
|
||||||
pmBuilder.updateStatus("my_job_id", JobStatus.OPENING, null);
|
|
||||||
cs1 = ClusterState.builder(new ClusterName("_cluster_name")).metaData(MetaData.builder()
|
|
||||||
.putCustom(MlMetadata.TYPE, pmBuilder.build()))
|
|
||||||
.nodes(DiscoveryNodes.builder()
|
|
||||||
.add(new DiscoveryNode("_node_id", new TransportAddress(InetAddress.getLoopbackAddress(), 9200), Version.CURRENT))
|
|
||||||
.localNodeId("_node_id"))
|
|
||||||
.build();
|
|
||||||
jobLifeCycleService.clusterChanged(new ClusterChangedEvent("_source", cs1, cs1));
|
|
||||||
assertFalse("Status not started", jobLifeCycleService.localAssignedJobs.contains("my_job_id"));
|
|
||||||
|
|
||||||
pmBuilder = new MlMetadata.Builder();
|
|
||||||
pmBuilder.putJob(buildJobBuilder("my_job_id").build(), false);
|
|
||||||
pmBuilder.updateStatus("my_job_id", JobStatus.OPENING, null);
|
|
||||||
pmBuilder.assignToNode("my_job_id", "_node_id");
|
|
||||||
cs1 = ClusterState.builder(new ClusterName("_cluster_name")).metaData(MetaData.builder()
|
|
||||||
.putCustom(MlMetadata.TYPE, pmBuilder.build()))
|
|
||||||
.nodes(DiscoveryNodes.builder()
|
|
||||||
.add(new DiscoveryNode("_node_id", new TransportAddress(InetAddress.getLoopbackAddress(), 9200), Version.CURRENT))
|
|
||||||
.localNodeId("_node_id"))
|
|
||||||
.build();
|
|
||||||
jobLifeCycleService.clusterChanged(new ClusterChangedEvent("_source", cs1, cs1));
|
|
||||||
assertTrue("Expect allocation, because job allocation says my_job_id should be allocated locally",
|
|
||||||
jobLifeCycleService.localAssignedJobs.contains("my_job_id"));
|
|
||||||
verify(dataProcessor, times(1)).openJob("my_job_id", false);
|
|
||||||
|
|
||||||
jobLifeCycleService.clusterChanged(new ClusterChangedEvent("_source", cs1, cs1));
|
|
||||||
verify(dataProcessor, times(1)).openJob("my_job_id", false);
|
|
||||||
}
|
|
||||||
|
|
||||||
public void testClusterChanged_stopJob() throws Exception {
|
|
||||||
jobLifeCycleService.localAssignedJobs.add("my_job_id");
|
|
||||||
|
|
||||||
MlMetadata.Builder pmBuilder = new MlMetadata.Builder();
|
|
||||||
pmBuilder.putJob(buildJobBuilder("my_job_id").build(), false);
|
|
||||||
ClusterState cs1 = ClusterState.builder(new ClusterName("_cluster_name")).metaData(MetaData.builder()
|
|
||||||
.putCustom(MlMetadata.TYPE, pmBuilder.build()))
|
|
||||||
.nodes(DiscoveryNodes.builder()
|
|
||||||
.add(new DiscoveryNode("_node_id", new TransportAddress(InetAddress.getLoopbackAddress(), 9200), Version.CURRENT))
|
|
||||||
.localNodeId("_node_id"))
|
|
||||||
.build();
|
|
||||||
jobLifeCycleService.clusterChanged(new ClusterChangedEvent("_source", cs1, cs1));
|
|
||||||
assertEquals("Status is not closing, so nothing happened", jobLifeCycleService.localAssignedJobs.size(), 1);
|
|
||||||
|
|
||||||
pmBuilder = new MlMetadata.Builder();
|
|
||||||
pmBuilder.putJob(buildJobBuilder("my_job_id").build(), false);
|
|
||||||
pmBuilder.updateStatus("my_job_id", JobStatus.OPENING, null);
|
|
||||||
pmBuilder.updateStatus("my_job_id", JobStatus.OPENED, null);
|
|
||||||
pmBuilder.updateStatus("my_job_id", JobStatus.CLOSING, null);
|
|
||||||
pmBuilder.assignToNode("my_job_id", "_node_id");
|
|
||||||
cs1 = ClusterState.builder(new ClusterName("_cluster_name")).metaData(MetaData.builder()
|
|
||||||
.putCustom(MlMetadata.TYPE, pmBuilder.build()))
|
|
||||||
.nodes(DiscoveryNodes.builder()
|
|
||||||
.add(new DiscoveryNode("_node_id", new TransportAddress(InetAddress.getLoopbackAddress(), 9200), Version.CURRENT))
|
|
||||||
.localNodeId("_node_id"))
|
|
||||||
.build();
|
|
||||||
jobLifeCycleService.clusterChanged(new ClusterChangedEvent("_source", cs1, cs1));
|
|
||||||
assertEquals(jobLifeCycleService.localAssignedJobs.size(), 0);
|
|
||||||
verify(dataProcessor, times(1)).closeJob("my_job_id");
|
|
||||||
}
|
|
||||||
|
|
||||||
public void testClusterChanged_allocationDeletingJob() throws Exception {
|
|
||||||
jobLifeCycleService.localAssignedJobs.add("my_job_id");
|
|
||||||
|
|
||||||
MlMetadata.Builder pmBuilder = new MlMetadata.Builder();
|
|
||||||
pmBuilder.putJob(buildJobBuilder("my_job_id").build(), false);
|
|
||||||
pmBuilder.updateStatus("my_job_id", JobStatus.DELETING, null);
|
|
||||||
ClusterState cs1 = ClusterState.builder(new ClusterName("_cluster_name")).metaData(MetaData.builder()
|
|
||||||
.putCustom(MlMetadata.TYPE, pmBuilder.build()))
|
|
||||||
.nodes(DiscoveryNodes.builder()
|
|
||||||
.add(new DiscoveryNode("_node_id", new TransportAddress(InetAddress.getLoopbackAddress(), 9200), Version.CURRENT))
|
|
||||||
.localNodeId("_node_id"))
|
|
||||||
.build();
|
|
||||||
jobLifeCycleService.clusterChanged(new ClusterChangedEvent("_source", cs1, cs1));
|
|
||||||
assertEquals(jobLifeCycleService.localAssignedJobs.size(), 1);
|
|
||||||
|
|
||||||
|
|
||||||
pmBuilder.deleteJob("my_job_id");
|
|
||||||
ClusterState cs2 = ClusterState.builder(new ClusterName("_cluster_name")).metaData(MetaData.builder()
|
|
||||||
.putCustom(MlMetadata.TYPE, pmBuilder.build()))
|
|
||||||
.nodes(DiscoveryNodes.builder()
|
|
||||||
.add(new DiscoveryNode("_node_id", new TransportAddress(InetAddress.getLoopbackAddress(), 9200), Version.CURRENT))
|
|
||||||
.localNodeId("_node_id"))
|
|
||||||
.build();
|
|
||||||
jobLifeCycleService.clusterChanged(new ClusterChangedEvent("_source", cs2, cs1));
|
|
||||||
|
|
||||||
assertEquals(jobLifeCycleService.localAssignedJobs.size(), 0);
|
|
||||||
verify(dataProcessor, times(1)).closeJob("my_job_id");
|
|
||||||
}
|
|
||||||
|
|
||||||
public void testClusterChanged_allocationDeletingClosedJob() {
|
|
||||||
jobLifeCycleService.localAssignedJobs.add("my_job_id");
|
|
||||||
|
|
||||||
MlMetadata.Builder pmBuilder = new MlMetadata.Builder();
|
|
||||||
pmBuilder.putJob(buildJobBuilder("my_job_id").build(), false);
|
|
||||||
|
|
||||||
expectThrows(ElasticsearchStatusException.class, () -> pmBuilder.deleteJob("my_job_id"));
|
|
||||||
}
|
|
||||||
|
|
||||||
public void testStart_openJobFails() {
|
|
||||||
doThrow(new RuntimeException("error")).when(dataProcessor).openJob("my_job_id", false);
|
|
||||||
Allocation.Builder allocation = new Allocation.Builder();
|
|
||||||
allocation.setJobId("my_job_id");
|
|
||||||
jobLifeCycleService.startJob(allocation.build());
|
|
||||||
assertTrue(jobLifeCycleService.localAssignedJobs.contains("my_job_id"));
|
|
||||||
verify(dataProcessor).openJob("my_job_id", false);
|
|
||||||
UpdateJobStatusAction.Request expectedRequest = new UpdateJobStatusAction.Request("my_job_id", JobStatus.FAILED);
|
|
||||||
expectedRequest.setReason("failed to open, error");
|
|
||||||
verify(client).execute(eq(UpdateJobStatusAction.INSTANCE), eq(expectedRequest), any());
|
|
||||||
}
|
|
||||||
|
|
||||||
public void testStart_closeJobFails() {
|
|
||||||
jobLifeCycleService.localAssignedJobs.add("my_job_id");
|
|
||||||
doThrow(new RuntimeException("error")).when(dataProcessor).closeJob("my_job_id");
|
|
||||||
jobLifeCycleService.stopJob("my_job_id");
|
|
||||||
assertEquals(jobLifeCycleService.localAssignedJobs.size(), 0);
|
|
||||||
verify(dataProcessor).closeJob("my_job_id");
|
|
||||||
UpdateJobStatusAction.Request expectedRequest = new UpdateJobStatusAction.Request("my_job_id", JobStatus.FAILED);
|
|
||||||
expectedRequest.setReason("failed to close, error");
|
|
||||||
verify(client).execute(eq(UpdateJobStatusAction.INSTANCE), eq(expectedRequest), any());
|
|
||||||
}
|
|
||||||
|
|
||||||
public void testStop() {
|
|
||||||
jobLifeCycleService.localAssignedJobs.add("job1");
|
|
||||||
jobLifeCycleService.localAssignedJobs.add("job2");
|
|
||||||
assertFalse(jobLifeCycleService.stopped);
|
|
||||||
|
|
||||||
jobLifeCycleService.stop();
|
|
||||||
assertTrue(jobLifeCycleService.stopped);
|
|
||||||
verify(dataProcessor, times(1)).closeJob("job1");
|
|
||||||
verify(dataProcessor, times(1)).closeJob("job2");
|
|
||||||
verifyNoMoreInteractions(dataProcessor);
|
|
||||||
}
|
|
||||||
|
|
||||||
public void testStop_failure() {
|
|
||||||
jobLifeCycleService.localAssignedJobs.add("job1");
|
|
||||||
jobLifeCycleService.localAssignedJobs.add("job2");
|
|
||||||
assertFalse(jobLifeCycleService.stopped);
|
|
||||||
|
|
||||||
doThrow(new RuntimeException()).when(dataProcessor).closeJob("job1");
|
|
||||||
jobLifeCycleService.stop();
|
|
||||||
assertTrue(jobLifeCycleService.stopped);
|
|
||||||
verify(dataProcessor, times(1)).closeJob("job1");
|
|
||||||
verify(dataProcessor, times(1)).closeJob("job2");
|
|
||||||
verifyNoMoreInteractions(dataProcessor);
|
|
||||||
}
|
|
||||||
|
|
||||||
}
|
|
|
@ -51,7 +51,6 @@ import java.util.Date;
|
||||||
import java.util.HashMap;
|
import java.util.HashMap;
|
||||||
import java.util.List;
|
import java.util.List;
|
||||||
import java.util.Map;
|
import java.util.Map;
|
||||||
import java.util.Optional;
|
|
||||||
import java.util.concurrent.ExecutionException;
|
import java.util.concurrent.ExecutionException;
|
||||||
import java.util.concurrent.atomic.AtomicReference;
|
import java.util.concurrent.atomic.AtomicReference;
|
||||||
import java.util.function.Consumer;
|
import java.util.function.Consumer;
|
||||||
|
@ -78,9 +77,10 @@ public class JobProviderTests extends ESTestCase {
|
||||||
Client client = getMockedClient(getResponse);
|
Client client = getMockedClient(getResponse);
|
||||||
JobProvider provider = createProvider(client);
|
JobProvider provider = createProvider(client);
|
||||||
|
|
||||||
Optional<Quantiles> quantiles = provider.getQuantiles(JOB_ID);
|
Quantiles[] holder = new Quantiles[1];
|
||||||
|
provider.getQuantiles(JOB_ID, quantiles -> holder[0] = quantiles, RuntimeException::new);
|
||||||
assertFalse(quantiles.isPresent());
|
Quantiles quantiles = holder[0];
|
||||||
|
assertNull(quantiles);
|
||||||
}
|
}
|
||||||
|
|
||||||
public void testGetQuantiles_GivenQuantilesHaveNonEmptyState() throws Exception {
|
public void testGetQuantiles_GivenQuantilesHaveNonEmptyState() throws Exception {
|
||||||
|
@ -93,10 +93,11 @@ public class JobProviderTests extends ESTestCase {
|
||||||
Client client = getMockedClient(getResponse);
|
Client client = getMockedClient(getResponse);
|
||||||
JobProvider provider = createProvider(client);
|
JobProvider provider = createProvider(client);
|
||||||
|
|
||||||
Optional<Quantiles> quantiles = provider.getQuantiles(JOB_ID);
|
Quantiles[] holder = new Quantiles[1];
|
||||||
|
provider.getQuantiles(JOB_ID, quantiles -> holder[0] = quantiles, RuntimeException::new);
|
||||||
assertTrue(quantiles.isPresent());
|
Quantiles quantiles = holder[0];
|
||||||
assertEquals("state", quantiles.get().getQuantileState());
|
assertNotNull(quantiles);
|
||||||
|
assertEquals("state", quantiles.getQuantileState());
|
||||||
}
|
}
|
||||||
|
|
||||||
public void testGetQuantiles_GivenQuantilesHaveEmptyState() throws Exception {
|
public void testGetQuantiles_GivenQuantilesHaveEmptyState() throws Exception {
|
||||||
|
@ -109,10 +110,11 @@ public class JobProviderTests extends ESTestCase {
|
||||||
Client client = getMockedClient(getResponse);
|
Client client = getMockedClient(getResponse);
|
||||||
JobProvider provider = createProvider(client);
|
JobProvider provider = createProvider(client);
|
||||||
|
|
||||||
Optional<Quantiles> quantiles = provider.getQuantiles(JOB_ID);
|
Quantiles[] holder = new Quantiles[1];
|
||||||
|
provider.getQuantiles(JOB_ID, quantiles -> holder[0] = quantiles, RuntimeException::new);
|
||||||
assertTrue(quantiles.isPresent());
|
Quantiles quantiles = holder[0];
|
||||||
assertEquals("", quantiles.get().getQuantileState());
|
assertNotNull(quantiles);
|
||||||
|
assertEquals("", quantiles.getQuantileState());
|
||||||
}
|
}
|
||||||
|
|
||||||
public void testCreateUsageMetering() throws InterruptedException, ExecutionException {
|
public void testCreateUsageMetering() throws InterruptedException, ExecutionException {
|
||||||
|
@ -912,7 +914,10 @@ public class JobProviderTests extends ESTestCase {
|
||||||
Client client = getMockedClient(qb -> {}, response);
|
Client client = getMockedClient(qb -> {}, response);
|
||||||
JobProvider provider = createProvider(client);
|
JobProvider provider = createProvider(client);
|
||||||
|
|
||||||
QueryPage<ModelSnapshot> page = provider.modelSnapshots(jobId, from, size);
|
@SuppressWarnings({"unchecked", "rawtypes"})
|
||||||
|
QueryPage<ModelSnapshot>[] holder = new QueryPage[1];
|
||||||
|
provider.modelSnapshots(jobId, from, size, r -> holder[0] = r, RuntimeException::new);
|
||||||
|
QueryPage<ModelSnapshot> page = holder[0];
|
||||||
assertEquals(2L, page.count());
|
assertEquals(2L, page.count());
|
||||||
List<ModelSnapshot> snapshots = page.results();
|
List<ModelSnapshot> snapshots = page.results();
|
||||||
|
|
||||||
|
@ -1174,6 +1179,13 @@ public class JobProviderTests extends ESTestCase {
|
||||||
ActionFuture<GetResponse> actionFuture = mock(ActionFuture.class);
|
ActionFuture<GetResponse> actionFuture = mock(ActionFuture.class);
|
||||||
when(client.get(any())).thenReturn(actionFuture);
|
when(client.get(any())).thenReturn(actionFuture);
|
||||||
when(actionFuture.actionGet()).thenReturn(response);
|
when(actionFuture.actionGet()).thenReturn(response);
|
||||||
|
|
||||||
|
doAnswer(invocationOnMock -> {
|
||||||
|
@SuppressWarnings("unchecked")
|
||||||
|
ActionListener<GetResponse> actionListener = (ActionListener<GetResponse>) invocationOnMock.getArguments()[1];
|
||||||
|
actionListener.onResponse(response);
|
||||||
|
return null;
|
||||||
|
}).when(client).get(any(), any());
|
||||||
return client;
|
return client;
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
|
@ -43,7 +43,7 @@ public class AutodetectCommunicatorTests extends ESTestCase {
|
||||||
DataLoadParams params = new DataLoadParams(TimeRange.builder().startTime("1").endTime("2").build(), false, Optional.empty());
|
DataLoadParams params = new DataLoadParams(TimeRange.builder().startTime("1").endTime("2").build(), false, Optional.empty());
|
||||||
AutodetectProcess process = mockAutodetectProcessWithOutputStream();
|
AutodetectProcess process = mockAutodetectProcessWithOutputStream();
|
||||||
try (AutodetectCommunicator communicator = createAutodetectCommunicator(process, mock(AutoDetectResultProcessor.class))) {
|
try (AutodetectCommunicator communicator = createAutodetectCommunicator(process, mock(AutoDetectResultProcessor.class))) {
|
||||||
communicator.writeToJob(new ByteArrayInputStream(new byte[0]), params, () -> false);
|
communicator.writeToJob(new ByteArrayInputStream(new byte[0]), params);
|
||||||
Mockito.verify(process).writeResetBucketsControlMessage(params);
|
Mockito.verify(process).writeResetBucketsControlMessage(params);
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
@ -134,7 +134,7 @@ public class AutodetectCommunicatorTests extends ESTestCase {
|
||||||
StatusReporter statusReporter = mock(StatusReporter.class);
|
StatusReporter statusReporter = mock(StatusReporter.class);
|
||||||
StateProcessor stateProcessor = mock(StateProcessor.class);
|
StateProcessor stateProcessor = mock(StateProcessor.class);
|
||||||
return new AutodetectCommunicator(executorService, createJobDetails(), autodetectProcess, statusReporter,
|
return new AutodetectCommunicator(executorService, createJobDetails(), autodetectProcess, statusReporter,
|
||||||
autoDetectResultProcessor, stateProcessor);
|
autoDetectResultProcessor, stateProcessor, e -> {});
|
||||||
}
|
}
|
||||||
|
|
||||||
public void testWriteToJobInUse() throws IOException {
|
public void testWriteToJobInUse() throws IOException {
|
||||||
|
@ -145,10 +145,10 @@ public class AutodetectCommunicatorTests extends ESTestCase {
|
||||||
|
|
||||||
communicator.inUse.set(new CountDownLatch(1));
|
communicator.inUse.set(new CountDownLatch(1));
|
||||||
expectThrows(ElasticsearchStatusException.class,
|
expectThrows(ElasticsearchStatusException.class,
|
||||||
() -> communicator.writeToJob(in, mock(DataLoadParams.class), () -> false));
|
() -> communicator.writeToJob(in, mock(DataLoadParams.class)));
|
||||||
|
|
||||||
communicator.inUse.set(null);
|
communicator.inUse.set(null);
|
||||||
communicator.writeToJob(in, new DataLoadParams(TimeRange.builder().build(), false, Optional.empty()), () -> false);
|
communicator.writeToJob(in, new DataLoadParams(TimeRange.builder().build(), false, Optional.empty()));
|
||||||
}
|
}
|
||||||
|
|
||||||
public void testFlushInUse() throws IOException {
|
public void testFlushInUse() throws IOException {
|
||||||
|
|
|
@ -5,32 +5,14 @@
|
||||||
*/
|
*/
|
||||||
package org.elasticsearch.xpack.ml.job.process.autodetect.writer;
|
package org.elasticsearch.xpack.ml.job.process.autodetect.writer;
|
||||||
|
|
||||||
import static org.mockito.Matchers.anyLong;
|
|
||||||
import static org.mockito.Mockito.never;
|
|
||||||
import static org.mockito.Mockito.times;
|
|
||||||
import static org.mockito.Mockito.verify;
|
|
||||||
|
|
||||||
import java.io.IOException;
|
|
||||||
import java.util.ArrayList;
|
|
||||||
import java.util.Arrays;
|
|
||||||
import java.util.Collections;
|
|
||||||
import java.util.HashSet;
|
|
||||||
import java.util.List;
|
|
||||||
import java.util.Map;
|
|
||||||
import java.util.Set;
|
|
||||||
|
|
||||||
import org.apache.logging.log4j.Logger;
|
import org.apache.logging.log4j.Logger;
|
||||||
import org.elasticsearch.test.ESTestCase;
|
import org.elasticsearch.test.ESTestCase;
|
||||||
import org.elasticsearch.xpack.ml.job.process.autodetect.AutodetectProcess;
|
|
||||||
import org.junit.Assert;
|
|
||||||
import org.junit.Before;
|
|
||||||
import org.mockito.Mockito;
|
|
||||||
|
|
||||||
import org.elasticsearch.xpack.ml.job.AnalysisConfig;
|
import org.elasticsearch.xpack.ml.job.AnalysisConfig;
|
||||||
import org.elasticsearch.xpack.ml.job.DataDescription;
|
import org.elasticsearch.xpack.ml.job.DataDescription;
|
||||||
import org.elasticsearch.xpack.ml.job.Detector;
|
import org.elasticsearch.xpack.ml.job.Detector;
|
||||||
import org.elasticsearch.xpack.ml.job.condition.Condition;
|
import org.elasticsearch.xpack.ml.job.condition.Condition;
|
||||||
import org.elasticsearch.xpack.ml.job.condition.Operator;
|
import org.elasticsearch.xpack.ml.job.condition.Operator;
|
||||||
|
import org.elasticsearch.xpack.ml.job.process.autodetect.AutodetectProcess;
|
||||||
import org.elasticsearch.xpack.ml.job.process.autodetect.writer.AbstractDataToProcessWriter.InputOutputMap;
|
import org.elasticsearch.xpack.ml.job.process.autodetect.writer.AbstractDataToProcessWriter.InputOutputMap;
|
||||||
import org.elasticsearch.xpack.ml.job.status.StatusReporter;
|
import org.elasticsearch.xpack.ml.job.status.StatusReporter;
|
||||||
import org.elasticsearch.xpack.ml.job.transform.TransformConfig;
|
import org.elasticsearch.xpack.ml.job.transform.TransformConfig;
|
||||||
|
@ -42,6 +24,23 @@ import org.elasticsearch.xpack.ml.transforms.RegexSplit;
|
||||||
import org.elasticsearch.xpack.ml.transforms.StringTransform;
|
import org.elasticsearch.xpack.ml.transforms.StringTransform;
|
||||||
import org.elasticsearch.xpack.ml.transforms.Transform;
|
import org.elasticsearch.xpack.ml.transforms.Transform;
|
||||||
import org.elasticsearch.xpack.ml.transforms.Transform.TransformIndex;
|
import org.elasticsearch.xpack.ml.transforms.Transform.TransformIndex;
|
||||||
|
import org.junit.Assert;
|
||||||
|
import org.junit.Before;
|
||||||
|
import org.mockito.Mockito;
|
||||||
|
|
||||||
|
import java.io.IOException;
|
||||||
|
import java.util.ArrayList;
|
||||||
|
import java.util.Arrays;
|
||||||
|
import java.util.Collections;
|
||||||
|
import java.util.HashSet;
|
||||||
|
import java.util.List;
|
||||||
|
import java.util.Map;
|
||||||
|
import java.util.Set;
|
||||||
|
|
||||||
|
import static org.mockito.Matchers.anyLong;
|
||||||
|
import static org.mockito.Mockito.never;
|
||||||
|
import static org.mockito.Mockito.times;
|
||||||
|
import static org.mockito.Mockito.verify;
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* Testing methods of AbstractDataToProcessWriter but uses the concrete
|
* Testing methods of AbstractDataToProcessWriter but uses the concrete
|
||||||
|
@ -343,7 +342,7 @@ public class AbstractDataToProcessWriterTests extends ESTestCase {
|
||||||
String[] input = { "1", "metricA", "0" };
|
String[] input = { "1", "metricA", "0" };
|
||||||
String[] output = new String[3];
|
String[] output = new String[3];
|
||||||
|
|
||||||
assertFalse(writer.applyTransformsAndWrite(() -> false, input, output, 3));
|
assertFalse(writer.applyTransformsAndWrite(input, output, 3));
|
||||||
|
|
||||||
verify(autodetectProcess, never()).writeRecord(output);
|
verify(autodetectProcess, never()).writeRecord(output);
|
||||||
verify(statusReporter, never()).reportRecordWritten(anyLong(), anyLong());
|
verify(statusReporter, never()).reportRecordWritten(anyLong(), anyLong());
|
||||||
|
@ -354,7 +353,7 @@ public class AbstractDataToProcessWriterTests extends ESTestCase {
|
||||||
// this is ok
|
// this is ok
|
||||||
input = new String[] { "2", "metricB", "0" };
|
input = new String[] { "2", "metricB", "0" };
|
||||||
String[] expectedOutput = { "2", null, null };
|
String[] expectedOutput = { "2", null, null };
|
||||||
assertTrue(writer.applyTransformsAndWrite(() -> false, input, output, 3));
|
assertTrue(writer.applyTransformsAndWrite(input, output, 3));
|
||||||
|
|
||||||
verify(autodetectProcess, times(1)).writeRecord(expectedOutput);
|
verify(autodetectProcess, times(1)).writeRecord(expectedOutput);
|
||||||
verify(statusReporter, times(1)).reportRecordWritten(3, 2000);
|
verify(statusReporter, times(1)).reportRecordWritten(3, 2000);
|
||||||
|
|
|
@ -5,15 +5,23 @@
|
||||||
*/
|
*/
|
||||||
package org.elasticsearch.xpack.ml.job.process.autodetect.writer;
|
package org.elasticsearch.xpack.ml.job.process.autodetect.writer;
|
||||||
|
|
||||||
import static org.elasticsearch.xpack.ml.job.process.autodetect.writer.JsonDataToProcessWriterTests.endLessStream;
|
import org.apache.logging.log4j.Logger;
|
||||||
import static org.mockito.Matchers.any;
|
import org.elasticsearch.test.ESTestCase;
|
||||||
import static org.mockito.Matchers.anyLong;
|
import org.elasticsearch.xpack.ml.job.AnalysisConfig;
|
||||||
import static org.mockito.Mockito.atLeastOnce;
|
import org.elasticsearch.xpack.ml.job.DataCounts;
|
||||||
import static org.mockito.Mockito.doAnswer;
|
import org.elasticsearch.xpack.ml.job.DataDescription;
|
||||||
import static org.mockito.Mockito.never;
|
import org.elasticsearch.xpack.ml.job.DataDescription.DataFormat;
|
||||||
import static org.mockito.Mockito.times;
|
import org.elasticsearch.xpack.ml.job.Detector;
|
||||||
import static org.mockito.Mockito.verify;
|
import org.elasticsearch.xpack.ml.job.process.autodetect.AutodetectProcess;
|
||||||
import static org.mockito.Mockito.when;
|
import org.elasticsearch.xpack.ml.job.status.StatusReporter;
|
||||||
|
import org.elasticsearch.xpack.ml.job.transform.TransformConfig;
|
||||||
|
import org.elasticsearch.xpack.ml.job.transform.TransformConfigs;
|
||||||
|
import org.elasticsearch.xpack.ml.job.transform.TransformType;
|
||||||
|
import org.junit.Before;
|
||||||
|
import org.mockito.Mockito;
|
||||||
|
import org.mockito.invocation.InvocationOnMock;
|
||||||
|
import org.mockito.stubbing.Answer;
|
||||||
|
import org.supercsv.exception.SuperCsvException;
|
||||||
|
|
||||||
import java.io.ByteArrayInputStream;
|
import java.io.ByteArrayInputStream;
|
||||||
import java.io.IOException;
|
import java.io.IOException;
|
||||||
|
@ -23,28 +31,14 @@ import java.util.ArrayList;
|
||||||
import java.util.Arrays;
|
import java.util.Arrays;
|
||||||
import java.util.Date;
|
import java.util.Date;
|
||||||
import java.util.List;
|
import java.util.List;
|
||||||
import java.util.concurrent.atomic.AtomicBoolean;
|
|
||||||
import java.util.concurrent.atomic.AtomicReference;
|
|
||||||
|
|
||||||
import org.apache.logging.log4j.Logger;
|
import static org.mockito.Matchers.any;
|
||||||
import org.elasticsearch.tasks.TaskCancelledException;
|
import static org.mockito.Matchers.anyLong;
|
||||||
import org.elasticsearch.test.ESTestCase;
|
import static org.mockito.Mockito.doAnswer;
|
||||||
import org.elasticsearch.xpack.ml.job.DataCounts;
|
import static org.mockito.Mockito.never;
|
||||||
import org.elasticsearch.xpack.ml.job.process.autodetect.AutodetectProcess;
|
import static org.mockito.Mockito.times;
|
||||||
import org.junit.Before;
|
import static org.mockito.Mockito.verify;
|
||||||
import org.mockito.Mockito;
|
import static org.mockito.Mockito.when;
|
||||||
import org.mockito.invocation.InvocationOnMock;
|
|
||||||
import org.mockito.stubbing.Answer;
|
|
||||||
import org.supercsv.exception.SuperCsvException;
|
|
||||||
|
|
||||||
import org.elasticsearch.xpack.ml.job.AnalysisConfig;
|
|
||||||
import org.elasticsearch.xpack.ml.job.DataDescription;
|
|
||||||
import org.elasticsearch.xpack.ml.job.DataDescription.DataFormat;
|
|
||||||
import org.elasticsearch.xpack.ml.job.Detector;
|
|
||||||
import org.elasticsearch.xpack.ml.job.status.StatusReporter;
|
|
||||||
import org.elasticsearch.xpack.ml.job.transform.TransformConfig;
|
|
||||||
import org.elasticsearch.xpack.ml.job.transform.TransformConfigs;
|
|
||||||
import org.elasticsearch.xpack.ml.job.transform.TransformType;
|
|
||||||
|
|
||||||
public class CsvDataToProcessWriterTests extends ESTestCase {
|
public class CsvDataToProcessWriterTests extends ESTestCase {
|
||||||
|
|
||||||
|
@ -85,33 +79,6 @@ public class CsvDataToProcessWriterTests extends ESTestCase {
|
||||||
analysisConfig = new AnalysisConfig.Builder(Arrays.asList(detector)).build();
|
analysisConfig = new AnalysisConfig.Builder(Arrays.asList(detector)).build();
|
||||||
}
|
}
|
||||||
|
|
||||||
public void testWrite_cancel() throws Exception {
|
|
||||||
InputStream inputStream = endLessStream("time,metric,value\n", "1,,foo\n");
|
|
||||||
CsvDataToProcessWriter writer = createWriter();
|
|
||||||
writer.writeHeader();
|
|
||||||
|
|
||||||
AtomicBoolean cancel = new AtomicBoolean(false);
|
|
||||||
AtomicReference<Exception> exception = new AtomicReference<>();
|
|
||||||
Thread t = new Thread(() -> {
|
|
||||||
try {
|
|
||||||
writer.write(inputStream, cancel::get);
|
|
||||||
} catch (Exception e) {
|
|
||||||
exception.set(e);
|
|
||||||
}
|
|
||||||
});
|
|
||||||
t.start();
|
|
||||||
try {
|
|
||||||
assertBusy(() -> verify(statusReporter, atLeastOnce()).reportRecordWritten(anyLong(), anyLong()));
|
|
||||||
} finally {
|
|
||||||
cancel.set(true);
|
|
||||||
t.join();
|
|
||||||
}
|
|
||||||
|
|
||||||
assertNotNull(exception.get());
|
|
||||||
assertEquals(TaskCancelledException.class, exception.get().getClass());
|
|
||||||
assertEquals("cancelled", exception.get().getMessage());
|
|
||||||
}
|
|
||||||
|
|
||||||
public void testWrite_GivenTimeFormatIsEpochAndDataIsValid()
|
public void testWrite_GivenTimeFormatIsEpochAndDataIsValid()
|
||||||
throws IOException {
|
throws IOException {
|
||||||
StringBuilder input = new StringBuilder();
|
StringBuilder input = new StringBuilder();
|
||||||
|
@ -121,7 +88,7 @@ public class CsvDataToProcessWriterTests extends ESTestCase {
|
||||||
InputStream inputStream = createInputStream(input.toString());
|
InputStream inputStream = createInputStream(input.toString());
|
||||||
CsvDataToProcessWriter writer = createWriter();
|
CsvDataToProcessWriter writer = createWriter();
|
||||||
writer.writeHeader();
|
writer.writeHeader();
|
||||||
writer.write(inputStream, () -> false);
|
writer.write(inputStream);
|
||||||
verify(statusReporter, times(1)).startNewIncrementalCount();
|
verify(statusReporter, times(1)).startNewIncrementalCount();
|
||||||
|
|
||||||
List<String[]> expectedRecords = new ArrayList<>();
|
List<String[]> expectedRecords = new ArrayList<>();
|
||||||
|
@ -153,7 +120,7 @@ public class CsvDataToProcessWriterTests extends ESTestCase {
|
||||||
InputStream inputStream = createInputStream(input.toString());
|
InputStream inputStream = createInputStream(input.toString());
|
||||||
CsvDataToProcessWriter writer = createWriter();
|
CsvDataToProcessWriter writer = createWriter();
|
||||||
writer.writeHeader();
|
writer.writeHeader();
|
||||||
writer.write(inputStream, () -> false);
|
writer.write(inputStream);
|
||||||
verify(statusReporter, times(1)).startNewIncrementalCount();
|
verify(statusReporter, times(1)).startNewIncrementalCount();
|
||||||
|
|
||||||
List<String[]> expectedRecords = new ArrayList<>();
|
List<String[]> expectedRecords = new ArrayList<>();
|
||||||
|
@ -176,7 +143,7 @@ public class CsvDataToProcessWriterTests extends ESTestCase {
|
||||||
InputStream inputStream = createInputStream(input.toString());
|
InputStream inputStream = createInputStream(input.toString());
|
||||||
CsvDataToProcessWriter writer = createWriter();
|
CsvDataToProcessWriter writer = createWriter();
|
||||||
writer.writeHeader();
|
writer.writeHeader();
|
||||||
writer.write(inputStream, () -> false);
|
writer.write(inputStream);
|
||||||
verify(statusReporter, times(1)).startNewIncrementalCount();
|
verify(statusReporter, times(1)).startNewIncrementalCount();
|
||||||
|
|
||||||
List<String[]> expectedRecords = new ArrayList<>();
|
List<String[]> expectedRecords = new ArrayList<>();
|
||||||
|
@ -201,7 +168,7 @@ public class CsvDataToProcessWriterTests extends ESTestCase {
|
||||||
when(statusReporter.getLatestRecordTime()).thenReturn(new Date(5000L));
|
when(statusReporter.getLatestRecordTime()).thenReturn(new Date(5000L));
|
||||||
CsvDataToProcessWriter writer = createWriter();
|
CsvDataToProcessWriter writer = createWriter();
|
||||||
writer.writeHeader();
|
writer.writeHeader();
|
||||||
writer.write(inputStream, () -> false);
|
writer.write(inputStream);
|
||||||
verify(statusReporter, times(1)).startNewIncrementalCount();
|
verify(statusReporter, times(1)).startNewIncrementalCount();
|
||||||
|
|
||||||
List<String[]> expectedRecords = new ArrayList<>();
|
List<String[]> expectedRecords = new ArrayList<>();
|
||||||
|
@ -232,7 +199,7 @@ public class CsvDataToProcessWriterTests extends ESTestCase {
|
||||||
InputStream inputStream = createInputStream(input.toString());
|
InputStream inputStream = createInputStream(input.toString());
|
||||||
CsvDataToProcessWriter writer = createWriter();
|
CsvDataToProcessWriter writer = createWriter();
|
||||||
writer.writeHeader();
|
writer.writeHeader();
|
||||||
writer.write(inputStream, () -> false);
|
writer.write(inputStream);
|
||||||
verify(statusReporter, times(1)).startNewIncrementalCount();
|
verify(statusReporter, times(1)).startNewIncrementalCount();
|
||||||
|
|
||||||
List<String[]> expectedRecords = new ArrayList<>();
|
List<String[]> expectedRecords = new ArrayList<>();
|
||||||
|
@ -266,7 +233,7 @@ public class CsvDataToProcessWriterTests extends ESTestCase {
|
||||||
InputStream inputStream = createInputStream(input.toString());
|
InputStream inputStream = createInputStream(input.toString());
|
||||||
CsvDataToProcessWriter writer = createWriter();
|
CsvDataToProcessWriter writer = createWriter();
|
||||||
writer.writeHeader();
|
writer.writeHeader();
|
||||||
writer.write(inputStream, () -> false);
|
writer.write(inputStream);
|
||||||
verify(statusReporter, times(1)).startNewIncrementalCount();
|
verify(statusReporter, times(1)).startNewIncrementalCount();
|
||||||
|
|
||||||
List<String[]> expectedRecords = new ArrayList<>();
|
List<String[]> expectedRecords = new ArrayList<>();
|
||||||
|
@ -298,7 +265,7 @@ public class CsvDataToProcessWriterTests extends ESTestCase {
|
||||||
CsvDataToProcessWriter writer = createWriter();
|
CsvDataToProcessWriter writer = createWriter();
|
||||||
writer.writeHeader();
|
writer.writeHeader();
|
||||||
|
|
||||||
DataCounts counts = writer.write(inputStream, () -> false);
|
DataCounts counts = writer.write(inputStream);
|
||||||
assertEquals(0L, counts.getInputBytes());
|
assertEquals(0L, counts.getInputBytes());
|
||||||
assertEquals(0L, counts.getInputRecordCount());
|
assertEquals(0L, counts.getInputRecordCount());
|
||||||
}
|
}
|
||||||
|
@ -326,7 +293,7 @@ public class CsvDataToProcessWriterTests extends ESTestCase {
|
||||||
input.append("1970-01-01,00:00:02Z,foo,6.0\n");
|
input.append("1970-01-01,00:00:02Z,foo,6.0\n");
|
||||||
InputStream inputStream = createInputStream(input.toString());
|
InputStream inputStream = createInputStream(input.toString());
|
||||||
|
|
||||||
writer.write(inputStream, () -> false);
|
writer.write(inputStream);
|
||||||
verify(statusReporter, times(1)).startNewIncrementalCount();
|
verify(statusReporter, times(1)).startNewIncrementalCount();
|
||||||
|
|
||||||
List<String[]> expectedRecords = new ArrayList<>();
|
List<String[]> expectedRecords = new ArrayList<>();
|
||||||
|
@ -365,7 +332,7 @@ public class CsvDataToProcessWriterTests extends ESTestCase {
|
||||||
InputStream inputStream = createInputStream(input.toString());
|
InputStream inputStream = createInputStream(input.toString());
|
||||||
CsvDataToProcessWriter writer = createWriter();
|
CsvDataToProcessWriter writer = createWriter();
|
||||||
writer.writeHeader();
|
writer.writeHeader();
|
||||||
writer.write(inputStream, () -> false);
|
writer.write(inputStream);
|
||||||
verify(statusReporter, times(1)).startNewIncrementalCount();
|
verify(statusReporter, times(1)).startNewIncrementalCount();
|
||||||
|
|
||||||
List<String[]> expectedRecords = new ArrayList<>();
|
List<String[]> expectedRecords = new ArrayList<>();
|
||||||
|
@ -392,7 +359,7 @@ public class CsvDataToProcessWriterTests extends ESTestCase {
|
||||||
CsvDataToProcessWriter writer = createWriter();
|
CsvDataToProcessWriter writer = createWriter();
|
||||||
writer.writeHeader();
|
writer.writeHeader();
|
||||||
|
|
||||||
SuperCsvException e = ESTestCase.expectThrows(SuperCsvException.class, () -> writer.write(inputStream, () -> false));
|
SuperCsvException e = ESTestCase.expectThrows(SuperCsvException.class, () -> writer.write(inputStream));
|
||||||
// Expected line numbers are 2 and 10001, but SuperCSV may print the
|
// Expected line numbers are 2 and 10001, but SuperCSV may print the
|
||||||
// numbers using a different locale's digit characters
|
// numbers using a different locale's digit characters
|
||||||
assertTrue(e.getMessage(), e.getMessage().matches(
|
assertTrue(e.getMessage(), e.getMessage().matches(
|
||||||
|
|
|
@ -68,7 +68,7 @@ public class DataWithTransformsToProcessWriterTests extends ESTestCase {
|
||||||
InputStream inputStream = createInputStream(input.toString());
|
InputStream inputStream = createInputStream(input.toString());
|
||||||
AbstractDataToProcessWriter writer = createWriter(true);
|
AbstractDataToProcessWriter writer = createWriter(true);
|
||||||
writer.writeHeader();
|
writer.writeHeader();
|
||||||
writer.write(inputStream, () -> false);
|
writer.write(inputStream);
|
||||||
|
|
||||||
List<String[]> expectedRecords = new ArrayList<>();
|
List<String[]> expectedRecords = new ArrayList<>();
|
||||||
// The final field is the control field
|
// The final field is the control field
|
||||||
|
@ -91,7 +91,7 @@ public class DataWithTransformsToProcessWriterTests extends ESTestCase {
|
||||||
InputStream inputStream = createInputStream(input.toString());
|
InputStream inputStream = createInputStream(input.toString());
|
||||||
AbstractDataToProcessWriter writer = createWriter(false);
|
AbstractDataToProcessWriter writer = createWriter(false);
|
||||||
writer.writeHeader();
|
writer.writeHeader();
|
||||||
writer.write(inputStream, () -> false);
|
writer.write(inputStream);
|
||||||
|
|
||||||
List<String[]> expectedRecords = new ArrayList<>();
|
List<String[]> expectedRecords = new ArrayList<>();
|
||||||
// The final field is the control field
|
// The final field is the control field
|
||||||
|
|
|
@ -7,7 +7,6 @@ package org.elasticsearch.xpack.ml.job.process.autodetect.writer;
|
||||||
|
|
||||||
import org.apache.logging.log4j.Logger;
|
import org.apache.logging.log4j.Logger;
|
||||||
import org.elasticsearch.ElasticsearchParseException;
|
import org.elasticsearch.ElasticsearchParseException;
|
||||||
import org.elasticsearch.tasks.TaskCancelledException;
|
|
||||||
import org.elasticsearch.test.ESTestCase;
|
import org.elasticsearch.test.ESTestCase;
|
||||||
import org.elasticsearch.xpack.ml.job.AnalysisConfig;
|
import org.elasticsearch.xpack.ml.job.AnalysisConfig;
|
||||||
import org.elasticsearch.xpack.ml.job.DataDescription;
|
import org.elasticsearch.xpack.ml.job.DataDescription;
|
||||||
|
@ -30,12 +29,9 @@ import java.nio.charset.StandardCharsets;
|
||||||
import java.util.ArrayList;
|
import java.util.ArrayList;
|
||||||
import java.util.Arrays;
|
import java.util.Arrays;
|
||||||
import java.util.List;
|
import java.util.List;
|
||||||
import java.util.concurrent.atomic.AtomicBoolean;
|
|
||||||
import java.util.concurrent.atomic.AtomicReference;
|
|
||||||
|
|
||||||
import static org.mockito.Matchers.any;
|
import static org.mockito.Matchers.any;
|
||||||
import static org.mockito.Matchers.anyLong;
|
import static org.mockito.Matchers.anyLong;
|
||||||
import static org.mockito.Mockito.atLeastOnce;
|
|
||||||
import static org.mockito.Mockito.doAnswer;
|
import static org.mockito.Mockito.doAnswer;
|
||||||
import static org.mockito.Mockito.never;
|
import static org.mockito.Mockito.never;
|
||||||
import static org.mockito.Mockito.times;
|
import static org.mockito.Mockito.times;
|
||||||
|
@ -80,33 +76,6 @@ public class JsonDataToProcessWriterTests extends ESTestCase {
|
||||||
analysisConfig = new AnalysisConfig.Builder(Arrays.asList(detector)).build();
|
analysisConfig = new AnalysisConfig.Builder(Arrays.asList(detector)).build();
|
||||||
}
|
}
|
||||||
|
|
||||||
public void testWrite_cancel() throws Exception {
|
|
||||||
InputStream inputStream = endLessStream("", "{\"time\":1}");
|
|
||||||
JsonDataToProcessWriter writer = createWriter();
|
|
||||||
writer.writeHeader();
|
|
||||||
|
|
||||||
AtomicBoolean cancel = new AtomicBoolean(false);
|
|
||||||
AtomicReference<Exception> exception = new AtomicReference<>();
|
|
||||||
Thread t = new Thread(() -> {
|
|
||||||
try {
|
|
||||||
writer.write(inputStream, cancel::get);
|
|
||||||
} catch (Exception e) {
|
|
||||||
exception.set(e);
|
|
||||||
}
|
|
||||||
});
|
|
||||||
t.start();
|
|
||||||
try {
|
|
||||||
assertBusy(() -> verify(statusReporter, atLeastOnce()).reportRecordWritten(anyLong(), anyLong()));
|
|
||||||
} finally {
|
|
||||||
cancel.set(true);
|
|
||||||
t.join();
|
|
||||||
}
|
|
||||||
|
|
||||||
assertNotNull(exception.get());
|
|
||||||
assertEquals(TaskCancelledException.class, exception.get().getClass());
|
|
||||||
assertEquals("cancelled", exception.get().getMessage());
|
|
||||||
}
|
|
||||||
|
|
||||||
public void testWrite_GivenTimeFormatIsEpochAndDataIsValid() throws Exception {
|
public void testWrite_GivenTimeFormatIsEpochAndDataIsValid() throws Exception {
|
||||||
StringBuilder input = new StringBuilder();
|
StringBuilder input = new StringBuilder();
|
||||||
input.append("{\"time\":\"1\", \"metric\":\"foo\", \"value\":\"1.0\"}");
|
input.append("{\"time\":\"1\", \"metric\":\"foo\", \"value\":\"1.0\"}");
|
||||||
|
@ -114,7 +83,7 @@ public class JsonDataToProcessWriterTests extends ESTestCase {
|
||||||
InputStream inputStream = createInputStream(input.toString());
|
InputStream inputStream = createInputStream(input.toString());
|
||||||
JsonDataToProcessWriter writer = createWriter();
|
JsonDataToProcessWriter writer = createWriter();
|
||||||
writer.writeHeader();
|
writer.writeHeader();
|
||||||
writer.write(inputStream, () -> false);
|
writer.write(inputStream);
|
||||||
verify(statusReporter, times(1)).startNewIncrementalCount();
|
verify(statusReporter, times(1)).startNewIncrementalCount();
|
||||||
|
|
||||||
List<String[]> expectedRecords = new ArrayList<>();
|
List<String[]> expectedRecords = new ArrayList<>();
|
||||||
|
@ -136,7 +105,7 @@ public class JsonDataToProcessWriterTests extends ESTestCase {
|
||||||
InputStream inputStream = createInputStream(input.toString());
|
InputStream inputStream = createInputStream(input.toString());
|
||||||
JsonDataToProcessWriter writer = createWriter();
|
JsonDataToProcessWriter writer = createWriter();
|
||||||
writer.writeHeader();
|
writer.writeHeader();
|
||||||
writer.write(inputStream, () -> false);
|
writer.write(inputStream);
|
||||||
verify(statusReporter, times(1)).startNewIncrementalCount();
|
verify(statusReporter, times(1)).startNewIncrementalCount();
|
||||||
|
|
||||||
List<String[]> expectedRecords = new ArrayList<>();
|
List<String[]> expectedRecords = new ArrayList<>();
|
||||||
|
@ -165,7 +134,7 @@ public class JsonDataToProcessWriterTests extends ESTestCase {
|
||||||
InputStream inputStream = createInputStream(input.toString());
|
InputStream inputStream = createInputStream(input.toString());
|
||||||
JsonDataToProcessWriter writer = createWriter();
|
JsonDataToProcessWriter writer = createWriter();
|
||||||
writer.writeHeader();
|
writer.writeHeader();
|
||||||
writer.write(inputStream, () -> false);
|
writer.write(inputStream);
|
||||||
|
|
||||||
List<String[]> expectedRecords = new ArrayList<>();
|
List<String[]> expectedRecords = new ArrayList<>();
|
||||||
// The final field is the control field
|
// The final field is the control field
|
||||||
|
@ -194,7 +163,7 @@ public class JsonDataToProcessWriterTests extends ESTestCase {
|
||||||
InputStream inputStream = createInputStream(input.toString());
|
InputStream inputStream = createInputStream(input.toString());
|
||||||
JsonDataToProcessWriter writer = createWriter();
|
JsonDataToProcessWriter writer = createWriter();
|
||||||
writer.writeHeader();
|
writer.writeHeader();
|
||||||
writer.write(inputStream, () -> false);
|
writer.write(inputStream);
|
||||||
verify(statusReporter, times(1)).startNewIncrementalCount();
|
verify(statusReporter, times(1)).startNewIncrementalCount();
|
||||||
|
|
||||||
List<String[]> expectedRecords = new ArrayList<>();
|
List<String[]> expectedRecords = new ArrayList<>();
|
||||||
|
@ -223,7 +192,7 @@ public class JsonDataToProcessWriterTests extends ESTestCase {
|
||||||
InputStream inputStream = createInputStream(input.toString());
|
InputStream inputStream = createInputStream(input.toString());
|
||||||
JsonDataToProcessWriter writer = createWriter();
|
JsonDataToProcessWriter writer = createWriter();
|
||||||
writer.writeHeader();
|
writer.writeHeader();
|
||||||
writer.write(inputStream, () -> false);
|
writer.write(inputStream);
|
||||||
verify(statusReporter, times(1)).startNewIncrementalCount();
|
verify(statusReporter, times(1)).startNewIncrementalCount();
|
||||||
|
|
||||||
List<String[]> expectedRecords = new ArrayList<>();
|
List<String[]> expectedRecords = new ArrayList<>();
|
||||||
|
@ -250,7 +219,7 @@ public class JsonDataToProcessWriterTests extends ESTestCase {
|
||||||
JsonDataToProcessWriter writer = createWriter();
|
JsonDataToProcessWriter writer = createWriter();
|
||||||
writer.writeHeader();
|
writer.writeHeader();
|
||||||
|
|
||||||
ESTestCase.expectThrows(ElasticsearchParseException.class, () -> writer.write(inputStream, () -> false));
|
ESTestCase.expectThrows(ElasticsearchParseException.class, () -> writer.write(inputStream));
|
||||||
}
|
}
|
||||||
|
|
||||||
public void testWrite_GivenJsonWithArrayField()
|
public void testWrite_GivenJsonWithArrayField()
|
||||||
|
@ -265,7 +234,7 @@ public class JsonDataToProcessWriterTests extends ESTestCase {
|
||||||
InputStream inputStream = createInputStream(input.toString());
|
InputStream inputStream = createInputStream(input.toString());
|
||||||
JsonDataToProcessWriter writer = createWriter();
|
JsonDataToProcessWriter writer = createWriter();
|
||||||
writer.writeHeader();
|
writer.writeHeader();
|
||||||
writer.write(inputStream, () -> false);
|
writer.write(inputStream);
|
||||||
verify(statusReporter, times(1)).startNewIncrementalCount();
|
verify(statusReporter, times(1)).startNewIncrementalCount();
|
||||||
|
|
||||||
List<String[]> expectedRecords = new ArrayList<>();
|
List<String[]> expectedRecords = new ArrayList<>();
|
||||||
|
@ -294,7 +263,7 @@ public class JsonDataToProcessWriterTests extends ESTestCase {
|
||||||
InputStream inputStream = createInputStream(input.toString());
|
InputStream inputStream = createInputStream(input.toString());
|
||||||
JsonDataToProcessWriter writer = createWriter();
|
JsonDataToProcessWriter writer = createWriter();
|
||||||
writer.writeHeader();
|
writer.writeHeader();
|
||||||
writer.write(inputStream, () -> false);
|
writer.write(inputStream);
|
||||||
verify(statusReporter, times(1)).startNewIncrementalCount();
|
verify(statusReporter, times(1)).startNewIncrementalCount();
|
||||||
|
|
||||||
List<String[]> expectedRecords = new ArrayList<>();
|
List<String[]> expectedRecords = new ArrayList<>();
|
||||||
|
@ -336,7 +305,7 @@ public class JsonDataToProcessWriterTests extends ESTestCase {
|
||||||
input.append("{\"date\":\"1970-01-01\", \"time-of-day\":\"00:00:02Z\", \"value\":\"6.0\"}");
|
input.append("{\"date\":\"1970-01-01\", \"time-of-day\":\"00:00:02Z\", \"value\":\"6.0\"}");
|
||||||
InputStream inputStream = createInputStream(input.toString());
|
InputStream inputStream = createInputStream(input.toString());
|
||||||
|
|
||||||
writer.write(inputStream, () -> false);
|
writer.write(inputStream);
|
||||||
verify(statusReporter, times(1)).startNewIncrementalCount();
|
verify(statusReporter, times(1)).startNewIncrementalCount();
|
||||||
|
|
||||||
List<String[]> expectedRecords = new ArrayList<>();
|
List<String[]> expectedRecords = new ArrayList<>();
|
||||||
|
@ -373,7 +342,7 @@ public class JsonDataToProcessWriterTests extends ESTestCase {
|
||||||
InputStream inputStream = createInputStream(input.toString());
|
InputStream inputStream = createInputStream(input.toString());
|
||||||
JsonDataToProcessWriter writer = createWriter();
|
JsonDataToProcessWriter writer = createWriter();
|
||||||
writer.writeHeader();
|
writer.writeHeader();
|
||||||
writer.write(inputStream, () -> false);
|
writer.write(inputStream);
|
||||||
verify(statusReporter, times(1)).startNewIncrementalCount();
|
verify(statusReporter, times(1)).startNewIncrementalCount();
|
||||||
|
|
||||||
List<String[]> expectedRecords = new ArrayList<>();
|
List<String[]> expectedRecords = new ArrayList<>();
|
||||||
|
@ -402,30 +371,4 @@ public class JsonDataToProcessWriterTests extends ESTestCase {
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
static InputStream endLessStream(String firstLine, String repeatLine) {
|
|
||||||
return new InputStream() {
|
|
||||||
|
|
||||||
int pos = 0;
|
|
||||||
boolean firstLineRead = false;
|
|
||||||
final byte[] first = firstLine.getBytes(StandardCharsets.UTF_8);
|
|
||||||
final byte[] repeat = repeatLine.getBytes(StandardCharsets.UTF_8);
|
|
||||||
|
|
||||||
@Override
|
|
||||||
public int read() throws IOException {
|
|
||||||
if (firstLineRead == false) {
|
|
||||||
if (pos == first.length) {
|
|
||||||
pos = 0;
|
|
||||||
firstLineRead = true;
|
|
||||||
} else {
|
|
||||||
return first[pos++];
|
|
||||||
}
|
|
||||||
}
|
|
||||||
|
|
||||||
if (pos == repeat.length) {
|
|
||||||
pos = 0;
|
|
||||||
}
|
|
||||||
return repeat[pos++];
|
|
||||||
}
|
|
||||||
};
|
|
||||||
}
|
|
||||||
}
|
}
|
||||||
|
|
|
@ -5,14 +5,20 @@
|
||||||
*/
|
*/
|
||||||
package org.elasticsearch.xpack.ml.job.process.autodetect.writer;
|
package org.elasticsearch.xpack.ml.job.process.autodetect.writer;
|
||||||
|
|
||||||
import static org.elasticsearch.xpack.ml.job.process.autodetect.writer.JsonDataToProcessWriterTests.endLessStream;
|
import org.apache.logging.log4j.Logger;
|
||||||
import static org.mockito.Matchers.any;
|
import org.elasticsearch.test.ESTestCase;
|
||||||
import static org.mockito.Matchers.anyLong;
|
import org.elasticsearch.xpack.ml.job.AnalysisConfig;
|
||||||
import static org.mockito.Mockito.atLeastOnce;
|
import org.elasticsearch.xpack.ml.job.DataDescription;
|
||||||
import static org.mockito.Mockito.doAnswer;
|
import org.elasticsearch.xpack.ml.job.DataDescription.DataFormat;
|
||||||
import static org.mockito.Mockito.times;
|
import org.elasticsearch.xpack.ml.job.Detector;
|
||||||
import static org.mockito.Mockito.verify;
|
import org.elasticsearch.xpack.ml.job.process.autodetect.AutodetectProcess;
|
||||||
import static org.mockito.Mockito.verifyNoMoreInteractions;
|
import org.elasticsearch.xpack.ml.job.status.StatusReporter;
|
||||||
|
import org.elasticsearch.xpack.ml.job.transform.TransformConfig;
|
||||||
|
import org.elasticsearch.xpack.ml.job.transform.TransformConfigs;
|
||||||
|
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.ByteArrayInputStream;
|
||||||
import java.io.IOException;
|
import java.io.IOException;
|
||||||
|
@ -21,25 +27,12 @@ import java.nio.charset.StandardCharsets;
|
||||||
import java.util.ArrayList;
|
import java.util.ArrayList;
|
||||||
import java.util.Arrays;
|
import java.util.Arrays;
|
||||||
import java.util.List;
|
import java.util.List;
|
||||||
import java.util.concurrent.atomic.AtomicBoolean;
|
|
||||||
import java.util.concurrent.atomic.AtomicReference;
|
|
||||||
|
|
||||||
import org.apache.logging.log4j.Logger;
|
import static org.mockito.Matchers.any;
|
||||||
import org.elasticsearch.tasks.TaskCancelledException;
|
import static org.mockito.Mockito.doAnswer;
|
||||||
import org.elasticsearch.test.ESTestCase;
|
import static org.mockito.Mockito.times;
|
||||||
import org.elasticsearch.xpack.ml.job.process.autodetect.AutodetectProcess;
|
import static org.mockito.Mockito.verify;
|
||||||
import org.junit.Before;
|
import static org.mockito.Mockito.verifyNoMoreInteractions;
|
||||||
import org.mockito.Mockito;
|
|
||||||
import org.mockito.invocation.InvocationOnMock;
|
|
||||||
import org.mockito.stubbing.Answer;
|
|
||||||
|
|
||||||
import org.elasticsearch.xpack.ml.job.AnalysisConfig;
|
|
||||||
import org.elasticsearch.xpack.ml.job.DataDescription;
|
|
||||||
import org.elasticsearch.xpack.ml.job.DataDescription.DataFormat;
|
|
||||||
import org.elasticsearch.xpack.ml.job.Detector;
|
|
||||||
import org.elasticsearch.xpack.ml.job.status.StatusReporter;
|
|
||||||
import org.elasticsearch.xpack.ml.job.transform.TransformConfig;
|
|
||||||
import org.elasticsearch.xpack.ml.job.transform.TransformConfigs;
|
|
||||||
|
|
||||||
public class SingleLineDataToProcessWriterTests extends ESTestCase {
|
public class SingleLineDataToProcessWriterTests extends ESTestCase {
|
||||||
private AutodetectProcess autodetectProcess;
|
private AutodetectProcess autodetectProcess;
|
||||||
|
@ -77,39 +70,6 @@ public class SingleLineDataToProcessWriterTests extends ESTestCase {
|
||||||
transformConfigs = new ArrayList<>();
|
transformConfigs = new ArrayList<>();
|
||||||
}
|
}
|
||||||
|
|
||||||
public void testWrite_cancel() throws Exception {
|
|
||||||
TransformConfig transformConfig = new TransformConfig("extract");
|
|
||||||
transformConfig.setInputs(Arrays.asList("raw"));
|
|
||||||
transformConfig.setOutputs(Arrays.asList("time", "message"));
|
|
||||||
transformConfig.setArguments(Arrays.asList("(.{20}) (.*)"));
|
|
||||||
transformConfigs.add(transformConfig);
|
|
||||||
|
|
||||||
InputStream inputStream = endLessStream("", "2015-04-29 10:00:00Z this is a message\n");
|
|
||||||
SingleLineDataToProcessWriter writer = createWriter();
|
|
||||||
writer.writeHeader();
|
|
||||||
|
|
||||||
AtomicBoolean cancel = new AtomicBoolean(false);
|
|
||||||
AtomicReference<Exception> exception = new AtomicReference<>();
|
|
||||||
Thread t = new Thread(() -> {
|
|
||||||
try {
|
|
||||||
writer.write(inputStream, cancel::get);
|
|
||||||
} catch (Exception e) {
|
|
||||||
exception.set(e);
|
|
||||||
}
|
|
||||||
});
|
|
||||||
t.start();
|
|
||||||
try {
|
|
||||||
assertBusy(() -> verify(statusReporter, atLeastOnce()).reportRecordWritten(anyLong(), anyLong()));
|
|
||||||
} finally {
|
|
||||||
cancel.set(true);
|
|
||||||
t.join();
|
|
||||||
}
|
|
||||||
|
|
||||||
assertNotNull(exception.get());
|
|
||||||
assertEquals(TaskCancelledException.class, exception.get().getClass());
|
|
||||||
assertEquals("cancelled", exception.get().getMessage());
|
|
||||||
}
|
|
||||||
|
|
||||||
public void testWrite_GivenDataIsValid() throws Exception {
|
public void testWrite_GivenDataIsValid() throws Exception {
|
||||||
TransformConfig transformConfig = new TransformConfig("extract");
|
TransformConfig transformConfig = new TransformConfig("extract");
|
||||||
transformConfig.setInputs(Arrays.asList("raw"));
|
transformConfig.setInputs(Arrays.asList("raw"));
|
||||||
|
@ -124,7 +84,7 @@ public class SingleLineDataToProcessWriterTests extends ESTestCase {
|
||||||
InputStream inputStream = createInputStream(input.toString());
|
InputStream inputStream = createInputStream(input.toString());
|
||||||
SingleLineDataToProcessWriter writer = createWriter();
|
SingleLineDataToProcessWriter writer = createWriter();
|
||||||
writer.writeHeader();
|
writer.writeHeader();
|
||||||
writer.write(inputStream, () -> false);
|
writer.write(inputStream);
|
||||||
verify(statusReporter, times(1)).getLatestRecordTime();
|
verify(statusReporter, times(1)).getLatestRecordTime();
|
||||||
verify(statusReporter, times(1)).startNewIncrementalCount();
|
verify(statusReporter, times(1)).startNewIncrementalCount();
|
||||||
verify(statusReporter, times(1)).setAnalysedFieldsPerRecord(1);
|
verify(statusReporter, times(1)).setAnalysedFieldsPerRecord(1);
|
||||||
|
@ -161,7 +121,7 @@ public class SingleLineDataToProcessWriterTests extends ESTestCase {
|
||||||
InputStream inputStream = createInputStream(input.toString());
|
InputStream inputStream = createInputStream(input.toString());
|
||||||
SingleLineDataToProcessWriter writer = createWriter();
|
SingleLineDataToProcessWriter writer = createWriter();
|
||||||
writer.writeHeader();
|
writer.writeHeader();
|
||||||
writer.write(inputStream, () -> false);
|
writer.write(inputStream);
|
||||||
verify(statusReporter, times(1)).getLatestRecordTime();
|
verify(statusReporter, times(1)).getLatestRecordTime();
|
||||||
verify(statusReporter, times(1)).startNewIncrementalCount();
|
verify(statusReporter, times(1)).startNewIncrementalCount();
|
||||||
verify(statusReporter, times(1)).setAnalysedFieldsPerRecord(1);
|
verify(statusReporter, times(1)).setAnalysedFieldsPerRecord(1);
|
||||||
|
@ -187,7 +147,7 @@ public class SingleLineDataToProcessWriterTests extends ESTestCase {
|
||||||
InputStream inputStream = createInputStream(input.toString());
|
InputStream inputStream = createInputStream(input.toString());
|
||||||
SingleLineDataToProcessWriter writer = createWriter();
|
SingleLineDataToProcessWriter writer = createWriter();
|
||||||
writer.writeHeader();
|
writer.writeHeader();
|
||||||
writer.write(inputStream, () -> false);
|
writer.write(inputStream);
|
||||||
verify(statusReporter, times(1)).startNewIncrementalCount();
|
verify(statusReporter, times(1)).startNewIncrementalCount();
|
||||||
verify(statusReporter, times(1)).setAnalysedFieldsPerRecord(1);
|
verify(statusReporter, times(1)).setAnalysedFieldsPerRecord(1);
|
||||||
verify(statusReporter, times(1)).reportDateParseError(1);
|
verify(statusReporter, times(1)).reportDateParseError(1);
|
||||||
|
|
|
@ -79,7 +79,7 @@ setup:
|
||||||
- do:
|
- do:
|
||||||
xpack.ml.flush_job:
|
xpack.ml.flush_job:
|
||||||
job_id: job-stats-test
|
job_id: job-stats-test
|
||||||
- match: { acknowledged: true }
|
- match: { flushed: true }
|
||||||
|
|
||||||
|
|
||||||
- do:
|
- do:
|
||||||
|
|
|
@ -58,12 +58,12 @@ setup:
|
||||||
- do:
|
- do:
|
||||||
xpack.ml.flush_job:
|
xpack.ml.flush_job:
|
||||||
job_id: farequote
|
job_id: farequote
|
||||||
- match: { acknowledged: true }
|
- match: { flushed: true }
|
||||||
|
|
||||||
- do:
|
- do:
|
||||||
xpack.ml.close_job:
|
xpack.ml.close_job:
|
||||||
job_id: farequote
|
job_id: farequote
|
||||||
- match: { acknowledged: true }
|
- match: { closed: true }
|
||||||
|
|
||||||
- do:
|
- do:
|
||||||
xpack.ml.get_job_stats:
|
xpack.ml.get_job_stats:
|
||||||
|
@ -91,12 +91,12 @@ setup:
|
||||||
- do:
|
- do:
|
||||||
xpack.ml.flush_job:
|
xpack.ml.flush_job:
|
||||||
job_id: farequote
|
job_id: farequote
|
||||||
- match: { acknowledged: true }
|
- match: { flushed: true }
|
||||||
|
|
||||||
- do:
|
- do:
|
||||||
xpack.ml.close_job:
|
xpack.ml.close_job:
|
||||||
job_id: farequote
|
job_id: farequote
|
||||||
- match: { acknowledged: true }
|
- match: { closed: true }
|
||||||
|
|
||||||
- do:
|
- do:
|
||||||
xpack.ml.get_job_stats:
|
xpack.ml.get_job_stats:
|
||||||
|
@ -169,17 +169,17 @@ setup:
|
||||||
---
|
---
|
||||||
"Test flushing, posting and closing a closed job":
|
"Test flushing, posting and closing a closed job":
|
||||||
- do:
|
- do:
|
||||||
catch: /illegal_argument_exception/
|
catch: /status_exception/
|
||||||
xpack.ml.flush_job:
|
xpack.ml.flush_job:
|
||||||
job_id: closed_job
|
job_id: closed_job
|
||||||
|
|
||||||
- do:
|
- do:
|
||||||
catch: /illegal_argument_exception/
|
catch: /status_exception/
|
||||||
xpack.ml.close_job:
|
xpack.ml.close_job:
|
||||||
job_id: closed_job
|
job_id: closed_job
|
||||||
|
|
||||||
- do:
|
- do:
|
||||||
catch: /illegal_argument_exception/
|
catch: /status_exception/
|
||||||
xpack.ml.post_data:
|
xpack.ml.post_data:
|
||||||
job_id: closed_job
|
job_id: closed_job
|
||||||
body: {}
|
body: {}
|
||||||
|
|
|
@ -0,0 +1,14 @@
|
||||||
|
apply plugin: 'elasticsearch.standalone-rest-test'
|
||||||
|
apply plugin: 'elasticsearch.rest-test'
|
||||||
|
|
||||||
|
dependencies {
|
||||||
|
testCompile project(path: ':elasticsearch', configuration: 'runtime')
|
||||||
|
}
|
||||||
|
|
||||||
|
integTest {
|
||||||
|
cluster {
|
||||||
|
numNodes = 3
|
||||||
|
distribution = 'zip'
|
||||||
|
plugin ':elasticsearch'
|
||||||
|
}
|
||||||
|
}
|
|
@ -0,0 +1,105 @@
|
||||||
|
/*
|
||||||
|
* Copyright Elasticsearch B.V. and/or licensed to Elasticsearch B.V. under one
|
||||||
|
* or more contributor license agreements. Licensed under the Elastic License;
|
||||||
|
* you may not use this file except in compliance with the Elastic License.
|
||||||
|
*/
|
||||||
|
package org.elasticsearch.xpack.ml.integration;
|
||||||
|
|
||||||
|
import org.apache.http.entity.StringEntity;
|
||||||
|
import org.elasticsearch.client.Response;
|
||||||
|
import org.elasticsearch.common.xcontent.XContentBuilder;
|
||||||
|
import org.elasticsearch.common.xcontent.XContentHelper;
|
||||||
|
import org.elasticsearch.test.rest.ESRestTestCase;
|
||||||
|
import org.elasticsearch.xpack.ml.MlPlugin;
|
||||||
|
|
||||||
|
import java.util.Collections;
|
||||||
|
import java.util.Map;
|
||||||
|
|
||||||
|
import static org.elasticsearch.common.xcontent.XContentFactory.jsonBuilder;
|
||||||
|
import static org.elasticsearch.common.xcontent.XContentType.JSON;
|
||||||
|
|
||||||
|
public class MlBasicMultiNodeIT extends ESRestTestCase {
|
||||||
|
|
||||||
|
public void testBasics() throws Exception {
|
||||||
|
String jobId = "foo";
|
||||||
|
createFarequoteJob(jobId);
|
||||||
|
|
||||||
|
Response response = client().performRequest("post", MlPlugin.BASE_PATH + "anomaly_detectors/" + jobId + "/_open");
|
||||||
|
assertEquals(200, response.getStatusLine().getStatusCode());
|
||||||
|
assertEquals(Collections.singletonMap("opened", true), responseEntityToMap(response));
|
||||||
|
|
||||||
|
String postData =
|
||||||
|
"{\"airline\":\"AAL\",\"responsetime\":\"132.2046\",\"sourcetype\":\"farequote\",\"time\":\"1403481600\"}\n" +
|
||||||
|
"{\"airline\":\"JZA\",\"responsetime\":\"990.4628\",\"sourcetype\":\"farequote\",\"time\":\"1403481700\"}";
|
||||||
|
response = client().performRequest("post", MlPlugin.BASE_PATH + "anomaly_detectors/" + jobId + "/_data",
|
||||||
|
Collections.emptyMap(), new StringEntity(postData));
|
||||||
|
assertEquals(202, response.getStatusLine().getStatusCode());
|
||||||
|
Map<String, Object> responseBody = responseEntityToMap(response);
|
||||||
|
assertEquals(2, responseBody.get("processed_record_count"));
|
||||||
|
assertEquals(4, responseBody.get("processed_field_count"));
|
||||||
|
assertEquals(177, responseBody.get("input_bytes"));
|
||||||
|
assertEquals(6, responseBody.get("input_field_count"));
|
||||||
|
assertEquals(0, responseBody.get("invalid_date_count"));
|
||||||
|
assertEquals(0, responseBody.get("missing_field_count"));
|
||||||
|
assertEquals(0, responseBody.get("out_of_order_timestamp_count"));
|
||||||
|
assertEquals(1403481600000L, responseBody.get("earliest_record_timestamp"));
|
||||||
|
assertEquals(1403481700000L, responseBody.get("latest_record_timestamp"));
|
||||||
|
|
||||||
|
response = client().performRequest("post", MlPlugin.BASE_PATH + "anomaly_detectors/" + jobId + "/_flush");
|
||||||
|
assertEquals(200, response.getStatusLine().getStatusCode());
|
||||||
|
assertEquals(Collections.singletonMap("flushed", true), responseEntityToMap(response));
|
||||||
|
|
||||||
|
response = client().performRequest("post", MlPlugin.BASE_PATH + "anomaly_detectors/" + jobId + "/_close");
|
||||||
|
assertEquals(200, response.getStatusLine().getStatusCode());
|
||||||
|
assertEquals(Collections.singletonMap("closed", true), responseEntityToMap(response));
|
||||||
|
|
||||||
|
response = client().performRequest("get", "/.ml-anomalies-" + jobId + "/data_counts/" + jobId + "-data-counts");
|
||||||
|
assertEquals(200, response.getStatusLine().getStatusCode());
|
||||||
|
@SuppressWarnings("unchecked")
|
||||||
|
Map<String, Object> dataCountsDoc = (Map<String, Object>) responseEntityToMap(response).get("_source");
|
||||||
|
assertEquals(2, dataCountsDoc.get("processed_record_count"));
|
||||||
|
assertEquals(4, dataCountsDoc.get("processed_field_count"));
|
||||||
|
assertEquals(177, dataCountsDoc.get("input_bytes"));
|
||||||
|
assertEquals(6, dataCountsDoc.get("input_field_count"));
|
||||||
|
assertEquals(0, dataCountsDoc.get("invalid_date_count"));
|
||||||
|
assertEquals(0, dataCountsDoc.get("missing_field_count"));
|
||||||
|
assertEquals(0, dataCountsDoc.get("out_of_order_timestamp_count"));
|
||||||
|
assertEquals(1403481600000L, dataCountsDoc.get("earliest_record_timestamp"));
|
||||||
|
assertEquals(1403481700000L, dataCountsDoc.get("latest_record_timestamp"));
|
||||||
|
|
||||||
|
response = client().performRequest("delete", MlPlugin.BASE_PATH + "anomaly_detectors/" + jobId);
|
||||||
|
assertEquals(200, response.getStatusLine().getStatusCode());
|
||||||
|
}
|
||||||
|
|
||||||
|
private Response createFarequoteJob(String jobId) throws Exception {
|
||||||
|
XContentBuilder xContentBuilder = jsonBuilder();
|
||||||
|
xContentBuilder.startObject();
|
||||||
|
xContentBuilder.field("job_id", jobId);
|
||||||
|
xContentBuilder.field("description", "Analysis of response time by airline");
|
||||||
|
|
||||||
|
xContentBuilder.startObject("analysis_config");
|
||||||
|
xContentBuilder.field("bucket_span", 3600);
|
||||||
|
xContentBuilder.startArray("detectors");
|
||||||
|
xContentBuilder.startObject();
|
||||||
|
xContentBuilder.field("function", "metric");
|
||||||
|
xContentBuilder.field("field_name", "responsetime");
|
||||||
|
xContentBuilder.field("by_field_name", "airline");
|
||||||
|
xContentBuilder.endObject();
|
||||||
|
xContentBuilder.endArray();
|
||||||
|
xContentBuilder.endObject();
|
||||||
|
|
||||||
|
xContentBuilder.startObject("data_description");
|
||||||
|
xContentBuilder.field("format", "JSON");
|
||||||
|
xContentBuilder.field("time_field", "time");
|
||||||
|
xContentBuilder.field("time_format", "epoch");
|
||||||
|
xContentBuilder.endObject();
|
||||||
|
xContentBuilder.endObject();
|
||||||
|
|
||||||
|
return client().performRequest("put", MlPlugin.BASE_PATH + "anomaly_detectors/" + jobId,
|
||||||
|
Collections.emptyMap(), new StringEntity(xContentBuilder.string()));
|
||||||
|
}
|
||||||
|
|
||||||
|
private static Map<String, Object> responseEntityToMap(Response response) throws Exception {
|
||||||
|
return XContentHelper.convertToMap(JSON.xContent(), response.getEntity().getContent(), false);
|
||||||
|
}
|
||||||
|
}
|
|
@ -2,3 +2,4 @@ rootProject.name = 'ml'
|
||||||
include ':elasticsearch'
|
include ':elasticsearch'
|
||||||
include ':docs'
|
include ':docs'
|
||||||
include ':kibana'
|
include ':kibana'
|
||||||
|
include ':qa:basic-multi-node'
|
||||||
|
|
Loading…
Reference in New Issue