Merge branch 'master' into ccr

* master:
  Match phrase queries against non-indexed fields should throw an exception (#31060)
  In the internal highlighter APIs, use the field type as opposed to the mapper. (#31039)
  [DOCS] Removes duplicated authentication pages
  Enable customizing REST tests blacklist (#31074)
  Make sure KeywordFieldMapper#clone preserves split_queries_on_whitespace. (#31049)
  [DOCS] Moves machine learning overview to stack-docs
  [ML] Add secondary sort to ML events (#31063)
  [Rollup] Specialize validation exception for easier management (#30339)
  Adapt bwc versions after backporting #31045 to 6.3
  Remove usage of explicit type in docs (#29667)
  Share common readFrom/writeTo code in AcknowledgeResponse (#30983)
  Adapt bwc versions after backporting #31045 to 6.x
  Mute MatchPhrase*QueryBuilderTests
  [Docs] Fix typo in watcher conditions documentation (#30989)
  Remove wrong link in index phrases doc
  Move pipeline APIs to ingest namespace (#31027)
  [DOCS] Fixes accounting setting names (#30863)
  [DOCS] Rewords _field_names documentation (#31029)
  Index phrases (#30450)
  Remove leftover debugging from PTCMDT
  Fix PTCMDT#testMinVersionSerialization
  Make Persistent Tasks implementations version and feature aware (#31045)
This commit is contained in:
Nhat Nguyen 2018-06-04 14:32:28 -04:00
commit d9a0a02e89
147 changed files with 1682 additions and 2548 deletions

View File

@ -23,11 +23,6 @@ import org.apache.http.Header;
import org.elasticsearch.action.ActionListener;
import org.elasticsearch.action.admin.cluster.settings.ClusterUpdateSettingsRequest;
import org.elasticsearch.action.admin.cluster.settings.ClusterUpdateSettingsResponse;
import org.elasticsearch.action.ingest.PutPipelineRequest;
import org.elasticsearch.action.ingest.GetPipelineRequest;
import org.elasticsearch.action.ingest.GetPipelineResponse;
import org.elasticsearch.action.ingest.DeletePipelineRequest;
import org.elasticsearch.action.ingest.WritePipelineResponse;
import java.io.IOException;
@ -68,72 +63,4 @@ public final class ClusterClient {
restHighLevelClient.performRequestAsyncAndParseEntity(clusterUpdateSettingsRequest, RequestConverters::clusterPutSettings,
ClusterUpdateSettingsResponse::fromXContent, listener, emptySet(), headers);
}
/**
* Add a pipeline or update an existing pipeline in the cluster
* <p>
* See
* <a href="https://www.elastic.co/guide/en/elasticsearch/reference/current/put-pipeline-api.html"> Put Pipeline API on elastic.co</a>
*/
public WritePipelineResponse putPipeline(PutPipelineRequest request, Header... headers) throws IOException {
return restHighLevelClient.performRequestAndParseEntity( request, RequestConverters::putPipeline,
WritePipelineResponse::fromXContent, emptySet(), headers);
}
/**
* Asynchronously add a pipeline or update an existing pipeline in the cluster
* <p>
* See
* <a href="https://www.elastic.co/guide/en/elasticsearch/reference/current/put-pipeline-api.html"> Put Pipeline API on elastic.co</a>
*/
public void putPipelineAsync(PutPipelineRequest request, ActionListener<WritePipelineResponse> listener, Header... headers) {
restHighLevelClient.performRequestAsyncAndParseEntity( request, RequestConverters::putPipeline,
WritePipelineResponse::fromXContent, listener, emptySet(), headers);
}
/**
* Get an existing pipeline
* <p>
* See
* <a href="https://www.elastic.co/guide/en/elasticsearch/reference/current/get-pipeline-api.html"> Get Pipeline API on elastic.co</a>
*/
public GetPipelineResponse getPipeline(GetPipelineRequest request, Header... headers) throws IOException {
return restHighLevelClient.performRequestAndParseEntity( request, RequestConverters::getPipeline,
GetPipelineResponse::fromXContent, emptySet(), headers);
}
/**
* Asynchronously get an existing pipeline
* <p>
* See
* <a href="https://www.elastic.co/guide/en/elasticsearch/reference/master/get-pipeline-api.html"> Get Pipeline API on elastic.co</a>
*/
public void getPipelineAsync(GetPipelineRequest request, ActionListener<GetPipelineResponse> listener, Header... headers) {
restHighLevelClient.performRequestAsyncAndParseEntity( request, RequestConverters::getPipeline,
GetPipelineResponse::fromXContent, listener, emptySet(), headers);
}
/**
* Delete an existing pipeline
* <p>
* See
* <a href="https://www.elastic.co/guide/en/elasticsearch/reference/current/delete-pipeline-api.html">
* Delete Pipeline API on elastic.co</a>
*/
public WritePipelineResponse deletePipeline(DeletePipelineRequest request, Header... headers) throws IOException {
return restHighLevelClient.performRequestAndParseEntity( request, RequestConverters::deletePipeline,
WritePipelineResponse::fromXContent, emptySet(), headers);
}
/**
* Asynchronously delete an existing pipeline
* <p>
* See
* <a href="https://www.elastic.co/guide/en/elasticsearch/reference/current/delete-pipeline-api.html">
* Delete Pipeline API on elastic.co</a>
*/
public void deletePipelineAsync(DeletePipelineRequest request, ActionListener<WritePipelineResponse> listener, Header... headers) {
restHighLevelClient.performRequestAsyncAndParseEntity( request, RequestConverters::deletePipeline,
WritePipelineResponse::fromXContent, listener, emptySet(), headers);
}
}

View File

@ -0,0 +1,114 @@
/*
* 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.apache.http.Header;
import org.elasticsearch.action.ActionListener;
import org.elasticsearch.action.ingest.DeletePipelineRequest;
import org.elasticsearch.action.ingest.GetPipelineRequest;
import org.elasticsearch.action.ingest.GetPipelineResponse;
import org.elasticsearch.action.ingest.PutPipelineRequest;
import org.elasticsearch.action.ingest.WritePipelineResponse;
import java.io.IOException;
import static java.util.Collections.emptySet;
/**
* A wrapper for the {@link RestHighLevelClient} that provides methods for accessing the Ingest API.
* <p>
* See <a href="https://www.elastic.co/guide/en/elasticsearch/reference/current/ingest.html">Ingest API on elastic.co</a>
*/
public final class IngestClient {
private final RestHighLevelClient restHighLevelClient;
IngestClient(RestHighLevelClient restHighLevelClient) {
this.restHighLevelClient = restHighLevelClient;
}
/**
* Add a pipeline or update an existing pipeline
* <p>
* See
* <a href="https://www.elastic.co/guide/en/elasticsearch/reference/current/put-pipeline-api.html"> Put Pipeline API on elastic.co</a>
*/
public WritePipelineResponse putPipeline(PutPipelineRequest request, Header... headers) throws IOException {
return restHighLevelClient.performRequestAndParseEntity( request, RequestConverters::putPipeline,
WritePipelineResponse::fromXContent, emptySet(), headers);
}
/**
* Asynchronously add a pipeline or update an existing pipeline
* <p>
* See
* <a href="https://www.elastic.co/guide/en/elasticsearch/reference/current/put-pipeline-api.html"> Put Pipeline API on elastic.co</a>
*/
public void putPipelineAsync(PutPipelineRequest request, ActionListener<WritePipelineResponse> listener, Header... headers) {
restHighLevelClient.performRequestAsyncAndParseEntity( request, RequestConverters::putPipeline,
WritePipelineResponse::fromXContent, listener, emptySet(), headers);
}
/**
* Get an existing pipeline
* <p>
* See
* <a href="https://www.elastic.co/guide/en/elasticsearch/reference/current/get-pipeline-api.html"> Get Pipeline API on elastic.co</a>
*/
public GetPipelineResponse getPipeline(GetPipelineRequest request, Header... headers) throws IOException {
return restHighLevelClient.performRequestAndParseEntity( request, RequestConverters::getPipeline,
GetPipelineResponse::fromXContent, emptySet(), headers);
}
/**
* Asynchronously get an existing pipeline
* <p>
* See
* <a href="https://www.elastic.co/guide/en/elasticsearch/reference/master/get-pipeline-api.html"> Get Pipeline API on elastic.co</a>
*/
public void getPipelineAsync(GetPipelineRequest request, ActionListener<GetPipelineResponse> listener, Header... headers) {
restHighLevelClient.performRequestAsyncAndParseEntity( request, RequestConverters::getPipeline,
GetPipelineResponse::fromXContent, listener, emptySet(), headers);
}
/**
* Delete an existing pipeline
* <p>
* See
* <a href="https://www.elastic.co/guide/en/elasticsearch/reference/current/delete-pipeline-api.html">
* Delete Pipeline API on elastic.co</a>
*/
public WritePipelineResponse deletePipeline(DeletePipelineRequest request, Header... headers) throws IOException {
return restHighLevelClient.performRequestAndParseEntity( request, RequestConverters::deletePipeline,
WritePipelineResponse::fromXContent, emptySet(), headers);
}
/**
* Asynchronously delete an existing pipeline
* <p>
* See
* <a href="https://www.elastic.co/guide/en/elasticsearch/reference/current/delete-pipeline-api.html">
* Delete Pipeline API on elastic.co</a>
*/
public void deletePipelineAsync(DeletePipelineRequest request, ActionListener<WritePipelineResponse> listener, Header... headers) {
restHighLevelClient.performRequestAsyncAndParseEntity( request, RequestConverters::deletePipeline,
WritePipelineResponse::fromXContent, listener, emptySet(), headers);
}
}

View File

@ -191,6 +191,7 @@ public class RestHighLevelClient implements Closeable {
private final IndicesClient indicesClient = new IndicesClient(this);
private final ClusterClient clusterClient = new ClusterClient(this);
private final IngestClient ingestClient = new IngestClient(this);
private final SnapshotClient snapshotClient = new SnapshotClient(this);
private final TasksClient tasksClient = new TasksClient(this);
@ -256,6 +257,15 @@ public class RestHighLevelClient implements Closeable {
return clusterClient;
}
/**
* Provides a {@link IngestClient} which can be used to access the Ingest API.
*
* See <a href="https://www.elastic.co/guide/en/elasticsearch/reference/current/ingest.html">Ingest API on elastic.co</a>
*/
public final IngestClient ingest() {
return ingestClient;
}
/**
* Provides a {@link SnapshotClient} which can be used to access the Snapshot API.
*

View File

@ -22,20 +22,12 @@ package org.elasticsearch.client;
import org.elasticsearch.ElasticsearchException;
import org.elasticsearch.action.admin.cluster.settings.ClusterUpdateSettingsRequest;
import org.elasticsearch.action.admin.cluster.settings.ClusterUpdateSettingsResponse;
import org.elasticsearch.action.ingest.GetPipelineRequest;
import org.elasticsearch.action.ingest.GetPipelineResponse;
import org.elasticsearch.action.ingest.PutPipelineRequest;
import org.elasticsearch.action.ingest.DeletePipelineRequest;
import org.elasticsearch.action.ingest.WritePipelineResponse;
import org.elasticsearch.cluster.routing.allocation.decider.EnableAllocationDecider;
import org.elasticsearch.common.bytes.BytesReference;
import org.elasticsearch.common.settings.Settings;
import org.elasticsearch.common.unit.ByteSizeUnit;
import org.elasticsearch.common.xcontent.XContentBuilder;
import org.elasticsearch.common.xcontent.XContentType;
import org.elasticsearch.common.xcontent.support.XContentMapValues;
import org.elasticsearch.indices.recovery.RecoverySettings;
import org.elasticsearch.ingest.PipelineConfiguration;
import org.elasticsearch.rest.RestStatus;
import java.io.IOException;
@ -113,53 +105,4 @@ public class ClusterClientIT extends ESRestHighLevelClientTestCase {
assertThat(exception.getMessage(), equalTo(
"Elasticsearch exception [type=illegal_argument_exception, reason=transient setting [" + setting + "], not recognized]"));
}
public void testPutPipeline() throws IOException {
String id = "some_pipeline_id";
XContentBuilder pipelineBuilder = buildRandomXContentPipeline();
PutPipelineRequest request = new PutPipelineRequest(
id,
BytesReference.bytes(pipelineBuilder),
pipelineBuilder.contentType());
WritePipelineResponse putPipelineResponse =
execute(request, highLevelClient().cluster()::putPipeline, highLevelClient().cluster()::putPipelineAsync);
assertTrue(putPipelineResponse.isAcknowledged());
}
public void testGetPipeline() throws IOException {
String id = "some_pipeline_id";
XContentBuilder pipelineBuilder = buildRandomXContentPipeline();
{
PutPipelineRequest request = new PutPipelineRequest(
id,
BytesReference.bytes(pipelineBuilder),
pipelineBuilder.contentType()
);
createPipeline(request);
}
GetPipelineRequest request = new GetPipelineRequest(id);
GetPipelineResponse response =
execute(request, highLevelClient().cluster()::getPipeline, highLevelClient().cluster()::getPipelineAsync);
assertTrue(response.isFound());
assertEquals(response.pipelines().get(0).getId(), id);
PipelineConfiguration expectedConfig =
new PipelineConfiguration(id, BytesReference.bytes(pipelineBuilder), pipelineBuilder.contentType());
assertEquals(expectedConfig.getConfigAsMap(), response.pipelines().get(0).getConfigAsMap());
}
public void testDeletePipeline() throws IOException {
String id = "some_pipeline_id";
{
createPipeline(id);
}
DeletePipelineRequest request = new DeletePipelineRequest(id);
WritePipelineResponse response =
execute(request, highLevelClient().cluster()::deletePipeline, highLevelClient().cluster()::deletePipelineAsync);
assertTrue(response.isAcknowledged());
}
}

View File

@ -0,0 +1,83 @@
/*
* 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.action.ingest.DeletePipelineRequest;
import org.elasticsearch.action.ingest.GetPipelineRequest;
import org.elasticsearch.action.ingest.GetPipelineResponse;
import org.elasticsearch.action.ingest.PutPipelineRequest;
import org.elasticsearch.action.ingest.WritePipelineResponse;
import org.elasticsearch.common.bytes.BytesReference;
import org.elasticsearch.common.xcontent.XContentBuilder;
import org.elasticsearch.ingest.PipelineConfiguration;
import java.io.IOException;
public class IngestClientIT extends ESRestHighLevelClientTestCase {
public void testPutPipeline() throws IOException {
String id = "some_pipeline_id";
XContentBuilder pipelineBuilder = buildRandomXContentPipeline();
PutPipelineRequest request = new PutPipelineRequest(
id,
BytesReference.bytes(pipelineBuilder),
pipelineBuilder.contentType());
WritePipelineResponse putPipelineResponse =
execute(request, highLevelClient().ingest()::putPipeline, highLevelClient().ingest()::putPipelineAsync);
assertTrue(putPipelineResponse.isAcknowledged());
}
public void testGetPipeline() throws IOException {
String id = "some_pipeline_id";
XContentBuilder pipelineBuilder = buildRandomXContentPipeline();
{
PutPipelineRequest request = new PutPipelineRequest(
id,
BytesReference.bytes(pipelineBuilder),
pipelineBuilder.contentType()
);
createPipeline(request);
}
GetPipelineRequest request = new GetPipelineRequest(id);
GetPipelineResponse response =
execute(request, highLevelClient().ingest()::getPipeline, highLevelClient().ingest()::getPipelineAsync);
assertTrue(response.isFound());
assertEquals(response.pipelines().get(0).getId(), id);
PipelineConfiguration expectedConfig =
new PipelineConfiguration(id, BytesReference.bytes(pipelineBuilder), pipelineBuilder.contentType());
assertEquals(expectedConfig.getConfigAsMap(), response.pipelines().get(0).getConfigAsMap());
}
public void testDeletePipeline() throws IOException {
String id = "some_pipeline_id";
{
createPipeline(id);
}
DeletePipelineRequest request = new DeletePipelineRequest(id);
WritePipelineResponse response =
execute(request, highLevelClient().ingest()::deletePipeline, highLevelClient().ingest()::deletePipelineAsync);
assertTrue(response.isAcknowledged());
}
}

View File

@ -186,220 +186,4 @@ public class ClusterClientDocumentationIT extends ESRestHighLevelClientTestCase
assertTrue(latch.await(30L, TimeUnit.SECONDS));
}
}
public void testPutPipeline() throws IOException {
RestHighLevelClient client = highLevelClient();
{
// tag::put-pipeline-request
String source =
"{\"description\":\"my set of processors\"," +
"\"processors\":[{\"set\":{\"field\":\"foo\",\"value\":\"bar\"}}]}";
PutPipelineRequest request = new PutPipelineRequest(
"my-pipeline-id", // <1>
new BytesArray(source.getBytes(StandardCharsets.UTF_8)), // <2>
XContentType.JSON // <3>
);
// end::put-pipeline-request
// tag::put-pipeline-request-timeout
request.timeout(TimeValue.timeValueMinutes(2)); // <1>
request.timeout("2m"); // <2>
// end::put-pipeline-request-timeout
// tag::put-pipeline-request-masterTimeout
request.masterNodeTimeout(TimeValue.timeValueMinutes(1)); // <1>
request.masterNodeTimeout("1m"); // <2>
// end::put-pipeline-request-masterTimeout
// tag::put-pipeline-execute
WritePipelineResponse response = client.cluster().putPipeline(request); // <1>
// end::put-pipeline-execute
// tag::put-pipeline-response
boolean acknowledged = response.isAcknowledged(); // <1>
// end::put-pipeline-response
assertTrue(acknowledged);
}
}
public void testPutPipelineAsync() throws Exception {
RestHighLevelClient client = highLevelClient();
{
String source =
"{\"description\":\"my set of processors\"," +
"\"processors\":[{\"set\":{\"field\":\"foo\",\"value\":\"bar\"}}]}";
PutPipelineRequest request = new PutPipelineRequest(
"my-pipeline-id",
new BytesArray(source.getBytes(StandardCharsets.UTF_8)),
XContentType.JSON
);
// tag::put-pipeline-execute-listener
ActionListener<WritePipelineResponse> listener =
new ActionListener<WritePipelineResponse>() {
@Override
public void onResponse(WritePipelineResponse response) {
// <1>
}
@Override
public void onFailure(Exception e) {
// <2>
}
};
// end::put-pipeline-execute-listener
// Replace the empty listener by a blocking listener in test
final CountDownLatch latch = new CountDownLatch(1);
listener = new LatchedActionListener<>(listener, latch);
// tag::put-pipeline-execute-async
client.cluster().putPipelineAsync(request, listener); // <1>
// end::put-pipeline-execute-async
assertTrue(latch.await(30L, TimeUnit.SECONDS));
}
}
public void testGetPipeline() throws IOException {
RestHighLevelClient client = highLevelClient();
{
createPipeline("my-pipeline-id");
}
{
// tag::get-pipeline-request
GetPipelineRequest request = new GetPipelineRequest("my-pipeline-id"); // <1>
// end::get-pipeline-request
// tag::get-pipeline-request-masterTimeout
request.masterNodeTimeout(TimeValue.timeValueMinutes(1)); // <1>
request.masterNodeTimeout("1m"); // <2>
// end::get-pipeline-request-masterTimeout
// tag::get-pipeline-execute
GetPipelineResponse response = client.cluster().getPipeline(request); // <1>
// end::get-pipeline-execute
// tag::get-pipeline-response
boolean successful = response.isFound(); // <1>
List<PipelineConfiguration> pipelines = response.pipelines(); // <2>
for(PipelineConfiguration pipeline: pipelines) {
Map<String, Object> config = pipeline.getConfigAsMap(); // <3>
}
// end::get-pipeline-response
assertTrue(successful);
}
}
public void testGetPipelineAsync() throws Exception {
RestHighLevelClient client = highLevelClient();
{
createPipeline("my-pipeline-id");
}
{
GetPipelineRequest request = new GetPipelineRequest("my-pipeline-id");
// tag::get-pipeline-execute-listener
ActionListener<GetPipelineResponse> listener =
new ActionListener<GetPipelineResponse>() {
@Override
public void onResponse(GetPipelineResponse response) {
// <1>
}
@Override
public void onFailure(Exception e) {
// <2>
}
};
// end::get-pipeline-execute-listener
// Replace the empty listener by a blocking listener in test
final CountDownLatch latch = new CountDownLatch(1);
listener = new LatchedActionListener<>(listener, latch);
// tag::get-pipeline-execute-async
client.cluster().getPipelineAsync(request, listener); // <1>
// end::get-pipeline-execute-async
assertTrue(latch.await(30L, TimeUnit.SECONDS));
}
}
public void testDeletePipeline() throws IOException {
RestHighLevelClient client = highLevelClient();
{
createPipeline("my-pipeline-id");
}
{
// tag::delete-pipeline-request
DeletePipelineRequest request = new DeletePipelineRequest("my-pipeline-id"); // <1>
// end::delete-pipeline-request
// tag::delete-pipeline-request-timeout
request.timeout(TimeValue.timeValueMinutes(2)); // <1>
request.timeout("2m"); // <2>
// end::delete-pipeline-request-timeout
// tag::delete-pipeline-request-masterTimeout
request.masterNodeTimeout(TimeValue.timeValueMinutes(1)); // <1>
request.masterNodeTimeout("1m"); // <2>
// end::delete-pipeline-request-masterTimeout
// tag::delete-pipeline-execute
WritePipelineResponse response = client.cluster().deletePipeline(request); // <1>
// end::delete-pipeline-execute
// tag::delete-pipeline-response
boolean acknowledged = response.isAcknowledged(); // <1>
// end::delete-pipeline-response
assertTrue(acknowledged);
}
}
public void testDeletePipelineAsync() throws Exception {
RestHighLevelClient client = highLevelClient();
{
createPipeline("my-pipeline-id");
}
{
DeletePipelineRequest request = new DeletePipelineRequest("my-pipeline-id");
// tag::delete-pipeline-execute-listener
ActionListener<WritePipelineResponse> listener =
new ActionListener<WritePipelineResponse>() {
@Override
public void onResponse(WritePipelineResponse response) {
// <1>
}
@Override
public void onFailure(Exception e) {
// <2>
}
};
// end::delete-pipeline-execute-listener
// Replace the empty listener by a blocking listener in test
final CountDownLatch latch = new CountDownLatch(1);
listener = new LatchedActionListener<>(listener, latch);
// tag::delete-pipeline-execute-async
client.cluster().deletePipelineAsync(request, listener); // <1>
// end::delete-pipeline-execute-async
assertTrue(latch.await(30L, TimeUnit.SECONDS));
}
}
}

View File

@ -0,0 +1,279 @@
/*
* 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.documentation;
import org.elasticsearch.action.ActionListener;
import org.elasticsearch.action.LatchedActionListener;
import org.elasticsearch.action.ingest.DeletePipelineRequest;
import org.elasticsearch.action.ingest.GetPipelineRequest;
import org.elasticsearch.action.ingest.GetPipelineResponse;
import org.elasticsearch.action.ingest.PutPipelineRequest;
import org.elasticsearch.action.ingest.WritePipelineResponse;
import org.elasticsearch.client.ESRestHighLevelClientTestCase;
import org.elasticsearch.client.RestHighLevelClient;
import org.elasticsearch.common.bytes.BytesArray;
import org.elasticsearch.common.unit.TimeValue;
import org.elasticsearch.common.xcontent.XContentType;
import org.elasticsearch.ingest.PipelineConfiguration;
import java.io.IOException;
import java.nio.charset.StandardCharsets;
import java.util.List;
import java.util.Map;
import java.util.concurrent.CountDownLatch;
import java.util.concurrent.TimeUnit;
/**
* This class is used to generate the Java Ingest API documentation.
* You need to wrap your code between two tags like:
* // tag::example
* // end::example
*
* Where example is your tag name.
*
* Then in the documentation, you can extract what is between tag and end tags with
* ["source","java",subs="attributes,callouts,macros"]
* --------------------------------------------------
* include-tagged::{doc-tests}/IngestClientDocumentationIT.java[example]
* --------------------------------------------------
*
* The column width of the code block is 84. If the code contains a line longer
* than 84, the line will be cut and a horizontal scroll bar will be displayed.
* (the code indentation of the tag is not included in the width)
*/
public class IngestClientDocumentationIT extends ESRestHighLevelClientTestCase {
public void testPutPipeline() throws IOException {
RestHighLevelClient client = highLevelClient();
{
// tag::put-pipeline-request
String source =
"{\"description\":\"my set of processors\"," +
"\"processors\":[{\"set\":{\"field\":\"foo\",\"value\":\"bar\"}}]}";
PutPipelineRequest request = new PutPipelineRequest(
"my-pipeline-id", // <1>
new BytesArray(source.getBytes(StandardCharsets.UTF_8)), // <2>
XContentType.JSON // <3>
);
// end::put-pipeline-request
// tag::put-pipeline-request-timeout
request.timeout(TimeValue.timeValueMinutes(2)); // <1>
request.timeout("2m"); // <2>
// end::put-pipeline-request-timeout
// tag::put-pipeline-request-masterTimeout
request.masterNodeTimeout(TimeValue.timeValueMinutes(1)); // <1>
request.masterNodeTimeout("1m"); // <2>
// end::put-pipeline-request-masterTimeout
// tag::put-pipeline-execute
WritePipelineResponse response = client.ingest().putPipeline(request); // <1>
// end::put-pipeline-execute
// tag::put-pipeline-response
boolean acknowledged = response.isAcknowledged(); // <1>
// end::put-pipeline-response
assertTrue(acknowledged);
}
}
public void testPutPipelineAsync() throws Exception {
RestHighLevelClient client = highLevelClient();
{
String source =
"{\"description\":\"my set of processors\"," +
"\"processors\":[{\"set\":{\"field\":\"foo\",\"value\":\"bar\"}}]}";
PutPipelineRequest request = new PutPipelineRequest(
"my-pipeline-id",
new BytesArray(source.getBytes(StandardCharsets.UTF_8)),
XContentType.JSON
);
// tag::put-pipeline-execute-listener
ActionListener<WritePipelineResponse> listener =
new ActionListener<WritePipelineResponse>() {
@Override
public void onResponse(WritePipelineResponse response) {
// <1>
}
@Override
public void onFailure(Exception e) {
// <2>
}
};
// end::put-pipeline-execute-listener
// Replace the empty listener by a blocking listener in test
final CountDownLatch latch = new CountDownLatch(1);
listener = new LatchedActionListener<>(listener, latch);
// tag::put-pipeline-execute-async
client.ingest().putPipelineAsync(request, listener); // <1>
// end::put-pipeline-execute-async
assertTrue(latch.await(30L, TimeUnit.SECONDS));
}
}
public void testGetPipeline() throws IOException {
RestHighLevelClient client = highLevelClient();
{
createPipeline("my-pipeline-id");
}
{
// tag::get-pipeline-request
GetPipelineRequest request = new GetPipelineRequest("my-pipeline-id"); // <1>
// end::get-pipeline-request
// tag::get-pipeline-request-masterTimeout
request.masterNodeTimeout(TimeValue.timeValueMinutes(1)); // <1>
request.masterNodeTimeout("1m"); // <2>
// end::get-pipeline-request-masterTimeout
// tag::get-pipeline-execute
GetPipelineResponse response = client.ingest().getPipeline(request); // <1>
// end::get-pipeline-execute
// tag::get-pipeline-response
boolean successful = response.isFound(); // <1>
List<PipelineConfiguration> pipelines = response.pipelines(); // <2>
for(PipelineConfiguration pipeline: pipelines) {
Map<String, Object> config = pipeline.getConfigAsMap(); // <3>
}
// end::get-pipeline-response
assertTrue(successful);
}
}
public void testGetPipelineAsync() throws Exception {
RestHighLevelClient client = highLevelClient();
{
createPipeline("my-pipeline-id");
}
{
GetPipelineRequest request = new GetPipelineRequest("my-pipeline-id");
// tag::get-pipeline-execute-listener
ActionListener<GetPipelineResponse> listener =
new ActionListener<GetPipelineResponse>() {
@Override
public void onResponse(GetPipelineResponse response) {
// <1>
}
@Override
public void onFailure(Exception e) {
// <2>
}
};
// end::get-pipeline-execute-listener
// Replace the empty listener by a blocking listener in test
final CountDownLatch latch = new CountDownLatch(1);
listener = new LatchedActionListener<>(listener, latch);
// tag::get-pipeline-execute-async
client.ingest().getPipelineAsync(request, listener); // <1>
// end::get-pipeline-execute-async
assertTrue(latch.await(30L, TimeUnit.SECONDS));
}
}
public void testDeletePipeline() throws IOException {
RestHighLevelClient client = highLevelClient();
{
createPipeline("my-pipeline-id");
}
{
// tag::delete-pipeline-request
DeletePipelineRequest request = new DeletePipelineRequest("my-pipeline-id"); // <1>
// end::delete-pipeline-request
// tag::delete-pipeline-request-timeout
request.timeout(TimeValue.timeValueMinutes(2)); // <1>
request.timeout("2m"); // <2>
// end::delete-pipeline-request-timeout
// tag::delete-pipeline-request-masterTimeout
request.masterNodeTimeout(TimeValue.timeValueMinutes(1)); // <1>
request.masterNodeTimeout("1m"); // <2>
// end::delete-pipeline-request-masterTimeout
// tag::delete-pipeline-execute
WritePipelineResponse response = client.ingest().deletePipeline(request); // <1>
// end::delete-pipeline-execute
// tag::delete-pipeline-response
boolean acknowledged = response.isAcknowledged(); // <1>
// end::delete-pipeline-response
assertTrue(acknowledged);
}
}
public void testDeletePipelineAsync() throws Exception {
RestHighLevelClient client = highLevelClient();
{
createPipeline("my-pipeline-id");
}
{
DeletePipelineRequest request = new DeletePipelineRequest("my-pipeline-id");
// tag::delete-pipeline-execute-listener
ActionListener<WritePipelineResponse> listener =
new ActionListener<WritePipelineResponse>() {
@Override
public void onResponse(WritePipelineResponse response) {
// <1>
}
@Override
public void onFailure(Exception e) {
// <2>
}
};
// end::delete-pipeline-execute-listener
// Replace the empty listener by a blocking listener in test
final CountDownLatch latch = new CountDownLatch(1);
listener = new LatchedActionListener<>(listener, latch);
// tag::delete-pipeline-execute-async
client.ingest().deletePipelineAsync(request, listener); // <1>
// end::delete-pipeline-execute-async
assertTrue(latch.await(30L, TimeUnit.SECONDS));
}
}
}

View File

@ -1,14 +1,14 @@
[[java-rest-high-cluster-delete-pipeline]]
[[java-rest-high-ingest-delete-pipeline]]
=== Delete Pipeline API
[[java-rest-high-cluster-delete-pipeline-request]]
[[java-rest-high-ingest-delete-pipeline-request]]
==== Delete Pipeline Request
A `DeletePipelineRequest` requires a pipeline `id` to delete.
["source","java",subs="attributes,callouts,macros"]
--------------------------------------------------
include-tagged::{doc-tests}/ClusterClientDocumentationIT.java[delete-pipeline-request]
include-tagged::{doc-tests}/IngestClientDocumentationIT.java[delete-pipeline-request]
--------------------------------------------------
<1> The pipeline id to delete
@ -17,28 +17,28 @@ The following arguments can optionally be provided:
["source","java",subs="attributes,callouts,macros"]
--------------------------------------------------
include-tagged::{doc-tests}/ClusterClientDocumentationIT.java[delete-pipeline-request-timeout]
include-tagged::{doc-tests}/IngestClientDocumentationIT.java[delete-pipeline-request-timeout]
--------------------------------------------------
<1> Timeout to wait for the all the nodes to acknowledge the pipeline deletion as a `TimeValue`
<2> Timeout to wait for the all the nodes to acknowledge the pipeline deletion as a `String`
["source","java",subs="attributes,callouts,macros"]
--------------------------------------------------
include-tagged::{doc-tests}/ClusterClientDocumentationIT.java[delete-pipeline-request-masterTimeout]
include-tagged::{doc-tests}/IngestClientDocumentationIT.java[delete-pipeline-request-masterTimeout]
--------------------------------------------------
<1> Timeout to connect to the master node as a `TimeValue`
<2> Timeout to connect to the master node as a `String`
[[java-rest-high-cluster-delete-pipeline-sync]]
[[java-rest-high-ingest-delete-pipeline-sync]]
==== Synchronous Execution
["source","java",subs="attributes,callouts,macros"]
--------------------------------------------------
include-tagged::{doc-tests}/ClusterClientDocumentationIT.java[delete-pipeline-execute]
include-tagged::{doc-tests}/IngestClientDocumentationIT.java[delete-pipeline-execute]
--------------------------------------------------
<1> Execute the request and get back the response in a `WritePipelineResponse` object.
[[java-rest-high-cluster-delete-pipeline-async]]
[[java-rest-high-ingest-delete-pipeline-async]]
==== Asynchronous Execution
The asynchronous execution of a delete pipeline request requires both the `DeletePipelineRequest`
@ -47,7 +47,7 @@ method:
["source","java",subs="attributes,callouts,macros"]
--------------------------------------------------
include-tagged::{doc-tests}/ClusterClientDocumentationIT.java[delete-pipeline-execute-async]
include-tagged::{doc-tests}/IngestClientDocumentationIT.java[delete-pipeline-execute-async]
--------------------------------------------------
<1> The `DeletePipelineRequest` to execute and the `ActionListener` to use when
the execution completes
@ -61,13 +61,13 @@ A typical listener for `WritePipelineResponse` looks like:
["source","java",subs="attributes,callouts,macros"]
--------------------------------------------------
include-tagged::{doc-tests}/ClusterClientDocumentationIT.java[delete-pipeline-execute-listener]
include-tagged::{doc-tests}/IngestClientDocumentationIT.java[delete-pipeline-execute-listener]
--------------------------------------------------
<1> Called when the execution is successfully completed. The response is
provided as an argument
<2> Called in case of failure. The raised exception is provided as an argument
[[java-rest-high-cluster-delete-pipeline-response]]
[[java-rest-high-ingest-delete-pipeline-response]]
==== Delete Pipeline Response
The returned `WritePipelineResponse` allows to retrieve information about the executed
@ -75,6 +75,6 @@ The returned `WritePipelineResponse` allows to retrieve information about the ex
["source","java",subs="attributes,callouts,macros"]
--------------------------------------------------
include-tagged::{doc-tests}/ClusterClientDocumentationIT.java[delete-pipeline-response]
include-tagged::{doc-tests}/IngestClientDocumentationIT.java[delete-pipeline-response]
--------------------------------------------------
<1> Indicates whether all of the nodes have acknowledged the request

View File

@ -1,14 +1,14 @@
[[java-rest-high-cluster-get-pipeline]]
[[java-rest-high-ingest-get-pipeline]]
=== Get Pipeline API
[[java-rest-high-cluster-get-pipeline-request]]
[[java-rest-high-ingest-get-pipeline-request]]
==== Get Pipeline Request
A `GetPipelineRequest` requires one or more `pipelineIds` to fetch.
["source","java",subs="attributes,callouts,macros"]
--------------------------------------------------
include-tagged::{doc-tests}/ClusterClientDocumentationIT.java[get-pipeline-request]
include-tagged::{doc-tests}/IngestClientDocumentationIT.java[get-pipeline-request]
--------------------------------------------------
<1> The pipeline id to fetch
@ -17,21 +17,21 @@ The following arguments can optionally be provided:
["source","java",subs="attributes,callouts,macros"]
--------------------------------------------------
include-tagged::{doc-tests}/ClusterClientDocumentationIT.java[get-pipeline-request-masterTimeout]
include-tagged::{doc-tests}/IngestClientDocumentationIT.java[get-pipeline-request-masterTimeout]
--------------------------------------------------
<1> Timeout to connect to the master node as a `TimeValue`
<2> Timeout to connect to the master node as a `String`
[[java-rest-high-cluster-get-pipeline-sync]]
[[java-rest-high-ingest-get-pipeline-sync]]
==== Synchronous Execution
["source","java",subs="attributes,callouts,macros"]
--------------------------------------------------
include-tagged::{doc-tests}/ClusterClientDocumentationIT.java[get-pipeline-execute]
include-tagged::{doc-tests}/IngestClientDocumentationIT.java[get-pipeline-execute]
--------------------------------------------------
<1> Execute the request and get back the response in a GetPipelineResponse object.
[[java-rest-high-cluster-get-pipeline-async]]
[[java-rest-high-ingest-get-pipeline-async]]
==== Asynchronous Execution
The asynchronous execution of a get pipeline request requires both the `GetPipelineRequest`
@ -40,7 +40,7 @@ method:
["source","java",subs="attributes,callouts,macros"]
--------------------------------------------------
include-tagged::{doc-tests}/ClusterClientDocumentationIT.java[get-pipeline-execute-async]
include-tagged::{doc-tests}/IngestClientDocumentationIT.java[get-pipeline-execute-async]
--------------------------------------------------
<1> The `GetPipelineRequest` to execute and the `ActionListener` to use when
the execution completes
@ -54,13 +54,13 @@ A typical listener for `GetPipelineResponse` looks like:
["source","java",subs="attributes,callouts,macros"]
--------------------------------------------------
include-tagged::{doc-tests}/ClusterClientDocumentationIT.java[get-pipeline-execute-listener]
include-tagged::{doc-tests}/IngestClientDocumentationIT.java[get-pipeline-execute-listener]
--------------------------------------------------
<1> Called when the execution is successfully completed. The response is
provided as an argument
<2> Called in case of failure. The raised exception is provided as an argument
[[java-rest-high-cluster-get-pipeline-response]]
[[java-rest-high-ingest-get-pipeline-response]]
==== Get Pipeline Response
The returned `GetPipelineResponse` allows to retrieve information about the executed
@ -68,7 +68,7 @@ The returned `GetPipelineResponse` allows to retrieve information about the exec
["source","java",subs="attributes,callouts,macros"]
--------------------------------------------------
include-tagged::{doc-tests}/ClusterClientDocumentationIT.java[get-pipeline-response]
include-tagged::{doc-tests}/IngestClientDocumentationIT.java[get-pipeline-response]
--------------------------------------------------
<1> Check if a matching pipeline id was found or not.
<2> Get the list of pipelines found as a list of `PipelineConfig` objects.

View File

@ -1,7 +1,7 @@
[[java-rest-high-cluster-put-pipeline]]
[[java-rest-high-ingest-put-pipeline]]
=== Put Pipeline API
[[java-rest-high-cluster-put-pipeline-request]]
[[java-rest-high-ingest-put-pipeline-request]]
==== Put Pipeline Request
A `PutPipelineRequest` requires an `id` argument, a source and a `XContentType`. The source consists
@ -9,7 +9,7 @@ of a description and a list of `Processor` objects.
["source","java",subs="attributes,callouts,macros"]
--------------------------------------------------
include-tagged::{doc-tests}/ClusterClientDocumentationIT.java[put-pipeline-request]
include-tagged::{doc-tests}/IngestClientDocumentationIT.java[put-pipeline-request]
--------------------------------------------------
<1> The pipeline id
<2> The source for the pipeline as a `ByteArray`.
@ -20,28 +20,28 @@ The following arguments can optionally be provided:
["source","java",subs="attributes,callouts,macros"]
--------------------------------------------------
include-tagged::{doc-tests}/ClusterClientDocumentationIT.java[put-pipeline-request-timeout]
include-tagged::{doc-tests}/IngestClientDocumentationIT.java[put-pipeline-request-timeout]
--------------------------------------------------
<1> Timeout to wait for the all the nodes to acknowledge the pipeline creation as a `TimeValue`
<2> Timeout to wait for the all the nodes to acknowledge the pipeline creation as a `String`
["source","java",subs="attributes,callouts,macros"]
--------------------------------------------------
include-tagged::{doc-tests}/ClusterClientDocumentationIT.java[put-pipeline-request-masterTimeout]
include-tagged::{doc-tests}/IngestClientDocumentationIT.java[put-pipeline-request-masterTimeout]
--------------------------------------------------
<1> Timeout to connect to the master node as a `TimeValue`
<2> Timeout to connect to the master node as a `String`
[[java-rest-high-cluster-put-pipeline-sync]]
[[java-rest-high-ingest-put-pipeline-sync]]
==== Synchronous Execution
["source","java",subs="attributes,callouts,macros"]
--------------------------------------------------
include-tagged::{doc-tests}/ClusterClientDocumentationIT.java[put-pipeline-execute]
include-tagged::{doc-tests}/IngestClientDocumentationIT.java[put-pipeline-execute]
--------------------------------------------------
<1> Execute the request and get back the response in a WritePipelineResponse object.
[[java-rest-high-cluster-put-pipeline-async]]
[[java-rest-high-ingest-put-pipeline-async]]
==== Asynchronous Execution
The asynchronous execution of a put pipeline request requires both the `PutPipelineRequest`
@ -50,7 +50,7 @@ method:
["source","java",subs="attributes,callouts,macros"]
--------------------------------------------------
include-tagged::{doc-tests}/ClusterClientDocumentationIT.java[put-pipeline-execute-async]
include-tagged::{doc-tests}/IngestClientDocumentationIT.java[put-pipeline-execute-async]
--------------------------------------------------
<1> The `PutPipelineRequest` to execute and the `ActionListener` to use when
the execution completes
@ -64,13 +64,13 @@ A typical listener for `WritePipelineResponse` looks like:
["source","java",subs="attributes,callouts,macros"]
--------------------------------------------------
include-tagged::{doc-tests}/ClusterClientDocumentationIT.java[put-pipeline-execute-listener]
include-tagged::{doc-tests}/IngestClientDocumentationIT.java[put-pipeline-execute-listener]
--------------------------------------------------
<1> Called when the execution is successfully completed. The response is
provided as an argument
<2> Called in case of failure. The raised exception is provided as an argument
[[java-rest-high-cluster-put-pipeline-response]]
[[java-rest-high-ingest-put-pipeline-response]]
==== Put Pipeline Response
The returned `WritePipelineResponse` allows to retrieve information about the executed
@ -78,6 +78,6 @@ The returned `WritePipelineResponse` allows to retrieve information about the ex
["source","java",subs="attributes,callouts,macros"]
--------------------------------------------------
include-tagged::{doc-tests}/ClusterClientDocumentationIT.java[put-pipeline-response]
include-tagged::{doc-tests}/IngestClientDocumentationIT.java[put-pipeline-response]
--------------------------------------------------
<1> Indicates whether all of the nodes have acknowledged the request

View File

@ -106,14 +106,19 @@ include::indices/put_template.asciidoc[]
The Java High Level REST Client supports the following Cluster APIs:
* <<java-rest-high-cluster-put-settings>>
* <<java-rest-high-cluster-put-pipeline>>
* <<java-rest-high-cluster-get-pipeline>>
* <<java-rest-high-cluster-delete-pipeline>>
include::cluster/put_settings.asciidoc[]
include::cluster/put_pipeline.asciidoc[]
include::cluster/get_pipeline.asciidoc[]
include::cluster/delete_pipeline.asciidoc[]
== Ingest APIs
The Java High Level REST Client supports the following Ingest APIs:
* <<java-rest-high-ingest-put-pipeline>>
* <<java-rest-high-ingest-get-pipeline>>
* <<java-rest-high-ingest-delete-pipeline>>
include::ingest/put_pipeline.asciidoc[]
include::ingest/get_pipeline.asciidoc[]
include::ingest/delete_pipeline.asciidoc[]
== Snapshot APIs

View File

@ -23,7 +23,7 @@ PUT _template/template_1
"number_of_shards": 1
},
"mappings": {
"type1": {
"_doc": {
"_source": {
"enabled": false
},
@ -157,7 +157,7 @@ PUT /_template/template_1
"number_of_shards" : 1
},
"mappings" : {
"type1" : {
"_doc" : {
"_source" : { "enabled" : false }
}
}
@ -171,7 +171,7 @@ PUT /_template/template_2
"number_of_shards" : 1
},
"mappings" : {
"type1" : {
"_doc" : {
"_source" : { "enabled" : true }
}
}
@ -180,7 +180,7 @@ PUT /_template/template_2
// CONSOLE
// TEST[s/^/DELETE _template\/template_1\n/]
The above will disable storing the `_source` on all `type1` types, but
The above will disable storing the `_source`, but
for indices that start with `te*`, `_source` will still be enabled.
Note, for mappings, the merging is "deep", meaning that specific
object/property based mappings can easily be added/overridden on higher

View File

@ -1,47 +1,23 @@
[[mapping-field-names-field]]
=== `_field_names` field
The `_field_names` field indexes the names of every field in a document that
contains any value other than `null`. This field is used by the
The `_field_names` field used to index the names of every field in a document that
contains any value other than `null`. This field was used by the
<<query-dsl-exists-query,`exists`>> query to find documents that
either have or don't have any non-+null+ value for a particular field.
The value of the `_field_names` field is accessible in queries:
[source,js]
--------------------------
# Example documents
PUT my_index/_doc/1
{
"title": "This is a document"
}
PUT my_index/_doc/2?refresh=true
{
"title": "This is another document",
"body": "This document has a body"
}
GET my_index/_search
{
"query": {
"terms": {
"_field_names": [ "title" ] <1>
}
}
}
--------------------------
// CONSOLE
<1> Querying on the `_field_names` field (also see the <<query-dsl-exists-query,`exists`>> query)
Now the `_field_names` field only indexes the names of fields that have
`doc_values` and `norms` disabled. For fields which have either `doc_values`
or `norm` enabled the <<query-dsl-exists-query,`exists`>> query will still
be available but will not use the `_field_names` field.
==== Disabling `_field_names`
Because `_field_names` introduce some index-time overhead, you might want to
disable this field if you want to optimize for indexing speed and do not need
`exists` queries.
Disabling `_field_names` is often not necessary because it no longer
carries the index overhead it once did. If you have a lot of fields
which have `doc_values` and `norms` disabled and you do not need to
execute `exists` queries using those fields you might want to disable
`_field_names` be adding the following to the mappings:
[source,js]
--------------------------------------------------

View File

@ -96,6 +96,14 @@ The following parameters are accepted by `text` fields:
the expense of a larger index. Accepts an
<<index-prefix-config,`index-prefix configuration block`>>
`index_phrases`::
If enabled, two-term word combinations ('shingles') are indexed into a separate
field. This allows exact phrase queries to run more efficiently, at the expense
of a larger index. Note that this works best when stopwords are not removed,
as phrases containing stopwords will not use the subsidiary field and will fall
back to a standard phrase query. Accepts `true` or `false` (default).
<<norms,`norms`>>::
Whether field-length should be taken into account when scoring queries.

View File

@ -15,6 +15,9 @@
* The boundary specified using geohashes in the `geo_bounding_box` query
now include entire geohash cell, instead of just geohash center.
* Attempts to generate multi-term phrase queries against non-text fields
with a custom analyzer will now throw an exception
==== Adaptive replica selection enabled by default
Adaptive replica selection has been enabled by default. If you wish to return to

View File

@ -80,12 +80,12 @@ The accounting circuit breaker allows Elasticsearch to limit the memory
usage of things held in memory that are not released when a request is
completed. This includes things like the Lucene segment memory.
`network.breaker.accounting.limit`::
`indices.breaker.accounting.limit`::
Limit for accounting breaker, defaults to 100% of JVM heap. This means that it is bound
by the limit configured for the parent circuit breaker.
`network.breaker.accounting.overhead`::
`indices.breaker.accounting.overhead`::
A constant that all accounting estimations are multiplied with to determine a
final estimation. Defaults to 1

View File

@ -0,0 +1,67 @@
---
"search with indexed phrases":
- skip:
version: " - 6.99.99"
reason: index_phrase is only available as of 7.0.0
- do:
indices.create:
index: test
body:
mappings:
test:
properties:
text:
type: text
index_phrases: true
- do:
index:
index: test
type: test
id: 1
body: { text: "peter piper picked a peck of pickled peppers" }
- do:
indices.refresh:
index: [test]
- do:
search:
index: test
body:
query:
match_phrase:
text:
query: "peter piper"
- match: {hits.total: 1}
- do:
search:
index: test
q: '"peter piper"~1'
df: text
- match: {hits.total: 1}
- do:
search:
index: test
body:
query:
match_phrase:
text: "peter piper picked"
- match: {hits.total: 1}
- do:
search:
index: test
body:
query:
match_phrase:
text: "piper"
- match: {hits.total: 1}

View File

@ -20,14 +20,9 @@
package org.elasticsearch.action.admin.cluster.repositories.delete;
import org.elasticsearch.action.support.master.AcknowledgedResponse;
import org.elasticsearch.common.io.stream.StreamInput;
import org.elasticsearch.common.io.stream.StreamOutput;
import org.elasticsearch.common.xcontent.ConstructingObjectParser;
import org.elasticsearch.common.xcontent.ToXContentObject;
import org.elasticsearch.common.xcontent.XContentParser;
import java.io.IOException;
/**
* Unregister repository response
*/
@ -47,18 +42,6 @@ public class DeleteRepositoryResponse extends AcknowledgedResponse {
super(acknowledged);
}
@Override
public void readFrom(StreamInput in) throws IOException {
super.readFrom(in);
readAcknowledged(in);
}
@Override
public void writeTo(StreamOutput out) throws IOException {
super.writeTo(out);
writeAcknowledged(out);
}
public static DeleteRepositoryResponse fromXContent(XContentParser parser) {
return PARSER.apply(parser, null);
}

View File

@ -20,13 +20,9 @@
package org.elasticsearch.action.admin.cluster.repositories.put;
import org.elasticsearch.action.support.master.AcknowledgedResponse;
import org.elasticsearch.common.io.stream.StreamInput;
import org.elasticsearch.common.io.stream.StreamOutput;
import org.elasticsearch.common.xcontent.ConstructingObjectParser;
import org.elasticsearch.common.xcontent.XContentParser;
import java.io.IOException;
/**
* Register repository response
*/
@ -46,18 +42,6 @@ public class PutRepositoryResponse extends AcknowledgedResponse {
super(acknowledged);
}
@Override
public void readFrom(StreamInput in) throws IOException {
super.readFrom(in);
readAcknowledged(in);
}
@Override
public void writeTo(StreamOutput out) throws IOException {
super.writeTo(out);
writeAcknowledged(out);
}
public static PutRepositoryResponse fromXContent(XContentParser parser) {
return PARSER.apply(parser, null);
}

View File

@ -63,22 +63,32 @@ public class ClusterRerouteResponse extends AcknowledgedResponse implements ToXC
@Override
public void readFrom(StreamInput in) throws IOException {
super.readFrom(in);
state = ClusterState.readFrom(in, null);
readAcknowledged(in);
explanations = RoutingExplanations.readFrom(in);
if (in.getVersion().onOrAfter(Version.V_7_0_0_alpha1)) {
super.readFrom(in);
state = ClusterState.readFrom(in, null);
explanations = RoutingExplanations.readFrom(in);
} else {
state = ClusterState.readFrom(in, null);
acknowledged = in.readBoolean();
explanations = RoutingExplanations.readFrom(in);
}
}
@Override
public void writeTo(StreamOutput out) throws IOException {
super.writeTo(out);
if (out.getVersion().onOrAfter(Version.V_6_3_0)) {
if (out.getVersion().onOrAfter(Version.V_7_0_0_alpha1)) {
super.writeTo(out);
state.writeTo(out);
RoutingExplanations.writeTo(explanations, out);
} else {
ClusterModule.filterCustomsForPre63Clients(state).writeTo(out);
if (out.getVersion().onOrAfter(Version.V_6_3_0)) {
state.writeTo(out);
} else {
ClusterModule.filterCustomsForPre63Clients(state).writeTo(out);
}
out.writeBoolean(acknowledged);
RoutingExplanations.writeTo(explanations, out);
}
writeAcknowledged(out);
RoutingExplanations.writeTo(explanations, out);
}
@Override

View File

@ -19,6 +19,7 @@
package org.elasticsearch.action.admin.cluster.settings;
import org.elasticsearch.Version;
import org.elasticsearch.action.support.master.AcknowledgedResponse;
import org.elasticsearch.common.ParseField;
import org.elasticsearch.common.io.stream.StreamInput;
@ -67,10 +68,15 @@ public class ClusterUpdateSettingsResponse extends AcknowledgedResponse {
@Override
public void readFrom(StreamInput in) throws IOException {
super.readFrom(in);
transientSettings = Settings.readSettingsFromStream(in);
persistentSettings = Settings.readSettingsFromStream(in);
readAcknowledged(in);
if (in.getVersion().onOrAfter(Version.V_7_0_0_alpha1)) {
super.readFrom(in);
transientSettings = Settings.readSettingsFromStream(in);
persistentSettings = Settings.readSettingsFromStream(in);
} else {
transientSettings = Settings.readSettingsFromStream(in);
persistentSettings = Settings.readSettingsFromStream(in);
acknowledged = in.readBoolean();
}
}
public Settings getTransientSettings() {
@ -83,10 +89,15 @@ public class ClusterUpdateSettingsResponse extends AcknowledgedResponse {
@Override
public void writeTo(StreamOutput out) throws IOException {
super.writeTo(out);
Settings.writeSettingsToStream(transientSettings, out);
Settings.writeSettingsToStream(persistentSettings, out);
writeAcknowledged(out);
if (out.getVersion().onOrAfter(Version.V_7_0_0_alpha1)) {
super.writeTo(out);
Settings.writeSettingsToStream(transientSettings, out);
Settings.writeSettingsToStream(persistentSettings, out);
} else {
Settings.writeSettingsToStream(transientSettings, out);
Settings.writeSettingsToStream(persistentSettings, out);
out.writeBoolean(acknowledged);
}
}
@Override

View File

@ -20,10 +20,6 @@
package org.elasticsearch.action.admin.cluster.snapshots.delete;
import org.elasticsearch.action.support.master.AcknowledgedResponse;
import org.elasticsearch.common.io.stream.StreamInput;
import org.elasticsearch.common.io.stream.StreamOutput;
import java.io.IOException;
/**
* Delete snapshot response
@ -36,17 +32,4 @@ public class DeleteSnapshotResponse extends AcknowledgedResponse {
DeleteSnapshotResponse(boolean acknowledged) {
super(acknowledged);
}
@Override
public void readFrom(StreamInput in) throws IOException {
super.readFrom(in);
readAcknowledged(in);
}
@Override
public void writeTo(StreamOutput out) throws IOException {
super.writeTo(out);
writeAcknowledged(out);
}
}

View File

@ -20,10 +20,6 @@
package org.elasticsearch.action.admin.cluster.storedscripts;
import org.elasticsearch.action.support.master.AcknowledgedResponse;
import org.elasticsearch.common.io.stream.StreamInput;
import org.elasticsearch.common.io.stream.StreamOutput;
import java.io.IOException;
public class DeleteStoredScriptResponse extends AcknowledgedResponse {
@ -33,16 +29,4 @@ public class DeleteStoredScriptResponse extends AcknowledgedResponse {
public DeleteStoredScriptResponse(boolean acknowledged) {
super(acknowledged);
}
@Override
public void readFrom(StreamInput in) throws IOException {
super.readFrom(in);
readAcknowledged(in);
}
@Override
public void writeTo(StreamOutput out) throws IOException {
super.writeTo(out);
writeAcknowledged(out);
}
}

View File

@ -20,10 +20,6 @@
package org.elasticsearch.action.admin.cluster.storedscripts;
import org.elasticsearch.action.support.master.AcknowledgedResponse;
import org.elasticsearch.common.io.stream.StreamInput;
import org.elasticsearch.common.io.stream.StreamOutput;
import java.io.IOException;
public class PutStoredScriptResponse extends AcknowledgedResponse {
@ -34,16 +30,4 @@ public class PutStoredScriptResponse extends AcknowledgedResponse {
super(acknowledged);
}
@Override
public void readFrom(StreamInput in) throws IOException {
super.readFrom(in);
readAcknowledged(in);
}
@Override
public void writeTo(StreamOutput out) throws IOException {
super.writeTo(out);
writeAcknowledged(out);
}
}

View File

@ -20,13 +20,9 @@
package org.elasticsearch.action.admin.indices.alias;
import org.elasticsearch.action.support.master.AcknowledgedResponse;
import org.elasticsearch.common.io.stream.StreamInput;
import org.elasticsearch.common.io.stream.StreamOutput;
import org.elasticsearch.common.xcontent.ConstructingObjectParser;
import org.elasticsearch.common.xcontent.XContentParser;
import java.io.IOException;
/**
* A response for a add/remove alias action.
*/
@ -45,18 +41,6 @@ public class IndicesAliasesResponse extends AcknowledgedResponse {
super(acknowledged);
}
@Override
public void readFrom(StreamInput in) throws IOException {
super.readFrom(in);
readAcknowledged(in);
}
@Override
public void writeTo(StreamOutput out) throws IOException {
super.writeTo(out);
writeAcknowledged(out);
}
public static IndicesAliasesResponse fromXContent(XContentParser parser) {
return PARSER.apply(parser, null);
}

View File

@ -20,13 +20,9 @@
package org.elasticsearch.action.admin.indices.close;
import org.elasticsearch.action.support.master.AcknowledgedResponse;
import org.elasticsearch.common.io.stream.StreamInput;
import org.elasticsearch.common.io.stream.StreamOutput;
import org.elasticsearch.common.xcontent.ConstructingObjectParser;
import org.elasticsearch.common.xcontent.XContentParser;
import java.io.IOException;
/**
* A response for a close index action.
*/
@ -45,18 +41,6 @@ public class CloseIndexResponse extends AcknowledgedResponse {
super(acknowledged);
}
@Override
public void readFrom(StreamInput in) throws IOException {
super.readFrom(in);
readAcknowledged(in);
}
@Override
public void writeTo(StreamOutput out) throws IOException {
super.writeTo(out);
writeAcknowledged(out);
}
public static CloseIndexResponse fromXContent(XContentParser parser) {
return PARSER.apply(parser, null);
}

View File

@ -66,7 +66,6 @@ public class CreateIndexResponse extends ShardsAcknowledgedResponse {
@Override
public void readFrom(StreamInput in) throws IOException {
super.readFrom(in);
readAcknowledged(in);
readShardsAcknowledged(in);
if (in.getVersion().onOrAfter(Version.V_5_6_0)) {
index = in.readString();
@ -76,7 +75,6 @@ public class CreateIndexResponse extends ShardsAcknowledgedResponse {
@Override
public void writeTo(StreamOutput out) throws IOException {
super.writeTo(out);
writeAcknowledged(out);
writeShardsAcknowledged(out);
if (out.getVersion().onOrAfter(Version.V_5_6_0)) {
out.writeString(index);

View File

@ -20,14 +20,9 @@
package org.elasticsearch.action.admin.indices.delete;
import org.elasticsearch.action.support.master.AcknowledgedResponse;
import org.elasticsearch.common.io.stream.StreamInput;
import org.elasticsearch.common.io.stream.StreamOutput;
import org.elasticsearch.common.xcontent.ConstructingObjectParser;
import org.elasticsearch.common.xcontent.XContentBuilder;
import org.elasticsearch.common.xcontent.XContentParser;
import java.io.IOException;
/**
* A response for a delete index action.
*/
@ -47,18 +42,6 @@ public class DeleteIndexResponse extends AcknowledgedResponse {
super(acknowledged);
}
@Override
public void readFrom(StreamInput in) throws IOException {
super.readFrom(in);
readAcknowledged(in);
}
@Override
public void writeTo(StreamOutput out) throws IOException {
super.writeTo(out);
writeAcknowledged(out);
}
public static DeleteIndexResponse fromXContent(XContentParser parser) {
return PARSER.apply(parser, null);
}

View File

@ -20,13 +20,9 @@
package org.elasticsearch.action.admin.indices.mapping.put;
import org.elasticsearch.action.support.master.AcknowledgedResponse;
import org.elasticsearch.common.io.stream.StreamInput;
import org.elasticsearch.common.io.stream.StreamOutput;
import org.elasticsearch.common.xcontent.ConstructingObjectParser;
import org.elasticsearch.common.xcontent.XContentParser;
import java.io.IOException;
/**
* The response of put mapping operation.
*/
@ -47,18 +43,6 @@ public class PutMappingResponse extends AcknowledgedResponse {
super(acknowledged);
}
@Override
public void readFrom(StreamInput in) throws IOException {
super.readFrom(in);
readAcknowledged(in);
}
@Override
public void writeTo(StreamOutput out) throws IOException {
super.writeTo(out);
writeAcknowledged(out);
}
public static PutMappingResponse fromXContent(XContentParser parser) {
return PARSER.apply(parser, null);
}

View File

@ -21,11 +21,9 @@ package org.elasticsearch.action.admin.indices.open;
import org.elasticsearch.Version;
import org.elasticsearch.action.support.master.ShardsAcknowledgedResponse;
import org.elasticsearch.common.ParseField;
import org.elasticsearch.common.io.stream.StreamInput;
import org.elasticsearch.common.io.stream.StreamOutput;
import org.elasticsearch.common.xcontent.ConstructingObjectParser;
import org.elasticsearch.common.xcontent.XContentBuilder;
import org.elasticsearch.common.xcontent.XContentParser;
import java.io.IOException;
@ -52,7 +50,6 @@ public class OpenIndexResponse extends ShardsAcknowledgedResponse {
@Override
public void readFrom(StreamInput in) throws IOException {
super.readFrom(in);
readAcknowledged(in);
if (in.getVersion().onOrAfter(Version.V_6_1_0)) {
readShardsAcknowledged(in);
}
@ -61,7 +58,6 @@ public class OpenIndexResponse extends ShardsAcknowledgedResponse {
@Override
public void writeTo(StreamOutput out) throws IOException {
super.writeTo(out);
writeAcknowledged(out);
if (out.getVersion().onOrAfter(Version.V_6_1_0)) {
writeShardsAcknowledged(out);
}

View File

@ -19,6 +19,7 @@
package org.elasticsearch.action.admin.indices.rollover;
import org.elasticsearch.Version;
import org.elasticsearch.action.support.master.ShardsAcknowledgedResponse;
import org.elasticsearch.common.ParseField;
import org.elasticsearch.common.io.stream.StreamInput;
@ -114,34 +115,60 @@ public final class RolloverResponse extends ShardsAcknowledgedResponse implement
@Override
public void readFrom(StreamInput in) throws IOException {
super.readFrom(in);
oldIndex = in.readString();
newIndex = in.readString();
int conditionSize = in.readVInt();
conditionStatus = new HashMap<>(conditionSize);
for (int i = 0; i < conditionSize; i++) {
conditionStatus.put(in.readString(), in.readBoolean());
if (in.getVersion().onOrAfter(Version.V_7_0_0_alpha1)) {
super.readFrom(in);
oldIndex = in.readString();
newIndex = in.readString();
int conditionSize = in.readVInt();
conditionStatus = new HashMap<>(conditionSize);
for (int i = 0; i < conditionSize; i++) {
conditionStatus.put(in.readString(), in.readBoolean());
}
dryRun = in.readBoolean();
rolledOver = in.readBoolean();
readShardsAcknowledged(in);
} else {
oldIndex = in.readString();
newIndex = in.readString();
int conditionSize = in.readVInt();
conditionStatus = new HashMap<>(conditionSize);
for (int i = 0; i < conditionSize; i++) {
conditionStatus.put(in.readString(), in.readBoolean());
}
dryRun = in.readBoolean();
rolledOver = in.readBoolean();
acknowledged = in.readBoolean();
readShardsAcknowledged(in);
}
dryRun = in.readBoolean();
rolledOver = in.readBoolean();
readAcknowledged(in);
readShardsAcknowledged(in);
}
@Override
public void writeTo(StreamOutput out) throws IOException {
super.writeTo(out);
out.writeString(oldIndex);
out.writeString(newIndex);
out.writeVInt(conditionStatus.size());
for (Map.Entry<String, Boolean> entry : conditionStatus.entrySet()) {
out.writeString(entry.getKey());
out.writeBoolean(entry.getValue());
if (out.getVersion().onOrAfter(Version.V_7_0_0_alpha1)) {
super.writeTo(out);
out.writeString(oldIndex);
out.writeString(newIndex);
out.writeVInt(conditionStatus.size());
for (Map.Entry<String, Boolean> entry : conditionStatus.entrySet()) {
out.writeString(entry.getKey());
out.writeBoolean(entry.getValue());
}
out.writeBoolean(dryRun);
out.writeBoolean(rolledOver);
writeShardsAcknowledged(out);
} else {
out.writeString(oldIndex);
out.writeString(newIndex);
out.writeVInt(conditionStatus.size());
for (Map.Entry<String, Boolean> entry : conditionStatus.entrySet()) {
out.writeString(entry.getKey());
out.writeBoolean(entry.getValue());
}
out.writeBoolean(dryRun);
out.writeBoolean(rolledOver);
out.writeBoolean(acknowledged);
writeShardsAcknowledged(out);
}
out.writeBoolean(dryRun);
out.writeBoolean(rolledOver);
writeAcknowledged(out);
writeShardsAcknowledged(out);
}
@Override

View File

@ -20,13 +20,9 @@
package org.elasticsearch.action.admin.indices.settings.put;
import org.elasticsearch.action.support.master.AcknowledgedResponse;
import org.elasticsearch.common.io.stream.StreamInput;
import org.elasticsearch.common.io.stream.StreamOutput;
import org.elasticsearch.common.xcontent.ConstructingObjectParser;
import org.elasticsearch.common.xcontent.XContentParser;
import java.io.IOException;
/**
* A response for an update index settings action
*/
@ -46,18 +42,6 @@ public class UpdateSettingsResponse extends AcknowledgedResponse {
super(acknowledged);
}
@Override
public void readFrom(StreamInput in) throws IOException {
super.readFrom(in);
readAcknowledged(in);
}
@Override
public void writeTo(StreamOutput out) throws IOException {
super.writeTo(out);
writeAcknowledged(out);
}
public static UpdateSettingsResponse fromXContent(XContentParser parser) {
return PARSER.apply(parser, null);
}

View File

@ -19,10 +19,6 @@
package org.elasticsearch.action.admin.indices.template.delete;
import org.elasticsearch.action.support.master.AcknowledgedResponse;
import org.elasticsearch.common.io.stream.StreamInput;
import org.elasticsearch.common.io.stream.StreamOutput;
import java.io.IOException;
/**
* A response for a delete index template.
@ -35,16 +31,4 @@ public class DeleteIndexTemplateResponse extends AcknowledgedResponse {
protected DeleteIndexTemplateResponse(boolean acknowledged) {
super(acknowledged);
}
@Override
public void readFrom(StreamInput in) throws IOException {
super.readFrom(in);
readAcknowledged(in);
}
@Override
public void writeTo(StreamOutput out) throws IOException {
super.writeTo(out);
writeAcknowledged(out);
}
}

View File

@ -19,13 +19,9 @@
package org.elasticsearch.action.admin.indices.template.put;
import org.elasticsearch.action.support.master.AcknowledgedResponse;
import org.elasticsearch.common.io.stream.StreamInput;
import org.elasticsearch.common.io.stream.StreamOutput;
import org.elasticsearch.common.xcontent.ConstructingObjectParser;
import org.elasticsearch.common.xcontent.XContentParser;
import java.io.IOException;
/**
* A response for a put index template action.
*/
@ -38,18 +34,6 @@ public class PutIndexTemplateResponse extends AcknowledgedResponse {
super(acknowledged);
}
@Override
public void readFrom(StreamInput in) throws IOException {
super.readFrom(in);
readAcknowledged(in);
}
@Override
public void writeTo(StreamOutput out) throws IOException {
super.writeTo(out);
writeAcknowledged(out);
}
private static final ConstructingObjectParser<PutIndexTemplateResponse, Void> PARSER;
static {
PARSER = new ConstructingObjectParser<>("put_index_template", true, args -> new PutIndexTemplateResponse((boolean) args[0]));

View File

@ -20,10 +20,6 @@
package org.elasticsearch.action.admin.indices.upgrade.post;
import org.elasticsearch.action.support.master.AcknowledgedResponse;
import org.elasticsearch.common.io.stream.StreamInput;
import org.elasticsearch.common.io.stream.StreamOutput;
import java.io.IOException;
/**
* A response for an update index settings action
@ -36,16 +32,4 @@ public class UpgradeSettingsResponse extends AcknowledgedResponse {
UpgradeSettingsResponse(boolean acknowledged) {
super(acknowledged);
}
@Override
public void readFrom(StreamInput in) throws IOException {
super.readFrom(in);
readAcknowledged(in);
}
@Override
public void writeTo(StreamOutput out) throws IOException {
super.writeTo(out);
writeAcknowledged(out);
}
}

View File

@ -20,14 +20,10 @@
package org.elasticsearch.action.ingest;
import org.elasticsearch.action.support.master.AcknowledgedResponse;
import org.elasticsearch.common.io.stream.StreamInput;
import org.elasticsearch.common.io.stream.StreamOutput;
import org.elasticsearch.common.xcontent.ConstructingObjectParser;
import org.elasticsearch.common.xcontent.ToXContentObject;
import org.elasticsearch.common.xcontent.XContentParser;
import java.io.IOException;
public class WritePipelineResponse extends AcknowledgedResponse implements ToXContentObject {
private static final ConstructingObjectParser<WritePipelineResponse, Void> PARSER = new ConstructingObjectParser<>(
@ -45,18 +41,6 @@ public class WritePipelineResponse extends AcknowledgedResponse implements ToXCo
super(acknowledged);
}
@Override
public void readFrom(StreamInput in) throws IOException {
super.readFrom(in);
readAcknowledged(in);
}
@Override
public void writeTo(StreamOutput out) throws IOException {
super.writeTo(out);
writeAcknowledged(out);
}
public static WritePipelineResponse fromXContent(XContentParser parser) {
return PARSER.apply(parser, null);
}

View File

@ -45,7 +45,7 @@ public abstract class AcknowledgedResponse extends ActionResponse implements ToX
ObjectParser.ValueType.BOOLEAN);
}
private boolean acknowledged;
protected boolean acknowledged;
protected AcknowledgedResponse() {
@ -63,17 +63,15 @@ public abstract class AcknowledgedResponse extends ActionResponse implements ToX
return acknowledged;
}
/**
* Reads the acknowledged value
*/
protected void readAcknowledged(StreamInput in) throws IOException {
@Override
public void readFrom(StreamInput in) throws IOException {
super.readFrom(in);
acknowledged = in.readBoolean();
}
/**
* Writes the acknowledged value
*/
protected void writeAcknowledged(StreamOutput out) throws IOException {
@Override
public void writeTo(StreamOutput out) throws IOException {
super.writeTo(out);
out.writeBoolean(acknowledged);
}

View File

@ -22,7 +22,6 @@ package org.elasticsearch.cluster;
import com.carrotsearch.hppc.cursors.IntObjectCursor;
import com.carrotsearch.hppc.cursors.ObjectCursor;
import com.carrotsearch.hppc.cursors.ObjectObjectCursor;
import org.elasticsearch.client.transport.TransportClient;
import org.elasticsearch.cluster.block.ClusterBlock;
import org.elasticsearch.cluster.block.ClusterBlocks;
@ -50,6 +49,7 @@ import org.elasticsearch.common.io.stream.NamedWriteableAwareStreamInput;
import org.elasticsearch.common.io.stream.NamedWriteableRegistry;
import org.elasticsearch.common.io.stream.StreamInput;
import org.elasticsearch.common.io.stream.StreamOutput;
import org.elasticsearch.common.io.stream.VersionedNamedWriteable;
import org.elasticsearch.common.settings.Settings;
import org.elasticsearch.common.xcontent.ToXContentFragment;
import org.elasticsearch.common.xcontent.XContentBuilder;
@ -122,7 +122,7 @@ public class ClusterState implements ToXContentFragment, Diffable<ClusterState>
* @param <T> the type of the custom
* @return true if the custom should be serialized and false otherwise
*/
static <T extends NamedDiffable & FeatureAware> boolean shouldSerializeCustom(final StreamOutput out, final T custom) {
static <T extends VersionedNamedWriteable & FeatureAware> boolean shouldSerialize(final StreamOutput out, final T custom) {
if (out.getVersion().before(custom.getMinimalSupportedVersion())) {
return false;
}
@ -748,13 +748,13 @@ public class ClusterState implements ToXContentFragment, Diffable<ClusterState>
// filter out custom states not supported by the other node
int numberOfCustoms = 0;
for (final ObjectCursor<Custom> cursor : customs.values()) {
if (FeatureAware.shouldSerializeCustom(out, cursor.value)) {
if (FeatureAware.shouldSerialize(out, cursor.value)) {
numberOfCustoms++;
}
}
out.writeVInt(numberOfCustoms);
for (final ObjectCursor<Custom> cursor : customs.values()) {
if (FeatureAware.shouldSerializeCustom(out, cursor.value)) {
if (FeatureAware.shouldSerialize(out, cursor.value)) {
out.writeNamedWriteable(cursor.value);
}
}

View File

@ -19,17 +19,10 @@
package org.elasticsearch.cluster;
import org.elasticsearch.Version;
import org.elasticsearch.common.io.stream.NamedWriteable;
import org.elasticsearch.common.io.stream.VersionedNamedWriteable;
/**
* Diff that also support NamedWriteable interface
* Diff that also support {@link VersionedNamedWriteable} interface
*/
public interface NamedDiffable<T> extends Diffable<T>, NamedWriteable {
/**
* The minimal version of the recipient this custom object can be sent to
*/
default Version getMinimalSupportedVersion() {
return Version.CURRENT.minimumIndexCompatibilityVersion();
}
public interface NamedDiffable<T> extends Diffable<T>, VersionedNamedWriteable {
}

View File

@ -20,14 +20,15 @@
package org.elasticsearch.cluster;
import com.carrotsearch.hppc.cursors.ObjectObjectCursor;
import org.elasticsearch.Version;
import org.elasticsearch.cluster.ClusterState.Custom;
import org.elasticsearch.snapshots.Snapshot;
import org.elasticsearch.common.collect.ImmutableOpenMap;
import org.elasticsearch.common.io.stream.StreamInput;
import org.elasticsearch.common.io.stream.StreamOutput;
import org.elasticsearch.common.xcontent.ToXContent;
import org.elasticsearch.common.xcontent.XContentBuilder;
import org.elasticsearch.index.shard.ShardId;
import org.elasticsearch.snapshots.Snapshot;
import java.io.IOException;
import java.util.ArrayList;
@ -382,6 +383,11 @@ public class RestoreInProgress extends AbstractNamedDiffable<Custom> implements
return TYPE;
}
@Override
public Version getMinimalSupportedVersion() {
return Version.CURRENT.minimumCompatibilityVersion();
}
public static NamedDiff<Custom> readDiffFrom(StreamInput in) throws IOException {
return readDiffFrom(Custom.class, TYPE, in);
}

View File

@ -395,6 +395,11 @@ public class SnapshotsInProgress extends AbstractNamedDiffable<Custom> implement
return TYPE;
}
@Override
public Version getMinimalSupportedVersion() {
return Version.CURRENT.minimumCompatibilityVersion();
}
public static NamedDiff<Custom> readDiffFrom(StreamInput in) throws IOException {
return readDiffFrom(Custom.class, TYPE, in);
}

View File

@ -19,6 +19,7 @@
package org.elasticsearch.cluster.metadata;
import org.elasticsearch.Version;
import org.elasticsearch.cluster.Diff;
import org.elasticsearch.cluster.NamedDiff;
import org.elasticsearch.common.ParseField;
@ -34,8 +35,6 @@ import org.elasticsearch.common.xcontent.ToXContentObject;
import org.elasticsearch.common.xcontent.XContentBuilder;
import org.elasticsearch.common.xcontent.XContentParser;
import org.elasticsearch.index.Index;
import org.joda.time.DateTime;
import org.joda.time.DateTimeZone;
import java.io.IOException;
import java.util.ArrayList;
@ -44,7 +43,6 @@ import java.util.Collections;
import java.util.EnumSet;
import java.util.List;
import java.util.Objects;
import java.util.concurrent.TimeUnit;
/**
* A collection of tombstones for explicitly marking indices as deleted in the cluster state.
@ -97,6 +95,11 @@ public final class IndexGraveyard implements MetaData.Custom {
return TYPE;
}
@Override
public Version getMinimalSupportedVersion() {
return Version.CURRENT.minimumCompatibilityVersion();
}
@Override
public EnumSet<MetaData.XContentContext> context() {
return MetaData.API_AND_GATEWAY;

View File

@ -786,13 +786,13 @@ public class MetaData implements Iterable<IndexMetaData>, Diffable<MetaData>, To
// filter out custom states not supported by the other node
int numberOfCustoms = 0;
for (final ObjectCursor<Custom> cursor : customs.values()) {
if (FeatureAware.shouldSerializeCustom(out, cursor.value)) {
if (FeatureAware.shouldSerialize(out, cursor.value)) {
numberOfCustoms++;
}
}
out.writeVInt(numberOfCustoms);
for (final ObjectCursor<Custom> cursor : customs.values()) {
if (FeatureAware.shouldSerializeCustom(out, cursor.value)) {
if (FeatureAware.shouldSerialize(out, cursor.value)) {
out.writeNamedWriteable(cursor.value);
}
}

View File

@ -20,6 +20,7 @@
package org.elasticsearch.cluster.metadata;
import org.elasticsearch.ElasticsearchParseException;
import org.elasticsearch.Version;
import org.elasticsearch.cluster.AbstractNamedDiffable;
import org.elasticsearch.cluster.NamedDiff;
import org.elasticsearch.cluster.metadata.MetaData.Custom;
@ -103,6 +104,11 @@ public class RepositoriesMetaData extends AbstractNamedDiffable<Custom> implemen
return TYPE;
}
@Override
public Version getMinimalSupportedVersion() {
return Version.CURRENT.minimumCompatibilityVersion();
}
public RepositoriesMetaData(StreamInput in) throws IOException {
RepositoryMetaData[] repository = new RepositoryMetaData[in.readVInt()];
for (int i = 0; i < repository.length; i++) {

View File

@ -0,0 +1,38 @@
/*
* 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.common.io.stream;
import org.elasticsearch.Version;
/**
* A {@link NamedWriteable} that has a minimum version associated with it.
*/
public interface VersionedNamedWriteable extends NamedWriteable {
/**
* Returns the name of the writeable object
*/
String getWriteableName();
/**
* The minimal version of the recipient this object can be sent to
*/
Version getMinimalSupportedVersion();
}

View File

@ -198,7 +198,7 @@ public final class KeywordFieldMapper extends FieldMapper {
protected KeywordFieldType(KeywordFieldType ref) {
super(ref);
this.normalizer = ref.normalizer;
this.splitQueriesOnWhitespace = splitQueriesOnWhitespace;
this.splitQueriesOnWhitespace = ref.splitQueriesOnWhitespace;
}
public KeywordFieldType clone() {

View File

@ -19,6 +19,7 @@
package org.elasticsearch.index.mapper;
import org.apache.lucene.analysis.TokenStream;
import org.apache.lucene.document.FieldType;
import org.apache.lucene.index.IndexOptions;
import org.apache.lucene.index.IndexReader;
@ -43,6 +44,7 @@ import org.elasticsearch.index.fielddata.IndexFieldData;
import org.elasticsearch.index.query.QueryRewriteContext;
import org.elasticsearch.index.query.QueryShardContext;
import org.elasticsearch.index.query.QueryShardException;
import org.elasticsearch.index.search.MatchQuery;
import org.elasticsearch.index.similarity.SimilarityProvider;
import org.elasticsearch.search.DocValueFormat;
import org.joda.time.DateTimeZone;
@ -353,6 +355,14 @@ public abstract class MappedFieldType extends FieldType {
public abstract Query existsQuery(QueryShardContext context);
public Query phraseQuery(String field, TokenStream stream, int slop, boolean enablePositionIncrements) throws IOException {
throw new IllegalArgumentException("Can only use phrase queries on text fields - not on [" + name + "] which is of type [" + typeName() + "]");
}
public Query multiPhraseQuery(String field, TokenStream stream, int slop, boolean enablePositionIncrements) throws IOException {
throw new IllegalArgumentException("Can only use phrase queries on text fields - not on [" + name + "] which is of type [" + typeName() + "]");
}
/**
* An enum used to describe the relation between the range of terms in a
* shard when compared with a query range

View File

@ -19,20 +19,29 @@
package org.elasticsearch.index.mapper;
import org.apache.logging.log4j.Logger;
import org.apache.lucene.analysis.Analyzer;
import org.apache.lucene.analysis.AnalyzerWrapper;
import org.apache.lucene.analysis.CachingTokenFilter;
import org.apache.lucene.analysis.TokenFilter;
import org.apache.lucene.analysis.TokenStream;
import org.apache.lucene.analysis.ngram.EdgeNGramTokenFilter;
import org.apache.lucene.analysis.shingle.FixedShingleFilter;
import org.apache.lucene.analysis.tokenattributes.PositionIncrementAttribute;
import org.apache.lucene.analysis.tokenattributes.TermToBytesRefAttribute;
import org.apache.lucene.document.Field;
import org.apache.lucene.index.IndexOptions;
import org.apache.lucene.index.IndexableField;
import org.apache.lucene.index.Term;
import org.apache.lucene.search.ConstantScoreQuery;
import org.apache.lucene.search.MultiPhraseQuery;
import org.apache.lucene.search.MultiTermQuery;
import org.apache.lucene.search.NormsFieldExistsQuery;
import org.apache.lucene.search.PhraseQuery;
import org.apache.lucene.search.Query;
import org.apache.lucene.search.TermQuery;
import org.elasticsearch.common.collect.Iterators;
import org.elasticsearch.common.logging.ESLoggerFactory;
import org.elasticsearch.common.settings.Settings;
import org.elasticsearch.common.xcontent.XContentBuilder;
import org.elasticsearch.common.xcontent.support.XContentMapValues;
@ -43,7 +52,7 @@ import org.elasticsearch.index.fielddata.plain.PagedBytesIndexFieldData;
import org.elasticsearch.index.query.QueryShardContext;
import java.io.IOException;
import java.util.Collections;
import java.util.ArrayList;
import java.util.Iterator;
import java.util.List;
import java.util.Map;
@ -54,9 +63,13 @@ import static org.elasticsearch.index.mapper.TypeParsers.parseTextField;
/** A {@link FieldMapper} for full-text fields. */
public class TextFieldMapper extends FieldMapper {
private static final Logger logger = ESLoggerFactory.getLogger(TextFieldMapper.class);
public static final String CONTENT_TYPE = "text";
private static final int POSITION_INCREMENT_GAP_USE_ANALYZER = -1;
public static final String FAST_PHRASE_SUFFIX = "._index_phrase";
public static class Defaults {
public static final double FIELDDATA_MIN_FREQUENCY = 0;
public static final double FIELDDATA_MAX_FREQUENCY = Integer.MAX_VALUE;
@ -105,6 +118,11 @@ public class TextFieldMapper extends FieldMapper {
return builder;
}
public Builder indexPhrases(boolean indexPhrases) {
fieldType().setIndexPhrases(indexPhrases);
return builder;
}
@Override
public Builder docValues(boolean docValues) {
if (docValues) {
@ -166,8 +184,16 @@ public class TextFieldMapper extends FieldMapper {
prefixFieldType.setAnalyzer(fieldType.indexAnalyzer());
prefixMapper = new PrefixFieldMapper(prefixFieldType, context.indexSettings());
}
if (fieldType().indexPhrases) {
if (fieldType().isSearchable() == false) {
throw new IllegalArgumentException("Cannot set index_phrases on unindexed field [" + name() + "]");
}
if (fieldType.indexOptions().compareTo(IndexOptions.DOCS_AND_FREQS_AND_POSITIONS) < 0) {
throw new IllegalArgumentException("Cannot set index_phrases on field [" + name() + "] if positions are not enabled");
}
}
return new TextFieldMapper(
name, fieldType, defaultFieldType, positionIncrementGap, prefixMapper,
name, fieldType(), defaultFieldType, positionIncrementGap, prefixMapper,
context.indexSettings(), multiFieldsBuilder.build(this, context), copyTo);
}
}
@ -211,12 +237,35 @@ public class TextFieldMapper extends FieldMapper {
builder.indexPrefixes(minChars, maxChars);
DocumentMapperParser.checkNoRemainingFields(propName, indexPrefix, parserContext.indexVersionCreated());
iterator.remove();
} else if (propName.equals("index_phrases")) {
builder.indexPhrases(XContentMapValues.nodeBooleanValue(propNode, "index_phrases"));
iterator.remove();
}
}
return builder;
}
}
private static class PhraseWrappedAnalyzer extends AnalyzerWrapper {
private final Analyzer delegate;
PhraseWrappedAnalyzer(Analyzer delegate) {
super(delegate.getReuseStrategy());
this.delegate = delegate;
}
@Override
protected Analyzer getWrappedAnalyzer(String fieldName) {
return delegate;
}
@Override
protected TokenStreamComponents wrapComponents(String fieldName, TokenStreamComponents components) {
return new TokenStreamComponents(components.getTokenizer(), new FixedShingleFilter(components.getTokenStream(), 2));
}
}
private static class PrefixWrappedAnalyzer extends AnalyzerWrapper {
private final int minChars;
@ -242,6 +291,46 @@ public class TextFieldMapper extends FieldMapper {
}
}
private static final class PhraseFieldType extends StringFieldType {
final TextFieldType parent;
PhraseFieldType(TextFieldType parent) {
setTokenized(true);
setIndexOptions(IndexOptions.DOCS_AND_FREQS_AND_POSITIONS);
if (parent.indexOptions() == IndexOptions.DOCS_AND_FREQS_AND_POSITIONS_AND_OFFSETS) {
setIndexOptions(IndexOptions.DOCS_AND_FREQS_AND_POSITIONS_AND_OFFSETS);
}
if (parent.storeTermVectorOffsets()) {
setStoreTermVectors(true);
setStoreTermVectorPositions(true);
setStoreTermVectorOffsets(true);
}
setAnalyzer(parent.indexAnalyzer().name(), parent.indexAnalyzer().analyzer());
setName(parent.name() + FAST_PHRASE_SUFFIX);
this.parent = parent;
}
void setAnalyzer(String name, Analyzer delegate) {
setIndexAnalyzer(new NamedAnalyzer(name, AnalyzerScope.INDEX, new PhraseWrappedAnalyzer(delegate)));
}
@Override
public MappedFieldType clone() {
return new PhraseFieldType(parent);
}
@Override
public String typeName() {
return "phrase";
}
@Override
public Query existsQuery(QueryShardContext context) {
throw new UnsupportedOperationException();
}
}
static final class PrefixFieldType extends StringFieldType {
final int minChars;
@ -310,6 +399,23 @@ public class TextFieldMapper extends FieldMapper {
}
}
private static final class PhraseFieldMapper extends FieldMapper {
PhraseFieldMapper(PhraseFieldType fieldType, Settings indexSettings) {
super(fieldType.name(), fieldType, fieldType, indexSettings, MultiFields.empty(), CopyTo.empty());
}
@Override
protected void parseCreateField(ParseContext context, List<IndexableField> fields) throws IOException {
throw new UnsupportedOperationException();
}
@Override
protected String contentType() {
return "phrase";
}
}
private static final class PrefixFieldMapper extends FieldMapper {
protected PrefixFieldMapper(PrefixFieldType fieldType, Settings indexSettings) {
@ -343,6 +449,7 @@ public class TextFieldMapper extends FieldMapper {
private double fielddataMaxFrequency;
private int fielddataMinSegmentSize;
private PrefixFieldType prefixFieldType;
private boolean indexPhrases = false;
public TextFieldType() {
setTokenized(true);
@ -358,6 +465,7 @@ public class TextFieldMapper extends FieldMapper {
this.fielddataMinFrequency = ref.fielddataMinFrequency;
this.fielddataMaxFrequency = ref.fielddataMaxFrequency;
this.fielddataMinSegmentSize = ref.fielddataMinSegmentSize;
this.indexPhrases = ref.indexPhrases;
if (ref.prefixFieldType != null) {
this.prefixFieldType = ref.prefixFieldType.clone();
}
@ -374,6 +482,7 @@ public class TextFieldMapper extends FieldMapper {
}
TextFieldType that = (TextFieldType) o;
return fielddata == that.fielddata
&& indexPhrases == that.indexPhrases
&& Objects.equals(prefixFieldType, that.prefixFieldType)
&& fielddataMinFrequency == that.fielddataMinFrequency
&& fielddataMaxFrequency == that.fielddataMaxFrequency
@ -382,7 +491,7 @@ public class TextFieldMapper extends FieldMapper {
@Override
public int hashCode() {
return Objects.hash(super.hashCode(), fielddata, prefixFieldType,
return Objects.hash(super.hashCode(), fielddata, indexPhrases, prefixFieldType,
fielddataMinFrequency, fielddataMaxFrequency, fielddataMinSegmentSize);
}
@ -427,6 +536,11 @@ public class TextFieldMapper extends FieldMapper {
this.prefixFieldType = prefixFieldType;
}
void setIndexPhrases(boolean indexPhrases) {
checkIfFrozen();
this.indexPhrases = indexPhrases;
}
public PrefixFieldType getPrefixFieldType() {
return this.prefixFieldType;
}
@ -458,6 +572,93 @@ public class TextFieldMapper extends FieldMapper {
}
}
@Override
public Query phraseQuery(String field, TokenStream stream, int slop, boolean enablePosIncrements) throws IOException {
if (indexPhrases && slop == 0 && hasGaps(cache(stream)) == false) {
stream = new FixedShingleFilter(stream, 2);
field = field + FAST_PHRASE_SUFFIX;
}
PhraseQuery.Builder builder = new PhraseQuery.Builder();
builder.setSlop(slop);
TermToBytesRefAttribute termAtt = stream.getAttribute(TermToBytesRefAttribute.class);
PositionIncrementAttribute posIncrAtt = stream.getAttribute(PositionIncrementAttribute.class);
int position = -1;
stream.reset();
while (stream.incrementToken()) {
if (enablePosIncrements) {
position += posIncrAtt.getPositionIncrement();
}
else {
position += 1;
}
builder.add(new Term(field, termAtt.getBytesRef()), position);
}
return builder.build();
}
@Override
public Query multiPhraseQuery(String field, TokenStream stream, int slop, boolean enablePositionIncrements) throws IOException {
if (indexPhrases && slop == 0 && hasGaps(cache(stream)) == false) {
stream = new FixedShingleFilter(stream, 2);
field = field + FAST_PHRASE_SUFFIX;
}
MultiPhraseQuery.Builder mpqb = new MultiPhraseQuery.Builder();
mpqb.setSlop(slop);
TermToBytesRefAttribute termAtt = stream.getAttribute(TermToBytesRefAttribute.class);
PositionIncrementAttribute posIncrAtt = stream.getAttribute(PositionIncrementAttribute.class);
int position = -1;
List<Term> multiTerms = new ArrayList<>();
stream.reset();
while (stream.incrementToken()) {
int positionIncrement = posIncrAtt.getPositionIncrement();
if (positionIncrement > 0 && multiTerms.size() > 0) {
if (enablePositionIncrements) {
mpqb.add(multiTerms.toArray(new Term[0]), position);
} else {
mpqb.add(multiTerms.toArray(new Term[0]));
}
multiTerms.clear();
}
position += positionIncrement;
multiTerms.add(new Term(field, termAtt.getBytesRef()));
}
if (enablePositionIncrements) {
mpqb.add(multiTerms.toArray(new Term[0]), position);
} else {
mpqb.add(multiTerms.toArray(new Term[0]));
}
return mpqb.build();
}
private static CachingTokenFilter cache(TokenStream in) {
if (in instanceof CachingTokenFilter) {
return (CachingTokenFilter) in;
}
return new CachingTokenFilter(in);
}
private static boolean hasGaps(CachingTokenFilter stream) throws IOException {
PositionIncrementAttribute posIncAtt = stream.getAttribute(PositionIncrementAttribute.class);
stream.reset();
while (stream.incrementToken()) {
if (posIncAtt.getPositionIncrement() > 1) {
return true;
}
}
return false;
}
@Override
public IndexFieldData.Builder fielddataBuilder(String fullyQualifiedIndexName) {
if (fielddata == false) {
@ -472,6 +673,9 @@ public class TextFieldMapper extends FieldMapper {
public void checkCompatibility(MappedFieldType other, List<String> conflicts) {
super.checkCompatibility(other, conflicts);
TextFieldType tft = (TextFieldType) other;
if (tft.indexPhrases != this.indexPhrases) {
conflicts.add("mapper [" + name() + "] has different [index_phrases] values");
}
if (Objects.equals(this.prefixFieldType, tft.prefixFieldType) == false) {
if (this.prefixFieldType == null) {
conflicts.add("mapper [" + name()
@ -490,8 +694,9 @@ public class TextFieldMapper extends FieldMapper {
private int positionIncrementGap;
private PrefixFieldMapper prefixFieldMapper;
private PhraseFieldMapper phraseFieldMapper;
protected TextFieldMapper(String simpleName, MappedFieldType fieldType, MappedFieldType defaultFieldType,
protected TextFieldMapper(String simpleName, TextFieldType fieldType, MappedFieldType defaultFieldType,
int positionIncrementGap, PrefixFieldMapper prefixFieldMapper,
Settings indexSettings, MultiFields multiFields, CopyTo copyTo) {
super(simpleName, fieldType, defaultFieldType, indexSettings, multiFields, copyTo);
@ -502,6 +707,7 @@ public class TextFieldMapper extends FieldMapper {
}
this.positionIncrementGap = positionIncrementGap;
this.prefixFieldMapper = prefixFieldMapper;
this.phraseFieldMapper = fieldType.indexPhrases ? new PhraseFieldMapper(new PhraseFieldType(fieldType), indexSettings) : null;
}
@Override
@ -535,15 +741,25 @@ public class TextFieldMapper extends FieldMapper {
if (prefixFieldMapper != null) {
prefixFieldMapper.addField(value, fields);
}
if (phraseFieldMapper != null) {
fields.add(new Field(phraseFieldMapper.fieldType.name(), value, phraseFieldMapper.fieldType));
}
}
}
@Override
public Iterator<Mapper> iterator() {
if (prefixFieldMapper == null) {
List<Mapper> subIterators = new ArrayList<>();
if (prefixFieldMapper != null) {
subIterators.add(prefixFieldMapper);
}
if (phraseFieldMapper != null) {
subIterators.add(phraseFieldMapper);
}
if (subIterators.size() == 0) {
return super.iterator();
}
return Iterators.concat(super.iterator(), Collections.singleton(prefixFieldMapper).iterator());
return Iterators.concat(super.iterator(), subIterators.iterator());
}
@Override
@ -562,6 +778,10 @@ public class TextFieldMapper extends FieldMapper {
throw new IllegalArgumentException("mapper [" + name() + "] has different index_prefix settings, current ["
+ this.prefixFieldMapper + "], merged [" + mw.prefixFieldMapper + "]");
}
else if (this.fieldType().indexPhrases != mw.fieldType().indexPhrases) {
throw new IllegalArgumentException("mapper [" + name() + "] has different index_phrases settings, current ["
+ this.fieldType().indexPhrases + "], merged [" + mw.fieldType().indexPhrases + "]");
}
}
@Override
@ -602,5 +822,8 @@ public class TextFieldMapper extends FieldMapper {
if (fieldType().prefixFieldType != null) {
fieldType().prefixFieldType.doXContent(builder);
}
if (fieldType().indexPhrases) {
builder.field("index_phrases", fieldType().indexPhrases);
}
}
}

View File

@ -28,6 +28,7 @@ import org.elasticsearch.common.io.stream.StreamInput;
import org.elasticsearch.common.io.stream.StreamOutput;
import org.elasticsearch.common.xcontent.XContentBuilder;
import org.elasticsearch.common.xcontent.XContentParser;
import org.elasticsearch.index.mapper.MappedFieldType;
import org.elasticsearch.index.search.MatchQuery;
import org.elasticsearch.index.search.MatchQuery.ZeroTermsQuery;

View File

@ -352,21 +352,41 @@ public class MatchQuery {
@Override
protected Query analyzePhrase(String field, TokenStream stream, int slop) throws IOException {
if (hasPositions(mapper) == false) {
IllegalStateException exc =
new IllegalStateException("field:[" + field + "] was indexed without position data; cannot run PhraseQuery");
if (lenient) {
return newLenientFieldQuery(field, exc);
} else {
throw exc;
try {
checkForPositions(field);
Query query = mapper.phraseQuery(field, stream, slop, enablePositionIncrements);
if (query instanceof PhraseQuery) {
// synonyms that expand to multiple terms can return a phrase query.
return blendPhraseQuery((PhraseQuery) query, mapper);
}
return query;
}
Query query = super.analyzePhrase(field, stream, slop);
if (query instanceof PhraseQuery) {
// synonyms that expand to multiple terms can return a phrase query.
return blendPhraseQuery((PhraseQuery) query, mapper);
catch (IllegalArgumentException | IllegalStateException e) {
if (lenient) {
return newLenientFieldQuery(field, e);
}
throw e;
}
}
@Override
protected Query analyzeMultiPhrase(String field, TokenStream stream, int slop) throws IOException {
try {
checkForPositions(field);
return mapper.multiPhraseQuery(field, stream, slop, enablePositionIncrements);
}
catch (IllegalArgumentException | IllegalStateException e) {
if (lenient) {
return newLenientFieldQuery(field, e);
}
throw e;
}
}
private void checkForPositions(String field) {
if (hasPositions(mapper) == false) {
throw new IllegalStateException("field:[" + field + "] was indexed without position data; cannot run PhraseQuery");
}
return query;
}
/**

View File

@ -19,6 +19,7 @@
package org.elasticsearch.ingest;
import org.elasticsearch.Version;
import org.elasticsearch.cluster.Diff;
import org.elasticsearch.cluster.DiffableUtils;
import org.elasticsearch.cluster.NamedDiff;
@ -69,6 +70,11 @@ public final class IngestMetadata implements MetaData.Custom {
return TYPE;
}
@Override
public Version getMinimalSupportedVersion() {
return Version.CURRENT.minimumCompatibilityVersion();
}
public Map<String, PipelineConfiguration> getPipelines() {
return pipelines;
}

View File

@ -35,7 +35,7 @@ public class NodePersistentTasksExecutor {
this.threadPool = threadPool;
}
public <Params extends PersistentTaskParams> void executeTask(@Nullable Params params,
public <Params extends PersistentTaskParams> void executeTask(Params params,
@Nullable Task.Status status,
AllocatedPersistentTask task,
PersistentTasksExecutor<Params> executor) {

View File

@ -19,12 +19,13 @@
package org.elasticsearch.persistent;
import org.elasticsearch.common.io.stream.NamedWriteable;
import org.elasticsearch.cluster.ClusterState;
import org.elasticsearch.common.io.stream.VersionedNamedWriteable;
import org.elasticsearch.common.xcontent.ToXContentObject;
/**
* Parameters used to start persistent task
*/
public interface PersistentTaskParams extends NamedWriteable, ToXContentObject {
public interface PersistentTaskParams extends VersionedNamedWriteable, ToXContentObject, ClusterState.FeatureAware {
}

View File

@ -29,7 +29,6 @@ import org.elasticsearch.cluster.ClusterStateUpdateTask;
import org.elasticsearch.cluster.metadata.MetaData;
import org.elasticsearch.cluster.node.DiscoveryNodes;
import org.elasticsearch.cluster.service.ClusterService;
import org.elasticsearch.common.Nullable;
import org.elasticsearch.common.component.AbstractComponent;
import org.elasticsearch.common.settings.Settings;
import org.elasticsearch.persistent.PersistentTasksCustomMetaData.Assignment;
@ -65,7 +64,7 @@ public class PersistentTasksClusterService extends AbstractComponent implements
* @param taskParams the task's parameters
* @param listener the listener that will be called when task is started
*/
public <Params extends PersistentTaskParams> void createPersistentTask(String taskId, String taskName, @Nullable Params taskParams,
public <Params extends PersistentTaskParams> void createPersistentTask(String taskId, String taskName, Params taskParams,
ActionListener<PersistentTask<?>> listener) {
clusterService.submitStateUpdateTask("create persistent task", new ClusterStateUpdateTask() {
@Override
@ -225,7 +224,7 @@ public class PersistentTasksClusterService extends AbstractComponent implements
* @return a new {@link Assignment}
*/
private <Params extends PersistentTaskParams> Assignment createAssignment(final String taskName,
final @Nullable Params taskParams,
final Params taskParams,
final ClusterState currentState) {
PersistentTasksExecutor<Params> persistentTasksExecutor = registry.getPersistentTaskExecutorSafe(taskName);

View File

@ -49,8 +49,8 @@ import java.util.HashMap;
import java.util.List;
import java.util.Map;
import java.util.Objects;
import java.util.Optional;
import java.util.Set;
import java.util.function.Function;
import java.util.function.Predicate;
import java.util.stream.Collectors;
@ -264,7 +264,6 @@ public final class PersistentTasksCustomMetaData extends AbstractNamedDiffable<M
private final String id;
private final long allocationId;
private final String taskName;
@Nullable
private final P params;
@Nullable
private final Status status;
@ -314,7 +313,11 @@ public final class PersistentTasksCustomMetaData extends AbstractNamedDiffable<M
id = in.readString();
allocationId = in.readLong();
taskName = in.readString();
params = (P) in.readOptionalNamedWriteable(PersistentTaskParams.class);
if (in.getVersion().onOrAfter(Version.V_6_3_0)) {
params = (P) in.readNamedWriteable(PersistentTaskParams.class);
} else {
params = (P) in.readOptionalNamedWriteable(PersistentTaskParams.class);
}
status = in.readOptionalNamedWriteable(Task.Status.class);
assignment = new Assignment(in.readOptionalString(), in.readString());
allocationIdOnLastStatusUpdate = in.readOptionalLong();
@ -325,7 +328,11 @@ public final class PersistentTasksCustomMetaData extends AbstractNamedDiffable<M
out.writeString(id);
out.writeLong(allocationId);
out.writeString(taskName);
out.writeOptionalNamedWriteable(params);
if (out.getVersion().onOrAfter(Version.V_6_3_0)) {
out.writeNamedWriteable(params);
} else {
out.writeOptionalNamedWriteable(params);
}
out.writeOptionalNamedWriteable(status);
out.writeOptionalString(assignment.executorNode);
out.writeString(assignment.explanation);
@ -500,7 +507,10 @@ public final class PersistentTasksCustomMetaData extends AbstractNamedDiffable<M
@Override
public void writeTo(StreamOutput out) throws IOException {
out.writeLong(lastAllocationId);
out.writeMap(tasks, StreamOutput::writeString, (stream, value) -> value.writeTo(stream));
Map<String, PersistentTask<?>> filteredTasks = tasks.values().stream()
.filter(t -> ClusterState.FeatureAware.shouldSerialize(out, t.getParams()))
.collect(Collectors.toMap(PersistentTask::getId, Function.identity()));
out.writeMap(filteredTasks, StreamOutput::writeString, (stream, value) -> value.writeTo(stream));
}
public static NamedDiff<MetaData.Custom> readDiffFrom(StreamInput in) throws IOException {

View File

@ -24,10 +24,10 @@ import org.elasticsearch.cluster.node.DiscoveryNode;
import org.elasticsearch.common.Nullable;
import org.elasticsearch.common.component.AbstractComponent;
import org.elasticsearch.common.settings.Settings;
import org.elasticsearch.tasks.Task;
import org.elasticsearch.tasks.TaskId;
import org.elasticsearch.persistent.PersistentTasksCustomMetaData.Assignment;
import org.elasticsearch.persistent.PersistentTasksCustomMetaData.PersistentTask;
import org.elasticsearch.tasks.Task;
import org.elasticsearch.tasks.TaskId;
import java.util.Map;
import java.util.function.Predicate;
@ -118,7 +118,7 @@ public abstract class PersistentTasksExecutor<Params extends PersistentTaskParam
* NOTE: The nodeOperation has to throw an exception, trigger task.markAsCompleted() or task.completeAndNotifyIfNeeded() methods to
* indicate that the persistent task has finished.
*/
protected abstract void nodeOperation(AllocatedPersistentTask task, @Nullable Params params, @Nullable Task.Status status);
protected abstract void nodeOperation(AllocatedPersistentTask task, Params params, @Nullable Task.Status status);
public String getExecutor() {
return executor;

View File

@ -21,7 +21,6 @@ package org.elasticsearch.persistent;
import org.elasticsearch.action.Action;
import org.elasticsearch.action.ActionListener;
import org.elasticsearch.action.ActionRequest;
import org.elasticsearch.action.ActionRequestBuilder;
import org.elasticsearch.action.admin.cluster.node.tasks.cancel.CancelTasksRequest;
import org.elasticsearch.action.admin.cluster.node.tasks.cancel.CancelTasksResponse;
import org.elasticsearch.action.support.ContextPreservingActionListener;
@ -69,7 +68,7 @@ public class PersistentTasksService extends AbstractComponent {
*/
public <Params extends PersistentTaskParams> void sendStartRequest(final String taskId,
final String taskName,
final @Nullable Params taskParams,
final Params taskParams,
final ActionListener<PersistentTask<Params>> listener) {
@SuppressWarnings("unchecked")
final ActionListener<PersistentTask<?>> wrappedListener =

View File

@ -18,6 +18,7 @@
*/
package org.elasticsearch.persistent;
import org.elasticsearch.Version;
import org.elasticsearch.action.Action;
import org.elasticsearch.action.ActionListener;
import org.elasticsearch.action.ActionRequestValidationException;
@ -36,9 +37,9 @@ import org.elasticsearch.common.inject.Inject;
import org.elasticsearch.common.io.stream.StreamInput;
import org.elasticsearch.common.io.stream.StreamOutput;
import org.elasticsearch.common.settings.Settings;
import org.elasticsearch.persistent.PersistentTasksCustomMetaData.PersistentTask;
import org.elasticsearch.threadpool.ThreadPool;
import org.elasticsearch.transport.TransportService;
import org.elasticsearch.persistent.PersistentTasksCustomMetaData.PersistentTask;
import java.io.IOException;
import java.util.Objects;
@ -66,7 +67,6 @@ public class StartPersistentTaskAction extends Action<StartPersistentTaskAction.
private String taskId;
@Nullable
private String taskName;
private PersistentTaskParams params;
@ -86,7 +86,11 @@ public class StartPersistentTaskAction extends Action<StartPersistentTaskAction.
super.readFrom(in);
taskId = in.readString();
taskName = in.readString();
params = in.readOptionalNamedWriteable(PersistentTaskParams.class);
if (in.getVersion().onOrAfter(Version.V_6_3_0)) {
params = in.readNamedWriteable(PersistentTaskParams.class);
} else {
params = in.readOptionalNamedWriteable(PersistentTaskParams.class);
}
}
@Override
@ -94,7 +98,11 @@ public class StartPersistentTaskAction extends Action<StartPersistentTaskAction.
super.writeTo(out);
out.writeString(taskId);
out.writeString(taskName);
out.writeOptionalNamedWriteable(params);
if (out.getVersion().onOrAfter(Version.V_6_3_0)) {
out.writeNamedWriteable(params);
} else {
out.writeOptionalNamedWriteable(params);
}
}
@Override

View File

@ -383,6 +383,11 @@ public final class ScriptMetaData implements MetaData.Custom, Writeable, ToXCont
return TYPE;
}
@Override
public Version getMinimalSupportedVersion() {
return Version.CURRENT.minimumCompatibilityVersion();
}
@Override
public EnumSet<MetaData.XContentContext> context() {
return MetaData.ALL_CONTEXTS;

View File

@ -36,7 +36,7 @@ import org.apache.lucene.search.vectorhighlight.SingleFragListBuilder;
import org.elasticsearch.common.settings.Setting;
import org.elasticsearch.common.settings.Settings;
import org.elasticsearch.common.text.Text;
import org.elasticsearch.index.mapper.FieldMapper;
import org.elasticsearch.index.mapper.MappedFieldType;
import org.elasticsearch.search.fetch.FetchPhaseExecutionException;
import org.elasticsearch.search.fetch.FetchSubPhase;
import org.elasticsearch.search.fetch.subphase.highlight.SearchContextHighlight.Field;
@ -71,9 +71,9 @@ public class FastVectorHighlighter implements Highlighter {
SearchContextHighlight.Field field = highlighterContext.field;
SearchContext context = highlighterContext.context;
FetchSubPhase.HitContext hitContext = highlighterContext.hitContext;
FieldMapper mapper = highlighterContext.mapper;
MappedFieldType fieldType = highlighterContext.fieldType;
if (canHighlight(mapper) == false) {
if (canHighlight(fieldType) == false) {
throw new IllegalArgumentException("the field [" + highlighterContext.fieldName +
"] should be indexed with term vector with position offsets to be used with fast vector highlighter");
}
@ -87,7 +87,7 @@ public class FastVectorHighlighter implements Highlighter {
HighlighterEntry cache = (HighlighterEntry) hitContext.cache().get(CACHE_KEY);
try {
MapperHighlightEntry entry = cache.mappers.get(mapper);
FieldHighlightEntry entry = cache.fields.get(fieldType);
if (entry == null) {
FragListBuilder fragListBuilder;
BaseFragmentsBuilder fragmentsBuilder;
@ -97,37 +97,37 @@ public class FastVectorHighlighter implements Highlighter {
if (field.fieldOptions().numberOfFragments() == 0) {
fragListBuilder = new SingleFragListBuilder();
if (!forceSource && mapper.fieldType().stored()) {
fragmentsBuilder = new SimpleFragmentsBuilder(mapper, field.fieldOptions().preTags(),
if (!forceSource && fieldType.stored()) {
fragmentsBuilder = new SimpleFragmentsBuilder(fieldType, field.fieldOptions().preTags(),
field.fieldOptions().postTags(), boundaryScanner);
} else {
fragmentsBuilder = new SourceSimpleFragmentsBuilder(mapper, context,
fragmentsBuilder = new SourceSimpleFragmentsBuilder(fieldType, context,
field.fieldOptions().preTags(), field.fieldOptions().postTags(), boundaryScanner);
}
} else {
fragListBuilder = field.fieldOptions().fragmentOffset() == -1 ?
new SimpleFragListBuilder() : new SimpleFragListBuilder(field.fieldOptions().fragmentOffset());
if (field.fieldOptions().scoreOrdered()) {
if (!forceSource && mapper.fieldType().stored()) {
if (!forceSource && fieldType.stored()) {
fragmentsBuilder = new ScoreOrderFragmentsBuilder(field.fieldOptions().preTags(),
field.fieldOptions().postTags(), boundaryScanner);
} else {
fragmentsBuilder = new SourceScoreOrderFragmentsBuilder(mapper, context,
fragmentsBuilder = new SourceScoreOrderFragmentsBuilder(fieldType, context,
field.fieldOptions().preTags(), field.fieldOptions().postTags(), boundaryScanner);
}
} else {
if (!forceSource && mapper.fieldType().stored()) {
fragmentsBuilder = new SimpleFragmentsBuilder(mapper, field.fieldOptions().preTags(),
if (!forceSource && fieldType.stored()) {
fragmentsBuilder = new SimpleFragmentsBuilder(fieldType, field.fieldOptions().preTags(),
field.fieldOptions().postTags(), boundaryScanner);
} else {
fragmentsBuilder =
new SourceSimpleFragmentsBuilder(mapper, context, field.fieldOptions().preTags(),
new SourceSimpleFragmentsBuilder(fieldType, context, field.fieldOptions().preTags(),
field.fieldOptions().postTags(), boundaryScanner);
}
}
}
fragmentsBuilder.setDiscreteMultiValueHighlighting(termVectorMultiValue);
entry = new MapperHighlightEntry();
entry = new FieldHighlightEntry();
if (field.fieldOptions().requireFieldMatch()) {
/**
* we use top level reader to rewrite the query against all readers,
@ -152,7 +152,7 @@ public class FastVectorHighlighter implements Highlighter {
cache.fvh = new org.apache.lucene.search.vectorhighlight.FastVectorHighlighter();
}
CustomFieldQuery.highlightFilters.set(field.fieldOptions().highlightFilter());
cache.mappers.put(mapper, entry);
cache.fields.put(fieldType, entry);
}
final FieldQuery fieldQuery;
if (field.fieldOptions().requireFieldMatch()) {
@ -173,12 +173,12 @@ public class FastVectorHighlighter implements Highlighter {
// Only send matched fields if they were requested to save time.
if (field.fieldOptions().matchedFields() != null && !field.fieldOptions().matchedFields().isEmpty()) {
fragments = cache.fvh.getBestFragments(fieldQuery, hitContext.reader(), hitContext.docId(),
mapper.fieldType().name(), field.fieldOptions().matchedFields(), fragmentCharSize,
fieldType.name(), field.fieldOptions().matchedFields(), fragmentCharSize,
numberOfFragments, entry.fragListBuilder, entry.fragmentsBuilder, field.fieldOptions().preTags(),
field.fieldOptions().postTags(), encoder);
} else {
fragments = cache.fvh.getBestFragments(fieldQuery, hitContext.reader(), hitContext.docId(),
mapper.fieldType().name(), fragmentCharSize, numberOfFragments, entry.fragListBuilder,
fieldType.name(), fragmentCharSize, numberOfFragments, entry.fragListBuilder,
entry.fragmentsBuilder, field.fieldOptions().preTags(), field.fieldOptions().postTags(), encoder);
}
@ -193,7 +193,7 @@ public class FastVectorHighlighter implements Highlighter {
FieldFragList fieldFragList = new SimpleFieldFragList(-1 /*ignored*/);
fieldFragList.add(0, noMatchSize, Collections.<WeightedPhraseInfo>emptyList());
fragments = entry.fragmentsBuilder.createFragments(hitContext.reader(), hitContext.docId(),
mapper.fieldType().name(), fieldFragList, 1, field.fieldOptions().preTags(),
fieldType.name(), fieldFragList, 1, field.fieldOptions().preTags(),
field.fieldOptions().postTags(), encoder);
if (fragments != null && fragments.length > 0) {
return new HighlightField(highlighterContext.fieldName, Text.convertFromStringArray(fragments));
@ -209,9 +209,10 @@ public class FastVectorHighlighter implements Highlighter {
}
@Override
public boolean canHighlight(FieldMapper fieldMapper) {
return fieldMapper.fieldType().storeTermVectors() && fieldMapper.fieldType().storeTermVectorOffsets()
&& fieldMapper.fieldType().storeTermVectorPositions();
public boolean canHighlight(MappedFieldType fieldType) {
return fieldType.storeTermVectors()
&& fieldType.storeTermVectorOffsets()
&& fieldType.storeTermVectorPositions();
}
private static BoundaryScanner getBoundaryScanner(Field field) {
@ -244,7 +245,7 @@ public class FastVectorHighlighter implements Highlighter {
}
}
private class MapperHighlightEntry {
private class FieldHighlightEntry {
public FragListBuilder fragListBuilder;
public FragmentsBuilder fragmentsBuilder;
public FieldQuery noFieldMatchFieldQuery;
@ -253,6 +254,6 @@ public class FastVectorHighlighter implements Highlighter {
private class HighlighterEntry {
public org.apache.lucene.search.vectorhighlight.FastVectorHighlighter fvh;
public Map<FieldMapper, MapperHighlightEntry> mappers = new HashMap<>();
public Map<MappedFieldType, FieldHighlightEntry> fields = new HashMap<>();
}
}

View File

@ -29,7 +29,7 @@ import org.apache.lucene.util.CollectionUtil;
import org.elasticsearch.index.analysis.CustomAnalyzer;
import org.elasticsearch.index.analysis.NamedAnalyzer;
import org.elasticsearch.index.analysis.TokenFilterFactory;
import org.elasticsearch.index.mapper.FieldMapper;
import org.elasticsearch.index.mapper.MappedFieldType;
import java.util.Comparator;
import java.util.List;
@ -47,10 +47,10 @@ public final class FragmentBuilderHelper {
* Fixes problems with broken analysis chains if positions and offsets are messed up that can lead to
* {@link StringIndexOutOfBoundsException} in the {@link FastVectorHighlighter}
*/
public static WeightedFragInfo fixWeightedFragInfo(FieldMapper mapper, Field[] values, WeightedFragInfo fragInfo) {
public static WeightedFragInfo fixWeightedFragInfo(MappedFieldType fieldType, Field[] values, WeightedFragInfo fragInfo) {
assert fragInfo != null : "FragInfo must not be null";
assert mapper.fieldType().name().equals(values[0].name()) : "Expected FieldMapper for field " + values[0].name();
if (!fragInfo.getSubInfos().isEmpty() && containsBrokenAnalysis(mapper.fieldType().indexAnalyzer())) {
assert fieldType.name().equals(values[0].name()) : "Expected MappedFieldType for field " + values[0].name();
if (!fragInfo.getSubInfos().isEmpty() && containsBrokenAnalysis(fieldType.indexAnalyzer())) {
/* This is a special case where broken analysis like WDF is used for term-vector creation at index-time
* which can potentially mess up the offsets. To prevent a SAIIOBException we need to resort
* the fragments based on their offsets rather than using soley the positions as it is done in

View File

@ -24,18 +24,16 @@ import org.elasticsearch.common.component.AbstractComponent;
import org.elasticsearch.common.regex.Regex;
import org.elasticsearch.common.settings.Settings;
import org.elasticsearch.index.mapper.DocumentMapper;
import org.elasticsearch.index.mapper.FieldMapper;
import org.elasticsearch.index.mapper.KeywordFieldMapper;
import org.elasticsearch.index.mapper.MappedFieldType;
import org.elasticsearch.index.mapper.SourceFieldMapper;
import org.elasticsearch.index.mapper.TextFieldMapper;
import org.elasticsearch.search.fetch.FetchSubPhase;
import org.elasticsearch.search.internal.SearchContext;
import java.util.Arrays;
import java.util.Collection;
import java.util.Collections;
import java.util.HashMap;
import java.util.List;
import java.util.Map;
public class HighlightPhase extends AbstractComponent implements FetchSubPhase {
@ -71,8 +69,8 @@ public class HighlightPhase extends AbstractComponent implements FetchSubPhase {
boolean fieldNameContainsWildcards = field.field().contains("*");
for (String fieldName : fieldNamesToHighlight) {
FieldMapper fieldMapper = getMapperForField(fieldName, context, hitContext);
if (fieldMapper == null) {
MappedFieldType fieldType = context.mapperService().fullName(fieldName);
if (fieldType == null) {
continue;
}
@ -85,8 +83,8 @@ public class HighlightPhase extends AbstractComponent implements FetchSubPhase {
// If the field was explicitly given we assume that whoever issued the query knew
// what they were doing and try to highlight anyway.
if (fieldNameContainsWildcards) {
if (fieldMapper.fieldType().typeName().equals(TextFieldMapper.CONTENT_TYPE) == false &&
fieldMapper.fieldType().typeName().equals(KeywordFieldMapper.CONTENT_TYPE) == false) {
if (fieldType.typeName().equals(TextFieldMapper.CONTENT_TYPE) == false &&
fieldType.typeName().equals(KeywordFieldMapper.CONTENT_TYPE) == false) {
continue;
}
}
@ -104,10 +102,10 @@ public class HighlightPhase extends AbstractComponent implements FetchSubPhase {
if (highlightQuery == null) {
highlightQuery = context.parsedQuery().query();
}
HighlighterContext highlighterContext = new HighlighterContext(fieldName, field, fieldMapper, context,
hitContext, highlightQuery);
HighlighterContext highlighterContext = new HighlighterContext(fieldName,
field, fieldType, context, hitContext, highlightQuery);
if ((highlighter.canHighlight(fieldMapper) == false) && fieldNameContainsWildcards) {
if ((highlighter.canHighlight(fieldType) == false) && fieldNameContainsWildcards) {
// if several fieldnames matched the wildcard then we want to skip those that we cannot highlight
continue;
}
@ -119,10 +117,4 @@ public class HighlightPhase extends AbstractComponent implements FetchSubPhase {
}
hitContext.hit().highlightFields(highlightFields);
}
private FieldMapper getMapperForField(String fieldName, SearchContext searchContext, HitContext hitContext) {
DocumentMapper documentMapper = searchContext.mapperService().documentMapper(hitContext.hit().getType());
// TODO: no need to lookup the doc mapper with unambiguous field names? just look at the mapper service
return documentMapper.mappers().smartNameFieldMapper(fieldName);
}
}

View File

@ -22,7 +22,7 @@ import org.apache.lucene.search.highlight.DefaultEncoder;
import org.apache.lucene.search.highlight.Encoder;
import org.apache.lucene.search.highlight.SimpleHTMLEncoder;
import org.elasticsearch.index.fieldvisitor.CustomFieldsVisitor;
import org.elasticsearch.index.mapper.FieldMapper;
import org.elasticsearch.index.mapper.MappedFieldType;
import org.elasticsearch.search.fetch.FetchSubPhase;
import org.elasticsearch.search.internal.SearchContext;
import org.elasticsearch.search.lookup.SourceLookup;
@ -46,15 +46,17 @@ public final class HighlightUtils {
/**
* Load field values for highlighting.
*/
public static List<Object> loadFieldValues(SearchContextHighlight.Field field, FieldMapper mapper, SearchContext searchContext,
FetchSubPhase.HitContext hitContext) throws IOException {
public static List<Object> loadFieldValues(SearchContextHighlight.Field field,
MappedFieldType fieldType,
SearchContext searchContext,
FetchSubPhase.HitContext hitContext) throws IOException {
//percolator needs to always load from source, thus it sets the global force source to true
boolean forceSource = searchContext.highlight().forceSource(field);
List<Object> textsToHighlight;
if (!forceSource && mapper.fieldType().stored()) {
CustomFieldsVisitor fieldVisitor = new CustomFieldsVisitor(singleton(mapper.fieldType().name()), false);
if (!forceSource && fieldType.stored()) {
CustomFieldsVisitor fieldVisitor = new CustomFieldsVisitor(singleton(fieldType.name()), false);
hitContext.reader().document(hitContext.docId(), fieldVisitor);
textsToHighlight = fieldVisitor.fields().get(mapper.fieldType().name());
textsToHighlight = fieldVisitor.fields().get(fieldType.name());
if (textsToHighlight == null) {
// Can happen if the document doesn't have the field to highlight
textsToHighlight = Collections.emptyList();
@ -62,7 +64,7 @@ public final class HighlightUtils {
} else {
SourceLookup sourceLookup = searchContext.lookup().source();
sourceLookup.setSegmentAndDocument(hitContext.readerContext(), hitContext.docId());
textsToHighlight = sourceLookup.extractRawValues(mapper.fieldType().name());
textsToHighlight = sourceLookup.extractRawValues(fieldType.name());
}
assert textsToHighlight != null;
return textsToHighlight;

View File

@ -18,7 +18,7 @@
*/
package org.elasticsearch.search.fetch.subphase.highlight;
import org.elasticsearch.index.mapper.FieldMapper;
import org.elasticsearch.index.mapper.MappedFieldType;
/**
* Highlights a search result.
@ -27,5 +27,5 @@ public interface Highlighter {
HighlightField highlight(HighlighterContext highlighterContext);
boolean canHighlight(FieldMapper fieldMapper);
boolean canHighlight(MappedFieldType fieldType);
}

View File

@ -19,7 +19,7 @@
package org.elasticsearch.search.fetch.subphase.highlight;
import org.apache.lucene.search.Query;
import org.elasticsearch.index.mapper.FieldMapper;
import org.elasticsearch.index.mapper.MappedFieldType;
import org.elasticsearch.search.fetch.FetchSubPhase;
import org.elasticsearch.search.internal.SearchContext;
@ -27,16 +27,20 @@ public class HighlighterContext {
public final String fieldName;
public final SearchContextHighlight.Field field;
public final FieldMapper mapper;
public final MappedFieldType fieldType;
public final SearchContext context;
public final FetchSubPhase.HitContext hitContext;
public final Query query;
public HighlighterContext(String fieldName, SearchContextHighlight.Field field, FieldMapper mapper, SearchContext context,
FetchSubPhase.HitContext hitContext, Query query) {
public HighlighterContext(String fieldName,
SearchContextHighlight.Field field,
MappedFieldType fieldType,
SearchContext context,
FetchSubPhase.HitContext hitContext,
Query query) {
this.fieldName = fieldName;
this.field = field;
this.mapper = mapper;
this.fieldType = fieldType;
this.context = context;
this.hitContext = hitContext;
this.query = query;

View File

@ -36,7 +36,7 @@ import org.apache.lucene.util.CollectionUtil;
import org.elasticsearch.ExceptionsHelper;
import org.elasticsearch.common.text.Text;
import org.elasticsearch.index.IndexSettings;
import org.elasticsearch.index.mapper.FieldMapper;
import org.elasticsearch.index.mapper.MappedFieldType;
import org.elasticsearch.search.fetch.FetchPhaseExecutionException;
import org.elasticsearch.search.fetch.FetchSubPhase;
import org.elasticsearch.search.internal.SearchContext;
@ -59,22 +59,21 @@ public class PlainHighlighter implements Highlighter {
SearchContextHighlight.Field field = highlighterContext.field;
SearchContext context = highlighterContext.context;
FetchSubPhase.HitContext hitContext = highlighterContext.hitContext;
FieldMapper mapper = highlighterContext.mapper;
MappedFieldType fieldType = highlighterContext.fieldType;
Encoder encoder = field.fieldOptions().encoder().equals("html") ? HighlightUtils.Encoders.HTML : HighlightUtils.Encoders.DEFAULT;
if (!hitContext.cache().containsKey(CACHE_KEY)) {
Map<FieldMapper, org.apache.lucene.search.highlight.Highlighter> mappers = new HashMap<>();
hitContext.cache().put(CACHE_KEY, mappers);
hitContext.cache().put(CACHE_KEY, new HashMap<>());
}
@SuppressWarnings("unchecked")
Map<FieldMapper, org.apache.lucene.search.highlight.Highlighter> cache =
(Map<FieldMapper, org.apache.lucene.search.highlight.Highlighter>) hitContext.cache().get(CACHE_KEY);
Map<MappedFieldType, org.apache.lucene.search.highlight.Highlighter> cache =
(Map<MappedFieldType, org.apache.lucene.search.highlight.Highlighter>) hitContext.cache().get(CACHE_KEY);
org.apache.lucene.search.highlight.Highlighter entry = cache.get(mapper);
org.apache.lucene.search.highlight.Highlighter entry = cache.get(fieldType);
if (entry == null) {
QueryScorer queryScorer = new CustomQueryScorer(highlighterContext.query,
field.fieldOptions().requireFieldMatch() ? mapper.fieldType().name() : null);
field.fieldOptions().requireFieldMatch() ? fieldType.name() : null);
queryScorer.setExpandMultiTermQuery(true);
Fragmenter fragmenter;
if (field.fieldOptions().numberOfFragments() == 0) {
@ -96,21 +95,21 @@ public class PlainHighlighter implements Highlighter {
// always highlight across all data
entry.setMaxDocCharsToAnalyze(Integer.MAX_VALUE);
cache.put(mapper, entry);
cache.put(fieldType, entry);
}
// a HACK to make highlighter do highlighting, even though its using the single frag list builder
int numberOfFragments = field.fieldOptions().numberOfFragments() == 0 ? 1 : field.fieldOptions().numberOfFragments();
ArrayList<TextFragment> fragsList = new ArrayList<>();
List<Object> textsToHighlight;
Analyzer analyzer = getAnalyzer(context.mapperService().documentMapper(hitContext.hit().getType()), mapper.fieldType());
Analyzer analyzer = getAnalyzer(context.mapperService().documentMapper(hitContext.hit().getType()), fieldType);
final int maxAnalyzedOffset = context.indexShard().indexSettings().getHighlightMaxAnalyzedOffset();
try {
textsToHighlight = HighlightUtils.loadFieldValues(field, mapper, context, hitContext);
textsToHighlight = HighlightUtils.loadFieldValues(field, fieldType, context, hitContext);
for (Object textToHighlight : textsToHighlight) {
String text = convertFieldValue(mapper.fieldType(), textToHighlight);
String text = convertFieldValue(fieldType, textToHighlight);
if (text.length() > maxAnalyzedOffset) {
throw new IllegalArgumentException(
"The length of [" + highlighterContext.fieldName + "] field of [" + hitContext.hit().getId() +
@ -121,7 +120,7 @@ public class PlainHighlighter implements Highlighter {
"with unified or fvh highlighter is recommended!");
}
try (TokenStream tokenStream = analyzer.tokenStream(mapper.fieldType().name(), text)) {
try (TokenStream tokenStream = analyzer.tokenStream(fieldType.name(), text)) {
if (!tokenStream.hasAttribute(CharTermAttribute.class) || !tokenStream.hasAttribute(OffsetAttribute.class)) {
// can't perform highlighting if the stream has no terms (binary token stream) or no offsets
continue;
@ -178,7 +177,7 @@ public class PlainHighlighter implements Highlighter {
String fieldContents = textsToHighlight.get(0).toString();
int end;
try {
end = findGoodEndForNoHighlightExcerpt(noMatchSize, analyzer, mapper.fieldType().name(), fieldContents);
end = findGoodEndForNoHighlightExcerpt(noMatchSize, analyzer, fieldType.name(), fieldContents);
} catch (Exception e) {
throw new FetchPhaseExecutionException(context, "Failed to highlight field [" + highlighterContext.fieldName + "]", e);
}
@ -190,7 +189,7 @@ public class PlainHighlighter implements Highlighter {
}
@Override
public boolean canHighlight(FieldMapper fieldMapper) {
public boolean canHighlight(MappedFieldType fieldType) {
return true;
}

View File

@ -23,24 +23,27 @@ import org.apache.lucene.search.highlight.Encoder;
import org.apache.lucene.search.vectorhighlight.BoundaryScanner;
import org.apache.lucene.search.vectorhighlight.FieldFragList.WeightedFragInfo;
import org.elasticsearch.index.mapper.FieldMapper;
import org.elasticsearch.index.mapper.MappedFieldType;
/**
* Direct Subclass of Lucene's org.apache.lucene.search.vectorhighlight.SimpleFragmentsBuilder
* that corrects offsets for broken analysis chains.
* Direct Subclass of Lucene's org.apache.lucene.search.vectorhighlight.SimpleFragmentsBuilder
* that corrects offsets for broken analysis chains.
*/
public class SimpleFragmentsBuilder extends org.apache.lucene.search.vectorhighlight.SimpleFragmentsBuilder {
protected final FieldMapper mapper;
protected final MappedFieldType fieldType;
public SimpleFragmentsBuilder(FieldMapper mapper,
String[] preTags, String[] postTags, BoundaryScanner boundaryScanner) {
public SimpleFragmentsBuilder(MappedFieldType fieldType,
String[] preTags,
String[] postTags,
BoundaryScanner boundaryScanner) {
super(preTags, postTags, boundaryScanner);
this.mapper = mapper;
this.fieldType = fieldType;
}
@Override
protected String makeFragment( StringBuilder buffer, int[] index, Field[] values, WeightedFragInfo fragInfo,
String[] preTags, String[] postTags, Encoder encoder ){
return super.makeFragment(buffer, index, values, FragmentBuilderHelper.fixWeightedFragInfo(mapper, values, fragInfo),
preTags, postTags, encoder);
WeightedFragInfo weightedFragInfo = FragmentBuilderHelper.fixWeightedFragInfo(fieldType, values, fragInfo);
return super.makeFragment(buffer, index, values, weightedFragInfo, preTags, postTags, encoder);
}
}

View File

@ -26,7 +26,7 @@ import org.apache.lucene.search.highlight.Encoder;
import org.apache.lucene.search.vectorhighlight.BoundaryScanner;
import org.apache.lucene.search.vectorhighlight.FieldFragList.WeightedFragInfo;
import org.apache.lucene.search.vectorhighlight.ScoreOrderFragmentsBuilder;
import org.elasticsearch.index.mapper.FieldMapper;
import org.elasticsearch.index.mapper.MappedFieldType;
import org.elasticsearch.search.internal.SearchContext;
import org.elasticsearch.search.lookup.SourceLookup;
@ -35,14 +35,17 @@ import java.util.List;
public class SourceScoreOrderFragmentsBuilder extends ScoreOrderFragmentsBuilder {
private final FieldMapper mapper;
private final MappedFieldType fieldType;
private final SearchContext searchContext;
public SourceScoreOrderFragmentsBuilder(FieldMapper mapper, SearchContext searchContext, String[] preTags, String[] postTags,
public SourceScoreOrderFragmentsBuilder(MappedFieldType fieldType,
SearchContext searchContext,
String[] preTags,
String[] postTags,
BoundaryScanner boundaryScanner) {
super(preTags, postTags, boundaryScanner);
this.mapper = mapper;
this.fieldType = fieldType;
this.searchContext = searchContext;
}
@ -52,10 +55,10 @@ public class SourceScoreOrderFragmentsBuilder extends ScoreOrderFragmentsBuilder
SourceLookup sourceLookup = searchContext.lookup().source();
sourceLookup.setSegmentAndDocument((LeafReaderContext) reader.getContext(), docId);
List<Object> values = sourceLookup.extractRawValues(mapper.fieldType().name());
List<Object> values = sourceLookup.extractRawValues(fieldType.name());
Field[] fields = new Field[values.size()];
for (int i = 0; i < values.size(); i++) {
fields[i] = new Field(mapper.fieldType().name(), values.get(i).toString(), TextField.TYPE_NOT_STORED);
fields[i] = new Field(fieldType.name(), values.get(i).toString(), TextField.TYPE_NOT_STORED);
}
return fields;
}
@ -63,7 +66,7 @@ public class SourceScoreOrderFragmentsBuilder extends ScoreOrderFragmentsBuilder
@Override
protected String makeFragment( StringBuilder buffer, int[] index, Field[] values, WeightedFragInfo fragInfo,
String[] preTags, String[] postTags, Encoder encoder ){
return super.makeFragment(buffer, index, values, FragmentBuilderHelper.fixWeightedFragInfo(mapper, values, fragInfo),
preTags, postTags, encoder);
WeightedFragInfo weightedFragInfo = FragmentBuilderHelper.fixWeightedFragInfo(fieldType, values, fragInfo);
return super.makeFragment(buffer, index, values, weightedFragInfo, preTags, postTags, encoder);
}
}

View File

@ -23,7 +23,7 @@ import org.apache.lucene.document.TextField;
import org.apache.lucene.index.IndexReader;
import org.apache.lucene.index.LeafReaderContext;
import org.apache.lucene.search.vectorhighlight.BoundaryScanner;
import org.elasticsearch.index.mapper.FieldMapper;
import org.elasticsearch.index.mapper.MappedFieldType;
import org.elasticsearch.search.internal.SearchContext;
import org.elasticsearch.search.lookup.SourceLookup;
@ -34,9 +34,12 @@ public class SourceSimpleFragmentsBuilder extends SimpleFragmentsBuilder {
private final SearchContext searchContext;
public SourceSimpleFragmentsBuilder(FieldMapper mapper, SearchContext searchContext, String[] preTags, String[] postTags,
public SourceSimpleFragmentsBuilder(MappedFieldType fieldType,
SearchContext searchContext,
String[] preTags,
String[] postTags,
BoundaryScanner boundaryScanner) {
super(mapper, preTags, postTags, boundaryScanner);
super(fieldType, preTags, postTags, boundaryScanner);
this.searchContext = searchContext;
}
@ -48,13 +51,13 @@ public class SourceSimpleFragmentsBuilder extends SimpleFragmentsBuilder {
SourceLookup sourceLookup = searchContext.lookup().source();
sourceLookup.setSegmentAndDocument((LeafReaderContext) reader.getContext(), docId);
List<Object> values = sourceLookup.extractRawValues(mapper.fieldType().name());
List<Object> values = sourceLookup.extractRawValues(fieldType.name());
if (values.isEmpty()) {
return EMPTY_FIELDS;
}
Field[] fields = new Field[values.size()];
for (int i = 0; i < values.size(); i++) {
fields[i] = new Field(mapper.fieldType().name(), values.get(i).toString(), TextField.TYPE_NOT_STORED);
fields[i] = new Field(fieldType.name(), values.get(i).toString(), TextField.TYPE_NOT_STORED);
}
return fields;
}

View File

@ -22,11 +22,11 @@ import org.apache.lucene.analysis.Analyzer;
import org.apache.lucene.index.IndexOptions;
import org.apache.lucene.search.IndexSearcher;
import org.apache.lucene.search.highlight.Encoder;
import org.apache.lucene.search.uhighlight.Snippet;
import org.apache.lucene.search.uhighlight.BoundedBreakIteratorScanner;
import org.apache.lucene.search.uhighlight.CustomPassageFormatter;
import org.apache.lucene.search.uhighlight.CustomSeparatorBreakIterator;
import org.apache.lucene.search.uhighlight.CustomUnifiedHighlighter;
import org.apache.lucene.search.uhighlight.Snippet;
import org.apache.lucene.search.uhighlight.UnifiedHighlighter.OffsetSource;
import org.apache.lucene.util.BytesRef;
import org.apache.lucene.util.CollectionUtil;
@ -34,7 +34,6 @@ import org.elasticsearch.common.Strings;
import org.elasticsearch.common.text.Text;
import org.elasticsearch.index.IndexSettings;
import org.elasticsearch.index.mapper.DocumentMapper;
import org.elasticsearch.index.mapper.FieldMapper;
import org.elasticsearch.index.mapper.KeywordFieldMapper;
import org.elasticsearch.index.mapper.MappedFieldType;
import org.elasticsearch.search.fetch.FetchPhaseExecutionException;
@ -52,13 +51,13 @@ import static org.apache.lucene.search.uhighlight.CustomUnifiedHighlighter.MULTI
public class UnifiedHighlighter implements Highlighter {
@Override
public boolean canHighlight(FieldMapper fieldMapper) {
public boolean canHighlight(MappedFieldType fieldType) {
return true;
}
@Override
public HighlightField highlight(HighlighterContext highlighterContext) {
FieldMapper fieldMapper = highlighterContext.mapper;
MappedFieldType fieldType = highlighterContext.fieldType;
SearchContextHighlight.Field field = highlighterContext.field;
SearchContext context = highlighterContext.context;
FetchSubPhase.HitContext hitContext = highlighterContext.hitContext;
@ -72,15 +71,15 @@ public class UnifiedHighlighter implements Highlighter {
try {
final Analyzer analyzer =
getAnalyzer(context.mapperService().documentMapper(hitContext.hit().getType()), fieldMapper.fieldType());
List<Object> fieldValues = HighlightUtils.loadFieldValues(field, fieldMapper, context, hitContext);
getAnalyzer(context.mapperService().documentMapper(hitContext.hit().getType()), fieldType);
List<Object> fieldValues = HighlightUtils.loadFieldValues(field, fieldType, context, hitContext);
fieldValues = fieldValues.stream()
.map((s) -> convertFieldValue(fieldMapper.fieldType(), s))
.map((s) -> convertFieldValue(fieldType, s))
.collect(Collectors.toList());
final IndexSearcher searcher = new IndexSearcher(hitContext.reader());
final CustomUnifiedHighlighter highlighter;
final String fieldValue = mergeFieldValues(fieldValues, MULTIVAL_SEP_CHAR);
final OffsetSource offsetSource = getOffsetSource(fieldMapper.fieldType());
final OffsetSource offsetSource = getOffsetSource(fieldType);
if ((offsetSource == OffsetSource.ANALYSIS) && (fieldValue.length() > maxAnalyzedOffset)) {
throw new IllegalArgumentException(
"The length of [" + highlighterContext.fieldName + "] field of [" + hitContext.hit().getId() +

View File

@ -19,13 +19,16 @@
package org.elasticsearch.action.admin.cluster.settings;
import org.elasticsearch.Version;
import org.elasticsearch.common.settings.ClusterSettings;
import org.elasticsearch.common.settings.Setting;
import org.elasticsearch.common.settings.Settings;
import org.elasticsearch.common.settings.Settings.Builder;
import org.elasticsearch.common.xcontent.XContentParser;
import org.elasticsearch.test.AbstractStreamableXContentTestCase;
import org.elasticsearch.test.VersionUtils;
import java.io.IOException;
import java.util.List;
import java.util.Set;
import java.util.function.Predicate;
@ -96,4 +99,9 @@ public class ClusterUpdateSettingsResponseTests extends AbstractStreamableXConte
protected ClusterUpdateSettingsResponse createBlankInstance() {
return new ClusterUpdateSettingsResponse();
}
public void testOldSerialisation() throws IOException {
ClusterUpdateSettingsResponse original = createTestInstance();
assertSerialization(original, VersionUtils.randomVersionBetween(random(), Version.V_6_0_0, Version.V_7_0_0_alpha1));
}
}

View File

@ -19,10 +19,13 @@
package org.elasticsearch.action.admin.indices.rollover;
import org.elasticsearch.Version;
import org.elasticsearch.common.unit.TimeValue;
import org.elasticsearch.common.xcontent.XContentParser;
import org.elasticsearch.test.AbstractStreamableXContentTestCase;
import org.elasticsearch.test.VersionUtils;
import java.io.IOException;
import java.util.ArrayList;
import java.util.HashMap;
import java.util.List;
@ -126,4 +129,9 @@ public class RolloverResponseTests extends AbstractStreamableXContentTestCase<Ro
throw new UnsupportedOperationException();
}
}
public void testOldSerialisation() throws IOException {
RolloverResponse original = createTestInstance();
assertSerialization(original, VersionUtils.randomVersionBetween(random(), Version.V_6_0_0, Version.V_7_0_0_alpha1));
}
}

View File

@ -308,6 +308,11 @@ public class ClusterChangedEventTests extends ESTestCase {
return "2";
}
@Override
public Version getMinimalSupportedVersion() {
return Version.CURRENT;
}
@Override
public EnumSet<MetaData.XContentContext> context() {
return EnumSet.of(MetaData.XContentContext.GATEWAY);
@ -324,6 +329,11 @@ public class ClusterChangedEventTests extends ESTestCase {
return "1";
}
@Override
public Version getMinimalSupportedVersion() {
return Version.CURRENT;
}
@Override
public EnumSet<MetaData.XContentContext> context() {
return EnumSet.of(MetaData.XContentContext.GATEWAY);

View File

@ -19,6 +19,7 @@
package org.elasticsearch.cluster;
import org.elasticsearch.Version;
import org.elasticsearch.action.admin.cluster.state.ClusterStateResponse;
import org.elasticsearch.client.Client;
import org.elasticsearch.cluster.metadata.IndexGraveyard;
@ -32,7 +33,6 @@ import org.elasticsearch.common.io.stream.NamedWriteableRegistry;
import org.elasticsearch.common.io.stream.StreamInput;
import org.elasticsearch.common.io.stream.StreamOutput;
import org.elasticsearch.common.io.stream.Writeable;
import org.elasticsearch.common.settings.Settings;
import org.elasticsearch.common.xcontent.NamedXContentRegistry;
import org.elasticsearch.common.xcontent.XContentBuilder;
import org.elasticsearch.common.xcontent.XContentParser;
@ -42,7 +42,6 @@ import org.elasticsearch.plugins.Plugin;
import org.elasticsearch.script.ScriptService;
import org.elasticsearch.test.ESIntegTestCase;
import org.elasticsearch.threadpool.ThreadPool;
import org.elasticsearch.transport.TcpTransport;
import org.elasticsearch.watcher.ResourceWatcherService;
import java.io.IOException;
@ -73,7 +72,8 @@ import static org.hamcrest.Matchers.instanceOf;
@ESIntegTestCase.ClusterScope(scope = TEST)
public class ClusterStateIT extends ESIntegTestCase {
public abstract static class Custom implements MetaData.Custom {
public abstract static
class Custom implements MetaData.Custom {
private static final ParseField VALUE = new ParseField("value");
@ -131,6 +131,11 @@ public class ClusterStateIT extends ESIntegTestCase {
return TYPE;
}
@Override
public Version getMinimalSupportedVersion() {
return Version.CURRENT;
}
@Override
public Optional<String> getRequiredFeature() {
return Optional.of("node");
@ -155,6 +160,11 @@ public class ClusterStateIT extends ESIntegTestCase {
return TYPE;
}
@Override
public Version getMinimalSupportedVersion() {
return Version.CURRENT;
}
/*
* This custom should always be returned yet we randomize whether it has a required feature that the client is expected to have
* versus not requiring any feature. We use a field to make the random choice exactly once.

View File

@ -116,7 +116,7 @@ public class FeatureAwareTests extends ESTestCase {
if (custom.getRequiredFeature().isPresent()) {
out.setFeatures(Collections.singleton(custom.getRequiredFeature().get()));
}
assertTrue(FeatureAware.shouldSerializeCustom(out, custom));
assertTrue(FeatureAware.shouldSerialize(out, custom));
}
{
final BytesStreamOutput out = new BytesStreamOutput();
@ -126,7 +126,7 @@ public class FeatureAwareTests extends ESTestCase {
if (custom.getRequiredFeature().isPresent() && randomBoolean()) {
out.setFeatures(Collections.singleton(custom.getRequiredFeature().get()));
}
assertFalse(FeatureAware.shouldSerializeCustom(out, custom));
assertFalse(FeatureAware.shouldSerialize(out, custom));
}
}
}
@ -141,7 +141,7 @@ public class FeatureAwareTests extends ESTestCase {
out.setVersion(afterVersion);
assertTrue(custom.getRequiredFeature().isPresent());
out.setFeatures(Collections.singleton(custom.getRequiredFeature().get()));
assertTrue(FeatureAware.shouldSerializeCustom(out, custom));
assertTrue(FeatureAware.shouldSerialize(out, custom));
}
{
// the feature is present and the client is a transport client
@ -149,7 +149,7 @@ public class FeatureAwareTests extends ESTestCase {
out.setVersion(afterVersion);
assertTrue(custom.getRequiredFeature().isPresent());
out.setFeatures(new HashSet<>(Arrays.asList(custom.getRequiredFeature().get(), TransportClient.TRANSPORT_CLIENT_FEATURE)));
assertTrue(FeatureAware.shouldSerializeCustom(out, custom));
assertTrue(FeatureAware.shouldSerialize(out, custom));
}
}
@ -161,14 +161,14 @@ public class FeatureAwareTests extends ESTestCase {
// the feature is missing but we should serialize it anyway because the client is not a transport client
final BytesStreamOutput out = new BytesStreamOutput();
out.setVersion(afterVersion);
assertTrue(FeatureAware.shouldSerializeCustom(out, custom));
assertTrue(FeatureAware.shouldSerialize(out, custom));
}
{
// the feature is missing and we should not serialize it because the client is a transport client
final BytesStreamOutput out = new BytesStreamOutput();
out.setVersion(afterVersion);
out.setFeatures(Collections.singleton(TransportClient.TRANSPORT_CLIENT_FEATURE));
assertFalse(FeatureAware.shouldSerializeCustom(out, custom));
assertFalse(FeatureAware.shouldSerialize(out, custom));
}
}

View File

@ -19,6 +19,7 @@
package org.elasticsearch.cluster;
import org.elasticsearch.Version;
import org.elasticsearch.action.admin.cluster.state.ClusterStateResponse;
import org.elasticsearch.action.admin.indices.template.get.GetIndexTemplatesResponse;
import org.elasticsearch.action.support.IndicesOptions;
@ -37,7 +38,6 @@ import org.elasticsearch.common.io.stream.StreamInput;
import org.elasticsearch.common.io.stream.StreamOutput;
import org.elasticsearch.common.settings.Settings;
import org.elasticsearch.common.unit.ByteSizeValue;
import org.elasticsearch.common.xcontent.ToXContent;
import org.elasticsearch.common.xcontent.XContentBuilder;
import org.elasticsearch.common.xcontent.XContentFactory;
import org.elasticsearch.index.IndexNotFoundException;
@ -304,6 +304,11 @@ public class SimpleClusterStateIT extends ESIntegTestCase {
return "test";
}
@Override
public Version getMinimalSupportedVersion() {
return Version.CURRENT;
}
@Override
public void writeTo(StreamOutput out) throws IOException {
out.writeInt(value);

View File

@ -45,7 +45,6 @@ import org.elasticsearch.snapshots.SnapshotId;
import org.elasticsearch.test.VersionUtils;
import java.io.IOException;
import java.util.ArrayList;
import java.util.Arrays;
import java.util.Collections;
@ -133,7 +132,7 @@ public class ClusterSerializationTests extends ESAllocationTestCase {
// serialize with current version
BytesStreamOutput outStream = new BytesStreamOutput();
Version version = VersionUtils.randomVersionBetween(random(), Version.CURRENT.minimumIndexCompatibilityVersion(), Version.CURRENT);
Version version = VersionUtils.randomVersionBetween(random(), Version.CURRENT.minimumCompatibilityVersion(), Version.CURRENT);
outStream.setVersion(version);
diffs.writeTo(outStream);
StreamInput inStream = outStream.bytes().streamInput();

View File

@ -18,6 +18,7 @@
*/
package org.elasticsearch.cluster.service;
import org.elasticsearch.Version;
import org.elasticsearch.cluster.ClusterState;
import org.elasticsearch.cluster.Diff;
import org.elasticsearch.common.io.stream.StreamOutput;
@ -43,6 +44,11 @@ public class ClusterSerivceTests extends ESTestCase {
return null;
}
@Override
public Version getMinimalSupportedVersion() {
return Version.CURRENT;
}
@Override
public void writeTo(StreamOutput out) throws IOException {

View File

@ -239,6 +239,11 @@ public class ZenDiscoveryIT extends ESIntegTestCase {
return TYPE;
}
@Override
public Version getMinimalSupportedVersion() {
return Version.CURRENT;
}
@Override
public EnumSet<MetaData.XContentContext> context() {
return EnumSet.of(MetaData.XContentContext.GATEWAY, MetaData.XContentContext.SNAPSHOT);

View File

@ -492,6 +492,11 @@ public class GatewayMetaStateTests extends ESAllocationTestCase {
return TYPE;
}
@Override
public Version getMinimalSupportedVersion() {
return Version.CURRENT;
}
@Override
public EnumSet<MetaData.XContentContext> context() {
return EnumSet.of(MetaData.XContentContext.GATEWAY);
@ -510,6 +515,11 @@ public class GatewayMetaStateTests extends ESAllocationTestCase {
return TYPE;
}
@Override
public Version getMinimalSupportedVersion() {
return Version.CURRENT;
}
@Override
public EnumSet<MetaData.XContentContext> context() {
return EnumSet.of(MetaData.XContentContext.GATEWAY);

View File

@ -52,8 +52,15 @@ public class KeywordFieldTypeTests extends FieldTypeTestCase {
public void setupProperties() {
addModifier(new Modifier("normalizer", false) {
@Override
public void modify(MappedFieldType ft) {
((KeywordFieldType) ft).setNormalizer(Lucene.KEYWORD_ANALYZER);
public void modify(MappedFieldType type) {
((KeywordFieldType) type).setNormalizer(Lucene.KEYWORD_ANALYZER);
}
});
addModifier(new Modifier("split_queries_on_whitespace", true) {
@Override
public void modify(MappedFieldType type) {
KeywordFieldType keywordType = (KeywordFieldType) type;
keywordType.setSplitQueriesOnWhitespace(!keywordType.splitQueriesOnWhitespace());
}
});
}

View File

@ -19,6 +19,8 @@
package org.elasticsearch.index.mapper;
import org.apache.lucene.analysis.TokenStream;
import org.apache.lucene.analysis.tokenattributes.CharTermAttribute;
import org.apache.lucene.document.FieldType;
import org.apache.lucene.index.DocValuesType;
import org.apache.lucene.index.IndexOptions;
@ -29,6 +31,8 @@ import org.apache.lucene.index.PostingsEnum;
import org.apache.lucene.index.Term;
import org.apache.lucene.index.TermsEnum;
import org.apache.lucene.search.ConstantScoreQuery;
import org.apache.lucene.search.MultiPhraseQuery;
import org.apache.lucene.search.PhraseQuery;
import org.apache.lucene.search.PrefixQuery;
import org.apache.lucene.search.Query;
import org.apache.lucene.search.TermQuery;
@ -38,6 +42,7 @@ import org.elasticsearch.common.Strings;
import org.elasticsearch.common.bytes.BytesReference;
import org.elasticsearch.common.compress.CompressedXContent;
import org.elasticsearch.common.lucene.uid.Versions;
import org.elasticsearch.common.settings.Settings;
import org.elasticsearch.common.xcontent.ToXContent;
import org.elasticsearch.common.xcontent.XContentBuilder;
import org.elasticsearch.common.xcontent.XContentFactory;
@ -47,7 +52,9 @@ import org.elasticsearch.index.VersionType;
import org.elasticsearch.index.engine.Engine;
import org.elasticsearch.index.mapper.MapperService.MergeReason;
import org.elasticsearch.index.mapper.TextFieldMapper.TextFieldType;
import org.elasticsearch.index.query.MatchPhraseQueryBuilder;
import org.elasticsearch.index.query.QueryShardContext;
import org.elasticsearch.index.search.MatchQuery;
import org.elasticsearch.index.shard.IndexShard;
import org.elasticsearch.plugins.Plugin;
import org.elasticsearch.test.ESSingleNodeTestCase;
@ -65,6 +72,7 @@ import static org.apache.lucene.search.MultiTermQuery.CONSTANT_SCORE_REWRITE;
import static org.hamcrest.Matchers.containsString;
import static org.hamcrest.Matchers.equalTo;
import static org.hamcrest.Matchers.instanceOf;
import static org.hamcrest.core.Is.is;
public class TextFieldMapperTests extends ESSingleNodeTestCase {
@ -73,7 +81,13 @@ public class TextFieldMapperTests extends ESSingleNodeTestCase {
@Before
public void setup() {
indexService = createIndex("test");
Settings settings = Settings.builder()
.put("index.analysis.filter.mySynonyms.type", "synonym")
.putList("index.analysis.filter.mySynonyms.synonyms", Collections.singletonList("car, auto"))
.put("index.analysis.analyzer.synonym.tokenizer", "standard")
.put("index.analysis.analyzer.synonym.filter", "mySynonyms")
.build();
indexService = createIndex("test", settings);
parser = indexService.mapperService().documentMapperParser();
}
@ -670,6 +684,102 @@ public class TextFieldMapperTests extends ESSingleNodeTestCase {
}
}
public void testFastPhraseMapping() throws IOException {
QueryShardContext queryShardContext = indexService.newQueryShardContext(
randomInt(20), null, () -> {
throw new UnsupportedOperationException();
}, null);
String mapping = Strings.toString(XContentFactory.jsonBuilder().startObject().startObject("type")
.startObject("properties")
.startObject("field")
.field("type", "text")
.field("analyzer", "english")
.field("index_phrases", true)
.endObject()
.startObject("synfield")
.field("type", "text")
.field("analyzer", "synonym")
.field("index_phrases", true)
.endObject()
.endObject()
.endObject().endObject());
DocumentMapper mapper = parser.parse("type", new CompressedXContent(mapping));
assertEquals(mapping, mapper.mappingSource().toString());
queryShardContext.getMapperService().merge("type", new CompressedXContent(mapping), MergeReason.MAPPING_UPDATE);
Query q = new MatchPhraseQueryBuilder("field", "two words").toQuery(queryShardContext);
assertThat(q, is(new PhraseQuery("field._index_phrase", "two word")));
Query q2 = new MatchPhraseQueryBuilder("field", "three words here").toQuery(queryShardContext);
assertThat(q2, is(new PhraseQuery("field._index_phrase", "three word", "word here")));
Query q3 = new MatchPhraseQueryBuilder("field", "two words").slop(1).toQuery(queryShardContext);
assertThat(q3, is(new PhraseQuery(1, "field", "two", "word")));
Query q4 = new MatchPhraseQueryBuilder("field", "singleton").toQuery(queryShardContext);
assertThat(q4, is(new TermQuery(new Term("field", "singleton"))));
Query q5 = new MatchPhraseQueryBuilder("field", "sparkle a stopword").toQuery(queryShardContext);
assertThat(q5,
is(new PhraseQuery.Builder().add(new Term("field", "sparkl")).add(new Term("field", "stopword"), 2).build()));
Query q6 = new MatchPhraseQueryBuilder("synfield", "motor car").toQuery(queryShardContext);
assertThat(q6, is(new MultiPhraseQuery.Builder()
.add(new Term[]{
new Term("synfield._index_phrase", "motor car"),
new Term("synfield._index_phrase", "motor auto")})
.build()));
ParsedDocument doc = mapper.parse(SourceToParse.source("test", "type", "1", BytesReference
.bytes(XContentFactory.jsonBuilder()
.startObject()
.field("field", "Some English text that is going to be very useful")
.endObject()),
XContentType.JSON));
IndexableField[] fields = doc.rootDoc().getFields("field._index_phrase");
assertEquals(1, fields.length);
try (TokenStream ts = fields[0].tokenStream(queryShardContext.getMapperService().indexAnalyzer(), null)) {
CharTermAttribute termAtt = ts.addAttribute(CharTermAttribute.class);
ts.reset();
assertTrue(ts.incrementToken());
assertEquals("some english", termAtt.toString());
}
{
String badConfigMapping = Strings.toString(XContentFactory.jsonBuilder().startObject().startObject("type")
.startObject("properties").startObject("field")
.field("type", "text")
.field("index", "false")
.field("index_phrases", true)
.endObject().endObject()
.endObject().endObject());
IllegalArgumentException e = expectThrows(IllegalArgumentException.class,
() -> parser.parse("type", new CompressedXContent(badConfigMapping))
);
assertThat(e.getMessage(), containsString("Cannot set index_phrases on unindexed field [field]"));
}
{
String badConfigMapping = Strings.toString(XContentFactory.jsonBuilder().startObject().startObject("type")
.startObject("properties").startObject("field")
.field("type", "text")
.field("index_options", "freqs")
.field("index_phrases", true)
.endObject().endObject()
.endObject().endObject());
IllegalArgumentException e = expectThrows(IllegalArgumentException.class,
() -> parser.parse("type", new CompressedXContent(badConfigMapping))
);
assertThat(e.getMessage(), containsString("Cannot set index_phrases on field [field] if positions are not enabled"));
}
}
public void testIndexPrefixMapping() throws IOException {
QueryShardContext queryShardContext = indexService.newQueryShardContext(

View File

@ -68,6 +68,13 @@ public class TextFieldTypeTests extends FieldTypeTestCase {
tft.setFielddataMinSegmentSize(1000);
}
});
addModifier(new Modifier("index_phrases", false) {
@Override
public void modify(MappedFieldType ft) {
TextFieldMapper.TextFieldType tft = (TextFieldMapper.TextFieldType) ft;
tft.setIndexPhrases(true);
}
});
addModifier(new Modifier("index_prefixes", false) {
@Override
public void modify(MappedFieldType ft) {

View File

@ -61,7 +61,7 @@ public class MatchPhrasePrefixQueryBuilderTests extends AbstractQueryTestCase<Ma
MatchPhrasePrefixQueryBuilder matchQuery = new MatchPhrasePrefixQueryBuilder(fieldName, value);
if (randomBoolean()) {
if (randomBoolean() && fieldName.equals(STRING_FIELD_NAME)) {
matchQuery.analyzer(randomFrom("simple", "keyword", "whitespace"));
}
@ -118,6 +118,12 @@ public class MatchPhrasePrefixQueryBuilderTests extends AbstractQueryTestCase<Ma
assertThat(e.getMessage(), containsString("analyzer [bogusAnalyzer] not found"));
}
public void testPhraseOnFieldWithNoTerms() {
MatchPhrasePrefixQueryBuilder matchQuery = new MatchPhrasePrefixQueryBuilder(DATE_FIELD_NAME, "three term phrase");
matchQuery.analyzer("whitespace");
expectThrows(IllegalArgumentException.class, () -> matchQuery.doToQuery(createShardContext()));
}
public void testPhrasePrefixMatchQuery() throws IOException {
String json1 = "{\n" +
" \"match_phrase_prefix\" : {\n" +

View File

@ -64,7 +64,7 @@ public class MatchPhraseQueryBuilderTests extends AbstractQueryTestCase<MatchPhr
MatchPhraseQueryBuilder matchQuery = new MatchPhraseQueryBuilder(fieldName, value);
if (randomBoolean()) {
if (randomBoolean() && fieldName.equals(STRING_FIELD_NAME)) {
matchQuery.analyzer(randomFrom("simple", "keyword", "whitespace"));
}

View File

@ -19,6 +19,7 @@
package org.elasticsearch.index.query;
import com.carrotsearch.randomizedtesting.annotations.Seed;
import org.apache.lucene.index.Term;
import org.apache.lucene.queries.ExtendedCommonTermsQuery;
import org.apache.lucene.search.BooleanQuery;

View File

@ -19,6 +19,8 @@
package org.elasticsearch.persistent;
import org.elasticsearch.ResourceNotFoundException;
import org.elasticsearch.Version;
import org.elasticsearch.client.transport.TransportClient;
import org.elasticsearch.cluster.Diff;
import org.elasticsearch.cluster.NamedDiff;
import org.elasticsearch.cluster.metadata.MetaData;
@ -26,8 +28,11 @@ import org.elasticsearch.cluster.metadata.MetaData.Custom;
import org.elasticsearch.common.ParseField;
import org.elasticsearch.common.UUIDs;
import org.elasticsearch.common.bytes.BytesReference;
import org.elasticsearch.common.io.stream.BytesStreamOutput;
import org.elasticsearch.common.io.stream.NamedWriteableAwareStreamInput;
import org.elasticsearch.common.io.stream.NamedWriteableRegistry;
import org.elasticsearch.common.io.stream.NamedWriteableRegistry.Entry;
import org.elasticsearch.common.io.stream.StreamInput;
import org.elasticsearch.common.io.stream.Writeable;
import org.elasticsearch.common.xcontent.NamedXContentRegistry;
import org.elasticsearch.common.xcontent.ToXContent;
@ -43,13 +48,24 @@ import org.elasticsearch.persistent.TestPersistentTasksPlugin.TestPersistentTask
import org.elasticsearch.tasks.Task;
import org.elasticsearch.test.AbstractDiffableSerializationTestCase;
import java.io.IOException;
import java.util.ArrayList;
import java.util.Arrays;
import java.util.Collections;
import java.util.HashSet;
import java.util.NoSuchElementException;
import java.util.Optional;
import java.util.Set;
import static org.elasticsearch.cluster.metadata.MetaData.CONTEXT_MODE_GATEWAY;
import static org.elasticsearch.cluster.metadata.MetaData.CONTEXT_MODE_SNAPSHOT;
import static org.elasticsearch.persistent.PersistentTasksExecutor.NO_NODE_FOUND;
import static org.elasticsearch.test.VersionUtils.allReleasedVersions;
import static org.elasticsearch.test.VersionUtils.compatibleFutureVersion;
import static org.elasticsearch.test.VersionUtils.getFirstVersion;
import static org.elasticsearch.test.VersionUtils.getPreviousVersion;
import static org.elasticsearch.test.VersionUtils.randomVersionBetween;
import static org.hamcrest.Matchers.equalTo;
public class PersistentTasksCustomMetaDataTests extends AbstractDiffableSerializationTestCase<Custom> {
@ -228,7 +244,65 @@ public class PersistentTasksCustomMetaDataTests extends AbstractDiffableSerializ
assertEquals(changed, builder.isChanged());
persistentTasks = builder.build();
}
}
public void testMinVersionSerialization() throws IOException {
PersistentTasksCustomMetaData.Builder tasks = PersistentTasksCustomMetaData.builder();
Version minVersion = allReleasedVersions().stream().filter(Version::isRelease).findFirst().orElseThrow(NoSuchElementException::new);
final Version streamVersion = randomVersionBetween(random(), minVersion, getPreviousVersion(Version.CURRENT));
tasks.addTask("test_compatible_version", TestPersistentTasksExecutor.NAME,
new TestParams(null, randomVersionBetween(random(), minVersion, streamVersion),
randomBoolean() ? Optional.empty() : Optional.of("test")),
randomAssignment());
tasks.addTask("test_incompatible_version", TestPersistentTasksExecutor.NAME,
new TestParams(null, randomVersionBetween(random(), compatibleFutureVersion(streamVersion), Version.CURRENT),
randomBoolean() ? Optional.empty() : Optional.of("test")),
randomAssignment());
final BytesStreamOutput out = new BytesStreamOutput();
out.setVersion(streamVersion);
Set<String> features = new HashSet<>();
final boolean transportClient = randomBoolean();
if (transportClient) {
features.add(TransportClient.TRANSPORT_CLIENT_FEATURE);
}
// if a transport client, then it must have the feature otherwise we add the feature randomly
if (transportClient || randomBoolean()) {
features.add("test");
}
out.setFeatures(features);
tasks.build().writeTo(out);
final StreamInput input = out.bytes().streamInput();
input.setVersion(streamVersion);
PersistentTasksCustomMetaData read =
new PersistentTasksCustomMetaData(new NamedWriteableAwareStreamInput(input, getNamedWriteableRegistry()));
assertThat(read.taskMap().keySet(), equalTo(Collections.singleton("test_compatible_version")));
}
public void testFeatureSerialization() throws IOException {
PersistentTasksCustomMetaData.Builder tasks = PersistentTasksCustomMetaData.builder();
Version minVersion = getFirstVersion();
tasks.addTask("test_compatible", TestPersistentTasksExecutor.NAME,
new TestParams(null, randomVersionBetween(random(), minVersion, Version.CURRENT),
randomBoolean() ? Optional.empty() : Optional.of("existing")),
randomAssignment());
tasks.addTask("test_incompatible", TestPersistentTasksExecutor.NAME,
new TestParams(null, randomVersionBetween(random(), minVersion, Version.CURRENT), Optional.of("non_existing")),
randomAssignment());
final BytesStreamOutput out = new BytesStreamOutput();
out.setVersion(Version.CURRENT);
Set<String> features = new HashSet<>();
features.add("existing");
features.add(TransportClient.TRANSPORT_CLIENT_FEATURE);
out.setFeatures(features);
tasks.build().writeTo(out);
PersistentTasksCustomMetaData read = new PersistentTasksCustomMetaData(
new NamedWriteableAwareStreamInput(out.bytes().streamInput(), getNamedWriteableRegistry()));
assertThat(read.taskMap().keySet(), equalTo(Collections.singleton("test_compatible")));
}
private Assignment randomAssignment() {

View File

@ -20,12 +20,12 @@ package org.elasticsearch.persistent;
import org.elasticsearch.action.support.PlainActionFuture;
import org.elasticsearch.common.UUIDs;
import org.elasticsearch.persistent.PersistentTasksCustomMetaData.PersistentTask;
import org.elasticsearch.persistent.TestPersistentTasksPlugin.TestParams;
import org.elasticsearch.persistent.TestPersistentTasksPlugin.TestPersistentTasksExecutor;
import org.elasticsearch.plugins.Plugin;
import org.elasticsearch.test.ESIntegTestCase;
import org.elasticsearch.test.junit.annotations.TestLogging;
import org.elasticsearch.persistent.PersistentTasksCustomMetaData.PersistentTask;
import org.elasticsearch.persistent.TestPersistentTasksPlugin.TestPersistentTasksExecutor;
import org.elasticsearch.persistent.TestPersistentTasksPlugin.TestParams;
import java.util.ArrayList;
import java.util.Collection;
@ -35,8 +35,6 @@ import java.util.List;
import static org.hamcrest.Matchers.empty;
import static org.hamcrest.Matchers.equalTo;
import static org.hamcrest.Matchers.greaterThan;
import static org.hamcrest.Matchers.notNullValue;
import static org.hamcrest.Matchers.nullValue;
@ESIntegTestCase.ClusterScope(scope = ESIntegTestCase.Scope.TEST, minNumDataNodes = 1)
public class PersistentTasksExecutorFullRestartIT extends ESIntegTestCase {
@ -65,7 +63,7 @@ public class PersistentTasksExecutorFullRestartIT extends ESIntegTestCase {
PlainActionFuture<PersistentTask<TestParams>> future = new PlainActionFuture<>();
futures.add(future);
taskIds[i] = UUIDs.base64UUID();
service.sendStartRequest(taskIds[i], TestPersistentTasksExecutor.NAME, randomBoolean() ? null : new TestParams("Blah"), future);
service.sendStartRequest(taskIds[i], TestPersistentTasksExecutor.NAME, new TestParams("Blah"), future);
}
for (int i = 0; i < numberOfTasks; i++) {

View File

@ -22,8 +22,8 @@ import org.elasticsearch.common.UUIDs;
import org.elasticsearch.common.io.stream.NamedWriteableRegistry;
import org.elasticsearch.common.io.stream.NamedWriteableRegistry.Entry;
import org.elasticsearch.persistent.StartPersistentTaskAction.Request;
import org.elasticsearch.persistent.TestPersistentTasksPlugin.TestPersistentTasksExecutor;
import org.elasticsearch.persistent.TestPersistentTasksPlugin.TestParams;
import org.elasticsearch.persistent.TestPersistentTasksPlugin.TestPersistentTasksExecutor;
import org.elasticsearch.test.AbstractStreamableTestCase;
import java.util.Collections;
@ -32,17 +32,12 @@ public class StartPersistentActionRequestTests extends AbstractStreamableTestCas
@Override
protected Request createTestInstance() {
TestParams testParams;
TestParams testParams = new TestParams();
if (randomBoolean()) {
testParams = new TestParams();
if (randomBoolean()) {
testParams.setTestParam(randomAlphaOfLengthBetween(1, 20));
}
if (randomBoolean()) {
testParams.setExecutorNodeAttr(randomAlphaOfLengthBetween(1, 20));
}
} else {
testParams = null;
testParams.setTestParam(randomAlphaOfLengthBetween(1, 20));
}
if (randomBoolean()) {
testParams.setExecutorNodeAttr(randomAlphaOfLengthBetween(1, 20));
}
return new Request(UUIDs.base64UUID(), randomAlphaOfLengthBetween(1, 20), testParams);
}

View File

@ -19,6 +19,7 @@
package org.elasticsearch.persistent;
import org.elasticsearch.Version;
import org.elasticsearch.action.Action;
import org.elasticsearch.action.ActionListener;
import org.elasticsearch.action.ActionRequest;
@ -49,6 +50,8 @@ import org.elasticsearch.common.xcontent.ConstructingObjectParser;
import org.elasticsearch.common.xcontent.NamedXContentRegistry;
import org.elasticsearch.common.xcontent.XContentBuilder;
import org.elasticsearch.common.xcontent.XContentParser;
import org.elasticsearch.persistent.PersistentTasksCustomMetaData.Assignment;
import org.elasticsearch.persistent.PersistentTasksCustomMetaData.PersistentTask;
import org.elasticsearch.plugins.ActionPlugin;
import org.elasticsearch.plugins.PersistentTaskPlugin;
import org.elasticsearch.plugins.Plugin;
@ -57,8 +60,6 @@ import org.elasticsearch.tasks.TaskCancelledException;
import org.elasticsearch.tasks.TaskId;
import org.elasticsearch.threadpool.ThreadPool;
import org.elasticsearch.transport.TransportService;
import org.elasticsearch.persistent.PersistentTasksCustomMetaData.Assignment;
import org.elasticsearch.persistent.PersistentTasksCustomMetaData.PersistentTask;
import java.io.IOException;
import java.util.ArrayList;
@ -67,6 +68,7 @@ import java.util.Collections;
import java.util.List;
import java.util.Map;
import java.util.Objects;
import java.util.Optional;
import java.util.concurrent.CountDownLatch;
import java.util.concurrent.TimeUnit;
import java.util.concurrent.atomic.AtomicInteger;
@ -120,6 +122,9 @@ public class TestPersistentTasksPlugin extends Plugin implements ActionPlugin, P
REQUEST_PARSER.declareString(constructorArg(), new ParseField("param"));
}
private final Version minVersion;
private final Optional<String> feature;
private String executorNodeAttr = null;
private String responseNode = null;
@ -127,17 +132,25 @@ public class TestPersistentTasksPlugin extends Plugin implements ActionPlugin, P
private String testParam = null;
public TestParams() {
this((String)null);
}
public TestParams(String testParam) {
this(testParam, Version.CURRENT, Optional.empty());
}
public TestParams(String testParam, Version minVersion, Optional<String> feature) {
this.testParam = testParam;
this.minVersion = minVersion;
this.feature = feature;
}
public TestParams(StreamInput in) throws IOException {
executorNodeAttr = in.readOptionalString();
responseNode = in.readOptionalString();
testParam = in.readOptionalString();
minVersion = Version.readVersion(in);
feature = Optional.ofNullable(in.readOptionalString());
}
@Override
@ -166,6 +179,8 @@ public class TestPersistentTasksPlugin extends Plugin implements ActionPlugin, P
out.writeOptionalString(executorNodeAttr);
out.writeOptionalString(responseNode);
out.writeOptionalString(testParam);
Version.writeVersion(minVersion, out);
out.writeOptionalString(feature.orElse(null));
}
@Override
@ -194,6 +209,16 @@ public class TestPersistentTasksPlugin extends Plugin implements ActionPlugin, P
public int hashCode() {
return Objects.hash(executorNodeAttr, responseNode, testParam);
}
@Override
public Version getMinimalSupportedVersion() {
return minVersion;
}
@Override
public Optional<String> getRequiredFeature() {
return feature;
}
}
public static class Status implements Task.Status {

View File

@ -71,7 +71,7 @@ public class EnableAssignmentDeciderIT extends ESIntegTestCase {
final CountDownLatch latch = new CountDownLatch(numberOfTasks);
for (int i = 0; i < numberOfTasks; i++) {
PersistentTasksService service = internalCluster().getInstance(PersistentTasksService.class);
service.sendStartRequest("task_" + i, TestPersistentTasksExecutor.NAME, randomTaskParams(),
service.sendStartRequest("task_" + i, TestPersistentTasksExecutor.NAME, new TestParams(randomAlphaOfLength(10)),
new ActionListener<PersistentTask<PersistentTaskParams>>() {
@Override
public void onResponse(PersistentTask<PersistentTaskParams> task) {
@ -163,11 +163,4 @@ public class EnableAssignmentDeciderIT extends ESIntegTestCase {
assertAcked(client().admin().cluster().prepareUpdateSettings().setPersistentSettings(settings));
}
/** Returns a random task parameter **/
private static PersistentTaskParams randomTaskParams() {
if (randomBoolean()) {
return null;
}
return new TestParams(randomAlphaOfLength(10));
}
}

View File

@ -19,11 +19,7 @@
package org.elasticsearch.search.fetch.subphase.highlight;
import org.elasticsearch.common.text.Text;
import org.elasticsearch.index.mapper.FieldMapper;
import org.elasticsearch.search.fetch.subphase.highlight.HighlightField;
import org.elasticsearch.search.fetch.subphase.highlight.Highlighter;
import org.elasticsearch.search.fetch.subphase.highlight.HighlighterContext;
import org.elasticsearch.search.fetch.subphase.highlight.SearchContextHighlight;
import org.elasticsearch.index.mapper.MappedFieldType;
import java.util.ArrayList;
import java.util.List;
@ -68,7 +64,7 @@ public class CustomHighlighter implements Highlighter {
}
@Override
public boolean canHighlight(FieldMapper fieldMapper) {
public boolean canHighlight(MappedFieldType fieldType) {
return true;
}

View File

@ -21,9 +21,9 @@ package org.elasticsearch.snapshots;
import com.carrotsearch.hppc.IntHashSet;
import com.carrotsearch.hppc.IntSet;
import org.elasticsearch.Version;
import org.elasticsearch.action.ActionFuture;
import org.elasticsearch.action.admin.cluster.repositories.put.PutRepositoryResponse;
import org.elasticsearch.action.admin.cluster.repositories.verify.VerifyRepositoryResponse;
import org.elasticsearch.action.admin.cluster.snapshots.create.CreateSnapshotResponse;
import org.elasticsearch.action.admin.cluster.snapshots.delete.DeleteSnapshotResponse;
import org.elasticsearch.action.admin.cluster.snapshots.get.GetSnapshotsResponse;
@ -1162,6 +1162,11 @@ public class DedicatedClusterSnapshotRestoreIT extends AbstractSnapshotIntegTest
return TYPE;
}
@Override
public Version getMinimalSupportedVersion() {
return Version.CURRENT;
}
public static SnapshottableMetadata readFrom(StreamInput in) throws IOException {
return readFrom(SnapshottableMetadata::new, in);
}
@ -1193,6 +1198,11 @@ public class DedicatedClusterSnapshotRestoreIT extends AbstractSnapshotIntegTest
return TYPE;
}
@Override
public Version getMinimalSupportedVersion() {
return Version.CURRENT;
}
public static NonSnapshottableMetadata readFrom(StreamInput in) throws IOException {
return readFrom(NonSnapshottableMetadata::new, in);
}
@ -1223,6 +1233,11 @@ public class DedicatedClusterSnapshotRestoreIT extends AbstractSnapshotIntegTest
return TYPE;
}
@Override
public Version getMinimalSupportedVersion() {
return Version.CURRENT;
}
public static SnapshottableGatewayMetadata readFrom(StreamInput in) throws IOException {
return readFrom(SnapshottableGatewayMetadata::new, in);
}
@ -1253,6 +1268,11 @@ public class DedicatedClusterSnapshotRestoreIT extends AbstractSnapshotIntegTest
return TYPE;
}
@Override
public Version getMinimalSupportedVersion() {
return Version.CURRENT;
}
public static NonSnapshottableGatewayMetadata readFrom(StreamInput in) throws IOException {
return readFrom(NonSnapshottableGatewayMetadata::new, in);
}
@ -1284,6 +1304,11 @@ public class DedicatedClusterSnapshotRestoreIT extends AbstractSnapshotIntegTest
return TYPE;
}
@Override
public Version getMinimalSupportedVersion() {
return Version.CURRENT;
}
public static SnapshotableGatewayNoApiMetadata readFrom(StreamInput in) throws IOException {
return readFrom(SnapshotableGatewayNoApiMetadata::new, in);
}

View File

@ -19,22 +19,19 @@
package org.elasticsearch.test;
import java.util.ArrayList;
import java.util.Collections;
import java.util.List;
import java.util.Locale;
import java.util.Map;
import java.util.Optional;
import java.util.Random;
import java.util.SortedSet;
import java.util.TreeSet;
import java.util.stream.Collectors;
import java.util.stream.Stream;
import org.elasticsearch.Version;
import org.elasticsearch.common.Nullable;
import org.elasticsearch.common.collect.Tuple;
import java.util.ArrayList;
import java.util.Collections;
import java.util.List;
import java.util.Map;
import java.util.Optional;
import java.util.Random;
import java.util.stream.Collectors;
import java.util.stream.Stream;
/** Utilities for selecting versions in tests */
public class VersionUtils {
@ -228,6 +225,13 @@ public class VersionUtils {
return opt.get();
}
/** returns the first future compatible version */
public static Version compatibleFutureVersion(Version version) {
final Optional<Version> opt = ALL_VERSIONS.stream().filter(version::before).filter(v -> v.isCompatible(version)).findAny();
assert opt.isPresent() : "no future compatible version for " + version;
return opt.get();
}
/** Returns the maximum {@link Version} that is compatible with the given version. */
public static Version maxCompatibleVersion(Version version) {
final List<Version> compatible = ALL_VERSIONS.stream().filter(version::isCompatible).filter(version::onOrBefore)

View File

@ -69,6 +69,11 @@ public abstract class ESClientYamlSuiteTestCase extends ESRestTestCase {
* e.g. "-Dtests.rest.blacklist=get/10_basic/*"
*/
public static final String REST_TESTS_BLACKLIST = "tests.rest.blacklist";
/**
* We use tests.rest.blacklist in build files to blacklist tests; this property enables a user to add additional blacklisted tests on
* top of the tests blacklisted in the build.
*/
public static final String REST_TESTS_BLACKLIST_ADDITIONS = "tests.rest.blacklist_additions";
/**
* Property that allows to control whether spec validation is enabled or not (default true).
*/
@ -125,6 +130,10 @@ public abstract class ESClientYamlSuiteTestCase extends ESRestTestCase {
for (final String entry : blacklist) {
blacklistPathMatchers.add(new BlacklistedPathPatternMatcher(entry));
}
final String[] blacklistAdditions = resolvePathsProperty(REST_TESTS_BLACKLIST_ADDITIONS, null);
for (final String entry : blacklistAdditions) {
blacklistPathMatchers.add(new BlacklistedPathPatternMatcher(entry));
}
}
assert restTestExecutionContext != null;
assert adminExecutionContext != null;

View File

@ -1,29 +0,0 @@
[float]
[[ml-analyzing]]
=== Analyzing the Past and Present
The {xpackml} features automate the analysis of time-series data by creating
accurate baselines of normal behavior in the data and identifying anomalous
patterns in that data. You can submit your data for analysis in batches or
continuously in real-time {dfeeds}.
Using proprietary {ml} algorithms, the following circumstances are detected,
scored, and linked with statistically significant influencers in the data:
* Anomalies related to temporal deviations in values, counts, or frequencies
* Statistical rarity
* Unusual behaviors for a member of a population
Automated periodicity detection and quick adaptation to changing data ensure
that you dont need to specify algorithms, models, or other data science-related
configurations in order to get the benefits of {ml}.
You can view the {ml} results in {kib} where, for example, charts illustrate the
actual data values, the bounds for the expected values, and the anomalies that
occur outside these bounds.
[role="screenshot"]
image::images/ml-gs-job-analysis.jpg["Example screenshot from the Machine Learning Single Metric Viewer in Kibana"]
For a more detailed walk-through of {xpackml} features, see
<<ml-getting-started>>.

View File

@ -1,10 +0,0 @@
[float]
[[ml-nodes]]
=== Machine learning nodes
A {ml} node is a node that has `xpack.ml.enabled` and `node.ml` set to `true`,
which is the default behavior. If you set `node.ml` to `false`, the node can
service API requests but it cannot run jobs. If you want to use {xpackml}
features, there must be at least one {ml} node in your cluster. For more
information about this setting, see
{ref}/ml-settings.html[{ml} settings in {es}].

View File

@ -1,26 +0,0 @@
[[ml-buckets]]
=== Buckets
++++
<titleabbrev>Buckets</titleabbrev>
++++
The {xpackml} features use the concept of a _bucket_ to divide the time series
into batches for processing.
The _bucket span_ is part of the configuration information for a job. It defines
the time interval that is used to summarize and model the data. This is
typically between 5 minutes to 1 hour and it depends on your data characteristics.
When you set the bucket span, take into account the granularity at which you
want to analyze, the frequency of the input data, the typical duration of the
anomalies, and the frequency at which alerting is required.
When you view your {ml} results, each bucket has an anomaly score. This score is
a statistically aggregated and normalized view of the combined anomalousness of
all the record results in the bucket. If you have more than one job, you can
also obtain overall bucket results, which combine and correlate anomalies from
multiple jobs into an overall score. When you view the results for jobs groups
in {kib}, it provides the overall bucket scores.
For more information, see
{ref}/ml-results-resource.html[Results Resources] and
{ref}/ml-get-overall-buckets.html[Get Overall Buckets API].

View File

@ -1,40 +0,0 @@
[[ml-calendars]]
=== Calendars and Scheduled Events
Sometimes there are periods when you expect unusual activity to take place,
such as bank holidays, "Black Friday", or planned system outages. If you
identify these events in advance, no anomalies are generated during that period.
The {ml} model is not ill-affected and you do not receive spurious results.
You can create calendars and scheduled events in the **Settings** pane on the
**Machine Learning** page in {kib} or by using {ref}/ml-apis.html[{ml} APIs].
A scheduled event must have a start time, end time, and description. In general,
scheduled events are short in duration (typically lasting from a few hours to a
day) and occur infrequently. If you have regularly occurring events, such as
weekly maintenance periods, you do not need to create scheduled events for these
circumstances; they are already handled by the {ml} analytics.
You can identify zero or more scheduled events in a calendar. Jobs can then
subscribe to calendars and the {ml} analytics handle all subsequent scheduled
events appropriately.
If you want to add multiple scheduled events at once, you can import an
iCalendar (`.ics`) file in {kib} or a JSON file in the
{ref}/ml-post-calendar-event.html[add events to calendar API].
[NOTE]
--
* You must identify scheduled events before your job analyzes the data for that
time period. Machine learning results are not updated retroactively.
* If your iCalendar file contains recurring events, only the first occurrence is
imported.
* Bucket results are generated during scheduled events but they have an
anomaly score of zero. For more information about bucket results, see
{ref}/ml-results-resource.html[Results Resources].
* If you use long or frequent scheduled events, it might take longer for the
{ml} analytics to learn to model your data and some anomalous behavior might be
missed.
--

View File

@ -1,40 +0,0 @@
[[ml-dfeeds]]
=== {dfeeds-cap}
Machine learning jobs can analyze data that is stored in {es} or data that is
sent from some other source via an API. _{dfeeds-cap}_ retrieve data from {es}
for analysis, which is the simpler and more common scenario.
If you create jobs in {kib}, you must use {dfeeds}. When you create a job, you
select an index pattern and {kib} configures the {dfeed} for you under the
covers. If you use {ml} APIs instead, you can create a {dfeed} by using the
{ref}/ml-put-datafeed.html[create {dfeeds} API] after you create a job. You can
associate only one {dfeed} with each job.
For a description of all the {dfeed} properties, see
{ref}/ml-datafeed-resource.html[Datafeed Resources].
To start retrieving data from {es}, you must start the {dfeed}. When you start
it, you can optionally specify start and end times. If you do not specify an
end time, the {dfeed} runs continuously. You can start and stop {dfeeds} in
{kib} or use the {ref}/ml-start-datafeed.html[start {dfeeds}] and
{ref}/ml-stop-datafeed.html[stop {dfeeds}] APIs. A {dfeed} can be started and
stopped multiple times throughout its lifecycle.
[IMPORTANT]
--
When {security} is enabled, a {dfeed} stores the roles of the user who created
or updated the {dfeed} at that time. This means that if those roles are updated,
the {dfeed} subsequently runs with the new permissions that are associated with
the roles. However, if the users roles are adjusted after creating or updating
the {dfeed}, the {dfeed} continues to run with the permissions that were
associated with the original roles.
One way to update the roles that are stored within the {dfeed} without changing
any other settings is to submit an empty JSON document ({}) to the
{ref}/ml-update-datafeed.html[update {dfeed} API].
--
If the data that you want to analyze is not stored in {es}, you cannot use
{dfeeds}. You can however send batches of data directly to the job by using the
{ref}/ml-post-data.html[post data to jobs API].

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