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:
Zachary Tong 2016-11-23 11:12:39 -05:00 committed by GitHub
parent 9fc3c77905
commit 17b3224e03
39 changed files with 264 additions and 238 deletions

View File

@ -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()));

View File

@ -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

View File

@ -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));

View File

@ -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);
}
}

View File

@ -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();
}
}

View File

@ -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()));

View File

@ -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 {

View File

@ -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
*/

View File

@ -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);
}
/**

View File

@ -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

View File

@ -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
* &amp; 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);
}
}

View File

@ -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(),

View File

@ -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)
*/

View File

@ -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
*/

View File

@ -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]));

View File

@ -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]));

View File

@ -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);

View File

@ -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);
}

View File

@ -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);
}

View File

@ -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);
}

View File

@ -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;
}

View File

@ -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);
}

View File

@ -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);
}

View File

@ -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));

View File

@ -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() {

View File

@ -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());

View File

@ -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

View File

@ -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 {

View File

@ -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());
}
}

View File

@ -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:

View File

@ -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 }

View File

@ -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:

View File

@ -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 }

View File

@ -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 }

View File

@ -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 }

View File

@ -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}

View File

@ -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 }

View File

@ -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

View File

@ -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 }