Browse Source

REST high-level client: add delete ingest pipeline API (#30865)

Relates to #27205
Sohaib Iftikhar 7 years ago
parent
commit
11887fa54a

+ 31 - 6
client/rest-high-level/src/main/java/org/elasticsearch/client/ClusterClient.java

@@ -23,10 +23,11 @@ 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.PutPipelineRequest;
-import org.elasticsearch.action.ingest.PutPipelineResponse;
+import org.elasticsearch.action.ingest.DeletePipelineRequest;
+import org.elasticsearch.action.ingest.WritePipelineResponse;
 
 import java.io.IOException;
 
@@ -74,9 +75,9 @@ public final class ClusterClient {
      * See
      * <a href="https://www.elastic.co/guide/en/elasticsearch/reference/current/put-pipeline-api.html"> Put Pipeline API on elastic.co</a>
      */
-    public PutPipelineResponse putPipeline(PutPipelineRequest request, Header... headers) throws IOException {
+    public WritePipelineResponse putPipeline(PutPipelineRequest request, Header... headers) throws IOException {
         return restHighLevelClient.performRequestAndParseEntity( request, RequestConverters::putPipeline,
-            PutPipelineResponse::fromXContent, emptySet(), headers);
+            WritePipelineResponse::fromXContent, emptySet(), headers);
     }
 
     /**
@@ -85,9 +86,9 @@ public final class ClusterClient {
      * 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<PutPipelineResponse> listener, Header... headers) {
+    public void putPipelineAsync(PutPipelineRequest request, ActionListener<WritePipelineResponse> 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
+     * <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);
+    }
 }

+ 15 - 0
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");

+ 16 - 2
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());
+    }
 }

+ 16 - 0
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<String, String> 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));

+ 76 - 5
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<PutPipelineResponse> listener =
-                new ActionListener<PutPipelineResponse>() {
+            ActionListener<WritePipelineResponse> listener =
+                new ActionListener<WritePipelineResponse>() {
                     @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<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));
+        }
+    }
 }

+ 80 - 0
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

+ 5 - 5
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"]

+ 2 - 0
docs/java-rest/high-level/supported-apis.asciidoc

@@ -108,10 +108,12 @@ 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[]
 
 == Snapshot APIs
 

+ 0 - 62
server/src/main/java/org/elasticsearch/action/ingest/PutPipelineResponse.java

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

+ 15 - 1
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<WritePipelineResponse, Void> 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);
+    }
 }

+ 0 - 53
server/src/test/java/org/elasticsearch/action/ingest/PutPipelineResponseTests.java

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

+ 30 - 2
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<WritePipelineResponse> {
 
     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);
+    }
 }