diff --git a/docs/reference/autoscaling/apis/autoscaling-apis.asciidoc b/docs/reference/autoscaling/apis/autoscaling-apis.asciidoc index 078c568305903..f05614b1e7b84 100644 --- a/docs/reference/autoscaling/apis/autoscaling-apis.asciidoc +++ b/docs/reference/autoscaling/apis/autoscaling-apis.asciidoc @@ -10,8 +10,10 @@ You can use the following APIs to perform autoscaling operations. === Top-Level * <> +* <> * <> // top-level include::get-autoscaling-decision.asciidoc[] +include::delete-autoscaling-policy.asciidoc[] include::put-autoscaling-policy.asciidoc[] diff --git a/docs/reference/autoscaling/apis/delete-autoscaling-policy.asciidoc b/docs/reference/autoscaling/apis/delete-autoscaling-policy.asciidoc new file mode 100644 index 0000000000000..2c41ed42432ee --- /dev/null +++ b/docs/reference/autoscaling/apis/delete-autoscaling-policy.asciidoc @@ -0,0 +1,64 @@ +[role="xpack"] +[testenv="platinum"] +[[autoscaling-delete-autoscaling-policy]] +=== Delete autoscaling policy API +++++ +Delete autoscaling policy +++++ + +Delete autoscaling policy. + +[[autoscaling-delete-autoscaling-policy-request]] +==== {api-request-title} + +[source,console] +-------------------------------------------------- +PUT /_autoscaling/policy/my_autoscaling_policy +{ + "policy": { + "deciders": { + "always": { + } + } + } +} +-------------------------------------------------- +// TESTSETUP + +[source,console] +-------------------------------------------------- +DELETE /_autoscaling/policy/ +-------------------------------------------------- +// TEST[s//my_autoscaling_policy/] + +[[autoscaling-delete-autoscaling-policy-prereqs]] +==== {api-prereq-title} + +* If the {es} {security-features} are enabled, you must have +`manage_autoscaling` cluster privileges. For more information, see +<>. + +[[autoscaling-delete-autoscaling-policy-desc]] +==== {api-description-title} + +This API deletes an autoscaling policy with the provided name. + +[[autoscaling-delete-autoscaling-policy-examples]] +==== {api-examples-title} + +This example deletes an autoscaling policy named `my_autosaling_policy`. + +[source,console] +-------------------------------------------------- +DELETE /_autoscaling/policy/my_autoscaling_policy +-------------------------------------------------- +// TEST + +The API returns the following result: + +[source,console-result] +-------------------------------------------------- +{ + "acknowledged": true +} +-------------------------------------------------- diff --git a/x-pack/plugin/autoscaling/qa/rest/src/test/resources/rest-api-spec/test/autoscaling/delete_autoscaling_policy.yml b/x-pack/plugin/autoscaling/qa/rest/src/test/resources/rest-api-spec/test/autoscaling/delete_autoscaling_policy.yml new file mode 100644 index 0000000000000..d8ed15a02d92b --- /dev/null +++ b/x-pack/plugin/autoscaling/qa/rest/src/test/resources/rest-api-spec/test/autoscaling/delete_autoscaling_policy.yml @@ -0,0 +1,22 @@ +--- +"Test delete autoscaling policy": + - do: + autoscaling.put_autoscaling_policy: + name: hot + body: + policy: + deciders: + always: {} + + - match: { "acknowledged": true } + + - do: + autoscaling.delete_autoscaling_policy: + name: hot + + # TODO: add validation that the policy is removed after we have a get policy API + + - do: + catch: bad_request + autoscaling.delete_autoscaling_policy: + name: does_not_exist diff --git a/x-pack/plugin/autoscaling/src/main/java/org/elasticsearch/xpack/autoscaling/Autoscaling.java b/x-pack/plugin/autoscaling/src/main/java/org/elasticsearch/xpack/autoscaling/Autoscaling.java index a806e0c1b69bf..f41c1e0af8281 100644 --- a/x-pack/plugin/autoscaling/src/main/java/org/elasticsearch/xpack/autoscaling/Autoscaling.java +++ b/x-pack/plugin/autoscaling/src/main/java/org/elasticsearch/xpack/autoscaling/Autoscaling.java @@ -26,12 +26,15 @@ import org.elasticsearch.plugins.Plugin; import org.elasticsearch.rest.RestController; import org.elasticsearch.rest.RestHandler; +import org.elasticsearch.xpack.autoscaling.action.DeleteAutoscalingPolicyAction; import org.elasticsearch.xpack.autoscaling.action.GetAutoscalingDecisionAction; import org.elasticsearch.xpack.autoscaling.action.PutAutoscalingPolicyAction; +import org.elasticsearch.xpack.autoscaling.action.TransportDeleteAutoscalingPolicyAction; import org.elasticsearch.xpack.autoscaling.action.TransportGetAutoscalingDecisionAction; import org.elasticsearch.xpack.autoscaling.action.TransportPutAutoscalingPolicyAction; import org.elasticsearch.xpack.autoscaling.decision.AlwaysAutoscalingDecider; import org.elasticsearch.xpack.autoscaling.decision.AutoscalingDecider; +import org.elasticsearch.xpack.autoscaling.rest.RestDeleteAutoscalingPolicyActionHandler; import org.elasticsearch.xpack.autoscaling.rest.RestGetAutoscalingDecisionHandler; import org.elasticsearch.xpack.autoscaling.rest.RestPutAutoscalingPolicyHandler; import org.elasticsearch.xpack.core.XPackPlugin; @@ -99,6 +102,7 @@ boolean isSnapshot() { if (enabled) { return List.of( new ActionHandler<>(GetAutoscalingDecisionAction.INSTANCE, TransportGetAutoscalingDecisionAction.class), + new ActionHandler<>(DeleteAutoscalingPolicyAction.INSTANCE, TransportDeleteAutoscalingPolicyAction.class), new ActionHandler<>(PutAutoscalingPolicyAction.INSTANCE, TransportPutAutoscalingPolicyAction.class) ); } else { @@ -117,7 +121,11 @@ public List getRestHandlers( final Supplier nodesInCluster ) { if (enabled) { - return List.of(new RestGetAutoscalingDecisionHandler(), new RestPutAutoscalingPolicyHandler()); + return List.of( + new RestGetAutoscalingDecisionHandler(), + new RestDeleteAutoscalingPolicyActionHandler(), + new RestPutAutoscalingPolicyHandler() + ); } else { return List.of(); } diff --git a/x-pack/plugin/autoscaling/src/main/java/org/elasticsearch/xpack/autoscaling/action/DeleteAutoscalingPolicyAction.java b/x-pack/plugin/autoscaling/src/main/java/org/elasticsearch/xpack/autoscaling/action/DeleteAutoscalingPolicyAction.java new file mode 100644 index 0000000000000..ee02bd2900fe4 --- /dev/null +++ b/x-pack/plugin/autoscaling/src/main/java/org/elasticsearch/xpack/autoscaling/action/DeleteAutoscalingPolicyAction.java @@ -0,0 +1,70 @@ +/* + * Copyright Elasticsearch B.V. and/or licensed to Elasticsearch B.V. under one + * or more contributor license agreements. Licensed under the Elastic License; + * you may not use this file except in compliance with the Elastic License. + */ + +package org.elasticsearch.xpack.autoscaling.action; + +import org.elasticsearch.action.ActionRequestValidationException; +import org.elasticsearch.action.ActionType; +import org.elasticsearch.action.support.master.AcknowledgedRequest; +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.ToXContentObject; +import org.elasticsearch.common.xcontent.XContentBuilder; + +import java.io.IOException; +import java.util.Objects; + +public class DeleteAutoscalingPolicyAction extends ActionType { + + public static final DeleteAutoscalingPolicyAction INSTANCE = new DeleteAutoscalingPolicyAction(); + public static final String NAME = "cluster:admin/autoscaling/delete_autoscaling_policy"; + + private DeleteAutoscalingPolicyAction() { + super(NAME, AcknowledgedResponse::new); + } + + public static class Request extends AcknowledgedRequest implements ToXContentObject { + + private final String name; + + public String name() { + return name; + } + + public Request(final String name) { + this.name = Objects.requireNonNull(name); + } + + public Request(final StreamInput in) throws IOException { + super(in); + name = in.readString(); + } + + @Override + public void writeTo(final StreamOutput out) throws IOException { + super.writeTo(out); + out.writeString(name); + } + + @Override + public ActionRequestValidationException validate() { + return null; + } + + @Override + public XContentBuilder toXContent(final XContentBuilder builder, final Params params) throws IOException { + builder.startObject(); + { + + } + builder.endObject(); + return builder; + } + + } + +} diff --git a/x-pack/plugin/autoscaling/src/main/java/org/elasticsearch/xpack/autoscaling/action/TransportDeleteAutoscalingPolicyAction.java b/x-pack/plugin/autoscaling/src/main/java/org/elasticsearch/xpack/autoscaling/action/TransportDeleteAutoscalingPolicyAction.java new file mode 100644 index 0000000000000..02af626fcb496 --- /dev/null +++ b/x-pack/plugin/autoscaling/src/main/java/org/elasticsearch/xpack/autoscaling/action/TransportDeleteAutoscalingPolicyAction.java @@ -0,0 +1,117 @@ +/* + * Copyright Elasticsearch B.V. and/or licensed to Elasticsearch B.V. under one + * or more contributor license agreements. Licensed under the Elastic License; + * you may not use this file except in compliance with the Elastic License. + */ + +package org.elasticsearch.xpack.autoscaling.action; + +import org.apache.logging.log4j.LogManager; +import org.apache.logging.log4j.Logger; +import org.elasticsearch.action.ActionListener; +import org.elasticsearch.action.support.ActionFilters; +import org.elasticsearch.action.support.master.AcknowledgedResponse; +import org.elasticsearch.action.support.master.TransportMasterNodeAction; +import org.elasticsearch.cluster.AckedClusterStateUpdateTask; +import org.elasticsearch.cluster.ClusterState; +import org.elasticsearch.cluster.block.ClusterBlockException; +import org.elasticsearch.cluster.block.ClusterBlockLevel; +import org.elasticsearch.cluster.metadata.IndexNameExpressionResolver; +import org.elasticsearch.cluster.metadata.Metadata; +import org.elasticsearch.cluster.service.ClusterService; +import org.elasticsearch.common.inject.Inject; +import org.elasticsearch.common.io.stream.StreamInput; +import org.elasticsearch.tasks.Task; +import org.elasticsearch.threadpool.ThreadPool; +import org.elasticsearch.transport.TransportService; +import org.elasticsearch.xpack.autoscaling.AutoscalingMetadata; +import org.elasticsearch.xpack.autoscaling.policy.AutoscalingPolicyMetadata; + +import java.io.IOException; +import java.util.SortedMap; +import java.util.TreeMap; + +public class TransportDeleteAutoscalingPolicyAction extends TransportMasterNodeAction< + DeleteAutoscalingPolicyAction.Request, + AcknowledgedResponse> { + + private static final Logger logger = LogManager.getLogger(TransportPutAutoscalingPolicyAction.class); + + @Inject + public TransportDeleteAutoscalingPolicyAction( + final TransportService transportService, + final ClusterService clusterService, + final ThreadPool threadPool, + final ActionFilters actionFilters, + final IndexNameExpressionResolver indexNameExpressionResolver + ) { + super( + DeleteAutoscalingPolicyAction.NAME, + transportService, + clusterService, + threadPool, + actionFilters, + DeleteAutoscalingPolicyAction.Request::new, + indexNameExpressionResolver + ); + } + + @Override + protected String executor() { + return ThreadPool.Names.SAME; + } + + @Override + protected AcknowledgedResponse read(final StreamInput in) throws IOException { + return new AcknowledgedResponse(in); + } + + @Override + protected void masterOperation( + final Task task, + final DeleteAutoscalingPolicyAction.Request request, + final ClusterState state, + final ActionListener listener + ) { + clusterService.submitStateUpdateTask("delete-autoscaling-policy", new AckedClusterStateUpdateTask<>(request, listener) { + + @Override + protected AcknowledgedResponse newResponse(final boolean acknowledged) { + return new AcknowledgedResponse(acknowledged); + } + + @Override + public ClusterState execute(final ClusterState currentState) { + return deleteAutoscalingPolicy(currentState, request.name(), logger); + } + + }); + } + + @Override + protected ClusterBlockException checkBlock(final DeleteAutoscalingPolicyAction.Request request, final ClusterState state) { + return state.blocks().globalBlockedException(ClusterBlockLevel.METADATA_WRITE); + } + + static ClusterState deleteAutoscalingPolicy(final ClusterState currentState, final String name, final Logger logger) { + final ClusterState.Builder builder = ClusterState.builder(currentState); + final AutoscalingMetadata currentMetadata; + if (currentState.metadata().custom(AutoscalingMetadata.NAME) != null) { + currentMetadata = currentState.metadata().custom(AutoscalingMetadata.NAME); + } else { + // we will reject the request below when we try to look up the policy by name + currentMetadata = AutoscalingMetadata.EMPTY; + } + if (currentMetadata.policies().containsKey(name) == false) { + throw new IllegalArgumentException("autoscaling policy with name [" + name + "] does not exist"); + } + final SortedMap newPolicies = new TreeMap<>(currentMetadata.policies()); + final AutoscalingPolicyMetadata policy = newPolicies.remove(name); + assert policy != null : name; + logger.info("deleting autoscaling policy [{}]", name); + final AutoscalingMetadata newMetadata = new AutoscalingMetadata(newPolicies); + builder.metadata(Metadata.builder(currentState.getMetadata()).putCustom(AutoscalingMetadata.NAME, newMetadata).build()); + return builder.build(); + } + +} diff --git a/x-pack/plugin/autoscaling/src/main/java/org/elasticsearch/xpack/autoscaling/rest/RestDeleteAutoscalingPolicyActionHandler.java b/x-pack/plugin/autoscaling/src/main/java/org/elasticsearch/xpack/autoscaling/rest/RestDeleteAutoscalingPolicyActionHandler.java new file mode 100644 index 0000000000000..aab89bb7dd39b --- /dev/null +++ b/x-pack/plugin/autoscaling/src/main/java/org/elasticsearch/xpack/autoscaling/rest/RestDeleteAutoscalingPolicyActionHandler.java @@ -0,0 +1,38 @@ +/* + * Copyright Elasticsearch B.V. and/or licensed to Elasticsearch B.V. under one + * or more contributor license agreements. Licensed under the Elastic License; + * you may not use this file except in compliance with the Elastic License. + */ + +package org.elasticsearch.xpack.autoscaling.rest; + +import org.elasticsearch.client.node.NodeClient; +import org.elasticsearch.rest.BaseRestHandler; +import org.elasticsearch.rest.RestRequest; +import org.elasticsearch.rest.action.RestToXContentListener; +import org.elasticsearch.xpack.autoscaling.action.DeleteAutoscalingPolicyAction; + +import java.util.List; + +import static org.elasticsearch.rest.RestRequest.Method.DELETE; + +public class RestDeleteAutoscalingPolicyActionHandler extends BaseRestHandler { + + @Override + public List routes() { + return List.of(new Route(DELETE, "/_autoscaling/policy/{name}")); + } + + @Override + public String getName() { + return "delete_autoscaling_policy"; + } + + @Override + protected RestChannelConsumer prepareRequest(final RestRequest restRequest, final NodeClient client) { + final String name = restRequest.param("name"); + final DeleteAutoscalingPolicyAction.Request request = new DeleteAutoscalingPolicyAction.Request(name); + return channel -> client.execute(DeleteAutoscalingPolicyAction.INSTANCE, request, new RestToXContentListener<>(channel)); + } + +} diff --git a/x-pack/plugin/autoscaling/src/test/java/org/elasticsearch/xpack/autoscaling/action/TransportDeleteAutoscalingPolicyActionIT.java b/x-pack/plugin/autoscaling/src/test/java/org/elasticsearch/xpack/autoscaling/action/TransportDeleteAutoscalingPolicyActionIT.java new file mode 100644 index 0000000000000..e492041585ef8 --- /dev/null +++ b/x-pack/plugin/autoscaling/src/test/java/org/elasticsearch/xpack/autoscaling/action/TransportDeleteAutoscalingPolicyActionIT.java @@ -0,0 +1,46 @@ +/* + * Copyright Elasticsearch B.V. and/or licensed to Elasticsearch B.V. under one + * or more contributor license agreements. Licensed under the Elastic License; + * you may not use this file except in compliance with the Elastic License. + */ + +package org.elasticsearch.xpack.autoscaling.action; + +import org.elasticsearch.cluster.ClusterState; +import org.elasticsearch.xpack.autoscaling.AutoscalingIntegTestCase; +import org.elasticsearch.xpack.autoscaling.AutoscalingMetadata; +import org.elasticsearch.xpack.autoscaling.policy.AutoscalingPolicy; + +import static org.elasticsearch.test.hamcrest.ElasticsearchAssertions.assertAcked; +import static org.elasticsearch.xpack.autoscaling.AutoscalingTestCase.randomAutoscalingPolicy; +import static org.hamcrest.Matchers.containsString; +import static org.hamcrest.Matchers.hasKey; +import static org.hamcrest.Matchers.not; + +public class TransportDeleteAutoscalingPolicyActionIT extends AutoscalingIntegTestCase { + + public void testDeletePolicy() { + final AutoscalingPolicy policy = randomAutoscalingPolicy(); + final PutAutoscalingPolicyAction.Request putRequest = new PutAutoscalingPolicyAction.Request(policy); + assertAcked(client().execute(PutAutoscalingPolicyAction.INSTANCE, putRequest).actionGet()); + // we trust that the policy is in the cluster state since we have tests for putting policies + final DeleteAutoscalingPolicyAction.Request deleteRequest = new DeleteAutoscalingPolicyAction.Request(policy.name()); + assertAcked(client().execute(DeleteAutoscalingPolicyAction.INSTANCE, deleteRequest).actionGet()); + // now verify that the policy is not in the cluster state + final ClusterState state = client().admin().cluster().prepareState().get().getState(); + final AutoscalingMetadata metadata = state.metadata().custom(AutoscalingMetadata.NAME); + assertNotNull(metadata); + assertThat(metadata.policies(), not(hasKey(policy.name()))); + } + + public void testDeleteNonExistentPolicy() { + final String name = randomAlphaOfLength(8); + final DeleteAutoscalingPolicyAction.Request deleteRequest = new DeleteAutoscalingPolicyAction.Request(name); + final IllegalArgumentException e = expectThrows( + IllegalArgumentException.class, + () -> client().execute(DeleteAutoscalingPolicyAction.INSTANCE, deleteRequest).actionGet() + ); + assertThat(e.getMessage(), containsString("autoscaling policy with name [" + name + "] does not exist")); + } + +} diff --git a/x-pack/plugin/autoscaling/src/test/java/org/elasticsearch/xpack/autoscaling/action/TransportDeleteAutoscalingPolicyActionTests.java b/x-pack/plugin/autoscaling/src/test/java/org/elasticsearch/xpack/autoscaling/action/TransportDeleteAutoscalingPolicyActionTests.java new file mode 100644 index 0000000000000..334e602f24edb --- /dev/null +++ b/x-pack/plugin/autoscaling/src/test/java/org/elasticsearch/xpack/autoscaling/action/TransportDeleteAutoscalingPolicyActionTests.java @@ -0,0 +1,125 @@ +/* + * Copyright Elasticsearch B.V. and/or licensed to Elasticsearch B.V. under one + * or more contributor license agreements. Licensed under the Elastic License; + * you may not use this file except in compliance with the Elastic License. + */ + +package org.elasticsearch.xpack.autoscaling.action; + +import org.apache.logging.log4j.Logger; +import org.elasticsearch.action.support.ActionFilters; +import org.elasticsearch.cluster.ClusterName; +import org.elasticsearch.cluster.ClusterState; +import org.elasticsearch.cluster.block.ClusterBlockException; +import org.elasticsearch.cluster.block.ClusterBlocks; +import org.elasticsearch.cluster.coordination.NoMasterBlockService; +import org.elasticsearch.cluster.metadata.IndexNameExpressionResolver; +import org.elasticsearch.cluster.metadata.Metadata; +import org.elasticsearch.cluster.service.ClusterService; +import org.elasticsearch.threadpool.ThreadPool; +import org.elasticsearch.transport.TransportService; +import org.elasticsearch.xpack.autoscaling.AutoscalingMetadata; +import org.elasticsearch.xpack.autoscaling.AutoscalingTestCase; +import org.elasticsearch.xpack.autoscaling.policy.AutoscalingPolicyMetadata; + +import java.util.Map; + +import static org.hamcrest.Matchers.containsString; +import static org.hamcrest.Matchers.equalTo; +import static org.hamcrest.Matchers.hasKey; +import static org.hamcrest.Matchers.not; +import static org.hamcrest.Matchers.nullValue; +import static org.mockito.Mockito.mock; +import static org.mockito.Mockito.verify; +import static org.mockito.Mockito.verifyNoMoreInteractions; + +public class TransportDeleteAutoscalingPolicyActionTests extends AutoscalingTestCase { + + public void testWriteBlock() { + final TransportDeleteAutoscalingPolicyAction action = new TransportDeleteAutoscalingPolicyAction( + mock(TransportService.class), + mock(ClusterService.class), + mock(ThreadPool.class), + mock(ActionFilters.class), + mock(IndexNameExpressionResolver.class) + ); + final ClusterBlocks blocks = ClusterBlocks.builder() + .addGlobalBlock( + randomFrom( + Metadata.CLUSTER_READ_ONLY_BLOCK, + Metadata.CLUSTER_READ_ONLY_ALLOW_DELETE_BLOCK, + NoMasterBlockService.NO_MASTER_BLOCK_WRITES + ) + ) + .build(); + final ClusterState state = ClusterState.builder(new ClusterName(randomAlphaOfLength(8))).blocks(blocks).build(); + final ClusterBlockException e = action.checkBlock(new DeleteAutoscalingPolicyAction.Request(randomAlphaOfLength(8)), state); + assertThat(e, not(nullValue())); + } + + public void testNoWriteBlock() { + final TransportDeleteAutoscalingPolicyAction action = new TransportDeleteAutoscalingPolicyAction( + mock(TransportService.class), + mock(ClusterService.class), + mock(ThreadPool.class), + mock(ActionFilters.class), + mock(IndexNameExpressionResolver.class) + ); + final ClusterBlocks blocks = ClusterBlocks.builder().build(); + final ClusterState state = ClusterState.builder(new ClusterName(randomAlphaOfLength(8))).blocks(blocks).build(); + final ClusterBlockException e = action.checkBlock(new DeleteAutoscalingPolicyAction.Request(randomAlphaOfLength(8)), state); + assertThat(e, nullValue()); + } + + public void testDeletePolicy() { + final ClusterState currentState; + { + final ClusterState.Builder builder = ClusterState.builder(new ClusterName(randomAlphaOfLength(8))); + builder.metadata( + Metadata.builder().putCustom(AutoscalingMetadata.NAME, randomAutoscalingMetadataOfPolicyCount(randomIntBetween(1, 8))) + ); + currentState = builder.build(); + } + final AutoscalingMetadata currentMetadata = currentState.metadata().custom(AutoscalingMetadata.NAME); + final String name = randomFrom(currentMetadata.policies().keySet()); + final Logger mockLogger = mock(Logger.class); + final ClusterState state = TransportDeleteAutoscalingPolicyAction.deleteAutoscalingPolicy(currentState, name, mockLogger); + + // ensure the policy is deleted from the cluster state + final AutoscalingMetadata metadata = state.metadata().custom(AutoscalingMetadata.NAME); + assertNotNull(metadata); + assertThat(metadata.policies(), not(hasKey(name))); + verify(mockLogger).info("deleting autoscaling policy [{}]", name); + verifyNoMoreInteractions(mockLogger); + + // ensure that existing policies were otherwise preserved + for (final Map.Entry entry : currentMetadata.policies().entrySet()) { + if (entry.getKey().equals(name)) { + continue; + } + assertThat(metadata.policies(), hasKey(entry.getKey())); + assertThat(metadata.policies().get(entry.getKey()).policy(), equalTo(entry.getValue().policy())); + } + } + + public void testDeleteNonExistentPolicy() { + final ClusterState currentState; + { + final ClusterState.Builder builder = ClusterState.builder(new ClusterName(randomAlphaOfLength(8))); + builder.metadata( + Metadata.builder().putCustom(AutoscalingMetadata.NAME, randomAutoscalingMetadataOfPolicyCount(randomIntBetween(1, 8))) + ); + currentState = builder.build(); + } + final AutoscalingMetadata currentMetadata = currentState.metadata().custom(AutoscalingMetadata.NAME); + final String name = randomValueOtherThanMany(currentMetadata.policies().keySet()::contains, () -> randomAlphaOfLength(8)); + final Logger mockLogger = mock(Logger.class); + final IllegalArgumentException e = expectThrows( + IllegalArgumentException.class, + () -> TransportDeleteAutoscalingPolicyAction.deleteAutoscalingPolicy(currentState, name, mockLogger) + ); + assertThat(e.getMessage(), containsString("autoscaling policy with name [" + name + "] does not exist")); + verifyNoMoreInteractions(mockLogger); + } + +} diff --git a/x-pack/plugin/src/test/resources/rest-api-spec/api/autoscaling.delete_autoscaling_policy.json b/x-pack/plugin/src/test/resources/rest-api-spec/api/autoscaling.delete_autoscaling_policy.json new file mode 100644 index 0000000000000..de5be0f94b811 --- /dev/null +++ b/x-pack/plugin/src/test/resources/rest-api-spec/api/autoscaling.delete_autoscaling_policy.json @@ -0,0 +1,24 @@ +{ + "autoscaling.delete_autoscaling_policy":{ + "documentation":{ + "url":"https://www.elastic.co/guide/en/elasticsearch/reference/current/autoscaling-delete-autoscaling-policy.html" + }, + "stability":"experimental", + "url":{ + "paths":[ + { + "path":"/_autoscaling/policy/{name}", + "methods":[ + "DELETE" + ], + "parts":{ + "name":{ + "type":"string", + "description":"the name of the autoscaling policy" + } + } + } + ] + } + } +}