Browse Source

Add delete autoscaling policy API (#54601)

This commit adds an API for deleting autoscaling policies.
Jason Tedor 5 years ago
parent
commit
6d0bd25545

+ 2 - 0
docs/reference/autoscaling/apis/autoscaling-apis.asciidoc

@@ -10,8 +10,10 @@ You can use the following APIs to perform autoscaling operations.
 === Top-Level
 
 * <<autoscaling-get-autoscaling-decision,Get autoscaling decision>>
+* <<autoscaling-delete-autoscaling-policy,Delete autoscaling policy>>
 * <<autoscaling-put-autoscaling-policy,Put autoscaling policy>>
 
 // top-level
 include::get-autoscaling-decision.asciidoc[]
+include::delete-autoscaling-policy.asciidoc[]
 include::put-autoscaling-policy.asciidoc[]

+ 64 - 0
docs/reference/autoscaling/apis/delete-autoscaling-policy.asciidoc

@@ -0,0 +1,64 @@
+[role="xpack"]
+[testenv="platinum"]
+[[autoscaling-delete-autoscaling-policy]]
+=== Delete autoscaling policy API
+++++
+<titleabbrev>Delete autoscaling policy</titleabbrev>
+++++
+
+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/<name>
+--------------------------------------------------
+// TEST[s/<name>/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
+<<security-privileges>>.
+
+[[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
+}
+--------------------------------------------------

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

+ 9 - 1
x-pack/plugin/autoscaling/src/main/java/org/elasticsearch/xpack/autoscaling/Autoscaling.java

@@ -26,12 +26,15 @@ import org.elasticsearch.plugins.ActionPlugin;
 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 @@ public class Autoscaling extends Plugin implements ActionPlugin {
         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 class Autoscaling extends Plugin implements ActionPlugin {
         final Supplier<DiscoveryNodes> nodesInCluster
     ) {
         if (enabled) {
-            return List.of(new RestGetAutoscalingDecisionHandler(), new RestPutAutoscalingPolicyHandler());
+            return List.of(
+                new RestGetAutoscalingDecisionHandler(),
+                new RestDeleteAutoscalingPolicyActionHandler(),
+                new RestPutAutoscalingPolicyHandler()
+            );
         } else {
             return List.of();
         }

+ 70 - 0
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<AcknowledgedResponse> {
+
+    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<DeleteAutoscalingPolicyAction.Request> 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;
+        }
+
+    }
+
+}

+ 117 - 0
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<AcknowledgedResponse> 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<String, AutoscalingPolicyMetadata> 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();
+    }
+
+}

+ 38 - 0
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<Route> 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));
+    }
+
+}

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

+ 125 - 0
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<String, AutoscalingPolicyMetadata> 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);
+    }
+
+}

+ 24 - 0
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"
+            }
+          }
+        }
+      ]
+    }
+  }
+}