Merge branch 'master' into ccr

* master:
  Painless: Add Bindings (#33042)
  Update version after client credentials backport
  Fix forbidden apis on FIPS (#33202)
  Remote 6.x transport BWC Layer for `_shrink` (#33236)
  Test fix - Graph HLRC tests needed another field adding to randomisation exception list
  HLRC: Add ML Get Records API (#33085)
  [ML] Fix character set finder bug with unencodable charsets (#33234)
  TESTS: Fix overly long lines (#33240)
  Test fix - Graph HLRC test was missing field name to be excluded from randomisation logic
  Remove unsupported group_shard_failures parameter (#33208)
  Update BucketUtils#suggestShardSideQueueSize signature (#33210)
  Parse PEM Key files leniantly (#33173)
  INGEST: Add Pipeline Processor (#32473)
  Core: Add java time xcontent serializers (#33120)
  Consider multi release jars when running third party audit (#33206)
  Update MSI documentation (#31950)
  HLRC: create base timed request class (#33216)
  [DOCS] Fixes command page titles
  HLRC: Move ML protocol classes into client ml package (#33203)
  Scroll queries asking for rescore are considered invalid (#32918)
  Painless: Fix Semicolon Regression (#33212)
  ingest: minor - update test to include dissect (#33211)
  Switch remaining LLREST usage to new style Requests (#33171)
  HLREST: add reindex API (#32679)
This commit is contained in:
Nhat Nguyen 2018-08-29 12:30:24 -04:00
commit 5632e31c74
222 changed files with 4884 additions and 1436 deletions

View File

@ -87,6 +87,7 @@ class PrecommitTasks {
dependsOn(buildResources) dependsOn(buildResources)
signatureFile = buildResources.copy("forbidden/third-party-audit.txt") signatureFile = buildResources.copy("forbidden/third-party-audit.txt")
javaHome = project.runtimeJavaHome javaHome = project.runtimeJavaHome
targetCompatibility = project.runtimeJavaVersion
} }
return thirdPartyAuditTask return thirdPartyAuditTask
} }

View File

@ -23,6 +23,7 @@ import org.elasticsearch.gradle.JdkJarHellCheck;
import org.elasticsearch.test.NamingConventionsCheck; import org.elasticsearch.test.NamingConventionsCheck;
import org.gradle.api.DefaultTask; import org.gradle.api.DefaultTask;
import org.gradle.api.GradleException; import org.gradle.api.GradleException;
import org.gradle.api.JavaVersion;
import org.gradle.api.artifacts.Configuration; import org.gradle.api.artifacts.Configuration;
import org.gradle.api.file.FileCollection; import org.gradle.api.file.FileCollection;
import org.gradle.api.tasks.Input; import org.gradle.api.tasks.Input;
@ -66,6 +67,17 @@ public class ThirdPartyAuditTask extends DefaultTask {
private String javaHome; private String javaHome;
private JavaVersion targetCompatibility;
@Input
public JavaVersion getTargetCompatibility() {
return targetCompatibility;
}
public void setTargetCompatibility(JavaVersion targetCompatibility) {
this.targetCompatibility = targetCompatibility;
}
@InputFiles @InputFiles
public Configuration getForbiddenAPIsConfiguration() { public Configuration getForbiddenAPIsConfiguration() {
return getProject().getConfigurations().getByName("forbiddenApisCliJar"); return getProject().getConfigurations().getByName("forbiddenApisCliJar");
@ -157,10 +169,19 @@ public class ThirdPartyAuditTask extends DefaultTask {
private void extractJars(FileCollection jars) { private void extractJars(FileCollection jars) {
File jarExpandDir = getJarExpandDir(); File jarExpandDir = getJarExpandDir();
// We need to clean up to make sure old dependencies don't linger
getProject().delete(jarExpandDir);
jars.forEach(jar -> jars.forEach(jar ->
getProject().copy(spec -> { getProject().copy(spec -> {
spec.from(getProject().zipTree(jar)); spec.from(getProject().zipTree(jar));
spec.into(jarExpandDir); spec.into(jarExpandDir);
// Exclude classes for multi release jars above target
for (int i = Integer.parseInt(targetCompatibility.getMajorVersion()) + 1;
i <= Integer.parseInt(JavaVersion.VERSION_HIGHER.getMajorVersion());
i++
) {
spec.exclude("META-INF/versions/" + i + "/**");
}
}) })
); );
} }

View File

@ -24,13 +24,14 @@ import org.apache.http.client.methods.HttpGet;
import org.apache.http.client.methods.HttpPost; import org.apache.http.client.methods.HttpPost;
import org.apache.http.client.methods.HttpPut; import org.apache.http.client.methods.HttpPut;
import org.elasticsearch.client.RequestConverters.EndpointBuilder; import org.elasticsearch.client.RequestConverters.EndpointBuilder;
import org.elasticsearch.client.ml.CloseJobRequest;
import org.elasticsearch.client.ml.DeleteJobRequest;
import org.elasticsearch.client.ml.GetBucketsRequest;
import org.elasticsearch.client.ml.GetJobRequest;
import org.elasticsearch.client.ml.GetRecordsRequest;
import org.elasticsearch.client.ml.OpenJobRequest;
import org.elasticsearch.client.ml.PutJobRequest;
import org.elasticsearch.common.Strings; import org.elasticsearch.common.Strings;
import org.elasticsearch.protocol.xpack.ml.CloseJobRequest;
import org.elasticsearch.protocol.xpack.ml.DeleteJobRequest;
import org.elasticsearch.protocol.xpack.ml.GetJobRequest;
import org.elasticsearch.protocol.xpack.ml.GetBucketsRequest;
import org.elasticsearch.protocol.xpack.ml.OpenJobRequest;
import org.elasticsearch.protocol.xpack.ml.PutJobRequest;
import java.io.IOException; import java.io.IOException;
@ -124,4 +125,18 @@ final class MLRequestConverters {
request.setEntity(createEntity(getBucketsRequest, REQUEST_BODY_CONTENT_TYPE)); request.setEntity(createEntity(getBucketsRequest, REQUEST_BODY_CONTENT_TYPE));
return request; return request;
} }
static Request getRecords(GetRecordsRequest getRecordsRequest) throws IOException {
String endpoint = new EndpointBuilder()
.addPathPartAsIs("_xpack")
.addPathPartAsIs("ml")
.addPathPartAsIs("anomaly_detectors")
.addPathPart(getRecordsRequest.getJobId())
.addPathPartAsIs("results")
.addPathPartAsIs("records")
.build();
Request request = new Request(HttpGet.METHOD_NAME, endpoint);
request.setEntity(createEntity(getRecordsRequest, REQUEST_BODY_CONTENT_TYPE));
return request;
}
} }

View File

@ -19,18 +19,20 @@
package org.elasticsearch.client; package org.elasticsearch.client;
import org.elasticsearch.action.ActionListener; import org.elasticsearch.action.ActionListener;
import org.elasticsearch.protocol.xpack.ml.CloseJobRequest; import org.elasticsearch.client.ml.CloseJobRequest;
import org.elasticsearch.protocol.xpack.ml.CloseJobResponse; import org.elasticsearch.client.ml.CloseJobResponse;
import org.elasticsearch.protocol.xpack.ml.DeleteJobRequest; import org.elasticsearch.client.ml.DeleteJobRequest;
import org.elasticsearch.protocol.xpack.ml.DeleteJobResponse; import org.elasticsearch.client.ml.DeleteJobResponse;
import org.elasticsearch.protocol.xpack.ml.GetBucketsRequest; import org.elasticsearch.client.ml.GetBucketsRequest;
import org.elasticsearch.protocol.xpack.ml.GetBucketsResponse; import org.elasticsearch.client.ml.GetBucketsResponse;
import org.elasticsearch.protocol.xpack.ml.GetJobRequest; import org.elasticsearch.client.ml.GetJobRequest;
import org.elasticsearch.protocol.xpack.ml.GetJobResponse; import org.elasticsearch.client.ml.GetJobResponse;
import org.elasticsearch.protocol.xpack.ml.OpenJobRequest; import org.elasticsearch.client.ml.GetRecordsRequest;
import org.elasticsearch.protocol.xpack.ml.OpenJobResponse; import org.elasticsearch.client.ml.GetRecordsResponse;
import org.elasticsearch.protocol.xpack.ml.PutJobRequest; import org.elasticsearch.client.ml.OpenJobRequest;
import org.elasticsearch.protocol.xpack.ml.PutJobResponse; import org.elasticsearch.client.ml.OpenJobResponse;
import org.elasticsearch.client.ml.PutJobRequest;
import org.elasticsearch.client.ml.PutJobResponse;
import java.io.IOException; import java.io.IOException;
import java.util.Collections; import java.util.Collections;
@ -56,9 +58,9 @@ public final class MachineLearningClient {
* For additional info * For additional info
* see <a href="https://www.elastic.co/guide/en/elasticsearch/reference/current/ml-put-job.html">ML PUT job documentation</a> * see <a href="https://www.elastic.co/guide/en/elasticsearch/reference/current/ml-put-job.html">ML PUT job documentation</a>
* *
* @param request The PutJobRequest containing the {@link org.elasticsearch.protocol.xpack.ml.job.config.Job} settings * @param request The PutJobRequest containing the {@link org.elasticsearch.client.ml.job.config.Job} settings
* @param options Additional request options (e.g. headers), use {@link RequestOptions#DEFAULT} if nothing needs to be customized * @param options Additional request options (e.g. headers), use {@link RequestOptions#DEFAULT} if nothing needs to be customized
* @return PutJobResponse with enclosed {@link org.elasticsearch.protocol.xpack.ml.job.config.Job} object * @return PutJobResponse with enclosed {@link org.elasticsearch.client.ml.job.config.Job} object
* @throws IOException when there is a serialization issue sending the request or receiving the response * @throws IOException when there is a serialization issue sending the request or receiving the response
*/ */
public PutJobResponse putJob(PutJobRequest request, RequestOptions options) throws IOException { public PutJobResponse putJob(PutJobRequest request, RequestOptions options) throws IOException {
@ -75,7 +77,7 @@ public final class MachineLearningClient {
* For additional info * For additional info
* see <a href="https://www.elastic.co/guide/en/elasticsearch/reference/current/ml-put-job.html">ML PUT job documentation</a> * see <a href="https://www.elastic.co/guide/en/elasticsearch/reference/current/ml-put-job.html">ML PUT job documentation</a>
* *
* @param request The request containing the {@link org.elasticsearch.protocol.xpack.ml.job.config.Job} settings * @param request The request containing the {@link org.elasticsearch.client.ml.job.config.Job} settings
* @param options Additional request options (e.g. headers), use {@link RequestOptions#DEFAULT} if nothing needs to be customized * @param options Additional request options (e.g. headers), use {@link RequestOptions#DEFAULT} if nothing needs to be customized
* @param listener Listener to be notified upon request completion * @param listener Listener to be notified upon request completion
*/ */
@ -98,7 +100,7 @@ public final class MachineLearningClient {
* @param request {@link GetJobRequest} Request containing a list of jobId(s) and additional options * @param request {@link GetJobRequest} Request containing a list of jobId(s) and additional options
* @param options Additional request options (e.g. headers), use {@link RequestOptions#DEFAULT} if nothing needs to be customized * @param options Additional request options (e.g. headers), use {@link RequestOptions#DEFAULT} if nothing needs to be customized
* @return {@link GetJobResponse} response object containing * @return {@link GetJobResponse} response object containing
* the {@link org.elasticsearch.protocol.xpack.ml.job.config.Job} objects and the number of jobs found * the {@link org.elasticsearch.client.ml.job.config.Job} objects and the number of jobs found
* @throws IOException when there is a serialization issue sending the request or receiving the response * @throws IOException when there is a serialization issue sending the request or receiving the response
*/ */
public GetJobResponse getJob(GetJobRequest request, RequestOptions options) throws IOException { public GetJobResponse getJob(GetJobRequest request, RequestOptions options) throws IOException {
@ -285,4 +287,40 @@ public final class MachineLearningClient {
listener, listener,
Collections.emptySet()); Collections.emptySet());
} }
/**
* Gets the records for a Machine Learning Job.
* <p>
* For additional info
* see <a href="https://www.elastic.co/guide/en/elasticsearch/reference/current/ml-get-record.html">ML GET records documentation</a>
*
* @param request the request
* @param options Additional request options (e.g. headers), use {@link RequestOptions#DEFAULT} if nothing needs to be customized
*/
public GetRecordsResponse getRecords(GetRecordsRequest request, RequestOptions options) throws IOException {
return restHighLevelClient.performRequestAndParseEntity(request,
MLRequestConverters::getRecords,
options,
GetRecordsResponse::fromXContent,
Collections.emptySet());
}
/**
* Gets the records for a Machine Learning Job, notifies listener once the requested records are retrieved.
* <p>
* For additional info
* see <a href="https://www.elastic.co/guide/en/elasticsearch/reference/current/ml-get-record.html">ML GET records documentation</a>
*
* @param request the request
* @param options Additional request options (e.g. headers), use {@link RequestOptions#DEFAULT} if nothing needs to be customized
* @param listener Listener to be notified upon request completion
*/
public void getRecordsAsync(GetRecordsRequest request, RequestOptions options, ActionListener<GetRecordsResponse> listener) {
restHighLevelClient.performRequestAsyncAndParseEntity(request,
MLRequestConverters::getRecords,
options,
GetRecordsResponse::fromXContent,
listener,
Collections.emptySet());
}
} }

View File

@ -106,6 +106,7 @@ import org.elasticsearch.common.xcontent.XContentParser;
import org.elasticsearch.common.xcontent.XContentType; import org.elasticsearch.common.xcontent.XContentType;
import org.elasticsearch.index.VersionType; import org.elasticsearch.index.VersionType;
import org.elasticsearch.index.rankeval.RankEvalRequest; import org.elasticsearch.index.rankeval.RankEvalRequest;
import org.elasticsearch.index.reindex.ReindexRequest;
import org.elasticsearch.protocol.xpack.XPackInfoRequest; import org.elasticsearch.protocol.xpack.XPackInfoRequest;
import org.elasticsearch.protocol.xpack.XPackUsageRequest; import org.elasticsearch.protocol.xpack.XPackUsageRequest;
import org.elasticsearch.protocol.xpack.license.DeleteLicenseRequest; import org.elasticsearch.protocol.xpack.license.DeleteLicenseRequest;
@ -820,6 +821,21 @@ final class RequestConverters {
return request; return request;
} }
static Request reindex(ReindexRequest reindexRequest) throws IOException {
String endpoint = new EndpointBuilder().addPathPart("_reindex").build();
Request request = new Request(HttpPost.METHOD_NAME, endpoint);
Params params = new Params(request)
.withRefresh(reindexRequest.isRefresh())
.withTimeout(reindexRequest.getTimeout())
.withWaitForActiveShards(reindexRequest.getWaitForActiveShards());
if (reindexRequest.getScrollTime() != null) {
params.putParam("scroll", reindexRequest.getScrollTime());
}
request.setEntity(createEntity(reindexRequest, REQUEST_BODY_CONTENT_TYPE));
return request;
}
static Request rollover(RolloverRequest rolloverRequest) throws IOException { static Request rollover(RolloverRequest rolloverRequest) throws IOException {
String endpoint = new EndpointBuilder().addPathPart(rolloverRequest.getAlias()).addPathPartAsIs("_rollover") String endpoint = new EndpointBuilder().addPathPart(rolloverRequest.getAlias()).addPathPartAsIs("_rollover")
.addPathPart(rolloverRequest.getNewIndexName()).build(); .addPathPart(rolloverRequest.getNewIndexName()).build();

View File

@ -64,6 +64,8 @@ import org.elasticsearch.common.xcontent.XContentParser;
import org.elasticsearch.common.xcontent.XContentType; import org.elasticsearch.common.xcontent.XContentType;
import org.elasticsearch.index.rankeval.RankEvalRequest; import org.elasticsearch.index.rankeval.RankEvalRequest;
import org.elasticsearch.index.rankeval.RankEvalResponse; import org.elasticsearch.index.rankeval.RankEvalResponse;
import org.elasticsearch.index.reindex.BulkByScrollResponse;
import org.elasticsearch.index.reindex.ReindexRequest;
import org.elasticsearch.plugins.spi.NamedXContentProvider; import org.elasticsearch.plugins.spi.NamedXContentProvider;
import org.elasticsearch.rest.BytesRestResponse; import org.elasticsearch.rest.BytesRestResponse;
import org.elasticsearch.rest.RestStatus; import org.elasticsearch.rest.RestStatus;
@ -395,6 +397,33 @@ public class RestHighLevelClient implements Closeable {
performRequestAsyncAndParseEntity(bulkRequest, RequestConverters::bulk, options, BulkResponse::fromXContent, listener, emptySet()); performRequestAsyncAndParseEntity(bulkRequest, RequestConverters::bulk, options, BulkResponse::fromXContent, listener, emptySet());
} }
/**
* Executes a reindex request.
* See <a href="https://www.elastic.co/guide/en/elasticsearch/reference/current/docs-reindex.html">Reindex API on elastic.co</a>
* @param reindexRequest the request
* @param options the request options (e.g. headers), use {@link RequestOptions#DEFAULT} if nothing needs to be customized
* @return the response
* @throws IOException in case there is a problem sending the request or parsing back the response
*/
public final BulkByScrollResponse reindex(ReindexRequest reindexRequest, RequestOptions options) throws IOException {
return performRequestAndParseEntity(
reindexRequest, RequestConverters::reindex, options, BulkByScrollResponse::fromXContent, emptySet()
);
}
/**
* Asynchronously executes a reindex request.
* See <a href="https://www.elastic.co/guide/en/elasticsearch/reference/current/docs-reindex.html">Reindex API on elastic.co</a>
* @param reindexRequest the request
* @param options the request options (e.g. headers), use {@link RequestOptions#DEFAULT} if nothing needs to be customized
* @param listener the listener to be notified upon request completion
*/
public final void reindexAsync(ReindexRequest reindexRequest, RequestOptions options, ActionListener<BulkByScrollResponse> listener) {
performRequestAsyncAndParseEntity(
reindexRequest, RequestConverters::reindex, options, BulkByScrollResponse::fromXContent, listener, emptySet()
);
}
/** /**
* Pings the remote Elasticsearch cluster and returns true if the ping succeeded, false otherwise * Pings the remote Elasticsearch cluster and returns true if the ping succeeded, false otherwise
* @param options the request options (e.g. headers), use {@link RequestOptions#DEFAULT} if nothing needs to be customized * @param options the request options (e.g. headers), use {@link RequestOptions#DEFAULT} if nothing needs to be customized

View File

@ -0,0 +1,56 @@
/*
* Licensed to Elasticsearch under one or more contributor
* license agreements. See the NOTICE file distributed with
* this work for additional information regarding copyright
* ownership. Elasticsearch licenses this file to you under
* the Apache License, Version 2.0 (the "License"); you may
* not use this file except in compliance with the License.
* You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing,
* software distributed under the License is distributed on an
* "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
* KIND, either express or implied. See the License for the
* specific language governing permissions and limitations
* under the License.
*/
package org.elasticsearch.client;
import org.elasticsearch.common.unit.TimeValue;
/**
* A base request for any requests that supply timeouts.
*
* Please note, any requests that use a ackTimeout should set timeout as they
* represent the same backing field on the server.
*/
public class TimedRequest implements Validatable {
private TimeValue timeout;
private TimeValue masterTimeout;
public void setTimeout(TimeValue timeout) {
this.timeout = timeout;
}
public void setMasterTimeout(TimeValue masterTimeout) {
this.masterTimeout = masterTimeout;
}
/**
* Returns the request timeout
*/
public TimeValue timeout() {
return timeout;
}
/**
* Returns the timeout for the request to be completed on the master node
*/
public TimeValue masterNodeTimeout() {
return masterTimeout;
}
}

View File

@ -16,7 +16,7 @@
* specific language governing permissions and limitations * specific language governing permissions and limitations
* under the License. * under the License.
*/ */
package org.elasticsearch.protocol.xpack.ml; package org.elasticsearch.client.ml;
import org.elasticsearch.action.ActionResponse; import org.elasticsearch.action.ActionResponse;
import org.elasticsearch.common.ParseField; import org.elasticsearch.common.ParseField;

View File

@ -16,7 +16,7 @@
* specific language governing permissions and limitations * specific language governing permissions and limitations
* under the License. * under the License.
*/ */
package org.elasticsearch.protocol.xpack.ml; package org.elasticsearch.client.ml;
import org.elasticsearch.action.ActionRequest; import org.elasticsearch.action.ActionRequest;
import org.elasticsearch.action.ActionRequestValidationException; import org.elasticsearch.action.ActionRequestValidationException;

View File

@ -16,7 +16,7 @@
* specific language governing permissions and limitations * specific language governing permissions and limitations
* under the License. * under the License.
*/ */
package org.elasticsearch.protocol.xpack.ml; package org.elasticsearch.client.ml;
import org.elasticsearch.action.ActionResponse; import org.elasticsearch.action.ActionResponse;
import org.elasticsearch.common.ParseField; import org.elasticsearch.common.ParseField;

View File

@ -16,7 +16,7 @@
* specific language governing permissions and limitations * specific language governing permissions and limitations
* under the License. * under the License.
*/ */
package org.elasticsearch.protocol.xpack.ml; package org.elasticsearch.client.ml;
import org.elasticsearch.action.ActionRequest; import org.elasticsearch.action.ActionRequest;
import org.elasticsearch.action.ActionRequestValidationException; import org.elasticsearch.action.ActionRequestValidationException;

View File

@ -16,7 +16,7 @@
* specific language governing permissions and limitations * specific language governing permissions and limitations
* under the License. * under the License.
*/ */
package org.elasticsearch.protocol.xpack.ml; package org.elasticsearch.client.ml;
import org.elasticsearch.action.support.master.AcknowledgedResponse; import org.elasticsearch.action.support.master.AcknowledgedResponse;
import org.elasticsearch.common.xcontent.XContentParser; import org.elasticsearch.common.xcontent.XContentParser;

View File

@ -16,17 +16,17 @@
* specific language governing permissions and limitations * specific language governing permissions and limitations
* under the License. * under the License.
*/ */
package org.elasticsearch.protocol.xpack.ml; package org.elasticsearch.client.ml;
import org.elasticsearch.action.ActionRequest; import org.elasticsearch.action.ActionRequest;
import org.elasticsearch.action.ActionRequestValidationException; import org.elasticsearch.action.ActionRequestValidationException;
import org.elasticsearch.client.ml.job.config.Job;
import org.elasticsearch.client.ml.job.results.Result;
import org.elasticsearch.client.ml.job.util.PageParams;
import org.elasticsearch.common.ParseField; import org.elasticsearch.common.ParseField;
import org.elasticsearch.common.xcontent.ObjectParser; import org.elasticsearch.common.xcontent.ObjectParser;
import org.elasticsearch.common.xcontent.ToXContentObject; import org.elasticsearch.common.xcontent.ToXContentObject;
import org.elasticsearch.common.xcontent.XContentBuilder; import org.elasticsearch.common.xcontent.XContentBuilder;
import org.elasticsearch.protocol.xpack.ml.job.config.Job;
import org.elasticsearch.protocol.xpack.ml.job.results.Result;
import org.elasticsearch.protocol.xpack.ml.job.util.PageParams;
import java.io.IOException; import java.io.IOException;
import java.util.Objects; import java.util.Objects;

View File

@ -16,12 +16,12 @@
* specific language governing permissions and limitations * specific language governing permissions and limitations
* under the License. * under the License.
*/ */
package org.elasticsearch.protocol.xpack.ml; package org.elasticsearch.client.ml;
import org.elasticsearch.client.ml.job.results.Bucket;
import org.elasticsearch.common.ParseField; import org.elasticsearch.common.ParseField;
import org.elasticsearch.common.xcontent.ConstructingObjectParser; import org.elasticsearch.common.xcontent.ConstructingObjectParser;
import org.elasticsearch.common.xcontent.XContentParser; import org.elasticsearch.common.xcontent.XContentParser;
import org.elasticsearch.protocol.xpack.ml.job.results.Bucket;
import java.io.IOException; import java.io.IOException;
import java.util.List; import java.util.List;

View File

@ -16,10 +16,11 @@
* specific language governing permissions and limitations * specific language governing permissions and limitations
* under the License. * under the License.
*/ */
package org.elasticsearch.protocol.xpack.ml; package org.elasticsearch.client.ml;
import org.elasticsearch.action.ActionRequest; import org.elasticsearch.action.ActionRequest;
import org.elasticsearch.action.ActionRequestValidationException; import org.elasticsearch.action.ActionRequestValidationException;
import org.elasticsearch.client.ml.job.config.Job;
import org.elasticsearch.common.ParseField; import org.elasticsearch.common.ParseField;
import org.elasticsearch.common.xcontent.ConstructingObjectParser; import org.elasticsearch.common.xcontent.ConstructingObjectParser;
import org.elasticsearch.common.xcontent.ToXContentObject; import org.elasticsearch.common.xcontent.ToXContentObject;
@ -32,7 +33,7 @@ import java.util.List;
import java.util.Objects; import java.util.Objects;
/** /**
* Request object to get {@link org.elasticsearch.protocol.xpack.ml.job.config.Job} objects with the matching `jobId`s or * Request object to get {@link Job} objects with the matching `jobId`s or
* `groupName`s. * `groupName`s.
* *
* `_all` explicitly gets all the jobs in the cluster * `_all` explicitly gets all the jobs in the cluster
@ -66,7 +67,7 @@ public class GetJobRequest extends ActionRequest implements ToXContentObject {
} }
/** /**
* Get the specified {@link org.elasticsearch.protocol.xpack.ml.job.config.Job} configurations via their unique jobIds * Get the specified {@link Job} configurations via their unique jobIds
* @param jobIds must not contain any null values * @param jobIds must not contain any null values
*/ */
public GetJobRequest(String... jobIds) { public GetJobRequest(String... jobIds) {

View File

@ -16,13 +16,13 @@
* specific language governing permissions and limitations * specific language governing permissions and limitations
* under the License. * under the License.
*/ */
package org.elasticsearch.protocol.xpack.ml; package org.elasticsearch.client.ml;
import org.elasticsearch.client.ml.job.config.Job;
import org.elasticsearch.common.ParseField; import org.elasticsearch.common.ParseField;
import org.elasticsearch.common.Strings; import org.elasticsearch.common.Strings;
import org.elasticsearch.common.xcontent.ConstructingObjectParser; import org.elasticsearch.common.xcontent.ConstructingObjectParser;
import org.elasticsearch.common.xcontent.XContentParser; import org.elasticsearch.common.xcontent.XContentParser;
import org.elasticsearch.protocol.xpack.ml.job.config.Job;
import java.io.IOException; import java.io.IOException;
import java.util.List; import java.util.List;

View File

@ -0,0 +1,222 @@
/*
* Licensed to Elasticsearch under one or more contributor
* license agreements. See the NOTICE file distributed with
* this work for additional information regarding copyright
* ownership. Elasticsearch licenses this file to you under
* the Apache License, Version 2.0 (the "License"); you may
* not use this file except in compliance with the License.
* You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing,
* software distributed under the License is distributed on an
* "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
* KIND, either express or implied. See the License for the
* specific language governing permissions and limitations
* under the License.
*/
package org.elasticsearch.client.ml;
import org.elasticsearch.client.Validatable;
import org.elasticsearch.client.ml.job.config.Job;
import org.elasticsearch.client.ml.job.util.PageParams;
import org.elasticsearch.common.ParseField;
import org.elasticsearch.common.xcontent.ObjectParser;
import org.elasticsearch.common.xcontent.ToXContentObject;
import org.elasticsearch.common.xcontent.XContentBuilder;
import java.io.IOException;
import java.util.Objects;
/**
* A request to retrieve records of a given job
*/
public class GetRecordsRequest implements ToXContentObject, Validatable {
public static final ParseField EXCLUDE_INTERIM = new ParseField("exclude_interim");
public static final ParseField START = new ParseField("start");
public static final ParseField END = new ParseField("end");
public static final ParseField RECORD_SCORE = new ParseField("record_score");
public static final ParseField SORT = new ParseField("sort");
public static final ParseField DESCENDING = new ParseField("desc");
public static final ObjectParser<GetRecordsRequest, Void> PARSER = new ObjectParser<>("get_buckets_request", GetRecordsRequest::new);
static {
PARSER.declareString((request, jobId) -> request.jobId = jobId, Job.ID);
PARSER.declareBoolean(GetRecordsRequest::setExcludeInterim, EXCLUDE_INTERIM);
PARSER.declareStringOrNull(GetRecordsRequest::setStart, START);
PARSER.declareStringOrNull(GetRecordsRequest::setEnd, END);
PARSER.declareObject(GetRecordsRequest::setPageParams, PageParams.PARSER, PageParams.PAGE);
PARSER.declareDouble(GetRecordsRequest::setRecordScore, RECORD_SCORE);
PARSER.declareString(GetRecordsRequest::setSort, SORT);
PARSER.declareBoolean(GetRecordsRequest::setDescending, DESCENDING);
}
private String jobId;
private Boolean excludeInterim;
private String start;
private String end;
private PageParams pageParams;
private Double recordScore;
private String sort;
private Boolean descending;
private GetRecordsRequest() {}
/**
* Constructs a request to retrieve records of a given job
* @param jobId id of the job to retrieve records of
*/
public GetRecordsRequest(String jobId) {
this.jobId = Objects.requireNonNull(jobId);
}
public String getJobId() {
return jobId;
}
public boolean isExcludeInterim() {
return excludeInterim;
}
/**
* Sets the value of "exclude_interim".
* When {@code true}, interim records will be filtered out.
* @param excludeInterim value of "exclude_interim" to be set
*/
public void setExcludeInterim(boolean excludeInterim) {
this.excludeInterim = excludeInterim;
}
public String getStart() {
return start;
}
/**
* Sets the value of "start" which is a timestamp.
* Only records whose timestamp is on or after the "start" value will be returned.
* @param start value of "start" to be set
*/
public void setStart(String start) {
this.start = start;
}
public String getEnd() {
return end;
}
/**
* Sets the value of "end" which is a timestamp.
* Only records whose timestamp is before the "end" value will be returned.
* @param end value of "end" to be set
*/
public void setEnd(String end) {
this.end = end;
}
public PageParams getPageParams() {
return pageParams;
}
/**
* Sets the paging parameters
* @param pageParams The paging parameters
*/
public void setPageParams(PageParams pageParams) {
this.pageParams = pageParams;
}
public Double getRecordScore() {
return recordScore;
}
/**
* Sets the value of "record_score".
* Only records with "record_score" equal or greater will be returned.
* @param recordScore value of "record_score".
*/
public void setRecordScore(double recordScore) {
this.recordScore = recordScore;
}
public String getSort() {
return sort;
}
/**
* Sets the value of "sort".
* Specifies the bucket field to sort on.
* @param sort value of "sort".
*/
public void setSort(String sort) {
this.sort = sort;
}
public boolean isDescending() {
return descending;
}
/**
* Sets the value of "desc".
* Specifies the sorting order.
* @param descending value of "desc"
*/
public void setDescending(boolean descending) {
this.descending = descending;
}
@Override
public XContentBuilder toXContent(XContentBuilder builder, Params params) throws IOException {
builder.startObject();
builder.field(Job.ID.getPreferredName(), jobId);
if (excludeInterim != null) {
builder.field(EXCLUDE_INTERIM.getPreferredName(), excludeInterim);
}
if (start != null) {
builder.field(START.getPreferredName(), start);
}
if (end != null) {
builder.field(END.getPreferredName(), end);
}
if (pageParams != null) {
builder.field(PageParams.PAGE.getPreferredName(), pageParams);
}
if (recordScore != null) {
builder.field(RECORD_SCORE.getPreferredName(), recordScore);
}
if (sort != null) {
builder.field(SORT.getPreferredName(), sort);
}
if (descending != null) {
builder.field(DESCENDING.getPreferredName(), descending);
}
builder.endObject();
return builder;
}
@Override
public int hashCode() {
return Objects.hash(jobId, excludeInterim, recordScore, pageParams, start, end, sort, descending);
}
@Override
public boolean equals(Object obj) {
if (obj == null) {
return false;
}
if (getClass() != obj.getClass()) {
return false;
}
GetRecordsRequest other = (GetRecordsRequest) obj;
return Objects.equals(jobId, other.jobId) &&
Objects.equals(excludeInterim, other.excludeInterim) &&
Objects.equals(recordScore, other.recordScore) &&
Objects.equals(pageParams, other.pageParams) &&
Objects.equals(start, other.start) &&
Objects.equals(end, other.end) &&
Objects.equals(sort, other.sort) &&
Objects.equals(descending, other.descending);
}
}

View File

@ -0,0 +1,78 @@
/*
* Licensed to Elasticsearch under one or more contributor
* license agreements. See the NOTICE file distributed with
* this work for additional information regarding copyright
* ownership. Elasticsearch licenses this file to you under
* the Apache License, Version 2.0 (the "License"); you may
* not use this file except in compliance with the License.
* You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing,
* software distributed under the License is distributed on an
* "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
* KIND, either express or implied. See the License for the
* specific language governing permissions and limitations
* under the License.
*/
package org.elasticsearch.client.ml;
import org.elasticsearch.client.ml.job.results.AnomalyRecord;
import org.elasticsearch.common.ParseField;
import org.elasticsearch.common.xcontent.ConstructingObjectParser;
import org.elasticsearch.common.xcontent.XContentParser;
import java.io.IOException;
import java.util.List;
import java.util.Objects;
/**
* A response containing the requested buckets
*/
public class GetRecordsResponse extends AbstractResultResponse<AnomalyRecord> {
public static final ParseField RECORDS = new ParseField("records");
@SuppressWarnings("unchecked")
public static final ConstructingObjectParser<GetRecordsResponse, Void> PARSER = new ConstructingObjectParser<>("get_records_response",
true, a -> new GetRecordsResponse((List<AnomalyRecord>) a[0], (long) a[1]));
static {
PARSER.declareObjectArray(ConstructingObjectParser.constructorArg(), AnomalyRecord.PARSER, RECORDS);
PARSER.declareLong(ConstructingObjectParser.constructorArg(), COUNT);
}
public static GetRecordsResponse fromXContent(XContentParser parser) throws IOException {
return PARSER.parse(parser, null);
}
GetRecordsResponse(List<AnomalyRecord> buckets, long count) {
super(RECORDS, buckets, count);
}
/**
* The retrieved records
* @return the retrieved records
*/
public List<AnomalyRecord> records() {
return results;
}
@Override
public int hashCode() {
return Objects.hash(count, results);
}
@Override
public boolean equals(Object obj) {
if (obj == null) {
return false;
}
if (getClass() != obj.getClass()) {
return false;
}
GetRecordsResponse other = (GetRecordsResponse) obj;
return count == other.count && Objects.equals(results, other.results);
}
}

View File

@ -16,19 +16,18 @@
* specific language governing permissions and limitations * specific language governing permissions and limitations
* under the License. * under the License.
*/ */
package org.elasticsearch.protocol.xpack.ml; package org.elasticsearch.client.ml;
import org.elasticsearch.action.ActionRequest; import org.elasticsearch.action.ActionRequest;
import org.elasticsearch.action.ActionRequestValidationException; import org.elasticsearch.action.ActionRequestValidationException;
import org.elasticsearch.client.ml.job.config.Job;
import org.elasticsearch.common.ParseField; import org.elasticsearch.common.ParseField;
import org.elasticsearch.common.Strings; import org.elasticsearch.common.Strings;
import org.elasticsearch.common.unit.TimeValue; import org.elasticsearch.common.unit.TimeValue;
import org.elasticsearch.common.xcontent.ConstructingObjectParser; import org.elasticsearch.common.xcontent.ConstructingObjectParser;
import org.elasticsearch.common.xcontent.ToXContent;
import org.elasticsearch.common.xcontent.ToXContentObject; import org.elasticsearch.common.xcontent.ToXContentObject;
import org.elasticsearch.common.xcontent.XContentBuilder; import org.elasticsearch.common.xcontent.XContentBuilder;
import org.elasticsearch.common.xcontent.XContentParser; import org.elasticsearch.common.xcontent.XContentParser;
import org.elasticsearch.protocol.xpack.ml.job.config.Job;
import java.io.IOException; import java.io.IOException;
import java.util.Objects; import java.util.Objects;
@ -95,7 +94,7 @@ public class OpenJobRequest extends ActionRequest implements ToXContentObject {
} }
@Override @Override
public XContentBuilder toXContent(XContentBuilder builder, ToXContent.Params params) throws IOException { public XContentBuilder toXContent(XContentBuilder builder, Params params) throws IOException {
builder.startObject(); builder.startObject();
builder.field(Job.ID.getPreferredName(), jobId); builder.field(Job.ID.getPreferredName(), jobId);
if (timeout != null) { if (timeout != null) {

View File

@ -16,7 +16,7 @@
* specific language governing permissions and limitations * specific language governing permissions and limitations
* under the License. * under the License.
*/ */
package org.elasticsearch.protocol.xpack.ml; package org.elasticsearch.client.ml;
import org.elasticsearch.action.ActionResponse; import org.elasticsearch.action.ActionResponse;
import org.elasticsearch.common.ParseField; import org.elasticsearch.common.ParseField;

View File

@ -16,14 +16,14 @@
* specific language governing permissions and limitations * specific language governing permissions and limitations
* under the License. * under the License.
*/ */
package org.elasticsearch.protocol.xpack.ml; package org.elasticsearch.client.ml;
import org.elasticsearch.action.ActionRequest; import org.elasticsearch.action.ActionRequest;
import org.elasticsearch.action.ActionRequestValidationException; import org.elasticsearch.action.ActionRequestValidationException;
import org.elasticsearch.client.ml.job.config.Job;
import org.elasticsearch.common.Strings; import org.elasticsearch.common.Strings;
import org.elasticsearch.common.xcontent.ToXContentObject; import org.elasticsearch.common.xcontent.ToXContentObject;
import org.elasticsearch.common.xcontent.XContentBuilder; import org.elasticsearch.common.xcontent.XContentBuilder;
import org.elasticsearch.protocol.xpack.ml.job.config.Job;
import java.io.IOException; import java.io.IOException;
import java.util.Objects; import java.util.Objects;

View File

@ -16,13 +16,12 @@
* specific language governing permissions and limitations * specific language governing permissions and limitations
* under the License. * under the License.
*/ */
package org.elasticsearch.protocol.xpack.ml; package org.elasticsearch.client.ml;
import org.elasticsearch.common.xcontent.ToXContent; import org.elasticsearch.client.ml.job.config.Job;
import org.elasticsearch.common.xcontent.ToXContentObject; import org.elasticsearch.common.xcontent.ToXContentObject;
import org.elasticsearch.common.xcontent.XContentBuilder; import org.elasticsearch.common.xcontent.XContentBuilder;
import org.elasticsearch.common.xcontent.XContentParser; import org.elasticsearch.common.xcontent.XContentParser;
import org.elasticsearch.protocol.xpack.ml.job.config.Job;
import java.io.IOException; import java.io.IOException;
import java.util.Objects; import java.util.Objects;
@ -47,7 +46,7 @@ public class PutJobResponse implements ToXContentObject {
} }
@Override @Override
public XContentBuilder toXContent(XContentBuilder builder, ToXContent.Params params) throws IOException { public XContentBuilder toXContent(XContentBuilder builder, Params params) throws IOException {
job.toXContent(builder, params); job.toXContent(builder, params);
return builder; return builder;
} }

View File

@ -16,7 +16,7 @@
* specific language governing permissions and limitations * specific language governing permissions and limitations
* under the License. * under the License.
*/ */
package org.elasticsearch.protocol.xpack.ml.datafeed; package org.elasticsearch.client.ml.datafeed;
import org.elasticsearch.common.Nullable; import org.elasticsearch.common.Nullable;
import org.elasticsearch.common.ParseField; import org.elasticsearch.common.ParseField;

View File

@ -16,8 +16,9 @@
* specific language governing permissions and limitations * specific language governing permissions and limitations
* under the License. * under the License.
*/ */
package org.elasticsearch.protocol.xpack.ml.datafeed; package org.elasticsearch.client.ml.datafeed;
import org.elasticsearch.client.ml.job.config.Job;
import org.elasticsearch.common.ParseField; import org.elasticsearch.common.ParseField;
import org.elasticsearch.common.unit.TimeValue; import org.elasticsearch.common.unit.TimeValue;
import org.elasticsearch.common.xcontent.ConstructingObjectParser; import org.elasticsearch.common.xcontent.ConstructingObjectParser;
@ -27,7 +28,6 @@ import org.elasticsearch.common.xcontent.XContentParser;
import org.elasticsearch.index.query.AbstractQueryBuilder; import org.elasticsearch.index.query.AbstractQueryBuilder;
import org.elasticsearch.index.query.QueryBuilder; import org.elasticsearch.index.query.QueryBuilder;
import org.elasticsearch.index.query.QueryBuilders; import org.elasticsearch.index.query.QueryBuilders;
import org.elasticsearch.protocol.xpack.ml.job.config.Job;
import org.elasticsearch.search.aggregations.AggregatorFactories; import org.elasticsearch.search.aggregations.AggregatorFactories;
import org.elasticsearch.search.builder.SearchSourceBuilder; import org.elasticsearch.search.builder.SearchSourceBuilder;

View File

@ -16,8 +16,9 @@
* specific language governing permissions and limitations * specific language governing permissions and limitations
* under the License. * under the License.
*/ */
package org.elasticsearch.protocol.xpack.ml.datafeed; package org.elasticsearch.client.ml.datafeed;
import org.elasticsearch.client.ml.job.config.Job;
import org.elasticsearch.common.ParseField; import org.elasticsearch.common.ParseField;
import org.elasticsearch.common.unit.TimeValue; import org.elasticsearch.common.unit.TimeValue;
import org.elasticsearch.common.xcontent.ConstructingObjectParser; import org.elasticsearch.common.xcontent.ConstructingObjectParser;
@ -26,7 +27,6 @@ import org.elasticsearch.common.xcontent.XContentBuilder;
import org.elasticsearch.common.xcontent.XContentParser; import org.elasticsearch.common.xcontent.XContentParser;
import org.elasticsearch.index.query.AbstractQueryBuilder; import org.elasticsearch.index.query.AbstractQueryBuilder;
import org.elasticsearch.index.query.QueryBuilder; import org.elasticsearch.index.query.QueryBuilder;
import org.elasticsearch.protocol.xpack.ml.job.config.Job;
import org.elasticsearch.search.aggregations.AggregatorFactories; import org.elasticsearch.search.aggregations.AggregatorFactories;
import org.elasticsearch.search.builder.SearchSourceBuilder; import org.elasticsearch.search.builder.SearchSourceBuilder;

View File

@ -16,7 +16,7 @@
* specific language governing permissions and limitations * specific language governing permissions and limitations
* under the License. * under the License.
*/ */
package org.elasticsearch.protocol.xpack.ml.job.config; package org.elasticsearch.client.ml.job.config;
import org.elasticsearch.common.ParseField; import org.elasticsearch.common.ParseField;
import org.elasticsearch.common.unit.TimeValue; import org.elasticsearch.common.unit.TimeValue;

View File

@ -16,7 +16,7 @@
* specific language governing permissions and limitations * specific language governing permissions and limitations
* under the License. * under the License.
*/ */
package org.elasticsearch.protocol.xpack.ml.job.config; package org.elasticsearch.client.ml.job.config;
import org.elasticsearch.common.Nullable; import org.elasticsearch.common.Nullable;
import org.elasticsearch.common.ParseField; import org.elasticsearch.common.ParseField;

View File

@ -16,7 +16,7 @@
* specific language governing permissions and limitations * specific language governing permissions and limitations
* under the License. * under the License.
*/ */
package org.elasticsearch.protocol.xpack.ml.job.config; package org.elasticsearch.client.ml.job.config;
import org.elasticsearch.common.ParseField; import org.elasticsearch.common.ParseField;
import org.elasticsearch.common.Strings; import org.elasticsearch.common.Strings;

View File

@ -16,7 +16,7 @@
* specific language governing permissions and limitations * specific language governing permissions and limitations
* under the License. * under the License.
*/ */
package org.elasticsearch.protocol.xpack.ml.job.config; package org.elasticsearch.client.ml.job.config;
import org.elasticsearch.common.ParseField; import org.elasticsearch.common.ParseField;
import org.elasticsearch.common.xcontent.ObjectParser; import org.elasticsearch.common.xcontent.ObjectParser;

View File

@ -16,7 +16,7 @@
* specific language governing permissions and limitations * specific language governing permissions and limitations
* under the License. * under the License.
*/ */
package org.elasticsearch.protocol.xpack.ml.job.config; package org.elasticsearch.client.ml.job.config;
import org.elasticsearch.common.Strings; import org.elasticsearch.common.Strings;

View File

@ -16,7 +16,7 @@
* specific language governing permissions and limitations * specific language governing permissions and limitations
* under the License. * under the License.
*/ */
package org.elasticsearch.protocol.xpack.ml.job.config; package org.elasticsearch.client.ml.job.config;
import org.elasticsearch.common.ParseField; import org.elasticsearch.common.ParseField;
import org.elasticsearch.common.xcontent.ObjectParser; import org.elasticsearch.common.xcontent.ObjectParser;

View File

@ -16,7 +16,7 @@
* specific language governing permissions and limitations * specific language governing permissions and limitations
* under the License. * under the License.
*/ */
package org.elasticsearch.protocol.xpack.ml.job.config; package org.elasticsearch.client.ml.job.config;
import org.elasticsearch.common.ParseField; import org.elasticsearch.common.ParseField;
import org.elasticsearch.common.xcontent.ObjectParser; import org.elasticsearch.common.xcontent.ObjectParser;

View File

@ -16,7 +16,7 @@
* specific language governing permissions and limitations * specific language governing permissions and limitations
* under the License. * under the License.
*/ */
package org.elasticsearch.protocol.xpack.ml.job.config; package org.elasticsearch.client.ml.job.config;
import java.util.Arrays; import java.util.Arrays;
import java.util.Collections; import java.util.Collections;

View File

@ -16,7 +16,7 @@
* specific language governing permissions and limitations * specific language governing permissions and limitations
* under the License. * under the License.
*/ */
package org.elasticsearch.protocol.xpack.ml.job.config; package org.elasticsearch.client.ml.job.config;
import org.elasticsearch.common.ParseField; import org.elasticsearch.common.ParseField;
import org.elasticsearch.common.xcontent.ConstructingObjectParser; import org.elasticsearch.common.xcontent.ConstructingObjectParser;

View File

@ -16,8 +16,9 @@
* specific language governing permissions and limitations * specific language governing permissions and limitations
* under the License. * under the License.
*/ */
package org.elasticsearch.protocol.xpack.ml.job.config; package org.elasticsearch.client.ml.job.config;
import org.elasticsearch.client.ml.job.util.TimeUtil;
import org.elasticsearch.common.ParseField; import org.elasticsearch.common.ParseField;
import org.elasticsearch.common.Strings; import org.elasticsearch.common.Strings;
import org.elasticsearch.common.unit.TimeValue; import org.elasticsearch.common.unit.TimeValue;
@ -25,7 +26,6 @@ import org.elasticsearch.common.xcontent.ObjectParser;
import org.elasticsearch.common.xcontent.ObjectParser.ValueType; import org.elasticsearch.common.xcontent.ObjectParser.ValueType;
import org.elasticsearch.common.xcontent.ToXContentObject; import org.elasticsearch.common.xcontent.ToXContentObject;
import org.elasticsearch.common.xcontent.XContentBuilder; import org.elasticsearch.common.xcontent.XContentBuilder;
import org.elasticsearch.protocol.xpack.ml.job.util.TimeUtil;
import java.io.IOException; import java.io.IOException;
import java.util.Collections; import java.util.Collections;

View File

@ -16,7 +16,7 @@
* specific language governing permissions and limitations * specific language governing permissions and limitations
* under the License. * under the License.
*/ */
package org.elasticsearch.protocol.xpack.ml.job.config; package org.elasticsearch.client.ml.job.config;
import org.elasticsearch.common.Nullable; import org.elasticsearch.common.Nullable;
import org.elasticsearch.common.ParseField; import org.elasticsearch.common.ParseField;

View File

@ -16,7 +16,7 @@
* specific language governing permissions and limitations * specific language governing permissions and limitations
* under the License. * under the License.
*/ */
package org.elasticsearch.protocol.xpack.ml.job.config; package org.elasticsearch.client.ml.job.config;
import org.elasticsearch.common.ParseField; import org.elasticsearch.common.ParseField;
import org.elasticsearch.common.xcontent.ConstructingObjectParser; import org.elasticsearch.common.xcontent.ConstructingObjectParser;

View File

@ -16,7 +16,7 @@
* specific language governing permissions and limitations * specific language governing permissions and limitations
* under the License. * under the License.
*/ */
package org.elasticsearch.protocol.xpack.ml.job.config; package org.elasticsearch.client.ml.job.config;
import org.elasticsearch.common.ParseField; import org.elasticsearch.common.ParseField;

View File

@ -16,7 +16,7 @@
* specific language governing permissions and limitations * specific language governing permissions and limitations
* under the License. * under the License.
*/ */
package org.elasticsearch.protocol.xpack.ml.job.config; package org.elasticsearch.client.ml.job.config;
import java.util.Locale; import java.util.Locale;

View File

@ -16,7 +16,7 @@
* specific language governing permissions and limitations * specific language governing permissions and limitations
* under the License. * under the License.
*/ */
package org.elasticsearch.protocol.xpack.ml.job.config; package org.elasticsearch.client.ml.job.config;
import org.elasticsearch.common.ParseField; import org.elasticsearch.common.ParseField;
import org.elasticsearch.common.xcontent.ConstructingObjectParser; import org.elasticsearch.common.xcontent.ConstructingObjectParser;

View File

@ -16,11 +16,11 @@
* specific language governing permissions and limitations * specific language governing permissions and limitations
* under the License. * under the License.
*/ */
package org.elasticsearch.protocol.xpack.ml.job.config; package org.elasticsearch.client.ml.job.config;
import org.elasticsearch.common.Strings; import org.elasticsearch.common.Strings;
import org.elasticsearch.common.xcontent.ContextParser; import org.elasticsearch.common.xcontent.ContextParser;
import org.elasticsearch.common.xcontent.LoggingDeprecationHandler; import org.elasticsearch.common.xcontent.DeprecationHandler;
import org.elasticsearch.common.xcontent.NamedXContentRegistry; import org.elasticsearch.common.xcontent.NamedXContentRegistry;
import org.elasticsearch.common.xcontent.ToXContentObject; import org.elasticsearch.common.xcontent.ToXContentObject;
import org.elasticsearch.common.xcontent.XContentBuilder; import org.elasticsearch.common.xcontent.XContentBuilder;
@ -50,7 +50,7 @@ public class RuleScope implements ToXContentObject {
Map<String, ?> value = (Map<String, ?>) entry.getValue(); Map<String, ?> value = (Map<String, ?>) entry.getValue();
builder.map(value); builder.map(value);
try (XContentParser scopeParser = XContentFactory.xContent(builder.contentType()).createParser( try (XContentParser scopeParser = XContentFactory.xContent(builder.contentType()).createParser(
NamedXContentRegistry.EMPTY, LoggingDeprecationHandler.INSTANCE, Strings.toString(builder))) { NamedXContentRegistry.EMPTY, DEPRECATION_HANDLER, Strings.toString(builder))) {
scope.put(entry.getKey(), FilterRef.PARSER.parse(scopeParser, null)); scope.put(entry.getKey(), FilterRef.PARSER.parse(scopeParser, null));
} }
} }
@ -59,6 +59,15 @@ public class RuleScope implements ToXContentObject {
}; };
} }
private static final DeprecationHandler DEPRECATION_HANDLER = new DeprecationHandler() {
@Override
public void usedDeprecatedName(String usedName, String modernName) {}
@Override
public void usedDeprecatedField(String usedName, String replacedWith) {}
};
private final Map<String, FilterRef> scope; private final Map<String, FilterRef> scope;
public RuleScope() { public RuleScope() {

View File

@ -16,15 +16,15 @@
* specific language governing permissions and limitations * specific language governing permissions and limitations
* under the License. * under the License.
*/ */
package org.elasticsearch.protocol.xpack.ml.job.process; package org.elasticsearch.client.ml.job.process;
import org.elasticsearch.client.ml.job.config.Job;
import org.elasticsearch.client.ml.job.util.TimeUtil;
import org.elasticsearch.common.ParseField; import org.elasticsearch.common.ParseField;
import org.elasticsearch.common.xcontent.ConstructingObjectParser; import org.elasticsearch.common.xcontent.ConstructingObjectParser;
import org.elasticsearch.common.xcontent.ObjectParser.ValueType; import org.elasticsearch.common.xcontent.ObjectParser.ValueType;
import org.elasticsearch.common.xcontent.ToXContentObject; import org.elasticsearch.common.xcontent.ToXContentObject;
import org.elasticsearch.common.xcontent.XContentBuilder; import org.elasticsearch.common.xcontent.XContentBuilder;
import org.elasticsearch.protocol.xpack.ml.job.config.Job;
import org.elasticsearch.protocol.xpack.ml.job.util.TimeUtil;
import java.io.IOException; import java.io.IOException;
import java.util.Date; import java.util.Date;

View File

@ -16,16 +16,16 @@
* specific language governing permissions and limitations * specific language governing permissions and limitations
* under the License. * under the License.
*/ */
package org.elasticsearch.protocol.xpack.ml.job.process; package org.elasticsearch.client.ml.job.process;
import org.elasticsearch.client.ml.job.config.Job;
import org.elasticsearch.client.ml.job.results.Result;
import org.elasticsearch.client.ml.job.util.TimeUtil;
import org.elasticsearch.common.ParseField; import org.elasticsearch.common.ParseField;
import org.elasticsearch.common.xcontent.ConstructingObjectParser; import org.elasticsearch.common.xcontent.ConstructingObjectParser;
import org.elasticsearch.common.xcontent.ObjectParser.ValueType; import org.elasticsearch.common.xcontent.ObjectParser.ValueType;
import org.elasticsearch.common.xcontent.ToXContentObject; import org.elasticsearch.common.xcontent.ToXContentObject;
import org.elasticsearch.common.xcontent.XContentBuilder; import org.elasticsearch.common.xcontent.XContentBuilder;
import org.elasticsearch.protocol.xpack.ml.job.config.Job;
import org.elasticsearch.protocol.xpack.ml.job.results.Result;
import org.elasticsearch.protocol.xpack.ml.job.util.TimeUtil;
import java.io.IOException; import java.io.IOException;
import java.util.Date; import java.util.Date;

View File

@ -16,16 +16,16 @@
* specific language governing permissions and limitations * specific language governing permissions and limitations
* under the License. * under the License.
*/ */
package org.elasticsearch.protocol.xpack.ml.job.process; package org.elasticsearch.client.ml.job.process;
import org.elasticsearch.Version; import org.elasticsearch.Version;
import org.elasticsearch.client.ml.job.config.Job;
import org.elasticsearch.client.ml.job.util.TimeUtil;
import org.elasticsearch.common.ParseField; import org.elasticsearch.common.ParseField;
import org.elasticsearch.common.xcontent.ObjectParser; import org.elasticsearch.common.xcontent.ObjectParser;
import org.elasticsearch.common.xcontent.ObjectParser.ValueType; import org.elasticsearch.common.xcontent.ObjectParser.ValueType;
import org.elasticsearch.common.xcontent.ToXContentObject; import org.elasticsearch.common.xcontent.ToXContentObject;
import org.elasticsearch.common.xcontent.XContentBuilder; import org.elasticsearch.common.xcontent.XContentBuilder;
import org.elasticsearch.protocol.xpack.ml.job.config.Job;
import org.elasticsearch.protocol.xpack.ml.job.util.TimeUtil;
import java.io.IOException; import java.io.IOException;
import java.util.Date; import java.util.Date;

View File

@ -16,14 +16,14 @@
* specific language governing permissions and limitations * specific language governing permissions and limitations
* under the License. * under the License.
*/ */
package org.elasticsearch.protocol.xpack.ml.job.process; package org.elasticsearch.client.ml.job.process;
import org.elasticsearch.client.ml.job.config.Job;
import org.elasticsearch.common.ParseField; import org.elasticsearch.common.ParseField;
import org.elasticsearch.common.xcontent.ConstructingObjectParser; import org.elasticsearch.common.xcontent.ConstructingObjectParser;
import org.elasticsearch.common.xcontent.ObjectParser.ValueType; import org.elasticsearch.common.xcontent.ObjectParser.ValueType;
import org.elasticsearch.common.xcontent.ToXContentObject; import org.elasticsearch.common.xcontent.ToXContentObject;
import org.elasticsearch.common.xcontent.XContentBuilder; import org.elasticsearch.common.xcontent.XContentBuilder;
import org.elasticsearch.protocol.xpack.ml.job.config.Job;
import java.io.IOException; import java.io.IOException;
import java.util.Date; import java.util.Date;

View File

@ -16,7 +16,7 @@
* specific language governing permissions and limitations * specific language governing permissions and limitations
* under the License. * under the License.
*/ */
package org.elasticsearch.protocol.xpack.ml.job.results; package org.elasticsearch.client.ml.job.results;
import org.elasticsearch.common.ParseField; import org.elasticsearch.common.ParseField;
import org.elasticsearch.common.xcontent.ObjectParser; import org.elasticsearch.common.xcontent.ObjectParser;

View File

@ -16,8 +16,9 @@
* specific language governing permissions and limitations * specific language governing permissions and limitations
* under the License. * under the License.
*/ */
package org.elasticsearch.protocol.xpack.ml.job.results; package org.elasticsearch.client.ml.job.results;
import org.elasticsearch.client.ml.job.config.Job;
import org.elasticsearch.common.ParseField; import org.elasticsearch.common.ParseField;
import org.elasticsearch.common.time.DateFormatters; import org.elasticsearch.common.time.DateFormatters;
import org.elasticsearch.common.xcontent.ConstructingObjectParser; import org.elasticsearch.common.xcontent.ConstructingObjectParser;
@ -25,7 +26,6 @@ import org.elasticsearch.common.xcontent.ObjectParser.ValueType;
import org.elasticsearch.common.xcontent.ToXContentObject; import org.elasticsearch.common.xcontent.ToXContentObject;
import org.elasticsearch.common.xcontent.XContentBuilder; import org.elasticsearch.common.xcontent.XContentBuilder;
import org.elasticsearch.common.xcontent.XContentParser.Token; import org.elasticsearch.common.xcontent.XContentParser.Token;
import org.elasticsearch.protocol.xpack.ml.job.config.Job;
import java.io.IOException; import java.io.IOException;
import java.time.format.DateTimeFormatter; import java.time.format.DateTimeFormatter;

View File

@ -16,8 +16,9 @@
* specific language governing permissions and limitations * specific language governing permissions and limitations
* under the License. * under the License.
*/ */
package org.elasticsearch.protocol.xpack.ml.job.results; package org.elasticsearch.client.ml.job.results;
import org.elasticsearch.client.ml.job.config.Job;
import org.elasticsearch.common.ParseField; import org.elasticsearch.common.ParseField;
import org.elasticsearch.common.time.DateFormatters; import org.elasticsearch.common.time.DateFormatters;
import org.elasticsearch.common.xcontent.ConstructingObjectParser; import org.elasticsearch.common.xcontent.ConstructingObjectParser;
@ -25,7 +26,6 @@ import org.elasticsearch.common.xcontent.ObjectParser.ValueType;
import org.elasticsearch.common.xcontent.ToXContentObject; import org.elasticsearch.common.xcontent.ToXContentObject;
import org.elasticsearch.common.xcontent.XContentBuilder; import org.elasticsearch.common.xcontent.XContentBuilder;
import org.elasticsearch.common.xcontent.XContentParser.Token; import org.elasticsearch.common.xcontent.XContentParser.Token;
import org.elasticsearch.protocol.xpack.ml.job.config.Job;
import java.io.IOException; import java.io.IOException;
import java.time.format.DateTimeFormatter; import java.time.format.DateTimeFormatter;

View File

@ -16,8 +16,9 @@
* specific language governing permissions and limitations * specific language governing permissions and limitations
* under the License. * under the License.
*/ */
package org.elasticsearch.protocol.xpack.ml.job.results; package org.elasticsearch.client.ml.job.results;
import org.elasticsearch.client.ml.job.config.Job;
import org.elasticsearch.common.ParseField; import org.elasticsearch.common.ParseField;
import org.elasticsearch.common.time.DateFormatters; import org.elasticsearch.common.time.DateFormatters;
import org.elasticsearch.common.xcontent.ConstructingObjectParser; import org.elasticsearch.common.xcontent.ConstructingObjectParser;
@ -25,7 +26,6 @@ import org.elasticsearch.common.xcontent.ObjectParser.ValueType;
import org.elasticsearch.common.xcontent.ToXContentObject; import org.elasticsearch.common.xcontent.ToXContentObject;
import org.elasticsearch.common.xcontent.XContentBuilder; import org.elasticsearch.common.xcontent.XContentBuilder;
import org.elasticsearch.common.xcontent.XContentParser.Token; import org.elasticsearch.common.xcontent.XContentParser.Token;
import org.elasticsearch.protocol.xpack.ml.job.config.Job;
import java.io.IOException; import java.io.IOException;
import java.time.format.DateTimeFormatter; import java.time.format.DateTimeFormatter;

View File

@ -16,13 +16,13 @@
* specific language governing permissions and limitations * specific language governing permissions and limitations
* under the License. * under the License.
*/ */
package org.elasticsearch.protocol.xpack.ml.job.results; package org.elasticsearch.client.ml.job.results;
import org.elasticsearch.client.ml.job.config.Job;
import org.elasticsearch.common.ParseField; import org.elasticsearch.common.ParseField;
import org.elasticsearch.common.xcontent.ConstructingObjectParser; import org.elasticsearch.common.xcontent.ConstructingObjectParser;
import org.elasticsearch.common.xcontent.ToXContentObject; import org.elasticsearch.common.xcontent.ToXContentObject;
import org.elasticsearch.common.xcontent.XContentBuilder; import org.elasticsearch.common.xcontent.XContentBuilder;
import org.elasticsearch.protocol.xpack.ml.job.config.Job;
import java.io.IOException; import java.io.IOException;
import java.util.ArrayList; import java.util.ArrayList;

View File

@ -16,7 +16,7 @@
* specific language governing permissions and limitations * specific language governing permissions and limitations
* under the License. * under the License.
*/ */
package org.elasticsearch.protocol.xpack.ml.job.results; package org.elasticsearch.client.ml.job.results;
import org.elasticsearch.common.ParseField; import org.elasticsearch.common.ParseField;
import org.elasticsearch.common.xcontent.ConstructingObjectParser; import org.elasticsearch.common.xcontent.ConstructingObjectParser;

View File

@ -16,8 +16,9 @@
* specific language governing permissions and limitations * specific language governing permissions and limitations
* under the License. * under the License.
*/ */
package org.elasticsearch.protocol.xpack.ml.job.results; package org.elasticsearch.client.ml.job.results;
import org.elasticsearch.client.ml.job.config.Job;
import org.elasticsearch.common.ParseField; import org.elasticsearch.common.ParseField;
import org.elasticsearch.common.time.DateFormatters; import org.elasticsearch.common.time.DateFormatters;
import org.elasticsearch.common.xcontent.ConstructingObjectParser; import org.elasticsearch.common.xcontent.ConstructingObjectParser;
@ -25,7 +26,6 @@ import org.elasticsearch.common.xcontent.ObjectParser.ValueType;
import org.elasticsearch.common.xcontent.ToXContentObject; import org.elasticsearch.common.xcontent.ToXContentObject;
import org.elasticsearch.common.xcontent.XContentBuilder; import org.elasticsearch.common.xcontent.XContentBuilder;
import org.elasticsearch.common.xcontent.XContentParser.Token; import org.elasticsearch.common.xcontent.XContentParser.Token;
import org.elasticsearch.protocol.xpack.ml.job.config.Job;
import java.io.IOException; import java.io.IOException;
import java.time.format.DateTimeFormatter; import java.time.format.DateTimeFormatter;

View File

@ -16,8 +16,9 @@
* specific language governing permissions and limitations * specific language governing permissions and limitations
* under the License. * under the License.
*/ */
package org.elasticsearch.protocol.xpack.ml.job.results; package org.elasticsearch.client.ml.job.results;
import org.elasticsearch.client.ml.job.config.Job;
import org.elasticsearch.common.ParseField; import org.elasticsearch.common.ParseField;
import org.elasticsearch.common.time.DateFormatters; import org.elasticsearch.common.time.DateFormatters;
import org.elasticsearch.common.xcontent.ConstructingObjectParser; import org.elasticsearch.common.xcontent.ConstructingObjectParser;
@ -25,7 +26,6 @@ import org.elasticsearch.common.xcontent.ObjectParser;
import org.elasticsearch.common.xcontent.ToXContentObject; import org.elasticsearch.common.xcontent.ToXContentObject;
import org.elasticsearch.common.xcontent.XContentBuilder; import org.elasticsearch.common.xcontent.XContentBuilder;
import org.elasticsearch.common.xcontent.XContentParser; import org.elasticsearch.common.xcontent.XContentParser;
import org.elasticsearch.protocol.xpack.ml.job.config.Job;
import java.io.IOException; import java.io.IOException;
import java.time.format.DateTimeFormatter; import java.time.format.DateTimeFormatter;

View File

@ -16,7 +16,7 @@
* specific language governing permissions and limitations * specific language governing permissions and limitations
* under the License. * under the License.
*/ */
package org.elasticsearch.protocol.xpack.ml.job.results; package org.elasticsearch.client.ml.job.results;
import org.elasticsearch.common.ParseField; import org.elasticsearch.common.ParseField;

View File

@ -16,7 +16,7 @@
* specific language governing permissions and limitations * specific language governing permissions and limitations
* under the License. * under the License.
*/ */
package org.elasticsearch.protocol.xpack.ml.job.util; package org.elasticsearch.client.ml.job.util;
import org.elasticsearch.common.Nullable; import org.elasticsearch.common.Nullable;
import org.elasticsearch.common.ParseField; import org.elasticsearch.common.ParseField;

View File

@ -16,7 +16,7 @@
* specific language governing permissions and limitations * specific language governing permissions and limitations
* under the License. * under the License.
*/ */
package org.elasticsearch.protocol.xpack.ml.job.util; package org.elasticsearch.client.ml.job.util;
import org.elasticsearch.common.time.DateFormatters; import org.elasticsearch.common.time.DateFormatters;
import org.elasticsearch.common.xcontent.XContentParser; import org.elasticsearch.common.xcontent.XContentParser;

View File

@ -41,12 +41,16 @@ import org.elasticsearch.action.update.UpdateRequest;
import org.elasticsearch.action.update.UpdateResponse; import org.elasticsearch.action.update.UpdateResponse;
import org.elasticsearch.common.Strings; import org.elasticsearch.common.Strings;
import org.elasticsearch.common.bytes.BytesReference; import org.elasticsearch.common.bytes.BytesReference;
import org.elasticsearch.common.settings.Settings;
import org.elasticsearch.common.unit.ByteSizeUnit; import org.elasticsearch.common.unit.ByteSizeUnit;
import org.elasticsearch.common.unit.ByteSizeValue; import org.elasticsearch.common.unit.ByteSizeValue;
import org.elasticsearch.common.xcontent.XContentBuilder; import org.elasticsearch.common.xcontent.XContentBuilder;
import org.elasticsearch.common.xcontent.XContentType; import org.elasticsearch.common.xcontent.XContentType;
import org.elasticsearch.index.VersionType; import org.elasticsearch.index.VersionType;
import org.elasticsearch.index.get.GetResult; import org.elasticsearch.index.get.GetResult;
import org.elasticsearch.index.query.IdsQueryBuilder;
import org.elasticsearch.index.reindex.BulkByScrollResponse;
import org.elasticsearch.index.reindex.ReindexRequest;
import org.elasticsearch.rest.RestStatus; import org.elasticsearch.rest.RestStatus;
import org.elasticsearch.script.Script; import org.elasticsearch.script.Script;
import org.elasticsearch.script.ScriptType; import org.elasticsearch.script.ScriptType;
@ -624,6 +628,69 @@ public class CrudIT extends ESRestHighLevelClientTestCase {
validateBulkResponses(nbItems, errors, bulkResponse, bulkRequest); validateBulkResponses(nbItems, errors, bulkResponse, bulkRequest);
} }
public void testReindex() throws IOException {
final String sourceIndex = "source1";
final String destinationIndex = "dest";
{
// Prepare
Settings settings = Settings.builder()
.put("number_of_shards", 1)
.put("number_of_replicas", 0)
.build();
createIndex(sourceIndex, settings);
createIndex(destinationIndex, settings);
assertEquals(
RestStatus.OK,
highLevelClient().bulk(
new BulkRequest()
.add(new IndexRequest(sourceIndex, "type", "1")
.source(Collections.singletonMap("foo", "bar"), XContentType.JSON))
.add(new IndexRequest(sourceIndex, "type", "2")
.source(Collections.singletonMap("foo2", "bar2"), XContentType.JSON))
.setRefreshPolicy(RefreshPolicy.IMMEDIATE),
RequestOptions.DEFAULT
).status()
);
}
{
// test1: create one doc in dest
ReindexRequest reindexRequest = new ReindexRequest();
reindexRequest.setSourceIndices(sourceIndex);
reindexRequest.setDestIndex(destinationIndex);
reindexRequest.setSourceQuery(new IdsQueryBuilder().addIds("1").types("type"));
reindexRequest.setRefresh(true);
BulkByScrollResponse bulkResponse = execute(reindexRequest, highLevelClient()::reindex, highLevelClient()::reindexAsync);
assertEquals(1, bulkResponse.getCreated());
assertEquals(1, bulkResponse.getTotal());
assertEquals(0, bulkResponse.getDeleted());
assertEquals(0, bulkResponse.getNoops());
assertEquals(0, bulkResponse.getVersionConflicts());
assertEquals(1, bulkResponse.getBatches());
assertTrue(bulkResponse.getTook().getMillis() > 0);
assertEquals(1, bulkResponse.getBatches());
assertEquals(0, bulkResponse.getBulkFailures().size());
assertEquals(0, bulkResponse.getSearchFailures().size());
}
{
// test2: create 1 and update 1
ReindexRequest reindexRequest = new ReindexRequest();
reindexRequest.setSourceIndices(sourceIndex);
reindexRequest.setDestIndex(destinationIndex);
BulkByScrollResponse bulkResponse = execute(reindexRequest, highLevelClient()::reindex, highLevelClient()::reindexAsync);
assertEquals(1, bulkResponse.getCreated());
assertEquals(2, bulkResponse.getTotal());
assertEquals(1, bulkResponse.getUpdated());
assertEquals(0, bulkResponse.getDeleted());
assertEquals(0, bulkResponse.getNoops());
assertEquals(0, bulkResponse.getVersionConflicts());
assertEquals(1, bulkResponse.getBatches());
assertTrue(bulkResponse.getTook().getMillis() > 0);
assertEquals(1, bulkResponse.getBatches());
assertEquals(0, bulkResponse.getBulkFailures().size());
assertEquals(0, bulkResponse.getSearchFailures().size());
}
}
public void testBulkProcessorIntegration() throws IOException { public void testBulkProcessorIntegration() throws IOException {
int nbItems = randomIntBetween(10, 100); int nbItems = randomIntBetween(10, 100);
boolean[] errors = new boolean[nbItems]; boolean[] errors = new boolean[nbItems];

View File

@ -23,19 +23,19 @@ import org.apache.http.client.methods.HttpDelete;
import org.apache.http.client.methods.HttpGet; import org.apache.http.client.methods.HttpGet;
import org.apache.http.client.methods.HttpPost; import org.apache.http.client.methods.HttpPost;
import org.apache.http.client.methods.HttpPut; import org.apache.http.client.methods.HttpPut;
import org.elasticsearch.client.ml.CloseJobRequest;
import org.elasticsearch.client.ml.DeleteJobRequest;
import org.elasticsearch.client.ml.GetBucketsRequest;
import org.elasticsearch.client.ml.GetJobRequest;
import org.elasticsearch.client.ml.OpenJobRequest;
import org.elasticsearch.client.ml.PutJobRequest;
import org.elasticsearch.client.ml.job.config.AnalysisConfig;
import org.elasticsearch.client.ml.job.config.Detector;
import org.elasticsearch.client.ml.job.config.Job;
import org.elasticsearch.client.ml.job.util.PageParams;
import org.elasticsearch.common.unit.TimeValue; import org.elasticsearch.common.unit.TimeValue;
import org.elasticsearch.common.xcontent.XContentParser; import org.elasticsearch.common.xcontent.XContentParser;
import org.elasticsearch.common.xcontent.json.JsonXContent; import org.elasticsearch.common.xcontent.json.JsonXContent;
import org.elasticsearch.protocol.xpack.ml.CloseJobRequest;
import org.elasticsearch.protocol.xpack.ml.DeleteJobRequest;
import org.elasticsearch.protocol.xpack.ml.GetBucketsRequest;
import org.elasticsearch.protocol.xpack.ml.GetJobRequest;
import org.elasticsearch.protocol.xpack.ml.OpenJobRequest;
import org.elasticsearch.protocol.xpack.ml.PutJobRequest;
import org.elasticsearch.protocol.xpack.ml.job.config.AnalysisConfig;
import org.elasticsearch.protocol.xpack.ml.job.config.Detector;
import org.elasticsearch.protocol.xpack.ml.job.config.Job;
import org.elasticsearch.protocol.xpack.ml.job.util.PageParams;
import org.elasticsearch.test.ESTestCase; import org.elasticsearch.test.ESTestCase;
import java.io.ByteArrayOutputStream; import java.io.ByteArrayOutputStream;

View File

@ -21,20 +21,26 @@ package org.elasticsearch.client;
import org.elasticsearch.action.bulk.BulkRequest; import org.elasticsearch.action.bulk.BulkRequest;
import org.elasticsearch.action.index.IndexRequest; import org.elasticsearch.action.index.IndexRequest;
import org.elasticsearch.action.support.WriteRequest; import org.elasticsearch.action.support.WriteRequest;
import org.elasticsearch.client.ml.GetBucketsRequest;
import org.elasticsearch.client.ml.GetBucketsResponse;
import org.elasticsearch.client.ml.GetRecordsRequest;
import org.elasticsearch.client.ml.GetRecordsResponse;
import org.elasticsearch.client.ml.PutJobRequest;
import org.elasticsearch.client.ml.job.config.Job;
import org.elasticsearch.client.ml.job.results.AnomalyRecord;
import org.elasticsearch.client.ml.job.results.Bucket;
import org.elasticsearch.client.ml.job.util.PageParams;
import org.elasticsearch.common.xcontent.XContentType; import org.elasticsearch.common.xcontent.XContentType;
import org.elasticsearch.protocol.xpack.ml.GetBucketsRequest;
import org.elasticsearch.protocol.xpack.ml.GetBucketsResponse;
import org.elasticsearch.protocol.xpack.ml.PutJobRequest;
import org.elasticsearch.protocol.xpack.ml.job.config.Job;
import org.elasticsearch.protocol.xpack.ml.job.results.Bucket;
import org.elasticsearch.protocol.xpack.ml.job.util.PageParams;
import org.junit.After; import org.junit.After;
import org.junit.Before; import org.junit.Before;
import java.io.IOException; import java.io.IOException;
import static org.hamcrest.Matchers.equalTo; import static org.hamcrest.Matchers.equalTo;
import static org.hamcrest.Matchers.greaterThan;
import static org.hamcrest.Matchers.greaterThanOrEqualTo;
import static org.hamcrest.Matchers.is; import static org.hamcrest.Matchers.is;
import static org.hamcrest.Matchers.lessThan;
import static org.hamcrest.Matchers.lessThanOrEqualTo; import static org.hamcrest.Matchers.lessThanOrEqualTo;
public class MachineLearningGetResultsIT extends ESRestHighLevelClientTestCase { public class MachineLearningGetResultsIT extends ESRestHighLevelClientTestCase {
@ -47,7 +53,8 @@ public class MachineLearningGetResultsIT extends ESRestHighLevelClientTestCase {
// 2018-08-01T00:00:00Z // 2018-08-01T00:00:00Z
private static final long START_TIME_EPOCH_MS = 1533081600000L; private static final long START_TIME_EPOCH_MS = 1533081600000L;
private BucketStats bucketStats = new BucketStats(); private Stats bucketStats = new Stats();
private Stats recordStats = new Stats();
@Before @Before
public void createJobAndIndexResults() throws IOException { public void createJobAndIndexResults() throws IOException {
@ -68,7 +75,7 @@ public class MachineLearningGetResultsIT extends ESRestHighLevelClientTestCase {
// Also index an interim bucket // Also index an interim bucket
addBucketIndexRequest(time, true, bulkRequest); addBucketIndexRequest(time, true, bulkRequest);
addRecordIndexRequests(time, true, bulkRequest); addRecordIndexRequest(time, true, bulkRequest);
highLevelClient().bulk(bulkRequest, RequestOptions.DEFAULT); highLevelClient().bulk(bulkRequest, RequestOptions.DEFAULT);
} }
@ -91,16 +98,21 @@ public class MachineLearningGetResultsIT extends ESRestHighLevelClientTestCase {
} }
int recordCount = randomIntBetween(1, 3); int recordCount = randomIntBetween(1, 3);
for (int i = 0; i < recordCount; ++i) { for (int i = 0; i < recordCount; ++i) {
IndexRequest indexRequest = new IndexRequest(RESULTS_INDEX, DOC); addRecordIndexRequest(timestamp, isInterim, bulkRequest);
double recordScore = randomDoubleBetween(0.0, 100.0, true);
double p = randomDoubleBetween(0.0, 0.05, false);
indexRequest.source("{\"job_id\":\"" + JOB_ID + "\", \"result_type\":\"record\", \"timestamp\": " + timestamp + "," +
"\"bucket_span\": 3600,\"is_interim\": " + isInterim + ", \"record_score\": " + recordScore + ", \"probability\": "
+ p + "}", XContentType.JSON);
bulkRequest.add(indexRequest);
} }
} }
private void addRecordIndexRequest(long timestamp, boolean isInterim, BulkRequest bulkRequest) {
IndexRequest indexRequest = new IndexRequest(RESULTS_INDEX, DOC);
double recordScore = randomDoubleBetween(0.0, 100.0, true);
recordStats.report(recordScore);
double p = randomDoubleBetween(0.0, 0.05, false);
indexRequest.source("{\"job_id\":\"" + JOB_ID + "\", \"result_type\":\"record\", \"timestamp\": " + timestamp + "," +
"\"bucket_span\": 3600,\"is_interim\": " + isInterim + ", \"record_score\": " + recordScore + ", \"probability\": "
+ p + "}", XContentType.JSON);
bulkRequest.add(indexRequest);
}
@After @After
public void deleteJob() throws IOException { public void deleteJob() throws IOException {
new MlRestTestStateCleaner(logger, client()).clearMlMetadata(); new MlRestTestStateCleaner(logger, client()).clearMlMetadata();
@ -194,7 +206,73 @@ public class MachineLearningGetResultsIT extends ESRestHighLevelClientTestCase {
} }
} }
private static class BucketStats { public void testGetRecords() throws IOException {
MachineLearningClient machineLearningClient = highLevelClient().machineLearning();
{
GetRecordsRequest request = new GetRecordsRequest(JOB_ID);
GetRecordsResponse response = execute(request, machineLearningClient::getRecords, machineLearningClient::getRecordsAsync);
assertThat(response.count(), greaterThan(0L));
assertThat(response.count(), equalTo(recordStats.totalCount()));
}
{
GetRecordsRequest request = new GetRecordsRequest(JOB_ID);
request.setRecordScore(50.0);
GetRecordsResponse response = execute(request, machineLearningClient::getRecords, machineLearningClient::getRecordsAsync);
long majorAndCriticalCount = recordStats.majorCount + recordStats.criticalCount;
assertThat(response.count(), equalTo(majorAndCriticalCount));
assertThat(response.records().size(), equalTo((int) Math.min(100, majorAndCriticalCount)));
assertThat(response.records().stream().anyMatch(r -> r.getRecordScore() < 50.0), is(false));
}
{
GetRecordsRequest request = new GetRecordsRequest(JOB_ID);
request.setExcludeInterim(true);
GetRecordsResponse response = execute(request, machineLearningClient::getRecords, machineLearningClient::getRecordsAsync);
assertThat(response.count(), equalTo(recordStats.totalCount() - 1));
}
{
long end = START_TIME_EPOCH_MS + 10 * 3600000;
GetRecordsRequest request = new GetRecordsRequest(JOB_ID);
request.setStart(String.valueOf(START_TIME_EPOCH_MS));
request.setEnd(String.valueOf(end));
GetRecordsResponse response = execute(request, machineLearningClient::getRecords, machineLearningClient::getRecordsAsync);
for (AnomalyRecord record : response.records()) {
assertThat(record.getTimestamp().getTime(), greaterThanOrEqualTo(START_TIME_EPOCH_MS));
assertThat(record.getTimestamp().getTime(), lessThan(end));
}
}
{
GetRecordsRequest request = new GetRecordsRequest(JOB_ID);
request.setPageParams(new PageParams(3, 3));
GetRecordsResponse response = execute(request, machineLearningClient::getRecords, machineLearningClient::getRecordsAsync);
assertThat(response.records().size(), equalTo(3));
}
{
GetRecordsRequest request = new GetRecordsRequest(JOB_ID);
request.setSort("probability");
request.setDescending(true);
GetRecordsResponse response = execute(request, machineLearningClient::getRecords, machineLearningClient::getRecordsAsync);
double previousProb = 1.0;
for (AnomalyRecord record : response.records()) {
assertThat(record.getProbability(), lessThanOrEqualTo(previousProb));
previousProb = record.getProbability();
}
}
}
private static class Stats {
// score < 50.0 // score < 50.0
private long minorCount; private long minorCount;
@ -204,14 +282,18 @@ public class MachineLearningGetResultsIT extends ESRestHighLevelClientTestCase {
// score > 75.0 // score > 75.0
private long criticalCount; private long criticalCount;
private void report(double anomalyScore) { private void report(double score) {
if (anomalyScore < 50.0) { if (score < 50.0) {
minorCount++; minorCount++;
} else if (anomalyScore < 75.0) { } else if (score < 75.0) {
majorCount++; majorCount++;
} else { } else {
criticalCount++; criticalCount++;
} }
} }
private long totalCount() {
return minorCount + majorCount + criticalCount;
}
} }
} }

View File

@ -19,21 +19,21 @@
package org.elasticsearch.client; package org.elasticsearch.client;
import com.carrotsearch.randomizedtesting.generators.CodepointSetGenerator; import com.carrotsearch.randomizedtesting.generators.CodepointSetGenerator;
import org.elasticsearch.client.ml.CloseJobRequest;
import org.elasticsearch.client.ml.CloseJobResponse;
import org.elasticsearch.client.ml.DeleteJobRequest;
import org.elasticsearch.client.ml.DeleteJobResponse;
import org.elasticsearch.client.ml.GetJobRequest;
import org.elasticsearch.client.ml.GetJobResponse;
import org.elasticsearch.client.ml.OpenJobRequest;
import org.elasticsearch.client.ml.OpenJobResponse;
import org.elasticsearch.client.ml.PutJobRequest;
import org.elasticsearch.client.ml.PutJobResponse;
import org.elasticsearch.client.ml.job.config.AnalysisConfig;
import org.elasticsearch.client.ml.job.config.DataDescription;
import org.elasticsearch.client.ml.job.config.Detector;
import org.elasticsearch.client.ml.job.config.Job;
import org.elasticsearch.common.unit.TimeValue; import org.elasticsearch.common.unit.TimeValue;
import org.elasticsearch.protocol.xpack.ml.CloseJobRequest;
import org.elasticsearch.protocol.xpack.ml.CloseJobResponse;
import org.elasticsearch.protocol.xpack.ml.DeleteJobRequest;
import org.elasticsearch.protocol.xpack.ml.DeleteJobResponse;
import org.elasticsearch.protocol.xpack.ml.GetJobRequest;
import org.elasticsearch.protocol.xpack.ml.GetJobResponse;
import org.elasticsearch.protocol.xpack.ml.OpenJobRequest;
import org.elasticsearch.protocol.xpack.ml.OpenJobResponse;
import org.elasticsearch.protocol.xpack.ml.PutJobRequest;
import org.elasticsearch.protocol.xpack.ml.PutJobResponse;
import org.elasticsearch.protocol.xpack.ml.job.config.AnalysisConfig;
import org.elasticsearch.protocol.xpack.ml.job.config.DataDescription;
import org.elasticsearch.protocol.xpack.ml.job.config.Detector;
import org.elasticsearch.protocol.xpack.ml.job.config.Job;
import org.junit.After; import org.junit.After;
import java.io.IOException; import java.io.IOException;

View File

@ -116,6 +116,7 @@ import org.elasticsearch.common.xcontent.XContentBuilder;
import org.elasticsearch.common.xcontent.XContentHelper; import org.elasticsearch.common.xcontent.XContentHelper;
import org.elasticsearch.common.xcontent.XContentParser; import org.elasticsearch.common.xcontent.XContentParser;
import org.elasticsearch.common.xcontent.XContentType; import org.elasticsearch.common.xcontent.XContentType;
import org.elasticsearch.common.xcontent.json.JsonXContent;
import org.elasticsearch.index.RandomCreateIndexGenerator; import org.elasticsearch.index.RandomCreateIndexGenerator;
import org.elasticsearch.index.VersionType; import org.elasticsearch.index.VersionType;
import org.elasticsearch.index.query.QueryBuilder; import org.elasticsearch.index.query.QueryBuilder;
@ -126,6 +127,8 @@ import org.elasticsearch.index.rankeval.RankEvalRequest;
import org.elasticsearch.index.rankeval.RankEvalSpec; import org.elasticsearch.index.rankeval.RankEvalSpec;
import org.elasticsearch.index.rankeval.RatedRequest; import org.elasticsearch.index.rankeval.RatedRequest;
import org.elasticsearch.index.rankeval.RestRankEvalAction; import org.elasticsearch.index.rankeval.RestRankEvalAction;
import org.elasticsearch.index.reindex.ReindexRequest;
import org.elasticsearch.index.reindex.RemoteInfo;
import org.elasticsearch.protocol.xpack.XPackInfoRequest; import org.elasticsearch.protocol.xpack.XPackInfoRequest;
import org.elasticsearch.protocol.xpack.migration.IndexUpgradeInfoRequest; import org.elasticsearch.protocol.xpack.migration.IndexUpgradeInfoRequest;
import org.elasticsearch.protocol.xpack.watcher.DeleteWatchRequest; import org.elasticsearch.protocol.xpack.watcher.DeleteWatchRequest;
@ -172,6 +175,7 @@ import java.util.function.Function;
import java.util.function.Supplier; import java.util.function.Supplier;
import java.util.stream.Collectors; import java.util.stream.Collectors;
import static java.util.Collections.emptyMap;
import static java.util.Collections.singletonMap; import static java.util.Collections.singletonMap;
import static org.elasticsearch.client.RequestConverters.REQUEST_BODY_CONTENT_TYPE; import static org.elasticsearch.client.RequestConverters.REQUEST_BODY_CONTENT_TYPE;
import static org.elasticsearch.client.RequestConverters.enforceSameContentType; import static org.elasticsearch.client.RequestConverters.enforceSameContentType;
@ -179,6 +183,7 @@ import static org.elasticsearch.index.RandomCreateIndexGenerator.randomAliases;
import static org.elasticsearch.index.RandomCreateIndexGenerator.randomCreateIndexRequest; import static org.elasticsearch.index.RandomCreateIndexGenerator.randomCreateIndexRequest;
import static org.elasticsearch.index.RandomCreateIndexGenerator.randomIndexSettings; import static org.elasticsearch.index.RandomCreateIndexGenerator.randomIndexSettings;
import static org.elasticsearch.index.alias.RandomAliasActionsGenerator.randomAliasAction; import static org.elasticsearch.index.alias.RandomAliasActionsGenerator.randomAliasAction;
import static org.elasticsearch.index.query.QueryBuilders.matchAllQuery;
import static org.elasticsearch.search.RandomSearchRequestGenerator.randomSearchRequest; import static org.elasticsearch.search.RandomSearchRequestGenerator.randomSearchRequest;
import static org.elasticsearch.test.hamcrest.ElasticsearchAssertions.assertToXContentEquivalent; import static org.elasticsearch.test.hamcrest.ElasticsearchAssertions.assertToXContentEquivalent;
import static org.hamcrest.CoreMatchers.equalTo; import static org.hamcrest.CoreMatchers.equalTo;
@ -407,6 +412,64 @@ public class RequestConvertersTests extends ESTestCase {
assertToXContentBody(indicesAliasesRequest, request.getEntity()); assertToXContentBody(indicesAliasesRequest, request.getEntity());
} }
public void testReindex() throws IOException {
ReindexRequest reindexRequest = new ReindexRequest();
reindexRequest.setSourceIndices("source_idx");
reindexRequest.setDestIndex("dest_idx");
Map<String, String> expectedParams = new HashMap<>();
if (randomBoolean()) {
XContentBuilder builder = JsonXContent.contentBuilder().prettyPrint();
RemoteInfo remoteInfo = new RemoteInfo("http", "remote-host", 9200, null,
BytesReference.bytes(matchAllQuery().toXContent(builder, ToXContent.EMPTY_PARAMS)),
"user",
"pass",
emptyMap(),
RemoteInfo.DEFAULT_SOCKET_TIMEOUT,
RemoteInfo.DEFAULT_CONNECT_TIMEOUT
);
reindexRequest.setRemoteInfo(remoteInfo);
}
if (randomBoolean()) {
reindexRequest.setSourceDocTypes("doc", "tweet");
}
if (randomBoolean()) {
reindexRequest.setSourceBatchSize(randomInt(100));
}
if (randomBoolean()) {
reindexRequest.setDestDocType("tweet_and_doc");
}
if (randomBoolean()) {
reindexRequest.setDestOpType("create");
}
if (randomBoolean()) {
reindexRequest.setDestPipeline("my_pipeline");
}
if (randomBoolean()) {
reindexRequest.setDestRouting("=cat");
}
if (randomBoolean()) {
reindexRequest.setSize(randomIntBetween(100, 1000));
}
if (randomBoolean()) {
reindexRequest.setAbortOnVersionConflict(false);
}
if (randomBoolean()) {
String ts = randomTimeValue();
reindexRequest.setScroll(TimeValue.parseTimeValue(ts, "scroll"));
}
if (reindexRequest.getRemoteInfo() == null && randomBoolean()) {
reindexRequest.setSourceQuery(new TermQueryBuilder("foo", "fooval"));
}
setRandomTimeout(reindexRequest::setTimeout, ReplicationRequest.DEFAULT_TIMEOUT, expectedParams);
setRandomWaitForActiveShards(reindexRequest::setWaitForActiveShards, ActiveShardCount.DEFAULT, expectedParams);
expectedParams.put("scroll", reindexRequest.getScrollTime().getStringRep());
Request request = RequestConverters.reindex(reindexRequest);
assertEquals("/_reindex", request.getEndpoint());
assertEquals(HttpPost.METHOD_NAME, request.getMethod());
assertEquals(expectedParams, request.getParameters());
assertToXContentBody(reindexRequest, request.getEntity());
}
public void testPutMapping() throws IOException { public void testPutMapping() throws IOException {
PutMappingRequest putMappingRequest = new PutMappingRequest(); PutMappingRequest putMappingRequest = new PutMappingRequest();

View File

@ -660,7 +660,6 @@ public class RestHighLevelClientTests extends ESTestCase {
"indices.put_alias", "indices.put_alias",
"mtermvectors", "mtermvectors",
"put_script", "put_script",
"reindex",
"reindex_rethrottle", "reindex_rethrottle",
"render_search_template", "render_search_template",
"scripts_painless_execute", "scripts_painless_execute",

View File

@ -50,6 +50,8 @@ import org.elasticsearch.client.RequestOptions;
import org.elasticsearch.client.Response; import org.elasticsearch.client.Response;
import org.elasticsearch.client.RestHighLevelClient; import org.elasticsearch.client.RestHighLevelClient;
import org.elasticsearch.common.Strings; import org.elasticsearch.common.Strings;
import org.elasticsearch.common.bytes.BytesArray;
import org.elasticsearch.common.settings.Settings;
import org.elasticsearch.common.unit.ByteSizeUnit; import org.elasticsearch.common.unit.ByteSizeUnit;
import org.elasticsearch.common.unit.ByteSizeValue; import org.elasticsearch.common.unit.ByteSizeValue;
import org.elasticsearch.common.unit.TimeValue; import org.elasticsearch.common.unit.TimeValue;
@ -59,13 +61,22 @@ import org.elasticsearch.common.xcontent.XContentType;
import org.elasticsearch.common.xcontent.json.JsonXContent; import org.elasticsearch.common.xcontent.json.JsonXContent;
import org.elasticsearch.index.VersionType; import org.elasticsearch.index.VersionType;
import org.elasticsearch.index.get.GetResult; import org.elasticsearch.index.get.GetResult;
import org.elasticsearch.index.query.MatchAllQueryBuilder;
import org.elasticsearch.index.query.TermQueryBuilder;
import org.elasticsearch.index.reindex.BulkByScrollResponse;
import org.elasticsearch.index.reindex.ReindexRequest;
import org.elasticsearch.index.reindex.RemoteInfo;
import org.elasticsearch.index.reindex.ScrollableHitSource;
import org.elasticsearch.rest.RestStatus; import org.elasticsearch.rest.RestStatus;
import org.elasticsearch.script.Script; import org.elasticsearch.script.Script;
import org.elasticsearch.script.ScriptType; import org.elasticsearch.script.ScriptType;
import org.elasticsearch.search.fetch.subphase.FetchSourceContext; import org.elasticsearch.search.fetch.subphase.FetchSourceContext;
import org.elasticsearch.search.sort.SortOrder;
import java.util.Collections;
import java.util.Date; import java.util.Date;
import java.util.HashMap; import java.util.HashMap;
import java.util.List;
import java.util.Map; import java.util.Map;
import java.util.concurrent.CountDownLatch; import java.util.concurrent.CountDownLatch;
import java.util.concurrent.TimeUnit; import java.util.concurrent.TimeUnit;
@ -750,6 +761,144 @@ public class CRUDDocumentationIT extends ESRestHighLevelClientTestCase {
} }
} }
public void testReindex() throws Exception {
RestHighLevelClient client = highLevelClient();
{
String mapping =
"\"doc\": {\n" +
" \"properties\": {\n" +
" \"user\": {\n" +
" \"type\": \"text\"\n" +
" },\n" +
" \"field1\": {\n" +
" \"type\": \"integer\"\n" +
" },\n" +
" \"field2\": {\n" +
" \"type\": \"integer\"\n" +
" }\n" +
" }\n" +
" }";
createIndex("source1", Settings.EMPTY, mapping);
createIndex("source2", Settings.EMPTY, mapping);
createPipeline("my_pipeline");
}
{
// tag::reindex-request
ReindexRequest request = new ReindexRequest(); // <1>
request.setSourceIndices("source1", "source2"); // <2>
request.setDestIndex("dest"); // <3>
// end::reindex-request
// tag::reindex-request-versionType
request.setDestVersionType(VersionType.EXTERNAL); // <1>
// end::reindex-request-versionType
// tag::reindex-request-opType
request.setDestOpType("create"); // <1>
// end::reindex-request-opType
// tag::reindex-request-conflicts
request.setConflicts("proceed"); // <1>
// end::reindex-request-conflicts
// tag::reindex-request-typeOrQuery
request.setSourceDocTypes("doc"); // <1>
request.setSourceQuery(new TermQueryBuilder("user", "kimchy")); // <2>
// end::reindex-request-typeOrQuery
// tag::reindex-request-size
request.setSize(10); // <1>
// end::reindex-request-size
// tag::reindex-request-sourceSize
request.setSourceBatchSize(100); // <1>
// end::reindex-request-sourceSize
// tag::reindex-request-pipeline
request.setDestPipeline("my_pipeline"); // <1>
// end::reindex-request-pipeline
// tag::reindex-request-sort
request.addSortField("field1", SortOrder.DESC); // <1>
request.addSortField("field2", SortOrder.ASC); // <2>
// end::reindex-request-sort
// tag::reindex-request-script
request.setScript(
new Script(
ScriptType.INLINE, "painless",
"if (ctx._source.user == 'kimchy') {ctx._source.likes++;}",
Collections.emptyMap())); // <1>
// end::reindex-request-script
// tag::reindex-request-remote
request.setRemoteInfo(
new RemoteInfo(
"https", "localhost", 9002, null, new BytesArray(new MatchAllQueryBuilder().toString()),
"user", "pass", Collections.emptyMap(), new TimeValue(100, TimeUnit.MILLISECONDS),
new TimeValue(100, TimeUnit.SECONDS)
)
); // <1>
// end::reindex-request-remote
request.setRemoteInfo(null); // Remove it for tests
// tag::reindex-request-timeout
request.setTimeout(TimeValue.timeValueMinutes(2)); // <1>
// end::reindex-request-timeout
// tag::reindex-request-refresh
request.setRefresh(true); // <1>
// end::reindex-request-refresh
// tag::reindex-request-slices
request.setSlices(2); // <1>
// end::reindex-request-slices
// tag::reindex-request-scroll
request.setScroll(TimeValue.timeValueMinutes(10)); // <1>
// end::reindex-request-scroll
// tag::reindex-execute
BulkByScrollResponse bulkResponse = client.reindex(request, RequestOptions.DEFAULT);
// end::reindex-execute
assertSame(0, bulkResponse.getSearchFailures().size());
assertSame(0, bulkResponse.getBulkFailures().size());
// tag::reindex-response
TimeValue timeTaken = bulkResponse.getTook(); // <1>
boolean timedOut = bulkResponse.isTimedOut(); // <2>
long totalDocs = bulkResponse.getTotal(); // <3>
long updatedDocs = bulkResponse.getUpdated(); // <4>
long createdDocs = bulkResponse.getCreated(); // <5>
long deletedDocs = bulkResponse.getDeleted(); // <6>
long batches = bulkResponse.getBatches(); // <7>
long noops = bulkResponse.getNoops(); // <8>
long versionConflicts = bulkResponse.getVersionConflicts(); // <9>
long bulkRetries = bulkResponse.getBulkRetries(); // <10>
long searchRetries = bulkResponse.getSearchRetries(); // <11>
TimeValue throttledMillis = bulkResponse.getStatus().getThrottled(); // <12>
TimeValue throttledUntilMillis = bulkResponse.getStatus().getThrottledUntil(); // <13>
List<ScrollableHitSource.SearchFailure> searchFailures = bulkResponse.getSearchFailures(); // <14>
List<BulkItemResponse.Failure> bulkFailures = bulkResponse.getBulkFailures(); // <15>
// end::reindex-response
}
{
ReindexRequest request = new ReindexRequest();
request.setSourceIndices("source1");
request.setDestIndex("dest");
// tag::reindex-execute-listener
ActionListener<BulkByScrollResponse> listener = new ActionListener<BulkByScrollResponse>() {
@Override
public void onResponse(BulkByScrollResponse bulkResponse) {
// <1>
}
@Override
public void onFailure(Exception e) {
// <2>
}
};
// end::reindex-execute-listener
// Replace the empty listener by a blocking listener in test
final CountDownLatch latch = new CountDownLatch(1);
listener = new LatchedActionListener<>(listener, latch);
// tag::reindex-execute-async
client.reindexAsync(request, RequestOptions.DEFAULT, listener); // <1>
// end::reindex-execute-async
assertTrue(latch.await(30L, TimeUnit.SECONDS));
}
}
public void testGet() throws Exception { public void testGet() throws Exception {
RestHighLevelClient client = highLevelClient(); RestHighLevelClient client = highLevelClient();
{ {

View File

@ -27,26 +27,29 @@ import org.elasticsearch.client.MachineLearningIT;
import org.elasticsearch.client.MlRestTestStateCleaner; import org.elasticsearch.client.MlRestTestStateCleaner;
import org.elasticsearch.client.RequestOptions; import org.elasticsearch.client.RequestOptions;
import org.elasticsearch.client.RestHighLevelClient; import org.elasticsearch.client.RestHighLevelClient;
import org.elasticsearch.client.ml.CloseJobRequest;
import org.elasticsearch.client.ml.CloseJobResponse;
import org.elasticsearch.client.ml.DeleteJobRequest;
import org.elasticsearch.client.ml.DeleteJobResponse;
import org.elasticsearch.client.ml.GetBucketsRequest;
import org.elasticsearch.client.ml.GetBucketsResponse;
import org.elasticsearch.client.ml.GetJobRequest;
import org.elasticsearch.client.ml.GetJobResponse;
import org.elasticsearch.client.ml.GetRecordsRequest;
import org.elasticsearch.client.ml.GetRecordsResponse;
import org.elasticsearch.client.ml.OpenJobRequest;
import org.elasticsearch.client.ml.OpenJobResponse;
import org.elasticsearch.client.ml.PutJobRequest;
import org.elasticsearch.client.ml.PutJobResponse;
import org.elasticsearch.client.ml.job.config.AnalysisConfig;
import org.elasticsearch.client.ml.job.config.DataDescription;
import org.elasticsearch.client.ml.job.config.Detector;
import org.elasticsearch.client.ml.job.config.Job;
import org.elasticsearch.client.ml.job.results.AnomalyRecord;
import org.elasticsearch.client.ml.job.results.Bucket;
import org.elasticsearch.client.ml.job.util.PageParams;
import org.elasticsearch.common.unit.TimeValue; import org.elasticsearch.common.unit.TimeValue;
import org.elasticsearch.common.xcontent.XContentType; import org.elasticsearch.common.xcontent.XContentType;
import org.elasticsearch.protocol.xpack.ml.CloseJobRequest;
import org.elasticsearch.protocol.xpack.ml.CloseJobResponse;
import org.elasticsearch.protocol.xpack.ml.DeleteJobRequest;
import org.elasticsearch.protocol.xpack.ml.DeleteJobResponse;
import org.elasticsearch.protocol.xpack.ml.GetBucketsRequest;
import org.elasticsearch.protocol.xpack.ml.GetBucketsResponse;
import org.elasticsearch.protocol.xpack.ml.GetJobRequest;
import org.elasticsearch.protocol.xpack.ml.GetJobResponse;
import org.elasticsearch.protocol.xpack.ml.OpenJobRequest;
import org.elasticsearch.protocol.xpack.ml.OpenJobResponse;
import org.elasticsearch.protocol.xpack.ml.PutJobRequest;
import org.elasticsearch.protocol.xpack.ml.PutJobResponse;
import org.elasticsearch.protocol.xpack.ml.job.config.AnalysisConfig;
import org.elasticsearch.protocol.xpack.ml.job.config.DataDescription;
import org.elasticsearch.protocol.xpack.ml.job.config.Detector;
import org.elasticsearch.protocol.xpack.ml.job.config.Job;
import org.elasticsearch.protocol.xpack.ml.job.results.Bucket;
import org.elasticsearch.protocol.xpack.ml.job.util.PageParams;
import org.junit.After; import org.junit.After;
import java.io.IOException; import java.io.IOException;
@ -454,4 +457,94 @@ public class MlClientDocumentationIT extends ESRestHighLevelClientTestCase {
assertTrue(latch.await(30L, TimeUnit.SECONDS)); assertTrue(latch.await(30L, TimeUnit.SECONDS));
} }
} }
public void testGetRecords() throws IOException, InterruptedException {
RestHighLevelClient client = highLevelClient();
String jobId = "test-get-records";
Job job = MachineLearningIT.buildJob(jobId);
client.machineLearning().putJob(new PutJobRequest(job), RequestOptions.DEFAULT);
// Let us index a record
IndexRequest indexRequest = new IndexRequest(".ml-anomalies-shared", "doc");
indexRequest.setRefreshPolicy(WriteRequest.RefreshPolicy.IMMEDIATE);
indexRequest.source("{\"job_id\":\"test-get-records\", \"result_type\":\"record\", \"timestamp\": 1533081600000," +
"\"bucket_span\": 600,\"is_interim\": false, \"record_score\": 80.0}", XContentType.JSON);
client.index(indexRequest, RequestOptions.DEFAULT);
{
// tag::x-pack-ml-get-records-request
GetRecordsRequest request = new GetRecordsRequest(jobId); // <1>
// end::x-pack-ml-get-records-request
// tag::x-pack-ml-get-records-desc
request.setDescending(true); // <1>
// end::x-pack-ml-get-records-desc
// tag::x-pack-ml-get-records-end
request.setEnd("2018-08-21T00:00:00Z"); // <1>
// end::x-pack-ml-get-records-end
// tag::x-pack-ml-get-records-exclude-interim
request.setExcludeInterim(true); // <1>
// end::x-pack-ml-get-records-exclude-interim
// tag::x-pack-ml-get-records-page
request.setPageParams(new PageParams(100, 200)); // <1>
// end::x-pack-ml-get-records-page
// Set page params back to null so the response contains the record we indexed
request.setPageParams(null);
// tag::x-pack-ml-get-records-record-score
request.setRecordScore(75.0); // <1>
// end::x-pack-ml-get-records-record-score
// tag::x-pack-ml-get-records-sort
request.setSort("probability"); // <1>
// end::x-pack-ml-get-records-sort
// tag::x-pack-ml-get-records-start
request.setStart("2018-08-01T00:00:00Z"); // <1>
// end::x-pack-ml-get-records-start
// tag::x-pack-ml-get-records-execute
GetRecordsResponse response = client.machineLearning().getRecords(request, RequestOptions.DEFAULT);
// end::x-pack-ml-get-records-execute
// tag::x-pack-ml-get-records-response
long count = response.count(); // <1>
List<AnomalyRecord> records = response.records(); // <2>
// end::x-pack-ml-get-records-response
assertEquals(1, records.size());
}
{
GetRecordsRequest request = new GetRecordsRequest(jobId);
// tag::x-pack-ml-get-records-listener
ActionListener<GetRecordsResponse> listener =
new ActionListener<GetRecordsResponse>() {
@Override
public void onResponse(GetRecordsResponse getRecordsResponse) {
// <1>
}
@Override
public void onFailure(Exception e) {
// <2>
}
};
// end::x-pack-ml-get-records-listener
// Replace the empty listener by a blocking listener in test
final CountDownLatch latch = new CountDownLatch(1);
listener = new LatchedActionListener<>(listener, latch);
// tag::x-pack-ml-get-records-execute-async
client.machineLearning().getRecordsAsync(request, RequestOptions.DEFAULT, listener); // <1>
// end::x-pack-ml-get-records-execute-async
assertTrue(latch.await(30L, TimeUnit.SECONDS));
}
}
} }

View File

@ -16,7 +16,7 @@
* specific language governing permissions and limitations * specific language governing permissions and limitations
* under the License. * under the License.
*/ */
package org.elasticsearch.protocol.xpack.ml; package org.elasticsearch.client.ml;
import org.elasticsearch.common.unit.TimeValue; import org.elasticsearch.common.unit.TimeValue;
import org.elasticsearch.common.xcontent.XContentParser; import org.elasticsearch.common.xcontent.XContentParser;

View File

@ -16,7 +16,7 @@
* specific language governing permissions and limitations * specific language governing permissions and limitations
* under the License. * under the License.
*/ */
package org.elasticsearch.protocol.xpack.ml; package org.elasticsearch.client.ml;
import org.elasticsearch.common.xcontent.XContentParser; import org.elasticsearch.common.xcontent.XContentParser;
import org.elasticsearch.test.AbstractXContentTestCase; import org.elasticsearch.test.AbstractXContentTestCase;

View File

@ -16,9 +16,9 @@
* specific language governing permissions and limitations * specific language governing permissions and limitations
* under the License. * under the License.
*/ */
package org.elasticsearch.protocol.xpack.ml; package org.elasticsearch.client.ml;
import org.elasticsearch.protocol.xpack.ml.job.config.JobTests; import org.elasticsearch.client.ml.job.config.JobTests;
import org.elasticsearch.test.ESTestCase; import org.elasticsearch.test.ESTestCase;
public class DeleteJobRequestTests extends ESTestCase { public class DeleteJobRequestTests extends ESTestCase {

View File

@ -16,7 +16,7 @@
* specific language governing permissions and limitations * specific language governing permissions and limitations
* under the License. * under the License.
*/ */
package org.elasticsearch.protocol.xpack.ml; package org.elasticsearch.client.ml;
import org.elasticsearch.common.xcontent.XContentParser; import org.elasticsearch.common.xcontent.XContentParser;
import org.elasticsearch.test.AbstractXContentTestCase; import org.elasticsearch.test.AbstractXContentTestCase;

View File

@ -16,10 +16,10 @@
* specific language governing permissions and limitations * specific language governing permissions and limitations
* under the License. * under the License.
*/ */
package org.elasticsearch.protocol.xpack.ml; package org.elasticsearch.client.ml;
import org.elasticsearch.client.ml.job.util.PageParams;
import org.elasticsearch.common.xcontent.XContentParser; import org.elasticsearch.common.xcontent.XContentParser;
import org.elasticsearch.protocol.xpack.ml.job.util.PageParams;
import org.elasticsearch.test.AbstractXContentTestCase; import org.elasticsearch.test.AbstractXContentTestCase;
import java.io.IOException; import java.io.IOException;

View File

@ -16,11 +16,11 @@
* specific language governing permissions and limitations * specific language governing permissions and limitations
* under the License. * under the License.
*/ */
package org.elasticsearch.protocol.xpack.ml; package org.elasticsearch.client.ml;
import org.elasticsearch.client.ml.job.results.Bucket;
import org.elasticsearch.client.ml.job.results.BucketTests;
import org.elasticsearch.common.xcontent.XContentParser; import org.elasticsearch.common.xcontent.XContentParser;
import org.elasticsearch.protocol.xpack.ml.job.results.Bucket;
import org.elasticsearch.protocol.xpack.ml.job.results.BucketTests;
import org.elasticsearch.test.AbstractXContentTestCase; import org.elasticsearch.test.AbstractXContentTestCase;
import java.io.IOException; import java.io.IOException;

View File

@ -16,7 +16,7 @@
* specific language governing permissions and limitations * specific language governing permissions and limitations
* under the License. * under the License.
*/ */
package org.elasticsearch.protocol.xpack.ml; package org.elasticsearch.client.ml;
import org.elasticsearch.common.xcontent.XContentParser; import org.elasticsearch.common.xcontent.XContentParser;
import org.elasticsearch.test.AbstractXContentTestCase; import org.elasticsearch.test.AbstractXContentTestCase;

View File

@ -16,11 +16,11 @@
* specific language governing permissions and limitations * specific language governing permissions and limitations
* under the License. * under the License.
*/ */
package org.elasticsearch.protocol.xpack.ml; package org.elasticsearch.client.ml;
import org.elasticsearch.client.ml.job.config.Job;
import org.elasticsearch.client.ml.job.config.JobTests;
import org.elasticsearch.common.xcontent.XContentParser; import org.elasticsearch.common.xcontent.XContentParser;
import org.elasticsearch.protocol.xpack.ml.job.config.Job;
import org.elasticsearch.protocol.xpack.ml.job.config.JobTests;
import org.elasticsearch.test.AbstractXContentTestCase; import org.elasticsearch.test.AbstractXContentTestCase;
import java.io.IOException; import java.io.IOException;

View File

@ -0,0 +1,72 @@
/*
* Licensed to Elasticsearch under one or more contributor
* license agreements. See the NOTICE file distributed with
* this work for additional information regarding copyright
* ownership. Elasticsearch licenses this file to you under
* the Apache License, Version 2.0 (the "License"); you may
* not use this file except in compliance with the License.
* You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing,
* software distributed under the License is distributed on an
* "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
* KIND, either express or implied. See the License for the
* specific language governing permissions and limitations
* under the License.
*/
package org.elasticsearch.client.ml;
import org.elasticsearch.client.ml.job.util.PageParams;
import org.elasticsearch.common.xcontent.XContentParser;
import org.elasticsearch.test.AbstractXContentTestCase;
import org.elasticsearch.test.ESTestCase;
import java.io.IOException;
public class GetRecordsRequestTests extends AbstractXContentTestCase<GetRecordsRequest> {
@Override
protected GetRecordsRequest createTestInstance() {
GetRecordsRequest request = new GetRecordsRequest(ESTestCase.randomAlphaOfLengthBetween(1, 20));
if (ESTestCase.randomBoolean()) {
request.setStart(String.valueOf(ESTestCase.randomLong()));
}
if (ESTestCase.randomBoolean()) {
request.setEnd(String.valueOf(ESTestCase.randomLong()));
}
if (ESTestCase.randomBoolean()) {
request.setExcludeInterim(ESTestCase.randomBoolean());
}
if (ESTestCase.randomBoolean()) {
request.setRecordScore(ESTestCase.randomDouble());
}
if (ESTestCase.randomBoolean()) {
int from = ESTestCase.randomInt(10000);
int size = ESTestCase.randomInt(10000);
request.setPageParams(new PageParams(from, size));
}
if (ESTestCase.randomBoolean()) {
request.setSort("anomaly_score");
}
if (ESTestCase.randomBoolean()) {
request.setDescending(ESTestCase.randomBoolean());
}
if (ESTestCase.randomBoolean()) {
request.setExcludeInterim(ESTestCase.randomBoolean());
}
return request;
}
@Override
protected GetRecordsRequest doParseInstance(XContentParser parser) throws IOException {
return GetRecordsRequest.PARSER.apply(parser, null);
}
@Override
protected boolean supportsUnknownFields() {
return false;
}
}

View File

@ -0,0 +1,54 @@
/*
* Licensed to Elasticsearch under one or more contributor
* license agreements. See the NOTICE file distributed with
* this work for additional information regarding copyright
* ownership. Elasticsearch licenses this file to you under
* the Apache License, Version 2.0 (the "License"); you may
* not use this file except in compliance with the License.
* You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing,
* software distributed under the License is distributed on an
* "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
* KIND, either express or implied. See the License for the
* specific language governing permissions and limitations
* under the License.
*/
package org.elasticsearch.client.ml;
import org.elasticsearch.client.ml.job.results.AnomalyRecord;
import org.elasticsearch.client.ml.job.results.AnomalyRecordTests;
import org.elasticsearch.common.xcontent.XContentParser;
import org.elasticsearch.test.AbstractXContentTestCase;
import org.elasticsearch.test.ESTestCase;
import java.io.IOException;
import java.util.ArrayList;
import java.util.List;
public class GetRecordsResponseTests extends AbstractXContentTestCase<GetRecordsResponse> {
@Override
protected GetRecordsResponse createTestInstance() {
String jobId = ESTestCase.randomAlphaOfLength(20);
int listSize = ESTestCase.randomInt(10);
List<AnomalyRecord> records = new ArrayList<>(listSize);
for (int j = 0; j < listSize; j++) {
AnomalyRecord record = AnomalyRecordTests.createTestInstance(jobId);
records.add(record);
}
return new GetRecordsResponse(records, listSize);
}
@Override
protected GetRecordsResponse doParseInstance(XContentParser parser) throws IOException {
return GetRecordsResponse.fromXContent(parser);
}
@Override
protected boolean supportsUnknownFields() {
return true;
}
}

View File

@ -16,11 +16,11 @@
* specific language governing permissions and limitations * specific language governing permissions and limitations
* under the License. * under the License.
*/ */
package org.elasticsearch.protocol.xpack.ml; package org.elasticsearch.client.ml;
import org.elasticsearch.client.ml.job.config.JobTests;
import org.elasticsearch.common.unit.TimeValue; import org.elasticsearch.common.unit.TimeValue;
import org.elasticsearch.common.xcontent.XContentParser; import org.elasticsearch.common.xcontent.XContentParser;
import org.elasticsearch.protocol.xpack.ml.job.config.JobTests;
import org.elasticsearch.test.AbstractXContentTestCase; import org.elasticsearch.test.AbstractXContentTestCase;
import java.io.IOException; import java.io.IOException;

View File

@ -16,7 +16,7 @@
* specific language governing permissions and limitations * specific language governing permissions and limitations
* under the License. * under the License.
*/ */
package org.elasticsearch.protocol.xpack.ml; package org.elasticsearch.client.ml;
import org.elasticsearch.common.xcontent.XContentParser; import org.elasticsearch.common.xcontent.XContentParser;
import org.elasticsearch.test.AbstractXContentTestCase; import org.elasticsearch.test.AbstractXContentTestCase;

View File

@ -16,11 +16,11 @@
* specific language governing permissions and limitations * specific language governing permissions and limitations
* under the License. * under the License.
*/ */
package org.elasticsearch.protocol.xpack.ml; package org.elasticsearch.client.ml;
import org.elasticsearch.client.ml.job.config.Job;
import org.elasticsearch.client.ml.job.config.JobTests;
import org.elasticsearch.common.xcontent.XContentParser; import org.elasticsearch.common.xcontent.XContentParser;
import org.elasticsearch.protocol.xpack.ml.job.config.Job;
import org.elasticsearch.protocol.xpack.ml.job.config.JobTests;
import org.elasticsearch.test.AbstractXContentTestCase; import org.elasticsearch.test.AbstractXContentTestCase;

View File

@ -16,10 +16,10 @@
* specific language governing permissions and limitations * specific language governing permissions and limitations
* under the License. * under the License.
*/ */
package org.elasticsearch.protocol.xpack.ml; package org.elasticsearch.client.ml;
import org.elasticsearch.client.ml.job.config.JobTests;
import org.elasticsearch.common.xcontent.XContentParser; import org.elasticsearch.common.xcontent.XContentParser;
import org.elasticsearch.protocol.xpack.ml.job.config.JobTests;
import org.elasticsearch.test.AbstractXContentTestCase; import org.elasticsearch.test.AbstractXContentTestCase;
import java.io.IOException; import java.io.IOException;

View File

@ -16,7 +16,7 @@
* specific language governing permissions and limitations * specific language governing permissions and limitations
* under the License. * under the License.
*/ */
package org.elasticsearch.protocol.xpack.ml.datafeed; package org.elasticsearch.client.ml.datafeed;
import org.elasticsearch.common.unit.TimeValue; import org.elasticsearch.common.unit.TimeValue;
import org.elasticsearch.common.xcontent.XContentParser; import org.elasticsearch.common.xcontent.XContentParser;

View File

@ -16,7 +16,7 @@
* specific language governing permissions and limitations * specific language governing permissions and limitations
* under the License. * under the License.
*/ */
package org.elasticsearch.protocol.xpack.ml.datafeed; package org.elasticsearch.client.ml.datafeed;
import com.carrotsearch.randomizedtesting.generators.CodepointSetGenerator; import com.carrotsearch.randomizedtesting.generators.CodepointSetGenerator;
import org.elasticsearch.common.settings.Settings; import org.elasticsearch.common.settings.Settings;

View File

@ -16,7 +16,7 @@
* specific language governing permissions and limitations * specific language governing permissions and limitations
* under the License. * under the License.
*/ */
package org.elasticsearch.protocol.xpack.ml.datafeed; package org.elasticsearch.client.ml.datafeed;
import org.elasticsearch.common.settings.Settings; import org.elasticsearch.common.settings.Settings;
import org.elasticsearch.common.unit.TimeValue; import org.elasticsearch.common.unit.TimeValue;

View File

@ -16,7 +16,7 @@
* specific language governing permissions and limitations * specific language governing permissions and limitations
* under the License. * under the License.
*/ */
package org.elasticsearch.protocol.xpack.ml.job.config; package org.elasticsearch.client.ml.job.config;
import org.elasticsearch.common.settings.Settings; import org.elasticsearch.common.settings.Settings;
import org.elasticsearch.common.unit.TimeValue; import org.elasticsearch.common.unit.TimeValue;

View File

@ -16,7 +16,7 @@
* specific language governing permissions and limitations * specific language governing permissions and limitations
* under the License. * under the License.
*/ */
package org.elasticsearch.protocol.xpack.ml.job.config; package org.elasticsearch.client.ml.job.config;
import org.elasticsearch.common.xcontent.DeprecationHandler; import org.elasticsearch.common.xcontent.DeprecationHandler;
import org.elasticsearch.common.xcontent.NamedXContentRegistry; import org.elasticsearch.common.xcontent.NamedXContentRegistry;

View File

@ -16,7 +16,7 @@
* specific language governing permissions and limitations * specific language governing permissions and limitations
* under the License. * under the License.
*/ */
package org.elasticsearch.protocol.xpack.ml.job.config; package org.elasticsearch.client.ml.job.config;
import org.elasticsearch.common.xcontent.XContentParser; import org.elasticsearch.common.xcontent.XContentParser;
import org.elasticsearch.test.AbstractXContentTestCase; import org.elasticsearch.test.AbstractXContentTestCase;

View File

@ -16,12 +16,12 @@
* specific language governing permissions and limitations * specific language governing permissions and limitations
* under the License. * under the License.
*/ */
package org.elasticsearch.protocol.xpack.ml.job.config; package org.elasticsearch.client.ml.job.config;
import org.elasticsearch.common.xcontent.XContentParser; import org.elasticsearch.common.xcontent.XContentParser;
import org.elasticsearch.protocol.xpack.ml.job.config.DataDescription.DataFormat;
import org.elasticsearch.test.AbstractXContentTestCase; import org.elasticsearch.test.AbstractXContentTestCase;
import static org.elasticsearch.client.ml.job.config.DataDescription.DataFormat;
import static org.hamcrest.Matchers.equalTo; import static org.hamcrest.Matchers.equalTo;
import static org.hamcrest.Matchers.nullValue; import static org.hamcrest.Matchers.nullValue;
import static org.hamcrest.core.Is.is; import static org.hamcrest.core.Is.is;

View File

@ -16,7 +16,7 @@
* specific language governing permissions and limitations * specific language governing permissions and limitations
* under the License. * under the License.
*/ */
package org.elasticsearch.protocol.xpack.ml.job.config; package org.elasticsearch.client.ml.job.config;
import org.elasticsearch.common.xcontent.XContentParser; import org.elasticsearch.common.xcontent.XContentParser;
import org.elasticsearch.test.AbstractXContentTestCase; import org.elasticsearch.test.AbstractXContentTestCase;

View File

@ -16,7 +16,7 @@
* specific language governing permissions and limitations * specific language governing permissions and limitations
* under the License. * under the License.
*/ */
package org.elasticsearch.protocol.xpack.ml.job.config; package org.elasticsearch.client.ml.job.config;
import org.elasticsearch.common.xcontent.XContentParser; import org.elasticsearch.common.xcontent.XContentParser;
import org.elasticsearch.test.AbstractXContentTestCase; import org.elasticsearch.test.AbstractXContentTestCase;

View File

@ -16,7 +16,7 @@
* specific language governing permissions and limitations * specific language governing permissions and limitations
* under the License. * under the License.
*/ */
package org.elasticsearch.protocol.xpack.ml.job.config; package org.elasticsearch.client.ml.job.config;
import org.elasticsearch.common.xcontent.XContentParser; import org.elasticsearch.common.xcontent.XContentParser;
import org.elasticsearch.test.AbstractXContentTestCase; import org.elasticsearch.test.AbstractXContentTestCase;

View File

@ -16,7 +16,7 @@
* specific language governing permissions and limitations * specific language governing permissions and limitations
* under the License. * under the License.
*/ */
package org.elasticsearch.protocol.xpack.ml.job.config; package org.elasticsearch.client.ml.job.config;
import com.carrotsearch.randomizedtesting.generators.CodepointSetGenerator; import com.carrotsearch.randomizedtesting.generators.CodepointSetGenerator;
import org.elasticsearch.common.settings.Settings; import org.elasticsearch.common.settings.Settings;

View File

@ -16,7 +16,7 @@
* specific language governing permissions and limitations * specific language governing permissions and limitations
* under the License. * under the License.
*/ */
package org.elasticsearch.protocol.xpack.ml.job.config; package org.elasticsearch.client.ml.job.config;
import org.elasticsearch.common.xcontent.XContentParser; import org.elasticsearch.common.xcontent.XContentParser;
import org.elasticsearch.test.AbstractXContentTestCase; import org.elasticsearch.test.AbstractXContentTestCase;

View File

@ -16,7 +16,7 @@
* specific language governing permissions and limitations * specific language governing permissions and limitations
* under the License. * under the License.
*/ */
package org.elasticsearch.protocol.xpack.ml.job.config; package org.elasticsearch.client.ml.job.config;
import org.elasticsearch.common.xcontent.XContentParser; import org.elasticsearch.common.xcontent.XContentParser;
import org.elasticsearch.test.AbstractXContentTestCase; import org.elasticsearch.test.AbstractXContentTestCase;

View File

@ -16,7 +16,7 @@
* specific language governing permissions and limitations * specific language governing permissions and limitations
* under the License. * under the License.
*/ */
package org.elasticsearch.protocol.xpack.ml.job.config; package org.elasticsearch.client.ml.job.config;
import org.elasticsearch.common.xcontent.XContentParser; import org.elasticsearch.common.xcontent.XContentParser;
import org.elasticsearch.test.AbstractXContentTestCase; import org.elasticsearch.test.AbstractXContentTestCase;

View File

@ -16,7 +16,7 @@
* specific language governing permissions and limitations * specific language governing permissions and limitations
* under the License. * under the License.
*/ */
package org.elasticsearch.protocol.xpack.ml.job.config; package org.elasticsearch.client.ml.job.config;
import org.elasticsearch.common.xcontent.XContentParser; import org.elasticsearch.common.xcontent.XContentParser;
import org.elasticsearch.test.AbstractXContentTestCase; import org.elasticsearch.test.AbstractXContentTestCase;

View File

@ -16,7 +16,7 @@
* specific language governing permissions and limitations * specific language governing permissions and limitations
* under the License. * under the License.
*/ */
package org.elasticsearch.protocol.xpack.ml.job.process; package org.elasticsearch.client.ml.job.process;
import org.elasticsearch.common.xcontent.XContentParser; import org.elasticsearch.common.xcontent.XContentParser;
import org.elasticsearch.test.AbstractXContentTestCase; import org.elasticsearch.test.AbstractXContentTestCase;
@ -113,7 +113,7 @@ public class DataCountsTests extends AbstractXContentTestCase<DataCounts> {
private static DataCounts createCounts( private static DataCounts createCounts(
long processedRecordCount, long processedFieldCount, long inputBytes, long inputFieldCount, long processedRecordCount, long processedFieldCount, long inputBytes, long inputFieldCount,
long invalidDateCount, long missingFieldCount, long outOfOrderTimeStampCount, long invalidDateCount, long missingFieldCount, long outOfOrderTimeStampCount,
long emptyBucketCount, long sparseBucketCount, long bucketCount, long emptyBucketCount, long sparseBucketCount, long bucketCount,
long earliestRecordTime, long latestRecordTime, long lastDataTimeStamp, long latestEmptyBucketTimeStamp, long earliestRecordTime, long latestRecordTime, long lastDataTimeStamp, long latestEmptyBucketTimeStamp,
long latestSparseBucketTimeStamp) { long latestSparseBucketTimeStamp) {

View File

@ -16,15 +16,16 @@
* specific language governing permissions and limitations * specific language governing permissions and limitations
* under the License. * under the License.
*/ */
package org.elasticsearch.protocol.xpack.ml.job.process; package org.elasticsearch.client.ml.job.process;
import org.elasticsearch.common.unit.TimeValue; import org.elasticsearch.common.unit.TimeValue;
import org.elasticsearch.common.xcontent.XContentParser; import org.elasticsearch.common.xcontent.XContentParser;
import org.elasticsearch.test.AbstractXContentTestCase; import org.elasticsearch.test.AbstractXContentTestCase;
import org.elasticsearch.protocol.xpack.ml.job.process.ModelSizeStats.MemoryStatus;
import java.util.Date; import java.util.Date;
import static org.elasticsearch.client.ml.job.process.ModelSizeStats.MemoryStatus;
public class ModelSizeStatsTests extends AbstractXContentTestCase<ModelSizeStats> { public class ModelSizeStatsTests extends AbstractXContentTestCase<ModelSizeStats> {
public void testDefaultConstructor() { public void testDefaultConstructor() {

View File

@ -16,7 +16,7 @@
* specific language governing permissions and limitations * specific language governing permissions and limitations
* under the License. * under the License.
*/ */
package org.elasticsearch.protocol.xpack.ml.job.process; package org.elasticsearch.client.ml.job.process;
import org.elasticsearch.Version; import org.elasticsearch.Version;
import org.elasticsearch.common.unit.TimeValue; import org.elasticsearch.common.unit.TimeValue;

View File

@ -16,7 +16,7 @@
* specific language governing permissions and limitations * specific language governing permissions and limitations
* under the License. * under the License.
*/ */
package org.elasticsearch.protocol.xpack.ml.job.process; package org.elasticsearch.client.ml.job.process;
import org.elasticsearch.common.unit.TimeValue; import org.elasticsearch.common.unit.TimeValue;
import org.elasticsearch.common.xcontent.XContentParser; import org.elasticsearch.common.xcontent.XContentParser;

View File

@ -16,7 +16,7 @@
* specific language governing permissions and limitations * specific language governing permissions and limitations
* under the License. * under the License.
*/ */
package org.elasticsearch.protocol.xpack.ml.job.results; package org.elasticsearch.client.ml.job.results;
import org.elasticsearch.common.xcontent.XContentParser; import org.elasticsearch.common.xcontent.XContentParser;
import org.elasticsearch.test.AbstractXContentTestCase; import org.elasticsearch.test.AbstractXContentTestCase;

View File

@ -16,7 +16,7 @@
* specific language governing permissions and limitations * specific language governing permissions and limitations
* under the License. * under the License.
*/ */
package org.elasticsearch.protocol.xpack.ml.job.results; package org.elasticsearch.client.ml.job.results;
import org.elasticsearch.common.xcontent.XContentParser; import org.elasticsearch.common.xcontent.XContentParser;
import org.elasticsearch.test.AbstractXContentTestCase; import org.elasticsearch.test.AbstractXContentTestCase;
@ -33,7 +33,7 @@ public class AnomalyRecordTests extends AbstractXContentTestCase<AnomalyRecord>
return createTestInstance("foo"); return createTestInstance("foo");
} }
public AnomalyRecord createTestInstance(String jobId) { public static AnomalyRecord createTestInstance(String jobId) {
AnomalyRecord anomalyRecord = new AnomalyRecord(jobId, new Date(randomNonNegativeLong()), randomNonNegativeLong()); AnomalyRecord anomalyRecord = new AnomalyRecord(jobId, new Date(randomNonNegativeLong()), randomNonNegativeLong());
anomalyRecord.setActual(Collections.singletonList(randomDouble())); anomalyRecord.setActual(Collections.singletonList(randomDouble()));
anomalyRecord.setTypical(Collections.singletonList(randomDouble())); anomalyRecord.setTypical(Collections.singletonList(randomDouble()));

View File

@ -16,7 +16,7 @@
* specific language governing permissions and limitations * specific language governing permissions and limitations
* under the License. * under the License.
*/ */
package org.elasticsearch.protocol.xpack.ml.job.results; package org.elasticsearch.client.ml.job.results;
import org.elasticsearch.common.xcontent.XContentParser; import org.elasticsearch.common.xcontent.XContentParser;
import org.elasticsearch.test.AbstractXContentTestCase; import org.elasticsearch.test.AbstractXContentTestCase;

Some files were not shown because too many files have changed in this diff Show More