listener, Header... headers) {
restHighLevelClient.performRequestAsyncAndParseEntity( request, RequestConverters::putPipeline,
- PutPipelineResponse::fromXContent, listener, emptySet(), headers);
+ WritePipelineResponse::fromXContent, listener, emptySet(), headers);
}
/**
@@ -111,4 +112,28 @@ public final class ClusterClient {
restHighLevelClient.performRequestAsyncAndParseEntity( request, RequestConverters::getPipeline,
GetPipelineResponse::fromXContent, listener, emptySet(), headers);
}
+
+ /**
+ * Delete an existing pipeline
+ *
+ * See
+ *
+ * Delete Pipeline API on elastic.co
+ */
+ public WritePipelineResponse deletePipeline(DeletePipelineRequest request, Header... headers) throws IOException {
+ return restHighLevelClient.performRequestAndParseEntity( request, RequestConverters::deletePipeline,
+ WritePipelineResponse::fromXContent, emptySet(), headers);
+ }
+
+ /**
+ * Asynchronously delete an existing pipeline
+ *
+ * See
+ *
+ * Delete Pipeline API on elastic.co
+ */
+ public void deletePipelineAsync(DeletePipelineRequest request, ActionListener listener, Header... headers) {
+ restHighLevelClient.performRequestAsyncAndParseEntity( request, RequestConverters::deletePipeline,
+ WritePipelineResponse::fromXContent, listener, emptySet(), headers);
+ }
}
diff --git a/client/rest-high-level/src/main/java/org/elasticsearch/client/RequestConverters.java b/client/rest-high-level/src/main/java/org/elasticsearch/client/RequestConverters.java
index e137d3d2f5c..6c8bb845259 100644
--- a/client/rest-high-level/src/main/java/org/elasticsearch/client/RequestConverters.java
+++ b/client/rest-high-level/src/main/java/org/elasticsearch/client/RequestConverters.java
@@ -60,6 +60,7 @@ import org.elasticsearch.action.fieldcaps.FieldCapabilitiesRequest;
import org.elasticsearch.action.get.GetRequest;
import org.elasticsearch.action.get.MultiGetRequest;
import org.elasticsearch.action.index.IndexRequest;
+import org.elasticsearch.action.ingest.DeletePipelineRequest;
import org.elasticsearch.action.ingest.PutPipelineRequest;
import org.elasticsearch.action.ingest.GetPipelineRequest;
import org.elasticsearch.action.search.ClearScrollRequest;
@@ -648,6 +649,20 @@ final class RequestConverters {
return request;
}
+ static Request deletePipeline(DeletePipelineRequest deletePipelineRequest) {
+ String endpoint = new EndpointBuilder()
+ .addPathPartAsIs("_ingest/pipeline")
+ .addPathPart(deletePipelineRequest.getId())
+ .build();
+ Request request = new Request(HttpDelete.METHOD_NAME, endpoint);
+
+ Params parameters = new Params(request);
+ parameters.withTimeout(deletePipelineRequest.timeout());
+ parameters.withMasterTimeout(deletePipelineRequest.masterNodeTimeout());
+
+ return request;
+ }
+
static Request listTasks(ListTasksRequest listTaskRequest) {
if (listTaskRequest.getTaskId() != null && listTaskRequest.getTaskId().isSet()) {
throw new IllegalArgumentException("TaskId cannot be used for list tasks request");
diff --git a/client/rest-high-level/src/test/java/org/elasticsearch/client/ClusterClientIT.java b/client/rest-high-level/src/test/java/org/elasticsearch/client/ClusterClientIT.java
index caab4c282f4..42db51e81b7 100644
--- a/client/rest-high-level/src/test/java/org/elasticsearch/client/ClusterClientIT.java
+++ b/client/rest-high-level/src/test/java/org/elasticsearch/client/ClusterClientIT.java
@@ -25,7 +25,8 @@ import org.elasticsearch.action.admin.cluster.settings.ClusterUpdateSettingsResp
import org.elasticsearch.action.ingest.GetPipelineRequest;
import org.elasticsearch.action.ingest.GetPipelineResponse;
import org.elasticsearch.action.ingest.PutPipelineRequest;
-import org.elasticsearch.action.ingest.PutPipelineResponse;
+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;
@@ -121,7 +122,7 @@ public class ClusterClientIT extends ESRestHighLevelClientTestCase {
BytesReference.bytes(pipelineBuilder),
pipelineBuilder.contentType());
- PutPipelineResponse putPipelineResponse =
+ WritePipelineResponse putPipelineResponse =
execute(request, highLevelClient().cluster()::putPipeline, highLevelClient().cluster()::putPipelineAsync);
assertTrue(putPipelineResponse.isAcknowledged());
}
@@ -148,4 +149,17 @@ public class ClusterClientIT extends ESRestHighLevelClientTestCase {
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());
+ }
}
diff --git a/client/rest-high-level/src/test/java/org/elasticsearch/client/RequestConvertersTests.java b/client/rest-high-level/src/test/java/org/elasticsearch/client/RequestConvertersTests.java
index bf69aa76636..f61d79b8d42 100644
--- a/client/rest-high-level/src/test/java/org/elasticsearch/client/RequestConvertersTests.java
+++ b/client/rest-high-level/src/test/java/org/elasticsearch/client/RequestConvertersTests.java
@@ -63,6 +63,7 @@ import org.elasticsearch.action.fieldcaps.FieldCapabilitiesRequest;
import org.elasticsearch.action.get.GetRequest;
import org.elasticsearch.action.get.MultiGetRequest;
import org.elasticsearch.action.index.IndexRequest;
+import org.elasticsearch.action.ingest.DeletePipelineRequest;
import org.elasticsearch.action.ingest.GetPipelineRequest;
import org.elasticsearch.action.ingest.PutPipelineRequest;
import org.elasticsearch.action.search.ClearScrollRequest;
@@ -1465,6 +1466,21 @@ public class RequestConvertersTests extends ESTestCase {
assertEquals(expectedParams, expectedRequest.getParameters());
}
+ public void testDeletePipeline() {
+ String pipelineId = "some_pipeline_id";
+ Map expectedParams = new HashMap<>();
+ DeletePipelineRequest request = new DeletePipelineRequest(pipelineId);
+ setRandomMasterTimeout(request, expectedParams);
+ setRandomTimeout(request::timeout, AcknowledgedRequest.DEFAULT_ACK_TIMEOUT, expectedParams);
+ Request expectedRequest = RequestConverters.deletePipeline(request);
+ StringJoiner endpoint = new StringJoiner("/", "/", "");
+ endpoint.add("_ingest/pipeline");
+ endpoint.add(pipelineId);
+ assertEquals(endpoint.toString(), expectedRequest.getEndpoint());
+ assertEquals(HttpDelete.METHOD_NAME, expectedRequest.getMethod());
+ assertEquals(expectedParams, expectedRequest.getParameters());
+ }
+
public void testRollover() throws IOException {
RolloverRequest rolloverRequest = new RolloverRequest(randomAlphaOfLengthBetween(3, 10),
randomBoolean() ? null : randomAlphaOfLengthBetween(3, 10));
diff --git a/client/rest-high-level/src/test/java/org/elasticsearch/client/documentation/ClusterClientDocumentationIT.java b/client/rest-high-level/src/test/java/org/elasticsearch/client/documentation/ClusterClientDocumentationIT.java
index 07785ecc03d..0da577f17e8 100644
--- a/client/rest-high-level/src/test/java/org/elasticsearch/client/documentation/ClusterClientDocumentationIT.java
+++ b/client/rest-high-level/src/test/java/org/elasticsearch/client/documentation/ClusterClientDocumentationIT.java
@@ -26,7 +26,8 @@ import org.elasticsearch.action.admin.cluster.settings.ClusterUpdateSettingsResp
import org.elasticsearch.action.ingest.GetPipelineRequest;
import org.elasticsearch.action.ingest.GetPipelineResponse;
import org.elasticsearch.action.ingest.PutPipelineRequest;
-import org.elasticsearch.action.ingest.PutPipelineResponse;
+import org.elasticsearch.action.ingest.DeletePipelineRequest;
+import org.elasticsearch.action.ingest.WritePipelineResponse;
import org.elasticsearch.client.ESRestHighLevelClientTestCase;
import org.elasticsearch.client.RestHighLevelClient;
import org.elasticsearch.cluster.routing.allocation.decider.EnableAllocationDecider;
@@ -212,7 +213,7 @@ public class ClusterClientDocumentationIT extends ESRestHighLevelClientTestCase
// end::put-pipeline-request-masterTimeout
// tag::put-pipeline-execute
- PutPipelineResponse response = client.cluster().putPipeline(request); // <1>
+ WritePipelineResponse response = client.cluster().putPipeline(request); // <1>
// end::put-pipeline-execute
// tag::put-pipeline-response
@@ -236,10 +237,10 @@ public class ClusterClientDocumentationIT extends ESRestHighLevelClientTestCase
);
// tag::put-pipeline-execute-listener
- ActionListener listener =
- new ActionListener() {
+ ActionListener listener =
+ new ActionListener() {
@Override
- public void onResponse(PutPipelineResponse response) {
+ public void onResponse(WritePipelineResponse response) {
// <1>
}
@@ -331,4 +332,74 @@ public class ClusterClientDocumentationIT extends ESRestHighLevelClientTestCase
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 listener =
+ new ActionListener() {
+ @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));
+ }
+ }
}
diff --git a/docs/java-rest/high-level/cluster/delete_pipeline.asciidoc b/docs/java-rest/high-level/cluster/delete_pipeline.asciidoc
new file mode 100644
index 00000000000..f809f831f78
--- /dev/null
+++ b/docs/java-rest/high-level/cluster/delete_pipeline.asciidoc
@@ -0,0 +1,80 @@
+[[java-rest-high-cluster-delete-pipeline]]
+=== Delete Pipeline API
+
+[[java-rest-high-cluster-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]
+--------------------------------------------------
+<1> The pipeline id to delete
+
+==== Optional arguments
+The following arguments can optionally be provided:
+
+["source","java",subs="attributes,callouts,macros"]
+--------------------------------------------------
+include-tagged::{doc-tests}/ClusterClientDocumentationIT.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]
+--------------------------------------------------
+<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]]
+==== Synchronous Execution
+
+["source","java",subs="attributes,callouts,macros"]
+--------------------------------------------------
+include-tagged::{doc-tests}/ClusterClientDocumentationIT.java[delete-pipeline-execute]
+--------------------------------------------------
+<1> Execute the request and get back the response in a `WritePipelineResponse` object.
+
+[[java-rest-high-cluster-delete-pipeline-async]]
+==== Asynchronous Execution
+
+The asynchronous execution of a delete pipeline request requires both the `DeletePipelineRequest`
+instance and an `ActionListener` instance to be passed to the asynchronous
+method:
+
+["source","java",subs="attributes,callouts,macros"]
+--------------------------------------------------
+include-tagged::{doc-tests}/ClusterClientDocumentationIT.java[delete-pipeline-execute-async]
+--------------------------------------------------
+<1> The `DeletePipelineRequest` to execute and the `ActionListener` to use when
+the execution completes
+
+The asynchronous method does not block and returns immediately. Once it is
+completed the `ActionListener` is called back using the `onResponse` method
+if the execution successfully completed or using the `onFailure` method if
+it failed.
+
+A typical listener for `WritePipelineResponse` looks like:
+
+["source","java",subs="attributes,callouts,macros"]
+--------------------------------------------------
+include-tagged::{doc-tests}/ClusterClientDocumentationIT.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]]
+==== Delete Pipeline Response
+
+The returned `WritePipelineResponse` allows to retrieve information about the executed
+ operation as follows:
+
+["source","java",subs="attributes,callouts,macros"]
+--------------------------------------------------
+include-tagged::{doc-tests}/ClusterClientDocumentationIT.java[delete-pipeline-response]
+--------------------------------------------------
+<1> Indicates whether all of the nodes have acknowledged the request
diff --git a/docs/java-rest/high-level/cluster/put_pipeline.asciidoc b/docs/java-rest/high-level/cluster/put_pipeline.asciidoc
index d50a6741cc0..942b75b74cd 100644
--- a/docs/java-rest/high-level/cluster/put_pipeline.asciidoc
+++ b/docs/java-rest/high-level/cluster/put_pipeline.asciidoc
@@ -22,8 +22,8 @@ The following arguments can optionally be provided:
--------------------------------------------------
include-tagged::{doc-tests}/ClusterClientDocumentationIT.java[put-pipeline-request-timeout]
--------------------------------------------------
-<1> Timeout to wait for the all the nodes to acknowledge the index creation as a `TimeValue`
-<2> Timeout to wait for the all the nodes to acknowledge the index creation as a `String`
+<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"]
--------------------------------------------------
@@ -39,7 +39,7 @@ include-tagged::{doc-tests}/ClusterClientDocumentationIT.java[put-pipeline-reque
--------------------------------------------------
include-tagged::{doc-tests}/ClusterClientDocumentationIT.java[put-pipeline-execute]
--------------------------------------------------
-<1> Execute the request and get back the response in a PutPipelineResponse object.
+<1> Execute the request and get back the response in a WritePipelineResponse object.
[[java-rest-high-cluster-put-pipeline-async]]
==== Asynchronous Execution
@@ -60,7 +60,7 @@ completed the `ActionListener` is called back using the `onResponse` method
if the execution successfully completed or using the `onFailure` method if
it failed.
-A typical listener for `PutPipelineResponse` looks like:
+A typical listener for `WritePipelineResponse` looks like:
["source","java",subs="attributes,callouts,macros"]
--------------------------------------------------
@@ -73,7 +73,7 @@ provided as an argument
[[java-rest-high-cluster-put-pipeline-response]]
==== Put Pipeline Response
-The returned `PutPipelineResponse` allows to retrieve information about the executed
+The returned `WritePipelineResponse` allows to retrieve information about the executed
operation as follows:
["source","java",subs="attributes,callouts,macros"]
diff --git a/docs/java-rest/high-level/supported-apis.asciidoc b/docs/java-rest/high-level/supported-apis.asciidoc
index 412fa2aec42..34873a248b7 100644
--- a/docs/java-rest/high-level/supported-apis.asciidoc
+++ b/docs/java-rest/high-level/supported-apis.asciidoc
@@ -108,10 +108,12 @@ The Java High Level REST Client supports the following Cluster APIs:
* <>
* <>
* <>
+* <>
include::cluster/put_settings.asciidoc[]
include::cluster/put_pipeline.asciidoc[]
include::cluster/get_pipeline.asciidoc[]
+include::cluster/delete_pipeline.asciidoc[]
== Snapshot APIs
diff --git a/server/src/main/java/org/elasticsearch/action/ingest/PutPipelineResponse.java b/server/src/main/java/org/elasticsearch/action/ingest/PutPipelineResponse.java
deleted file mode 100644
index 13960ca99ef..00000000000
--- a/server/src/main/java/org/elasticsearch/action/ingest/PutPipelineResponse.java
+++ /dev/null
@@ -1,62 +0,0 @@
-/*
- * 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.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 PutPipelineResponse extends AcknowledgedResponse implements ToXContentObject {
-
- private static final ConstructingObjectParser PARSER = new ConstructingObjectParser<>(
- "cluster_put_pipeline", true, args -> new PutPipelineResponse((boolean) args[0]));
-
- static {
- declareAcknowledgedField(PARSER);
- }
-
- public PutPipelineResponse() {
- }
-
- public PutPipelineResponse(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);
- }
-
- public static PutPipelineResponse fromXContent(XContentParser parser) {
- return PARSER.apply(parser, null);
- }
-}
diff --git a/server/src/main/java/org/elasticsearch/action/ingest/WritePipelineResponse.java b/server/src/main/java/org/elasticsearch/action/ingest/WritePipelineResponse.java
index ced6085e0ac..36301d6735a 100644
--- a/server/src/main/java/org/elasticsearch/action/ingest/WritePipelineResponse.java
+++ b/server/src/main/java/org/elasticsearch/action/ingest/WritePipelineResponse.java
@@ -22,10 +22,20 @@ 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 {
+public class WritePipelineResponse extends AcknowledgedResponse implements ToXContentObject {
+
+ private static final ConstructingObjectParser PARSER = new ConstructingObjectParser<>(
+ "write_pipeline_response", true, args -> new WritePipelineResponse((boolean) args[0]));
+
+ static {
+ declareAcknowledgedField(PARSER);
+ }
WritePipelineResponse() {
@@ -46,4 +56,8 @@ public class WritePipelineResponse extends AcknowledgedResponse {
super.writeTo(out);
writeAcknowledged(out);
}
+
+ public static WritePipelineResponse fromXContent(XContentParser parser) {
+ return PARSER.apply(parser, null);
+ }
}
diff --git a/server/src/test/java/org/elasticsearch/action/ingest/PutPipelineResponseTests.java b/server/src/test/java/org/elasticsearch/action/ingest/PutPipelineResponseTests.java
deleted file mode 100644
index 438d3e55044..00000000000
--- a/server/src/test/java/org/elasticsearch/action/ingest/PutPipelineResponseTests.java
+++ /dev/null
@@ -1,53 +0,0 @@
-/*
- * 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.action.ingest;
-
-import org.elasticsearch.common.Strings;
-import org.elasticsearch.common.xcontent.XContentParser;
-import org.elasticsearch.test.AbstractStreamableXContentTestCase;
-
-public class PutPipelineResponseTests extends AbstractStreamableXContentTestCase {
-
- public void testToXContent() {
- PutPipelineResponse response = new PutPipelineResponse(true);
- String output = Strings.toString(response);
- assertEquals("{\"acknowledged\":true}", output);
- }
-
- @Override
- protected PutPipelineResponse doParseInstance(XContentParser parser) {
- return PutPipelineResponse.fromXContent(parser);
- }
-
- @Override
- protected PutPipelineResponse createTestInstance() {
- return new PutPipelineResponse(randomBoolean());
- }
-
- @Override
- protected PutPipelineResponse createBlankInstance() {
- return new PutPipelineResponse();
- }
-
- @Override
- protected PutPipelineResponse mutateInstance(PutPipelineResponse response) {
- return new PutPipelineResponse(response.isAcknowledged() == false);
- }
-}
diff --git a/server/src/test/java/org/elasticsearch/action/ingest/WritePipelineResponseTests.java b/server/src/test/java/org/elasticsearch/action/ingest/WritePipelineResponseTests.java
index 00327603ba8..f68545c3f79 100644
--- a/server/src/test/java/org/elasticsearch/action/ingest/WritePipelineResponseTests.java
+++ b/server/src/test/java/org/elasticsearch/action/ingest/WritePipelineResponseTests.java
@@ -19,15 +19,17 @@
package org.elasticsearch.action.ingest;
+import org.elasticsearch.common.Strings;
import org.elasticsearch.common.io.stream.BytesStreamOutput;
import org.elasticsearch.common.io.stream.StreamInput;
-import org.elasticsearch.test.ESTestCase;
+import org.elasticsearch.common.xcontent.XContentParser;
+import org.elasticsearch.test.AbstractStreamableXContentTestCase;
import java.io.IOException;
import static org.hamcrest.CoreMatchers.equalTo;
-public class WritePipelineResponseTests extends ESTestCase {
+public class WritePipelineResponseTests extends AbstractStreamableXContentTestCase {
public void testSerializationWithoutError() throws IOException {
boolean isAcknowledged = randomBoolean();
@@ -52,4 +54,30 @@ public class WritePipelineResponseTests extends ESTestCase {
assertThat(otherResponse.isAcknowledged(), equalTo(response.isAcknowledged()));
}
+
+ public void testToXContent() {
+ WritePipelineResponse response = new WritePipelineResponse(true);
+ String output = Strings.toString(response);
+ assertEquals("{\"acknowledged\":true}", output);
+ }
+
+ @Override
+ protected WritePipelineResponse doParseInstance(XContentParser parser) {
+ return WritePipelineResponse.fromXContent(parser);
+ }
+
+ @Override
+ protected WritePipelineResponse createTestInstance() {
+ return new WritePipelineResponse(randomBoolean());
+ }
+
+ @Override
+ protected WritePipelineResponse createBlankInstance() {
+ return new WritePipelineResponse();
+ }
+
+ @Override
+ protected WritePipelineResponse mutateInstance(WritePipelineResponse response) {
+ return new WritePipelineResponse(response.isAcknowledged() == false);
+ }
}