Parameterize the result field name in QueryPage (elastic/elasticsearch#364)
QueryPage now requires a ParseField in the constructor, which is used for the name of the array of results. I considered putting an enum of different field names in QueryPage itself, but it seemed better to keep the field name local to each respective object. hitCount was also renamed to count, and getters updated appropriately. Finally, added a static helper to throw the ResourceNotFoundException, just to make life easier/more consistent. Closes elastic/elasticsearch#359 Original commit: elastic/x-pack-elasticsearch@9ba42ad4a1
This commit is contained in:
parent
9fc3c77905
commit
17b3224e03
|
@ -155,7 +155,7 @@ public class DeleteModelSnapshotAction extends Action<DeleteModelSnapshotAction.
|
|||
List<ModelSnapshot> deleteCandidates;
|
||||
deleteCandidates = jobProvider.modelSnapshots(
|
||||
request.getJobId(), 0, 1, null, null, null, true, request.getSnapshotId(), null
|
||||
).hits();
|
||||
).results();
|
||||
|
||||
if (deleteCandidates.size() > 1) {
|
||||
logger.warn("More than one model found for [jobId: " + request.getJobId()
|
||||
|
@ -172,8 +172,8 @@ public class DeleteModelSnapshotAction extends Action<DeleteModelSnapshotAction.
|
|||
// NORELEASE: technically, this could be stale and refuse a delete, but I think that's acceptable
|
||||
// since it is non-destructive
|
||||
QueryPage<Job> job = jobManager.getJob(request.getJobId(), clusterService.state());
|
||||
if (job.hitCount() > 0) {
|
||||
String currentModelInUse = job.hits().get(0).getModelSnapshotId();
|
||||
if (job.count() > 0) {
|
||||
String currentModelInUse = job.results().get(0).getModelSnapshotId();
|
||||
if (currentModelInUse != null && currentModelInUse.equals(request.getSnapshotId())) {
|
||||
throw new IllegalArgumentException(Messages.getMessage(Messages.REST_CANNOT_DELETE_HIGHEST_PRIORITY,
|
||||
request.getSnapshotId(), request.getJobId()));
|
||||
|
|
|
@ -332,7 +332,7 @@ public class GetBucketsAction extends Action<GetBucketsAction.Request, GetBucket
|
|||
|
||||
@Override
|
||||
public RestStatus status() {
|
||||
return buckets.hitCount() == 0 ? RestStatus.NOT_FOUND : RestStatus.OK;
|
||||
return buckets.count() == 0 ? RestStatus.NOT_FOUND : RestStatus.OK;
|
||||
}
|
||||
|
||||
@Override
|
||||
|
|
|
@ -356,7 +356,7 @@ public class GetJobsAction extends Action<GetJobsAction.Request, GetJobsAction.R
|
|||
|
||||
@Override
|
||||
public RestStatus status() {
|
||||
return jobs.hitCount() == 0 ? RestStatus.NOT_FOUND : RestStatus.OK;
|
||||
return jobs.count() == 0 ? RestStatus.NOT_FOUND : RestStatus.OK;
|
||||
}
|
||||
|
||||
@Override
|
||||
|
@ -438,30 +438,28 @@ public class GetJobsAction extends Action<GetJobsAction.Request, GetJobsAction.R
|
|||
// always get the job regardless of the request.config param because if the job
|
||||
// can't be found a different response is returned.
|
||||
QueryPage<Job> jobs = jobManager.getJob(request.getJobId(), state);
|
||||
if (jobs.hitCount() == 0) {
|
||||
if (jobs.count() == 0) {
|
||||
logger.debug(String.format(Locale.ROOT, "Cannot find job '%s'", request.getJobId()));
|
||||
response = new QueryPage<>(Collections.emptyList(), 0);
|
||||
listener.onResponse(new Response(response));
|
||||
return;
|
||||
} else if (jobs.hitCount() > 1) {
|
||||
throw QueryPage.emptyQueryPage(Job.RESULTS_FIELD);
|
||||
} else if (jobs.count() > 1) {
|
||||
logger.error(String.format(Locale.ROOT, "More than one job found for jobId [%s]", request.getJobId()));
|
||||
}
|
||||
|
||||
logger.debug("Returning job [" + request.getJobId() + "]");
|
||||
Job jobConfig = request.config() ? jobs.hits().get(0) : null;
|
||||
Job jobConfig = request.config() ? jobs.results().get(0) : null;
|
||||
DataCounts dataCounts = readDataCounts(request.dataCounts(), request.getJobId());
|
||||
ModelSizeStats modelSizeStats = readModelSizeStats(request.modelSizeStats(), request.getJobId());
|
||||
SchedulerState schedulerStatus = readSchedulerState(request.schedulerStatus(), request.getJobId());
|
||||
JobStatus jobStatus = readJobStatus(request.status(), request.getJobId());
|
||||
|
||||
Response.JobInfo jobInfo = new Response.JobInfo(jobConfig, dataCounts, modelSizeStats, schedulerStatus, jobStatus);
|
||||
response = new QueryPage<>(Collections.singletonList(jobInfo), 1);
|
||||
response = new QueryPage<>(Collections.singletonList(jobInfo), 1, Job.RESULTS_FIELD);
|
||||
|
||||
} else {
|
||||
// Multiple Jobs
|
||||
QueryPage<Job> jobsPage = jobManager.getJobs(request.pageParams.getFrom(), request.pageParams.getSize(), state);
|
||||
List<Response.JobInfo> jobInfoList = new ArrayList<>();
|
||||
for (Job job : jobsPage.hits()) {
|
||||
for (Job job : jobsPage.results()) {
|
||||
Job jobConfig = request.config() ? job : null;
|
||||
DataCounts dataCounts = readDataCounts(request.dataCounts(), job.getJobId());
|
||||
ModelSizeStats modelSizeStats = readModelSizeStats(request.modelSizeStats(), job.getJobId());
|
||||
|
@ -470,7 +468,7 @@ public class GetJobsAction extends Action<GetJobsAction.Request, GetJobsAction.R
|
|||
Response.JobInfo jobInfo = new Response.JobInfo(jobConfig, dataCounts, modelSizeStats, schedulerStatus, jobStatus);
|
||||
jobInfoList.add(jobInfo);
|
||||
}
|
||||
response = new QueryPage<>(jobInfoList, jobsPage.hitCount());
|
||||
response = new QueryPage<>(jobInfoList, jobsPage.count(), Job.RESULTS_FIELD);
|
||||
}
|
||||
|
||||
listener.onResponse(new Response(response));
|
||||
|
|
|
@ -325,7 +325,7 @@ extends Action<GetModelSnapshotsAction.Request, GetModelSnapshotsAction.Response
|
|||
|
||||
QueryPage<ModelSnapshot> page = doGetPage(jobProvider, request);
|
||||
|
||||
logger.debug(String.format(Locale.ROOT, "Return %d model snapshots for job %s", page.hitCount(), request.getJobId()));
|
||||
logger.debug(String.format(Locale.ROOT, "Return %d model snapshots for job %s", page.count(), request.getJobId()));
|
||||
listener.onResponse(new Response(page));
|
||||
}
|
||||
|
||||
|
@ -337,8 +337,8 @@ extends Action<GetModelSnapshotsAction.Request, GetModelSnapshotsAction.Response
|
|||
// The quantiles can be large, and totally dominate the output -
|
||||
// it's
|
||||
// clearer to remove them
|
||||
if (page.hits() != null) {
|
||||
for (ModelSnapshot modelSnapshot : page.hits()) {
|
||||
if (page.results() != null) {
|
||||
for (ModelSnapshot modelSnapshot : page.results()) {
|
||||
modelSnapshot.setQuantiles(null);
|
||||
}
|
||||
}
|
||||
|
|
|
@ -294,7 +294,7 @@ PutModelSnapshotDescriptionAction.RequestBuilder> {
|
|||
}
|
||||
|
||||
private List<ModelSnapshot> getModelSnapshots(String jobId, String snapshotId, String description) {
|
||||
return jobProvider.modelSnapshots(jobId, 0, 1, null, null, null, true, snapshotId, description).hits();
|
||||
return jobProvider.modelSnapshots(jobId, 0, 1, null, null, null, true, snapshotId, description).results();
|
||||
}
|
||||
|
||||
}
|
||||
|
|
|
@ -340,7 +340,7 @@ extends Action<RevertModelSnapshotAction.Request, RevertModelSnapshotAction.Resp
|
|||
|
||||
QueryPage<Job> job = jobManager.getJob(request.getJobId(), clusterService.state());
|
||||
Allocation allocation = jobManager.getJobAllocation(request.getJobId());
|
||||
if (job.hitCount() > 0 && allocation.getStatus().equals(JobStatus.RUNNING)) {
|
||||
if (job.count() > 0 && allocation.getStatus().equals(JobStatus.RUNNING)) {
|
||||
throw ExceptionsHelper.conflictStatusException(Messages.getMessage(Messages.REST_JOB_NOT_CLOSED_REVERT));
|
||||
}
|
||||
|
||||
|
@ -356,7 +356,7 @@ extends Action<RevertModelSnapshotAction.Request, RevertModelSnapshotAction.Resp
|
|||
|
||||
List<ModelSnapshot> revertCandidates;
|
||||
revertCandidates = provider.modelSnapshots(request.getJobId(), 0, 1, null, request.getTime(),
|
||||
ModelSnapshot.TIMESTAMP.getPreferredName(), true, request.getSnapshotId(), request.getDescription()).hits();
|
||||
ModelSnapshot.TIMESTAMP.getPreferredName(), true, request.getSnapshotId(), request.getDescription()).results();
|
||||
|
||||
if (revertCandidates == null || revertCandidates.isEmpty()) {
|
||||
throw new ResourceNotFoundException(Messages.getMessage(Messages.REST_NO_SUCH_MODEL_SNAPSHOT, request.getJobId()));
|
||||
|
|
|
@ -76,6 +76,9 @@ public class Job extends AbstractDiffable<Job> implements Writeable, ToXContent
|
|||
public static final ParseField TRANSFORMS = new ParseField("transforms");
|
||||
public static final ParseField MODEL_SNAPSHOT_ID = new ParseField("modelSnapshotId");
|
||||
|
||||
// Used for QueryPage
|
||||
public static final ParseField RESULTS_FIELD = new ParseField("jobs");
|
||||
|
||||
public static final ObjectParser<Builder, ParseFieldMatcherSupplier> PARSER = new ObjectParser<>("job_details", Builder::new);
|
||||
|
||||
static {
|
||||
|
|
|
@ -39,6 +39,9 @@ public class ModelSnapshot extends ToXContentToBytes implements Writeable {
|
|||
public static final ParseField LATEST_RECORD_TIME = new ParseField("latestRecordTimeStamp");
|
||||
public static final ParseField LATEST_RESULT_TIME = new ParseField("latestResultTimeStamp");
|
||||
|
||||
// Used for QueryPage
|
||||
public static final ParseField RESULTS_FIELD = new ParseField("modelSnapshots");
|
||||
|
||||
/**
|
||||
* Elasticsearch type
|
||||
*/
|
||||
|
|
|
@ -13,6 +13,7 @@ import org.elasticsearch.cluster.ClusterState;
|
|||
import org.elasticsearch.cluster.ClusterStateUpdateTask;
|
||||
import org.elasticsearch.cluster.metadata.MetaData;
|
||||
import org.elasticsearch.cluster.service.ClusterService;
|
||||
import org.elasticsearch.common.ParseField;
|
||||
import org.elasticsearch.common.logging.Loggers;
|
||||
import org.elasticsearch.common.settings.Settings;
|
||||
import org.elasticsearch.env.Environment;
|
||||
|
@ -98,10 +99,10 @@ public class JobManager {
|
|||
PrelertMetadata prelertMetadata = clusterState.getMetaData().custom(PrelertMetadata.TYPE);
|
||||
Job job = prelertMetadata.getJobs().get(jobId);
|
||||
if (job == null) {
|
||||
return new QueryPage<>(Collections.emptyList(), 0);
|
||||
throw QueryPage.emptyQueryPage(Job.RESULTS_FIELD);
|
||||
}
|
||||
|
||||
return new QueryPage<>(Collections.singletonList(job), 1);
|
||||
return new QueryPage<>(Collections.singletonList(job), 1, Job.RESULTS_FIELD);
|
||||
}
|
||||
|
||||
/**
|
||||
|
@ -123,7 +124,7 @@ public class JobManager {
|
|||
.limit(size)
|
||||
.map(Map.Entry::getValue)
|
||||
.collect(Collectors.toList());
|
||||
return new QueryPage<>(jobs, prelertMetadata.getJobs().size());
|
||||
return new QueryPage<>(jobs, prelertMetadata.getJobs().size(), Job.RESULTS_FIELD);
|
||||
}
|
||||
|
||||
/**
|
||||
|
|
|
@ -333,7 +333,7 @@ public class ElasticsearchJobProvider implements JobProvider
|
|||
QueryPage<Bucket> buckets = buckets(jobId, query.isIncludeInterim(), query.getFrom(), query.getSize(), fb, sortBuilder);
|
||||
|
||||
if (Strings.isNullOrEmpty(query.getPartitionValue())) {
|
||||
for (Bucket b : buckets.hits()) {
|
||||
for (Bucket b : buckets.results()) {
|
||||
if (query.isExpand() && b.getRecordCount() > 0) {
|
||||
expandBucket(jobId, query.isIncludeInterim(), b);
|
||||
}
|
||||
|
@ -342,9 +342,9 @@ public class ElasticsearchJobProvider implements JobProvider
|
|||
List<ScoreTimestamp> scores =
|
||||
partitionScores(jobId, query.getEpochStart(), query.getEpochEnd(), query.getPartitionValue());
|
||||
|
||||
mergePartitionScoresIntoBucket(scores, buckets.hits());
|
||||
mergePartitionScoresIntoBucket(scores, buckets.results());
|
||||
|
||||
for (Bucket b : buckets.hits()) {
|
||||
for (Bucket b : buckets.results()) {
|
||||
if (query.isExpand() && b.getRecordCount() > 0) {
|
||||
this.expandBucketForPartitionValue(jobId, query.isIncludeInterim(), b, query.getPartitionValue());
|
||||
}
|
||||
|
@ -410,7 +410,7 @@ public class ElasticsearchJobProvider implements JobProvider
|
|||
}
|
||||
}
|
||||
|
||||
return new QueryPage<>(results, searchResponse.getHits().getTotalHits());
|
||||
return new QueryPage<>(results, searchResponse.getHits().getTotalHits(), Bucket.RESULTS_FIELD);
|
||||
}
|
||||
|
||||
|
||||
|
@ -435,7 +435,7 @@ public class ElasticsearchJobProvider implements JobProvider
|
|||
}
|
||||
|
||||
if (hits.getTotalHits() == 0) {
|
||||
return new QueryPage<>(Collections.emptyList(), 0);
|
||||
throw QueryPage.emptyQueryPage(Bucket.RESULTS_FIELD);
|
||||
} else if (hits.getTotalHits() > 1L) {
|
||||
LOGGER.error("Found more than one bucket with timestamp [" + query.getTimestamp() + "]" +
|
||||
" from index " + indexName);
|
||||
|
@ -454,7 +454,7 @@ public class ElasticsearchJobProvider implements JobProvider
|
|||
|
||||
// don't return interim buckets if not requested
|
||||
if (bucket.isInterim() && query.isIncludeInterim() == false) {
|
||||
return new QueryPage<>(Collections.emptyList(), 0);
|
||||
throw QueryPage.emptyQueryPage(Bucket.RESULTS_FIELD);
|
||||
}
|
||||
|
||||
if (Strings.isNullOrEmpty(query.getPartitionValue())) {
|
||||
|
@ -477,7 +477,7 @@ public class ElasticsearchJobProvider implements JobProvider
|
|||
bucket.setAnomalyScore(
|
||||
bucket.partitionAnomalyScore(query.getPartitionValue()));
|
||||
}
|
||||
return new QueryPage<>(Collections.singletonList(bucket), 1);
|
||||
return new QueryPage<>(Collections.singletonList(bucket), 1, Bucket.RESULTS_FIELD);
|
||||
}
|
||||
|
||||
final class ScoreTimestamp
|
||||
|
@ -549,15 +549,15 @@ public class ElasticsearchJobProvider implements JobProvider
|
|||
QueryPage<AnomalyRecord> page = bucketRecords(
|
||||
jobId, bucket, from, RECORDS_SIZE_PARAM, includeInterim,
|
||||
AnomalyRecord.PROBABILITY.getPreferredName(), false, partitionFieldValue);
|
||||
bucket.setRecords(page.hits());
|
||||
bucket.setRecords(page.results());
|
||||
|
||||
while (page.hitCount() > from + RECORDS_SIZE_PARAM)
|
||||
while (page.count() > from + RECORDS_SIZE_PARAM)
|
||||
{
|
||||
from += RECORDS_SIZE_PARAM;
|
||||
page = bucketRecords(
|
||||
jobId, bucket, from, RECORDS_SIZE_PARAM, includeInterim,
|
||||
AnomalyRecord.PROBABILITY.getPreferredName(), false, partitionFieldValue);
|
||||
bucket.getRecords().addAll(page.hits());
|
||||
bucket.getRecords().addAll(page.results());
|
||||
}
|
||||
|
||||
return bucket.getRecords().size();
|
||||
|
@ -577,15 +577,15 @@ public class ElasticsearchJobProvider implements JobProvider
|
|||
QueryPage<AnomalyRecord> page = bucketRecords(
|
||||
jobId, bucket, from, RECORDS_SIZE_PARAM, includeInterim,
|
||||
AnomalyRecord.PROBABILITY.getPreferredName(), false, null);
|
||||
bucket.setRecords(page.hits());
|
||||
bucket.setRecords(page.results());
|
||||
|
||||
while (page.hitCount() > from + RECORDS_SIZE_PARAM)
|
||||
while (page.count() > from + RECORDS_SIZE_PARAM)
|
||||
{
|
||||
from += RECORDS_SIZE_PARAM;
|
||||
page = bucketRecords(
|
||||
jobId, bucket, from, RECORDS_SIZE_PARAM, includeInterim,
|
||||
AnomalyRecord.PROBABILITY.getPreferredName(), false, null);
|
||||
bucket.getRecords().addAll(page.hits());
|
||||
bucket.getRecords().addAll(page.results());
|
||||
}
|
||||
|
||||
return bucket.getRecords().size();
|
||||
|
@ -652,7 +652,7 @@ public class ElasticsearchJobProvider implements JobProvider
|
|||
results.add(categoryDefinition);
|
||||
}
|
||||
|
||||
return new QueryPage<>(results, searchResponse.getHits().getTotalHits());
|
||||
return new QueryPage<>(results, searchResponse.getHits().getTotalHits(), CategoryDefinition.RESULTS_FIELD);
|
||||
}
|
||||
|
||||
|
||||
|
@ -678,9 +678,9 @@ public class ElasticsearchJobProvider implements JobProvider
|
|||
throw new ElasticsearchParseException("failed to parser category definition", e);
|
||||
}
|
||||
CategoryDefinition definition = CategoryDefinition.PARSER.apply(parser, () -> parseFieldMatcher);
|
||||
return new QueryPage<>(Collections.singletonList(definition), 1);
|
||||
return new QueryPage<>(Collections.singletonList(definition), 1, CategoryDefinition.RESULTS_FIELD);
|
||||
}
|
||||
return new QueryPage<>(Collections.emptyList(), 0);
|
||||
throw QueryPage.emptyQueryPage(Bucket.RESULTS_FIELD);
|
||||
}
|
||||
|
||||
@Override
|
||||
|
@ -766,7 +766,7 @@ public class ElasticsearchJobProvider implements JobProvider
|
|||
results.add(record);
|
||||
}
|
||||
|
||||
return new QueryPage<>(results, searchResponse.getHits().getTotalHits());
|
||||
return new QueryPage<>(results, searchResponse.getHits().getTotalHits(), AnomalyRecord.RESULTS_FIELD);
|
||||
}
|
||||
|
||||
@Override
|
||||
|
@ -826,7 +826,7 @@ public class ElasticsearchJobProvider implements JobProvider
|
|||
influencers.add(influencer);
|
||||
}
|
||||
|
||||
return new QueryPage<>(influencers, response.getHits().getTotalHits());
|
||||
return new QueryPage<>(influencers, response.getHits().getTotalHits(), Influencer.RESULTS_FIELD);
|
||||
}
|
||||
|
||||
@Override
|
||||
|
@ -981,7 +981,7 @@ public class ElasticsearchJobProvider implements JobProvider
|
|||
results.add(modelSnapshot);
|
||||
}
|
||||
|
||||
return new QueryPage<>(results, searchResponse.getHits().getTotalHits());
|
||||
return new QueryPage<>(results, searchResponse.getHits().getTotalHits(), ModelSnapshot.RESULTS_FIELD);
|
||||
}
|
||||
|
||||
@Override
|
||||
|
|
|
@ -5,6 +5,7 @@
|
|||
*/
|
||||
package org.elasticsearch.xpack.prelert.job.persistence;
|
||||
|
||||
import org.elasticsearch.ResourceNotFoundException;
|
||||
import org.elasticsearch.action.support.ToXContentToBytes;
|
||||
import org.elasticsearch.common.ParseField;
|
||||
import org.elasticsearch.common.io.stream.StreamInput;
|
||||
|
@ -12,6 +13,7 @@ import org.elasticsearch.common.io.stream.StreamOutput;
|
|||
import org.elasticsearch.common.io.stream.Writeable;
|
||||
import org.elasticsearch.common.xcontent.ToXContent;
|
||||
import org.elasticsearch.common.xcontent.XContentBuilder;
|
||||
import org.elasticsearch.xpack.prelert.utils.ExceptionsHelper;
|
||||
|
||||
import java.io.IOException;
|
||||
import java.util.List;
|
||||
|
@ -19,33 +21,40 @@ import java.util.Objects;
|
|||
|
||||
/**
|
||||
* Generic wrapper class for a page of query results and the total number of
|
||||
* query hits.<br>
|
||||
* {@linkplain #hitCount()} is the total number of results but that value may
|
||||
* not be equal to the actual length of the {@linkplain #hits()} list if from
|
||||
* query results.<br>
|
||||
* {@linkplain #count()} is the total number of results but that value may
|
||||
* not be equal to the actual length of the {@linkplain #results()} list if from
|
||||
* & take or some cursor was used in the database query.
|
||||
*/
|
||||
public final class QueryPage<T extends ToXContent & Writeable> extends ToXContentToBytes implements Writeable {
|
||||
|
||||
public static final ParseField HITS = new ParseField("hits");
|
||||
public static final ParseField HIT_COUNT = new ParseField("hitCount");
|
||||
public static final ParseField COUNT = new ParseField("count");
|
||||
|
||||
private final List<T> hits;
|
||||
private final long hitCount;
|
||||
private final ParseField resultsField;
|
||||
private final List<T> results;
|
||||
private final long count;
|
||||
|
||||
public QueryPage(List<T> hits, long hitCount) {
|
||||
this.hits = hits;
|
||||
this.hitCount = hitCount;
|
||||
public QueryPage(List<T> results, long count, ParseField resultsField) {
|
||||
this.results = results;
|
||||
this.count = count;
|
||||
this.resultsField = ExceptionsHelper.requireNonNull(resultsField, "resultsField");
|
||||
}
|
||||
|
||||
public QueryPage(StreamInput in, Reader<T> hitReader) throws IOException {
|
||||
hits = in.readList(hitReader);
|
||||
hitCount = in.readLong();
|
||||
resultsField = new ParseField(in.readString());
|
||||
results = in.readList(hitReader);
|
||||
count = in.readLong();
|
||||
}
|
||||
|
||||
public static ResourceNotFoundException emptyQueryPage(ParseField resultsField) {
|
||||
return new ResourceNotFoundException("Could not find requested " + resultsField.getPreferredName());
|
||||
}
|
||||
|
||||
@Override
|
||||
public void writeTo(StreamOutput out) throws IOException {
|
||||
out.writeList(hits);
|
||||
out.writeLong(hitCount);
|
||||
out.writeString(resultsField.getPreferredName());
|
||||
out.writeList(results);
|
||||
out.writeLong(count);
|
||||
}
|
||||
|
||||
@Override
|
||||
|
@ -57,22 +66,22 @@ public final class QueryPage<T extends ToXContent & Writeable> extends ToXConten
|
|||
}
|
||||
|
||||
public XContentBuilder doXContentBody(XContentBuilder builder, Params params) throws IOException {
|
||||
builder.field(HITS.getPreferredName(), hits);
|
||||
builder.field(HIT_COUNT.getPreferredName(), hitCount);
|
||||
builder.field(resultsField.getPreferredName(), results);
|
||||
builder.field(COUNT.getPreferredName(), count);
|
||||
return builder;
|
||||
}
|
||||
|
||||
public List<T> hits() {
|
||||
return hits;
|
||||
public List<T> results() {
|
||||
return results;
|
||||
}
|
||||
|
||||
public long hitCount() {
|
||||
return hitCount;
|
||||
public long count() {
|
||||
return count;
|
||||
}
|
||||
|
||||
@Override
|
||||
public int hashCode() {
|
||||
return Objects.hash(hits, hitCount);
|
||||
return Objects.hash(results, count);
|
||||
}
|
||||
|
||||
@Override
|
||||
|
@ -87,7 +96,7 @@ public final class QueryPage<T extends ToXContent & Writeable> extends ToXConten
|
|||
|
||||
@SuppressWarnings("unchecked")
|
||||
QueryPage<T> other = (QueryPage<T>) obj;
|
||||
return Objects.equals(hits, other.hits) &&
|
||||
Objects.equals(hitCount, other.hitCount);
|
||||
return Objects.equals(results, other.results) &&
|
||||
Objects.equals(count, other.count);
|
||||
}
|
||||
}
|
||||
|
|
|
@ -51,7 +51,7 @@ public class NativeAutodetectProcessFactory implements AutodetectProcessFactory
|
|||
@Override
|
||||
public AutodetectProcess createAutodetectProcess(Job job, boolean ignoreDowntime) {
|
||||
List<Path> filesToDelete = new ArrayList<>();
|
||||
List<ModelSnapshot> modelSnapshots = jobProvider.modelSnapshots(job.getId(), 0, 1).hits();
|
||||
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(),
|
||||
|
|
|
@ -55,6 +55,9 @@ public class AnomalyRecord extends ToXContentToBytes implements Writeable {
|
|||
public static final ParseField BUCKET_SPAN = new ParseField("bucketSpan");
|
||||
public static final ParseField TIMESTAMP = new ParseField("timestamp");
|
||||
|
||||
// Used for QueryPage
|
||||
public static final ParseField RESULTS_FIELD = new ParseField("records");
|
||||
|
||||
/**
|
||||
* Metric Results (including population metrics)
|
||||
*/
|
||||
|
|
|
@ -50,6 +50,9 @@ public class Bucket extends ToXContentToBytes implements Writeable {
|
|||
public static final ParseField PROCESSING_TIME_MS = new ParseField("processingTimeMs");
|
||||
public static final ParseField PARTITION_SCORES = new ParseField("partitionScores");
|
||||
|
||||
// Used for QueryPage
|
||||
public static final ParseField RESULTS_FIELD = new ParseField("buckets");
|
||||
|
||||
/**
|
||||
* Elasticsearch type
|
||||
*/
|
||||
|
|
|
@ -32,6 +32,9 @@ public class CategoryDefinition extends ToXContentToBytes implements Writeable {
|
|||
public static final ParseField MAX_MATCHING_LENGTH = new ParseField("maxMatchingLength");
|
||||
public static final ParseField EXAMPLES = new ParseField("examples");
|
||||
|
||||
// Used for QueryPage
|
||||
public static final ParseField RESULTS_FIELD = new ParseField("categories");
|
||||
|
||||
public static final ConstructingObjectParser<CategoryDefinition, ParseFieldMatcherSupplier> PARSER =
|
||||
new ConstructingObjectParser<>(TYPE.getPreferredName(), a -> new CategoryDefinition((String) a[0]));
|
||||
|
||||
|
|
|
@ -38,6 +38,9 @@ public class Influencer extends ToXContentToBytes implements Writeable {
|
|||
public static final ParseField INITIAL_ANOMALY_SCORE = new ParseField("initialAnomalyScore");
|
||||
public static final ParseField ANOMALY_SCORE = new ParseField("anomalyScore");
|
||||
|
||||
// Used for QueryPage
|
||||
public static final ParseField RESULTS_FIELD = new ParseField("influencers");
|
||||
|
||||
public static final ConstructingObjectParser<Influencer, ParseFieldMatcherSupplier> PARSER = new ConstructingObjectParser<>(
|
||||
TYPE.getPreferredName(), a -> new Influencer((String) a[0], (String) a[1], (String) a[2]));
|
||||
|
||||
|
|
|
@ -192,8 +192,8 @@ public class ScheduledJobService extends AbstractComponent {
|
|||
QueryPage<Bucket> buckets;
|
||||
try {
|
||||
buckets = jobProvider.buckets(job.getId(), latestBucketQuery);
|
||||
if (buckets.hits().size() == 1) {
|
||||
latestFinalBucketEndMs = buckets.hits().get(0).getTimestamp().getTime() + bucketSpan.toMillis() - 1;
|
||||
if (buckets.results().size() == 1) {
|
||||
latestFinalBucketEndMs = buckets.results().get(0).getTimestamp().getTime() + bucketSpan.toMillis() - 1;
|
||||
}
|
||||
} catch (ResourceNotFoundException e) {
|
||||
logger.error("Could not retrieve latest bucket timestamp", e);
|
||||
|
|
|
@ -126,7 +126,7 @@ public class GetBucketActionResponseTests extends AbstractStreamableTestCase<Get
|
|||
}
|
||||
hits.add(bucket);
|
||||
}
|
||||
QueryPage<Bucket> buckets = new QueryPage<>(hits, listSize);
|
||||
QueryPage<Bucket> buckets = new QueryPage<>(hits, listSize, Bucket.RESULTS_FIELD);
|
||||
return new Response(buckets);
|
||||
}
|
||||
|
||||
|
|
|
@ -15,8 +15,9 @@ public class GetCategoryDefinitionResponseTests extends AbstractStreamableTestCa
|
|||
|
||||
@Override
|
||||
protected GetCategoriesDefinitionAction.Response createTestInstance() {
|
||||
CategoryDefinition definition = new CategoryDefinition(randomAsciiOfLength(10));
|
||||
QueryPage<CategoryDefinition> queryPage =
|
||||
new QueryPage<>(Collections.singletonList(new CategoryDefinition(randomAsciiOfLength(10))), 1L);
|
||||
new QueryPage<>(Collections.singletonList(definition), 1L, CategoryDefinition.RESULTS_FIELD);
|
||||
return new GetCategoriesDefinitionAction.Response(queryPage);
|
||||
}
|
||||
|
||||
|
|
|
@ -31,7 +31,7 @@ public class GetInfluencersActionResponseTests extends AbstractStreamableTestCas
|
|||
influencer.setTimestamp(new Date(randomLong()));
|
||||
hits.add(influencer);
|
||||
}
|
||||
QueryPage<Influencer> buckets = new QueryPage<>(hits, listSize);
|
||||
QueryPage<Influencer> buckets = new QueryPage<>(hits, listSize, Influencer.RESULTS_FIELD);
|
||||
return new Response(buckets);
|
||||
}
|
||||
|
||||
|
|
|
@ -101,7 +101,7 @@ public class GetJobActionResponseTests extends AbstractStreamableTestCase<GetJob
|
|||
jobInfoList.add(jobInfo);
|
||||
}
|
||||
|
||||
result = new Response(new QueryPage<>(jobInfoList, jobInfoList.size()));
|
||||
result = new Response(new QueryPage<>(jobInfoList, jobInfoList.size(), Job.RESULTS_FIELD));
|
||||
|
||||
return result;
|
||||
}
|
||||
|
|
|
@ -24,7 +24,7 @@ public class GetModelSnapshotsActionResponseTests extends AbstractStreamableTest
|
|||
snapshot.setDescription(randomAsciiOfLengthBetween(1, 20));
|
||||
hits.add(snapshot);
|
||||
}
|
||||
QueryPage<ModelSnapshot> snapshots = new QueryPage<>(hits, listSize);
|
||||
QueryPage<ModelSnapshot> snapshots = new QueryPage<>(hits, listSize, ModelSnapshot.RESULTS_FIELD);
|
||||
return new Response(snapshots);
|
||||
}
|
||||
|
||||
|
|
|
@ -25,7 +25,7 @@ public class GetRecordsActionResponseTests extends AbstractStreamableTestCase<Ge
|
|||
record.setId(randomAsciiOfLengthBetween(1, 20));
|
||||
hits.add(record);
|
||||
}
|
||||
QueryPage<AnomalyRecord> snapshots = new QueryPage<>(hits, listSize);
|
||||
QueryPage<AnomalyRecord> snapshots = new QueryPage<>(hits, listSize, AnomalyRecord.RESULTS_FIELD);
|
||||
return new Response(snapshots);
|
||||
}
|
||||
|
||||
|
|
|
@ -44,7 +44,7 @@ public class PrelertJobIT extends ESRestTestCase {
|
|||
() -> client().performRequest("get", PrelertPlugin.BASE_PATH + "jobs/non-existing-job"));
|
||||
|
||||
assertThat(e.getResponse().getStatusLine().getStatusCode(), equalTo(404));
|
||||
assertThat(e.getMessage(), containsString("\"hitCount\":0"));
|
||||
assertThat(e.getMessage(), containsString("Could not find requested jobs"));
|
||||
}
|
||||
|
||||
public void testGetJob_GivenJobExists() throws Exception {
|
||||
|
@ -54,7 +54,7 @@ public class PrelertJobIT extends ESRestTestCase {
|
|||
|
||||
assertThat(response.getStatusLine().getStatusCode(), equalTo(200));
|
||||
String responseAsString = responseEntityToString(response);
|
||||
assertThat(responseAsString, containsString("\"hitCount\":1"));
|
||||
assertThat(responseAsString, containsString("\"count\":1"));
|
||||
assertThat(responseAsString, containsString("\"jobId\":\"farequote\""));
|
||||
}
|
||||
|
||||
|
@ -89,7 +89,7 @@ public class PrelertJobIT extends ESRestTestCase {
|
|||
|
||||
assertThat(response.getStatusLine().getStatusCode(), equalTo(200));
|
||||
String responseAsString = responseEntityToString(response);
|
||||
assertThat(responseAsString, containsString("\"hitCount\":1"));
|
||||
assertThat(responseAsString, containsString("\"count\":1"));
|
||||
assertThat(responseAsString, containsString("\"jobId\":\"farequote\""));
|
||||
}
|
||||
|
||||
|
@ -102,7 +102,7 @@ public class PrelertJobIT extends ESRestTestCase {
|
|||
|
||||
assertThat(response.getStatusLine().getStatusCode(), equalTo(200));
|
||||
String responseAsString = responseEntityToString(response);
|
||||
assertThat(responseAsString, containsString("\"hitCount\":3"));
|
||||
assertThat(responseAsString, containsString("\"count\":3"));
|
||||
assertThat(responseAsString, containsString("\"jobId\":\"farequote_1\""));
|
||||
assertThat(responseAsString, containsString("\"jobId\":\"farequote_2\""));
|
||||
assertThat(responseAsString, containsString("\"jobId\":\"farequote_3\""));
|
||||
|
@ -117,7 +117,7 @@ public class PrelertJobIT extends ESRestTestCase {
|
|||
|
||||
assertThat(response.getStatusLine().getStatusCode(), equalTo(200));
|
||||
String responseAsString = responseEntityToString(response);
|
||||
assertThat(responseAsString, containsString("\"hitCount\":3"));
|
||||
assertThat(responseAsString, containsString("\"count\":3"));
|
||||
assertThat(responseAsString, not(containsString("\"jobId\":\"farequote_1\"")));
|
||||
assertThat(responseAsString, containsString("\"jobId\":\"farequote_2\""));
|
||||
assertThat(responseAsString, containsString("\"jobId\":\"farequote_3\""));
|
||||
|
@ -132,7 +132,7 @@ public class PrelertJobIT extends ESRestTestCase {
|
|||
|
||||
assertThat(response.getStatusLine().getStatusCode(), equalTo(200));
|
||||
String responseAsString = responseEntityToString(response);
|
||||
assertThat(responseAsString, containsString("\"hitCount\":3"));
|
||||
assertThat(responseAsString, containsString("\"count\":3"));
|
||||
assertThat(responseAsString, containsString("\"jobId\":\"farequote_1\""));
|
||||
assertThat(responseAsString, not(containsString("\"jobId\":\"farequote_2\"")));
|
||||
assertThat(responseAsString, not(containsString("\"jobId\":\"farequote_3\"")));
|
||||
|
@ -147,7 +147,7 @@ public class PrelertJobIT extends ESRestTestCase {
|
|||
|
||||
assertThat(response.getStatusLine().getStatusCode(), equalTo(200));
|
||||
String responseAsString = responseEntityToString(response);
|
||||
assertThat(responseAsString, containsString("\"hitCount\":3"));
|
||||
assertThat(responseAsString, containsString("\"count\":3"));
|
||||
assertThat(responseAsString, not(containsString("\"jobId\":\"farequote_1\"")));
|
||||
assertThat(responseAsString, containsString("\"jobId\":\"farequote_2\""));
|
||||
assertThat(responseAsString, not(containsString("\"jobId\":\"farequote_3\"")));
|
||||
|
@ -183,13 +183,13 @@ public class PrelertJobIT extends ESRestTestCase {
|
|||
Response response = client().performRequest("get", PrelertPlugin.BASE_PATH + "results/1/buckets", params);
|
||||
assertThat(response.getStatusLine().getStatusCode(), equalTo(200));
|
||||
String responseAsString = responseEntityToString(response);
|
||||
assertThat(responseAsString, containsString("\"hitCount\":3"));
|
||||
assertThat(responseAsString, containsString("\"count\":3"));
|
||||
|
||||
params.put("end", "1235");
|
||||
response = client().performRequest("get", PrelertPlugin.BASE_PATH + "results/1/buckets", params);
|
||||
assertThat(response.getStatusLine().getStatusCode(), equalTo(200));
|
||||
responseAsString = responseEntityToString(response);
|
||||
assertThat(responseAsString, containsString("\"hitCount\":1"));
|
||||
assertThat(responseAsString, containsString("\"count\":1"));
|
||||
|
||||
e = expectThrows(ResponseException.class, () -> client().performRequest("get", PrelertPlugin.BASE_PATH + "results/2/buckets/1234"));
|
||||
assertThat(e.getResponse().getStatusLine().getStatusCode(), equalTo(404));
|
||||
|
|
|
@ -30,7 +30,6 @@ import org.junit.Before;
|
|||
|
||||
import java.util.Arrays;
|
||||
import java.util.HashSet;
|
||||
import java.util.Optional;
|
||||
import java.util.Set;
|
||||
import java.util.stream.Collectors;
|
||||
|
||||
|
@ -65,8 +64,8 @@ public class JobManagerTests extends ESTestCase {
|
|||
ClusterState clusterState = ClusterState.builder(new ClusterName("name"))
|
||||
.metaData(MetaData.builder().putCustom(PrelertMetadata.TYPE, builder.build())).build();
|
||||
QueryPage<Job> doc = jobManager.getJob("foo", clusterState);
|
||||
assertTrue(doc.hitCount() > 0);
|
||||
assertThat(doc.hits().get(0).getJobId(), equalTo("foo"));
|
||||
assertTrue(doc.count() > 0);
|
||||
assertThat(doc.results().get(0).getJobId(), equalTo("foo"));
|
||||
}
|
||||
|
||||
public void testFilter() {
|
||||
|
@ -162,41 +161,41 @@ public class JobManagerTests extends ESTestCase {
|
|||
|
||||
JobManager jobManager = createJobManager();
|
||||
QueryPage<Job> result = jobManager.getJobs(0, 10, clusterState);
|
||||
assertThat(result.hitCount(), equalTo(10L));
|
||||
assertThat(result.hits().get(0).getId(), equalTo("0"));
|
||||
assertThat(result.hits().get(1).getId(), equalTo("1"));
|
||||
assertThat(result.hits().get(2).getId(), equalTo("2"));
|
||||
assertThat(result.hits().get(3).getId(), equalTo("3"));
|
||||
assertThat(result.hits().get(4).getId(), equalTo("4"));
|
||||
assertThat(result.hits().get(5).getId(), equalTo("5"));
|
||||
assertThat(result.hits().get(6).getId(), equalTo("6"));
|
||||
assertThat(result.hits().get(7).getId(), equalTo("7"));
|
||||
assertThat(result.hits().get(8).getId(), equalTo("8"));
|
||||
assertThat(result.hits().get(9).getId(), equalTo("9"));
|
||||
assertThat(result.count(), equalTo(10L));
|
||||
assertThat(result.results().get(0).getId(), equalTo("0"));
|
||||
assertThat(result.results().get(1).getId(), equalTo("1"));
|
||||
assertThat(result.results().get(2).getId(), equalTo("2"));
|
||||
assertThat(result.results().get(3).getId(), equalTo("3"));
|
||||
assertThat(result.results().get(4).getId(), equalTo("4"));
|
||||
assertThat(result.results().get(5).getId(), equalTo("5"));
|
||||
assertThat(result.results().get(6).getId(), equalTo("6"));
|
||||
assertThat(result.results().get(7).getId(), equalTo("7"));
|
||||
assertThat(result.results().get(8).getId(), equalTo("8"));
|
||||
assertThat(result.results().get(9).getId(), equalTo("9"));
|
||||
|
||||
result = jobManager.getJobs(0, 5, clusterState);
|
||||
assertThat(result.hitCount(), equalTo(10L));
|
||||
assertThat(result.hits().get(0).getId(), equalTo("0"));
|
||||
assertThat(result.hits().get(1).getId(), equalTo("1"));
|
||||
assertThat(result.hits().get(2).getId(), equalTo("2"));
|
||||
assertThat(result.hits().get(3).getId(), equalTo("3"));
|
||||
assertThat(result.hits().get(4).getId(), equalTo("4"));
|
||||
assertThat(result.count(), equalTo(10L));
|
||||
assertThat(result.results().get(0).getId(), equalTo("0"));
|
||||
assertThat(result.results().get(1).getId(), equalTo("1"));
|
||||
assertThat(result.results().get(2).getId(), equalTo("2"));
|
||||
assertThat(result.results().get(3).getId(), equalTo("3"));
|
||||
assertThat(result.results().get(4).getId(), equalTo("4"));
|
||||
|
||||
result = jobManager.getJobs(5, 5, clusterState);
|
||||
assertThat(result.hitCount(), equalTo(10L));
|
||||
assertThat(result.hits().get(0).getId(), equalTo("5"));
|
||||
assertThat(result.hits().get(1).getId(), equalTo("6"));
|
||||
assertThat(result.hits().get(2).getId(), equalTo("7"));
|
||||
assertThat(result.hits().get(3).getId(), equalTo("8"));
|
||||
assertThat(result.hits().get(4).getId(), equalTo("9"));
|
||||
assertThat(result.count(), equalTo(10L));
|
||||
assertThat(result.results().get(0).getId(), equalTo("5"));
|
||||
assertThat(result.results().get(1).getId(), equalTo("6"));
|
||||
assertThat(result.results().get(2).getId(), equalTo("7"));
|
||||
assertThat(result.results().get(3).getId(), equalTo("8"));
|
||||
assertThat(result.results().get(4).getId(), equalTo("9"));
|
||||
|
||||
result = jobManager.getJobs(9, 1, clusterState);
|
||||
assertThat(result.hitCount(), equalTo(10L));
|
||||
assertThat(result.hits().get(0).getId(), equalTo("9"));
|
||||
assertThat(result.count(), equalTo(10L));
|
||||
assertThat(result.results().get(0).getId(), equalTo("9"));
|
||||
|
||||
result = jobManager.getJobs(9, 10, clusterState);
|
||||
assertThat(result.hitCount(), equalTo(10L));
|
||||
assertThat(result.hits().get(0).getId(), equalTo("9"));
|
||||
assertThat(result.count(), equalTo(10L));
|
||||
assertThat(result.results().get(0).getId(), equalTo("9"));
|
||||
}
|
||||
|
||||
public void testInnerPutJob() {
|
||||
|
|
|
@ -5,6 +5,7 @@
|
|||
*/
|
||||
package org.elasticsearch.xpack.prelert.job.persistence;
|
||||
|
||||
import org.elasticsearch.ResourceNotFoundException;
|
||||
import org.elasticsearch.action.ActionListener;
|
||||
import org.elasticsearch.action.get.GetResponse;
|
||||
import org.elasticsearch.action.search.SearchResponse;
|
||||
|
@ -228,7 +229,7 @@ public class ElasticsearchJobProviderTests extends ESTestCase {
|
|||
.normalizedProbabilityThreshold(1.0);
|
||||
|
||||
QueryPage<Bucket> buckets = provider.buckets(jobId, bq.build());
|
||||
assertEquals(1L, buckets.hitCount());
|
||||
assertEquals(1L, buckets.count());
|
||||
QueryBuilder query = queryBuilder.getValue();
|
||||
String queryString = query.toString();
|
||||
assertTrue(
|
||||
|
@ -261,7 +262,7 @@ public class ElasticsearchJobProviderTests extends ESTestCase {
|
|||
.normalizedProbabilityThreshold(10.9).includeInterim(true);
|
||||
|
||||
QueryPage<Bucket> buckets = provider.buckets(jobId, bq.build());
|
||||
assertEquals(1L, buckets.hitCount());
|
||||
assertEquals(1L, buckets.count());
|
||||
QueryBuilder query = queryBuilder.getValue();
|
||||
String queryString = query.toString();
|
||||
assertTrue(queryString.matches("(?s).*maxNormalizedProbability[^}]*from. : 10\\.9.*"));
|
||||
|
@ -299,7 +300,7 @@ public class ElasticsearchJobProviderTests extends ESTestCase {
|
|||
bq.includeInterim(true);
|
||||
|
||||
QueryPage<Bucket> buckets = provider.buckets(jobId, bq.build());
|
||||
assertEquals(1L, buckets.hitCount());
|
||||
assertEquals(1L, buckets.count());
|
||||
QueryBuilder query = queryBuilder.getValue();
|
||||
String queryString = query.toString();
|
||||
assertTrue(queryString.matches("(?s).*maxNormalizedProbability[^}]*from. : 10\\.9.*"));
|
||||
|
@ -329,9 +330,8 @@ public class ElasticsearchJobProviderTests extends ESTestCase {
|
|||
|
||||
BucketQueryBuilder bq = new BucketQueryBuilder(Long.toString(timestamp));
|
||||
|
||||
QueryPage<Bucket> bucket = provider.bucket(jobId, bq.build());
|
||||
assertThat(bucket.hitCount(), equalTo(0L));
|
||||
assertThat(bucket.hits(), empty());
|
||||
ResourceNotFoundException e = expectThrows(ResourceNotFoundException.class,
|
||||
() ->provider.bucket(jobId, bq.build()));
|
||||
}
|
||||
|
||||
public void testBucket_OneBucketNoExpandNoInterim()
|
||||
|
@ -357,8 +357,8 @@ public class ElasticsearchJobProviderTests extends ESTestCase {
|
|||
BucketQueryBuilder bq = new BucketQueryBuilder(Long.toString(now.getTime()));
|
||||
|
||||
QueryPage<Bucket> bucketHolder = provider.bucket(jobId, bq.build());
|
||||
assertThat(bucketHolder.hitCount(), equalTo(1L));
|
||||
Bucket b = bucketHolder.hits().get(0);
|
||||
assertThat(bucketHolder.count(), equalTo(1L));
|
||||
Bucket b = bucketHolder.results().get(0);
|
||||
assertEquals(now, b.getTimestamp());
|
||||
}
|
||||
|
||||
|
@ -385,9 +385,8 @@ public class ElasticsearchJobProviderTests extends ESTestCase {
|
|||
|
||||
BucketQueryBuilder bq = new BucketQueryBuilder(Long.toString(now.getTime()));
|
||||
|
||||
QueryPage<Bucket> bucketHolder = provider.bucket(jobId, bq.build());
|
||||
assertThat(bucketHolder.hitCount(), equalTo(0L));
|
||||
assertThat(bucketHolder.hits(), empty());
|
||||
ResourceNotFoundException e = expectThrows(ResourceNotFoundException.class,
|
||||
() ->provider.bucket(jobId, bq.build()));
|
||||
}
|
||||
|
||||
public void testRecords() throws InterruptedException, ExecutionException, IOException {
|
||||
|
@ -429,8 +428,8 @@ public class ElasticsearchJobProviderTests extends ESTestCase {
|
|||
.normalizedProbability(2.2);
|
||||
|
||||
QueryPage<AnomalyRecord> recordPage = provider.records(jobId, rqb.build());
|
||||
assertEquals(2L, recordPage.hitCount());
|
||||
List<AnomalyRecord> records = recordPage.hits();
|
||||
assertEquals(2L, recordPage.count());
|
||||
List<AnomalyRecord> records = recordPage.results();
|
||||
assertEquals(22.4, records.get(0).getTypical().get(0), 0.000001);
|
||||
assertEquals(33.3, records.get(0).getActual().get(0), 0.000001);
|
||||
assertEquals("irritable", records.get(0).getFunction());
|
||||
|
@ -485,8 +484,8 @@ public class ElasticsearchJobProviderTests extends ESTestCase {
|
|||
rqb.normalizedProbability(2.2);
|
||||
|
||||
QueryPage<AnomalyRecord> recordPage = provider.records(jobId, rqb.build());
|
||||
assertEquals(2L, recordPage.hitCount());
|
||||
List<AnomalyRecord> records = recordPage.hits();
|
||||
assertEquals(2L, recordPage.count());
|
||||
List<AnomalyRecord> records = recordPage.results();
|
||||
assertEquals(22.4, records.get(0).getTypical().get(0), 0.000001);
|
||||
assertEquals(33.3, records.get(0).getActual().get(0), 0.000001);
|
||||
assertEquals("irritable", records.get(0).getFunction());
|
||||
|
@ -533,8 +532,8 @@ public class ElasticsearchJobProviderTests extends ESTestCase {
|
|||
|
||||
QueryPage<AnomalyRecord> recordPage = provider.bucketRecords(jobId, bucket, from, size, true, sortfield, true, "");
|
||||
|
||||
assertEquals(2L, recordPage.hitCount());
|
||||
List<AnomalyRecord> records = recordPage.hits();
|
||||
assertEquals(2L, recordPage.count());
|
||||
List<AnomalyRecord> records = recordPage.results();
|
||||
assertEquals(22.4, records.get(0).getTypical().get(0), 0.000001);
|
||||
assertEquals(33.3, records.get(0).getActual().get(0), 0.000001);
|
||||
assertEquals("irritable", records.get(0).getFunction());
|
||||
|
@ -633,8 +632,8 @@ public class ElasticsearchJobProviderTests extends ESTestCase {
|
|||
Client client = clientBuilder.build();
|
||||
ElasticsearchJobProvider provider = createProvider(client);
|
||||
QueryPage<CategoryDefinition> categoryDefinitions = provider.categoryDefinitions(jobId, from, size);
|
||||
assertEquals(1L, categoryDefinitions.hitCount());
|
||||
assertEquals(terms, categoryDefinitions.hits().get(0).getTerms());
|
||||
assertEquals(1L, categoryDefinitions.count());
|
||||
assertEquals(terms, categoryDefinitions.results().get(0).getTerms());
|
||||
}
|
||||
|
||||
public void testCategoryDefinition()
|
||||
|
@ -657,8 +656,8 @@ public class ElasticsearchJobProviderTests extends ESTestCase {
|
|||
Client client = clientBuilder.build();
|
||||
ElasticsearchJobProvider provider = createProvider(client);
|
||||
QueryPage<CategoryDefinition> categoryDefinitions = provider.categoryDefinition(jobId, categoryId);
|
||||
assertEquals(1L, categoryDefinitions.hitCount());
|
||||
assertEquals(terms, categoryDefinitions.hits().get(0).getTerms());
|
||||
assertEquals(1L, categoryDefinitions.count());
|
||||
assertEquals(terms, categoryDefinitions.results().get(0).getTerms());
|
||||
}
|
||||
|
||||
public void testInfluencers_NoInterim()
|
||||
|
@ -699,12 +698,12 @@ public class ElasticsearchJobProviderTests extends ESTestCase {
|
|||
|
||||
InfluencersQuery query = new InfluencersQueryBuilder().from(from).size(size).includeInterim(false).build();
|
||||
QueryPage<Influencer> page = provider.influencers(jobId, query);
|
||||
assertEquals(2L, page.hitCount());
|
||||
assertEquals(2L, page.count());
|
||||
|
||||
String queryString = queryBuilder.getValue().toString();
|
||||
assertTrue(queryString.matches("(?s).*must_not[^}]*term[^}]*isInterim.*value. : .true.*"));
|
||||
|
||||
List<Influencer> records = page.hits();
|
||||
List<Influencer> records = page.results();
|
||||
assertEquals("foo", records.get(0).getJobId());
|
||||
assertEquals("Bob", records.get(0).getInfluencerFieldValue());
|
||||
assertEquals("Builder", records.get(0).getInfluencerFieldName());
|
||||
|
@ -760,12 +759,12 @@ public class ElasticsearchJobProviderTests extends ESTestCase {
|
|||
InfluencersQuery query = new InfluencersQueryBuilder().from(from).size(size).epochStart("0").epochEnd("0").sortField("sort")
|
||||
.sortDescending(true).anomalyScoreThreshold(0.0).includeInterim(true).build();
|
||||
QueryPage<Influencer> page = provider.influencers(jobId, query);
|
||||
assertEquals(2L, page.hitCount());
|
||||
assertEquals(2L, page.count());
|
||||
|
||||
String queryString = queryBuilder.getValue().toString();
|
||||
assertFalse(queryString.matches("(?s).*isInterim.*"));
|
||||
|
||||
List<Influencer> records = page.hits();
|
||||
List<Influencer> records = page.results();
|
||||
assertEquals("Bob", records.get(0).getInfluencerFieldValue());
|
||||
assertEquals("Builder", records.get(0).getInfluencerFieldName());
|
||||
assertEquals(now, records.get(0).getTimestamp());
|
||||
|
@ -834,8 +833,8 @@ public class ElasticsearchJobProviderTests extends ESTestCase {
|
|||
ElasticsearchJobProvider provider = createProvider(client);
|
||||
|
||||
QueryPage<ModelSnapshot> page = provider.modelSnapshots(jobId, from, size);
|
||||
assertEquals(2L, page.hitCount());
|
||||
List<ModelSnapshot> snapshots = page.hits();
|
||||
assertEquals(2L, page.count());
|
||||
List<ModelSnapshot> snapshots = page.results();
|
||||
|
||||
assertEquals("foo", snapshots.get(0).getJobId());
|
||||
assertEquals(now, snapshots.get(0).getTimestamp());
|
||||
|
@ -891,8 +890,8 @@ public class ElasticsearchJobProviderTests extends ESTestCase {
|
|||
|
||||
QueryPage<ModelSnapshot> page = provider.modelSnapshots(jobId, from, size, null, null, "sortfield", true, "snappyId",
|
||||
"description1");
|
||||
assertEquals(2L, page.hitCount());
|
||||
List<ModelSnapshot> snapshots = page.hits();
|
||||
assertEquals(2L, page.count());
|
||||
List<ModelSnapshot> snapshots = page.results();
|
||||
|
||||
assertEquals(now, snapshots.get(0).getTimestamp());
|
||||
assertEquals(now, snapshots.get(0).getLatestRecordTimeStamp());
|
||||
|
|
|
@ -5,6 +5,7 @@
|
|||
*/
|
||||
package org.elasticsearch.xpack.prelert.job.persistence;
|
||||
|
||||
import org.elasticsearch.common.ParseField;
|
||||
import org.elasticsearch.common.io.stream.Writeable.Reader;
|
||||
import org.elasticsearch.xpack.prelert.job.results.Influencer;
|
||||
import org.elasticsearch.xpack.prelert.support.AbstractWireSerializingTestCase;
|
||||
|
@ -20,7 +21,7 @@ public class QueryPageTests extends AbstractWireSerializingTestCase<QueryPage<In
|
|||
hits.add(new Influencer(randomAsciiOfLengthBetween(1, 20), randomAsciiOfLengthBetween(1, 20),
|
||||
randomAsciiOfLengthBetween(1, 20)));
|
||||
}
|
||||
return new QueryPage<>(hits, hitCount);
|
||||
return new QueryPage<>(hits, hitCount, new ParseField("test"));
|
||||
}
|
||||
|
||||
@Override
|
||||
|
|
|
@ -87,8 +87,8 @@ public class ScheduledJobServiceTests extends ESTestCase {
|
|||
new ScheduledJobService(threadPool, client, jobProvider, dataProcessor, dataExtractorFactory, () -> currentTime);
|
||||
|
||||
when(jobProvider.audit(anyString())).thenReturn(auditor);
|
||||
when(jobProvider.buckets(anyString(), any(BucketsQueryBuilder.BucketsQuery.class))).thenReturn(
|
||||
new QueryPage<>(Collections.emptyList(), 0));
|
||||
when(jobProvider.buckets(anyString(), any(BucketsQueryBuilder.BucketsQuery.class))).thenThrow(
|
||||
QueryPage.emptyQueryPage(Job.RESULTS_FIELD));
|
||||
}
|
||||
|
||||
public void testStart_GivenNewlyCreatedJobLoopBack() throws IOException {
|
||||
|
|
|
@ -33,7 +33,7 @@ public class GetModelSnapshotsTests extends ESTestCase {
|
|||
|
||||
public void testModelSnapshots_GivenNoStartOrEndParams() {
|
||||
ModelSnapshot modelSnapshot = new ModelSnapshot(randomAsciiOfLengthBetween(1, 20));
|
||||
QueryPage<ModelSnapshot> queryResult = new QueryPage<>(Collections.singletonList(modelSnapshot), 300);
|
||||
QueryPage<ModelSnapshot> queryResult = new QueryPage<>(Collections.singletonList(modelSnapshot), 300, ModelSnapshot.RESULTS_FIELD);
|
||||
|
||||
JobProvider jobProvider = mock(JobProvider.class);
|
||||
when(jobProvider.modelSnapshots("foo", 0, 100, null, null, null, true, null, null)).thenReturn(queryResult);
|
||||
|
@ -43,12 +43,12 @@ public class GetModelSnapshotsTests extends ESTestCase {
|
|||
request.setDescOrder(true);
|
||||
|
||||
QueryPage<ModelSnapshot> page = GetModelSnapshotsAction.TransportAction.doGetPage(jobProvider, request);
|
||||
assertEquals(300, page.hitCount());
|
||||
assertEquals(300, page.count());
|
||||
}
|
||||
|
||||
public void testModelSnapshots_GivenEpochStartAndEpochEndParams() {
|
||||
ModelSnapshot modelSnapshot = new ModelSnapshot(randomAsciiOfLengthBetween(1, 20));
|
||||
QueryPage<ModelSnapshot> queryResult = new QueryPage<>(Collections.singletonList(modelSnapshot), 300);
|
||||
QueryPage<ModelSnapshot> queryResult = new QueryPage<>(Collections.singletonList(modelSnapshot), 300, ModelSnapshot.RESULTS_FIELD);
|
||||
|
||||
JobProvider jobProvider = mock(JobProvider.class);
|
||||
when(jobProvider.modelSnapshots("foo", 0, 100, "1", "2", null, true, null, null)).thenReturn(queryResult);
|
||||
|
@ -60,12 +60,12 @@ public class GetModelSnapshotsTests extends ESTestCase {
|
|||
request.setDescOrder(true);
|
||||
|
||||
QueryPage<ModelSnapshot> page = GetModelSnapshotsAction.TransportAction.doGetPage(jobProvider, request);
|
||||
assertEquals(300, page.hitCount());
|
||||
assertEquals(300, page.count());
|
||||
}
|
||||
|
||||
public void testModelSnapshots_GivenIsoWithMillisStartAndEpochEndParams() {
|
||||
ModelSnapshot modelSnapshot = new ModelSnapshot(randomAsciiOfLengthBetween(1, 20));
|
||||
QueryPage<ModelSnapshot> queryResult = new QueryPage<>(Collections.singletonList(modelSnapshot), 300);
|
||||
QueryPage<ModelSnapshot> queryResult = new QueryPage<>(Collections.singletonList(modelSnapshot), 300, ModelSnapshot.RESULTS_FIELD);
|
||||
|
||||
JobProvider jobProvider = mock(JobProvider.class);
|
||||
when(jobProvider.modelSnapshots("foo", 0, 100, "2015-01-01T12:00:00.042Z", "2015-01-01T13:00:00.142+00:00", null, true, null, null))
|
||||
|
@ -78,12 +78,12 @@ public class GetModelSnapshotsTests extends ESTestCase {
|
|||
request.setDescOrder(true);
|
||||
|
||||
QueryPage<ModelSnapshot> page = GetModelSnapshotsAction.TransportAction.doGetPage(jobProvider, request);
|
||||
assertEquals(300, page.hitCount());
|
||||
assertEquals(300, page.count());
|
||||
}
|
||||
|
||||
public void testModelSnapshots_GivenIsoWithoutMillisStartAndEpochEndParams() {
|
||||
ModelSnapshot modelSnapshot = new ModelSnapshot(randomAsciiOfLengthBetween(1, 20));
|
||||
QueryPage<ModelSnapshot> queryResult = new QueryPage<>(Collections.singletonList(modelSnapshot), 300);
|
||||
QueryPage<ModelSnapshot> queryResult = new QueryPage<>(Collections.singletonList(modelSnapshot), 300, ModelSnapshot.RESULTS_FIELD);
|
||||
|
||||
JobProvider jobProvider = mock(JobProvider.class);
|
||||
when(jobProvider.modelSnapshots("foo", 0, 100, "2015-01-01T12:00:00Z", "2015-01-01T13:00:00Z", null, true, null, null))
|
||||
|
@ -96,6 +96,6 @@ public class GetModelSnapshotsTests extends ESTestCase {
|
|||
request.setDescOrder(true);
|
||||
|
||||
QueryPage<ModelSnapshot> page = GetModelSnapshotsAction.TransportAction.doGetPage(jobProvider, request);
|
||||
assertEquals(300, page.hitCount());
|
||||
assertEquals(300, page.count());
|
||||
}
|
||||
}
|
||||
|
|
|
@ -92,7 +92,7 @@ setup:
|
|||
- do:
|
||||
xpack.prelert.get_model_snapshots:
|
||||
jobId: "foo"
|
||||
- match: { hitCount: 2 }
|
||||
- match: { count: 2 }
|
||||
|
||||
- do:
|
||||
count:
|
||||
|
@ -114,8 +114,8 @@ setup:
|
|||
- do:
|
||||
xpack.prelert.get_model_snapshots:
|
||||
jobId: "foo"
|
||||
- match: { hitCount: 1 }
|
||||
- match: { hits.0.snapshotId: "foo2"}
|
||||
- match: { count: 1 }
|
||||
- match: { modelSnapshots.0.snapshotId: "foo2"}
|
||||
|
||||
- do:
|
||||
count:
|
||||
|
|
|
@ -35,9 +35,9 @@ setup:
|
|||
jobId: "foo"
|
||||
|
||||
|
||||
- match: { hitCount: 2 }
|
||||
- match: { hits.0.restorePriority: 2 }
|
||||
- match: { hits.0.timestamp: 1464739200000 }
|
||||
- match: { count: 2 }
|
||||
- match: { modelSnapshots.0.restorePriority: 2 }
|
||||
- match: { modelSnapshots.0.timestamp: 1464739200000 }
|
||||
|
||||
---
|
||||
"Test get model snapshots API with start/end":
|
||||
|
@ -48,9 +48,9 @@ setup:
|
|||
end: "2016-07-01T00:00:00Z"
|
||||
|
||||
|
||||
- match: { hitCount: 2 }
|
||||
- match: { hits.0.restorePriority: 2 }
|
||||
- match: { hits.0.timestamp: 1464739200000 }
|
||||
- match: { count: 2 }
|
||||
- match: { modelSnapshots.0.restorePriority: 2 }
|
||||
- match: { modelSnapshots.0.timestamp: 1464739200000 }
|
||||
|
||||
---
|
||||
"Test get model snapshots API with ascending":
|
||||
|
@ -60,9 +60,9 @@ setup:
|
|||
desc: false
|
||||
|
||||
|
||||
- match: { hitCount: 2 }
|
||||
- match: { hits.0.restorePriority: 1 }
|
||||
- match: { hits.0.timestamp: 1464825600000 }
|
||||
- match: { count: 2 }
|
||||
- match: { modelSnapshots.0.restorePriority: 1 }
|
||||
- match: { modelSnapshots.0.timestamp: 1464825600000 }
|
||||
|
||||
---
|
||||
"Test get model snapshots API with size":
|
||||
|
@ -72,10 +72,10 @@ setup:
|
|||
size: 1
|
||||
|
||||
|
||||
- match: { hitCount: 2 }
|
||||
- match: { hits.0.restorePriority: 2 }
|
||||
- match: { hits.0.timestamp: 1464739200000 }
|
||||
- length: { hits: 1 }
|
||||
- match: { count: 2 }
|
||||
- match: { modelSnapshots.0.restorePriority: 2 }
|
||||
- match: { modelSnapshots.0.timestamp: 1464739200000 }
|
||||
- length: { modelSnapshots: 1 }
|
||||
|
||||
---
|
||||
"Test get model snapshots API with from":
|
||||
|
@ -85,7 +85,7 @@ setup:
|
|||
from: 1
|
||||
|
||||
|
||||
- match: { hitCount: 2 }
|
||||
- match: { hits.0.restorePriority: 1 }
|
||||
- match: { hits.0.timestamp: 1464825600000 }
|
||||
- length: { hits: 1 }
|
||||
- match: { count: 2 }
|
||||
- match: { modelSnapshots.0.restorePriority: 1 }
|
||||
- match: { modelSnapshots.0.timestamp: 1464825600000 }
|
||||
- length: { modelSnapshots: 1 }
|
||||
|
|
|
@ -27,14 +27,14 @@
|
|||
xpack.prelert.get_jobs:
|
||||
from: 0
|
||||
size: 100
|
||||
- match: { hitCount: 1 }
|
||||
- match: { hits.0.config.jobId: "farequote" }
|
||||
- match: { count: 1 }
|
||||
- match: { jobs.0.config.jobId: "farequote" }
|
||||
|
||||
- do:
|
||||
xpack.prelert.get_jobs:
|
||||
job_id: "farequote"
|
||||
- match: { hitCount: 1 }
|
||||
- match: { hits.0.config.jobId: "farequote" }
|
||||
- match: { count: 1 }
|
||||
- match: { jobs.0.config.jobId: "farequote" }
|
||||
|
||||
- do:
|
||||
xpack.prelert.delete_job:
|
||||
|
|
|
@ -27,9 +27,9 @@ setup:
|
|||
start: "2016-05-01T00:00:00Z"
|
||||
end: "2016-07-01T00:00:00Z"
|
||||
|
||||
- match: { hitCount: 1 }
|
||||
- match: { hits.0.timestamp: 1464739200000 }
|
||||
- match: { hits.0.jobId: farequote}
|
||||
- match: { count: 1 }
|
||||
- match: { buckets.0.timestamp: 1464739200000 }
|
||||
- match: { buckets.0.jobId: farequote}
|
||||
|
||||
---
|
||||
"Test result bucket api":
|
||||
|
@ -38,5 +38,5 @@ setup:
|
|||
jobId: "farequote"
|
||||
timestamp: "2016-06-01T00:00:00Z"
|
||||
|
||||
- match: { hits.0.timestamp: 1464739200000}
|
||||
- match: { hits.0.jobId: farequote }
|
||||
- match: { buckets.0.timestamp: 1464739200000}
|
||||
- match: { buckets.0.jobId: farequote }
|
||||
|
|
|
@ -28,11 +28,11 @@ setup:
|
|||
xpack.prelert.get_categories:
|
||||
jobId: "farequote"
|
||||
|
||||
- match: { hitCount: 2 }
|
||||
- match: { hits.0.jobId: farequote }
|
||||
- match: { hits.0.categoryId: 1 }
|
||||
- match: { hits.1.jobId: farequote }
|
||||
- match: { hits.1.categoryId: 2 }
|
||||
- match: { count: 2 }
|
||||
- match: { categories.0.jobId: farequote }
|
||||
- match: { categories.0.categoryId: 1 }
|
||||
- match: { categories.1.jobId: farequote }
|
||||
- match: { categories.1.categoryId: 2 }
|
||||
|
||||
---
|
||||
"Test result category api":
|
||||
|
@ -41,5 +41,5 @@ setup:
|
|||
jobId: "farequote"
|
||||
categoryId: "1"
|
||||
|
||||
- match: { hits.0.jobId: farequote }
|
||||
- match: { hits.0.categoryId: 1 }
|
||||
- match: { categories.0.jobId: farequote }
|
||||
- match: { categories.0.categoryId: 1 }
|
||||
|
|
|
@ -37,6 +37,6 @@ setup:
|
|||
start: "2016-05-01T00:00:00Z"
|
||||
end: "2016-07-01T00:00:00Z"
|
||||
|
||||
- match: { hitCount: 1 }
|
||||
- match: { hits.0.timestamp: 1464739200000 }
|
||||
- match: { count: 1 }
|
||||
- match: { influencers.0.timestamp: 1464739200000 }
|
||||
|
||||
|
|
|
@ -46,7 +46,7 @@ setup:
|
|||
start: "2016-05-01T00:00:00Z"
|
||||
end: "2016-07-01T00:00:00Z"
|
||||
|
||||
- match: { hitCount: 1 }
|
||||
- match: { hits.0.timestamp: 1464739200000 }
|
||||
- match: { hits.0.jobId: farequote}
|
||||
- match: { hits.0.result_type: record}
|
||||
- match: { count: 1 }
|
||||
- match: { records.0.timestamp: 1464739200000 }
|
||||
- match: { records.0.jobId: farequote}
|
||||
- match: { records.0.result_type: record}
|
||||
|
|
|
@ -46,10 +46,10 @@ setup:
|
|||
xpack.prelert.get_jobs:
|
||||
job_id: job-stats-test
|
||||
|
||||
- is_true: hits.0.config
|
||||
- is_false: hits.0.data_counts
|
||||
- is_false: hits.0.model_size_stats
|
||||
- is_false: hits.0.scheduler_state
|
||||
- is_true: jobs.0.config
|
||||
- is_false: jobs.0.data_counts
|
||||
- is_false: jobs.0.model_size_stats
|
||||
- is_false: jobs.0.scheduler_state
|
||||
|
||||
|
||||
- do:
|
||||
|
@ -70,9 +70,9 @@ setup:
|
|||
job_id: job-stats-test
|
||||
metric: data_counts
|
||||
|
||||
- match: { hits.0.data_counts.processed_record_count: 2 }
|
||||
- match: { hits.0.data_counts.processed_field_count: 4}
|
||||
- match: { hits.0.data_counts.input_field_count: 4 }
|
||||
- match: { jobs.0.data_counts.processed_record_count: 2 }
|
||||
- match: { jobs.0.data_counts.processed_field_count: 4}
|
||||
- match: { jobs.0.data_counts.input_field_count: 4 }
|
||||
|
||||
# Test filters
|
||||
# It's difficult to test for the presence of model_size_stats as they
|
||||
|
@ -81,51 +81,51 @@ setup:
|
|||
xpack.prelert.get_jobs:
|
||||
job_id: "job-stats-test"
|
||||
metric: "data_counts"
|
||||
- is_false: hits.0.config
|
||||
- is_true: hits.0.data_counts
|
||||
- is_false: hits.0.model_size_stats
|
||||
- is_false: hits.0.scheduler_state
|
||||
- is_false: jobs.0.config
|
||||
- is_true: jobs.0.data_counts
|
||||
- is_false: jobs.0.model_size_stats
|
||||
- is_false: jobs.0.scheduler_state
|
||||
|
||||
- do:
|
||||
xpack.prelert.get_jobs:
|
||||
job_id: "job-stats-test"
|
||||
metric: "model_size_stats"
|
||||
- is_false: hits.0.config
|
||||
- is_false: hits.0.data_counts
|
||||
- is_false: hits.0.scheduler_state
|
||||
- is_false: jobs.0.config
|
||||
- is_false: jobs.0.data_counts
|
||||
- is_false: jobs.0.scheduler_state
|
||||
|
||||
- do:
|
||||
xpack.prelert.get_jobs:
|
||||
job_id: "job-stats-test"
|
||||
metric: "scheduler_state"
|
||||
- is_false: hits.0.config
|
||||
- is_false: hits.0.data_counts
|
||||
- is_false: jobs.0.config
|
||||
- is_false: jobs.0.data_counts
|
||||
|
||||
- do:
|
||||
xpack.prelert.get_jobs:
|
||||
job_id: "job-stats-test"
|
||||
metric: "status"
|
||||
- is_false: hits.0.config
|
||||
- is_false: hits.0.data_counts
|
||||
- is_false: hits.0.model_size_stats
|
||||
- is_false: hits.0.scheduler_state
|
||||
- match: { hits.0.status: CLOSED }
|
||||
- is_false: jobs.0.config
|
||||
- is_false: jobs.0.data_counts
|
||||
- is_false: jobs.0.model_size_stats
|
||||
- is_false: jobs.0.scheduler_state
|
||||
- match: { jobs.0.status: CLOSED }
|
||||
|
||||
- do:
|
||||
xpack.prelert.get_jobs:
|
||||
job_id: "job-stats-test"
|
||||
metric: "_all"
|
||||
- is_true: hits.0.config
|
||||
- is_true: hits.0.data_counts
|
||||
- is_false: hits.0.scheduler_state
|
||||
- match: { hits.0.status: CLOSED }
|
||||
- is_true: jobs.0.config
|
||||
- is_true: jobs.0.data_counts
|
||||
- is_false: jobs.0.scheduler_state
|
||||
- match: { jobs.0.status: CLOSED }
|
||||
|
||||
- do:
|
||||
xpack.prelert.get_jobs:
|
||||
job_id: "scheduled-job"
|
||||
metric: "scheduler_state"
|
||||
- is_false: hits.0.config
|
||||
- is_false: hits.0.data_counts
|
||||
- is_false: hits.0.model_size_stats
|
||||
- match: { hits.0.scheduler_state.status: STOPPED }
|
||||
- is_false: jobs.0.config
|
||||
- is_false: jobs.0.data_counts
|
||||
- is_false: jobs.0.model_size_stats
|
||||
- match: { jobs.0.scheduler_state.status: STOPPED }
|
||||
|
||||
|
|
|
@ -60,8 +60,8 @@ setup:
|
|||
jobId: "foo"
|
||||
description: "new_description"
|
||||
|
||||
- match: { hitCount: 0 }
|
||||
- length: { hits: 0 }
|
||||
- match: { count: 0 }
|
||||
- length: { modelSnapshots: 0 }
|
||||
|
||||
- do:
|
||||
xpack.prelert.put_model_snapshot_description:
|
||||
|
@ -81,9 +81,9 @@ setup:
|
|||
jobId: "foo"
|
||||
description: "new_description"
|
||||
|
||||
- match: { hitCount: 1 }
|
||||
- match: { hits.0.restorePriority: 1 }
|
||||
- match: { hits.0.timestamp: 1464825600000 }
|
||||
- match: { count: 1 }
|
||||
- match: { modelSnapshots.0.restorePriority: 1 }
|
||||
- match: { modelSnapshots.0.timestamp: 1464825600000 }
|
||||
|
||||
---
|
||||
"Test with conflict against existing description":
|
||||
|
@ -92,10 +92,10 @@ setup:
|
|||
jobId: "foo"
|
||||
description: "bar"
|
||||
|
||||
- match: { hitCount: 1 }
|
||||
- length: { hits: 1 }
|
||||
- match: { hits.0.jobId: "foo" }
|
||||
- match: { hits.0.description: "bar" }
|
||||
- match: { count: 1 }
|
||||
- length: { modelSnapshots: 1 }
|
||||
- match: { modelSnapshots.0.jobId: "foo" }
|
||||
- match: { modelSnapshots.0.description: "bar" }
|
||||
|
||||
- do:
|
||||
catch: request
|
||||
|
|
|
@ -195,7 +195,7 @@ setup:
|
|||
start: "2016-01-01T00:00:00Z"
|
||||
end: "2016-12-01T00:00:00Z"
|
||||
|
||||
- match: { hitCount: 3 }
|
||||
- match: { count: 3 }
|
||||
|
||||
- do:
|
||||
xpack.prelert.revert_model_snapshot:
|
||||
|
@ -213,15 +213,15 @@ setup:
|
|||
start: "2016-01-01T00:00:00Z"
|
||||
end: "2016-12-01T00:00:00Z"
|
||||
|
||||
- match: { hitCount: 1 }
|
||||
- match: { hits.0.jobId: "foo" }
|
||||
- match: { hits.0.timestamp: 1462060800000 }
|
||||
- match: { count: 1 }
|
||||
- match: { buckets.0.jobId: "foo" }
|
||||
- match: { buckets.0.timestamp: 1462060800000 }
|
||||
|
||||
- do:
|
||||
xpack.prelert.get_jobs:
|
||||
job_id: foo
|
||||
metric: data_counts
|
||||
|
||||
- match: { hits.0.data_counts.latest_record_timestamp: 1464739200000 }
|
||||
- match: { jobs.0.data_counts.latest_record_timestamp: 1464739200000 }
|
||||
|
||||
|
||||
|
|
Loading…
Reference in New Issue