[ML] Add overall buckets api (elastic/x-pack-elasticsearch#2713)

Adds the GET overall_buckets API.

The REST end point is: GET
/_xpack/ml/anomaly_detectors/job_id/results/overall_buckets

The API returns overall bucket results. An overall bucket
is a summarized bucket result over multiple jobs.
It has the `bucket_span` of the longest job's `bucket_span`.
It also has an `overall_score` that is the `top_n` average of the
max anomaly scores per job.

relates elastic/x-pack-elasticsearch#2693

Original commit: elastic/x-pack-elasticsearch@ba6061482d
This commit is contained in:
Dimitris Athanasiou 2017-10-10 14:41:24 +01:00 committed by GitHub
parent 90e327032e
commit 5eea355b33
19 changed files with 1590 additions and 4 deletions

View File

@ -69,6 +69,7 @@ buildRestTests.expectedUnconvertedCandidates = [
'en/rest-api/ml/delete-snapshot.asciidoc',
'en/rest-api/ml/flush-job.asciidoc',
'en/rest-api/ml/get-bucket.asciidoc',
'en/rest-api/ml/get-overall-buckets.asciidoc',
'en/rest-api/ml/get-category.asciidoc',
'en/rest-api/ml/get-datafeed-stats.asciidoc',
'en/rest-api/ml/get-job-stats.asciidoc',

View File

@ -51,6 +51,7 @@ The main {ml} resources can be accessed with a variety of endpoints:
* {ref}/ml-get-bucket.html[GET /results/buckets]: List the buckets in the results
* {ref}/ml-get-bucket.html[GET /results/buckets/<bucket_id+++>+++]: Get bucket details
* {ref}/ml-get-overall-buckets.html[GET /results/overall_buckets]: Get overall bucket results for multiple jobs
* {ref}/ml-get-category.html[GET /results/categories]: List the categories in the results
* {ref}/ml-get-category.html[GET /results/categories/<category_id+++>+++]: Get category details
* {ref}/ml-get-influencer.html[GET /results/influencers]: Get influencer details

View File

@ -46,6 +46,7 @@ machine learning APIs and in advanced job configuration options in Kibana.
=== Results
* <<ml-get-bucket,Get buckets>>
* <<ml-get-overall-buckets,Get overall buckets>>
* <<ml-get-category,Get categories>>
* <<ml-get-influencer,Get influencers>>
* <<ml-get-record,Get records>>
@ -64,6 +65,7 @@ include::ml/delete-snapshot.asciidoc[]
include::ml/flush-job.asciidoc[]
//GET
include::ml/get-bucket.asciidoc[]
include::ml/get-overall-buckets.asciidoc[]
include::ml/get-category.asciidoc[]
include::ml/get-datafeed.asciidoc[]
include::ml/get-datafeed-stats.asciidoc[]

View File

@ -0,0 +1,126 @@
[role="xpack"]
[[ml-get-overall-buckets]]
=== Get Overall Buckets
This API enables you to retrieve overall bucket results
that summarize the bucket results of multiple jobs.
==== Request
`GET _xpack/ml/anomaly_detectors/<job_id>/results/overall_buckets`
==== Description
An overall bucket has a span equal to the largest job's `bucket_span`.
The `overall_score` is calculated by combining the scores of all the
buckets within the overall bucket span. The max `anomaly_score` per
job is calculated. Then the `top_n` of those scores are averaged to
result into the `overall_score`. This means that you can fine-tune
the `overall_score` so that it is more or less sensitive to the number
of jobs that detect an anomaly at the same time.
For example, set `top_n` to 1 and the `overall_score` will be the
max bucket score in the overall bucket interval. To the contrary,
set `top_n` to the number of jobs and the `overall_score` will
only be high when all jobs present anomalies in that overall bucket.
==== Path Parameters
`job_id`::
(string) Identifier for the job. It can be a job identifier, a group name,
or a wildcard expression. If set to `_all`, the API returns overall buckets
for all jobs.
==== Request Body
`top_n`::
(integer) The number of top job bucket scores to be used in the
`overall_score` calculation. The default value is `1`.
`overall_score`::
(double) Returns overall buckets with overall scores higher than this value.
`exclude_interim`::
(boolean) If true, the output excludes interim overall buckets.
Overall buckets are interim if any of the job buckets within
the overall bucket interval is interim.
By default, interim results are included.
`start`::
(string) Returns overall buckets with timestamps after this time.
`end`::
(string) Returns overall buckets with timestamps earlier than this time.
`allow_no_jobs`::
(boolean) If `false` and the `job_id` does not match any job an error will
be returned. The default value is `true`.
===== Results
The API returns the following information:
`overall_buckets`::
(array) An array of overall bucket objects. For more information, see
<<ml-results-overall-buckets,Overall Buckets>>.
==== Authorization
You must have `monitor_ml`, `monitor`, `manage_ml`, or `manage` cluster
privileges to use this API. You also need `read` index privilege on the index
that stores the results. The `machine_learning_admin` and `machine_learning_user`
roles provide these privileges. For more information, see
{xpack-ref}/security-privileges.html[Security Privileges] and
{xpack-ref}/built-in-roles.html[Built-in Roles].
//<<security-privileges>> and <<built-in-roles>>.
==== Examples
The following example gets overall buckets for jobs with IDs matching `job-*`:
[source,js]
--------------------------------------------------
GET _xpack/ml/anomaly_detectors/job-*/results/overall_buckets
{
"overall_score": 90,
"start": "1403532000000"
}
--------------------------------------------------
// CONSOLE
// TEST[skip:todo]
In this example, the API returns a single result that matches the specified
score and time constraints:
[source,js]
----
{
"count": 1,
"overall_buckets": [
{
"timestamp" : 1403532000000,
"bucket_span" : 3600,
"overall_score" : 40.0,
"jobs" : [
{
"job_id" : "job-1",
"max_anomaly_score" : 30.0
},
{
"job_id" : "job-2",
"max_anomaly_score" : 10.0
},
{
"job_id" : "job-3",
"max_anomaly_score" : 80.0
}
],
"is_interim" : false,
"result_type" : "overall_bucket"
}
]
}
----

View File

@ -4,6 +4,8 @@
Several different result types are created for each job. You can query anomaly
results for _buckets_, _influencers_, and _records_ by using the results API.
Summarized bucket results over multiple jobs can be queried as well; those
results are called _overall buckets_.
Results are written for each `bucket_span`. The timestamp for the results is the
start of the bucket time interval.
@ -38,10 +40,12 @@ timestamp or any calculated scores. For more information, see
{xpack-ref}/ml-configuring-categories.html[Categorizing Log Messages].
//<<ml-configuring-categories>>.
* <<ml-results-buckets,Buckets>>
* <<ml-results-influencers,Influencers>>
* <<ml-results-records,Records>>
* <<ml-results-categories,Categories>>
* <<ml-results-overall-buckets,Overall Buckets>>
[float]
[[ml-results-buckets]]
@ -415,3 +419,36 @@ A category resource has the following properties:
`terms`::
(string) A space separated list of the common tokens that are matched in
values of the category.
[float]
[[ml-results-overall-buckets]]
==== Overall Buckets
Overall buckets provide a summary of bucket results over multiple jobs.
Their `bucket_span` equals the longest `bucket_span` of the jobs in question.
The `overall_score` is the `top_n` average of the max `anomaly_score` per job
within the overall bucket time interval.
This means that you can fine-tune the `overall_score` so that it is more
or less sensitive to the number of jobs that detect an anomaly at the same time.
An overall bucket resource has the following properties:
`timestamp`::
(date) The start time of the overall bucket.
`bucket_span`::
(number) The length of the bucket in seconds. Matches the `bucket_span`
of the job with the longest one.
`overall_score`::
(number) The `top_n` average of the max bucket `anomaly_score` per job.
`jobs`::
(array) An array of objects that contain the `max_anomaly_score` per `job_id`.
`is_interim`::
(boolean) If true, this is an interim result. In other words, the anomaly
record is calculated based on partial input data.
`result_type`::
(string) Internal. This is always set to `overall_bucket`.

View File

@ -54,6 +54,7 @@ import org.elasticsearch.xpack.ml.action.GetInfluencersAction;
import org.elasticsearch.xpack.ml.action.GetJobsAction;
import org.elasticsearch.xpack.ml.action.GetJobsStatsAction;
import org.elasticsearch.xpack.ml.action.GetModelSnapshotsAction;
import org.elasticsearch.xpack.ml.action.GetOverallBucketsAction;
import org.elasticsearch.xpack.ml.action.GetRecordsAction;
import org.elasticsearch.xpack.ml.action.IsolateDatafeedAction;
import org.elasticsearch.xpack.ml.action.KillProcessAction;
@ -123,6 +124,7 @@ import org.elasticsearch.xpack.ml.rest.modelsnapshots.RestUpdateModelSnapshotAct
import org.elasticsearch.xpack.ml.rest.results.RestGetBucketsAction;
import org.elasticsearch.xpack.ml.rest.results.RestGetCategoriesAction;
import org.elasticsearch.xpack.ml.rest.results.RestGetInfluencersAction;
import org.elasticsearch.xpack.ml.rest.results.RestGetOverallBucketsAction;
import org.elasticsearch.xpack.ml.rest.results.RestGetRecordsAction;
import org.elasticsearch.xpack.ml.rest.validate.RestValidateDetectorAction;
import org.elasticsearch.xpack.ml.rest.validate.RestValidateJobConfigAction;
@ -367,6 +369,7 @@ public class MachineLearning implements ActionPlugin {
new RestGetInfluencersAction(settings, restController),
new RestGetRecordsAction(settings, restController),
new RestGetBucketsAction(settings, restController),
new RestGetOverallBucketsAction(settings, restController),
new RestPostDataAction(settings, restController),
new RestCloseJobAction(settings, restController),
new RestFlushJobAction(settings, restController),
@ -408,6 +411,7 @@ public class MachineLearning implements ActionPlugin {
new ActionHandler<>(KillProcessAction.INSTANCE, KillProcessAction.TransportAction.class),
new ActionHandler<>(GetBucketsAction.INSTANCE, GetBucketsAction.TransportAction.class),
new ActionHandler<>(GetInfluencersAction.INSTANCE, GetInfluencersAction.TransportAction.class),
new ActionHandler<>(GetOverallBucketsAction.INSTANCE, GetOverallBucketsAction.TransportAction.class),
new ActionHandler<>(GetRecordsAction.INSTANCE, GetRecordsAction.TransportAction.class),
new ActionHandler<>(PostDataAction.INSTANCE, PostDataAction.TransportAction.class),
new ActionHandler<>(CloseJobAction.INSTANCE, CloseJobAction.TransportAction.class),

View File

@ -0,0 +1,480 @@
/*
* 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.apache.lucene.util.PriorityQueue;
import org.elasticsearch.ElasticsearchParseException;
import org.elasticsearch.action.Action;
import org.elasticsearch.action.ActionListener;
import org.elasticsearch.action.ActionRequest;
import org.elasticsearch.action.ActionRequestBuilder;
import org.elasticsearch.action.ActionRequestValidationException;
import org.elasticsearch.action.ActionResponse;
import org.elasticsearch.action.search.SearchRequest;
import org.elasticsearch.action.search.SearchResponse;
import org.elasticsearch.action.support.ActionFilters;
import org.elasticsearch.action.support.HandledTransportAction;
import org.elasticsearch.client.Client;
import org.elasticsearch.client.ElasticsearchClient;
import org.elasticsearch.cluster.metadata.IndexNameExpressionResolver;
import org.elasticsearch.cluster.service.ClusterService;
import org.elasticsearch.common.ParseField;
import org.elasticsearch.common.Strings;
import org.elasticsearch.common.inject.Inject;
import org.elasticsearch.common.io.stream.StreamInput;
import org.elasticsearch.common.io.stream.StreamOutput;
import org.elasticsearch.common.joda.DateMathParser;
import org.elasticsearch.common.settings.Settings;
import org.elasticsearch.common.unit.TimeValue;
import org.elasticsearch.common.xcontent.ObjectParser;
import org.elasticsearch.common.xcontent.ToXContentObject;
import org.elasticsearch.common.xcontent.XContentBuilder;
import org.elasticsearch.common.xcontent.XContentParser;
import org.elasticsearch.index.mapper.DateFieldMapper;
import org.elasticsearch.search.aggregations.AggregationBuilder;
import org.elasticsearch.search.aggregations.AggregationBuilders;
import org.elasticsearch.search.aggregations.Aggregations;
import org.elasticsearch.search.aggregations.bucket.histogram.Histogram;
import org.elasticsearch.search.aggregations.bucket.terms.Terms;
import org.elasticsearch.search.aggregations.metrics.max.Max;
import org.elasticsearch.search.builder.SearchSourceBuilder;
import org.elasticsearch.threadpool.ThreadPool;
import org.elasticsearch.transport.TransportService;
import org.elasticsearch.xpack.ml.action.util.QueryPage;
import org.elasticsearch.xpack.ml.job.JobManager;
import org.elasticsearch.xpack.ml.job.config.Job;
import org.elasticsearch.xpack.ml.job.messages.Messages;
import org.elasticsearch.xpack.ml.job.persistence.AnomalyDetectorsIndex;
import org.elasticsearch.xpack.ml.job.persistence.BucketsQueryBuilder;
import org.elasticsearch.xpack.ml.job.persistence.JobProvider;
import org.elasticsearch.xpack.ml.job.results.Bucket;
import org.elasticsearch.xpack.ml.job.results.OverallBucket;
import org.elasticsearch.xpack.ml.job.results.Result;
import org.elasticsearch.xpack.ml.utils.ExceptionsHelper;
import org.elasticsearch.xpack.ml.utils.Intervals;
import org.joda.time.DateTime;
import java.io.IOException;
import java.util.ArrayList;
import java.util.Collections;
import java.util.Date;
import java.util.List;
import java.util.Objects;
import java.util.function.LongSupplier;
/**
* This action returns summarized bucket results over multiple jobs.
* Overall buckets have the span of the largest job's bucket_span.
* Their score is calculated by finding the max anomaly score per job
* and then averaging the top N.
*/
public class GetOverallBucketsAction
extends Action<GetOverallBucketsAction.Request, GetOverallBucketsAction.Response, GetOverallBucketsAction.RequestBuilder> {
public static final GetOverallBucketsAction INSTANCE = new GetOverallBucketsAction();
public static final String NAME = "cluster:monitor/xpack/ml/job/results/overall_buckets/get";
private GetOverallBucketsAction() {
super(NAME);
}
@Override
public RequestBuilder newRequestBuilder(ElasticsearchClient client) {
return new RequestBuilder(client);
}
@Override
public Response newResponse() {
return new Response();
}
public static class Request extends ActionRequest implements ToXContentObject {
public static final ParseField TOP_N = new ParseField("top_n");
public static final ParseField OVERALL_SCORE = new ParseField("overall_score");
public static final ParseField EXCLUDE_INTERIM = new ParseField("exclude_interim");
public static final ParseField START = new ParseField("start");
public static final ParseField END = new ParseField("end");
public static final ParseField ALLOW_NO_JOBS = new ParseField("allow_no_jobs");
private static final ObjectParser<Request, Void> PARSER = new ObjectParser<>(NAME, Request::new);
static {
PARSER.declareString((request, jobId) -> request.jobId = jobId, Job.ID);
PARSER.declareInt(Request::setTopN, TOP_N);
PARSER.declareDouble(Request::setOverallScore, OVERALL_SCORE);
PARSER.declareBoolean(Request::setExcludeInterim, EXCLUDE_INTERIM);
PARSER.declareString((request, startTime) -> request.setStart(parseDateOrThrow(
startTime, START, System::currentTimeMillis)), START);
PARSER.declareString((request, endTime) -> request.setEnd(parseDateOrThrow(
endTime, END, System::currentTimeMillis)), END);
PARSER.declareBoolean(Request::setAllowNoJobs, ALLOW_NO_JOBS);
}
static long parseDateOrThrow(String date, ParseField paramName, LongSupplier now) {
DateMathParser dateMathParser = new DateMathParser(DateFieldMapper.DEFAULT_DATE_TIME_FORMATTER);
try {
return dateMathParser.parse(date, now);
} catch (Exception e) {
String msg = Messages.getMessage(Messages.REST_INVALID_DATETIME_PARAMS, paramName.getPreferredName(), date);
throw new ElasticsearchParseException(msg, e);
}
}
public static Request parseRequest(String jobId, XContentParser parser) {
Request request = PARSER.apply(parser, null);
if (jobId != null) {
request.jobId = jobId;
}
return request;
}
private String jobId;
private int topN = 1;
private double overallScore = 0.0;
private boolean excludeInterim = false;
private Long start;
private Long end;
private boolean allowNoJobs = true;
Request() {
}
public Request(String jobId) {
this.jobId = ExceptionsHelper.requireNonNull(jobId, Job.ID.getPreferredName());
}
public String getJobId() {
return jobId;
}
public int getTopN() {
return topN;
}
public void setTopN(int topN) {
if (topN <= 0) {
throw new IllegalArgumentException("[topN] parameter must be positive, found [" + topN + "]");
}
this.topN = topN;
}
public double getOverallScore() {
return overallScore;
}
public void setOverallScore(double overallScore) {
this.overallScore = overallScore;
}
public boolean isExcludeInterim() {
return excludeInterim;
}
public void setExcludeInterim(boolean excludeInterim) {
this.excludeInterim = excludeInterim;
}
public Long getStart() {
return start;
}
public void setStart(Long start) {
this.start = start;
}
public void setStart(String start) {
setStart(parseDateOrThrow(start, START, System::currentTimeMillis));
}
public Long getEnd() {
return end;
}
public void setEnd(Long end) {
this.end = end;
}
public void setEnd(String end) {
setEnd(parseDateOrThrow(end, END, System::currentTimeMillis));
}
public boolean allowNoJobs() {
return allowNoJobs;
}
public void setAllowNoJobs(boolean allowNoJobs) {
this.allowNoJobs = allowNoJobs;
}
@Override
public ActionRequestValidationException validate() {
return null;
}
@Override
public void readFrom(StreamInput in) throws IOException {
super.readFrom(in);
jobId = in.readString();
topN = in.readVInt();
overallScore = in.readDouble();
excludeInterim = in.readBoolean();
start = in.readOptionalLong();
end = in.readOptionalLong();
allowNoJobs = in.readBoolean();
}
@Override
public void writeTo(StreamOutput out) throws IOException {
super.writeTo(out);
out.writeString(jobId);
out.writeVInt(topN);
out.writeDouble(overallScore);
out.writeBoolean(excludeInterim);
out.writeOptionalLong(start);
out.writeOptionalLong(end);
out.writeBoolean(allowNoJobs);
}
@Override
public XContentBuilder toXContent(XContentBuilder builder, Params params) throws IOException {
builder.startObject();
builder.field(Job.ID.getPreferredName(), jobId);
builder.field(TOP_N.getPreferredName(), topN);
builder.field(OVERALL_SCORE.getPreferredName(), overallScore);
builder.field(EXCLUDE_INTERIM.getPreferredName(), excludeInterim);
if (start != null) {
builder.field(START.getPreferredName(), String.valueOf(start));
}
if (end != null) {
builder.field(END.getPreferredName(), String.valueOf(end));
}
builder.field(ALLOW_NO_JOBS.getPreferredName(), allowNoJobs);
builder.endObject();
return builder;
}
@Override
public int hashCode() {
return Objects.hash(jobId, topN, overallScore, excludeInterim, start, end, allowNoJobs);
}
@Override
public boolean equals(Object other) {
if (other == null) {
return false;
}
if (getClass() != other.getClass()) {
return false;
}
Request that = (Request) other;
return Objects.equals(jobId, that.jobId) &&
this.topN == that.topN &&
this.excludeInterim == that.excludeInterim &&
this.overallScore == that.overallScore &&
Objects.equals(start, that.start) &&
Objects.equals(end, that.end) &&
this.allowNoJobs == that.allowNoJobs;
}
}
static class RequestBuilder extends ActionRequestBuilder<Request, Response, RequestBuilder> {
RequestBuilder(ElasticsearchClient client) {
super(client, INSTANCE, new Request());
}
}
public static class Response extends ActionResponse implements ToXContentObject {
private QueryPage<OverallBucket> overallBuckets;
Response() {
overallBuckets = new QueryPage<>(Collections.emptyList(), 0, OverallBucket.RESULTS_FIELD);
}
Response(QueryPage<OverallBucket> overallBuckets) {
this.overallBuckets = overallBuckets;
}
@Override
public void readFrom(StreamInput in) throws IOException {
super.readFrom(in);
overallBuckets = new QueryPage<>(in, OverallBucket::new);
}
@Override
public void writeTo(StreamOutput out) throws IOException {
super.writeTo(out);
overallBuckets.writeTo(out);
}
@Override
public XContentBuilder toXContent(XContentBuilder builder, Params params) throws IOException {
builder.startObject();
overallBuckets.doXContentBody(builder, params);
builder.endObject();
return builder;
}
@Override
public int hashCode() {
return Objects.hash(overallBuckets);
}
@Override
public boolean equals(Object obj) {
if (obj == null) {
return false;
}
if (getClass() != obj.getClass()) {
return false;
}
Response other = (Response) obj;
return Objects.equals(overallBuckets, other.overallBuckets);
}
@Override
public final String toString() {
return Strings.toString(this);
}
}
public static class TransportAction extends HandledTransportAction<Request, Response> {
private final Client client;
private final ClusterService clusterService;
private final JobManager jobManager;
@Inject
public TransportAction(Settings settings, ThreadPool threadPool, TransportService transportService,
ActionFilters actionFilters, IndexNameExpressionResolver indexNameExpressionResolver,
ClusterService clusterService, JobManager jobManager, Client client) {
super(settings, NAME, threadPool, transportService, actionFilters, indexNameExpressionResolver, Request::new);
this.clusterService = clusterService;
this.client = client;
this.jobManager = jobManager;
}
@Override
protected void doExecute(Request request, ActionListener<Response> listener) {
QueryPage<Job> jobsPage = jobManager.expandJobs(request.getJobId(), request.allowNoJobs(), clusterService.state());
if (jobsPage.count() == 0) {
listener.onResponse(new Response());
return;
}
List<String> indices = new ArrayList<>();
TimeValue maxBucketSpan = TimeValue.ZERO;
for (Job job : jobsPage.results()) {
indices.add(AnomalyDetectorsIndex.jobResultsAliasedName(job.getId()));
TimeValue bucketSpan = job.getAnalysisConfig().getBucketSpan();
if (maxBucketSpan.compareTo(bucketSpan) < 0) {
maxBucketSpan = bucketSpan;
}
}
final long maxBucketSpanSeconds = maxBucketSpan.seconds();
SearchRequest searchRequest = buildSearchRequest(request, maxBucketSpan.millis(), indices);
client.search(searchRequest, ActionListener.wrap(searchResponse -> {
List<OverallBucket> overallBuckets = computeOverallBuckets(request, maxBucketSpanSeconds, searchResponse);
listener.onResponse(new Response(new QueryPage<>(overallBuckets, overallBuckets.size(), OverallBucket.RESULTS_FIELD)));
}, listener::onFailure));
}
private static SearchRequest buildSearchRequest(Request request, long bucketSpanMillis, List<String> indices) {
String startTime = request.getStart() == null ? null : String.valueOf(
Intervals.alignToCeil(request.getStart(), bucketSpanMillis));
String endTime = request.getEnd() == null ? null : String.valueOf(Intervals.alignToFloor(request.getEnd(), bucketSpanMillis));
SearchSourceBuilder searchSourceBuilder = new BucketsQueryBuilder()
.size(0)
.includeInterim(request.isExcludeInterim() == false)
.start(startTime)
.end(endTime)
.build();
searchSourceBuilder.aggregation(buildAggregations(bucketSpanMillis));
SearchRequest searchRequest = new SearchRequest(indices.toArray(new String[indices.size()]));
searchRequest.indicesOptions(JobProvider.addIgnoreUnavailable(SearchRequest.DEFAULT_INDICES_OPTIONS));
searchRequest.source(searchSourceBuilder);
return searchRequest;
}
private static AggregationBuilder buildAggregations(long bucketSpanMillis) {
AggregationBuilder overallScoreAgg = AggregationBuilders.max(OverallBucket.OVERALL_SCORE.getPreferredName())
.field(Bucket.ANOMALY_SCORE.getPreferredName());
AggregationBuilder jobsAgg = AggregationBuilders.terms(Job.ID.getPreferredName())
.field(Job.ID.getPreferredName()).subAggregation(overallScoreAgg);
AggregationBuilder interimAgg = AggregationBuilders.max(Result.IS_INTERIM.getPreferredName())
.field(Result.IS_INTERIM.getPreferredName());
return AggregationBuilders.dateHistogram(Result.TIMESTAMP.getPreferredName())
.field(Result.TIMESTAMP.getPreferredName())
.interval(bucketSpanMillis)
.subAggregation(jobsAgg)
.subAggregation(interimAgg);
}
private List<OverallBucket> computeOverallBuckets(Request request, long bucketSpanSeconds, SearchResponse searchResponse) {
List<OverallBucket> overallBuckets = new ArrayList<>();
Histogram histogram = searchResponse.getAggregations().get(Result.TIMESTAMP.getPreferredName());
for (Histogram.Bucket histogramBucket : histogram.getBuckets()) {
Aggregations histogramBucketAggs = histogramBucket.getAggregations();
Terms jobsAgg = histogramBucketAggs.get(Job.ID.getPreferredName());
int jobsCount = jobsAgg.getBuckets().size();
int topN = Math.min(request.getTopN(), jobsCount);
List<OverallBucket.JobInfo> jobs = new ArrayList<>(jobsCount);
TopNScores topNScores = new TopNScores(topN);
for (Terms.Bucket jobsBucket : jobsAgg.getBuckets()) {
Max maxScore = jobsBucket.getAggregations().get(OverallBucket.OVERALL_SCORE.getPreferredName());
topNScores.insertWithOverflow(maxScore.getValue());
jobs.add(new OverallBucket.JobInfo((String) jobsBucket.getKey(), maxScore.getValue()));
}
double overallScore = topNScores.overallScore();
if (overallScore < request.getOverallScore()) {
continue;
}
Max interimAgg = histogramBucketAggs.get(Result.IS_INTERIM.getPreferredName());
boolean isInterim = interimAgg.getValue() > 0;
if (request.isExcludeInterim() && isInterim) {
continue;
}
overallBuckets.add(new OverallBucket(getHistogramBucketTimestamp(histogramBucket),
bucketSpanSeconds, overallScore, jobs, isInterim));
}
return overallBuckets;
}
private static Date getHistogramBucketTimestamp(Histogram.Bucket bucket) {
DateTime bucketTimestamp = (DateTime) bucket.getKey();
return new Date(bucketTimestamp.getMillis());
}
static class TopNScores extends PriorityQueue<Double> {
TopNScores(int n) {
super(n, false);
}
@Override
protected boolean lessThan(Double a, Double b) {
return a < b;
}
double overallScore() {
double overallScore = 0.0;
for (double score : this) {
overallScore += score;
}
return size() > 0 ? overallScore / size() : 0.0;
}
}
}
}

View File

@ -163,7 +163,7 @@ public final class Messages {
public static final String REST_CANNOT_DELETE_HIGHEST_PRIORITY =
"Model snapshot ''{0}'' is the active snapshot for job ''{1}'', so cannot be deleted";
public static final String REST_INVALID_DATETIME_PARAMS =
"Query param ''{0}'' with value ''{1}'' cannot be parsed as a date or converted to a number (epoch).";
"Query param [{0}] with value [{1}] cannot be parsed as a date or converted to a number (epoch).";
public static final String REST_INVALID_FLUSH_PARAMS_MISSING = "Invalid flush parameters: ''{0}'' has not been specified.";
public static final String REST_INVALID_FLUSH_PARAMS_UNEXPECTED = "Invalid flush parameters: unexpected ''{0}''.";
public static final String REST_JOB_NOT_CLOSED_REVERT = "Can only revert to a model snapshot when the job is closed.";

View File

@ -0,0 +1,178 @@
/*
* 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.results;
import org.elasticsearch.common.ParseField;
import org.elasticsearch.common.io.stream.StreamInput;
import org.elasticsearch.common.io.stream.StreamOutput;
import org.elasticsearch.common.io.stream.Writeable;
import org.elasticsearch.common.xcontent.ToXContentObject;
import org.elasticsearch.common.xcontent.XContentBuilder;
import org.elasticsearch.xpack.ml.job.config.Job;
import org.elasticsearch.xpack.ml.utils.ExceptionsHelper;
import java.io.IOException;
import java.util.Date;
import java.util.List;
import java.util.Objects;
/**
* Overall Bucket Result POJO
*/
public class OverallBucket implements ToXContentObject, Writeable {
public static final ParseField OVERALL_SCORE = new ParseField("overall_score");
public static final ParseField BUCKET_SPAN = new ParseField("bucket_span");
public static final ParseField JOBS = new ParseField("jobs");
// Used for QueryPage
public static final ParseField RESULTS_FIELD = new ParseField("overall_buckets");
/**
* Result type
*/
public static final String RESULT_TYPE_VALUE = "overall_bucket";
private final Date timestamp;
private final long bucketSpan;
private final double overallScore;
private final List<JobInfo> jobs;
private final boolean isInterim;
public OverallBucket(Date timestamp, long bucketSpan, double overallScore, List<JobInfo> jobs, boolean isInterim) {
this.timestamp = ExceptionsHelper.requireNonNull(timestamp, Result.TIMESTAMP.getPreferredName());
this.bucketSpan = bucketSpan;
this.overallScore = overallScore;
this.jobs = jobs;
this.isInterim = isInterim;
}
public OverallBucket(StreamInput in) throws IOException {
timestamp = new Date(in.readLong());
bucketSpan = in.readLong();
overallScore = in.readDouble();
jobs = in.readList(JobInfo::new);
isInterim = in.readBoolean();
}
@Override
public void writeTo(StreamOutput out) throws IOException {
out.writeLong(timestamp.getTime());
out.writeLong(bucketSpan);
out.writeDouble(overallScore);
out.writeList(jobs);
out.writeBoolean(isInterim);
}
@Override
public XContentBuilder toXContent(XContentBuilder builder, Params params) throws IOException {
builder.startObject();
builder.dateField(Result.TIMESTAMP.getPreferredName(), Result.TIMESTAMP.getPreferredName() + "_string", timestamp.getTime());
builder.field(BUCKET_SPAN.getPreferredName(), bucketSpan);
builder.field(OVERALL_SCORE.getPreferredName(), overallScore);
builder.field(JOBS.getPreferredName(), jobs);
builder.field(Result.IS_INTERIM.getPreferredName(), isInterim);
builder.field(Result.RESULT_TYPE.getPreferredName(), RESULT_TYPE_VALUE);
builder.endObject();
return builder;
}
public Date getTimestamp() {
return timestamp;
}
/**
* Bucketspan expressed in seconds
*/
public long getBucketSpan() {
return bucketSpan;
}
public double getOverallScore() {
return overallScore;
}
public boolean isInterim() {
return isInterim;
}
@Override
public int hashCode() {
return Objects.hash(timestamp, bucketSpan, overallScore, jobs, isInterim);
}
/**
* Compare all the fields and embedded anomaly records (if any)
*/
@Override
public boolean equals(Object other) {
if (this == other) {
return true;
}
if (other instanceof OverallBucket == false) {
return false;
}
OverallBucket that = (OverallBucket) other;
return Objects.equals(this.timestamp, that.timestamp)
&& this.bucketSpan == that.bucketSpan
&& this.overallScore == that.overallScore
&& Objects.equals(this.jobs, that.jobs)
&& this.isInterim == that.isInterim;
}
public static class JobInfo implements ToXContentObject, Writeable {
private static final ParseField MAX_ANOMALY_SCORE = new ParseField("max_anomaly_score");
private final String jobId;
private final double maxAnomalyScore;
public JobInfo(String jobId, double maxAnomalyScore) {
this.jobId = Objects.requireNonNull(jobId);
this.maxAnomalyScore = maxAnomalyScore;
}
public JobInfo(StreamInput in) throws IOException {
jobId = in.readString();
maxAnomalyScore = in.readDouble();
}
@Override
public void writeTo(StreamOutput out) throws IOException {
out.writeString(jobId);
out.writeDouble(maxAnomalyScore);
}
@Override
public XContentBuilder toXContent(XContentBuilder builder, Params params) throws IOException {
builder.startObject();
builder.field(Job.ID.getPreferredName(), jobId);
builder.field(MAX_ANOMALY_SCORE.getPreferredName(), maxAnomalyScore);
builder.endObject();
return builder;
}
@Override
public int hashCode() {
return Objects.hash(jobId, maxAnomalyScore);
}
@Override
public boolean equals(Object other) {
if (this == other) {
return true;
}
if (other instanceof JobInfo == false) {
return false;
}
JobInfo that = (JobInfo) other;
return Objects.equals(this.jobId, that.jobId) && this.maxAnomalyScore == that.maxAnomalyScore;
}
}
}

View File

@ -0,0 +1,60 @@
/*
* 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.rest.results;
import org.elasticsearch.client.node.NodeClient;
import org.elasticsearch.common.settings.Settings;
import org.elasticsearch.common.xcontent.XContentParser;
import org.elasticsearch.rest.BaseRestHandler;
import org.elasticsearch.rest.RestController;
import org.elasticsearch.rest.RestRequest;
import org.elasticsearch.rest.action.RestToXContentListener;
import org.elasticsearch.xpack.ml.MachineLearning;
import org.elasticsearch.xpack.ml.action.GetOverallBucketsAction;
import org.elasticsearch.xpack.ml.action.GetOverallBucketsAction.Request;
import org.elasticsearch.xpack.ml.job.config.Job;
import java.io.IOException;
public class RestGetOverallBucketsAction extends BaseRestHandler {
public RestGetOverallBucketsAction(Settings settings, RestController controller) {
super(settings);
controller.registerHandler(RestRequest.Method.GET,
MachineLearning.BASE_PATH + "anomaly_detectors/{" + Job.ID.getPreferredName() + "}/results/overall_buckets", this);
controller.registerHandler(RestRequest.Method.POST,
MachineLearning.BASE_PATH + "anomaly_detectors/{" + Job.ID.getPreferredName() + "}/results/overall_buckets", this);
}
@Override
public String getName() {
return "xpack_ml_get_overall_buckets_action";
}
@Override
protected RestChannelConsumer prepareRequest(RestRequest restRequest, NodeClient client) throws IOException {
String jobId = restRequest.param(Job.ID.getPreferredName());
final Request request;
if (restRequest.hasContentOrSourceParam()) {
XContentParser parser = restRequest.contentOrSourceParamParser();
request = Request.parseRequest(jobId, parser);
} else {
request = new Request(jobId);
request.setTopN(restRequest.paramAsInt(Request.TOP_N.getPreferredName(), request.getTopN()));
request.setOverallScore(Double.parseDouble(restRequest.param(Request.OVERALL_SCORE.getPreferredName(), "0.0")));
request.setExcludeInterim(restRequest.paramAsBoolean(Request.EXCLUDE_INTERIM.getPreferredName(), request.isExcludeInterim()));
if (restRequest.hasParam(Request.START.getPreferredName())) {
request.setStart(restRequest.param(Request.START.getPreferredName()));
}
if (restRequest.hasParam(Request.END.getPreferredName())) {
request.setEnd(restRequest.param(Request.END.getPreferredName()));
}
request.setAllowNoJobs(restRequest.paramAsBoolean(Request.ALLOW_NO_JOBS.getPreferredName(), request.allowNoJobs()));
}
return channel -> client.execute(GetOverallBucketsAction.INSTANCE, request, new RestToXContentListener<>(channel));
}
}

View File

@ -0,0 +1,52 @@
/*
* 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.common.xcontent.XContentParser;
import org.elasticsearch.test.AbstractStreamableXContentTestCase;
import org.elasticsearch.xpack.ml.action.GetOverallBucketsAction.Request;
public class GetOverallBucketsActionRequestTests extends AbstractStreamableXContentTestCase<Request> {
@Override
protected Request createTestInstance() {
Request request = new Request(randomAlphaOfLengthBetween(1, 20));
if (randomBoolean()) {
request.setTopN(randomIntBetween(1, 1000));
}
request.setAllowNoJobs(randomBoolean());
if (randomBoolean()) {
request.setStart(randomNonNegativeLong());
}
if (randomBoolean()) {
request.setExcludeInterim(randomBoolean());
}
if (randomBoolean()) {
request.setOverallScore(randomDouble());
}
if (randomBoolean()) {
request.setEnd(randomNonNegativeLong());
}
return request;
}
@Override
protected boolean supportsUnknownFields() {
return false;
}
@Override
protected Request createBlankInstance() {
return new Request();
}
@Override
protected Request doParseInstance(XContentParser parser) {
return Request.parseRequest(null, parser);
}
}

View File

@ -0,0 +1,40 @@
/*
* 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.test.AbstractStreamableTestCase;
import org.elasticsearch.xpack.ml.action.GetOverallBucketsAction.Response;
import org.elasticsearch.xpack.ml.action.util.QueryPage;
import org.elasticsearch.xpack.ml.job.results.OverallBucket;
import java.util.ArrayList;
import java.util.Date;
import java.util.List;
public class GetOverallBucketsActionResponseTests extends AbstractStreamableTestCase<Response> {
@Override
protected Response createTestInstance() {
int listSize = randomInt(10);
List<OverallBucket> hits = new ArrayList<>(listSize);
String jobId = randomAlphaOfLengthBetween(1, 20);
for (int bucketIndex = 0; bucketIndex < listSize; bucketIndex++) {
int jobsCount = randomInt(5);
List<OverallBucket.JobInfo> jobs = new ArrayList<>(jobsCount);
for (int jobIndex = 0; jobIndex < jobsCount; jobIndex++) {
jobs.add(new OverallBucket.JobInfo(jobId, randomDouble()));
}
hits.add(new OverallBucket(new Date(randomNonNegativeLong()), randomNonNegativeLong(), randomDouble(), jobs, randomBoolean()));
}
QueryPage<OverallBucket> snapshots = new QueryPage<>(hits, listSize, OverallBucket.RESULTS_FIELD);
return new Response(snapshots);
}
@Override
protected Response createBlankInstance() {
return new Response();
}
}

View File

@ -0,0 +1,29 @@
/*
* 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.test.ESTestCase;
import org.elasticsearch.xpack.ml.action.GetOverallBucketsAction.TransportAction.TopNScores;
import static org.hamcrest.Matchers.equalTo;
public class GetOverallBucketsActionTests extends ESTestCase {
public void testTopNScores() {
TopNScores topNScores = new TopNScores(3);
assertThat(topNScores.overallScore(), equalTo(0.0));
topNScores.insertWithOverflow(5.0);
assertThat(topNScores.overallScore(), equalTo(5.0));
topNScores.insertWithOverflow(4.0);
assertThat(topNScores.overallScore(), equalTo(4.5));
topNScores.insertWithOverflow(3.0);
assertThat(topNScores.overallScore(), equalTo(4.0));
topNScores.insertWithOverflow(6.0);
assertThat(topNScores.overallScore(), equalTo(5.0));
topNScores.insertWithOverflow(10.0);
assertThat(topNScores.overallScore(), equalTo(7.0));
}
}

View File

@ -54,7 +54,7 @@ public class StartDatafeedActionRequestTests extends AbstractStreamableXContentT
Exception e = expectThrows(ElasticsearchParseException.class,
() -> StartDatafeedAction.DatafeedParams.parseDateOrThrow("not-a-date",
StartDatafeedAction.START_TIME, () -> System.currentTimeMillis()));
assertEquals("Query param 'start' with value 'not-a-date' cannot be parsed as a date or converted to a number (epoch).",
assertEquals("Query param [start] with value [not-a-date] cannot be parsed as a date or converted to a number (epoch).",
e.getMessage());
}
}

View File

@ -32,7 +32,7 @@ public class RestStartDatafeedActionTests extends ESTestCase {
.withParams(params).build();
ElasticsearchParseException e = expectThrows(ElasticsearchParseException.class,
() -> action.prepareRequest(restRequest1, mock(NodeClient.class)));
assertEquals("Query param 'start' with value 'not-a-date' cannot be parsed as a date or " +
assertEquals("Query param [start] with value [not-a-date] cannot be parsed as a date or " +
"converted to a number (epoch).",
e.getMessage());
@ -44,7 +44,7 @@ public class RestStartDatafeedActionTests extends ESTestCase {
.withParams(params).build();
e = expectThrows(ElasticsearchParseException.class,
() -> action.prepareRequest(restRequest2, mock(NodeClient.class)));
assertEquals("Query param 'end' with value 'not-a-date' cannot be parsed as a date or " +
assertEquals("Query param [end] with value [not-a-date] cannot be parsed as a date or " +
"converted to a number (epoch).", e.getMessage());
}
}

View File

@ -115,6 +115,7 @@ cluster:admin/xpack/ml/filters/get
cluster:monitor/xpack/ml/job/results/categories/get
cluster:monitor/xpack/ml/job/stats/get
cluster:monitor/xpack/ml/job/results/buckets/get
cluster:monitor/xpack/ml/job/results/overall_buckets/get
cluster:monitor/xpack/ml/job/model_snapshots/get
cluster:monitor/xpack/ml/job/results/records/get
cluster:monitor/xpack/ml/job/results/influencers/get

View File

@ -0,0 +1,49 @@
{
"xpack.ml.get_overall_buckets": {
"documentation": "http://www.elastic.co/guide/en/elasticsearch/reference/current/ml-get-overall-buckets.html",
"methods": [ "GET", "POST" ],
"url": {
"path": "/_xpack/ml/anomaly_detectors/{job_id}/results/overall_buckets",
"paths": [
"/_xpack/ml/anomaly_detectors/{job_id}/results/overall_buckets"
],
"parts": {
"job_id": {
"type" : "string",
"required": true,
"description": "The job IDs for which to calculate overall bucket results"
}
},
"params": {
"top_n": {
"type": "int",
"description": "The number of top job bucket scores to be used in the overall_score calculation"
},
"overall_score": {
"type": "double",
"description": "Returns overall buckets with overall scores higher than this value"
},
"exclude_interim": {
"type": "boolean",
"description" : "If true overall buckets that include interim buckets will be excluded"
},
"start": {
"type": "string",
"description" : "Returns overall buckets with timestamps after this time"
},
"end": {
"type": "string",
"description" : "Returns overall buckets with timestamps earlier than this time"
},
"allow_no_jobs": {
"type": "boolean",
"required": false,
"description": "Whether to ignore if a wildcard expression matches no jobs. (This includes `_all` string or when no jobs have been specified)"
}
}
},
"body": {
"description" : "Overall bucket selection details if not provided in URI"
}
}
}

View File

@ -0,0 +1,520 @@
setup:
- do:
headers:
Authorization: "Basic eF9wYWNrX3Jlc3RfdXNlcjp4LXBhY2stdGVzdC1wYXNzd29yZA==" # run as x_pack_rest_user, i.e. the test setup superuser
xpack.ml.put_job:
job_id: jobs-get-result-overall-buckets-60
body: >
{
"groups": [ "jobs-get-result-overall-buckets-group"],
"analysis_config" : {
"bucket_span": "60m",
"detectors" :[{"function":"metric","field_name":"responsetime","by_field_name":"airline"}]
},
"data_description" : {
"time_field":"time"
}
}
- do:
headers:
Authorization: "Basic eF9wYWNrX3Jlc3RfdXNlcjp4LXBhY2stdGVzdC1wYXNzd29yZA==" # run as x_pack_rest_user, i.e. the test setup superuser
xpack.ml.put_job:
job_id: jobs-get-result-overall-buckets-30
body: >
{
"groups": [ "jobs-get-result-overall-buckets-group"],
"analysis_config" : {
"bucket_span": "30m",
"detectors" :[{"function":"metric","field_name":"responsetime","by_field_name":"airline"}]
},
"data_description" : {
"time_field":"time"
}
}
- do:
headers:
Authorization: "Basic eF9wYWNrX3Jlc3RfdXNlcjp4LXBhY2stdGVzdC1wYXNzd29yZA==" # run as x_pack_rest_user, i.e. the test setup superuser
xpack.ml.put_job:
job_id: jobs-get-result-overall-buckets-17
body: >
{
"groups": [ "jobs-get-result-overall-buckets-group"],
"analysis_config" : {
"bucket_span": "17m",
"detectors" :[{"function":"metric","field_name":"responsetime","by_field_name":"airline"}]
},
"data_description" : {
"time_field":"time"
}
}
# Now index some buckets
# The buckets are:
# job-60: [ 30.0] [0.0] [ 20.0 ]
# job-30: [ ] [0.0] [ 10.0, 40.0 ]
# job-17: [ ] [0.0] [ 1.0, 0.0, 60.0 ]
- do:
headers:
Authorization: "Basic eF9wYWNrX3Jlc3RfdXNlcjp4LXBhY2stdGVzdC1wYXNzd29yZA==" # run as x_pack_rest_user, i.e. the test setup superuser
index:
index: .ml-anomalies-shared
type: doc
id: "jobs-get-result-overall-buckets-60_1"
body:
{
"job_id": "jobs-get-result-overall-buckets-60",
"result_type": "bucket",
"timestamp": "2016-06-01T00:00:00Z",
"anomaly_score": 30.0,
"bucket_span": 3600
}
- do:
headers:
Authorization: "Basic eF9wYWNrX3Jlc3RfdXNlcjp4LXBhY2stdGVzdC1wYXNzd29yZA==" # run as x_pack_rest_user, i.e. the test setup superuser
index:
index: .ml-anomalies-shared
type: doc
id: "jobs-get-result-overall-buckets-60_2"
body:
{
"job_id": "jobs-get-result-overall-buckets-60",
"result_type": "bucket",
"timestamp": "2016-06-01T01:00:00Z",
"anomaly_score": 0.0,
"bucket_span": 3600
}
- do:
headers:
Authorization: "Basic eF9wYWNrX3Jlc3RfdXNlcjp4LXBhY2stdGVzdC1wYXNzd29yZA==" # run as x_pack_rest_user, i.e. the test setup superuser
index:
index: .ml-anomalies-shared
type: doc
id: "jobs-get-result-overall-buckets-60_3"
body:
{
"job_id": "jobs-get-result-overall-buckets-60",
"result_type": "bucket",
"timestamp": "2016-06-01T02:00:00Z",
"anomaly_score": 20.0,
"bucket_span": 3600
}
- do:
headers:
Authorization: "Basic eF9wYWNrX3Jlc3RfdXNlcjp4LXBhY2stdGVzdC1wYXNzd29yZA==" # run as x_pack_rest_user, i.e. the test setup superuser
index:
index: .ml-anomalies-shared
type: doc
id: "jobs-get-result-overall-buckets-30_1"
body:
{
"job_id": "jobs-get-result-overall-buckets-30",
"result_type": "bucket",
"timestamp": "2016-06-01T01:00:00Z",
"anomaly_score": 0.0,
"bucket_span": 1800
}
- do:
headers:
Authorization: "Basic eF9wYWNrX3Jlc3RfdXNlcjp4LXBhY2stdGVzdC1wYXNzd29yZA==" # run as x_pack_rest_user, i.e. the test setup superuser
index:
index: .ml-anomalies-shared
type: doc
id: "jobs-get-result-overall-buckets-30_2"
body:
{
"job_id": "jobs-get-result-overall-buckets-30",
"result_type": "bucket",
"timestamp": "2016-06-01T02:00:00Z",
"anomaly_score": 10.0,
"bucket_span": 1800
}
- do:
headers:
Authorization: "Basic eF9wYWNrX3Jlc3RfdXNlcjp4LXBhY2stdGVzdC1wYXNzd29yZA==" # run as x_pack_rest_user, i.e. the test setup superuser
index:
index: .ml-anomalies-shared
type: doc
id: "jobs-get-result-overall-buckets-30_3"
body:
{
"job_id": "jobs-get-result-overall-buckets-30",
"result_type": "bucket",
"timestamp": "2016-06-01T02:30:00Z",
"anomaly_score": 40.0,
"bucket_span": 1800
}
- do:
headers:
Authorization: "Basic eF9wYWNrX3Jlc3RfdXNlcjp4LXBhY2stdGVzdC1wYXNzd29yZA==" # run as x_pack_rest_user, i.e. the test setup superuser
index:
index: .ml-anomalies-shared
type: doc
id: "jobs-get-result-overall-buckets-17_1"
body:
{
"job_id": "jobs-get-result-overall-buckets-17",
"result_type": "bucket",
"timestamp": "2016-06-01T01:00:00Z",
"anomaly_score": 0.0,
"bucket_span": 1020
}
- do:
headers:
Authorization: "Basic eF9wYWNrX3Jlc3RfdXNlcjp4LXBhY2stdGVzdC1wYXNzd29yZA==" # run as x_pack_rest_user, i.e. the test setup superuser
index:
index: .ml-anomalies-shared
type: doc
id: "jobs-get-result-overall-buckets-17_2"
body:
{
"job_id": "jobs-get-result-overall-buckets-17",
"result_type": "bucket",
"timestamp": "2016-06-01T02:08:00Z",
"anomaly_score": 1.0,
"bucket_span": 1020
}
- do:
headers:
Authorization: "Basic eF9wYWNrX3Jlc3RfdXNlcjp4LXBhY2stdGVzdC1wYXNzd29yZA==" # run as x_pack_rest_user, i.e. the test setup superuser
index:
index: .ml-anomalies-shared
type: doc
id: "jobs-get-result-overall-buckets-17_3"
body:
{
"job_id": "jobs-get-result-overall-buckets-17",
"result_type": "bucket",
"timestamp": "2016-06-01T02:25:00Z",
"anomaly_score": 0.0,
"bucket_span": 1020
}
- do:
headers:
Authorization: "Basic eF9wYWNrX3Jlc3RfdXNlcjp4LXBhY2stdGVzdC1wYXNzd29yZA==" # run as x_pack_rest_user, i.e. the test setup superuser
index:
index: .ml-anomalies-shared
type: doc
id: "jobs-get-result-overall-buckets-17_4"
body:
{
"job_id": "jobs-get-result-overall-buckets-17",
"result_type": "bucket",
"timestamp": "2016-06-01T02:42:00Z",
"anomaly_score": 60.0,
"bucket_span": 1020,
"is_interim": true
}
- do:
headers:
Authorization: "Basic eF9wYWNrX3Jlc3RfdXNlcjp4LXBhY2stdGVzdC1wYXNzd29yZA==" # run as x_pack_rest_user, i.e. the test setup superuser
indices.refresh:
index: .ml-anomalies-shared
---
"Test overall buckets given missing job":
- do:
catch: missing
xpack.ml.get_overall_buckets:
job_id: "missing-job"
---
"Test overall buckets given non-matching expression and allow_no_jobs":
- do:
xpack.ml.get_overall_buckets:
job_id: "none-matching-*"
- match: { count: 0 }
---
"Test overall buckets given non-matching expression and not allow_no_jobs":
- do:
catch: missing
xpack.ml.get_overall_buckets:
job_id: "none-matching-*"
allow_no_jobs: false
---
"Test overall buckets given top_n is 0":
- do:
catch: /\[topN\] parameter must be positive, found \[0\]/
xpack.ml.get_overall_buckets:
job_id: "jobs-get-result-overall-buckets-*"
top_n: 0
---
"Test overall buckets given top_n is negative":
- do:
catch: /\[topN\] parameter must be positive, found \[-1\]/
xpack.ml.get_overall_buckets:
job_id: "jobs-get-result-overall-buckets-*"
top_n: -1
---
"Test overall buckets given default":
- do:
xpack.ml.get_overall_buckets:
job_id: "jobs-get-result-overall-buckets-*"
- match: { count: 3 }
- match: { overall_buckets.0.timestamp: 1464739200000 }
- match: { overall_buckets.0.bucket_span: 3600 }
- match: { overall_buckets.0.overall_score: 30.0 }
- length: { overall_buckets.0.jobs: 1}
- match: {overall_buckets.0.jobs.0.job_id: jobs-get-result-overall-buckets-60 }
- match: {overall_buckets.0.jobs.0.max_anomaly_score: 30.0 }
- match: { overall_buckets.0.is_interim: false }
- match: { overall_buckets.0.result_type: overall_bucket }
- match: { overall_buckets.1.timestamp: 1464742800000 }
- match: { overall_buckets.1.bucket_span: 3600 }
- match: { overall_buckets.1.overall_score: 0.0 }
- length: { overall_buckets.1.jobs: 3}
- match: {overall_buckets.1.jobs.0.job_id: jobs-get-result-overall-buckets-17 }
- match: {overall_buckets.1.jobs.0.max_anomaly_score: 0.0 }
- match: {overall_buckets.1.jobs.1.job_id: jobs-get-result-overall-buckets-30 }
- match: {overall_buckets.1.jobs.1.max_anomaly_score: 0.0 }
- match: {overall_buckets.1.jobs.2.job_id: jobs-get-result-overall-buckets-60 }
- match: {overall_buckets.1.jobs.2.max_anomaly_score: 0.0 }
- match: { overall_buckets.1.is_interim: false }
- match: { overall_buckets.1.result_type: overall_bucket }
- match: { overall_buckets.2.timestamp: 1464746400000 }
- match: { overall_buckets.2.bucket_span: 3600 }
- match: { overall_buckets.2.overall_score: 60.0 }
- length: { overall_buckets.2.jobs: 3}
- match: {overall_buckets.2.jobs.0.job_id: jobs-get-result-overall-buckets-17 }
- match: {overall_buckets.2.jobs.0.max_anomaly_score: 60.0 }
- match: {overall_buckets.2.jobs.1.job_id: jobs-get-result-overall-buckets-30 }
- match: {overall_buckets.2.jobs.1.max_anomaly_score: 40.0 }
- match: {overall_buckets.2.jobs.2.job_id: jobs-get-result-overall-buckets-60 }
- match: {overall_buckets.2.jobs.2.max_anomaly_score: 20.0 }
- match: { overall_buckets.2.is_interim: true }
- match: { overall_buckets.2.result_type: overall_bucket }
---
"Test overall buckets given top_n is 2":
- do:
xpack.ml.get_overall_buckets:
job_id: "jobs-get-result-overall-buckets-group"
top_n: 2
- match: { count: 3 }
- match: { overall_buckets.0.timestamp: 1464739200000 }
- match: { overall_buckets.0.bucket_span: 3600 }
- match: { overall_buckets.0.overall_score: 30.0 }
- length: { overall_buckets.0.jobs: 1}
- match: {overall_buckets.0.jobs.0.job_id: jobs-get-result-overall-buckets-60 }
- match: {overall_buckets.0.jobs.0.max_anomaly_score: 30.0 }
- match: { overall_buckets.0.is_interim: false }
- match: { overall_buckets.0.result_type: overall_bucket }
- match: { overall_buckets.1.timestamp: 1464742800000 }
- match: { overall_buckets.1.bucket_span: 3600 }
- match: { overall_buckets.1.overall_score: 0.0 }
- length: { overall_buckets.1.jobs: 3}
- match: {overall_buckets.1.jobs.0.job_id: jobs-get-result-overall-buckets-17 }
- match: {overall_buckets.1.jobs.0.max_anomaly_score: 0.0 }
- match: {overall_buckets.1.jobs.1.job_id: jobs-get-result-overall-buckets-30 }
- match: {overall_buckets.1.jobs.1.max_anomaly_score: 0.0 }
- match: {overall_buckets.1.jobs.2.job_id: jobs-get-result-overall-buckets-60 }
- match: {overall_buckets.1.jobs.2.max_anomaly_score: 0.0 }
- match: { overall_buckets.1.is_interim: false }
- match: { overall_buckets.1.result_type: overall_bucket }
- match: { overall_buckets.2.timestamp: 1464746400000 }
- match: { overall_buckets.2.bucket_span: 3600 }
- match: { overall_buckets.2.overall_score: 50.0 }
- length: { overall_buckets.2.jobs: 3}
- match: {overall_buckets.2.jobs.0.job_id: jobs-get-result-overall-buckets-17 }
- match: {overall_buckets.2.jobs.0.max_anomaly_score: 60.0 }
- match: {overall_buckets.2.jobs.1.job_id: jobs-get-result-overall-buckets-30 }
- match: {overall_buckets.2.jobs.1.max_anomaly_score: 40.0 }
- match: {overall_buckets.2.jobs.2.job_id: jobs-get-result-overall-buckets-60 }
- match: {overall_buckets.2.jobs.2.max_anomaly_score: 20.0 }
- match: { overall_buckets.2.is_interim: true }
- match: { overall_buckets.2.result_type: overall_bucket }
---
"Test overall buckets given top_n is 3":
- do:
xpack.ml.get_overall_buckets:
job_id: "jobs-get-result-overall-buckets-group"
top_n: 3
- match: { count: 3 }
- match: { overall_buckets.0.timestamp: 1464739200000 }
- match: { overall_buckets.0.bucket_span: 3600 }
- match: { overall_buckets.0.overall_score: 30.0 }
- length: { overall_buckets.0.jobs: 1}
- match: {overall_buckets.0.jobs.0.job_id: jobs-get-result-overall-buckets-60 }
- match: {overall_buckets.0.jobs.0.max_anomaly_score: 30.0 }
- match: { overall_buckets.0.is_interim: false }
- match: { overall_buckets.0.result_type: overall_bucket }
- match: { overall_buckets.1.timestamp: 1464742800000 }
- match: { overall_buckets.1.bucket_span: 3600 }
- match: { overall_buckets.1.overall_score: 0.0 }
- length: { overall_buckets.1.jobs: 3}
- match: {overall_buckets.1.jobs.0.job_id: jobs-get-result-overall-buckets-17 }
- match: {overall_buckets.1.jobs.0.max_anomaly_score: 0.0 }
- match: {overall_buckets.1.jobs.1.job_id: jobs-get-result-overall-buckets-30 }
- match: {overall_buckets.1.jobs.1.max_anomaly_score: 0.0 }
- match: {overall_buckets.1.jobs.2.job_id: jobs-get-result-overall-buckets-60 }
- match: {overall_buckets.1.jobs.2.max_anomaly_score: 0.0 }
- match: { overall_buckets.1.is_interim: false }
- match: { overall_buckets.1.result_type: overall_bucket }
- match: { overall_buckets.2.timestamp: 1464746400000 }
- match: { overall_buckets.2.bucket_span: 3600 }
- match: { overall_buckets.2.overall_score: 40.0 }
- length: { overall_buckets.2.jobs: 3}
- match: {overall_buckets.2.jobs.0.job_id: jobs-get-result-overall-buckets-17 }
- match: {overall_buckets.2.jobs.0.max_anomaly_score: 60.0 }
- match: {overall_buckets.2.jobs.1.job_id: jobs-get-result-overall-buckets-30 }
- match: {overall_buckets.2.jobs.1.max_anomaly_score: 40.0 }
- match: {overall_buckets.2.jobs.2.job_id: jobs-get-result-overall-buckets-60 }
- match: {overall_buckets.2.jobs.2.max_anomaly_score: 20.0 }
- match: { overall_buckets.2.is_interim: true }
- match: { overall_buckets.2.result_type: overall_bucket }
---
"Test overall buckets given top_n is greater than the job count":
- do:
xpack.ml.get_overall_buckets:
job_id: "jobs-get-result-overall-buckets-*"
top_n: 333
- match: { count: 3 }
- match: { overall_buckets.0.timestamp: 1464739200000 }
- match: { overall_buckets.0.bucket_span: 3600 }
- match: { overall_buckets.0.overall_score: 30.0 }
- length: { overall_buckets.0.jobs: 1}
- match: {overall_buckets.0.jobs.0.job_id: jobs-get-result-overall-buckets-60 }
- match: {overall_buckets.0.jobs.0.max_anomaly_score: 30.0 }
- match: { overall_buckets.0.is_interim: false }
- match: { overall_buckets.0.result_type: overall_bucket }
- match: { overall_buckets.1.timestamp: 1464742800000 }
- match: { overall_buckets.1.bucket_span: 3600 }
- match: { overall_buckets.1.overall_score: 0.0 }
- length: { overall_buckets.1.jobs: 3}
- match: {overall_buckets.1.jobs.0.job_id: jobs-get-result-overall-buckets-17 }
- match: {overall_buckets.1.jobs.0.max_anomaly_score: 0.0 }
- match: {overall_buckets.1.jobs.1.job_id: jobs-get-result-overall-buckets-30 }
- match: {overall_buckets.1.jobs.1.max_anomaly_score: 0.0 }
- match: {overall_buckets.1.jobs.2.job_id: jobs-get-result-overall-buckets-60 }
- match: {overall_buckets.1.jobs.2.max_anomaly_score: 0.0 }
- match: { overall_buckets.1.is_interim: false }
- match: { overall_buckets.1.result_type: overall_bucket }
- match: { overall_buckets.2.timestamp: 1464746400000 }
- match: { overall_buckets.2.bucket_span: 3600 }
- match: { overall_buckets.2.overall_score: 40.0 }
- length: { overall_buckets.2.jobs: 3}
- match: {overall_buckets.2.jobs.0.job_id: jobs-get-result-overall-buckets-17 }
- match: {overall_buckets.2.jobs.0.max_anomaly_score: 60.0 }
- match: {overall_buckets.2.jobs.1.job_id: jobs-get-result-overall-buckets-30 }
- match: {overall_buckets.2.jobs.1.max_anomaly_score: 40.0 }
- match: {overall_buckets.2.jobs.2.job_id: jobs-get-result-overall-buckets-60 }
- match: {overall_buckets.2.jobs.2.max_anomaly_score: 20.0 }
- match: { overall_buckets.2.is_interim: true }
- match: { overall_buckets.2.result_type: overall_bucket }
---
"Test overall buckets given overall_score filter":
- do:
xpack.ml.get_overall_buckets:
job_id: "jobs-get-result-overall-buckets-*"
top_n: 2
overall_score: 50.0
- match: { count: 1 }
- match: { overall_buckets.0.timestamp: 1464746400000 }
- match: { overall_buckets.0.overall_score: 50.0 }
---
"Test overall buckets given exclude_interim":
- do:
xpack.ml.get_overall_buckets:
job_id: "jobs-get-result-overall-buckets-*"
exclude_interim: true
- match: { count: 3 }
- match: { overall_buckets.0.timestamp: 1464739200000 }
- match: { overall_buckets.0.bucket_span: 3600 }
- match: { overall_buckets.0.overall_score: 30.0 }
- length: { overall_buckets.0.jobs: 1}
- match: {overall_buckets.0.jobs.0.job_id: jobs-get-result-overall-buckets-60 }
- match: {overall_buckets.0.jobs.0.max_anomaly_score: 30.0 }
- match: { overall_buckets.0.is_interim: false }
- match: { overall_buckets.0.result_type: overall_bucket }
- match: { overall_buckets.1.timestamp: 1464742800000 }
- match: { overall_buckets.1.bucket_span: 3600 }
- match: { overall_buckets.1.overall_score: 0.0 }
- length: { overall_buckets.1.jobs: 3}
- match: {overall_buckets.1.jobs.0.job_id: jobs-get-result-overall-buckets-17 }
- match: {overall_buckets.1.jobs.0.max_anomaly_score: 0.0 }
- match: {overall_buckets.1.jobs.1.job_id: jobs-get-result-overall-buckets-30 }
- match: {overall_buckets.1.jobs.1.max_anomaly_score: 0.0 }
- match: {overall_buckets.1.jobs.2.job_id: jobs-get-result-overall-buckets-60 }
- match: {overall_buckets.1.jobs.2.max_anomaly_score: 0.0 }
- match: { overall_buckets.1.is_interim: false }
- match: { overall_buckets.1.result_type: overall_bucket }
- match: { overall_buckets.2.timestamp: 1464746400000 }
- match: { overall_buckets.2.bucket_span: 3600 }
- match: { overall_buckets.2.overall_score: 40.0 }
- length: { overall_buckets.2.jobs: 3}
- match: {overall_buckets.2.jobs.0.job_id: jobs-get-result-overall-buckets-17 }
- match: {overall_buckets.2.jobs.0.max_anomaly_score: 1.0 }
- match: {overall_buckets.2.jobs.1.job_id: jobs-get-result-overall-buckets-30 }
- match: {overall_buckets.2.jobs.1.max_anomaly_score: 40.0 }
- match: {overall_buckets.2.jobs.2.job_id: jobs-get-result-overall-buckets-60 }
- match: {overall_buckets.2.jobs.2.max_anomaly_score: 20.0 }
- match: { overall_buckets.2.is_interim: false }
- match: { overall_buckets.2.result_type: overall_bucket }
---
"Test overall buckets given string start and end params":
- do:
xpack.ml.get_overall_buckets:
job_id: "jobs-get-result-overall-buckets-*"
top_n: 2
start: "2016-06-01T00:30:00Z"
end: "2016-06-01T02:30:00Z"
- match: { count: 1 }
- match: { overall_buckets.0.timestamp: 1464742800000 }
- match: { overall_buckets.0.overall_score: 0.0 }
- length: { overall_buckets.0.jobs: 3}
---
"Test overall buckets given epoch start and end params":
- do:
xpack.ml.get_overall_buckets:
job_id: "jobs-get-result-overall-buckets-*"
top_n: 2
start: 1464744600000
end: 1464751800000
- match: { count: 1 }
- match: { overall_buckets.0.timestamp: 1464746400000 }
- match: { overall_buckets.0.overall_score: 50.0 }
- length: { overall_buckets.0.jobs: 3}
- match: {overall_buckets.0.jobs.0.job_id: jobs-get-result-overall-buckets-17 }
- match: {overall_buckets.0.jobs.0.max_anomaly_score: 60.0 }
- match: {overall_buckets.0.jobs.1.job_id: jobs-get-result-overall-buckets-30 }
- match: {overall_buckets.0.jobs.1.max_anomaly_score: 40.0 }
- match: {overall_buckets.0.jobs.2.job_id: jobs-get-result-overall-buckets-60 }
- match: {overall_buckets.0.jobs.2.max_anomaly_score: 20.0 }
---
"Test overall buckets given invalid start param":
- do:
catch: /.*Query param \[start\] with value \[invalid\] cannot be parsed as a date or converted to a number.*/
xpack.ml.get_overall_buckets:
job_id: "jobs-get-result-overall-buckets-*"
start: "invalid"
---
"Test overall buckets given invalid end param":
- do:
catch: /.*Query param \[end\] with value \[invalid\] cannot be parsed as a date or converted to a number.*/
xpack.ml.get_overall_buckets:
job_id: "jobs-get-result-overall-buckets-*"
end: "invalid"

View File

@ -46,6 +46,12 @@ integTestRunner {
'ml/jobs_get_result_buckets/Test mutually-exclusive params via body',
'ml/jobs_get_result_categories/Test with invalid param combinations',
'ml/jobs_get_result_categories/Test with invalid param combinations via body',
'ml/jobs_get_result_overall_buckets/Test overall buckets given missing job',
'ml/jobs_get_result_overall_buckets/Test overall buckets given non-matching expression and not allow_no_jobs',
'ml/jobs_get_result_overall_buckets/Test overall buckets given top_n is 0',
'ml/jobs_get_result_overall_buckets/Test overall buckets given top_n is negative',
'ml/jobs_get_result_overall_buckets/Test overall buckets given invalid start param',
'ml/jobs_get_result_overall_buckets/Test overall buckets given invalid end param',
'ml/jobs_get_stats/Test get job stats given missing job',
'ml/jobs_get_stats/Test no exception on get job stats with missing index',
'ml/job_groups/Test put job with empty group',