From 56aabcdd69b1625bbaa3b53dda2edb7ffbc9c249 Mon Sep 17 00:00:00 2001 From: Lee Hinman Date: Mon, 9 Sep 2019 09:55:34 -0600 Subject: [PATCH] Add retention to Snapshot Lifecycle Management (#46407) MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit This commit adds retention to the existing Snapshot Lifecycle Management feature (#38461) as described in #43663. This allows a user to configure SLM to automatically delete older snapshots based on a number of criteria. An example policy would look like: ``` PUT /_slm/policy/snapshot-every-day { "schedule": "0 30 2 * * ?", "name": "", "repository": "my-s3-repository", "config": { "indices": ["foo-*", "important"] }, // Newly configured retention options "retention": { // Snapshots should be deleted after 14 days "expire_after": "14d", // Keep a maximum of thirty snapshots "max_count": 30, // Keep a minimum of the four most recent snapshots "min_count": 4 } } ``` SLM Retention is run on a scheduled configurable with the `slm.retention_schedule` setting, which supports cron expressions. Deletions are run for a configurable time bounded by the `slm.retention_duration` setting, which defaults to 1 hour. Included in this work is a new SLM stats API endpoint available through ``` json GET /_slm/stats ``` That returns statistics about snapshot taken and deleted, as well as successful retention runs, failures, and the time spent deleting snapshots. #45362 has more information as well as an example of the output. These stats are also included when retrieving SLM policies via the API. * Add base framework for snapshot retention (#43605) * Add base framework for snapshot retention This adds a basic `SnapshotRetentionService` and `SnapshotRetentionTask` to start as the basis for SLM's retention implementation. Relates to #38461 * Remove extraneous 'public' * Use a local var instead of reading class var repeatedly * Add SnapshotRetentionConfiguration for retention configuration (#43777) * Add SnapshotRetentionConfiguration for retention configuration This commit adds the `SnapshotRetentionConfiguration` class and its HLRC counterpart to encapsulate the configuration for SLM retention. Currently only a single parameter is supported as an example (we still need to discuss the different options we want to support and their names) to keep the size of the PR down. It also does not yet include version serialization checks since the original SLM branch has not yet been merged. Relates to #43663 * Fix REST tests * Fix more documentation * Use Objects.equals to avoid NPE * Put `randomSnapshotLifecyclePolicy` in only one place * Occasionally return retention with no configuration * Implement SnapshotRetentionTask's snapshot filtering and delet… (#44764) * Implement SnapshotRetentionTask's snapshot filtering and deletion This commit implements the snapshot filtering and deletion for `SnapshotRetentionTask`. Currently only the expire-after age is used for determining whether a snapshot is eligible for deletion. Relates to #43663 * Fix deletes running on the wrong thread * Handle missing or null policy in snap metadata differently * Convert Tuple> to Map> * Use the `OriginSettingClient` to work with security, enhance logging * Prevent NPE in test by mocking Client * Allow empty/missing SLM retention configuration (#45018) Semi-related to #44465, this allows the `"retention"` configuration map to be missing. Relates to #43663 * Add min_count and max_count as SLM retention predicates (#44926) This adds the configuration options for `min_count` and `max_count` as well as the logic for determining whether a snapshot meets this criteria to SLM's retention feature. These options are optional and one, two, or all three can be specified in an SLM policy. Relates to #43663 * Time-bound deletion of snapshots in retention delete function (#45065) * Time-bound deletion of snapshots in retention delete function With a cluster that has a large number of snapshots, it's possible that snapshot deletion can take a very long time (especially since deletes currently have to happen in a serial fashion). To prevent snapshot deletion from taking forever in a cluster and blocking other operations, this commit adds a setting to allow configuring a maximum time to spend deletion snapshots during retention. This dynamic setting defaults to 1 hour and is best-effort, meaning that it doesn't hard stop a deletion at an hour mark, but ensures that once the time has passed, all subsequent deletions are deferred until the next retention cycle. Relates to #43663 * Wow snapshots suuuure can take a long time. * Use a LongSupplier instead of actually sleeping * Remove TestLogging annotation * Remove rate limiting * Add SLM metrics gathering and endpoint (#45362) * Add SLM metrics gathering and endpoint This commit adds the infrastructure to gather metrics about the different SLM actions that a cluster takes. These actions are stored in `SnapshotLifecycleStats` and perpetuated in cluster state. The stats stored include the number of snapshots taken, failed, deleted, the number of retention runs, as well as per-policy counts for snapshots taken, failed, and deleted. It also includes the amount of time spent deleting snapshots from SLM retention. This commit also adds an endpoint for retrieving all stats (further commits will expose this in the SLM get-policy API) that looks like: ``` GET /_slm/stats { "retention_runs" : 13, "retention_failed" : 0, "retention_timed_out" : 0, "retention_deletion_time" : "1.4s", "retention_deletion_time_millis" : 1404, "policy_metrics" : { "daily-snapshots2" : { "snapshots_taken" : 7, "snapshots_failed" : 0, "snapshots_deleted" : 6, "snapshot_deletion_failures" : 0 }, "daily-snapshots" : { "snapshots_taken" : 12, "snapshots_failed" : 0, "snapshots_deleted" : 12, "snapshot_deletion_failures" : 6 } }, "total_snapshots_taken" : 19, "total_snapshots_failed" : 0, "total_snapshots_deleted" : 18, "total_snapshot_deletion_failures" : 6 } ``` This does not yet include HLRC for this, as this commit is quite large on its own. That will be added in a subsequent commit. Relates to #43663 * Version qualify serialization * Initialize counters outside constructor * Use computeIfAbsent instead of being too verbose * Move part of XContent generation into subclass * Fix REST action for master merge * Unused import * Record history of SLM retention actions (#45513) This commit records the deletion of snapshots by the retention component of SLM into the SLM history index for the purposes of reviewing operations taken by SLM and alerting. * Retry SLM retention after currently running snapshot completes (#45802) * Retry SLM retention after currently running snapshot completes This commit adds a ClusterStateObserver to wait until the currently running snapshot is complete before proceeding with snapshot deletion. SLM retention waits for the maximum allowed deletion time for the snapshot to complete, however, the waiting time is not factored into the limit on actual deletions. Relates to #43663 * Increase timeout waiting for snapshot completion * Apply patch From https://github.com/original-brownbear/elasticsearch/commit/2374316f0d1912c9e1498bece195546a1dc60bce.patch * Rename test variables * [TEST] Be less strict for stats checking * Skip SLM retention if ILM is STOPPING or STOPPED (#45869) This adds a check to ensure we take no action during SLM retention if ILM is currently stopped or in the process of stopping. Relates to #43663 * Check all actions preventing snapshot delete during retention (#45992) * Check all actions preventing snapshot delete during retention run Previously we only checked to see if a snapshot was currently running, but it turns out that more things can block snapshot deletion. This changes the check to be a check for: - a snapshot currently running - a deletion already in progress - a repo cleanup in progress - a restore currently running This was found by CI where a third party delete in a test caused SLM retention deletion to throw an exception. Relates to #43663 * Add unit test for okayToDeleteSnapshots * Fix bug where SLM retention task would be scheduled on every node * Enhance test logging * Ignore if snapshot is already deleted * Missing import * Fix SnapshotRetentionServiceTests * Expose SLM policy stats in get SLM policy API (#45989) This also adds support for the SLM stats endpoint to the high level rest client. Retrieving a policy now looks like: ```json { "daily-snapshots" : { "version": 1, "modified_date": "2019-04-23T01:30:00.000Z", "modified_date_millis": 1556048137314, "policy" : { "schedule": "0 30 1 * * ?", "name": "", "repository": "my_repository", "config": { "indices": ["data-*", "important"], "ignore_unavailable": false, "include_global_state": false }, "retention": {} }, "stats": { "snapshots_taken": 0, "snapshots_failed": 0, "snapshots_deleted": 0, "snapshot_deletion_failures": 0 }, "next_execution": "2019-04-24T01:30:00.000Z", "next_execution_millis": 1556048160000 } } ``` Relates to #43663 * Rewrite SnapshotLifecycleIT as as ESIntegTestCase (#46356) * Rewrite SnapshotLifecycleIT as as ESIntegTestCase This commit splits `SnapshotLifecycleIT` into two different tests. `SnapshotLifecycleRestIT` which includes the tests that do not require slow repositories, and `SLMSnapshotBlockingIntegTests` which is now an integration test using `MockRepository` to simulate a snapshot being in progress. Relates to #43663 Resolves #46205 * Add error logging when exceptions are thrown --- .../client/IndexLifecycleClient.java | 38 ++ .../IndexLifecycleRequestConverters.java | 11 + .../slm/GetSnapshotLifecycleStatsRequest.java | 30 ++ .../GetSnapshotLifecycleStatsResponse.java | 68 +++ .../client/slm/SnapshotLifecyclePolicy.java | 31 +- .../slm/SnapshotLifecyclePolicyMetadata.java | 26 +- .../client/slm/SnapshotLifecycleStats.java | 261 ++++++++++ .../slm/SnapshotRetentionConfiguration.java | 133 +++++ .../documentation/ILMDocumentationIT.java | 26 +- .../ilm/get_snapshot_lifecycle_stats.asciidoc | 35 ++ docs/reference/ilm/apis/slm-api.asciidoc | 71 ++- .../ilm/getting-started-slm.asciidoc | 6 +- .../elasticsearch/common/unit/TimeValue.java | 8 + .../cluster/RepositoryCleanupInProgress.java | 2 +- .../cluster/SnapshotDeletionsInProgress.java | 2 +- .../xpack/core/XPackClientPlugin.java | 2 + .../xpack/core/ilm/LifecycleSettings.java | 18 + .../core/slm/SnapshotLifecycleMetadata.java | 55 +- .../core/slm/SnapshotLifecyclePolicy.java | 25 +- .../core/slm/SnapshotLifecyclePolicyItem.java | 48 +- .../slm/SnapshotRetentionConfiguration.java | 245 +++++++++ .../action/GetSnapshotLifecycleAction.java | 4 + .../GetSnapshotLifecycleStatsAction.java | 102 ++++ .../core/slm/history/SnapshotHistoryItem.java | 49 +- .../xpack/slm/SnapshotLifecycleStats.java | 383 ++++++++++++++ .../slm/SnapshotLifecycleMetadataTests.java | 41 ++ .../slm/SnapshotLifecyclePolicyItemTests.java | 38 +- .../SnapshotLifecyclePolicyMetadataTests.java | 19 +- .../history/SnapshotHistoryStoreTests.java | 9 +- .../slm/SnapshotLifecycleStatsTests.java | 60 +++ .../SnapshotRetentionConfigurationTests.java | 109 ++++ x-pack/plugin/ilm/build.gradle | 1 - x-pack/plugin/ilm/qa/multi-node/build.gradle | 4 - .../ilm/TimeSeriesLifecycleActionsIT.java | 7 +- ...leIT.java => SnapshotLifecycleRestIT.java} | 247 ++++++--- .../xpack/security/PermissionsIT.java | 3 +- .../xpack/ilm/IndexLifecycle.java | 21 +- .../xpack/ilm/OperationModeUpdateTask.java | 3 +- .../xpack/slm/SnapshotLifecycleService.java | 2 +- .../xpack/slm/SnapshotLifecycleTask.java | 12 +- .../xpack/slm/SnapshotRetentionService.java | 103 ++++ .../xpack/slm/SnapshotRetentionTask.java | 491 ++++++++++++++++++ .../slm/UpdateSnapshotLifecycleStatsTask.java | 56 ++ .../RestGetSnapshotLifecycleStatsAction.java | 35 ++ ...ransportDeleteSnapshotLifecycleAction.java | 5 +- .../TransportGetSnapshotLifecycleAction.java | 6 +- ...nsportGetSnapshotLifecycleStatsAction.java | 64 +++ .../TransportPutSnapshotLifecycleAction.java | 7 +- .../ilm/OperationModeUpdateTaskTests.java | 4 +- .../slm/SLMSnapshotBlockingIntegTests.java | 281 ++++++++++ .../slm/SnapshotLifecyclePolicyTests.java | 77 ++- .../slm/SnapshotLifecycleServiceTests.java | 38 +- .../xpack/slm/SnapshotLifecycleTaskTests.java | 9 +- .../slm/SnapshotRetentionServiceTests.java | 80 +++ .../xpack/slm/SnapshotRetentionTaskTests.java | 463 +++++++++++++++++ .../rest-api-spec/api/slm.get_stats.json | 19 + 56 files changed, 3767 insertions(+), 226 deletions(-) create mode 100644 client/rest-high-level/src/main/java/org/elasticsearch/client/slm/GetSnapshotLifecycleStatsRequest.java create mode 100644 client/rest-high-level/src/main/java/org/elasticsearch/client/slm/GetSnapshotLifecycleStatsResponse.java create mode 100644 client/rest-high-level/src/main/java/org/elasticsearch/client/slm/SnapshotLifecycleStats.java create mode 100644 client/rest-high-level/src/main/java/org/elasticsearch/client/slm/SnapshotRetentionConfiguration.java create mode 100644 docs/java-rest/high-level/ilm/get_snapshot_lifecycle_stats.asciidoc create mode 100644 x-pack/plugin/core/src/main/java/org/elasticsearch/xpack/core/slm/SnapshotRetentionConfiguration.java create mode 100644 x-pack/plugin/core/src/main/java/org/elasticsearch/xpack/core/slm/action/GetSnapshotLifecycleStatsAction.java create mode 100644 x-pack/plugin/core/src/main/java/org/elasticsearch/xpack/slm/SnapshotLifecycleStats.java create mode 100644 x-pack/plugin/core/src/test/java/org/elasticsearch/xpack/core/slm/SnapshotLifecycleMetadataTests.java create mode 100644 x-pack/plugin/core/src/test/java/org/elasticsearch/xpack/slm/SnapshotLifecycleStatsTests.java create mode 100644 x-pack/plugin/core/src/test/java/org/elasticsearch/xpack/slm/SnapshotRetentionConfigurationTests.java rename x-pack/plugin/ilm/qa/multi-node/src/test/java/org/elasticsearch/xpack/slm/{SnapshotLifecycleIT.java => SnapshotLifecycleRestIT.java} (58%) create mode 100644 x-pack/plugin/ilm/src/main/java/org/elasticsearch/xpack/slm/SnapshotRetentionService.java create mode 100644 x-pack/plugin/ilm/src/main/java/org/elasticsearch/xpack/slm/SnapshotRetentionTask.java create mode 100644 x-pack/plugin/ilm/src/main/java/org/elasticsearch/xpack/slm/UpdateSnapshotLifecycleStatsTask.java create mode 100644 x-pack/plugin/ilm/src/main/java/org/elasticsearch/xpack/slm/action/RestGetSnapshotLifecycleStatsAction.java create mode 100644 x-pack/plugin/ilm/src/main/java/org/elasticsearch/xpack/slm/action/TransportGetSnapshotLifecycleStatsAction.java create mode 100644 x-pack/plugin/ilm/src/test/java/org/elasticsearch/xpack/slm/SLMSnapshotBlockingIntegTests.java create mode 100644 x-pack/plugin/ilm/src/test/java/org/elasticsearch/xpack/slm/SnapshotRetentionServiceTests.java create mode 100644 x-pack/plugin/ilm/src/test/java/org/elasticsearch/xpack/slm/SnapshotRetentionTaskTests.java create mode 100644 x-pack/plugin/src/test/resources/rest-api-spec/api/slm.get_stats.json diff --git a/client/rest-high-level/src/main/java/org/elasticsearch/client/IndexLifecycleClient.java b/client/rest-high-level/src/main/java/org/elasticsearch/client/IndexLifecycleClient.java index 8bfd8fe8ac0f3..80cee2c420ef3 100644 --- a/client/rest-high-level/src/main/java/org/elasticsearch/client/IndexLifecycleClient.java +++ b/client/rest-high-level/src/main/java/org/elasticsearch/client/IndexLifecycleClient.java @@ -39,6 +39,8 @@ import org.elasticsearch.client.slm.ExecuteSnapshotLifecyclePolicyResponse; import org.elasticsearch.client.slm.GetSnapshotLifecyclePolicyRequest; import org.elasticsearch.client.slm.GetSnapshotLifecyclePolicyResponse; +import org.elasticsearch.client.slm.GetSnapshotLifecycleStatsRequest; +import org.elasticsearch.client.slm.GetSnapshotLifecycleStatsResponse; import org.elasticsearch.client.slm.PutSnapshotLifecyclePolicyRequest; import java.io.IOException; @@ -464,4 +466,40 @@ public Cancellable executeSnapshotLifecyclePolicyAsync( request, IndexLifecycleRequestConverters::executeSnapshotLifecyclePolicy, options, ExecuteSnapshotLifecyclePolicyResponse::fromXContent, listener, emptySet()); } + + /** + * Retrieve snapshot lifecycle statistics. + * See
+     *  https://www.elastic.co/guide/en/elasticsearch/client/java-rest/current/
+     *  java-rest-high-ilm-slm-get-snapshot-lifecycle-stats.html
+     * 
+ * for more. + * @param request the request + * @param options the request options (e.g. headers), use {@link RequestOptions#DEFAULT} if nothing needs to be customized + * @return the response + * @throws IOException in case there is a problem sending the request or parsing back the response + */ + public GetSnapshotLifecycleStatsResponse getSnapshotLifecycleStats(GetSnapshotLifecycleStatsRequest request, + RequestOptions options) throws IOException { + return restHighLevelClient.performRequestAndParseEntity(request, IndexLifecycleRequestConverters::getSnapshotLifecycleStats, + options, GetSnapshotLifecycleStatsResponse::fromXContent, emptySet()); + } + + /** + * Asynchronously retrieve snapshot lifecycle statistics. + * See
+     *  https://www.elastic.co/guide/en/elasticsearch/client/java-rest/current/
+     *  java-rest-high-ilm-slm-get-snapshot-lifecycle-stats.html
+     * 
+ * for more. + * @param request the request + * @param options the request options (e.g. headers), use {@link RequestOptions#DEFAULT} if nothing needs to be customized + * @param listener the listener to be notified upon request completion + * @return cancellable that may be used to cancel the request + */ + public Cancellable getSnapshotLifecycleStatsAsync(GetSnapshotLifecycleStatsRequest request, RequestOptions options, + ActionListener listener) { + return restHighLevelClient.performRequestAsyncAndParseEntity(request, IndexLifecycleRequestConverters::getSnapshotLifecycleStats, + options, GetSnapshotLifecycleStatsResponse::fromXContent, listener, emptySet()); + } } diff --git a/client/rest-high-level/src/main/java/org/elasticsearch/client/IndexLifecycleRequestConverters.java b/client/rest-high-level/src/main/java/org/elasticsearch/client/IndexLifecycleRequestConverters.java index 6f4e991f0dbc7..563f178711e45 100644 --- a/client/rest-high-level/src/main/java/org/elasticsearch/client/IndexLifecycleRequestConverters.java +++ b/client/rest-high-level/src/main/java/org/elasticsearch/client/IndexLifecycleRequestConverters.java @@ -35,6 +35,7 @@ import org.elasticsearch.client.slm.DeleteSnapshotLifecyclePolicyRequest; import org.elasticsearch.client.slm.ExecuteSnapshotLifecyclePolicyRequest; import org.elasticsearch.client.slm.GetSnapshotLifecyclePolicyRequest; +import org.elasticsearch.client.slm.GetSnapshotLifecycleStatsRequest; import org.elasticsearch.client.slm.PutSnapshotLifecyclePolicyRequest; import org.elasticsearch.common.Strings; @@ -215,4 +216,14 @@ static Request executeSnapshotLifecyclePolicy(ExecuteSnapshotLifecyclePolicyRequ request.addParameters(params.asMap()); return request; } + + static Request getSnapshotLifecycleStats(GetSnapshotLifecycleStatsRequest getSnapshotLifecycleStatsRequest) { + String endpoint = new RequestConverters.EndpointBuilder().addPathPartAsIs("_slm/stats").build(); + Request request = new Request(HttpGet.METHOD_NAME, endpoint); + RequestConverters.Params params = new RequestConverters.Params(); + params.withMasterTimeout(getSnapshotLifecycleStatsRequest.masterNodeTimeout()); + params.withTimeout(getSnapshotLifecycleStatsRequest.timeout()); + request.addParameters(params.asMap()); + return request; + } } diff --git a/client/rest-high-level/src/main/java/org/elasticsearch/client/slm/GetSnapshotLifecycleStatsRequest.java b/client/rest-high-level/src/main/java/org/elasticsearch/client/slm/GetSnapshotLifecycleStatsRequest.java new file mode 100644 index 0000000000000..285a179e3e612 --- /dev/null +++ b/client/rest-high-level/src/main/java/org/elasticsearch/client/slm/GetSnapshotLifecycleStatsRequest.java @@ -0,0 +1,30 @@ +/* + * Licensed to Elasticsearch under one or more contributor + * license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright + * ownership. Elasticsearch licenses this file to you under + * the Apache License, Version 2.0 (the "License"); you may + * not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package org.elasticsearch.client.slm; + +import org.elasticsearch.client.TimedRequest; + +public class GetSnapshotLifecycleStatsRequest extends TimedRequest { + + public GetSnapshotLifecycleStatsRequest() { + super(); + } + +} diff --git a/client/rest-high-level/src/main/java/org/elasticsearch/client/slm/GetSnapshotLifecycleStatsResponse.java b/client/rest-high-level/src/main/java/org/elasticsearch/client/slm/GetSnapshotLifecycleStatsResponse.java new file mode 100644 index 0000000000000..1aed51afc72fd --- /dev/null +++ b/client/rest-high-level/src/main/java/org/elasticsearch/client/slm/GetSnapshotLifecycleStatsResponse.java @@ -0,0 +1,68 @@ +/* + * Licensed to Elasticsearch under one or more contributor + * license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright + * ownership. Elasticsearch licenses this file to you under + * the Apache License, Version 2.0 (the "License"); you may + * not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package org.elasticsearch.client.slm; + +import org.elasticsearch.common.xcontent.ToXContentObject; +import org.elasticsearch.common.xcontent.XContentBuilder; +import org.elasticsearch.common.xcontent.XContentParser; + +import java.io.IOException; +import java.util.Objects; + +public class GetSnapshotLifecycleStatsResponse implements ToXContentObject { + + private final SnapshotLifecycleStats stats; + + public GetSnapshotLifecycleStatsResponse(SnapshotLifecycleStats stats) { + this.stats = stats; + } + + public SnapshotLifecycleStats getStats() { + return this.stats; + } + + public static GetSnapshotLifecycleStatsResponse fromXContent(XContentParser parser) throws IOException { + return new GetSnapshotLifecycleStatsResponse(SnapshotLifecycleStats.parse(parser)); + } + + @Override + public XContentBuilder toXContent(XContentBuilder builder, Params params) throws IOException { + return stats.toXContent(builder, params); + } + + @Override + public boolean equals(Object o) { + if (this == o) { + return true; + } + + if (o == null || getClass() != o.getClass()) { + return false; + } + + GetSnapshotLifecycleStatsResponse other = (GetSnapshotLifecycleStatsResponse) o; + return Objects.equals(this.stats, other.stats); + } + + @Override + public int hashCode() { + return Objects.hash(this.stats); + } +} diff --git a/client/rest-high-level/src/main/java/org/elasticsearch/client/slm/SnapshotLifecyclePolicy.java b/client/rest-high-level/src/main/java/org/elasticsearch/client/slm/SnapshotLifecyclePolicy.java index 0370cb262f125..e9c521772a589 100644 --- a/client/rest-high-level/src/main/java/org/elasticsearch/client/slm/SnapshotLifecyclePolicy.java +++ b/client/rest-high-level/src/main/java/org/elasticsearch/client/slm/SnapshotLifecyclePolicy.java @@ -38,11 +38,13 @@ public class SnapshotLifecyclePolicy implements ToXContentObject { private final String schedule; private final String repository; private final Map configuration; + private final SnapshotRetentionConfiguration retentionPolicy; private static final ParseField NAME = new ParseField("name"); private static final ParseField SCHEDULE = new ParseField("schedule"); private static final ParseField REPOSITORY = new ParseField("repository"); private static final ParseField CONFIG = new ParseField("config"); + private static final ParseField RETENTION = new ParseField("retention"); @SuppressWarnings("unchecked") private static final ConstructingObjectParser PARSER = @@ -52,7 +54,8 @@ public class SnapshotLifecyclePolicy implements ToXContentObject { String schedule = (String) a[1]; String repo = (String) a[2]; Map config = (Map) a[3]; - return new SnapshotLifecyclePolicy(id, name, schedule, repo, config); + SnapshotRetentionConfiguration retention = (SnapshotRetentionConfiguration) a[4]; + return new SnapshotLifecyclePolicy(id, name, schedule, repo, config, retention); }); static { @@ -60,15 +63,18 @@ public class SnapshotLifecyclePolicy implements ToXContentObject { PARSER.declareString(ConstructingObjectParser.constructorArg(), SCHEDULE); PARSER.declareString(ConstructingObjectParser.constructorArg(), REPOSITORY); PARSER.declareObject(ConstructingObjectParser.optionalConstructorArg(), (p, c) -> p.map(), CONFIG); + PARSER.declareObject(ConstructingObjectParser.optionalConstructorArg(), SnapshotRetentionConfiguration::parse, RETENTION); } public SnapshotLifecyclePolicy(final String id, final String name, final String schedule, - final String repository, @Nullable Map configuration) { - this.id = Objects.requireNonNull(id); - this.name = name; - this.schedule = schedule; - this.repository = repository; + final String repository, @Nullable final Map configuration, + @Nullable final SnapshotRetentionConfiguration retentionPolicy) { + this.id = Objects.requireNonNull(id, "policy id is required"); + this.name = Objects.requireNonNull(name, "policy snapshot name is required"); + this.schedule = Objects.requireNonNull(schedule, "policy schedule is required"); + this.repository = Objects.requireNonNull(repository, "policy snapshot repository is required"); this.configuration = configuration; + this.retentionPolicy = retentionPolicy; } public String getId() { @@ -92,6 +98,11 @@ public Map getConfig() { return this.configuration; } + @Nullable + public SnapshotRetentionConfiguration getRetentionPolicy() { + return this.retentionPolicy; + } + public static SnapshotLifecyclePolicy parse(XContentParser parser, String id) { return PARSER.apply(parser, id); } @@ -105,13 +116,16 @@ public XContentBuilder toXContent(XContentBuilder builder, Params params) throws if (this.configuration != null) { builder.field(CONFIG.getPreferredName(), this.configuration); } + if (this.retentionPolicy != null) { + builder.field(RETENTION.getPreferredName(), this.retentionPolicy); + } builder.endObject(); return builder; } @Override public int hashCode() { - return Objects.hash(id, name, schedule, repository, configuration); + return Objects.hash(id, name, schedule, repository, configuration, retentionPolicy); } @Override @@ -128,7 +142,8 @@ public boolean equals(Object obj) { Objects.equals(name, other.name) && Objects.equals(schedule, other.schedule) && Objects.equals(repository, other.repository) && - Objects.equals(configuration, other.configuration); + Objects.equals(configuration, other.configuration) && + Objects.equals(retentionPolicy, other.retentionPolicy); } @Override diff --git a/client/rest-high-level/src/main/java/org/elasticsearch/client/slm/SnapshotLifecyclePolicyMetadata.java b/client/rest-high-level/src/main/java/org/elasticsearch/client/slm/SnapshotLifecyclePolicyMetadata.java index 9b967e8c33b07..d459069a2906e 100644 --- a/client/rest-high-level/src/main/java/org/elasticsearch/client/slm/SnapshotLifecyclePolicyMetadata.java +++ b/client/rest-high-level/src/main/java/org/elasticsearch/client/slm/SnapshotLifecyclePolicyMetadata.java @@ -42,6 +42,7 @@ public class SnapshotLifecyclePolicyMetadata implements ToXContentObject { static final ParseField NEXT_EXECUTION_MILLIS = new ParseField("next_execution_millis"); static final ParseField NEXT_EXECUTION = new ParseField("next_execution"); static final ParseField SNAPSHOT_IN_PROGRESS = new ParseField("in_progress"); + static final ParseField POLICY_STATS = new ParseField("stats"); private final SnapshotLifecyclePolicy policy; private final long version; @@ -53,6 +54,7 @@ public class SnapshotLifecyclePolicyMetadata implements ToXContentObject { private final SnapshotInvocationRecord lastFailure; @Nullable private final SnapshotInProgress snapshotInProgress; + private final SnapshotLifecycleStats.SnapshotPolicyStats policyStats; @SuppressWarnings("unchecked") public static final ConstructingObjectParser PARSER = @@ -65,8 +67,9 @@ public class SnapshotLifecyclePolicyMetadata implements ToXContentObject { SnapshotInvocationRecord lastFailure = (SnapshotInvocationRecord) a[4]; long nextExecution = (long) a[5]; SnapshotInProgress sip = (SnapshotInProgress) a[6]; - - return new SnapshotLifecyclePolicyMetadata(policy, version, modifiedDate, lastSuccess, lastFailure, nextExecution, sip); + SnapshotLifecycleStats.SnapshotPolicyStats stats = (SnapshotLifecycleStats.SnapshotPolicyStats) a[7]; + return new SnapshotLifecyclePolicyMetadata(policy, version, modifiedDate, lastSuccess, + lastFailure, nextExecution, sip, stats); }); static { @@ -77,6 +80,9 @@ public class SnapshotLifecyclePolicyMetadata implements ToXContentObject { PARSER.declareObject(ConstructingObjectParser.optionalConstructorArg(), SnapshotInvocationRecord::parse, LAST_FAILURE); PARSER.declareLong(ConstructingObjectParser.constructorArg(), NEXT_EXECUTION_MILLIS); PARSER.declareObject(ConstructingObjectParser.optionalConstructorArg(), SnapshotInProgress::parse, SNAPSHOT_IN_PROGRESS); + PARSER.declareObject(ConstructingObjectParser.constructorArg(), + (p, c) -> SnapshotLifecycleStats.SnapshotPolicyStats.parse(p, "policy"), POLICY_STATS); + } public static SnapshotLifecyclePolicyMetadata parse(XContentParser parser, String id) { @@ -86,7 +92,8 @@ public static SnapshotLifecyclePolicyMetadata parse(XContentParser parser, Strin public SnapshotLifecyclePolicyMetadata(SnapshotLifecyclePolicy policy, long version, long modifiedDate, SnapshotInvocationRecord lastSuccess, SnapshotInvocationRecord lastFailure, long nextExecution, - @Nullable SnapshotInProgress snapshotInProgress) { + @Nullable SnapshotInProgress snapshotInProgress, + SnapshotLifecycleStats.SnapshotPolicyStats policyStats) { this.policy = policy; this.version = version; this.modifiedDate = modifiedDate; @@ -94,6 +101,7 @@ public SnapshotLifecyclePolicyMetadata(SnapshotLifecyclePolicy policy, long vers this.lastFailure = lastFailure; this.nextExecution = nextExecution; this.snapshotInProgress = snapshotInProgress; + this.policyStats = policyStats; } public SnapshotLifecyclePolicy getPolicy() { @@ -124,6 +132,10 @@ public long getNextExecution() { return this.nextExecution; } + public SnapshotLifecycleStats.SnapshotPolicyStats getPolicyStats() { + return this.policyStats; + } + @Nullable public SnapshotInProgress getSnapshotInProgress() { return this.snapshotInProgress; @@ -145,13 +157,16 @@ public XContentBuilder toXContent(XContentBuilder builder, Params params) throws if (snapshotInProgress != null) { builder.field(SNAPSHOT_IN_PROGRESS.getPreferredName(), snapshotInProgress); } + builder.startObject(POLICY_STATS.getPreferredName()); + this.policyStats.toXContent(builder, params); + builder.endObject(); builder.endObject(); return builder; } @Override public int hashCode() { - return Objects.hash(policy, version, modifiedDate, lastSuccess, lastFailure, nextExecution); + return Objects.hash(policy, version, modifiedDate, lastSuccess, lastFailure, nextExecution, policyStats); } @Override @@ -168,7 +183,8 @@ public boolean equals(Object obj) { Objects.equals(modifiedDate, other.modifiedDate) && Objects.equals(lastSuccess, other.lastSuccess) && Objects.equals(lastFailure, other.lastFailure) && - Objects.equals(nextExecution, other.nextExecution); + Objects.equals(nextExecution, other.nextExecution) && + Objects.equals(policyStats, other.policyStats); } public static class SnapshotInProgress implements ToXContentObject { diff --git a/client/rest-high-level/src/main/java/org/elasticsearch/client/slm/SnapshotLifecycleStats.java b/client/rest-high-level/src/main/java/org/elasticsearch/client/slm/SnapshotLifecycleStats.java new file mode 100644 index 0000000000000..fc54f74649b01 --- /dev/null +++ b/client/rest-high-level/src/main/java/org/elasticsearch/client/slm/SnapshotLifecycleStats.java @@ -0,0 +1,261 @@ +/* + * Licensed to Elasticsearch under one or more contributor + * license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright + * ownership. Elasticsearch licenses this file to you under + * the Apache License, Version 2.0 (the "License"); you may + * not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package org.elasticsearch.client.slm; + +import org.elasticsearch.common.ParseField; +import org.elasticsearch.common.Strings; +import org.elasticsearch.common.unit.TimeValue; +import org.elasticsearch.common.xcontent.ConstructingObjectParser; +import org.elasticsearch.common.xcontent.ToXContentFragment; +import org.elasticsearch.common.xcontent.ToXContentObject; +import org.elasticsearch.common.xcontent.XContentBuilder; +import org.elasticsearch.common.xcontent.XContentParser; + +import java.io.IOException; +import java.util.Collections; +import java.util.List; +import java.util.Map; +import java.util.Objects; +import java.util.function.Function; +import java.util.stream.Collectors; + +public class SnapshotLifecycleStats implements ToXContentObject { + + private final long retentionRunCount; + private final long retentionFailedCount; + private final long retentionTimedOut; + private final long retentionTimeMs; + private final Map policyStats; + + public static final ParseField RETENTION_RUNS = new ParseField("retention_runs"); + public static final ParseField RETENTION_FAILED = new ParseField("retention_failed"); + public static final ParseField RETENTION_TIMED_OUT = new ParseField("retention_timed_out"); + public static final ParseField RETENTION_TIME = new ParseField("retention_deletion_time"); + public static final ParseField RETENTION_TIME_MILLIS = new ParseField("retention_deletion_time_millis"); + public static final ParseField POLICY_STATS = new ParseField("policy_stats"); + public static final ParseField TOTAL_TAKEN = new ParseField("total_snapshots_taken"); + public static final ParseField TOTAL_FAILED = new ParseField("total_snapshots_failed"); + public static final ParseField TOTAL_DELETIONS = new ParseField("total_snapshots_deleted"); + public static final ParseField TOTAL_DELETION_FAILURES = new ParseField("total_snapshot_deletion_failures"); + + + @SuppressWarnings("unchecked") + private static final ConstructingObjectParser PARSER = + new ConstructingObjectParser<>("snapshot_policy_stats", true, + a -> { + long runs = (long) a[0]; + long failed = (long) a[1]; + long timedOut = (long) a[2]; + long timeMs = (long) a[3]; + Map policyStatsMap = ((List) a[4]).stream() + .collect(Collectors.toMap(m -> m.policyId, Function.identity())); + return new SnapshotLifecycleStats(runs, failed, timedOut, timeMs, policyStatsMap); + }); + + static { + PARSER.declareLong(ConstructingObjectParser.constructorArg(), RETENTION_RUNS); + PARSER.declareLong(ConstructingObjectParser.constructorArg(), RETENTION_FAILED); + PARSER.declareLong(ConstructingObjectParser.constructorArg(), RETENTION_TIMED_OUT); + PARSER.declareLong(ConstructingObjectParser.constructorArg(), RETENTION_TIME_MILLIS); + PARSER.declareNamedObjects(ConstructingObjectParser.constructorArg(), (p, c, n) -> SnapshotPolicyStats.parse(p, n), POLICY_STATS); + } + + // Package visible for testing + private SnapshotLifecycleStats(long retentionRuns, long retentionFailed, long retentionTimedOut, long retentionTimeMs, + Map policyStats) { + this.retentionRunCount = retentionRuns; + this.retentionFailedCount = retentionFailed; + this.retentionTimedOut = retentionTimedOut; + this.retentionTimeMs = retentionTimeMs; + this.policyStats = policyStats; + } + + public static SnapshotLifecycleStats parse(XContentParser parser) { + return PARSER.apply(parser, null); + } + + public long getRetentionRunCount() { + return retentionRunCount; + } + + public long getRetentionFailedCount() { + return retentionFailedCount; + } + + public long getRetentionTimedOut() { + return retentionTimedOut; + } + + public long getRetentionTimeMillis() { + return retentionTimeMs; + } + + /** + * @return a map of per-policy stats for each SLM policy + */ + public Map getMetrics() { + return Collections.unmodifiableMap(this.policyStats); + } + + @Override + public XContentBuilder toXContent(XContentBuilder builder, Params params) throws IOException { + builder.startObject(); + builder.field(RETENTION_RUNS.getPreferredName(), this.retentionRunCount); + builder.field(RETENTION_FAILED.getPreferredName(), this.retentionFailedCount); + builder.field(RETENTION_TIMED_OUT.getPreferredName(), this.retentionTimedOut); + TimeValue retentionTime = TimeValue.timeValueMillis(this.retentionTimeMs); + builder.field(RETENTION_TIME.getPreferredName(), retentionTime); + builder.field(RETENTION_TIME_MILLIS.getPreferredName(), retentionTime.millis()); + + Map metrics = getMetrics(); + long totalTaken = metrics.values().stream().mapToLong(s -> s.snapshotsTaken).sum(); + long totalFailed = metrics.values().stream().mapToLong(s -> s.snapshotsFailed).sum(); + long totalDeleted = metrics.values().stream().mapToLong(s -> s.snapshotsDeleted).sum(); + long totalDeleteFailures = metrics.values().stream().mapToLong(s -> s.snapshotDeleteFailures).sum(); + builder.field(TOTAL_TAKEN.getPreferredName(), totalTaken); + builder.field(TOTAL_FAILED.getPreferredName(), totalFailed); + builder.field(TOTAL_DELETIONS.getPreferredName(), totalDeleted); + builder.field(TOTAL_DELETION_FAILURES.getPreferredName(), totalDeleteFailures); + builder.startObject(POLICY_STATS.getPreferredName()); + for (Map.Entry policy : metrics.entrySet()) { + SnapshotPolicyStats perPolicyMetrics = policy.getValue(); + builder.startObject(perPolicyMetrics.policyId); + perPolicyMetrics.toXContent(builder, params); + builder.endObject(); + } + builder.endObject(); + builder.endObject(); + return builder; + } + + @Override + public int hashCode() { + return Objects.hash(retentionRunCount, retentionFailedCount, retentionTimedOut, retentionTimeMs, policyStats); + } + + @Override + public boolean equals(Object obj) { + if (obj == null) { + return false; + } + if (obj.getClass() != getClass()) { + return false; + } + SnapshotLifecycleStats other = (SnapshotLifecycleStats) obj; + return retentionRunCount == other.retentionRunCount && + retentionFailedCount == other.retentionFailedCount && + retentionTimedOut == other.retentionTimedOut && + retentionTimeMs == other.retentionTimeMs && + Objects.equals(policyStats, other.policyStats); + } + + @Override + public String toString() { + return Strings.toString(this); + } + + public static class SnapshotPolicyStats implements ToXContentFragment { + private final String policyId; + private final long snapshotsTaken; + private final long snapshotsFailed; + private final long snapshotsDeleted; + private final long snapshotDeleteFailures; + + static final ParseField SNAPSHOTS_TAKEN = new ParseField("snapshots_taken"); + static final ParseField SNAPSHOTS_FAILED = new ParseField("snapshots_failed"); + static final ParseField SNAPSHOTS_DELETED = new ParseField("snapshots_deleted"); + static final ParseField SNAPSHOT_DELETION_FAILURES = new ParseField("snapshot_deletion_failures"); + + private static final ConstructingObjectParser PARSER = + new ConstructingObjectParser<>("snapshot_policy_stats", true, + (a, id) -> { + long taken = (long) a[0]; + long failed = (long) a[1]; + long deleted = (long) a[2]; + long deleteFailed = (long) a[3]; + return new SnapshotPolicyStats(id, taken, failed, deleted, deleteFailed); + }); + + static { + PARSER.declareLong(ConstructingObjectParser.constructorArg(), SNAPSHOTS_TAKEN); + PARSER.declareLong(ConstructingObjectParser.constructorArg(), SNAPSHOTS_FAILED); + PARSER.declareLong(ConstructingObjectParser.constructorArg(), SNAPSHOTS_DELETED); + PARSER.declareLong(ConstructingObjectParser.constructorArg(), SNAPSHOT_DELETION_FAILURES); + } + + public SnapshotPolicyStats(String policyId, long snapshotsTaken, long snapshotsFailed, long deleted, long failedDeletes) { + this.policyId = policyId; + this.snapshotsTaken = snapshotsTaken; + this.snapshotsFailed = snapshotsFailed; + this.snapshotsDeleted = deleted; + this.snapshotDeleteFailures = failedDeletes; + } + + public static SnapshotPolicyStats parse(XContentParser parser, String policyId) { + return PARSER.apply(parser, policyId); + } + + public long getSnapshotsTaken() { + return snapshotsTaken; + } + + public long getSnapshotsFailed() { + return snapshotsFailed; + } + + public long getSnapshotsDeleted() { + return snapshotsDeleted; + } + + public long getSnapshotDeleteFailures() { + return snapshotDeleteFailures; + } + + @Override + public int hashCode() { + return Objects.hash(policyId, snapshotsTaken, snapshotsFailed, snapshotsDeleted, snapshotDeleteFailures); + } + + @Override + public boolean equals(Object obj) { + if (obj == null) { + return false; + } + if (obj.getClass() != getClass()) { + return false; + } + SnapshotPolicyStats other = (SnapshotPolicyStats) obj; + return Objects.equals(policyId, other.policyId) && + snapshotsTaken == other.snapshotsTaken && + snapshotsFailed == other.snapshotsFailed && + snapshotsDeleted == other.snapshotsDeleted && + snapshotDeleteFailures == other.snapshotDeleteFailures; + } + + @Override + public XContentBuilder toXContent(XContentBuilder builder, Params params) throws IOException { + builder.field(SnapshotPolicyStats.SNAPSHOTS_TAKEN.getPreferredName(), snapshotsTaken); + builder.field(SnapshotPolicyStats.SNAPSHOTS_FAILED.getPreferredName(), snapshotsFailed); + builder.field(SnapshotPolicyStats.SNAPSHOTS_DELETED.getPreferredName(), snapshotsDeleted); + builder.field(SnapshotPolicyStats.SNAPSHOT_DELETION_FAILURES.getPreferredName(), snapshotDeleteFailures); + return builder; + } + } +} diff --git a/client/rest-high-level/src/main/java/org/elasticsearch/client/slm/SnapshotRetentionConfiguration.java b/client/rest-high-level/src/main/java/org/elasticsearch/client/slm/SnapshotRetentionConfiguration.java new file mode 100644 index 0000000000000..f98e61fef170d --- /dev/null +++ b/client/rest-high-level/src/main/java/org/elasticsearch/client/slm/SnapshotRetentionConfiguration.java @@ -0,0 +1,133 @@ +/* + * Licensed to Elasticsearch under one or more contributor + * license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright + * ownership. Elasticsearch licenses this file to you under + * the Apache License, Version 2.0 (the "License"); you may + * not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package org.elasticsearch.client.slm; + +import org.elasticsearch.common.Nullable; +import org.elasticsearch.common.ParseField; +import org.elasticsearch.common.Strings; +import org.elasticsearch.common.unit.TimeValue; +import org.elasticsearch.common.xcontent.ConstructingObjectParser; +import org.elasticsearch.common.xcontent.ToXContentObject; +import org.elasticsearch.common.xcontent.XContentBuilder; +import org.elasticsearch.common.xcontent.XContentParser; + +import java.io.IOException; +import java.util.Objects; + +public class SnapshotRetentionConfiguration implements ToXContentObject { + + public static final SnapshotRetentionConfiguration EMPTY = new SnapshotRetentionConfiguration(null, null, null); + + private static final ParseField EXPIRE_AFTER = new ParseField("expire_after"); + private static final ParseField MINIMUM_SNAPSHOT_COUNT = new ParseField("min_count"); + private static final ParseField MAXIMUM_SNAPSHOT_COUNT = new ParseField("max_count"); + + private static final ConstructingObjectParser PARSER = + new ConstructingObjectParser<>("snapshot_retention", true, a -> { + TimeValue expireAfter = a[0] == null ? null : TimeValue.parseTimeValue((String) a[0], EXPIRE_AFTER.getPreferredName()); + Integer minCount = (Integer) a[1]; + Integer maxCount = (Integer) a[2]; + return new SnapshotRetentionConfiguration(expireAfter, minCount, maxCount); + }); + + static { + PARSER.declareString(ConstructingObjectParser.optionalConstructorArg(), EXPIRE_AFTER); + PARSER.declareInt(ConstructingObjectParser.optionalConstructorArg(), MINIMUM_SNAPSHOT_COUNT); + PARSER.declareInt(ConstructingObjectParser.optionalConstructorArg(), MAXIMUM_SNAPSHOT_COUNT); + } + + private final TimeValue expireAfter; + private final Integer minimumSnapshotCount; + private final Integer maximumSnapshotCount; + + public SnapshotRetentionConfiguration(@Nullable TimeValue expireAfter, + @Nullable Integer minimumSnapshotCount, + @Nullable Integer maximumSnapshotCount) { + this.expireAfter = expireAfter; + this.minimumSnapshotCount = minimumSnapshotCount; + this.maximumSnapshotCount = maximumSnapshotCount; + if (this.minimumSnapshotCount != null && this.minimumSnapshotCount < 1) { + throw new IllegalArgumentException("minimum snapshot count must be at least 1, but was: " + this.minimumSnapshotCount); + } + if (this.maximumSnapshotCount != null && this.maximumSnapshotCount < 1) { + throw new IllegalArgumentException("maximum snapshot count must be at least 1, but was: " + this.maximumSnapshotCount); + } + if ((maximumSnapshotCount != null && minimumSnapshotCount != null) && this.minimumSnapshotCount > this.maximumSnapshotCount) { + throw new IllegalArgumentException("minimum snapshot count " + this.minimumSnapshotCount + + " cannot be larger than maximum snapshot count " + this.maximumSnapshotCount); + } + } + + public static SnapshotRetentionConfiguration parse(XContentParser parser, String name) { + return PARSER.apply(parser, null); + } + + public TimeValue getExpireAfter() { + return this.expireAfter; + } + + public Integer getMinimumSnapshotCount() { + return this.minimumSnapshotCount; + } + + public Integer getMaximumSnapshotCount() { + return this.maximumSnapshotCount; + } + + @Override + public XContentBuilder toXContent(XContentBuilder builder, Params params) throws IOException { + builder.startObject(); + if (expireAfter != null) { + builder.field(EXPIRE_AFTER.getPreferredName(), expireAfter.getStringRep()); + } + if (minimumSnapshotCount != null) { + builder.field(MINIMUM_SNAPSHOT_COUNT.getPreferredName(), minimumSnapshotCount); + } + if (maximumSnapshotCount != null) { + builder.field(MAXIMUM_SNAPSHOT_COUNT.getPreferredName(), maximumSnapshotCount); + } + builder.endObject(); + return builder; + } + + @Override + public int hashCode() { + return Objects.hash(expireAfter, minimumSnapshotCount, maximumSnapshotCount); + } + + @Override + public boolean equals(Object obj) { + if (obj == null) { + return false; + } + if (obj.getClass() != getClass()) { + return false; + } + SnapshotRetentionConfiguration other = (SnapshotRetentionConfiguration) obj; + return Objects.equals(this.expireAfter, other.expireAfter) && + Objects.equals(minimumSnapshotCount, other.minimumSnapshotCount) && + Objects.equals(maximumSnapshotCount, other.maximumSnapshotCount); + } + + @Override + public String toString() { + return Strings.toString(this); + } +} diff --git a/client/rest-high-level/src/test/java/org/elasticsearch/client/documentation/ILMDocumentationIT.java b/client/rest-high-level/src/test/java/org/elasticsearch/client/documentation/ILMDocumentationIT.java index 4a4a40dec57b5..5d367430afbea 100644 --- a/client/rest-high-level/src/test/java/org/elasticsearch/client/documentation/ILMDocumentationIT.java +++ b/client/rest-high-level/src/test/java/org/elasticsearch/client/documentation/ILMDocumentationIT.java @@ -59,10 +59,14 @@ import org.elasticsearch.client.slm.ExecuteSnapshotLifecyclePolicyResponse; import org.elasticsearch.client.slm.GetSnapshotLifecyclePolicyRequest; import org.elasticsearch.client.slm.GetSnapshotLifecyclePolicyResponse; +import org.elasticsearch.client.slm.GetSnapshotLifecycleStatsRequest; +import org.elasticsearch.client.slm.GetSnapshotLifecycleStatsResponse; import org.elasticsearch.client.slm.PutSnapshotLifecyclePolicyRequest; import org.elasticsearch.client.slm.SnapshotInvocationRecord; import org.elasticsearch.client.slm.SnapshotLifecyclePolicy; import org.elasticsearch.client.slm.SnapshotLifecyclePolicyMetadata; +import org.elasticsearch.client.slm.SnapshotLifecycleStats; +import org.elasticsearch.client.slm.SnapshotRetentionConfiguration; import org.elasticsearch.cluster.metadata.IndexMetaData; import org.elasticsearch.common.Strings; import org.elasticsearch.common.collect.ImmutableOpenMap; @@ -88,6 +92,7 @@ import java.util.concurrent.TimeUnit; import static org.hamcrest.Matchers.equalTo; +import static org.hamcrest.Matchers.greaterThanOrEqualTo; public class ILMDocumentationIT extends ESRestHighLevelClientTestCase { @@ -773,8 +778,11 @@ public void testAddSnapshotLifecyclePolicy() throws Exception { // tag::slm-put-snapshot-lifecycle-policy Map config = new HashMap<>(); config.put("indices", Collections.singletonList("idx")); + SnapshotRetentionConfiguration retention = + new SnapshotRetentionConfiguration(TimeValue.timeValueDays(30), 2, 10); SnapshotLifecyclePolicy policy = new SnapshotLifecyclePolicy( - "policy_id", "name", "1 2 3 * * ?", "my_repository", config); + "policy_id", "name", "1 2 3 * * ?", + "my_repository", config, retention); PutSnapshotLifecyclePolicyRequest request = new PutSnapshotLifecyclePolicyRequest(policy); // end::slm-put-snapshot-lifecycle-policy @@ -933,6 +941,22 @@ public void onFailure(Exception e) { // end::slm-execute-snapshot-lifecycle-policy-execute-async latch.await(5, TimeUnit.SECONDS); + // tag::slm-get-snapshot-lifecycle-stats + GetSnapshotLifecycleStatsRequest getStatsRequest = + new GetSnapshotLifecycleStatsRequest(); + // end::slm-get-snapshot-lifecycle-stats + + // tag::slm-get-snapshot-lifecycle-stats-execute + GetSnapshotLifecycleStatsResponse statsResp = client.indexLifecycle() + .getSnapshotLifecycleStats(getStatsRequest, RequestOptions.DEFAULT); + SnapshotLifecycleStats stats = statsResp.getStats(); + SnapshotLifecycleStats.SnapshotPolicyStats policyStats = + stats.getMetrics().get("policy_id"); + // end::slm-get-snapshot-lifecycle-stats-execute + assertThat( + statsResp.getStats().getMetrics().get("policy_id").getSnapshotsTaken(), + greaterThanOrEqualTo(1L)); + //////// DELETE // tag::slm-delete-snapshot-lifecycle-policy DeleteSnapshotLifecyclePolicyRequest deleteRequest = diff --git a/docs/java-rest/high-level/ilm/get_snapshot_lifecycle_stats.asciidoc b/docs/java-rest/high-level/ilm/get_snapshot_lifecycle_stats.asciidoc new file mode 100644 index 0000000000000..4f42b92ac64a7 --- /dev/null +++ b/docs/java-rest/high-level/ilm/get_snapshot_lifecycle_stats.asciidoc @@ -0,0 +1,35 @@ +-- +:api: slm-get-snapshot-lifecycle-stats +:request: GetSnapshotLifecycleStatsRequest +:response: GetSnapshotLifecycleStatsResponse +-- + +[id="{upid}-{api}"] +=== Get Snapshot Lifecycle Stats API + + +[id="{upid}-{api}-request"] +==== Request + +The Get Snapshot Lifecycle Stats API allows you to retrieve statistics about snapshots taken or +deleted, as well as retention runs by the snapshot lifecycle service. + +["source","java",subs="attributes,callouts,macros"] +-------------------------------------------------- +include-tagged::{doc-tests-file}[{api}-request] +-------------------------------------------------- + +[id="{upid}-{api}-response"] +==== Response + +The returned +{response}+ contains global statistics as well as a map of `SnapshotPolicyStats`, +accessible by the id of the policy, which contains statistics about each policy. + +["source","java",subs="attributes,callouts,macros"] +-------------------------------------------------- +include-tagged::{doc-tests-file}[{api}-response] +-------------------------------------------------- + +include::../execution.asciidoc[] + + diff --git a/docs/reference/ilm/apis/slm-api.asciidoc b/docs/reference/ilm/apis/slm-api.asciidoc index 0466924e3dee2..6d4c31f9ef4ac 100644 --- a/docs/reference/ilm/apis/slm-api.asciidoc +++ b/docs/reference/ilm/apis/slm-api.asciidoc @@ -59,7 +59,8 @@ PUT /_slm/policy/daily-snapshots "indices": ["data-*", "important"], <5> "ignore_unavailable": false, "include_global_state": false - } + }, + "retention": {} } -------------------------------------------------- // CONSOLE @@ -136,7 +137,14 @@ The output looks similar to the following: "indices": ["data-*", "important"], "ignore_unavailable": false, "include_global_state": false - } + }, + "retention": {} + }, + "stats": { + "snapshots_taken": 0, + "snapshots_failed": 0, + "snapshots_deleted": 0, + "snapshot_deletion_failures": 0 }, "next_execution": "2019-04-24T01:30:00.000Z", <3> "next_execution_millis": 1556048160000 @@ -221,8 +229,15 @@ Which, in this case shows an error because the index did not exist: "indices": ["data-*", "important"], "ignore_unavailable": false, "include_global_state": false - } + }, + "retention": {} }, + "stats": { + "snapshots_taken": 0, + "snapshots_failed": 1, + "snapshots_deleted": 0, + "snapshot_deletion_failures": 0 + } "last_failure": { <1> "snapshot_name": "daily-snap-2019.04.02-lohisb5ith2n8hxacaq3mw", "time_string": "2019-04-02T01:30:00.000Z", @@ -305,7 +320,14 @@ Which now includes the successful snapshot information: "indices": ["data-*", "important"], "ignore_unavailable": true, "include_global_state": false - } + }, + "retention": {} + }, + "stats": { + "snapshots_taken": 1, + "snapshots_failed": 1, + "snapshots_deleted": 0, + "snapshot_deletion_failures": 0 }, "last_success": { <2> "snapshot_name": "daily-snap-2019.04.24-tmtnyjtrsxkhbrrdcgg18a", @@ -330,6 +352,47 @@ Which now includes the successful snapshot information: It is a good idea to test policies using the execute API to ensure they work. +[[slm-get-stats]] +=== Get Snapshot Lifecycle Stats API + +SLM stores statistics on a global and per-policy level about actions taken. These stats can be +retrieved by using the following API: + +==== Example + +[source,js] +-------------------------------------------------- +GET /_slm/stats +-------------------------------------------------- +// CONSOLE +// TEST[continued] + +Which returns a response similar to: + +[source,js] +-------------------------------------------------- +{ + "retention_runs": 13, + "retention_failed": 0, + "retention_timed_out": 0, + "retention_deletion_time": "1.4s", + "retention_deletion_time_millis": 1404, + "policy_metrics": { + "daily-snapshots": { + "snapshots_taken": 1, + "snapshots_failed": 1, + "snapshots_deleted": 0, + "snapshot_deletion_failures": 0 + } + }, + "total_snapshots_taken": 1, + "total_snapshots_failed": 1, + "total_snapshots_deleted": 0, + "total_snapshot_deletion_failures": 0 +} +-------------------------------------------------- +// TESTRESPONSE[s/runs": 13/runs": $body.retention_runs/ s/_failed": 0/_failed": $body.retention_failed/ s/_timed_out": 0/_timed_out": $body.retention_timed_out/ s/"1.4s"/$body.retention_deletion_time/ s/1404/$body.retention_deletion_time_millis/] + [[slm-api-delete]] === Delete Snapshot Lifecycle Policy API diff --git a/docs/reference/ilm/getting-started-slm.asciidoc b/docs/reference/ilm/getting-started-slm.asciidoc index baef2021f7796..5d6e7a9fea4d5 100644 --- a/docs/reference/ilm/getting-started-slm.asciidoc +++ b/docs/reference/ilm/getting-started-slm.asciidoc @@ -95,7 +95,8 @@ PUT /_slm/policy/nightly-snapshots "repository": "my_repository", <3> "config": { <4> "indices": ["*"] <5> - } + }, + "retention": {} } -------------------------------------------------- // CONSOLE @@ -171,7 +172,8 @@ next time the policy will be executed. "repository": "my_repository", "config": { "indices": ["*"], - } + }, + "retention": {} }, "last_success": { <1> "snapshot_name": "nightly-snap-2019.04.24-tmtnyjtrsxkhbrrdcgg18a", <2> diff --git a/libs/core/src/main/java/org/elasticsearch/common/unit/TimeValue.java b/libs/core/src/main/java/org/elasticsearch/common/unit/TimeValue.java index c208e7d795391..edca86637e1b5 100644 --- a/libs/core/src/main/java/org/elasticsearch/common/unit/TimeValue.java +++ b/libs/core/src/main/java/org/elasticsearch/common/unit/TimeValue.java @@ -71,6 +71,14 @@ public static TimeValue timeValueHours(long hours) { return new TimeValue(hours, TimeUnit.HOURS); } + public static TimeValue timeValueDays(long days) { + // 106751.9 days is Long.MAX_VALUE nanoseconds, so we cannot store 106752 days + if (days > 106751) { + throw new IllegalArgumentException("time value cannot store values greater than 106751 days"); + } + return new TimeValue(days, TimeUnit.DAYS); + } + /** * @return the unit used for the this time value, see {@link #duration()} */ diff --git a/server/src/main/java/org/elasticsearch/cluster/RepositoryCleanupInProgress.java b/server/src/main/java/org/elasticsearch/cluster/RepositoryCleanupInProgress.java index a8cb897f0d369..e7c8e995dd61c 100644 --- a/server/src/main/java/org/elasticsearch/cluster/RepositoryCleanupInProgress.java +++ b/server/src/main/java/org/elasticsearch/cluster/RepositoryCleanupInProgress.java @@ -101,7 +101,7 @@ private Entry(StreamInput in) throws IOException { repositoryStateId = in.readLong(); } - private Entry(String repository, long repositoryStateId) { + public Entry(String repository, long repositoryStateId) { this.repository = repository; this.repositoryStateId = repositoryStateId; } diff --git a/server/src/main/java/org/elasticsearch/cluster/SnapshotDeletionsInProgress.java b/server/src/main/java/org/elasticsearch/cluster/SnapshotDeletionsInProgress.java index 8e702fbdceea8..2ac12d3e93922 100644 --- a/server/src/main/java/org/elasticsearch/cluster/SnapshotDeletionsInProgress.java +++ b/server/src/main/java/org/elasticsearch/cluster/SnapshotDeletionsInProgress.java @@ -44,7 +44,7 @@ public class SnapshotDeletionsInProgress extends AbstractNamedDiffable i // the list of snapshot deletion request entries private final List entries; - private SnapshotDeletionsInProgress(List entries) { + public SnapshotDeletionsInProgress(List entries) { this.entries = Collections.unmodifiableList(entries); } diff --git a/x-pack/plugin/core/src/main/java/org/elasticsearch/xpack/core/XPackClientPlugin.java b/x-pack/plugin/core/src/main/java/org/elasticsearch/xpack/core/XPackClientPlugin.java index fbb919490d0ae..da9fdd051e6a9 100644 --- a/x-pack/plugin/core/src/main/java/org/elasticsearch/xpack/core/XPackClientPlugin.java +++ b/x-pack/plugin/core/src/main/java/org/elasticsearch/xpack/core/XPackClientPlugin.java @@ -199,6 +199,7 @@ import org.elasticsearch.xpack.core.slm.action.DeleteSnapshotLifecycleAction; import org.elasticsearch.xpack.core.slm.action.ExecuteSnapshotLifecycleAction; import org.elasticsearch.xpack.core.slm.action.GetSnapshotLifecycleAction; +import org.elasticsearch.xpack.core.slm.action.GetSnapshotLifecycleStatsAction; import org.elasticsearch.xpack.core.slm.action.PutSnapshotLifecycleAction; import org.elasticsearch.xpack.core.spatial.SpatialFeatureSetUsage; import org.elasticsearch.xpack.core.sql.SqlFeatureSetUsage; @@ -382,6 +383,7 @@ public List> getClientActions() { GetSnapshotLifecycleAction.INSTANCE, DeleteSnapshotLifecycleAction.INSTANCE, ExecuteSnapshotLifecycleAction.INSTANCE, + GetSnapshotLifecycleStatsAction.INSTANCE, // Freeze FreezeIndexAction.INSTANCE, // Data Frame diff --git a/x-pack/plugin/core/src/main/java/org/elasticsearch/xpack/core/ilm/LifecycleSettings.java b/x-pack/plugin/core/src/main/java/org/elasticsearch/xpack/core/ilm/LifecycleSettings.java index b8d33112a336f..3e2623c35250c 100644 --- a/x-pack/plugin/core/src/main/java/org/elasticsearch/xpack/core/ilm/LifecycleSettings.java +++ b/x-pack/plugin/core/src/main/java/org/elasticsearch/xpack/core/ilm/LifecycleSettings.java @@ -5,8 +5,10 @@ */ package org.elasticsearch.xpack.core.ilm; +import org.elasticsearch.common.Strings; import org.elasticsearch.common.settings.Setting; import org.elasticsearch.common.unit.TimeValue; +import org.elasticsearch.xpack.core.scheduler.CronSchedule; /** * Class encapsulating settings related to Index Lifecycle Management X-Pack Plugin @@ -17,6 +19,9 @@ public class LifecycleSettings { public static final String LIFECYCLE_INDEXING_COMPLETE = "index.lifecycle.indexing_complete"; public static final String SLM_HISTORY_INDEX_ENABLED = "slm.history_index_enabled"; + public static final String SLM_RETENTION_SCHEDULE = "slm.retention_schedule"; + public static final String SLM_RETENTION_DURATION = "slm.retention_duration"; + public static final Setting LIFECYCLE_POLL_INTERVAL_SETTING = Setting.timeSetting(LIFECYCLE_POLL_INTERVAL, TimeValue.timeValueMinutes(10), TimeValue.timeValueSeconds(1), Setting.Property.Dynamic, Setting.Property.NodeScope); @@ -27,4 +32,17 @@ public class LifecycleSettings { public static final Setting SLM_HISTORY_INDEX_ENABLED_SETTING = Setting.boolSetting(SLM_HISTORY_INDEX_ENABLED, true, Setting.Property.NodeScope); + public static final Setting SLM_RETENTION_SCHEDULE_SETTING = Setting.simpleString(SLM_RETENTION_SCHEDULE, str -> { + try { + if (Strings.hasText(str)) { + // Test that the setting is a valid cron syntax + new CronSchedule(str); + } + } catch (Exception e) { + throw new IllegalArgumentException("invalid cron expression [" + str + "] for SLM retention schedule [" + + SLM_RETENTION_SCHEDULE + "]", e); + } + }, Setting.Property.Dynamic, Setting.Property.NodeScope); + public static final Setting SLM_RETENTION_DURATION_SETTING = Setting.timeSetting(SLM_RETENTION_DURATION, + TimeValue.timeValueHours(1), TimeValue.timeValueMillis(500), Setting.Property.Dynamic, Setting.Property.NodeScope); } diff --git a/x-pack/plugin/core/src/main/java/org/elasticsearch/xpack/core/slm/SnapshotLifecycleMetadata.java b/x-pack/plugin/core/src/main/java/org/elasticsearch/xpack/core/slm/SnapshotLifecycleMetadata.java index 2786fac735b43..84e8f288564da 100644 --- a/x-pack/plugin/core/src/main/java/org/elasticsearch/xpack/core/slm/SnapshotLifecycleMetadata.java +++ b/x-pack/plugin/core/src/main/java/org/elasticsearch/xpack/core/slm/SnapshotLifecycleMetadata.java @@ -19,6 +19,7 @@ import org.elasticsearch.common.xcontent.ConstructingObjectParser; import org.elasticsearch.common.xcontent.XContentBuilder; import org.elasticsearch.xpack.core.ilm.OperationMode; +import org.elasticsearch.xpack.slm.SnapshotLifecycleStats; import java.io.IOException; import java.util.Collections; @@ -38,36 +39,52 @@ public class SnapshotLifecycleMetadata implements MetaData.Custom { public static final String TYPE = "snapshot_lifecycle"; - public static final ParseField OPERATION_MODE_FIELD = new ParseField("operation_mode"); - public static final ParseField POLICIES_FIELD = new ParseField("policies"); - public static final SnapshotLifecycleMetadata EMPTY = new SnapshotLifecycleMetadata(Collections.emptyMap(), OperationMode.RUNNING); + private static final ParseField OPERATION_MODE_FIELD = new ParseField("operation_mode"); + private static final ParseField POLICIES_FIELD = new ParseField("policies"); + private static final ParseField STATS_FIELD = new ParseField("stats"); + + public static final SnapshotLifecycleMetadata EMPTY = + new SnapshotLifecycleMetadata(Collections.emptyMap(), OperationMode.RUNNING, new SnapshotLifecycleStats()); @SuppressWarnings("unchecked") public static final ConstructingObjectParser PARSER = new ConstructingObjectParser<>(TYPE, a -> new SnapshotLifecycleMetadata( ((List) a[0]).stream() .collect(Collectors.toMap(m -> m.getPolicy().getId(), Function.identity())), - OperationMode.valueOf((String) a[1]))); + OperationMode.valueOf((String) a[1]), + (SnapshotLifecycleStats) a[2])); static { PARSER.declareNamedObjects(ConstructingObjectParser.constructorArg(), (p, c, n) -> SnapshotLifecyclePolicyMetadata.parse(p, n), v -> { throw new IllegalArgumentException("ordered " + POLICIES_FIELD.getPreferredName() + " are not supported"); }, POLICIES_FIELD); + PARSER.declareString(ConstructingObjectParser.constructorArg(), OPERATION_MODE_FIELD); + PARSER.declareObject(ConstructingObjectParser.constructorArg(), (v, o) -> SnapshotLifecycleStats.parse(v), STATS_FIELD); } private final Map snapshotConfigurations; private final OperationMode operationMode; + private final SnapshotLifecycleStats slmStats; - public SnapshotLifecycleMetadata(Map snapshotConfigurations, OperationMode operationMode) { + public SnapshotLifecycleMetadata(Map snapshotConfigurations, + OperationMode operationMode, + SnapshotLifecycleStats slmStats) { this.snapshotConfigurations = new HashMap<>(snapshotConfigurations); this.operationMode = operationMode; + this.slmStats = slmStats; } public SnapshotLifecycleMetadata(StreamInput in) throws IOException { this.snapshotConfigurations = in.readMap(StreamInput::readString, SnapshotLifecyclePolicyMetadata::new); this.operationMode = in.readEnum(OperationMode.class); + // TODO: version qualify this with the correct version (7.5) once available + if (in.getVersion().onOrAfter(Version.V_8_0_0)) { + this.slmStats = new SnapshotLifecycleStats(in); + } else { + this.slmStats = new SnapshotLifecycleStats(); + } } public Map getSnapshotConfigurations() { @@ -78,6 +95,10 @@ public OperationMode getOperationMode() { return operationMode; } + public SnapshotLifecycleStats getStats() { + return this.slmStats; + } + @Override public EnumSet context() { return MetaData.ALL_CONTEXTS; @@ -102,12 +123,17 @@ public Version getMinimalSupportedVersion() { public void writeTo(StreamOutput out) throws IOException { out.writeMap(this.snapshotConfigurations, StreamOutput::writeString, (out1, value) -> value.writeTo(out1)); out.writeEnum(this.operationMode); + // TODO: version qualify this with the correct version (7.5) once available + if (out.getVersion().onOrAfter(Version.V_8_0_0)) { + this.slmStats.writeTo(out); + } } @Override public XContentBuilder toXContent(XContentBuilder builder, Params params) throws IOException { builder.field(POLICIES_FIELD.getPreferredName(), this.snapshotConfigurations); builder.field(OPERATION_MODE_FIELD.getPreferredName(), operationMode); + builder.field(STATS_FIELD.getPreferredName(), this.slmStats); return builder; } @@ -118,7 +144,7 @@ public String toString() { @Override public int hashCode() { - return Objects.hash(this.snapshotConfigurations, this.operationMode); + return Objects.hash(this.snapshotConfigurations, this.operationMode, this.slmStats); } @Override @@ -131,18 +157,21 @@ public boolean equals(Object obj) { } SnapshotLifecycleMetadata other = (SnapshotLifecycleMetadata) obj; return this.snapshotConfigurations.equals(other.snapshotConfigurations) && - this.operationMode.equals(other.operationMode); + this.operationMode.equals(other.operationMode) && + this.slmStats.equals(other.slmStats); } public static class SnapshotLifecycleMetadataDiff implements NamedDiff { final Diff> lifecycles; final OperationMode operationMode; + final SnapshotLifecycleStats slmStats; SnapshotLifecycleMetadataDiff(SnapshotLifecycleMetadata before, SnapshotLifecycleMetadata after) { this.lifecycles = DiffableUtils.diff(before.snapshotConfigurations, after.snapshotConfigurations, DiffableUtils.getStringKeySerializer()); this.operationMode = after.operationMode; + this.slmStats = after.slmStats; } public SnapshotLifecycleMetadataDiff(StreamInput in) throws IOException { @@ -150,13 +179,19 @@ public SnapshotLifecycleMetadataDiff(StreamInput in) throws IOException { SnapshotLifecyclePolicyMetadata::new, SnapshotLifecycleMetadataDiff::readLifecyclePolicyDiffFrom); this.operationMode = in.readEnum(OperationMode.class); + // TODO: version qualify this with the correct version (7.5) once available + if (in.getVersion().onOrAfter(Version.V_8_0_0)) { + this.slmStats = new SnapshotLifecycleStats(in); + } else { + this.slmStats = new SnapshotLifecycleStats(); + } } @Override public MetaData.Custom apply(MetaData.Custom part) { TreeMap newLifecycles = new TreeMap<>( lifecycles.apply(((SnapshotLifecycleMetadata) part).snapshotConfigurations)); - return new SnapshotLifecycleMetadata(newLifecycles, this.operationMode); + return new SnapshotLifecycleMetadata(newLifecycles, this.operationMode, this.slmStats); } @Override @@ -168,6 +203,10 @@ public String getWriteableName() { public void writeTo(StreamOutput out) throws IOException { lifecycles.writeTo(out); out.writeEnum(this.operationMode); + // TODO: version qualify this with the correct version (7.5) once available + if (out.getVersion().onOrAfter(Version.V_8_0_0)) { + this.slmStats.writeTo(out); + } } static Diff readLifecyclePolicyDiffFrom(StreamInput in) throws IOException { diff --git a/x-pack/plugin/core/src/main/java/org/elasticsearch/xpack/core/slm/SnapshotLifecyclePolicy.java b/x-pack/plugin/core/src/main/java/org/elasticsearch/xpack/core/slm/SnapshotLifecyclePolicy.java index ddb05ad1df142..802f2e26452a2 100644 --- a/x-pack/plugin/core/src/main/java/org/elasticsearch/xpack/core/slm/SnapshotLifecyclePolicy.java +++ b/x-pack/plugin/core/src/main/java/org/elasticsearch/xpack/core/slm/SnapshotLifecyclePolicy.java @@ -54,11 +54,13 @@ public class SnapshotLifecyclePolicy extends AbstractDiffable configuration; + private final SnapshotRetentionConfiguration retentionPolicy; private static final ParseField NAME = new ParseField("name"); private static final ParseField SCHEDULE = new ParseField("schedule"); private static final ParseField REPOSITORY = new ParseField("repository"); private static final ParseField CONFIG = new ParseField("config"); + private static final ParseField RETENTION = new ParseField("retention"); private static final IndexNameExpressionResolver.DateMathExpressionResolver DATE_MATH_RESOLVER = new IndexNameExpressionResolver.DateMathExpressionResolver(); private static final String METADATA_FIELD_NAME = "metadata"; @@ -71,7 +73,8 @@ public class SnapshotLifecyclePolicy extends AbstractDiffable config = (Map) a[3]; - return new SnapshotLifecyclePolicy(id, name, schedule, repo, config); + SnapshotRetentionConfiguration retention = (SnapshotRetentionConfiguration) a[4]; + return new SnapshotLifecyclePolicy(id, name, schedule, repo, config, retention); }); static { @@ -79,15 +82,18 @@ public class SnapshotLifecyclePolicy extends AbstractDiffable p.map(), CONFIG); + PARSER.declareObject(ConstructingObjectParser.optionalConstructorArg(), SnapshotRetentionConfiguration::parse, RETENTION); } public SnapshotLifecyclePolicy(final String id, final String name, final String schedule, - final String repository, @Nullable Map configuration) { + final String repository, @Nullable final Map configuration, + @Nullable final SnapshotRetentionConfiguration retentionPolicy) { this.id = Objects.requireNonNull(id, "policy id is required"); this.name = Objects.requireNonNull(name, "policy snapshot name is required"); this.schedule = Objects.requireNonNull(schedule, "policy schedule is required"); this.repository = Objects.requireNonNull(repository, "policy snapshot repository is required"); this.configuration = configuration; + this.retentionPolicy = retentionPolicy; } public SnapshotLifecyclePolicy(StreamInput in) throws IOException { @@ -96,6 +102,7 @@ public SnapshotLifecyclePolicy(StreamInput in) throws IOException { this.schedule = in.readString(); this.repository = in.readString(); this.configuration = in.readMap(); + this.retentionPolicy = in.readOptionalWriteable(SnapshotRetentionConfiguration::new); } public String getId() { @@ -119,6 +126,11 @@ public Map getConfig() { return this.configuration; } + @Nullable + public SnapshotRetentionConfiguration getRetentionPolicy() { + return this.retentionPolicy; + } + public long calculateNextExecution() { final Cron schedule = new Cron(this.schedule); return schedule.getNextValidTimeAfter(System.currentTimeMillis()); @@ -258,6 +270,7 @@ public void writeTo(StreamOutput out) throws IOException { out.writeString(this.schedule); out.writeString(this.repository); out.writeMap(this.configuration); + out.writeOptionalWriteable(this.retentionPolicy); } @Override @@ -269,13 +282,16 @@ public XContentBuilder toXContent(XContentBuilder builder, Params params) throws if (this.configuration != null) { builder.field(CONFIG.getPreferredName(), this.configuration); } + if (this.retentionPolicy != null) { + builder.field(RETENTION.getPreferredName(), this.retentionPolicy); + } builder.endObject(); return builder; } @Override public int hashCode() { - return Objects.hash(id, name, schedule, repository, configuration); + return Objects.hash(id, name, schedule, repository, configuration, retentionPolicy); } @Override @@ -292,7 +308,8 @@ public boolean equals(Object obj) { Objects.equals(name, other.name) && Objects.equals(schedule, other.schedule) && Objects.equals(repository, other.repository) && - Objects.equals(configuration, other.configuration); + Objects.equals(configuration, other.configuration) && + Objects.equals(retentionPolicy, other.retentionPolicy); } @Override diff --git a/x-pack/plugin/core/src/main/java/org/elasticsearch/xpack/core/slm/SnapshotLifecyclePolicyItem.java b/x-pack/plugin/core/src/main/java/org/elasticsearch/xpack/core/slm/SnapshotLifecyclePolicyItem.java index 25832ec1cfbed..70f108957d731 100644 --- a/x-pack/plugin/core/src/main/java/org/elasticsearch/xpack/core/slm/SnapshotLifecyclePolicyItem.java +++ b/x-pack/plugin/core/src/main/java/org/elasticsearch/xpack/core/slm/SnapshotLifecyclePolicyItem.java @@ -6,6 +6,7 @@ package org.elasticsearch.xpack.core.slm; +import org.elasticsearch.Version; import org.elasticsearch.cluster.SnapshotsInProgress; import org.elasticsearch.common.Nullable; import org.elasticsearch.common.ParseField; @@ -17,6 +18,7 @@ import org.elasticsearch.common.xcontent.ToXContentObject; import org.elasticsearch.common.xcontent.XContentBuilder; import org.elasticsearch.snapshots.SnapshotId; +import org.elasticsearch.xpack.slm.SnapshotLifecycleStats; import java.io.IOException; import java.util.Objects; @@ -29,12 +31,14 @@ public class SnapshotLifecyclePolicyItem implements ToXContentFragment, Writeable { private static final ParseField SNAPSHOT_IN_PROGRESS = new ParseField("in_progress"); + private static final ParseField POLICY_STATS = new ParseField("stats"); private final SnapshotLifecyclePolicy policy; private final long version; private final long modifiedDate; @Nullable private final SnapshotInProgress snapshotInProgress; + private final SnapshotLifecycleStats.SnapshotPolicyStats policyStats; @Nullable private final SnapshotInvocationRecord lastSuccess; @@ -42,13 +46,15 @@ public class SnapshotLifecyclePolicyItem implements ToXContentFragment, Writeabl @Nullable private final SnapshotInvocationRecord lastFailure; public SnapshotLifecyclePolicyItem(SnapshotLifecyclePolicyMetadata policyMetadata, - @Nullable SnapshotInProgress snapshotInProgress) { + @Nullable SnapshotInProgress snapshotInProgress, + @Nullable SnapshotLifecycleStats.SnapshotPolicyStats policyStats) { this.policy = policyMetadata.getPolicy(); this.version = policyMetadata.getVersion(); this.modifiedDate = policyMetadata.getModifiedDate(); this.lastSuccess = policyMetadata.getLastSuccess(); this.lastFailure = policyMetadata.getLastFailure(); this.snapshotInProgress = snapshotInProgress; + this.policyStats = policyStats == null ? new SnapshotLifecycleStats.SnapshotPolicyStats(policy.getId()) : policyStats; } public SnapshotLifecyclePolicyItem(StreamInput in) throws IOException { @@ -58,19 +64,26 @@ public SnapshotLifecyclePolicyItem(StreamInput in) throws IOException { this.lastSuccess = in.readOptionalWriteable(SnapshotInvocationRecord::new); this.lastFailure = in.readOptionalWriteable(SnapshotInvocationRecord::new); this.snapshotInProgress = in.readOptionalWriteable(SnapshotInProgress::new); + if (in.getVersion().onOrAfter(Version.V_8_0_0)) { + this.policyStats = new SnapshotLifecycleStats.SnapshotPolicyStats(in); + } else { + this.policyStats = new SnapshotLifecycleStats.SnapshotPolicyStats(this.policy.getId()); + } } // For testing SnapshotLifecyclePolicyItem(SnapshotLifecyclePolicy policy, long version, long modifiedDate, SnapshotInvocationRecord lastSuccess, SnapshotInvocationRecord lastFailure, - @Nullable SnapshotInProgress snapshotInProgress) { + @Nullable SnapshotInProgress snapshotInProgress, + SnapshotLifecycleStats.SnapshotPolicyStats policyStats) { this.policy = policy; this.version = version; this.modifiedDate = modifiedDate; this.lastSuccess = lastSuccess; this.lastFailure = lastFailure; this.snapshotInProgress = snapshotInProgress; + this.policyStats = policyStats; } public SnapshotLifecyclePolicy getPolicy() { return policy; @@ -97,6 +110,10 @@ public SnapshotInProgress getSnapshotInProgress() { return this.snapshotInProgress; } + public SnapshotLifecycleStats.SnapshotPolicyStats getPolicyStats() { + return this.policyStats; + } + @Override public void writeTo(StreamOutput out) throws IOException { policy.writeTo(out); @@ -105,11 +122,14 @@ public void writeTo(StreamOutput out) throws IOException { out.writeOptionalWriteable(lastSuccess); out.writeOptionalWriteable(lastFailure); out.writeOptionalWriteable(snapshotInProgress); + if (out.getVersion().onOrAfter(Version.V_8_0_0)) { + this.policyStats.writeTo(out); + } } @Override public int hashCode() { - return Objects.hash(policy, version, modifiedDate, lastSuccess, lastFailure); + return Objects.hash(policy, version, modifiedDate, lastSuccess, lastFailure, policyStats); } @Override @@ -126,7 +146,8 @@ public boolean equals(Object obj) { modifiedDate == other.modifiedDate && Objects.equals(lastSuccess, other.lastSuccess) && Objects.equals(lastFailure, other.lastFailure) && - Objects.equals(snapshotInProgress, other.snapshotInProgress); + Objects.equals(snapshotInProgress, other.snapshotInProgress) && + Objects.equals(policyStats, other.policyStats); } @Override @@ -147,6 +168,9 @@ public XContentBuilder toXContent(XContentBuilder builder, Params params) throws if (snapshotInProgress != null) { builder.field(SNAPSHOT_IN_PROGRESS.getPreferredName(), snapshotInProgress); } + builder.startObject(POLICY_STATS.getPreferredName()); + this.policyStats.toXContent(builder, params); + builder.endObject(); builder.endObject(); return builder; } @@ -187,6 +211,22 @@ public static SnapshotInProgress fromEntry(SnapshotsInProgress.Entry entry) { entry.state(), entry.startTime(), entry.failure()); } + public SnapshotId getSnapshotId() { + return snapshotId; + } + + public SnapshotsInProgress.State getState() { + return state; + } + + public long getStartTime() { + return startTime; + } + + public String getFailure() { + return failure; + } + @Override public void writeTo(StreamOutput out) throws IOException { this.snapshotId.writeTo(out); diff --git a/x-pack/plugin/core/src/main/java/org/elasticsearch/xpack/core/slm/SnapshotRetentionConfiguration.java b/x-pack/plugin/core/src/main/java/org/elasticsearch/xpack/core/slm/SnapshotRetentionConfiguration.java new file mode 100644 index 0000000000000..1d3d4bd7a82fc --- /dev/null +++ b/x-pack/plugin/core/src/main/java/org/elasticsearch/xpack/core/slm/SnapshotRetentionConfiguration.java @@ -0,0 +1,245 @@ +/* + * 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.core.slm; + +import org.apache.logging.log4j.LogManager; +import org.apache.logging.log4j.Logger; +import org.elasticsearch.common.Nullable; +import org.elasticsearch.common.ParseField; +import org.elasticsearch.common.Strings; +import org.elasticsearch.common.io.stream.StreamInput; +import org.elasticsearch.common.io.stream.StreamOutput; +import org.elasticsearch.common.io.stream.Writeable; +import org.elasticsearch.common.unit.TimeValue; +import org.elasticsearch.common.xcontent.ConstructingObjectParser; +import org.elasticsearch.common.xcontent.ToXContentObject; +import org.elasticsearch.common.xcontent.XContentBuilder; +import org.elasticsearch.common.xcontent.XContentParser; +import org.elasticsearch.snapshots.SnapshotInfo; + +import java.io.IOException; +import java.util.Comparator; +import java.util.List; +import java.util.Objects; +import java.util.Set; +import java.util.function.LongSupplier; +import java.util.function.Predicate; +import java.util.stream.Collectors; + +public class SnapshotRetentionConfiguration implements ToXContentObject, Writeable { + + public static final SnapshotRetentionConfiguration EMPTY = new SnapshotRetentionConfiguration(null, null, null); + + private static final ParseField EXPIRE_AFTER = new ParseField("expire_after"); + private static final ParseField MINIMUM_SNAPSHOT_COUNT = new ParseField("min_count"); + private static final ParseField MAXIMUM_SNAPSHOT_COUNT = new ParseField("max_count"); + private static final Logger logger = LogManager.getLogger(SnapshotRetentionConfiguration.class); + + private static final ConstructingObjectParser PARSER = + new ConstructingObjectParser<>("snapshot_retention", true, a -> { + TimeValue expireAfter = a[0] == null ? null : TimeValue.parseTimeValue((String) a[0], EXPIRE_AFTER.getPreferredName()); + Integer minCount = (Integer) a[1]; + Integer maxCount = (Integer) a[2]; + return new SnapshotRetentionConfiguration(expireAfter, minCount, maxCount); + }); + + static { + PARSER.declareString(ConstructingObjectParser.optionalConstructorArg(), EXPIRE_AFTER); + PARSER.declareInt(ConstructingObjectParser.optionalConstructorArg(), MINIMUM_SNAPSHOT_COUNT); + PARSER.declareInt(ConstructingObjectParser.optionalConstructorArg(), MAXIMUM_SNAPSHOT_COUNT); + } + + private final LongSupplier nowSupplier; + private final TimeValue expireAfter; + private final Integer minimumSnapshotCount; + private final Integer maximumSnapshotCount; + + SnapshotRetentionConfiguration(StreamInput in) throws IOException { + nowSupplier = System::currentTimeMillis; + this.expireAfter = in.readOptionalTimeValue(); + this.minimumSnapshotCount = in.readOptionalVInt(); + this.maximumSnapshotCount = in.readOptionalVInt(); + } + + public SnapshotRetentionConfiguration(@Nullable TimeValue expireAfter, + @Nullable Integer minimumSnapshotCount, + @Nullable Integer maximumSnapshotCount) { + this(System::currentTimeMillis, expireAfter, minimumSnapshotCount, maximumSnapshotCount); + } + + public SnapshotRetentionConfiguration(LongSupplier nowSupplier, + @Nullable TimeValue expireAfter, + @Nullable Integer minimumSnapshotCount, + @Nullable Integer maximumSnapshotCount) { + this.nowSupplier = nowSupplier; + this.expireAfter = expireAfter; + this.minimumSnapshotCount = minimumSnapshotCount; + this.maximumSnapshotCount = maximumSnapshotCount; + if (this.minimumSnapshotCount != null && this.minimumSnapshotCount < 1) { + throw new IllegalArgumentException("minimum snapshot count must be at least 1, but was: " + this.minimumSnapshotCount); + } + if (this.maximumSnapshotCount != null && this.maximumSnapshotCount < 1) { + throw new IllegalArgumentException("maximum snapshot count must be at least 1, but was: " + this.maximumSnapshotCount); + } + if ((maximumSnapshotCount != null && minimumSnapshotCount != null) && this.minimumSnapshotCount > this.maximumSnapshotCount) { + throw new IllegalArgumentException("minimum snapshot count " + this.minimumSnapshotCount + + " cannot be larger than maximum snapshot count " + this.maximumSnapshotCount); + } + } + + public static SnapshotRetentionConfiguration parse(XContentParser parser, String name) { + return PARSER.apply(parser, null); + } + + public TimeValue getExpireAfter() { + return this.expireAfter; + } + + public Integer getMinimumSnapshotCount() { + return this.minimumSnapshotCount; + } + + public Integer getMaximumSnapshotCount() { + return this.maximumSnapshotCount; + } + + /** + * Return a predicate by which a SnapshotInfo can be tested to see + * whether it should be deleted according to this retention policy. + * @param allSnapshots a list of all snapshot pertaining to this SLM policy and repository + */ + public Predicate getSnapshotDeletionPredicate(final List allSnapshots) { + final int snapCount = allSnapshots.size(); + List sortedSnapshots = allSnapshots.stream() + .sorted(Comparator.comparingLong(SnapshotInfo::startTime)) + .collect(Collectors.toList()); + + return si -> { + final String snapName = si.snapshotId().getName(); + + // First, enforce the maximum count, if the size is over the maximum number of + // snapshots, then allow the oldest N (where N is the number over the maximum snapshot + // count) snapshots to be eligible for deletion + if (this.maximumSnapshotCount != null) { + if (allSnapshots.size() > this.maximumSnapshotCount) { + int snapsToDelete = allSnapshots.size() - this.maximumSnapshotCount; + boolean eligible = sortedSnapshots.stream() + .limit(snapsToDelete) + .anyMatch(s -> s.equals(si)); + + if (eligible) { + logger.trace("[{}]: ELIGIBLE as it is one of the {} oldest snapshots with " + + "{} total snapshots, over the limit of {} maximum snapshots", + snapName, snapsToDelete, snapCount, this.maximumSnapshotCount); + return true; + } else { + logger.trace("[{}]: INELIGIBLE as it is not one of the {} oldest snapshots with " + + "{} total snapshots, over the limit of {} maximum snapshots", + snapName, snapsToDelete, snapCount, this.maximumSnapshotCount); + return false; + } + } + } + + // Next check the minimum count, since that is a blanket requirement regardless of time, + // if we haven't hit the minimum then we need to keep the snapshot regardless of + // expiration time + if (this.minimumSnapshotCount != null) { + if (allSnapshots.size() <= this.minimumSnapshotCount) { + logger.trace("[{}]: INELIGIBLE as there are {} snapshots and {} minimum snapshots needed", + snapName, snapCount, this.minimumSnapshotCount); + return false; + } + } + + // Finally, check the expiration time of the snapshot, if it is past, then it is + // eligible for deletion + if (this.expireAfter != null) { + TimeValue snapshotAge = new TimeValue(nowSupplier.getAsLong() - si.startTime()); + + if (this.minimumSnapshotCount != null) { + int eligibleForExpiration = snapCount - minimumSnapshotCount; + + // Only the oldest N snapshots are actually eligible, since if we went below this we + // would fall below the configured minimum number of snapshots to keep + Set snapsEligibleForExpiration = sortedSnapshots.stream() + .limit(eligibleForExpiration) + .collect(Collectors.toSet()); + + if (snapsEligibleForExpiration.contains(si) == false) { + // This snapshot is *not* one of the N oldest snapshots, so even if it were + // old enough, the other snapshots would be deleted before it + logger.trace("[{}]: INELIGIBLE as snapshot expiration would pass the " + + "minimum number of configured snapshots ({}) to keep, regardless of age", + snapName, this.minimumSnapshotCount); + return false; + } + } + + if (snapshotAge.compareTo(this.expireAfter) > 0) { + logger.trace("[{}]: ELIGIBLE as snapshot age of {} is older than {}", + snapName, snapshotAge.toHumanReadableString(3), this.expireAfter.toHumanReadableString(3)); + return true; + } else { + logger.trace("[{}]: INELIGIBLE as snapshot age of {} is newer than {}", + snapName, snapshotAge.toHumanReadableString(3), this.expireAfter.toHumanReadableString(3)); + return false; + } + } + // If nothing matched, the snapshot is not eligible for deletion + logger.trace("[{}]: INELIGIBLE as no retention predicates matched", snapName); + return false; + }; + } + + @Override + public void writeTo(StreamOutput out) throws IOException { + out.writeOptionalTimeValue(this.expireAfter); + out.writeOptionalVInt(this.minimumSnapshotCount); + out.writeOptionalVInt(this.maximumSnapshotCount); + } + + @Override + public XContentBuilder toXContent(XContentBuilder builder, Params params) throws IOException { + builder.startObject(); + if (expireAfter != null) { + builder.field(EXPIRE_AFTER.getPreferredName(), expireAfter.getStringRep()); + } + if (minimumSnapshotCount != null) { + builder.field(MINIMUM_SNAPSHOT_COUNT.getPreferredName(), minimumSnapshotCount); + } + if (maximumSnapshotCount != null) { + builder.field(MAXIMUM_SNAPSHOT_COUNT.getPreferredName(), maximumSnapshotCount); + } + builder.endObject(); + return builder; + } + + @Override + public int hashCode() { + return Objects.hash(expireAfter, minimumSnapshotCount, maximumSnapshotCount); + } + + @Override + public boolean equals(Object obj) { + if (obj == null) { + return false; + } + if (obj.getClass() != getClass()) { + return false; + } + SnapshotRetentionConfiguration other = (SnapshotRetentionConfiguration) obj; + return Objects.equals(this.expireAfter, other.expireAfter) && + Objects.equals(minimumSnapshotCount, other.minimumSnapshotCount) && + Objects.equals(maximumSnapshotCount, other.maximumSnapshotCount); + } + + @Override + public String toString() { + return Strings.toString(this); + } +} diff --git a/x-pack/plugin/core/src/main/java/org/elasticsearch/xpack/core/slm/action/GetSnapshotLifecycleAction.java b/x-pack/plugin/core/src/main/java/org/elasticsearch/xpack/core/slm/action/GetSnapshotLifecycleAction.java index 96d7c19f56f2c..5821f19fc9b21 100644 --- a/x-pack/plugin/core/src/main/java/org/elasticsearch/xpack/core/slm/action/GetSnapshotLifecycleAction.java +++ b/x-pack/plugin/core/src/main/java/org/elasticsearch/xpack/core/slm/action/GetSnapshotLifecycleAction.java @@ -94,6 +94,10 @@ public Response(StreamInput in) throws IOException { this.lifecycles = in.readList(SnapshotLifecyclePolicyItem::new); } + public List getPolicies() { + return this.lifecycles; + } + @Override public String toString() { return Strings.toString(this); diff --git a/x-pack/plugin/core/src/main/java/org/elasticsearch/xpack/core/slm/action/GetSnapshotLifecycleStatsAction.java b/x-pack/plugin/core/src/main/java/org/elasticsearch/xpack/core/slm/action/GetSnapshotLifecycleStatsAction.java new file mode 100644 index 0000000000000..ff37feb11642e --- /dev/null +++ b/x-pack/plugin/core/src/main/java/org/elasticsearch/xpack/core/slm/action/GetSnapshotLifecycleStatsAction.java @@ -0,0 +1,102 @@ +/* + * 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.core.slm.action; + +import org.elasticsearch.action.ActionRequestValidationException; +import org.elasticsearch.action.ActionResponse; +import org.elasticsearch.action.ActionType; +import org.elasticsearch.action.support.master.AcknowledgedRequest; +import org.elasticsearch.common.Strings; +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 org.elasticsearch.xpack.slm.SnapshotLifecycleStats; + +import java.io.IOException; +import java.util.Objects; + +/** + * This class represents the action of retriving the stats for snapshot lifecycle management. + * These are retrieved from the master's cluster state and contain numbers related to the count of + * snapshots taken or deleted, as well as retention runs and time spent deleting snapshots. + */ +public class GetSnapshotLifecycleStatsAction extends ActionType { + public static final GetSnapshotLifecycleStatsAction INSTANCE = new GetSnapshotLifecycleStatsAction(); + public static final String NAME = "cluster:admin/slm/stats"; + + protected GetSnapshotLifecycleStatsAction() { + super(NAME, GetSnapshotLifecycleStatsAction.Response::new); + } + + public static class Request extends AcknowledgedRequest { + + public Request() { } + + public Request(StreamInput in) throws IOException { + super(in); + } + + @Override + public ActionRequestValidationException validate() { + return null; + } + + @Override + public void writeTo(StreamOutput out) throws IOException { + super.writeTo(out); + } + } + + public static class Response extends ActionResponse implements ToXContentObject { + + private SnapshotLifecycleStats slmStats; + + public Response() { } + + public Response(SnapshotLifecycleStats slmStats) { + this.slmStats = slmStats; + } + + public Response(StreamInput in) throws IOException { + this.slmStats = new SnapshotLifecycleStats(in); + } + + @Override + public String toString() { + return Strings.toString(this); + } + + @Override + public XContentBuilder toXContent(XContentBuilder builder, Params params) throws IOException { + return this.slmStats.toXContent(builder, params); + } + + @Override + public void writeTo(StreamOutput out) throws IOException { + this.slmStats.writeTo(out); + } + + @Override + public int hashCode() { + return Objects.hash(this.slmStats); + } + + @Override + public boolean equals(Object obj) { + if (obj == null) { + return false; + } + if (obj.getClass() != getClass()) { + return false; + } + GetSnapshotLifecycleStatsAction.Response other = (GetSnapshotLifecycleStatsAction.Response) obj; + return this.slmStats.equals(other.slmStats); + } + } + +} diff --git a/x-pack/plugin/core/src/main/java/org/elasticsearch/xpack/core/slm/history/SnapshotHistoryItem.java b/x-pack/plugin/core/src/main/java/org/elasticsearch/xpack/core/slm/history/SnapshotHistoryItem.java index 8bd51e88704d0..380eaa8a65104 100644 --- a/x-pack/plugin/core/src/main/java/org/elasticsearch/xpack/core/slm/history/SnapshotHistoryItem.java +++ b/x-pack/plugin/core/src/main/java/org/elasticsearch/xpack/core/slm/history/SnapshotHistoryItem.java @@ -15,7 +15,6 @@ import org.elasticsearch.common.io.stream.StreamOutput; import org.elasticsearch.common.io.stream.Writeable; import org.elasticsearch.common.xcontent.ConstructingObjectParser; -import org.elasticsearch.common.xcontent.ToXContent; import org.elasticsearch.common.xcontent.ToXContentObject; import org.elasticsearch.common.xcontent.XContentBuilder; import org.elasticsearch.common.xcontent.XContentParser; @@ -40,7 +39,10 @@ public class SnapshotHistoryItem implements Writeable, ToXContentObject { static final ParseField SNAPSHOT_NAME = new ParseField("snapshot_name"); static final ParseField OPERATION = new ParseField("operation"); static final ParseField SUCCESS = new ParseField("success"); - private static final String CREATE_OPERATION = "CREATE"; + + public static final String CREATE_OPERATION = "CREATE"; + public static final String DELETE_OPERATION = "DELETE"; + protected final long timestamp; protected final String policyId; protected final String repository; @@ -98,25 +100,34 @@ public static SnapshotHistoryItem parse(XContentParser parser, String name) { this.errorDetails = errorDetails; } - public static SnapshotHistoryItem successRecord(long timestamp, SnapshotLifecyclePolicy policy, String snapshotName) { + public static SnapshotHistoryItem creationSuccessRecord(long timestamp, SnapshotLifecyclePolicy policy, String snapshotName) { return new SnapshotHistoryItem(timestamp, policy.getId(), policy.getRepository(), snapshotName, CREATE_OPERATION, true, policy.getConfig(), null); } - public static SnapshotHistoryItem failureRecord(long timeStamp, SnapshotLifecyclePolicy policy, String snapshotName, - Exception exception) throws IOException { - ToXContent.Params stacktraceParams = new ToXContent.MapParams(Collections.singletonMap(REST_EXCEPTION_SKIP_STACK_TRACE, "false")); - String exceptionString; - try (XContentBuilder causeXContentBuilder = JsonXContent.contentBuilder()) { - causeXContentBuilder.startObject(); - ElasticsearchException.generateThrowableXContent(causeXContentBuilder, stacktraceParams, exception); - causeXContentBuilder.endObject(); - exceptionString = BytesReference.bytes(causeXContentBuilder).utf8ToString(); - } + public static SnapshotHistoryItem creationFailureRecord(long timeStamp, SnapshotLifecyclePolicy policy, String snapshotName, + Exception exception) throws IOException { + String exceptionString = exceptionToString(exception); return new SnapshotHistoryItem(timeStamp, policy.getId(), policy.getRepository(), snapshotName, CREATE_OPERATION, false, policy.getConfig(), exceptionString); } + public static SnapshotHistoryItem deletionSuccessRecord(long timestamp, String snapshotName, String policyId, String repository) { + return new SnapshotHistoryItem(timestamp, policyId, repository, snapshotName, DELETE_OPERATION, true, null, null); + } + + public static SnapshotHistoryItem deletionPossibleSuccessRecord(long timestamp, String snapshotName, String policyId, String repository, + String details) { + return new SnapshotHistoryItem(timestamp, policyId, repository, snapshotName, DELETE_OPERATION, true, null, details); + } + + public static SnapshotHistoryItem deletionFailureRecord(long timestamp, String snapshotName, String policyId, String repository, + Exception exception) throws IOException { + String exceptionString = exceptionToString(exception); + return new SnapshotHistoryItem(timestamp, policyId, repository, snapshotName, DELETE_OPERATION, false, + null, exceptionString); + } + public SnapshotHistoryItem(StreamInput in) throws IOException { this.timestamp = in.readVLong(); this.policyId = in.readString(); @@ -220,4 +231,16 @@ public int hashCode() { public String toString() { return Strings.toString(this); } + + private static String exceptionToString(Exception exception) throws IOException { + Params stacktraceParams = new MapParams(Collections.singletonMap(REST_EXCEPTION_SKIP_STACK_TRACE, "false")); + String exceptionString; + try (XContentBuilder causeXContentBuilder = JsonXContent.contentBuilder()) { + causeXContentBuilder.startObject(); + ElasticsearchException.generateThrowableXContent(causeXContentBuilder, stacktraceParams, exception); + causeXContentBuilder.endObject(); + exceptionString = BytesReference.bytes(causeXContentBuilder).utf8ToString(); + } + return exceptionString; + } } diff --git a/x-pack/plugin/core/src/main/java/org/elasticsearch/xpack/slm/SnapshotLifecycleStats.java b/x-pack/plugin/core/src/main/java/org/elasticsearch/xpack/slm/SnapshotLifecycleStats.java new file mode 100644 index 0000000000000..fa018abc6c43e --- /dev/null +++ b/x-pack/plugin/core/src/main/java/org/elasticsearch/xpack/slm/SnapshotLifecycleStats.java @@ -0,0 +1,383 @@ +/* + * 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.slm; + +import org.elasticsearch.common.ParseField; +import org.elasticsearch.common.Strings; +import org.elasticsearch.common.io.stream.StreamInput; +import org.elasticsearch.common.io.stream.StreamOutput; +import org.elasticsearch.common.io.stream.Writeable; +import org.elasticsearch.common.metrics.CounterMetric; +import org.elasticsearch.common.unit.TimeValue; +import org.elasticsearch.common.xcontent.ConstructingObjectParser; +import org.elasticsearch.common.xcontent.ToXContentFragment; +import org.elasticsearch.common.xcontent.ToXContentObject; +import org.elasticsearch.common.xcontent.XContentBuilder; +import org.elasticsearch.common.xcontent.XContentParser; + +import java.io.IOException; +import java.util.Collections; +import java.util.HashMap; +import java.util.List; +import java.util.Map; +import java.util.Objects; +import java.util.concurrent.ConcurrentHashMap; +import java.util.function.Function; +import java.util.stream.Collectors; + +/** + * SnapshotLifecycleStats contains metrics and stats about snapshot lifecycle policy execution - how + * many snapshots were taken, deleted, how many failures, etc. It contains both global stats + * (snapshots taken, retention runs), and per-policy stats. + */ +public class SnapshotLifecycleStats implements Writeable, ToXContentObject { + + private final CounterMetric retentionRunCount = new CounterMetric(); + private final CounterMetric retentionFailedCount = new CounterMetric(); + private final CounterMetric retentionTimedOut = new CounterMetric(); + private final CounterMetric retentionTimeMs = new CounterMetric(); + private final Map policyStats; + + public static final ParseField RETENTION_RUNS = new ParseField("retention_runs"); + public static final ParseField RETENTION_FAILED = new ParseField("retention_failed"); + public static final ParseField RETENTION_TIMED_OUT = new ParseField("retention_timed_out"); + public static final ParseField RETENTION_TIME = new ParseField("retention_deletion_time"); + public static final ParseField RETENTION_TIME_MILLIS = new ParseField("retention_deletion_time_millis"); + public static final ParseField POLICY_STATS = new ParseField("policy_stats"); + public static final ParseField TOTAL_TAKEN = new ParseField("total_snapshots_taken"); + public static final ParseField TOTAL_FAILED = new ParseField("total_snapshots_failed"); + public static final ParseField TOTAL_DELETIONS = new ParseField("total_snapshots_deleted"); + public static final ParseField TOTAL_DELETION_FAILURES = new ParseField("total_snapshot_deletion_failures"); + + @SuppressWarnings("unchecked") + private static final ConstructingObjectParser PARSER = + new ConstructingObjectParser<>("snapshot_policy_stats", true, + a -> { + long runs = (long) a[0]; + long failed = (long) a[1]; + long timedOut = (long) a[2]; + long timeMs = (long) a[3]; + Map policyStatsMap = ((List) a[4]).stream() + .collect(Collectors.toMap(m -> m.policyId, Function.identity())); + return new SnapshotLifecycleStats(runs, failed, timedOut, timeMs, policyStatsMap); + }); + + static { + PARSER.declareLong(ConstructingObjectParser.constructorArg(), RETENTION_RUNS); + PARSER.declareLong(ConstructingObjectParser.constructorArg(), RETENTION_FAILED); + PARSER.declareLong(ConstructingObjectParser.constructorArg(), RETENTION_TIMED_OUT); + PARSER.declareLong(ConstructingObjectParser.constructorArg(), RETENTION_TIME_MILLIS); + PARSER.declareNamedObjects(ConstructingObjectParser.constructorArg(), (p, c, n) -> SnapshotPolicyStats.parse(p, n), POLICY_STATS); + } + + public SnapshotLifecycleStats() { + this.policyStats = new ConcurrentHashMap<>(); + } + + // Package visible for testing + SnapshotLifecycleStats(long retentionRuns, long retentionFailed, long retentionTimedOut, long retentionTimeMs, + Map policyStats) { + this.retentionRunCount.inc(retentionRuns); + this.retentionFailedCount.inc(retentionFailed); + this.retentionTimedOut.inc(retentionTimedOut); + this.retentionTimeMs.inc(retentionTimeMs); + this.policyStats = policyStats; + } + + public SnapshotLifecycleStats(StreamInput in) throws IOException { + this.policyStats = new ConcurrentHashMap<>(in.readMap(StreamInput::readString, SnapshotPolicyStats::new)); + this.retentionRunCount.inc(in.readVLong()); + this.retentionFailedCount.inc(in.readVLong()); + this.retentionTimedOut.inc(in.readVLong()); + this.retentionTimeMs.inc(in.readVLong()); + } + + public static SnapshotLifecycleStats parse(XContentParser parser) { + return PARSER.apply(parser, null); + } + + public SnapshotLifecycleStats merge(SnapshotLifecycleStats other) { + + HashMap newPolicyStats = new HashMap<>(this.policyStats); + // Merges the per-run stats (the stats in "other") with the stats already present + other.policyStats + .forEach((policyId, perRunPolicyStats) -> { + newPolicyStats.compute(policyId, (k, existingPolicyMetrics) -> { + if (existingPolicyMetrics == null) { + return perRunPolicyStats; + } else { + return existingPolicyMetrics.merge(perRunPolicyStats); + } + }); + }); + + return new SnapshotLifecycleStats(this.retentionRunCount.count() + other.retentionRunCount.count(), + this.retentionFailedCount.count() + other.retentionFailedCount.count(), + this.retentionTimedOut.count() + other.retentionTimedOut.count(), + this.retentionTimeMs.count() + other.retentionTimeMs.count(), + newPolicyStats); + } + + public SnapshotLifecycleStats removePolicy(String policyId) { + Map policyStats = new HashMap<>(this.policyStats); + policyStats.remove(policyId); + return new SnapshotLifecycleStats(this.retentionRunCount.count(), this.retentionFailedCount.count(), + this.retentionTimedOut.count(), this.retentionTimeMs.count(), + policyStats); + } + + /** + * @return a map of per-policy stats for each SLM policy + */ + public Map getMetrics() { + return Collections.unmodifiableMap(this.policyStats); + } + + /** + * Increment the number of times SLM retention has been run + */ + public void retentionRun() { + this.retentionRunCount.inc(); + } + + /** + * Increment the number of times SLM retention has failed + */ + public void retentionFailed() { + this.retentionFailedCount.inc(); + } + + /** + * Increment the number of times that SLM retention timed out due to the max delete time + * window being exceeded. + */ + public void retentionTimedOut() { + this.retentionTimedOut.inc(); + } + + /** + * Register the amount of time taken for deleting snapshots during SLM retention + */ + public void deletionTime(TimeValue elapsedTime) { + this.retentionTimeMs.inc(elapsedTime.millis()); + } + + /** + * Increment the per-policy snapshot taken count for the given policy id + */ + public void snapshotTaken(String slmPolicy) { + this.policyStats.computeIfAbsent(slmPolicy, SnapshotPolicyStats::new).snapshotTaken(); + } + + /** + * Increment the per-policy snapshot failure count for the given policy id + */ + public void snapshotFailed(String slmPolicy) { + this.policyStats.computeIfAbsent(slmPolicy, SnapshotPolicyStats::new).snapshotFailed(); + } + + /** + * Increment the per-policy snapshot deleted count for the given policy id + */ + public void snapshotDeleted(String slmPolicy) { + this.policyStats.computeIfAbsent(slmPolicy, SnapshotPolicyStats::new).snapshotDeleted(); + } + + /** + * Increment the per-policy snapshot deletion failure count for the given policy id + */ + public void snapshotDeleteFailure(String slmPolicy) { + this.policyStats.computeIfAbsent(slmPolicy, SnapshotPolicyStats::new).snapshotDeleteFailure(); + } + + @Override + public void writeTo(StreamOutput out) throws IOException { + out.writeMap(policyStats, StreamOutput::writeString, (v, o) -> o.writeTo(v)); + out.writeVLong(retentionRunCount.count()); + out.writeVLong(retentionFailedCount.count()); + out.writeVLong(retentionTimedOut.count()); + out.writeVLong(retentionTimeMs.count()); + } + + @Override + public XContentBuilder toXContent(XContentBuilder builder, Params params) throws IOException { + builder.startObject(); + builder.field(RETENTION_RUNS.getPreferredName(), this.retentionRunCount.count()); + builder.field(RETENTION_FAILED.getPreferredName(), this.retentionFailedCount.count()); + builder.field(RETENTION_TIMED_OUT.getPreferredName(), this.retentionTimedOut.count()); + TimeValue retentionTime = TimeValue.timeValueMillis(this.retentionTimeMs.count()); + builder.field(RETENTION_TIME.getPreferredName(), retentionTime); + builder.field(RETENTION_TIME_MILLIS.getPreferredName(), retentionTime.millis()); + + Map metrics = getMetrics(); + long totalTaken = metrics.values().stream().mapToLong(s -> s.snapshotsTaken.count()).sum(); + long totalFailed = metrics.values().stream().mapToLong(s -> s.snapshotsFailed.count()).sum(); + long totalDeleted = metrics.values().stream().mapToLong(s -> s.snapshotsDeleted.count()).sum(); + long totalDeleteFailures = metrics.values().stream().mapToLong(s -> s.snapshotDeleteFailures.count()).sum(); + builder.field(TOTAL_TAKEN.getPreferredName(), totalTaken); + builder.field(TOTAL_FAILED.getPreferredName(), totalFailed); + builder.field(TOTAL_DELETIONS.getPreferredName(), totalDeleted); + builder.field(TOTAL_DELETION_FAILURES.getPreferredName(), totalDeleteFailures); + builder.startObject(POLICY_STATS.getPreferredName()); + for (Map.Entry policy : metrics.entrySet()) { + SnapshotPolicyStats perPolicyMetrics = policy.getValue(); + builder.startObject(perPolicyMetrics.policyId); + perPolicyMetrics.toXContent(builder, params); + builder.endObject(); + } + builder.endObject(); + builder.endObject(); + return builder; + } + + @Override + public int hashCode() { + return Objects.hash(retentionRunCount.count(), retentionFailedCount.count(), + retentionTimedOut.count(), retentionTimeMs.count(), policyStats); + } + + @Override + public boolean equals(Object obj) { + if (obj == null) { + return false; + } + if (obj.getClass() != getClass()) { + return false; + } + SnapshotLifecycleStats other = (SnapshotLifecycleStats) obj; + return Objects.equals(retentionRunCount.count(), other.retentionRunCount.count()) && + Objects.equals(retentionFailedCount.count(), other.retentionFailedCount.count()) && + Objects.equals(retentionTimedOut.count(), other.retentionTimedOut.count()) && + Objects.equals(retentionTimeMs.count(), other.retentionTimeMs.count()) && + Objects.equals(policyStats, other.policyStats); + } + + @Override + public String toString() { + return Strings.toString(this); + } + + public static class SnapshotPolicyStats implements Writeable, ToXContentFragment { + private final String policyId; + private final CounterMetric snapshotsTaken = new CounterMetric(); + private final CounterMetric snapshotsFailed = new CounterMetric(); + private final CounterMetric snapshotsDeleted = new CounterMetric(); + private final CounterMetric snapshotDeleteFailures = new CounterMetric(); + + public static final ParseField SNAPSHOTS_TAKEN = new ParseField("snapshots_taken"); + public static final ParseField SNAPSHOTS_FAILED = new ParseField("snapshots_failed"); + public static final ParseField SNAPSHOTS_DELETED = new ParseField("snapshots_deleted"); + public static final ParseField SNAPSHOT_DELETION_FAILURES = new ParseField("snapshot_deletion_failures"); + + private static final ConstructingObjectParser PARSER = + new ConstructingObjectParser<>("snapshot_policy_stats", true, + (a, id) -> { + long taken = (long) a[0]; + long failed = (long) a[1]; + long deleted = (long) a[2]; + long deleteFailed = (long) a[3]; + return new SnapshotPolicyStats(id, taken, failed, deleted, deleteFailed); + }); + + static { + PARSER.declareLong(ConstructingObjectParser.constructorArg(), SNAPSHOTS_TAKEN); + PARSER.declareLong(ConstructingObjectParser.constructorArg(), SNAPSHOTS_FAILED); + PARSER.declareLong(ConstructingObjectParser.constructorArg(), SNAPSHOTS_DELETED); + PARSER.declareLong(ConstructingObjectParser.constructorArg(), SNAPSHOT_DELETION_FAILURES); + } + + public SnapshotPolicyStats(String slmPolicy) { + this.policyId = slmPolicy; + } + + public SnapshotPolicyStats(String policyId, long snapshotsTaken, long snapshotsFailed, long deleted, long failedDeletes) { + this.policyId = policyId; + this.snapshotsTaken.inc(snapshotsTaken); + this.snapshotsFailed.inc(snapshotsFailed); + this.snapshotsDeleted.inc(deleted); + this.snapshotDeleteFailures.inc(failedDeletes); + } + + public SnapshotPolicyStats(StreamInput in) throws IOException { + this.policyId = in.readString(); + this.snapshotsTaken.inc(in.readVLong()); + this.snapshotsFailed.inc(in.readVLong()); + this.snapshotsDeleted.inc(in.readVLong()); + this.snapshotDeleteFailures.inc(in.readVLong()); + } + + public static SnapshotPolicyStats parse(XContentParser parser, String policyId) { + return PARSER.apply(parser, policyId); + } + + public SnapshotPolicyStats merge(SnapshotPolicyStats other) { + return new SnapshotPolicyStats( + this.policyId, + this.snapshotsTaken.count() + other.snapshotsTaken.count(), + this.snapshotsFailed.count() + other.snapshotsFailed.count(), + this.snapshotsDeleted.count() + other.snapshotsDeleted.count(), + this.snapshotDeleteFailures.count() + other.snapshotDeleteFailures.count()); + } + + void snapshotTaken() { + snapshotsTaken.inc(); + } + + void snapshotFailed() { + snapshotsFailed.inc(); + } + + void snapshotDeleted() { + snapshotsDeleted.inc(); + } + + void snapshotDeleteFailure() { + snapshotDeleteFailures.inc(); + } + + @Override + public void writeTo(StreamOutput out) throws IOException { + out.writeString(policyId); + out.writeVLong(snapshotsTaken.count()); + out.writeVLong(snapshotsFailed.count()); + out.writeVLong(snapshotsDeleted.count()); + out.writeVLong(snapshotDeleteFailures.count()); + } + + @Override + public int hashCode() { + return Objects.hash(policyId, snapshotsTaken.count(), snapshotsFailed.count(), + snapshotsDeleted.count(), snapshotDeleteFailures.count()); + } + + @Override + public boolean equals(Object obj) { + if (obj == null) { + return false; + } + if (obj.getClass() != getClass()) { + return false; + } + SnapshotPolicyStats other = (SnapshotPolicyStats) obj; + return Objects.equals(policyId, other.policyId) && + Objects.equals(snapshotsTaken.count(), other.snapshotsTaken.count()) && + Objects.equals(snapshotsFailed.count(), other.snapshotsFailed.count()) && + Objects.equals(snapshotsDeleted.count(), other.snapshotsDeleted.count()) && + Objects.equals(snapshotDeleteFailures.count(), other.snapshotDeleteFailures.count()); + } + + @Override + public XContentBuilder toXContent(XContentBuilder builder, Params params) throws IOException { + builder.field(SnapshotPolicyStats.SNAPSHOTS_TAKEN.getPreferredName(), snapshotsTaken.count()); + builder.field(SnapshotPolicyStats.SNAPSHOTS_FAILED.getPreferredName(), snapshotsFailed.count()); + builder.field(SnapshotPolicyStats.SNAPSHOTS_DELETED.getPreferredName(), snapshotsDeleted.count()); + builder.field(SnapshotPolicyStats.SNAPSHOT_DELETION_FAILURES.getPreferredName(), snapshotDeleteFailures.count()); + return builder; + } + } + +} diff --git a/x-pack/plugin/core/src/test/java/org/elasticsearch/xpack/core/slm/SnapshotLifecycleMetadataTests.java b/x-pack/plugin/core/src/test/java/org/elasticsearch/xpack/core/slm/SnapshotLifecycleMetadataTests.java new file mode 100644 index 0000000000000..1e9b1fa717809 --- /dev/null +++ b/x-pack/plugin/core/src/test/java/org/elasticsearch/xpack/core/slm/SnapshotLifecycleMetadataTests.java @@ -0,0 +1,41 @@ +/* + * 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.core.slm; + +import org.elasticsearch.common.io.stream.Writeable; +import org.elasticsearch.common.xcontent.XContentParser; +import org.elasticsearch.test.AbstractSerializingTestCase; +import org.elasticsearch.xpack.core.ilm.OperationMode; +import org.elasticsearch.xpack.slm.SnapshotLifecycleStatsTests; + +import java.io.IOException; +import java.util.HashMap; +import java.util.Map; + +public class SnapshotLifecycleMetadataTests extends AbstractSerializingTestCase { + @Override + protected SnapshotLifecycleMetadata doParseInstance(XContentParser parser) throws IOException { + return SnapshotLifecycleMetadata.PARSER.apply(parser, null); + } + + @Override + protected SnapshotLifecycleMetadata createTestInstance() { + int policyCount = randomIntBetween(0, 3); + Map policies = new HashMap<>(policyCount); + for (int i = 0; i < policyCount; i++) { + String id = "policy-" + randomAlphaOfLength(3); + policies.put(id, SnapshotLifecyclePolicyMetadataTests.createRandomPolicyMetadata(id)); + } + return new SnapshotLifecycleMetadata(policies, randomFrom(OperationMode.values()), + SnapshotLifecycleStatsTests.randomLifecycleStats()); + } + + @Override + protected Writeable.Reader instanceReader() { + return SnapshotLifecycleMetadata::new; + } +} diff --git a/x-pack/plugin/core/src/test/java/org/elasticsearch/xpack/core/slm/SnapshotLifecyclePolicyItemTests.java b/x-pack/plugin/core/src/test/java/org/elasticsearch/xpack/core/slm/SnapshotLifecyclePolicyItemTests.java index 04ab84d5ef935..183b0141caa43 100644 --- a/x-pack/plugin/core/src/test/java/org/elasticsearch/xpack/core/slm/SnapshotLifecyclePolicyItemTests.java +++ b/x-pack/plugin/core/src/test/java/org/elasticsearch/xpack/core/slm/SnapshotLifecyclePolicyItemTests.java @@ -11,8 +11,9 @@ import org.elasticsearch.snapshots.SnapshotId; import org.elasticsearch.test.AbstractWireSerializingTestCase; import org.elasticsearch.test.ESTestCase; +import org.elasticsearch.xpack.slm.SnapshotLifecycleStatsTests; -import static org.elasticsearch.xpack.core.slm.SnapshotLifecyclePolicyMetadataTests.createRandomPolicy; +import static org.elasticsearch.xpack.core.slm.SnapshotLifecyclePolicyMetadataTests.randomSnapshotLifecyclePolicy; import static org.elasticsearch.xpack.core.slm.SnapshotLifecyclePolicyMetadataTests.createRandomPolicyMetadata; public class SnapshotLifecyclePolicyItemTests extends AbstractWireSerializingTestCase { @@ -27,34 +28,39 @@ public static SnapshotLifecyclePolicyItem.SnapshotInProgress randomSnapshotInPro @Override protected SnapshotLifecyclePolicyItem createTestInstance() { - return new SnapshotLifecyclePolicyItem(createRandomPolicyMetadata(randomAlphaOfLengthBetween(5, 10)), randomSnapshotInProgress()); + String policyId = randomAlphaOfLengthBetween(5, 10); + return new SnapshotLifecyclePolicyItem(createRandomPolicyMetadata(policyId), randomSnapshotInProgress(), + SnapshotLifecycleStatsTests.randomPolicyStats(policyId)); } @Override protected SnapshotLifecyclePolicyItem mutateInstance(SnapshotLifecyclePolicyItem instance) { - switch (between(0, 5)) { + switch (between(0, 6)) { case 0: String newPolicyId = randomValueOtherThan(instance.getPolicy().getId(), () -> randomAlphaOfLengthBetween(5, 10)); - return new SnapshotLifecyclePolicyItem(createRandomPolicy(newPolicyId), + return new SnapshotLifecyclePolicyItem(randomSnapshotLifecyclePolicy(newPolicyId), instance.getVersion(), instance.getModifiedDate(), instance.getLastSuccess(), instance.getLastFailure(), - instance.getSnapshotInProgress()); + instance.getSnapshotInProgress(), + instance.getPolicyStats()); case 1: return new SnapshotLifecyclePolicyItem(instance.getPolicy(), randomValueOtherThan(instance.getVersion(), ESTestCase::randomNonNegativeLong), instance.getModifiedDate(), instance.getLastSuccess(), instance.getLastFailure(), - instance.getSnapshotInProgress()); + instance.getSnapshotInProgress(), + instance.getPolicyStats()); case 2: return new SnapshotLifecyclePolicyItem(instance.getPolicy(), instance.getVersion(), randomValueOtherThan(instance.getModifiedDate(), ESTestCase::randomNonNegativeLong), instance.getLastSuccess(), instance.getLastFailure(), - instance.getSnapshotInProgress()); + instance.getSnapshotInProgress(), + instance.getPolicyStats()); case 3: return new SnapshotLifecyclePolicyItem(instance.getPolicy(), instance.getVersion(), @@ -62,7 +68,8 @@ protected SnapshotLifecyclePolicyItem mutateInstance(SnapshotLifecyclePolicyItem randomValueOtherThan(instance.getLastSuccess(), SnapshotInvocationRecordTests::randomSnapshotInvocationRecord), instance.getLastFailure(), - instance.getSnapshotInProgress()); + instance.getSnapshotInProgress(), + instance.getPolicyStats()); case 4: return new SnapshotLifecyclePolicyItem(instance.getPolicy(), instance.getVersion(), @@ -70,7 +77,8 @@ protected SnapshotLifecyclePolicyItem mutateInstance(SnapshotLifecyclePolicyItem instance.getLastSuccess(), randomValueOtherThan(instance.getLastFailure(), SnapshotInvocationRecordTests::randomSnapshotInvocationRecord), - instance.getSnapshotInProgress()); + instance.getSnapshotInProgress(), + instance.getPolicyStats()); case 5: return new SnapshotLifecyclePolicyItem(instance.getPolicy(), instance.getVersion(), @@ -78,7 +86,17 @@ protected SnapshotLifecyclePolicyItem mutateInstance(SnapshotLifecyclePolicyItem instance.getLastSuccess(), instance.getLastFailure(), randomValueOtherThan(instance.getSnapshotInProgress(), - SnapshotLifecyclePolicyItemTests::randomSnapshotInProgress)); + SnapshotLifecyclePolicyItemTests::randomSnapshotInProgress), + instance.getPolicyStats()); + case 6: + return new SnapshotLifecyclePolicyItem(instance.getPolicy(), + instance.getVersion(), + instance.getModifiedDate(), + instance.getLastSuccess(), + instance.getLastFailure(), + instance.getSnapshotInProgress(), + randomValueOtherThan(instance.getPolicyStats(), + () -> SnapshotLifecycleStatsTests.randomPolicyStats(instance.getPolicy().getId()))); default: throw new AssertionError("failure, got illegal switch case"); } diff --git a/x-pack/plugin/core/src/test/java/org/elasticsearch/xpack/core/slm/SnapshotLifecyclePolicyMetadataTests.java b/x-pack/plugin/core/src/test/java/org/elasticsearch/xpack/core/slm/SnapshotLifecyclePolicyMetadataTests.java index 964cfd733b319..cf48615f67559 100644 --- a/x-pack/plugin/core/src/test/java/org/elasticsearch/xpack/core/slm/SnapshotLifecyclePolicyMetadataTests.java +++ b/x-pack/plugin/core/src/test/java/org/elasticsearch/xpack/core/slm/SnapshotLifecyclePolicyMetadataTests.java @@ -7,6 +7,7 @@ package org.elasticsearch.xpack.core.slm; import org.elasticsearch.common.io.stream.Writeable; +import org.elasticsearch.common.unit.TimeValue; import org.elasticsearch.common.xcontent.XContentParser; import org.elasticsearch.test.AbstractSerializingTestCase; import org.elasticsearch.test.ESTestCase; @@ -50,7 +51,7 @@ protected SnapshotLifecyclePolicyMetadata mutateInstance(SnapshotLifecyclePolicy switch (between(0, 5)) { case 0: return SnapshotLifecyclePolicyMetadata.builder(instance) - .setPolicy(randomValueOtherThan(instance.getPolicy(), () -> createRandomPolicy(randomAlphaOfLength(10)))) + .setPolicy(randomValueOtherThan(instance.getPolicy(), () -> randomSnapshotLifecyclePolicy(randomAlphaOfLength(10)))) .build(); case 1: return SnapshotLifecyclePolicyMetadata.builder(instance) @@ -81,7 +82,7 @@ protected SnapshotLifecyclePolicyMetadata mutateInstance(SnapshotLifecyclePolicy public static SnapshotLifecyclePolicyMetadata createRandomPolicyMetadata(String policyId) { SnapshotLifecyclePolicyMetadata.Builder builder = SnapshotLifecyclePolicyMetadata.builder() - .setPolicy(createRandomPolicy(policyId)) + .setPolicy(randomSnapshotLifecyclePolicy(policyId)) .setVersion(randomNonNegativeLong()) .setModifiedDate(randomNonNegativeLong()); if (randomBoolean()) { @@ -96,7 +97,7 @@ public static SnapshotLifecyclePolicyMetadata createRandomPolicyMetadata(String return builder.build(); } - public static SnapshotLifecyclePolicy createRandomPolicy(String policyId) { + public static SnapshotLifecyclePolicy randomSnapshotLifecyclePolicy(String policyId) { Map config = new HashMap<>(); for (int i = 0; i < randomIntBetween(2, 5); i++) { config.put(randomAlphaOfLength(4), randomAlphaOfLength(4)); @@ -105,10 +106,18 @@ public static SnapshotLifecyclePolicy createRandomPolicy(String policyId) { randomAlphaOfLength(4), randomSchedule(), randomAlphaOfLength(4), - config); + config, + randomRetention()); } - private static String randomSchedule() { + public static SnapshotRetentionConfiguration randomRetention() { + return rarely() ? null : new SnapshotRetentionConfiguration( + rarely() ? null : TimeValue.parseTimeValue(randomTimeValue(), "random retention generation"), + rarely() ? null : randomIntBetween(1, 10), + rarely() ? null : randomIntBetween(15, 30)); + } + + public static String randomSchedule() { return randomIntBetween(0, 59) + " " + randomIntBetween(0, 59) + " " + randomIntBetween(0, 12) + " * * ?"; diff --git a/x-pack/plugin/core/src/test/java/org/elasticsearch/xpack/core/slm/history/SnapshotHistoryStoreTests.java b/x-pack/plugin/core/src/test/java/org/elasticsearch/xpack/core/slm/history/SnapshotHistoryStoreTests.java index 59dd546cba2a3..0a6a635490e2c 100644 --- a/x-pack/plugin/core/src/test/java/org/elasticsearch/xpack/core/slm/history/SnapshotHistoryStoreTests.java +++ b/x-pack/plugin/core/src/test/java/org/elasticsearch/xpack/core/slm/history/SnapshotHistoryStoreTests.java @@ -77,7 +77,7 @@ public void testNoActionIfDisabled() { final long timestamp = randomNonNegativeLong(); SnapshotLifecyclePolicy.ResolverContext context = new SnapshotLifecyclePolicy.ResolverContext(timestamp); String snapshotId = policy.generateSnapshotName(context); - SnapshotHistoryItem record = SnapshotHistoryItem.successRecord(timestamp, policy, snapshotId); + SnapshotHistoryItem record = SnapshotHistoryItem.creationSuccessRecord(timestamp, policy, snapshotId); client.setVerifier((a, r, l) -> { fail("the history store is disabled, no action should have been taken"); @@ -94,7 +94,7 @@ public void testPut() throws Exception { SnapshotLifecyclePolicy.ResolverContext context = new SnapshotLifecyclePolicy.ResolverContext(timestamp); String snapshotId = policy.generateSnapshotName(context); { - SnapshotHistoryItem record = SnapshotHistoryItem.successRecord(timestamp, policy, snapshotId); + SnapshotHistoryItem record = SnapshotHistoryItem.creationSuccessRecord(timestamp, policy, snapshotId); AtomicInteger calledTimes = new AtomicInteger(0); client.setVerifier((action, request, listener) -> { @@ -132,7 +132,7 @@ public void testPut() throws Exception { { final String cause = randomAlphaOfLength(9); Exception failureException = new RuntimeException(cause); - SnapshotHistoryItem record = SnapshotHistoryItem.failureRecord(timestamp, policy, snapshotId, failureException); + SnapshotHistoryItem record = SnapshotHistoryItem.creationFailureRecord(timestamp, policy, snapshotId, failureException); AtomicInteger calledTimes = new AtomicInteger(0); client.setVerifier((action, request, listener) -> { @@ -373,7 +373,8 @@ public static SnapshotLifecyclePolicy randomSnapshotLifecyclePolicy(String id) { randomAlphaOfLength(4), randomSchedule(), randomAlphaOfLength(4), - config); + config, + null); } private static String randomSchedule() { diff --git a/x-pack/plugin/core/src/test/java/org/elasticsearch/xpack/slm/SnapshotLifecycleStatsTests.java b/x-pack/plugin/core/src/test/java/org/elasticsearch/xpack/slm/SnapshotLifecycleStatsTests.java new file mode 100644 index 0000000000000..25b6c26998d21 --- /dev/null +++ b/x-pack/plugin/core/src/test/java/org/elasticsearch/xpack/slm/SnapshotLifecycleStatsTests.java @@ -0,0 +1,60 @@ +/* + * 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.slm; + +import org.elasticsearch.common.io.stream.Writeable; +import org.elasticsearch.common.xcontent.XContentParser; +import org.elasticsearch.test.AbstractSerializingTestCase; + +import java.io.IOException; +import java.util.HashMap; +import java.util.Map; + +public class SnapshotLifecycleStatsTests extends AbstractSerializingTestCase { + @Override + protected SnapshotLifecycleStats doParseInstance(XContentParser parser) throws IOException { + return SnapshotLifecycleStats.parse(parser); + } + + public static SnapshotLifecycleStats.SnapshotPolicyStats randomPolicyStats(String policyId) { + return new SnapshotLifecycleStats.SnapshotPolicyStats(policyId, + randomBoolean() ? 0 : randomNonNegativeLong(), + randomBoolean() ? 0 : randomNonNegativeLong(), + randomBoolean() ? 0 : randomNonNegativeLong(), + randomBoolean() ? 0 : randomNonNegativeLong()); + } + + public static SnapshotLifecycleStats randomLifecycleStats() { + int policies = randomIntBetween(0, 5); + Map policyStats = new HashMap<>(policies); + for (int i = 0; i < policies; i++) { + String policy = "policy-" + randomAlphaOfLength(4); + policyStats.put(policy, randomPolicyStats(policy)); + } + return new SnapshotLifecycleStats( + randomBoolean() ? 0 : randomNonNegativeLong(), + randomBoolean() ? 0 : randomNonNegativeLong(), + randomBoolean() ? 0 : randomNonNegativeLong(), + randomBoolean() ? 0 : randomNonNegativeLong(), + policyStats); + } + + @Override + protected SnapshotLifecycleStats createTestInstance() { + return randomLifecycleStats(); + } + + @Override + protected SnapshotLifecycleStats mutateInstance(SnapshotLifecycleStats instance) throws IOException { + return randomValueOtherThan(instance, () -> instance.merge(createTestInstance())); + } + + @Override + protected Writeable.Reader instanceReader() { + return SnapshotLifecycleStats::new; + } +} diff --git a/x-pack/plugin/core/src/test/java/org/elasticsearch/xpack/slm/SnapshotRetentionConfigurationTests.java b/x-pack/plugin/core/src/test/java/org/elasticsearch/xpack/slm/SnapshotRetentionConfigurationTests.java new file mode 100644 index 0000000000000..378fe0c2d774d --- /dev/null +++ b/x-pack/plugin/core/src/test/java/org/elasticsearch/xpack/slm/SnapshotRetentionConfigurationTests.java @@ -0,0 +1,109 @@ +/* + * 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.slm; + +import org.elasticsearch.common.unit.TimeValue; +import org.elasticsearch.snapshots.SnapshotId; +import org.elasticsearch.snapshots.SnapshotInfo; +import org.elasticsearch.test.ESTestCase; +import org.elasticsearch.xpack.core.slm.SnapshotLifecyclePolicy; +import org.elasticsearch.xpack.core.slm.SnapshotRetentionConfiguration; + +import java.util.ArrayList; +import java.util.Arrays; +import java.util.Collections; +import java.util.HashMap; +import java.util.List; +import java.util.Map; + +import static org.hamcrest.Matchers.containsString; +import static org.hamcrest.Matchers.equalTo; + +public class SnapshotRetentionConfigurationTests extends ESTestCase { + + private static final String REPO = "repo"; + + public void testConflictingSettings() { + IllegalArgumentException e; + e = expectThrows(IllegalArgumentException.class, () -> new SnapshotRetentionConfiguration(null, 0, null)); + assertThat(e.getMessage(), containsString("minimum snapshot count must be at least 1, but was: 0")); + e = expectThrows(IllegalArgumentException.class, () -> new SnapshotRetentionConfiguration(null, -2, null)); + assertThat(e.getMessage(), containsString("minimum snapshot count must be at least 1, but was: -2")); + e = expectThrows(IllegalArgumentException.class, () -> new SnapshotRetentionConfiguration(null, null, 0)); + assertThat(e.getMessage(), containsString("maximum snapshot count must be at least 1, but was: 0")); + e = expectThrows(IllegalArgumentException.class, () -> new SnapshotRetentionConfiguration(null, null, -2)); + assertThat(e.getMessage(), containsString("maximum snapshot count must be at least 1, but was: -2")); + e = expectThrows(IllegalArgumentException.class, () -> new SnapshotRetentionConfiguration(null, 3, 1)); + assertThat(e.getMessage(), containsString("minimum snapshot count 3 cannot be larger than maximum snapshot count 1")); + } + + public void testExpireAfter() { + SnapshotRetentionConfiguration conf = new SnapshotRetentionConfiguration( + () -> TimeValue.timeValueDays(1).millis() + 1, + TimeValue.timeValueDays(1), null, null); + SnapshotInfo oldInfo = makeInfo(0); + assertThat(conf.getSnapshotDeletionPredicate(Collections.singletonList(oldInfo)).test(oldInfo), equalTo(true)); + + SnapshotInfo newInfo = makeInfo(1); + assertThat(conf.getSnapshotDeletionPredicate(Collections.singletonList(newInfo)).test(newInfo), equalTo(false)); + + List infos = new ArrayList<>(); + infos.add(newInfo); + infos.add(oldInfo); + assertThat(conf.getSnapshotDeletionPredicate(infos).test(newInfo), equalTo(false)); + assertThat(conf.getSnapshotDeletionPredicate(infos).test(oldInfo), equalTo(true)); + } + + public void testExpiredWithMinimum() { + SnapshotRetentionConfiguration conf = new SnapshotRetentionConfiguration(() -> TimeValue.timeValueDays(1).millis() + 1, + TimeValue.timeValueDays(1), 2, null); + SnapshotInfo oldInfo = makeInfo(0); + SnapshotInfo newInfo = makeInfo(1); + + List infos = new ArrayList<>(); + infos.add(newInfo); + infos.add(oldInfo); + assertThat(conf.getSnapshotDeletionPredicate(infos).test(newInfo), equalTo(false)); + assertThat(conf.getSnapshotDeletionPredicate(infos).test(oldInfo), equalTo(false)); + + conf = new SnapshotRetentionConfiguration(() -> TimeValue.timeValueDays(1).millis() + 1, + TimeValue.timeValueDays(1), 1, null); + assertThat(conf.getSnapshotDeletionPredicate(infos).test(newInfo), equalTo(false)); + assertThat(conf.getSnapshotDeletionPredicate(infos).test(oldInfo), equalTo(true)); + } + + public void testMaximum() { + SnapshotRetentionConfiguration conf = new SnapshotRetentionConfiguration(() -> 1, null, 2, 5); + SnapshotInfo s1 = makeInfo(1); + SnapshotInfo s2 = makeInfo(2); + SnapshotInfo s3 = makeInfo(3); + SnapshotInfo s4 = makeInfo(4); + SnapshotInfo s5 = makeInfo(5); + SnapshotInfo s6 = makeInfo(6); + SnapshotInfo s7 = makeInfo(7); + SnapshotInfo s8 = makeInfo(8); + SnapshotInfo s9 = makeInfo(9); + + List infos = Arrays.asList(s1 , s2, s3, s4, s5, s6, s7, s8, s9); + assertThat(conf.getSnapshotDeletionPredicate(infos).test(s1), equalTo(true)); + assertThat(conf.getSnapshotDeletionPredicate(infos).test(s2), equalTo(true)); + assertThat(conf.getSnapshotDeletionPredicate(infos).test(s3), equalTo(true)); + assertThat(conf.getSnapshotDeletionPredicate(infos).test(s4), equalTo(true)); + assertThat(conf.getSnapshotDeletionPredicate(infos).test(s5), equalTo(false)); + assertThat(conf.getSnapshotDeletionPredicate(infos).test(s6), equalTo(false)); + assertThat(conf.getSnapshotDeletionPredicate(infos).test(s7), equalTo(false)); + assertThat(conf.getSnapshotDeletionPredicate(infos).test(s8), equalTo(false)); + assertThat(conf.getSnapshotDeletionPredicate(infos).test(s9), equalTo(false)); + } + + private SnapshotInfo makeInfo(long startTime) { + final Map meta = new HashMap<>(); + meta.put(SnapshotLifecyclePolicy.POLICY_ID_METADATA_FIELD, REPO); + return new SnapshotInfo(new SnapshotId("snap-" + randomAlphaOfLength(3), "uuid"), + Collections.singletonList("foo"), startTime, false, meta); + } +} diff --git a/x-pack/plugin/ilm/build.gradle b/x-pack/plugin/ilm/build.gradle index 17b66741a7a9f..74c239414a390 100644 --- a/x-pack/plugin/ilm/build.gradle +++ b/x-pack/plugin/ilm/build.gradle @@ -30,4 +30,3 @@ gradle.projectsEvaluated { } integTest.enabled = false - diff --git a/x-pack/plugin/ilm/qa/multi-node/build.gradle b/x-pack/plugin/ilm/qa/multi-node/build.gradle index 1746cc8f840ce..8e8df478d1235 100644 --- a/x-pack/plugin/ilm/qa/multi-node/build.gradle +++ b/x-pack/plugin/ilm/qa/multi-node/build.gradle @@ -25,8 +25,4 @@ testClusters.integTest { setting 'xpack.ml.enabled', 'false' setting 'xpack.license.self_generated.type', 'trial' setting 'indices.lifecycle.poll_interval', '1000ms' - // TODO: Find a way to run these tests with more than one snapshot pool thread. Currently we need to limit to one thread so that the - // rate limiting settings in SnapshotLifecycleIT doesn't result in blocked snapshot threads because multiple threads overshoot - // the limit simultaneously and the rate limiter then moves to wait minutes to make up for this. - setting 'thread_pool.snapshot.max', '1' } diff --git a/x-pack/plugin/ilm/qa/multi-node/src/test/java/org/elasticsearch/xpack/ilm/TimeSeriesLifecycleActionsIT.java b/x-pack/plugin/ilm/qa/multi-node/src/test/java/org/elasticsearch/xpack/ilm/TimeSeriesLifecycleActionsIT.java index 9b258471d61a0..ac385ce1c54b9 100644 --- a/x-pack/plugin/ilm/qa/multi-node/src/test/java/org/elasticsearch/xpack/ilm/TimeSeriesLifecycleActionsIT.java +++ b/x-pack/plugin/ilm/qa/multi-node/src/test/java/org/elasticsearch/xpack/ilm/TimeSeriesLifecycleActionsIT.java @@ -387,7 +387,8 @@ public void testDeleteDuringSnapshot() throws Exception { // index document so snapshot actually does something indexDocument(); // start snapshot - request = new Request("PUT", "/_snapshot/repo/snapshot"); + String snapName = "snapshot-" + randomAlphaOfLength(6).toLowerCase(Locale.ROOT); + request = new Request("PUT", "/_snapshot/repo/" + snapName); request.addParameter("wait_for_completion", "false"); request.setJsonEntity("{\"indices\": \"" + index + "\"}"); assertOK(client().performRequest(request)); @@ -396,8 +397,8 @@ public void testDeleteDuringSnapshot() throws Exception { // assert that index was deleted assertBusy(() -> assertFalse(indexExists(index)), 2, TimeUnit.MINUTES); // assert that snapshot is still in progress and clean up - assertThat(getSnapshotState("snapshot"), equalTo("SUCCESS")); - assertOK(client().performRequest(new Request("DELETE", "/_snapshot/repo/snapshot"))); + assertThat(getSnapshotState(snapName), equalTo("SUCCESS")); + assertOK(client().performRequest(new Request("DELETE", "/_snapshot/repo/" + snapName))); } public void testReadOnly() throws Exception { diff --git a/x-pack/plugin/ilm/qa/multi-node/src/test/java/org/elasticsearch/xpack/slm/SnapshotLifecycleIT.java b/x-pack/plugin/ilm/qa/multi-node/src/test/java/org/elasticsearch/xpack/slm/SnapshotLifecycleRestIT.java similarity index 58% rename from x-pack/plugin/ilm/qa/multi-node/src/test/java/org/elasticsearch/xpack/slm/SnapshotLifecycleIT.java rename to x-pack/plugin/ilm/qa/multi-node/src/test/java/org/elasticsearch/xpack/slm/SnapshotLifecycleRestIT.java index 8c1f1343c0411..43194823281d7 100644 --- a/x-pack/plugin/ilm/qa/multi-node/src/test/java/org/elasticsearch/xpack/slm/SnapshotLifecycleIT.java +++ b/x-pack/plugin/ilm/qa/multi-node/src/test/java/org/elasticsearch/xpack/slm/SnapshotLifecycleRestIT.java @@ -7,7 +7,8 @@ package org.elasticsearch.xpack.slm; import org.apache.http.util.EntityUtils; -import org.apache.lucene.util.LuceneTestCase.AwaitsFix; +import org.elasticsearch.ElasticsearchException; +import org.elasticsearch.action.admin.cluster.settings.ClusterUpdateSettingsRequest; import org.elasticsearch.action.index.IndexRequestBuilder; import org.elasticsearch.client.Request; import org.elasticsearch.client.Response; @@ -15,6 +16,8 @@ import org.elasticsearch.client.RestClient; import org.elasticsearch.client.ilm.RolloverAction; import org.elasticsearch.common.Strings; +import org.elasticsearch.common.settings.Settings; +import org.elasticsearch.common.unit.TimeValue; import org.elasticsearch.common.xcontent.DeprecationHandler; import org.elasticsearch.common.xcontent.NamedXContentRegistry; import org.elasticsearch.common.xcontent.ToXContent; @@ -24,9 +27,11 @@ import org.elasticsearch.common.xcontent.XContentType; import org.elasticsearch.common.xcontent.json.JsonXContent; import org.elasticsearch.test.rest.ESRestTestCase; +import org.elasticsearch.xpack.core.ilm.LifecycleSettings; import org.elasticsearch.xpack.core.ilm.Step; import org.elasticsearch.xpack.core.ilm.WaitForRolloverReadyStep; import org.elasticsearch.xpack.core.slm.SnapshotLifecyclePolicy; +import org.elasticsearch.xpack.core.slm.SnapshotRetentionConfiguration; import java.io.IOException; import java.io.InputStream; @@ -35,9 +40,11 @@ import java.util.HashMap; import java.util.List; import java.util.Map; -import java.util.Optional; +import java.util.concurrent.TimeUnit; import static org.elasticsearch.common.xcontent.XContentFactory.jsonBuilder; +import static org.elasticsearch.xpack.core.slm.history.SnapshotHistoryItem.CREATE_OPERATION; +import static org.elasticsearch.xpack.core.slm.history.SnapshotHistoryItem.DELETE_OPERATION; import static org.elasticsearch.xpack.core.slm.history.SnapshotHistoryStore.SLM_HISTORY_INDEX_PREFIX; import static org.elasticsearch.xpack.ilm.TimeSeriesLifecycleActionsIT.getStepKeyForIndex; import static org.hamcrest.Matchers.containsString; @@ -46,8 +53,7 @@ import static org.hamcrest.Matchers.greaterThanOrEqualTo; import static org.hamcrest.Matchers.startsWith; -@AwaitsFix(bugUrl = "https://github.com/elastic/elasticsearch/pull/46205") -public class SnapshotLifecycleIT extends ESRestTestCase { +public class SnapshotLifecycleRestIT extends ESRestTestCase { @Override protected boolean waitForAllSnapshotsWiped() { @@ -55,10 +61,8 @@ protected boolean waitForAllSnapshotsWiped() { } public void testMissingRepo() throws Exception { - final String policyId = "test-policy"; - final String missingRepoName = "missing-repo"; - SnapshotLifecyclePolicy policy = new SnapshotLifecyclePolicy(policyId, "snap", - "*/1 * * * * ?", missingRepoName, Collections.emptyMap()); + SnapshotLifecyclePolicy policy = new SnapshotLifecyclePolicy("test-policy", "snap", + "*/1 * * * * ?", "missing-repo", Collections.emptyMap(), SnapshotRetentionConfiguration.EMPTY); Request putLifecycle = new Request("PUT", "/_slm/policy/test-policy"); XContentBuilder lifecycleBuilder = JsonXContent.contentBuilder(); @@ -84,7 +88,7 @@ public void testFullPolicySnapshot() throws Exception { } // Create a snapshot repo - inializeRepo(repoId); + initializeRepo(repoId); createSnapshotPolicy(policyName, "snap", "*/1 * * * * ?", repoId, indexName, true); @@ -106,7 +110,7 @@ public void testFullPolicySnapshot() throws Exception { Map metadata = (Map) snapResponse.get(0).get("metadata"); assertNotNull(metadata); assertThat(metadata.get("policy"), equalTo(policyName)); - assertHistoryIsPresent(policyName, true, repoId); + assertHistoryIsPresent(policyName, true, repoId, CREATE_OPERATION); // Check that the last success date was written to the cluster state Request getReq = new Request("GET", "/_slm/policy/" + policyName); @@ -127,7 +131,15 @@ public void testFullPolicySnapshot() throws Exception { String lastSnapshotName = (String) lastSuccessObject.get("snapshot_name"); assertThat(lastSnapshotName, startsWith("snap-")); - assertHistoryIsPresent(policyName, true, repoId); + assertHistoryIsPresent(policyName, true, repoId, CREATE_OPERATION); + + Map stats = getSLMStats(); + Map policyStats = (Map) stats.get(SnapshotLifecycleStats.POLICY_STATS.getPreferredName()); + Map policyIdStats = (Map) policyStats.get(policyName); + int snapsTaken = (int) policyIdStats.get(SnapshotLifecycleStats.SnapshotPolicyStats.SNAPSHOTS_TAKEN.getPreferredName()); + int totalTaken = (int) stats.get(SnapshotLifecycleStats.TOTAL_TAKEN.getPreferredName()); + assertThat(snapsTaken, greaterThanOrEqualTo(1)); + assertThat(totalTaken, greaterThanOrEqualTo(1)); }); Request delReq = new Request("DELETE", "/_slm/policy/" + policyName); @@ -139,7 +151,7 @@ public void testPolicyFailure() throws Exception { final String policyName = "test-policy"; final String repoName = "test-repo"; final String indexPattern = "index-doesnt-exist"; - inializeRepo(repoName); + initializeRepo(repoName); // Create a policy with ignore_unvailable: false and an index that doesn't exist createSnapshotPolicy(policyName, "snap", "*/1 * * * * ?", repoName, indexPattern, false); @@ -168,10 +180,19 @@ public void testPolicyFailure() throws Exception { assertNotNull(snapshotName); assertThat(snapshotName, startsWith("snap-")); } - assertHistoryIsPresent(policyName, false, repoName); + assertHistoryIsPresent(policyName, false, repoName, CREATE_OPERATION); + + Map stats = getSLMStats(); + Map policyStats = (Map) stats.get(SnapshotLifecycleStats.POLICY_STATS.getPreferredName()); + Map policyIdStats = (Map) policyStats.get(policyName); + int snapsFailed = (int) policyIdStats.get(SnapshotLifecycleStats.SnapshotPolicyStats.SNAPSHOTS_FAILED.getPreferredName()); + int totalFailed = (int) stats.get(SnapshotLifecycleStats.TOTAL_FAILED.getPreferredName()); + assertThat(snapsFailed, greaterThanOrEqualTo(1)); + assertThat(totalFailed, greaterThanOrEqualTo(1)); }); } + @SuppressWarnings("unchecked") public void testPolicyManualExecution() throws Exception { final String indexName = "test"; final String policyName = "test-policy"; @@ -182,7 +203,7 @@ public void testPolicyManualExecution() throws Exception { } // Create a snapshot repo - inializeRepo(repoId); + initializeRepo(repoId); createSnapshotPolicy(policyName, "snap", "1 2 3 4 5 ?", repoId, indexName, true); @@ -191,82 +212,138 @@ public void testPolicyManualExecution() throws Exception { assertThat(EntityUtils.toString(badResp.getResponse().getEntity()), containsString("no such snapshot lifecycle policy [" + policyName + "-bad]")); - Response goodResp = client().performRequest(new Request("PUT", "/_slm/policy/" + policyName + "/_execute")); - - try (XContentParser parser = JsonXContent.jsonXContent.createParser(NamedXContentRegistry.EMPTY, - DeprecationHandler.THROW_UNSUPPORTED_OPERATION, EntityUtils.toByteArray(goodResp.getEntity()))) { - final String snapshotName = parser.mapStrings().get("snapshot_name"); + final String snapshotName = executePolicy(policyName); - // Check that the executed snapshot is created - assertBusy(() -> { - try { - Response response = client().performRequest(new Request("GET", "/_snapshot/" + repoId + "/" + snapshotName)); - Map snapshotResponseMap; - try (InputStream is = response.getEntity().getContent()) { - snapshotResponseMap = XContentHelper.convertToMap(XContentType.JSON.xContent(), is, true); - } - assertThat(snapshotResponseMap.size(), greaterThan(0)); - final Map metadata = extractMetadata(snapshotResponseMap, snapshotName); - assertNotNull(metadata); - assertThat(metadata.get("policy"), equalTo(policyName)); - assertHistoryIsPresent(policyName, true, repoId); - } catch (ResponseException e) { - fail("expected snapshot to exist but it does not: " + EntityUtils.toString(e.getResponse().getEntity())); + // Check that the executed snapshot is created + assertBusy(() -> { + try { + Response response = client().performRequest(new Request("GET", "/_snapshot/" + repoId + "/" + snapshotName)); + Map snapshotResponseMap; + try (InputStream is = response.getEntity().getContent()) { + snapshotResponseMap = XContentHelper.convertToMap(XContentType.JSON.xContent(), is, true); } - }); - } + assertThat(snapshotResponseMap.size(), greaterThan(0)); + final Map metadata = extractMetadata(snapshotResponseMap, snapshotName); + assertNotNull(metadata); + assertThat(metadata.get("policy"), equalTo(policyName)); + assertHistoryIsPresent(policyName, true, repoId, CREATE_OPERATION); + } catch (ResponseException e) { + fail("expected snapshot to exist but it does not: " + EntityUtils.toString(e.getResponse().getEntity())); + } + + Map stats = getSLMStats(); + Map policyStats = (Map) stats.get(SnapshotLifecycleStats.POLICY_STATS.getPreferredName()); + Map policyIdStats = (Map) policyStats.get(policyName); + int snapsTaken = (int) policyIdStats.get(SnapshotLifecycleStats.SnapshotPolicyStats.SNAPSHOTS_TAKEN.getPreferredName()); + int totalTaken = (int) stats.get(SnapshotLifecycleStats.TOTAL_TAKEN.getPreferredName()); + assertThat(snapsTaken, equalTo(1)); + assertThat(totalTaken, equalTo(1)); + }); } @SuppressWarnings("unchecked") - public void testSnapshotInProgress() throws Exception { + public void testBasicTimeBasedRetenion() throws Exception { final String indexName = "test"; final String policyName = "test-policy"; final String repoId = "my-repo"; - int docCount = 20; + int docCount = randomIntBetween(10, 50); + List indexReqs = new ArrayList<>(); for (int i = 0; i < docCount; i++) { index(client(), indexName, "" + i, "foo", "bar"); } // Create a snapshot repo - inializeRepo(repoId, 1); + initializeRepo(repoId); - createSnapshotPolicy(policyName, "snap", "1 2 3 4 5 ?", repoId, indexName, true); + // Create a policy with a retention period of 1 millisecond + createSnapshotPolicy(policyName, "snap", "1 2 3 4 5 ?", repoId, indexName, true, + new SnapshotRetentionConfiguration(TimeValue.timeValueMillis(1), null, null)); + + // Manually create a snapshot + final String snapshotName = executePolicy(policyName); - Response executeRepsonse = client().performRequest(new Request("PUT", "/_slm/policy/" + policyName + "/_execute")); + // Check that the executed snapshot is created + assertBusy(() -> { + try { + Response response = client().performRequest(new Request("GET", "/_snapshot/" + repoId + "/" + snapshotName)); + Map snapshotResponseMap; + try (InputStream is = response.getEntity().getContent()) { + snapshotResponseMap = XContentHelper.convertToMap(XContentType.JSON.xContent(), is, true); + } + assertThat(snapshotResponseMap.size(), greaterThan(0)); + final Map metadata = extractMetadata(snapshotResponseMap, snapshotName); + assertNotNull(metadata); + assertThat(metadata.get("policy"), equalTo(policyName)); + assertHistoryIsPresent(policyName, true, repoId, CREATE_OPERATION); + } catch (ResponseException e) { + fail("expected snapshot to exist but it does not: " + EntityUtils.toString(e.getResponse().getEntity())); + } + }); - try (XContentParser parser = JsonXContent.jsonXContent.createParser(NamedXContentRegistry.EMPTY, - DeprecationHandler.THROW_UNSUPPORTED_OPERATION, EntityUtils.toByteArray(executeRepsonse.getEntity()))) { - final String snapshotName = parser.mapStrings().get("snapshot_name"); + // Run retention every second + ClusterUpdateSettingsRequest req = new ClusterUpdateSettingsRequest(); + req.transientSettings(Settings.builder().put(LifecycleSettings.SLM_RETENTION_SCHEDULE, "*/1 * * * * ?")); + try (XContentBuilder builder = jsonBuilder()) { + req.toXContent(builder, ToXContent.EMPTY_PARAMS); + Request r = new Request("PUT", "/_cluster/settings"); + r.setJsonEntity(Strings.toString(builder)); + Response updateSettingsResp = client().performRequest(r); + } - // Check that the executed snapshot shows up in the SLM output + try { + // Check that the snapshot created by the policy has been removed by retention assertBusy(() -> { + // We expect a failed response because the snapshot should not exist try { - Response response = client().performRequest(new Request("GET", "/_slm/policy" + (randomBoolean() ? "" : "?human"))); - Map policyResponseMap; - try (InputStream content = response.getEntity().getContent()) { - policyResponseMap = XContentHelper.convertToMap(XContentType.JSON.xContent(), content, true); - } - assertThat(policyResponseMap.size(), greaterThan(0)); - Optional> inProgress = Optional.ofNullable((Map) policyResponseMap.get(policyName)) - .map(policy -> (Map) policy.get("in_progress")); - - if (inProgress.isPresent()) { - Map inProgressMap = inProgress.get(); - assertThat(inProgressMap.get("name"), equalTo(snapshotName)); - assertNotNull(inProgressMap.get("uuid")); - assertThat(inProgressMap.get("state"), equalTo("STARTED")); - assertThat((long) inProgressMap.get("start_time_millis"), greaterThan(0L)); - assertNull(inProgressMap.get("failure")); - } else { - fail("expected in_progress to contain a running snapshot, but the response was " + policyResponseMap); - } + logger.info("--> checking to see if snapshot has been deleted..."); + Response response = client().performRequest(new Request("GET", "/_snapshot/" + repoId + "/" + snapshotName)); + assertThat(EntityUtils.toString(response.getEntity()), containsString("snapshot_missing_exception")); } catch (ResponseException e) { - fail("expected policy to exist but it does not: " + EntityUtils.toString(e.getResponse().getEntity())); + assertThat(EntityUtils.toString(e.getResponse().getEntity()), containsString("snapshot_missing_exception")); } - }); + assertHistoryIsPresent(policyName, true, repoId, DELETE_OPERATION); + + Map stats = getSLMStats(); + Map policyStats = (Map) stats.get(SnapshotLifecycleStats.POLICY_STATS.getPreferredName()); + Map policyIdStats = (Map) policyStats.get(policyName); + int snapsTaken = (int) policyIdStats.get(SnapshotLifecycleStats.SnapshotPolicyStats.SNAPSHOTS_TAKEN.getPreferredName()); + int snapsDeleted = (int) policyIdStats.get(SnapshotLifecycleStats.SnapshotPolicyStats.SNAPSHOTS_DELETED.getPreferredName()); + int retentionRun = (int) stats.get(SnapshotLifecycleStats.RETENTION_RUNS.getPreferredName()); + int totalTaken = (int) stats.get(SnapshotLifecycleStats.TOTAL_TAKEN.getPreferredName()); + int totalDeleted = (int) stats.get(SnapshotLifecycleStats.TOTAL_DELETIONS.getPreferredName()); + assertThat(snapsTaken, equalTo(1)); + assertThat(totalTaken, equalTo(1)); + assertThat(retentionRun, greaterThanOrEqualTo(1)); + assertThat(snapsDeleted, greaterThanOrEqualTo(1)); + assertThat(totalDeleted, greaterThanOrEqualTo(1)); + }, 60, TimeUnit.SECONDS); + + } finally { + // Unset retention + ClusterUpdateSettingsRequest unsetRequest = new ClusterUpdateSettingsRequest(); + unsetRequest.transientSettings(Settings.builder().put(LifecycleSettings.SLM_RETENTION_SCHEDULE, (String) null)); + try (XContentBuilder builder = jsonBuilder()) { + unsetRequest.toXContent(builder, ToXContent.EMPTY_PARAMS); + Request r = new Request("PUT", "/_cluster/settings"); + r.setJsonEntity(Strings.toString(builder)); + client().performRequest(r); + } + } + } - // Cancel the snapshot since it is not going to complete quickly - assertOK(client().performRequest(new Request("DELETE", "/_snapshot/" + repoId + "/" + snapshotName))); + /** + * Execute the given policy and return the generated snapshot name + */ + private String executePolicy(String policyId) { + try { + Response executeRepsonse = client().performRequest(new Request("PUT", "/_slm/policy/" + policyId + "/_execute")); + try (XContentParser parser = JsonXContent.jsonXContent.createParser(NamedXContentRegistry.EMPTY, + DeprecationHandler.THROW_UNSUPPORTED_OPERATION, EntityUtils.toByteArray(executeRepsonse.getEntity()))) { + return parser.mapStrings().get("snapshot_name"); + } + } catch (Exception e) { + fail("failed to execute policy " + policyId + " - got: " + e); + throw new RuntimeException(e); } } @@ -283,8 +360,20 @@ private static Map extractMetadata(Map snapshotR .orElse(null); } + private Map getSLMStats() { + try { + Response response = client().performRequest(new Request("GET", "/_slm/stats")); + try (InputStream content = response.getEntity().getContent()) { + return XContentHelper.convertToMap(XContentType.JSON.xContent(), content, true); + } + } catch (Exception e) { + fail("exception retrieving stats: " + e); + throw new ElasticsearchException(e); + } + } + // This method should be called inside an assertBusy, it has no retry logic of its own - private void assertHistoryIsPresent(String policyName, boolean success, String repository) throws IOException { + private void assertHistoryIsPresent(String policyName, boolean success, String repository, String operation) throws IOException { final Request historySearchRequest = new Request("GET", ".slm-history*/_search"); historySearchRequest.setJsonEntity("{\n" + " \"query\": {\n" + @@ -307,7 +396,7 @@ private void assertHistoryIsPresent(String policyName, boolean success, String r " },\n" + " {\n" + " \"term\": {\n" + - " \"operation\": \"CREATE\"\n" + + " \"operation\": \"" + operation + "\"\n" + " }\n" + " }\n" + " ]\n" + @@ -342,6 +431,13 @@ private void assertHistoryIndexWaitingForRollover() throws IOException { private void createSnapshotPolicy(String policyName, String snapshotNamePattern, String schedule, String repoId, String indexPattern, boolean ignoreUnavailable) throws IOException { + createSnapshotPolicy(policyName, snapshotNamePattern, schedule, repoId, indexPattern, + ignoreUnavailable, SnapshotRetentionConfiguration.EMPTY); + } + + private void createSnapshotPolicy(String policyName, String snapshotNamePattern, String schedule, String repoId, + String indexPattern, boolean ignoreUnavailable, + SnapshotRetentionConfiguration retention) throws IOException { Map snapConfig = new HashMap<>(); snapConfig.put("indices", Collections.singletonList(indexPattern)); snapConfig.put("ignore_unavailable", ignoreUnavailable); @@ -353,7 +449,8 @@ private void createSnapshotPolicy(String policyName, String snapshotNamePattern, () -> randomAlphaOfLength(5)), randomAlphaOfLength(4)); } } - SnapshotLifecyclePolicy policy = new SnapshotLifecyclePolicy(policyName, snapshotNamePattern, schedule, repoId, snapConfig); + SnapshotLifecyclePolicy policy = new SnapshotLifecyclePolicy(policyName, snapshotNamePattern, schedule, + repoId, snapConfig, retention); Request putLifecycle = new Request("PUT", "/_slm/policy/" + policyName); XContentBuilder lifecycleBuilder = JsonXContent.contentBuilder(); @@ -362,11 +459,11 @@ private void createSnapshotPolicy(String policyName, String snapshotNamePattern, assertOK(client().performRequest(putLifecycle)); } - private void inializeRepo(String repoName) throws IOException { - inializeRepo(repoName, 256); + private void initializeRepo(String repoName) throws IOException { + initializeRepo(repoName, "40mb"); } - private void inializeRepo(String repoName, int maxBytesPerSecond) throws IOException { + private void initializeRepo(String repoName, String maxBytesPerSecond) throws IOException { Request request = new Request("PUT", "/_snapshot/" + repoName); request.setJsonEntity(Strings .toString(JsonXContent.contentBuilder() @@ -375,7 +472,7 @@ private void inializeRepo(String repoName, int maxBytesPerSecond) throws IOExcep .startObject("settings") .field("compress", randomBoolean()) .field("location", System.getProperty("tests.path.repo")) - .field("max_snapshot_bytes_per_sec", maxBytesPerSecond + "b") + .field("max_snapshot_bytes_per_sec", maxBytesPerSecond) .endObject() .endObject())); assertOK(client().performRequest(request)); diff --git a/x-pack/plugin/ilm/qa/with-security/src/test/java/org/elasticsearch/xpack/security/PermissionsIT.java b/x-pack/plugin/ilm/qa/with-security/src/test/java/org/elasticsearch/xpack/security/PermissionsIT.java index ed3128c85c978..b69584ddef866 100644 --- a/x-pack/plugin/ilm/qa/with-security/src/test/java/org/elasticsearch/xpack/security/PermissionsIT.java +++ b/x-pack/plugin/ilm/qa/with-security/src/test/java/org/elasticsearch/xpack/security/PermissionsIT.java @@ -25,6 +25,7 @@ import org.elasticsearch.client.slm.GetSnapshotLifecyclePolicyRequest; import org.elasticsearch.client.slm.PutSnapshotLifecyclePolicyRequest; import org.elasticsearch.client.slm.SnapshotLifecyclePolicy; +import org.elasticsearch.client.slm.SnapshotRetentionConfiguration; import org.elasticsearch.common.Strings; import org.elasticsearch.common.settings.SecureString; import org.elasticsearch.common.settings.Settings; @@ -190,7 +191,7 @@ public void testSLMWithPermissions() throws Exception { Map config = new HashMap<>(); config.put("indices", Collections.singletonList("index")); SnapshotLifecyclePolicy policy = new SnapshotLifecyclePolicy( - "policy_id", "name", "1 2 3 * * ?", "my_repository", config); + "policy_id", "name", "1 2 3 * * ?", "my_repository", config, SnapshotRetentionConfiguration.EMPTY); PutSnapshotLifecyclePolicyRequest request = new PutSnapshotLifecyclePolicyRequest(policy); expectThrows(ElasticsearchStatusException.class, diff --git a/x-pack/plugin/ilm/src/main/java/org/elasticsearch/xpack/ilm/IndexLifecycle.java b/x-pack/plugin/ilm/src/main/java/org/elasticsearch/xpack/ilm/IndexLifecycle.java index 82d09f7eeeede..b210c8f992964 100644 --- a/x-pack/plugin/ilm/src/main/java/org/elasticsearch/xpack/ilm/IndexLifecycle.java +++ b/x-pack/plugin/ilm/src/main/java/org/elasticsearch/xpack/ilm/IndexLifecycle.java @@ -65,6 +65,7 @@ import org.elasticsearch.xpack.core.slm.action.DeleteSnapshotLifecycleAction; import org.elasticsearch.xpack.core.slm.action.ExecuteSnapshotLifecycleAction; import org.elasticsearch.xpack.core.slm.action.GetSnapshotLifecycleAction; +import org.elasticsearch.xpack.core.slm.action.GetSnapshotLifecycleStatsAction; import org.elasticsearch.xpack.core.slm.action.PutSnapshotLifecycleAction; import org.elasticsearch.xpack.core.slm.history.SnapshotHistoryStore; import org.elasticsearch.xpack.core.slm.history.SnapshotLifecycleTemplateRegistry; @@ -90,13 +91,17 @@ import org.elasticsearch.xpack.ilm.action.TransportStopILMAction; import org.elasticsearch.xpack.slm.SnapshotLifecycleService; import org.elasticsearch.xpack.slm.SnapshotLifecycleTask; +import org.elasticsearch.xpack.slm.SnapshotRetentionService; +import org.elasticsearch.xpack.slm.SnapshotRetentionTask; import org.elasticsearch.xpack.slm.action.RestDeleteSnapshotLifecycleAction; import org.elasticsearch.xpack.slm.action.RestExecuteSnapshotLifecycleAction; import org.elasticsearch.xpack.slm.action.RestGetSnapshotLifecycleAction; +import org.elasticsearch.xpack.slm.action.RestGetSnapshotLifecycleStatsAction; import org.elasticsearch.xpack.slm.action.RestPutSnapshotLifecycleAction; import org.elasticsearch.xpack.slm.action.TransportDeleteSnapshotLifecycleAction; import org.elasticsearch.xpack.slm.action.TransportExecuteSnapshotLifecycleAction; import org.elasticsearch.xpack.slm.action.TransportGetSnapshotLifecycleAction; +import org.elasticsearch.xpack.slm.action.TransportGetSnapshotLifecycleStatsAction; import org.elasticsearch.xpack.slm.action.TransportPutSnapshotLifecycleAction; import java.io.IOException; @@ -113,6 +118,7 @@ public class IndexLifecycle extends Plugin implements ActionPlugin { private final SetOnce indexLifecycleInitialisationService = new SetOnce<>(); private final SetOnce snapshotLifecycleService = new SetOnce<>(); + private final SetOnce snapshotRetentionService = new SetOnce<>(); private final SetOnce snapshotHistoryStore = new SetOnce<>(); private Settings settings; private boolean enabled; @@ -134,7 +140,8 @@ public List> getSettings() { LifecycleSettings.LIFECYCLE_NAME_SETTING, LifecycleSettings.LIFECYCLE_INDEXING_COMPLETE_SETTING, RolloverAction.LIFECYCLE_ROLLOVER_ALIAS_SETTING, - LifecycleSettings.SLM_HISTORY_INDEX_ENABLED_SETTING); + LifecycleSettings.SLM_HISTORY_INDEX_ENABLED_SETTING, + LifecycleSettings.SLM_RETENTION_SCHEDULE_SETTING); } @Override @@ -153,7 +160,11 @@ public Collection createComponents(Client client, ClusterService cluster )); snapshotLifecycleService.set(new SnapshotLifecycleService(settings, () -> new SnapshotLifecycleTask(client, clusterService, snapshotHistoryStore.get()), clusterService, getClock())); - return Arrays.asList(indexLifecycleInitialisationService.get(), snapshotLifecycleService.get(), snapshotHistoryStore.get()); + snapshotRetentionService.set(new SnapshotRetentionService(settings, + () -> new SnapshotRetentionTask(client, clusterService, System::nanoTime, snapshotHistoryStore.get(), threadPool), + clusterService, getClock())); + return Arrays.asList(indexLifecycleInitialisationService.get(), snapshotLifecycleService.get(), snapshotHistoryStore.get(), + snapshotRetentionService.get()); } @Override @@ -207,7 +218,8 @@ public List getRestHandlers(Settings settings, RestController restC new RestPutSnapshotLifecycleAction(restController), new RestDeleteSnapshotLifecycleAction(restController), new RestGetSnapshotLifecycleAction(restController), - new RestExecuteSnapshotLifecycleAction(restController) + new RestExecuteSnapshotLifecycleAction(restController), + new RestGetSnapshotLifecycleStatsAction(restController) ); } @@ -236,6 +248,7 @@ public List getRestHandlers(Settings settings, RestController restC new ActionHandler<>(DeleteSnapshotLifecycleAction.INSTANCE, TransportDeleteSnapshotLifecycleAction.class), new ActionHandler<>(GetSnapshotLifecycleAction.INSTANCE, TransportGetSnapshotLifecycleAction.class), new ActionHandler<>(ExecuteSnapshotLifecycleAction.INSTANCE, TransportExecuteSnapshotLifecycleAction.class), + new ActionHandler<>(GetSnapshotLifecycleStatsAction.INSTANCE, TransportGetSnapshotLifecycleStatsAction.class), usageAction, infoAction); } @@ -243,7 +256,7 @@ public List getRestHandlers(Settings settings, RestController restC @Override public void close() { try { - IOUtils.close(indexLifecycleInitialisationService.get(), snapshotLifecycleService.get()); + IOUtils.close(indexLifecycleInitialisationService.get(), snapshotLifecycleService.get(), snapshotRetentionService.get()); } catch (IOException e) { throw new ElasticsearchException("unable to close index lifecycle services", e); } diff --git a/x-pack/plugin/ilm/src/main/java/org/elasticsearch/xpack/ilm/OperationModeUpdateTask.java b/x-pack/plugin/ilm/src/main/java/org/elasticsearch/xpack/ilm/OperationModeUpdateTask.java index c899a51c28f60..53d4a5307b0dc 100644 --- a/x-pack/plugin/ilm/src/main/java/org/elasticsearch/xpack/ilm/OperationModeUpdateTask.java +++ b/x-pack/plugin/ilm/src/main/java/org/elasticsearch/xpack/ilm/OperationModeUpdateTask.java @@ -74,7 +74,8 @@ private ClusterState updateSLMState(final ClusterState currentState) { return ClusterState.builder(currentState) .metaData(MetaData.builder(currentState.metaData()) .putCustom(SnapshotLifecycleMetadata.TYPE, - new SnapshotLifecycleMetadata(currentMetadata.getSnapshotConfigurations(), newMode))) + new SnapshotLifecycleMetadata(currentMetadata.getSnapshotConfigurations(), + newMode, currentMetadata.getStats()))) .build(); } diff --git a/x-pack/plugin/ilm/src/main/java/org/elasticsearch/xpack/slm/SnapshotLifecycleService.java b/x-pack/plugin/ilm/src/main/java/org/elasticsearch/xpack/slm/SnapshotLifecycleService.java index d2435324ea5d6..0d27584d83eb8 100644 --- a/x-pack/plugin/ilm/src/main/java/org/elasticsearch/xpack/slm/SnapshotLifecycleService.java +++ b/x-pack/plugin/ilm/src/main/java/org/elasticsearch/xpack/slm/SnapshotLifecycleService.java @@ -104,7 +104,7 @@ SchedulerEngine getScheduler() { /** * Returns true if ILM is in the stopped or stopped state */ - private static boolean ilmStoppedOrStopping(ClusterState state) { + static boolean ilmStoppedOrStopping(ClusterState state) { return Optional.ofNullable((SnapshotLifecycleMetadata) state.metaData().custom(SnapshotLifecycleMetadata.TYPE)) .map(SnapshotLifecycleMetadata::getOperationMode) .map(mode -> OperationMode.STOPPING == mode || OperationMode.STOPPED == mode) diff --git a/x-pack/plugin/ilm/src/main/java/org/elasticsearch/xpack/slm/SnapshotLifecycleTask.java b/x-pack/plugin/ilm/src/main/java/org/elasticsearch/xpack/slm/SnapshotLifecycleTask.java index 7b5f2bcec415a..993dac40252e6 100644 --- a/x-pack/plugin/ilm/src/main/java/org/elasticsearch/xpack/slm/SnapshotLifecycleTask.java +++ b/x-pack/plugin/ilm/src/main/java/org/elasticsearch/xpack/slm/SnapshotLifecycleTask.java @@ -94,7 +94,8 @@ public void onResponse(CreateSnapshotResponse createSnapshotResponse) { final long timestamp = Instant.now().toEpochMilli(); clusterService.submitStateUpdateTask("slm-record-success-" + policyMetadata.getPolicy().getId(), WriteJobStatus.success(policyMetadata.getPolicy().getId(), request.snapshot(), timestamp)); - historyStore.putAsync(SnapshotHistoryItem.successRecord(timestamp, policyMetadata.getPolicy(), request.snapshot())); + historyStore.putAsync(SnapshotHistoryItem.creationSuccessRecord(timestamp, policyMetadata.getPolicy(), + request.snapshot())); } @Override @@ -106,7 +107,8 @@ public void onFailure(Exception e) { WriteJobStatus.failure(policyMetadata.getPolicy().getId(), request.snapshot(), timestamp, e)); final SnapshotHistoryItem failureRecord; try { - failureRecord = SnapshotHistoryItem.failureRecord(timestamp, policyMetadata.getPolicy(), request.snapshot(), e); + failureRecord = SnapshotHistoryItem.creationFailureRecord(timestamp, policyMetadata.getPolicy(), + request.snapshot(), e); historyStore.putAsync(failureRecord); } catch (IOException ex) { // This shouldn't happen unless there's an issue with serializing the original exception, which shouldn't happen @@ -192,15 +194,19 @@ public ClusterState execute(ClusterState currentState) throws Exception { } SnapshotLifecyclePolicyMetadata.Builder newPolicyMetadata = SnapshotLifecyclePolicyMetadata.builder(policyMetadata); + final SnapshotLifecycleStats stats = snapMeta.getStats(); if (exception.isPresent()) { + stats.snapshotFailed(policyName); newPolicyMetadata.setLastFailure(new SnapshotInvocationRecord(snapshotName, timestamp, exceptionToString())); } else { + stats.snapshotTaken(policyName); newPolicyMetadata.setLastSuccess(new SnapshotInvocationRecord(snapshotName, timestamp, null)); } snapLifecycles.put(policyName, newPolicyMetadata.build()); - SnapshotLifecycleMetadata lifecycleMetadata = new SnapshotLifecycleMetadata(snapLifecycles, snapMeta.getOperationMode()); + SnapshotLifecycleMetadata lifecycleMetadata = new SnapshotLifecycleMetadata(snapLifecycles, + snapMeta.getOperationMode(), stats); MetaData currentMeta = currentState.metaData(); return ClusterState.builder(currentState) .metaData(MetaData.builder(currentMeta) diff --git a/x-pack/plugin/ilm/src/main/java/org/elasticsearch/xpack/slm/SnapshotRetentionService.java b/x-pack/plugin/ilm/src/main/java/org/elasticsearch/xpack/slm/SnapshotRetentionService.java new file mode 100644 index 0000000000000..36a60ffdf9365 --- /dev/null +++ b/x-pack/plugin/ilm/src/main/java/org/elasticsearch/xpack/slm/SnapshotRetentionService.java @@ -0,0 +1,103 @@ +/* + * 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.slm; + +import org.apache.logging.log4j.LogManager; +import org.apache.logging.log4j.Logger; +import org.elasticsearch.cluster.LocalNodeMasterListener; +import org.elasticsearch.cluster.service.ClusterService; +import org.elasticsearch.common.Strings; +import org.elasticsearch.common.settings.Settings; +import org.elasticsearch.threadpool.ThreadPool; +import org.elasticsearch.xpack.core.ilm.LifecycleSettings; +import org.elasticsearch.xpack.core.scheduler.CronSchedule; +import org.elasticsearch.xpack.core.scheduler.SchedulerEngine; +import org.elasticsearch.xpack.core.slm.SnapshotLifecyclePolicy; + +import java.io.Closeable; +import java.time.Clock; +import java.util.function.Supplier; + +/** + * The {@code SnapshotRetentionService} is responsible for scheduling the period kickoff of SLM's + * snapshot retention. This means that when the retention schedule setting is configured, the + * scheduler schedules a job that, when triggered, will delete snapshots according to the retention + * policy configured in the {@link SnapshotLifecyclePolicy}. + */ +public class SnapshotRetentionService implements LocalNodeMasterListener, Closeable { + + static final String SLM_RETENTION_JOB_ID = "slm-retention-job"; + + private static final Logger logger = LogManager.getLogger(SnapshotRetentionService.class); + + private final SchedulerEngine scheduler; + + private volatile String slmRetentionSchedule; + private volatile boolean isMaster = false; + + public SnapshotRetentionService(Settings settings, + Supplier taskSupplier, + ClusterService clusterService, + Clock clock) { + this.scheduler = new SchedulerEngine(settings, clock); + this.scheduler.register(taskSupplier.get()); + this.slmRetentionSchedule = LifecycleSettings.SLM_RETENTION_SCHEDULE_SETTING.get(settings); + clusterService.addLocalNodeMasterListener(this); + clusterService.getClusterSettings().addSettingsUpdateConsumer(LifecycleSettings.SLM_RETENTION_SCHEDULE_SETTING, + this::setUpdateSchedule); + } + + void setUpdateSchedule(String retentionSchedule) { + this.slmRetentionSchedule = retentionSchedule; + // The schedule has changed, so reschedule the retention job + rescheduleRetentionJob(); + } + + // Only used for testing + SchedulerEngine getScheduler() { + return this.scheduler; + } + + @Override + public void onMaster() { + this.isMaster = true; + rescheduleRetentionJob(); + } + + @Override + public void offMaster() { + this.isMaster = false; + cancelRetentionJob(); + } + + private void rescheduleRetentionJob() { + final String schedule = this.slmRetentionSchedule; + if (this.isMaster && Strings.hasText(schedule)) { + final SchedulerEngine.Job retentionJob = new SchedulerEngine.Job(SLM_RETENTION_JOB_ID, + new CronSchedule(schedule)); + logger.debug("scheduling SLM retention job for [{}]", schedule); + this.scheduler.add(retentionJob); + } else { + // The schedule has been unset, so cancel the scheduled retention job + cancelRetentionJob(); + } + } + + private void cancelRetentionJob() { + this.scheduler.scheduledJobIds().forEach(this.scheduler::remove); + } + + @Override + public String executorName() { + return ThreadPool.Names.SNAPSHOT; + } + + @Override + public void close() { + this.scheduler.stop(); + } +} diff --git a/x-pack/plugin/ilm/src/main/java/org/elasticsearch/xpack/slm/SnapshotRetentionTask.java b/x-pack/plugin/ilm/src/main/java/org/elasticsearch/xpack/slm/SnapshotRetentionTask.java new file mode 100644 index 0000000000000..27c40fbe15ca9 --- /dev/null +++ b/x-pack/plugin/ilm/src/main/java/org/elasticsearch/xpack/slm/SnapshotRetentionTask.java @@ -0,0 +1,491 @@ +/* + * 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.slm; + +import org.apache.logging.log4j.LogManager; +import org.apache.logging.log4j.Logger; +import org.apache.logging.log4j.message.ParameterizedMessage; +import org.elasticsearch.ElasticsearchException; +import org.elasticsearch.action.ActionListener; +import org.elasticsearch.action.LatchedActionListener; +import org.elasticsearch.action.admin.cluster.snapshots.get.GetSnapshotsResponse; +import org.elasticsearch.action.support.master.AcknowledgedResponse; +import org.elasticsearch.client.Client; +import org.elasticsearch.client.OriginSettingClient; +import org.elasticsearch.cluster.ClusterState; +import org.elasticsearch.cluster.ClusterStateObserver; +import org.elasticsearch.cluster.RepositoryCleanupInProgress; +import org.elasticsearch.cluster.RestoreInProgress; +import org.elasticsearch.cluster.SnapshotDeletionsInProgress; +import org.elasticsearch.cluster.SnapshotsInProgress; +import org.elasticsearch.cluster.service.ClusterService; +import org.elasticsearch.common.Strings; +import org.elasticsearch.common.unit.TimeValue; +import org.elasticsearch.snapshots.SnapshotId; +import org.elasticsearch.snapshots.SnapshotInfo; +import org.elasticsearch.snapshots.SnapshotState; +import org.elasticsearch.threadpool.ThreadPool; +import org.elasticsearch.xpack.core.ClientHelper; +import org.elasticsearch.xpack.core.ilm.LifecycleSettings; +import org.elasticsearch.xpack.core.scheduler.SchedulerEngine; +import org.elasticsearch.xpack.core.slm.SnapshotLifecycleMetadata; +import org.elasticsearch.xpack.core.slm.SnapshotLifecyclePolicy; +import org.elasticsearch.xpack.core.slm.SnapshotRetentionConfiguration; +import org.elasticsearch.xpack.core.slm.history.SnapshotHistoryItem; +import org.elasticsearch.xpack.core.slm.history.SnapshotHistoryStore; + +import java.io.IOException; +import java.time.Instant; +import java.util.Collection; +import java.util.Collections; +import java.util.HashMap; +import java.util.List; +import java.util.Map; +import java.util.Optional; +import java.util.Set; +import java.util.concurrent.CountDownLatch; +import java.util.concurrent.atomic.AtomicBoolean; +import java.util.concurrent.atomic.AtomicInteger; +import java.util.function.Consumer; +import java.util.function.LongSupplier; +import java.util.stream.Collectors; + +import static org.elasticsearch.xpack.core.slm.SnapshotLifecyclePolicy.POLICY_ID_METADATA_FIELD; + +/** + * The {@code SnapshotRetentionTask} is invoked by the scheduled job from the + * {@link SnapshotRetentionService}. It is responsible for retrieving the snapshots for repositories + * that have an SLM policy configured, and then deleting the snapshots that fall outside the + * retention policy. + */ +public class SnapshotRetentionTask implements SchedulerEngine.Listener { + + private static final Logger logger = LogManager.getLogger(SnapshotRetentionTask.class); + private static final AtomicBoolean running = new AtomicBoolean(false); + + private final Client client; + private final ClusterService clusterService; + private final LongSupplier nowNanoSupplier; + private final ThreadPool threadPool; + private final SnapshotHistoryStore historyStore; + + public SnapshotRetentionTask(Client client, ClusterService clusterService, LongSupplier nowNanoSupplier, + SnapshotHistoryStore historyStore, ThreadPool threadPool) { + this.client = new OriginSettingClient(client, ClientHelper.INDEX_LIFECYCLE_ORIGIN); + this.clusterService = clusterService; + this.nowNanoSupplier = nowNanoSupplier; + this.historyStore = historyStore; + this.threadPool = threadPool; + } + + @Override + public void triggered(SchedulerEngine.Event event) { + assert event.getJobName().equals(SnapshotRetentionService.SLM_RETENTION_JOB_ID) : + "expected id to be " + SnapshotRetentionService.SLM_RETENTION_JOB_ID + " but it was " + event.getJobName(); + + final ClusterState state = clusterService.state(); + if (SnapshotLifecycleService.ilmStoppedOrStopping(state)) { + logger.debug("skipping SLM retention as ILM is currently stopped or stopping"); + return; + } + + if (running.compareAndSet(false, true)) { + final SnapshotLifecycleStats slmStats = new SnapshotLifecycleStats(); + + // Defined here so it can be re-used without having to repeat it + final Consumer failureHandler = e -> { + try { + logger.error("error during snapshot retention task", e); + slmStats.retentionFailed(); + updateStateWithStats(slmStats); + } finally { + running.set(false); + } + }; + + try { + final TimeValue maxDeletionTime = LifecycleSettings.SLM_RETENTION_DURATION_SETTING.get(state.metaData().settings()); + + logger.info("starting SLM retention snapshot cleanup task"); + slmStats.retentionRun(); + // Find all SLM policies that have retention enabled + final Map policiesWithRetention = getAllPoliciesWithRetentionEnabled(state); + + // For those policies (there may be more than one for the same repo), + // return the repos that we need to get the snapshots for + final Set repositioriesToFetch = policiesWithRetention.values().stream() + .map(SnapshotLifecyclePolicy::getRepository) + .collect(Collectors.toSet()); + + if (repositioriesToFetch.isEmpty()) { + running.set(false); + return; + } + + // Finally, asynchronously retrieve all the snapshots, deleting them serially, + // before updating the cluster state with the new metrics and setting 'running' + // back to false + getAllSuccessfulSnapshots(repositioriesToFetch, new ActionListener<>() { + @Override + public void onResponse(Map> allSnapshots) { + try { + // Find all the snapshots that are past their retention date + final Map> snapshotsToBeDeleted = allSnapshots.entrySet().stream() + .collect(Collectors.toMap(Map.Entry::getKey, + e -> e.getValue().stream() + .filter(snapshot -> snapshotEligibleForDeletion(snapshot, allSnapshots, policiesWithRetention)) + .collect(Collectors.toList()))); + + // Finally, delete the snapshots that need to be deleted + maybeDeleteSnapshots(snapshotsToBeDeleted, maxDeletionTime, slmStats); + + updateStateWithStats(slmStats); + } finally { + running.set(false); + } + } + + @Override + public void onFailure(Exception e) { + failureHandler.accept(e); + } + }, failureHandler); + } catch (Exception e) { + failureHandler.accept(e); + } + } else { + logger.trace("snapshot lifecycle retention task started, but a task is already running, skipping"); + } + } + + static Map getAllPoliciesWithRetentionEnabled(final ClusterState state) { + final SnapshotLifecycleMetadata snapMeta = state.metaData().custom(SnapshotLifecycleMetadata.TYPE); + if (snapMeta == null) { + return Collections.emptyMap(); + } + return snapMeta.getSnapshotConfigurations().entrySet().stream() + .filter(e -> e.getValue().getPolicy().getRetentionPolicy() != null) + .filter(e -> e.getValue().getPolicy().getRetentionPolicy().equals(SnapshotRetentionConfiguration.EMPTY) == false) + .collect(Collectors.toMap(Map.Entry::getKey, e -> e.getValue().getPolicy())); + } + + static boolean snapshotEligibleForDeletion(SnapshotInfo snapshot, Map> allSnapshots, + Map policies) { + if (snapshot.userMetadata() == null) { + // This snapshot has no metadata, it is not eligible for deletion + return false; + } + + final String policyId; + try { + policyId = (String) snapshot.userMetadata().get(POLICY_ID_METADATA_FIELD); + } catch (Exception e) { + logger.debug("unable to retrieve policy id from snapshot metadata [" + snapshot.userMetadata() + "]", e); + return false; + } + + if (policyId == null) { + // policyId was null in the metadata, so it's not eligible + return false; + } + + SnapshotLifecyclePolicy policy = policies.get(policyId); + if (policy == null) { + // This snapshot was taking by a policy that doesn't exist, so it's not eligible + return false; + } + + SnapshotRetentionConfiguration retention = policy.getRetentionPolicy(); + if (retention == null || retention.equals(SnapshotRetentionConfiguration.EMPTY)) { + // Retention is not configured + return false; + } + + final String repository = policy.getRepository(); + // Retrieve the predicate based on the retention policy, passing in snapshots pertaining only to *this* policy and repository + boolean eligible = retention.getSnapshotDeletionPredicate( + allSnapshots.get(repository).stream() + .filter(info -> Optional.ofNullable(info.userMetadata()) + .map(meta -> meta.get(POLICY_ID_METADATA_FIELD)) + .map(pId -> pId.equals(policyId)) + .orElse(false)) + .collect(Collectors.toList())) + .test(snapshot); + logger.debug("[{}] testing snapshot [{}] deletion eligibility: {}", + repository, snapshot.snapshotId(), eligible ? "ELIGIBLE" : "INELIGIBLE"); + return eligible; + } + + void getAllSuccessfulSnapshots(Collection repositories, ActionListener>> listener, + Consumer errorHandler) { + if (repositories.isEmpty()) { + // Skip retrieving anything if there are no repositories to fetch + listener.onResponse(Collections.emptyMap()); + } + + client.admin().cluster() + .prepareGetSnapshots(repositories.toArray(Strings.EMPTY_ARRAY)) + .setIgnoreUnavailable(true) + .execute(new ActionListener<>() { + @Override + public void onResponse(final GetSnapshotsResponse resp) { + Map> snapshots = new HashMap<>(); + repositories.forEach(repo -> { + snapshots.put(repo, + // Only return snapshots in the SUCCESS state + resp.getSnapshots(repo).stream() + .filter(info -> info.state() == SnapshotState.SUCCESS) + .collect(Collectors.toList())); + }); + listener.onResponse(snapshots); + } + + @Override + public void onFailure(Exception e) { + logger.debug(new ParameterizedMessage("unable to retrieve snapshots for [{}] repositories", repositories), e); + errorHandler.accept(e); + } + }); + } + + static String getPolicyId(SnapshotInfo snapshotInfo) { + return Optional.ofNullable(snapshotInfo.userMetadata()) + .filter(meta -> meta.get(SnapshotLifecyclePolicy.POLICY_ID_METADATA_FIELD) != null) + .filter(meta -> meta.get(SnapshotLifecyclePolicy.POLICY_ID_METADATA_FIELD) instanceof String) + .map(meta -> (String) meta.get(SnapshotLifecyclePolicy.POLICY_ID_METADATA_FIELD)) + .orElseThrow(() -> new IllegalStateException("expected snapshot " + snapshotInfo + + " to have a policy in its metadata, but it did not")); + } + + /** + * Maybe delete the given snapshots. If a snapshot is currently running according to the cluster + * state, this waits (using a {@link ClusterStateObserver} until a cluster state with no running + * snapshots before executing the blocking + * {@link #deleteSnapshots(Map, TimeValue, SnapshotLifecycleStats)} request. At most, we wait + * for the maximum allowed deletion time before timing out waiting for a state with no + * running snapshots. + * + * It's possible the task may still run into a SnapshotInProgressException, if a snapshot is + * started between the state retrieved here and the actual deletion. Since is is expected to be + * a rare case, no special handling is present. + */ + private void maybeDeleteSnapshots(Map> snapshotsToDelete, + TimeValue maximumTime, + SnapshotLifecycleStats slmStats) { + int count = snapshotsToDelete.values().stream().mapToInt(List::size).sum(); + if (count == 0) { + logger.debug("no snapshots are eligible for deletion"); + return; + } + + ClusterState state = clusterService.state(); + if (okayToDeleteSnapshots(state)) { + deleteSnapshots(snapshotsToDelete, maximumTime, slmStats); + } else { + logger.debug("a snapshot is currently running, rescheduling SLM retention for after snapshot has completed"); + ClusterStateObserver observer = new ClusterStateObserver(clusterService, maximumTime, logger, threadPool.getThreadContext()); + CountDownLatch latch = new CountDownLatch(1); + observer.waitForNextChange( + new NoSnapshotRunningListener(observer, + newState -> threadPool.executor(ThreadPool.Names.MANAGEMENT).execute(() -> { + try { + deleteSnapshots(snapshotsToDelete, maximumTime, slmStats); + } finally { + latch.countDown(); + } + }), + e -> { + latch.countDown(); + throw new ElasticsearchException(e); + })); + try { + latch.await(); + } catch (InterruptedException e) { + throw new ElasticsearchException(e); + } + } + } + + void deleteSnapshots(Map> snapshotsToDelete, + TimeValue maximumTime, + SnapshotLifecycleStats slmStats) { + int count = snapshotsToDelete.values().stream().mapToInt(List::size).sum(); + + logger.info("starting snapshot retention deletion for [{}] snapshots", count); + long startTime = nowNanoSupplier.getAsLong(); + final AtomicInteger deleted = new AtomicInteger(0); + final AtomicInteger failed = new AtomicInteger(0); + for (Map.Entry> entry : snapshotsToDelete.entrySet()) { + String repo = entry.getKey(); + List snapshots = entry.getValue(); + for (SnapshotInfo info : snapshots) { + final String policyId = getPolicyId(info); + deleteSnapshot(policyId, repo, info.snapshotId(), slmStats, ActionListener.wrap(acknowledgedResponse -> { + deleted.incrementAndGet(); + if (acknowledgedResponse.isAcknowledged()) { + historyStore.putAsync(SnapshotHistoryItem.deletionSuccessRecord(Instant.now().toEpochMilli(), + info.snapshotId().getName(), policyId, repo)); + } else { + SnapshotHistoryItem.deletionPossibleSuccessRecord(Instant.now().toEpochMilli(), + info.snapshotId().getName(), policyId, repo, + "deletion request issued successfully, no acknowledgement received"); + } + }, e -> { + failed.incrementAndGet(); + try { + final SnapshotHistoryItem result = SnapshotHistoryItem.deletionFailureRecord(Instant.now().toEpochMilli(), + info.snapshotId().getName(), policyId, repo, e); + historyStore.putAsync(result); + } catch (IOException ex) { + // This shouldn't happen unless there's an issue with serializing the original exception + logger.error(new ParameterizedMessage( + "failed to record snapshot deletion failure for snapshot lifecycle policy [{}]", + policyId), ex); + } + })); + // Check whether we have exceeded the maximum time allowed to spend deleting + // snapshots, if we have, short-circuit the rest of the deletions + TimeValue elapsedDeletionTime = TimeValue.timeValueNanos(nowNanoSupplier.getAsLong() - startTime); + logger.debug("elapsed time for deletion of [{}] snapshot: {}", info.snapshotId(), elapsedDeletionTime); + if (elapsedDeletionTime.compareTo(maximumTime) > 0) { + logger.info("maximum snapshot retention deletion time reached, time spent: [{}]," + + " maximum allowed time: [{}], deleted [{}] out of [{}] snapshots scheduled for deletion, failed to delete [{}]", + elapsedDeletionTime, maximumTime, deleted, count, failed); + slmStats.deletionTime(elapsedDeletionTime); + slmStats.retentionTimedOut(); + return; + } + } + } + TimeValue totalElapsedTime = TimeValue.timeValueNanos(nowNanoSupplier.getAsLong() - startTime); + logger.debug("total elapsed time for deletion of [{}] snapshots: {}", deleted, totalElapsedTime); + slmStats.deletionTime(totalElapsedTime); + } + + /** + * Delete the given snapshot from the repository in blocking manner + * + * @param repo The repository the snapshot is in + * @param snapshot The snapshot metadata + * @param listener {@link ActionListener#onResponse(Object)} is called if a {@link SnapshotHistoryItem} can be created representing a + * successful or failed deletion call. {@link ActionListener#onFailure(Exception)} is called only if interrupted. + */ + void deleteSnapshot(String slmPolicy, String repo, SnapshotId snapshot, SnapshotLifecycleStats slmStats, + ActionListener listener) { + logger.info("[{}] snapshot retention deleting snapshot [{}]", repo, snapshot); + CountDownLatch latch = new CountDownLatch(1); + client.admin().cluster().prepareDeleteSnapshot(repo, snapshot.getName()) + .execute(new LatchedActionListener<>(new ActionListener<>() { + @Override + public void onResponse(AcknowledgedResponse acknowledgedResponse) { + if (acknowledgedResponse.isAcknowledged()) { + logger.debug("[{}] snapshot [{}] deleted successfully", repo, snapshot); + } else { + logger.warn("[{}] snapshot [{}] delete issued but the request was not acknowledged", repo, snapshot); + } + listener.onResponse(acknowledgedResponse); + slmStats.snapshotDeleted(slmPolicy); + } + + @Override + public void onFailure(Exception e) { + logger.warn(new ParameterizedMessage("[{}] failed to delete snapshot [{}] for retention", + repo, snapshot), e); + slmStats.snapshotDeleteFailure(slmPolicy); + listener.onFailure(e); + } + }, latch)); + try { + // Deletes cannot occur simultaneously, so wait for this + // deletion to complete before attempting the next one + latch.await(); + } catch (InterruptedException e) { + logger.error(new ParameterizedMessage("[{}] deletion of snapshot [{}] interrupted", + repo, snapshot), e); + listener.onFailure(e); + slmStats.snapshotDeleteFailure(slmPolicy); + } + } + + void updateStateWithStats(SnapshotLifecycleStats newStats) { + clusterService.submitStateUpdateTask("update_slm_stats", new UpdateSnapshotLifecycleStatsTask(newStats)); + } + + public static boolean okayToDeleteSnapshots(ClusterState state) { + // Cannot delete during a snapshot + final SnapshotsInProgress snapshotsInProgress = state.custom(SnapshotsInProgress.TYPE); + if (snapshotsInProgress != null && snapshotsInProgress.entries().size() > 0) { + return false; + } + + // Cannot delete during an existing delete + final SnapshotDeletionsInProgress deletionsInProgress = state.custom(SnapshotDeletionsInProgress.TYPE); + if (deletionsInProgress != null && deletionsInProgress.hasDeletionsInProgress()) { + return false; + } + + // Cannot delete while a repository is being cleaned + final RepositoryCleanupInProgress repositoryCleanupInProgress = state.custom(RepositoryCleanupInProgress.TYPE); + if (repositoryCleanupInProgress != null && repositoryCleanupInProgress.cleanupInProgress() == false) { + return false; + } + + // Cannot delete during a restore + final RestoreInProgress restoreInProgress = state.custom(RestoreInProgress.TYPE); + if (restoreInProgress != null) { + return false; + } + + // It's okay to delete snapshots + return true; + } + + /** + * A {@link ClusterStateObserver.Listener} that invokes the given function with the new state, + * once no snapshots are running. If a snapshot is still running it registers a new listener + * and tries again. Passes any exceptions to the original exception listener if they occur. + */ + class NoSnapshotRunningListener implements ClusterStateObserver.Listener { + + private final Consumer reRun; + private final Consumer exceptionConsumer; + private final ClusterStateObserver observer; + + NoSnapshotRunningListener(ClusterStateObserver observer, + Consumer reRun, + Consumer exceptionConsumer) { + this.observer = observer; + this.reRun = reRun; + this.exceptionConsumer = exceptionConsumer; + } + + @Override + public void onNewClusterState(ClusterState state) { + try { + if (okayToDeleteSnapshots(state)) { + logger.debug("retrying SLM snapshot retention deletion after snapshot operation has completed"); + reRun.accept(state); + } else { + observer.waitForNextChange(this); + } + } catch (Exception e) { + exceptionConsumer.accept(e); + } + } + + @Override + public void onClusterServiceClose() { + // This means the cluster is being shut down, so nothing to do here + } + + @Override + public void onTimeout(TimeValue timeout) { + exceptionConsumer.accept( + new IllegalStateException("slm retention snapshot deletion out while waiting for ongoing snapshot operations to complete")); + } + } +} diff --git a/x-pack/plugin/ilm/src/main/java/org/elasticsearch/xpack/slm/UpdateSnapshotLifecycleStatsTask.java b/x-pack/plugin/ilm/src/main/java/org/elasticsearch/xpack/slm/UpdateSnapshotLifecycleStatsTask.java new file mode 100644 index 0000000000000..7d3946b57ceab --- /dev/null +++ b/x-pack/plugin/ilm/src/main/java/org/elasticsearch/xpack/slm/UpdateSnapshotLifecycleStatsTask.java @@ -0,0 +1,56 @@ +/* + * 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.slm; + +import org.apache.logging.log4j.LogManager; +import org.apache.logging.log4j.Logger; +import org.apache.logging.log4j.message.ParameterizedMessage; +import org.elasticsearch.cluster.ClusterState; +import org.elasticsearch.cluster.ClusterStateUpdateTask; +import org.elasticsearch.cluster.metadata.MetaData; +import org.elasticsearch.xpack.core.slm.SnapshotLifecycleMetadata; + +/** + * {@link UpdateSnapshotLifecycleStatsTask} is a cluster state update task that retrieves the + * current SLM stats, merges them with the newly produced stats (non-mutating), and then updates + * the cluster state with the new stats numbers + */ +public class UpdateSnapshotLifecycleStatsTask extends ClusterStateUpdateTask { + private static final Logger logger = LogManager.getLogger(SnapshotRetentionTask.class); + + private final SnapshotLifecycleStats runStats; + + UpdateSnapshotLifecycleStatsTask(SnapshotLifecycleStats runStats) { + this.runStats = runStats; + } + + @Override + public ClusterState execute(ClusterState currentState) { + final MetaData currentMeta = currentState.metaData(); + final SnapshotLifecycleMetadata currentSlmMeta = currentMeta.custom(SnapshotLifecycleMetadata.TYPE); + + if (currentSlmMeta == null) { + return currentState; + } + + SnapshotLifecycleStats newMetrics = currentSlmMeta.getStats().merge(runStats); + SnapshotLifecycleMetadata newSlmMeta = new SnapshotLifecycleMetadata(currentSlmMeta.getSnapshotConfigurations(), + currentSlmMeta.getOperationMode(), newMetrics); + + return ClusterState.builder(currentState) + .metaData(MetaData.builder(currentMeta) + .putCustom(SnapshotLifecycleMetadata.TYPE, newSlmMeta)) + .build(); + } + + @Override + public void onFailure(String source, Exception e) { + logger.error(new ParameterizedMessage("failed to update cluster state with snapshot lifecycle stats, " + + "source: [{}], missing stats: [{}]", source, runStats), + e); + } +} diff --git a/x-pack/plugin/ilm/src/main/java/org/elasticsearch/xpack/slm/action/RestGetSnapshotLifecycleStatsAction.java b/x-pack/plugin/ilm/src/main/java/org/elasticsearch/xpack/slm/action/RestGetSnapshotLifecycleStatsAction.java new file mode 100644 index 0000000000000..b8629c2db5760 --- /dev/null +++ b/x-pack/plugin/ilm/src/main/java/org/elasticsearch/xpack/slm/action/RestGetSnapshotLifecycleStatsAction.java @@ -0,0 +1,35 @@ +/* + * 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.slm.action; + +import org.elasticsearch.client.node.NodeClient; +import org.elasticsearch.rest.BaseRestHandler; +import org.elasticsearch.rest.RestController; +import org.elasticsearch.rest.RestRequest; +import org.elasticsearch.rest.action.RestToXContentListener; +import org.elasticsearch.xpack.core.slm.action.GetSnapshotLifecycleStatsAction; + +public class RestGetSnapshotLifecycleStatsAction extends BaseRestHandler { + + public RestGetSnapshotLifecycleStatsAction(RestController controller) { + controller.registerHandler(RestRequest.Method.GET, "/_slm/stats", this); + } + + @Override + public String getName() { + return "slm_get_lifecycle_stats"; + } + + @Override + protected RestChannelConsumer prepareRequest(RestRequest request, NodeClient client) { + GetSnapshotLifecycleStatsAction.Request req = new GetSnapshotLifecycleStatsAction.Request(); + req.timeout(request.paramAsTime("timeout", req.timeout())); + req.masterNodeTimeout(request.paramAsTime("master_timeout", req.masterNodeTimeout())); + + return channel -> client.execute(GetSnapshotLifecycleStatsAction.INSTANCE, req, new RestToXContentListener<>(channel)); + } +} diff --git a/x-pack/plugin/ilm/src/main/java/org/elasticsearch/xpack/slm/action/TransportDeleteSnapshotLifecycleAction.java b/x-pack/plugin/ilm/src/main/java/org/elasticsearch/xpack/slm/action/TransportDeleteSnapshotLifecycleAction.java index 443bc7696822c..39b85c712e906 100644 --- a/x-pack/plugin/ilm/src/main/java/org/elasticsearch/xpack/slm/action/TransportDeleteSnapshotLifecycleAction.java +++ b/x-pack/plugin/ilm/src/main/java/org/elasticsearch/xpack/slm/action/TransportDeleteSnapshotLifecycleAction.java @@ -55,7 +55,7 @@ protected void masterOperation(Task task, DeleteSnapshotLifecycleAction.Request ClusterState state, ActionListener listener) throws Exception { clusterService.submitStateUpdateTask("delete-snapshot-lifecycle-" + request.getLifecycleId(), - new AckedClusterStateUpdateTask(request, listener) { + new AckedClusterStateUpdateTask<>(request, listener) { @Override protected DeleteSnapshotLifecycleAction.Response newResponse(boolean acknowledged) { return new DeleteSnapshotLifecycleAction.Response(acknowledged); @@ -82,7 +82,8 @@ public ClusterState execute(ClusterState currentState) { return ClusterState.builder(currentState) .metaData(MetaData.builder(metaData) .putCustom(SnapshotLifecycleMetadata.TYPE, - new SnapshotLifecycleMetadata(newConfigs, snapMeta.getOperationMode()))) + new SnapshotLifecycleMetadata(newConfigs, + snapMeta.getOperationMode(), snapMeta.getStats().removePolicy(request.getLifecycleId())))) .build(); } }); diff --git a/x-pack/plugin/ilm/src/main/java/org/elasticsearch/xpack/slm/action/TransportGetSnapshotLifecycleAction.java b/x-pack/plugin/ilm/src/main/java/org/elasticsearch/xpack/slm/action/TransportGetSnapshotLifecycleAction.java index d07e71e129fab..90d302eb403d2 100644 --- a/x-pack/plugin/ilm/src/main/java/org/elasticsearch/xpack/slm/action/TransportGetSnapshotLifecycleAction.java +++ b/x-pack/plugin/ilm/src/main/java/org/elasticsearch/xpack/slm/action/TransportGetSnapshotLifecycleAction.java @@ -26,6 +26,7 @@ import org.elasticsearch.xpack.core.slm.SnapshotLifecyclePolicy; import org.elasticsearch.xpack.core.slm.SnapshotLifecyclePolicyItem; import org.elasticsearch.xpack.core.slm.action.GetSnapshotLifecycleAction; +import org.elasticsearch.xpack.slm.SnapshotLifecycleStats; import java.io.IOException; import java.util.Arrays; @@ -86,6 +87,7 @@ protected void masterOperation(final Task task, final GetSnapshotLifecycleAction } final Set ids = new HashSet<>(Arrays.asList(request.getLifecycleIds())); + final SnapshotLifecycleStats slmStats = snapMeta.getStats(); List lifecycles = snapMeta.getSnapshotConfigurations().values().stream() .filter(meta -> { if (ids.isEmpty()) { @@ -94,7 +96,9 @@ protected void masterOperation(final Task task, final GetSnapshotLifecycleAction return ids.contains(meta.getPolicy().getId()); } }) - .map(policyMeta -> new SnapshotLifecyclePolicyItem(policyMeta, inProgress.get(policyMeta.getPolicy().getId()))) + .map(policyMeta -> + new SnapshotLifecyclePolicyItem(policyMeta, inProgress.get(policyMeta.getPolicy().getId()), + slmStats.getMetrics().get(policyMeta.getPolicy().getId()))) .collect(Collectors.toList()); listener.onResponse(new GetSnapshotLifecycleAction.Response(lifecycles)); } diff --git a/x-pack/plugin/ilm/src/main/java/org/elasticsearch/xpack/slm/action/TransportGetSnapshotLifecycleStatsAction.java b/x-pack/plugin/ilm/src/main/java/org/elasticsearch/xpack/slm/action/TransportGetSnapshotLifecycleStatsAction.java new file mode 100644 index 0000000000000..9e677e13f025d --- /dev/null +++ b/x-pack/plugin/ilm/src/main/java/org/elasticsearch/xpack/slm/action/TransportGetSnapshotLifecycleStatsAction.java @@ -0,0 +1,64 @@ +/* + * 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.slm.action; + +import org.elasticsearch.action.ActionListener; +import org.elasticsearch.action.support.ActionFilters; +import org.elasticsearch.action.support.master.TransportMasterNodeAction; +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.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.core.slm.SnapshotLifecycleMetadata; +import org.elasticsearch.xpack.core.slm.action.GetSnapshotLifecycleStatsAction; +import org.elasticsearch.xpack.slm.SnapshotLifecycleStats; + +import java.io.IOException; + +public class TransportGetSnapshotLifecycleStatsAction extends + TransportMasterNodeAction { + + @Inject + public TransportGetSnapshotLifecycleStatsAction(TransportService transportService, ClusterService clusterService, + ThreadPool threadPool, ActionFilters actionFilters, + IndexNameExpressionResolver indexNameExpressionResolver) { + super(GetSnapshotLifecycleStatsAction.NAME, transportService, clusterService, threadPool, actionFilters, + GetSnapshotLifecycleStatsAction.Request::new, indexNameExpressionResolver); + } + + @Override + protected String executor() { + return ThreadPool.Names.SAME; + } + + @Override + protected GetSnapshotLifecycleStatsAction.Response read(StreamInput in) throws IOException { + return new GetSnapshotLifecycleStatsAction.Response(in); + } + + @Override + protected void masterOperation(Task task, GetSnapshotLifecycleStatsAction.Request request, + ClusterState state, ActionListener listener) { + SnapshotLifecycleMetadata slmMeta = state.metaData().custom(SnapshotLifecycleMetadata.TYPE); + if (slmMeta == null) { + listener.onResponse(new GetSnapshotLifecycleStatsAction.Response(new SnapshotLifecycleStats())); + } else { + listener.onResponse(new GetSnapshotLifecycleStatsAction.Response(slmMeta.getStats())); + } + } + + @Override + protected ClusterBlockException checkBlock(GetSnapshotLifecycleStatsAction.Request request, ClusterState state) { + return state.blocks().globalBlockedException(ClusterBlockLevel.METADATA_READ); + } +} diff --git a/x-pack/plugin/ilm/src/main/java/org/elasticsearch/xpack/slm/action/TransportPutSnapshotLifecycleAction.java b/x-pack/plugin/ilm/src/main/java/org/elasticsearch/xpack/slm/action/TransportPutSnapshotLifecycleAction.java index 1bcc700433965..e487ba3709127 100644 --- a/x-pack/plugin/ilm/src/main/java/org/elasticsearch/xpack/slm/action/TransportPutSnapshotLifecycleAction.java +++ b/x-pack/plugin/ilm/src/main/java/org/elasticsearch/xpack/slm/action/TransportPutSnapshotLifecycleAction.java @@ -31,6 +31,7 @@ import org.elasticsearch.xpack.core.slm.SnapshotLifecyclePolicyMetadata; import org.elasticsearch.xpack.core.slm.action.PutSnapshotLifecycleAction; import org.elasticsearch.xpack.slm.SnapshotLifecycleService; +import org.elasticsearch.xpack.slm.SnapshotLifecycleStats; import java.io.IOException; import java.time.Instant; @@ -93,7 +94,8 @@ public ClusterState execute(ClusterState currentState) { OperationMode mode = Optional.ofNullable(ilmMeta) .map(IndexLifecycleMetadata::getOperationMode) .orElse(OperationMode.RUNNING); - lifecycleMetadata = new SnapshotLifecycleMetadata(Collections.singletonMap(id, meta), mode); + lifecycleMetadata = new SnapshotLifecycleMetadata(Collections.singletonMap(id, meta), + mode, new SnapshotLifecycleStats()); logger.info("adding new snapshot lifecycle [{}]", id); } else { Map snapLifecycles = new HashMap<>(snapMeta.getSnapshotConfigurations()); @@ -105,7 +107,8 @@ public ClusterState execute(ClusterState currentState) { .setModifiedDate(Instant.now().toEpochMilli()) .build(); snapLifecycles.put(id, newLifecycle); - lifecycleMetadata = new SnapshotLifecycleMetadata(snapLifecycles, snapMeta.getOperationMode()); + lifecycleMetadata = new SnapshotLifecycleMetadata(snapLifecycles, + snapMeta.getOperationMode(), snapMeta.getStats()); if (oldLifecycle == null) { logger.info("adding new snapshot lifecycle [{}]", id); } else { diff --git a/x-pack/plugin/ilm/src/test/java/org/elasticsearch/xpack/ilm/OperationModeUpdateTaskTests.java b/x-pack/plugin/ilm/src/test/java/org/elasticsearch/xpack/ilm/OperationModeUpdateTaskTests.java index bcd268b8b3e92..f3ed5924cfebc 100644 --- a/x-pack/plugin/ilm/src/test/java/org/elasticsearch/xpack/ilm/OperationModeUpdateTaskTests.java +++ b/x-pack/plugin/ilm/src/test/java/org/elasticsearch/xpack/ilm/OperationModeUpdateTaskTests.java @@ -14,6 +14,7 @@ import org.elasticsearch.xpack.core.ilm.IndexLifecycleMetadata; import org.elasticsearch.xpack.core.ilm.OperationMode; import org.elasticsearch.xpack.core.slm.SnapshotLifecycleMetadata; +import org.elasticsearch.xpack.slm.SnapshotLifecycleStats; import java.util.Collections; @@ -58,7 +59,8 @@ private void assertNoMove(OperationMode currentMode, OperationMode requestedMode private OperationMode executeUpdate(boolean metadataInstalled, OperationMode currentMode, OperationMode requestMode, boolean assertSameClusterState) { IndexLifecycleMetadata indexLifecycleMetadata = new IndexLifecycleMetadata(Collections.emptyMap(), currentMode); - SnapshotLifecycleMetadata snapshotLifecycleMetadata = new SnapshotLifecycleMetadata(Collections.emptyMap(), currentMode); + SnapshotLifecycleMetadata snapshotLifecycleMetadata = + new SnapshotLifecycleMetadata(Collections.emptyMap(), currentMode, new SnapshotLifecycleStats()); ImmutableOpenMap.Builder customsMapBuilder = ImmutableOpenMap.builder(); MetaData.Builder metaData = MetaData.builder() .persistentSettings(settings(Version.CURRENT).build()); diff --git a/x-pack/plugin/ilm/src/test/java/org/elasticsearch/xpack/slm/SLMSnapshotBlockingIntegTests.java b/x-pack/plugin/ilm/src/test/java/org/elasticsearch/xpack/slm/SLMSnapshotBlockingIntegTests.java new file mode 100644 index 0000000000000..52d5244aeb08c --- /dev/null +++ b/x-pack/plugin/ilm/src/test/java/org/elasticsearch/xpack/slm/SLMSnapshotBlockingIntegTests.java @@ -0,0 +1,281 @@ +/* + * 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.slm; + +import org.elasticsearch.action.admin.cluster.snapshots.status.SnapshotStatus; +import org.elasticsearch.action.admin.cluster.snapshots.status.SnapshotsStatusResponse; +import org.elasticsearch.cluster.SnapshotsInProgress; +import org.elasticsearch.common.settings.Settings; +import org.elasticsearch.common.unit.TimeValue; +import org.elasticsearch.plugins.Plugin; +import org.elasticsearch.repositories.RepositoriesService; +import org.elasticsearch.snapshots.SnapshotMissingException; +import org.elasticsearch.snapshots.mockstore.MockRepository; +import org.elasticsearch.test.ESIntegTestCase; +import org.elasticsearch.xpack.core.LocalStateCompositeXPackPlugin; +import org.elasticsearch.xpack.core.ilm.LifecycleSettings; +import org.elasticsearch.xpack.core.slm.SnapshotLifecyclePolicy; +import org.elasticsearch.xpack.core.slm.SnapshotLifecyclePolicyItem; +import org.elasticsearch.xpack.core.slm.SnapshotRetentionConfiguration; +import org.elasticsearch.xpack.core.slm.action.ExecuteSnapshotLifecycleAction; +import org.elasticsearch.xpack.core.slm.action.GetSnapshotLifecycleAction; +import org.elasticsearch.xpack.core.slm.action.PutSnapshotLifecycleAction; +import org.elasticsearch.xpack.ilm.IndexLifecycle; +import org.junit.After; + +import java.util.Arrays; +import java.util.Collection; +import java.util.Collections; +import java.util.HashMap; +import java.util.Map; + +import static org.hamcrest.Matchers.anyOf; +import static org.hamcrest.Matchers.equalTo; +import static org.hamcrest.Matchers.greaterThan; + +/** + * Tests for Snapshot Lifecycle Management that require a slow or blocked snapshot repo (using {@link MockRepository} + */ +public class SLMSnapshotBlockingIntegTests extends ESIntegTestCase { + + @After + public void resetSLMSettings() { + // unset retention settings + client().admin().cluster().prepareUpdateSettings() + .setTransientSettings(Settings.builder() + .put(LifecycleSettings.SLM_RETENTION_SCHEDULE, (String) null) + .build()) + .get(); + } + + @Override + protected Collection> nodePlugins() { + return Arrays.asList(MockRepository.Plugin.class, LocalStateCompositeXPackPlugin.class, IndexLifecycle.class); + } + + public void testSnapshotInProgress() throws Exception { + final String indexName = "test"; + final String policyName = "test-policy"; + final String repoId = "my-repo"; + int docCount = 20; + for (int i = 0; i < docCount; i++) { + index(indexName, "_doc", i + "", Collections.singletonMap("foo", "bar")); + } + + // Create a snapshot repo + initializeRepo(repoId); + + logger.info("--> creating policy {}", policyName); + createSnapshotPolicy(policyName, "snap", "1 2 3 4 5 ?", repoId, indexName, true); + + logger.info("--> blocking master from completing snapshot"); + blockMasterFromFinalizingSnapshotOnIndexFile(repoId); + + logger.info("--> executing snapshot lifecycle"); + final String snapshotName = executePolicy(policyName); + + // Check that the executed snapshot shows up in the SLM output + assertBusy(() -> { + GetSnapshotLifecycleAction.Response getResp = + client().execute(GetSnapshotLifecycleAction.INSTANCE, new GetSnapshotLifecycleAction.Request(policyName)).get(); + logger.info("--> checking for in progress snapshot..."); + + assertThat(getResp.getPolicies().size(), greaterThan(0)); + SnapshotLifecyclePolicyItem item = getResp.getPolicies().get(0); + assertNotNull(item.getSnapshotInProgress()); + SnapshotLifecyclePolicyItem.SnapshotInProgress inProgress = item.getSnapshotInProgress(); + assertThat(inProgress.getSnapshotId().getName(), equalTo(snapshotName)); + assertThat(inProgress.getStartTime(), greaterThan(0L)); + assertThat(inProgress.getState(), anyOf(equalTo(SnapshotsInProgress.State.INIT), equalTo(SnapshotsInProgress.State.STARTED))); + assertNull(inProgress.getFailure()); + }); + + logger.info("--> unblocking snapshots"); + unblockRepo(repoId); + + // Cancel/delete the snapshot + try { + client().admin().cluster().prepareDeleteSnapshot(repoId, snapshotName).get(); + } catch (SnapshotMissingException e) { + // ignore + } + } + + public void testRetentionWhileSnapshotInProgress() throws Exception { + final String indexName = "test"; + final String policyId = "slm-policy"; + final String repoId = "slm-repo"; + int docCount = 20; + for (int i = 0; i < docCount; i++) { + index(indexName, "_doc", i + "", Collections.singletonMap("foo", "bar")); + } + + initializeRepo(repoId); + + logger.info("--> creating policy {}", policyId); + createSnapshotPolicy(policyId, "snap", "1 2 3 4 5 ?", repoId, indexName, true, + new SnapshotRetentionConfiguration(TimeValue.timeValueSeconds(0), null, null)); + + // Create a snapshot and wait for it to be complete (need something that can be deleted) + final String completedSnapshotName = executePolicy(policyId); + logger.info("--> kicked off snapshot {}", completedSnapshotName); + assertBusy(() -> { + try { + SnapshotsStatusResponse s = + client().admin().cluster().prepareSnapshotStatus(repoId).setSnapshots(completedSnapshotName).get(); + assertThat("expected a snapshot but none were returned", s.getSnapshots().size(), equalTo(1)); + SnapshotStatus status = s.getSnapshots().get(0); + logger.info("--> waiting for snapshot {} to be completed, got: {}", completedSnapshotName, status.getState()); + assertThat(status.getState(), equalTo(SnapshotsInProgress.State.SUCCESS)); + } catch (SnapshotMissingException e) { + logger.error("expected a snapshot but it was missing", e); + fail("expected a snapshot with name " + completedSnapshotName + " but it does not exist"); + } + }); + + // Take another snapshot, but before doing that, block it from completing + logger.info("--> blocking nodes from completing snapshot"); + blockAllDataNodes(repoId); + final String secondSnapName = executePolicy(policyId); + + // Check that the executed snapshot shows up in the SLM output as in_progress + assertBusy(() -> { + GetSnapshotLifecycleAction.Response getResp = + client().execute(GetSnapshotLifecycleAction.INSTANCE, new GetSnapshotLifecycleAction.Request(policyId)).get(); + logger.info("--> checking for in progress snapshot..."); + + assertThat(getResp.getPolicies().size(), greaterThan(0)); + SnapshotLifecyclePolicyItem item = getResp.getPolicies().get(0); + assertNotNull(item.getSnapshotInProgress()); + SnapshotLifecyclePolicyItem.SnapshotInProgress inProgress = item.getSnapshotInProgress(); + assertThat(inProgress.getSnapshotId().getName(), equalTo(secondSnapName)); + assertThat(inProgress.getStartTime(), greaterThan(0L)); + assertThat(inProgress.getState(), anyOf(equalTo(SnapshotsInProgress.State.INIT), equalTo(SnapshotsInProgress.State.STARTED))); + assertNull(inProgress.getFailure()); + }); + + // Run retention every second + client().admin().cluster().prepareUpdateSettings() + .setTransientSettings(Settings.builder() + .put(LifecycleSettings.SLM_RETENTION_SCHEDULE, "*/1 * * * * ?") + .build()) + .get(); + // Guarantee that retention gets a chance to run before unblocking, I know sleeps are not + // ideal, but we don't currently have a way to force retention to run, so waiting at least + // a second is the best we can do for now. + Thread.sleep(1500); + + logger.info("--> unblocking snapshots"); + unblockRepo(repoId); + unblockAllDataNodes(repoId); + + // Check that the snapshot created by the policy has been removed by retention + assertBusy(() -> { + // Trigger a cluster state update so that it re-checks for a snapshot in progress + client().admin().cluster().prepareReroute().get(); + logger.info("--> waiting for snapshot to be deleted"); + try { + SnapshotsStatusResponse s = + client().admin().cluster().prepareSnapshotStatus(repoId).setSnapshots(completedSnapshotName).get(); + assertNull("expected no snapshot but one was returned", s.getSnapshots().get(0)); + } catch (SnapshotMissingException e) { + // Great, we wanted it to be deleted! + } + }); + + // Cancel/delete the snapshot + try { + client().admin().cluster().prepareDeleteSnapshot(repoId, secondSnapName).get(); + } catch (SnapshotMissingException e) { + // ignore + } + } + + private void initializeRepo(String repoName) { + client().admin().cluster().preparePutRepository(repoName) + .setType("mock") + .setSettings(Settings.builder() + .put("compress", randomBoolean()) + .put("location", randomAlphaOfLength(6)) + .build()) + .get(); + } + + private void createSnapshotPolicy(String policyName, String snapshotNamePattern, String schedule, String repoId, + String indexPattern, boolean ignoreUnavailable) { + createSnapshotPolicy(policyName, snapshotNamePattern, schedule, repoId, indexPattern, + ignoreUnavailable, SnapshotRetentionConfiguration.EMPTY); + } + + private void createSnapshotPolicy(String policyName, String snapshotNamePattern, String schedule, String repoId, + String indexPattern, boolean ignoreUnavailable, + SnapshotRetentionConfiguration retention) { + Map snapConfig = new HashMap<>(); + snapConfig.put("indices", Collections.singletonList(indexPattern)); + snapConfig.put("ignore_unavailable", ignoreUnavailable); + if (randomBoolean()) { + Map metadata = new HashMap<>(); + int fieldCount = randomIntBetween(2,5); + for (int i = 0; i < fieldCount; i++) { + metadata.put(randomValueOtherThanMany(key -> "policy".equals(key) || metadata.containsKey(key), + () -> randomAlphaOfLength(5)), randomAlphaOfLength(4)); + } + } + SnapshotLifecyclePolicy policy = new SnapshotLifecyclePolicy(policyName, snapshotNamePattern, schedule, + repoId, snapConfig, retention); + + PutSnapshotLifecycleAction.Request putLifecycle = new PutSnapshotLifecycleAction.Request(policyName, policy); + try { + client().execute(PutSnapshotLifecycleAction.INSTANCE, putLifecycle).get(); + } catch (Exception e) { + logger.error("failed to create slm policy", e); + fail("failed to create policy " + policy + " got: " + e); + } + } + + /** + * Execute the given policy and return the generated snapshot name + */ + private String executePolicy(String policyId) { + ExecuteSnapshotLifecycleAction.Request executeReq = new ExecuteSnapshotLifecycleAction.Request(policyId); + ExecuteSnapshotLifecycleAction.Response resp = null; + try { + resp = client().execute(ExecuteSnapshotLifecycleAction.INSTANCE, executeReq).get(); + return resp.getSnapshotName(); + } catch (Exception e) { + logger.error("failed to execute policy", e); + fail("failed to execute policy " + policyId + " got: " + e); + return "bad"; + } + } + + public static String blockMasterFromFinalizingSnapshotOnIndexFile(final String repositoryName) { + final String masterName = internalCluster().getMasterName(); + ((MockRepository)internalCluster().getInstance(RepositoriesService.class, masterName) + .repository(repositoryName)).setBlockOnWriteIndexFile(true); + return masterName; + } + + public static String unblockRepo(final String repositoryName) { + final String masterName = internalCluster().getMasterName(); + ((MockRepository)internalCluster().getInstance(RepositoriesService.class, masterName) + .repository(repositoryName)).unblock(); + return masterName; + } + + public static void blockAllDataNodes(String repository) { + for(RepositoriesService repositoriesService : internalCluster().getDataNodeInstances(RepositoriesService.class)) { + ((MockRepository)repositoriesService.repository(repository)).blockOnDataFiles(true); + } + } + + public static void unblockAllDataNodes(String repository) { + for(RepositoriesService repositoriesService : internalCluster().getDataNodeInstances(RepositoriesService.class)) { + ((MockRepository)repositoriesService.repository(repository)).unblock(); + } + } +} diff --git a/x-pack/plugin/ilm/src/test/java/org/elasticsearch/xpack/slm/SnapshotLifecyclePolicyTests.java b/x-pack/plugin/ilm/src/test/java/org/elasticsearch/xpack/slm/SnapshotLifecyclePolicyTests.java index 9370cad7f8771..6523d7a07228a 100644 --- a/x-pack/plugin/ilm/src/test/java/org/elasticsearch/xpack/slm/SnapshotLifecyclePolicyTests.java +++ b/x-pack/plugin/ilm/src/test/java/org/elasticsearch/xpack/slm/SnapshotLifecyclePolicyTests.java @@ -11,12 +11,15 @@ import org.elasticsearch.common.xcontent.XContentParser; import org.elasticsearch.test.AbstractSerializingTestCase; import org.elasticsearch.xpack.core.slm.SnapshotLifecyclePolicy; +import org.elasticsearch.xpack.core.slm.SnapshotLifecyclePolicyMetadataTests; +import org.elasticsearch.xpack.core.slm.SnapshotRetentionConfiguration; import java.io.IOException; import java.util.Collections; import java.util.HashMap; import java.util.Map; +import static org.elasticsearch.xpack.core.slm.SnapshotLifecyclePolicyMetadataTests.randomSnapshotLifecyclePolicy; import static org.hamcrest.Matchers.contains; import static org.hamcrest.Matchers.containsInAnyOrder; import static org.hamcrest.Matchers.equalTo; @@ -30,29 +33,34 @@ public class SnapshotLifecyclePolicyTests extends AbstractSerializingTestCase", "1 * * * * ?", "repo", Collections.emptyMap()); + p = new SnapshotLifecyclePolicy("id", "", "1 * * * * ?", "repo", Collections.emptyMap(), + SnapshotRetentionConfiguration.EMPTY); assertThat(p.generateSnapshotName(context), startsWith("name-2019.03.15-")); assertThat(p.generateSnapshotName(context).length(), greaterThan("name-2019.03.15-".length())); - p = new SnapshotLifecyclePolicy("id", "", "1 * * * * ?", "repo", Collections.emptyMap()); + p = new SnapshotLifecyclePolicy("id", "", "1 * * * * ?", "repo", Collections.emptyMap(), + SnapshotRetentionConfiguration.EMPTY); assertThat(p.generateSnapshotName(context), startsWith("name-2019.03.01-")); - p = new SnapshotLifecyclePolicy("id", "", "1 * * * * ?", "repo", Collections.emptyMap()); + p = new SnapshotLifecyclePolicy("id", "", "1 * * * * ?", "repo", Collections.emptyMap(), + SnapshotRetentionConfiguration.EMPTY); assertThat(p.generateSnapshotName(context), startsWith("name-2019-03-15.21:09:00-")); } public void testNextExecutionTime() { - SnapshotLifecyclePolicy p = new SnapshotLifecyclePolicy("id", "name", "0 1 2 3 4 ? 2099", "repo", Collections.emptyMap()); + SnapshotLifecyclePolicy p = new SnapshotLifecyclePolicy("id", "name", "0 1 2 3 4 ? 2099", "repo", Collections.emptyMap(), + SnapshotRetentionConfiguration.EMPTY); assertThat(p.calculateNextExecution(), equalTo(4078864860000L)); } public void testValidation() { SnapshotLifecyclePolicy policy = new SnapshotLifecyclePolicy("a,b", "", - "* * * * * L", " ", Collections.emptyMap()); + "* * * * * L", " ", Collections.emptyMap(), SnapshotRetentionConfiguration.EMPTY); ValidationException e = policy.validate(); assertThat(e.validationErrors(), @@ -64,7 +72,7 @@ public void testValidation() { "invalid schedule: invalid cron expression [* * * * * L]")); policy = new SnapshotLifecyclePolicy("_my_policy", "mySnap", - " ", "repo", Collections.emptyMap()); + " ", "repo", Collections.emptyMap(), SnapshotRetentionConfiguration.EMPTY); e = policy.validate(); assertThat(e.validationErrors(), @@ -80,7 +88,7 @@ public void testMetadataValidation() { configuration.put("metadata", metadataString); SnapshotLifecyclePolicy policy = new SnapshotLifecyclePolicy("mypolicy", "", - "1 * * * * ?", "myrepo", configuration); + "1 * * * * ?", "myrepo", configuration, SnapshotRetentionConfiguration.EMPTY); ValidationException e = policy.validate(); assertThat(e.validationErrors(), contains("invalid configuration.metadata [" + metadataString + "]: must be an object if present")); @@ -93,7 +101,7 @@ public void testMetadataValidation() { configuration.put("metadata", metadata); SnapshotLifecyclePolicy policy = new SnapshotLifecyclePolicy("mypolicy", "", - "1 * * * * ?", "myrepo", configuration); + "1 * * * * ?", "myrepo", configuration, SnapshotRetentionConfiguration.EMPTY); ValidationException e = policy.validate(); assertThat(e.validationErrors(), contains("invalid configuration.metadata: field name [policy] is reserved and " + "will be added automatically")); @@ -113,7 +121,7 @@ public void testMetadataValidation() { configuration.put("metadata", metadata); SnapshotLifecyclePolicy policy = new SnapshotLifecyclePolicy("mypolicy", "", - "1 * * * * ?", "myrepo", configuration); + "1 * * * * ?", "myrepo", configuration, SnapshotRetentionConfiguration.EMPTY); ValidationException e = policy.validate(); assertThat(e.validationErrors(), contains("invalid configuration.metadata: must be smaller than [1004] bytes, but is [" + totalBytes + "] bytes")); @@ -131,54 +139,37 @@ protected SnapshotLifecyclePolicy createTestInstance() { return randomSnapshotLifecyclePolicy(id); } - public static SnapshotLifecyclePolicy randomSnapshotLifecyclePolicy(String id) { - Map config = null; - if (randomBoolean()) { - config = new HashMap<>(); - for (int i = 0; i < randomIntBetween(2, 5); i++) { - config.put(randomAlphaOfLength(4), randomAlphaOfLength(4)); - } - } - return new SnapshotLifecyclePolicy(id, - randomAlphaOfLength(4), - randomSchedule(), - randomAlphaOfLength(4), - config); - } - - private static String randomSchedule() { - return randomIntBetween(0, 59) + " " + - randomIntBetween(0, 59) + " " + - randomIntBetween(0, 12) + " * * ?"; - } - @Override - protected SnapshotLifecyclePolicy mutateInstance(SnapshotLifecyclePolicy instance) throws IOException { - switch (between(0, 4)) { + protected SnapshotLifecyclePolicy mutateInstance(SnapshotLifecyclePolicy instance) { + switch (between(0, 5)) { case 0: return new SnapshotLifecyclePolicy(instance.getId() + randomAlphaOfLength(2), instance.getName(), instance.getSchedule(), instance.getRepository(), - instance.getConfig()); + instance.getConfig(), + instance.getRetentionPolicy()); case 1: return new SnapshotLifecyclePolicy(instance.getId(), instance.getName() + randomAlphaOfLength(2), instance.getSchedule(), instance.getRepository(), - instance.getConfig()); + instance.getConfig(), + instance.getRetentionPolicy()); case 2: return new SnapshotLifecyclePolicy(instance.getId(), instance.getName(), - randomValueOtherThan(instance.getSchedule(), SnapshotLifecyclePolicyTests::randomSchedule), + randomValueOtherThan(instance.getSchedule(), SnapshotLifecyclePolicyMetadataTests::randomSchedule), instance.getRepository(), - instance.getConfig()); + instance.getConfig(), + instance.getRetentionPolicy()); case 3: return new SnapshotLifecyclePolicy(instance.getId(), instance.getName(), instance.getSchedule(), instance.getRepository() + randomAlphaOfLength(2), - instance.getConfig()); + instance.getConfig(), + instance.getRetentionPolicy()); case 4: Map newConfig = new HashMap<>(); for (int i = 0; i < randomIntBetween(2, 5); i++) { @@ -188,7 +179,15 @@ protected SnapshotLifecyclePolicy mutateInstance(SnapshotLifecyclePolicy instanc instance.getName() + randomAlphaOfLength(2), instance.getSchedule(), instance.getRepository(), - newConfig); + newConfig, + instance.getRetentionPolicy()); + case 5: + return new SnapshotLifecyclePolicy(instance.getId(), + instance.getName(), + instance.getSchedule(), + instance.getRepository(), + instance.getConfig(), + randomValueOtherThan(instance.getRetentionPolicy(), SnapshotLifecyclePolicyMetadataTests::randomRetention)); default: throw new AssertionError("failure, got illegal switch case"); } diff --git a/x-pack/plugin/ilm/src/test/java/org/elasticsearch/xpack/slm/SnapshotLifecycleServiceTests.java b/x-pack/plugin/ilm/src/test/java/org/elasticsearch/xpack/slm/SnapshotLifecycleServiceTests.java index ab32d9bec0b4f..2a8868c480c14 100644 --- a/x-pack/plugin/ilm/src/test/java/org/elasticsearch/xpack/slm/SnapshotLifecycleServiceTests.java +++ b/x-pack/plugin/ilm/src/test/java/org/elasticsearch/xpack/slm/SnapshotLifecycleServiceTests.java @@ -23,6 +23,7 @@ import org.elasticsearch.xpack.core.slm.SnapshotLifecycleMetadata; import org.elasticsearch.xpack.core.slm.SnapshotLifecyclePolicy; import org.elasticsearch.xpack.core.slm.SnapshotLifecyclePolicyMetadata; +import org.elasticsearch.xpack.core.slm.SnapshotRetentionConfiguration; import org.elasticsearch.xpack.core.watcher.watch.ClockMock; import java.util.ArrayList; @@ -90,7 +91,8 @@ public void testNothingScheduledWhenNotRunning() { .setModifiedDate(1) .build(); ClusterState initialState = createState(new SnapshotLifecycleMetadata( - Collections.singletonMap(initialPolicy.getPolicy().getId(), initialPolicy), OperationMode.RUNNING)); + Collections.singletonMap(initialPolicy.getPolicy().getId(), initialPolicy), + OperationMode.RUNNING, new SnapshotLifecycleStats())); try (ThreadPool threadPool = new TestThreadPool("test"); ClusterService clusterService = ClusterServiceUtils.createClusterService(initialState, threadPool); SnapshotLifecycleService sls = new SnapshotLifecycleService(Settings.EMPTY, @@ -106,8 +108,10 @@ public void testNothingScheduledWhenNotRunning() { .build(); Map policies = new HashMap<>(); policies.put(newPolicy.getPolicy().getId(), newPolicy); - ClusterState emptyState = createState(new SnapshotLifecycleMetadata(Collections.emptyMap(), OperationMode.RUNNING)); - ClusterState state = createState(new SnapshotLifecycleMetadata(policies, OperationMode.RUNNING)); + ClusterState emptyState = + createState(new SnapshotLifecycleMetadata(Collections.emptyMap(), OperationMode.RUNNING, new SnapshotLifecycleStats())); + ClusterState state = + createState(new SnapshotLifecycleMetadata(policies, OperationMode.RUNNING, new SnapshotLifecycleStats())); sls.clusterChanged(new ClusterChangedEvent("1", state, emptyState)); @@ -117,13 +121,13 @@ public void testNothingScheduledWhenNotRunning() { sls.onMaster(); assertThat(sls.getScheduler().scheduledJobIds(), equalTo(Collections.singleton("initial-1"))); - state = createState(new SnapshotLifecycleMetadata(policies, OperationMode.STOPPING)); + state = createState(new SnapshotLifecycleMetadata(policies, OperationMode.STOPPING, new SnapshotLifecycleStats())); sls.clusterChanged(new ClusterChangedEvent("2", state, emptyState)); // Since the service is stopping, jobs should have been cancelled assertThat(sls.getScheduler().scheduledJobIds(), equalTo(Collections.emptySet())); - state = createState(new SnapshotLifecycleMetadata(policies, OperationMode.STOPPED)); + state = createState(new SnapshotLifecycleMetadata(policies, OperationMode.STOPPED, new SnapshotLifecycleStats())); sls.clusterChanged(new ClusterChangedEvent("3", state, emptyState)); // Since the service is stopped, jobs should have been cancelled @@ -148,7 +152,8 @@ public void testPolicyCRUD() throws Exception { () -> new FakeSnapshotTask(e -> trigger.get().accept(e)), clusterService, clock)) { sls.offMaster(); - SnapshotLifecycleMetadata snapMeta = new SnapshotLifecycleMetadata(Collections.emptyMap(), OperationMode.RUNNING); + SnapshotLifecycleMetadata snapMeta = + new SnapshotLifecycleMetadata(Collections.emptyMap(), OperationMode.RUNNING, new SnapshotLifecycleStats()); ClusterState previousState = createState(snapMeta); Map policies = new HashMap<>(); @@ -158,7 +163,7 @@ public void testPolicyCRUD() throws Exception { .setModifiedDate(1) .build(); policies.put(policy.getPolicy().getId(), policy); - snapMeta = new SnapshotLifecycleMetadata(policies, OperationMode.RUNNING); + snapMeta = new SnapshotLifecycleMetadata(policies, OperationMode.RUNNING, new SnapshotLifecycleStats()); ClusterState state = createState(snapMeta); ClusterChangedEvent event = new ClusterChangedEvent("1", state, previousState); trigger.set(e -> { @@ -187,7 +192,7 @@ public void testPolicyCRUD() throws Exception { .setModifiedDate(2) .build(); policies.put(policy.getPolicy().getId(), newPolicy); - state = createState(new SnapshotLifecycleMetadata(policies, OperationMode.RUNNING)); + state = createState(new SnapshotLifecycleMetadata(policies, OperationMode.RUNNING, new SnapshotLifecycleStats())); event = new ClusterChangedEvent("2", state, previousState); sls.clusterChanged(event); assertThat(sls.getScheduler().scheduledJobIds(), equalTo(Collections.singleton("foo-2"))); @@ -204,7 +209,8 @@ public void testPolicyCRUD() throws Exception { final int currentCount2 = triggerCount.get(); previousState = state; // Create a state simulating the policy being deleted - state = createState(new SnapshotLifecycleMetadata(Collections.emptyMap(), OperationMode.RUNNING)); + state = + createState(new SnapshotLifecycleMetadata(Collections.emptyMap(), OperationMode.RUNNING, new SnapshotLifecycleStats())); event = new ClusterChangedEvent("2", state, previousState); sls.clusterChanged(event); clock.fastForwardSeconds(2); @@ -221,7 +227,7 @@ public void testPolicyCRUD() throws Exception { .setModifiedDate(1) .build(); policies.put(policy.getPolicy().getId(), policy); - snapMeta = new SnapshotLifecycleMetadata(policies, OperationMode.RUNNING); + snapMeta = new SnapshotLifecycleMetadata(policies, OperationMode.RUNNING, new SnapshotLifecycleStats()); previousState = state; state = createState(snapMeta); event = new ClusterChangedEvent("1", state, previousState); @@ -254,7 +260,8 @@ public void testPolicyNamesEndingInNumbers() throws Exception { () -> new FakeSnapshotTask(e -> trigger.get().accept(e)), clusterService, clock)) { sls.onMaster(); - SnapshotLifecycleMetadata snapMeta = new SnapshotLifecycleMetadata(Collections.emptyMap(), OperationMode.RUNNING); + SnapshotLifecycleMetadata snapMeta = + new SnapshotLifecycleMetadata(Collections.emptyMap(), OperationMode.RUNNING, new SnapshotLifecycleStats()); ClusterState previousState = createState(snapMeta); Map policies = new HashMap<>(); @@ -265,7 +272,7 @@ public void testPolicyNamesEndingInNumbers() throws Exception { .setModifiedDate(1) .build(); policies.put(policy.getPolicy().getId(), policy); - snapMeta = new SnapshotLifecycleMetadata(policies, OperationMode.RUNNING); + snapMeta = new SnapshotLifecycleMetadata(policies, OperationMode.RUNNING, new SnapshotLifecycleStats()); ClusterState state = createState(snapMeta); ClusterChangedEvent event = new ClusterChangedEvent("1", state, previousState); sls.clusterChanged(event); @@ -280,7 +287,7 @@ public void testPolicyNamesEndingInNumbers() throws Exception { .setModifiedDate(1) .build(); policies.put(secondPolicy.getPolicy().getId(), secondPolicy); - snapMeta = new SnapshotLifecycleMetadata(policies, OperationMode.RUNNING); + snapMeta = new SnapshotLifecycleMetadata(policies, OperationMode.RUNNING, new SnapshotLifecycleStats()); state = createState(snapMeta); event = new ClusterChangedEvent("2", state, previousState); sls.clusterChanged(event); @@ -329,10 +336,11 @@ public static SnapshotLifecyclePolicy createPolicy(String id, String schedule) { indices.add("foo-*"); indices.add(randomAlphaOfLength(4)); config.put("indices", indices); - return new SnapshotLifecyclePolicy(id, randomAlphaOfLength(4), schedule, randomAlphaOfLength(4), config); + return new SnapshotLifecyclePolicy(id, randomAlphaOfLength(4), schedule, randomAlphaOfLength(4), config, + SnapshotRetentionConfiguration.EMPTY); } - private static String randomSchedule() { + public static String randomSchedule() { return randomIntBetween(0, 59) + " " + randomIntBetween(0, 59) + " " + randomIntBetween(0, 12) + " * * ?"; diff --git a/x-pack/plugin/ilm/src/test/java/org/elasticsearch/xpack/slm/SnapshotLifecycleTaskTests.java b/x-pack/plugin/ilm/src/test/java/org/elasticsearch/xpack/slm/SnapshotLifecycleTaskTests.java index 65897c7e1ee3b..84c1d12cce65e 100644 --- a/x-pack/plugin/ilm/src/test/java/org/elasticsearch/xpack/slm/SnapshotLifecycleTaskTests.java +++ b/x-pack/plugin/ilm/src/test/java/org/elasticsearch/xpack/slm/SnapshotLifecycleTaskTests.java @@ -56,7 +56,8 @@ public class SnapshotLifecycleTaskTests extends ESTestCase { public void testGetSnapMetadata() { final String id = randomAlphaOfLength(4); final SnapshotLifecyclePolicyMetadata slpm = makePolicyMeta(id); - final SnapshotLifecycleMetadata meta = new SnapshotLifecycleMetadata(Collections.singletonMap(id, slpm), OperationMode.RUNNING); + final SnapshotLifecycleMetadata meta = + new SnapshotLifecycleMetadata(Collections.singletonMap(id, slpm), OperationMode.RUNNING, new SnapshotLifecycleStats()); final ClusterState state = ClusterState.builder(new ClusterName("test")) .metaData(MetaData.builder() @@ -76,7 +77,8 @@ public void testGetSnapMetadata() { public void testSkipCreatingSnapshotWhenJobDoesNotMatch() { final String id = randomAlphaOfLength(4); final SnapshotLifecyclePolicyMetadata slpm = makePolicyMeta(id); - final SnapshotLifecycleMetadata meta = new SnapshotLifecycleMetadata(Collections.singletonMap(id, slpm), OperationMode.RUNNING); + final SnapshotLifecycleMetadata meta = + new SnapshotLifecycleMetadata(Collections.singletonMap(id, slpm), OperationMode.RUNNING, new SnapshotLifecycleStats()); final ClusterState state = ClusterState.builder(new ClusterName("test")) .metaData(MetaData.builder() @@ -106,7 +108,8 @@ public void testSkipCreatingSnapshotWhenJobDoesNotMatch() { public void testCreateSnapshotOnTrigger() { final String id = randomAlphaOfLength(4); final SnapshotLifecyclePolicyMetadata slpm = makePolicyMeta(id); - final SnapshotLifecycleMetadata meta = new SnapshotLifecycleMetadata(Collections.singletonMap(id, slpm), OperationMode.RUNNING); + final SnapshotLifecycleMetadata meta = + new SnapshotLifecycleMetadata(Collections.singletonMap(id, slpm), OperationMode.RUNNING, new SnapshotLifecycleStats()); final ClusterState state = ClusterState.builder(new ClusterName("test")) .metaData(MetaData.builder() diff --git a/x-pack/plugin/ilm/src/test/java/org/elasticsearch/xpack/slm/SnapshotRetentionServiceTests.java b/x-pack/plugin/ilm/src/test/java/org/elasticsearch/xpack/slm/SnapshotRetentionServiceTests.java new file mode 100644 index 0000000000000..972f0d57db453 --- /dev/null +++ b/x-pack/plugin/ilm/src/test/java/org/elasticsearch/xpack/slm/SnapshotRetentionServiceTests.java @@ -0,0 +1,80 @@ +/* + * 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.slm; + +import org.elasticsearch.Version; +import org.elasticsearch.client.Client; +import org.elasticsearch.cluster.node.DiscoveryNode; +import org.elasticsearch.cluster.node.DiscoveryNodeRole; +import org.elasticsearch.cluster.service.ClusterService; +import org.elasticsearch.common.settings.ClusterSettings; +import org.elasticsearch.common.settings.Setting; +import org.elasticsearch.common.settings.Settings; +import org.elasticsearch.test.ClusterServiceUtils; +import org.elasticsearch.test.ESTestCase; +import org.elasticsearch.threadpool.TestThreadPool; +import org.elasticsearch.threadpool.ThreadPool; +import org.elasticsearch.xpack.core.ilm.LifecycleSettings; +import org.elasticsearch.xpack.core.scheduler.SchedulerEngine; +import org.elasticsearch.xpack.core.slm.history.SnapshotHistoryStore; +import org.elasticsearch.xpack.core.watcher.watch.ClockMock; + +import java.util.Collections; +import java.util.HashSet; +import java.util.Set; + +import static org.hamcrest.Matchers.containsInAnyOrder; +import static org.hamcrest.Matchers.equalTo; +import static org.mockito.Mockito.mock; + +public class SnapshotRetentionServiceTests extends ESTestCase { + + private static final ClusterSettings clusterSettings; + static { + Set> internalSettings = new HashSet<>(ClusterSettings.BUILT_IN_CLUSTER_SETTINGS); + internalSettings.add(LifecycleSettings.SLM_RETENTION_SCHEDULE_SETTING); + clusterSettings = new ClusterSettings(Settings.EMPTY, internalSettings); + } + + public void testJobsAreScheduled() { + final DiscoveryNode discoveryNode = new DiscoveryNode("node", ESTestCase.buildNewFakeTransportAddress(), + Collections.emptyMap(), DiscoveryNodeRole.BUILT_IN_ROLES, Version.CURRENT); + ClockMock clock = new ClockMock(); + + try (ThreadPool threadPool = new TestThreadPool("test"); + ClusterService clusterService = ClusterServiceUtils.createClusterService(threadPool, discoveryNode, clusterSettings); + SnapshotRetentionService service = new SnapshotRetentionService(Settings.EMPTY, + FakeRetentionTask::new, clusterService, clock)) { + assertThat(service.getScheduler().jobCount(), equalTo(0)); + + service.onMaster(); + service.setUpdateSchedule(SnapshotLifecycleServiceTests.randomSchedule()); + assertThat(service.getScheduler().scheduledJobIds(), containsInAnyOrder(SnapshotRetentionService.SLM_RETENTION_JOB_ID)); + + service.offMaster(); + assertThat(service.getScheduler().jobCount(), equalTo(0)); + + service.onMaster(); + assertThat(service.getScheduler().scheduledJobIds(), containsInAnyOrder(SnapshotRetentionService.SLM_RETENTION_JOB_ID)); + + service.setUpdateSchedule(""); + assertThat(service.getScheduler().jobCount(), equalTo(0)); + threadPool.shutdownNow(); + } + } + + private static class FakeRetentionTask extends SnapshotRetentionTask { + FakeRetentionTask() { + super(mock(Client.class), null, System::nanoTime, mock(SnapshotHistoryStore.class), mock(ThreadPool.class)); + } + + @Override + public void triggered(SchedulerEngine.Event event) { + super.triggered(event); + } + } +} diff --git a/x-pack/plugin/ilm/src/test/java/org/elasticsearch/xpack/slm/SnapshotRetentionTaskTests.java b/x-pack/plugin/ilm/src/test/java/org/elasticsearch/xpack/slm/SnapshotRetentionTaskTests.java new file mode 100644 index 0000000000000..53c85c5e23027 --- /dev/null +++ b/x-pack/plugin/ilm/src/test/java/org/elasticsearch/xpack/slm/SnapshotRetentionTaskTests.java @@ -0,0 +1,463 @@ +/* + * 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.slm; + +import org.elasticsearch.action.ActionListener; +import org.elasticsearch.action.support.master.AcknowledgedResponse; +import org.elasticsearch.client.Client; +import org.elasticsearch.cluster.ClusterName; +import org.elasticsearch.cluster.ClusterState; +import org.elasticsearch.cluster.RepositoryCleanupInProgress; +import org.elasticsearch.cluster.RestoreInProgress; +import org.elasticsearch.cluster.SnapshotDeletionsInProgress; +import org.elasticsearch.cluster.SnapshotsInProgress; +import org.elasticsearch.cluster.metadata.MetaData; +import org.elasticsearch.cluster.service.ClusterService; +import org.elasticsearch.common.collect.ImmutableOpenMap; +import org.elasticsearch.common.settings.Settings; +import org.elasticsearch.common.unit.TimeValue; +import org.elasticsearch.index.shard.ShardId; +import org.elasticsearch.repositories.IndexId; +import org.elasticsearch.snapshots.Snapshot; +import org.elasticsearch.snapshots.SnapshotId; +import org.elasticsearch.snapshots.SnapshotInfo; +import org.elasticsearch.test.ClusterServiceUtils; +import org.elasticsearch.test.ESTestCase; +import org.elasticsearch.test.client.NoOpClient; +import org.elasticsearch.threadpool.TestThreadPool; +import org.elasticsearch.threadpool.ThreadPool; +import org.elasticsearch.xpack.core.ilm.LifecycleSettings; +import org.elasticsearch.xpack.core.ilm.OperationMode; +import org.elasticsearch.xpack.core.scheduler.SchedulerEngine; +import org.elasticsearch.xpack.core.slm.SnapshotLifecycleMetadata; +import org.elasticsearch.xpack.core.slm.SnapshotLifecyclePolicy; +import org.elasticsearch.xpack.core.slm.SnapshotLifecyclePolicyMetadata; +import org.elasticsearch.xpack.core.slm.SnapshotRetentionConfiguration; +import org.elasticsearch.xpack.core.slm.history.SnapshotHistoryStore; + +import java.time.ZoneOffset; +import java.util.ArrayList; +import java.util.Arrays; +import java.util.Collection; +import java.util.Collections; +import java.util.List; +import java.util.Map; +import java.util.Set; +import java.util.concurrent.ConcurrentHashMap; +import java.util.concurrent.CountDownLatch; +import java.util.concurrent.TimeUnit; +import java.util.concurrent.atomic.AtomicLong; +import java.util.function.Consumer; +import java.util.function.Function; +import java.util.function.LongSupplier; +import java.util.function.Supplier; +import java.util.stream.Collectors; + +import static org.elasticsearch.xpack.core.slm.history.SnapshotHistoryItem.DELETE_OPERATION; +import static org.hamcrest.Matchers.contains; +import static org.hamcrest.Matchers.containsInAnyOrder; +import static org.hamcrest.Matchers.containsString; +import static org.hamcrest.Matchers.empty; +import static org.hamcrest.Matchers.equalTo; +import static org.hamcrest.Matchers.hasSize; +import static org.hamcrest.Matchers.not; +import static org.mockito.Mockito.mock; + +public class SnapshotRetentionTaskTests extends ESTestCase { + + public void testGetAllPoliciesWithRetentionEnabled() { + SnapshotLifecyclePolicy policyWithout = new SnapshotLifecyclePolicy("policyWithout", "snap", "1 * * * * ?", + "repo", null, SnapshotRetentionConfiguration.EMPTY); + SnapshotLifecyclePolicy policyWithout2 = new SnapshotLifecyclePolicy("policyWithout2", "snap", "1 * * * * ?", + "repo", null, new SnapshotRetentionConfiguration(null, null, null)); + SnapshotLifecyclePolicy policyWith = new SnapshotLifecyclePolicy("policyWith", "snap", "1 * * * * ?", + "repo", null, new SnapshotRetentionConfiguration(TimeValue.timeValueDays(30), null, null)); + + // Test with no SLM metadata + ClusterState state = ClusterState.builder(new ClusterName("cluster")).build(); + assertThat(SnapshotRetentionTask.getAllPoliciesWithRetentionEnabled(state), equalTo(Collections.emptyMap())); + + // Test with empty SLM metadata + MetaData metaData = MetaData.builder() + .putCustom(SnapshotLifecycleMetadata.TYPE, + new SnapshotLifecycleMetadata(Collections.emptyMap(), OperationMode.RUNNING, new SnapshotLifecycleStats())) + .build(); + state = ClusterState.builder(new ClusterName("cluster")).metaData(metaData).build(); + assertThat(SnapshotRetentionTask.getAllPoliciesWithRetentionEnabled(state), equalTo(Collections.emptyMap())); + + // Test with metadata containing only a policy without retention + state = createState(policyWithout); + assertThat(SnapshotRetentionTask.getAllPoliciesWithRetentionEnabled(state), equalTo(Collections.emptyMap())); + + // Test with metadata containing a couple of policies + state = createState(policyWithout, policyWithout2, policyWith); + Map policyMap = SnapshotRetentionTask.getAllPoliciesWithRetentionEnabled(state); + assertThat(policyMap.size(), equalTo(1)); + assertThat(policyMap.get("policyWith"), equalTo(policyWith)); + } + + public void testSnapshotEligibleForDeletion() { + SnapshotLifecyclePolicy policy = new SnapshotLifecyclePolicy("policy", "snap", "1 * * * * ?", + "repo", null, new SnapshotRetentionConfiguration(TimeValue.timeValueDays(30), null, null)); + SnapshotLifecyclePolicy policyWithNoRetention = new SnapshotLifecyclePolicy("policy", "snap", "1 * * * * ?", + "repo", null, randomBoolean() ? null : SnapshotRetentionConfiguration.EMPTY); + Map policyMap = Collections.singletonMap("policy", policy); + Map policyWithNoRetentionMap = Collections.singletonMap("policy", policyWithNoRetention); + Function>> mkInfos = i -> + Collections.singletonMap("repo", Collections.singletonList(i)); + + // Test when user metadata is null + SnapshotInfo info = new SnapshotInfo(new SnapshotId("name", "uuid"), Collections.singletonList("index"), + 0L, null, 1L, 1, Collections.emptyList(), true, null); + assertThat(SnapshotRetentionTask.snapshotEligibleForDeletion(info, mkInfos.apply(info), policyMap), equalTo(false)); + + // Test when no retention is configured + info = new SnapshotInfo(new SnapshotId("name", "uuid"), Collections.singletonList("index"), + 0L, null, 1L, 1, Collections.emptyList(), true, null); + assertThat(SnapshotRetentionTask.snapshotEligibleForDeletion(info, mkInfos.apply(info), policyWithNoRetentionMap), equalTo(false)); + + // Test when user metadata is a map that doesn't contain "policy" + info = new SnapshotInfo(new SnapshotId("name", "uuid"), Collections.singletonList("index"), + 0L, null, 1L, 1, Collections.emptyList(), true, Collections.singletonMap("foo", "bar")); + assertThat(SnapshotRetentionTask.snapshotEligibleForDeletion(info, mkInfos.apply(info), policyMap), equalTo(false)); + + // Test with an ancient snapshot that should be expunged + info = new SnapshotInfo(new SnapshotId("name", "uuid"), Collections.singletonList("index"), + 0L, null, 1L, 1, Collections.emptyList(), true, Collections.singletonMap("policy", "policy")); + assertThat(SnapshotRetentionTask.snapshotEligibleForDeletion(info, mkInfos.apply(info), policyMap), equalTo(true)); + + // Test with a snapshot that's start date is old enough to be expunged (but the finish date is not) + long time = System.currentTimeMillis() - TimeValue.timeValueDays(30).millis() - 1; + info = new SnapshotInfo(new SnapshotId("name", "uuid"), Collections.singletonList("index"), + time, null, time + TimeValue.timeValueDays(4).millis(), 1, Collections.emptyList(), + true, Collections.singletonMap("policy", "policy")); + assertThat(SnapshotRetentionTask.snapshotEligibleForDeletion(info, mkInfos.apply(info), policyMap), equalTo(true)); + + // Test with a fresh snapshot that should not be expunged + info = new SnapshotInfo(new SnapshotId("name", "uuid"), Collections.singletonList("index"), + System.currentTimeMillis(), null, System.currentTimeMillis() + 1, + 1, Collections.emptyList(), true, Collections.singletonMap("policy", "policy")); + assertThat(SnapshotRetentionTask.snapshotEligibleForDeletion(info, mkInfos.apply(info), policyMap), equalTo(false)); + } + + public void testRetentionTaskSuccess() throws Exception { + retentionTaskTest(true); + } + + public void testRetentionTaskFailure() throws Exception { + retentionTaskTest(false); + } + + private void retentionTaskTest(final boolean deletionSuccess) throws Exception { + try (ThreadPool threadPool = new TestThreadPool("slm-test"); + ClusterService clusterService = ClusterServiceUtils.createClusterService(threadPool); + Client noOpClient = new NoOpClient("slm-test")) { + + final String policyId = "policy"; + final String repoId = "repo"; + SnapshotLifecyclePolicy policy = new SnapshotLifecyclePolicy(policyId, "snap", "1 * * * * ?", + repoId, null, new SnapshotRetentionConfiguration(TimeValue.timeValueDays(30), null, null)); + + ClusterState state = createState(policy); + ClusterServiceUtils.setState(clusterService, state); + + final SnapshotInfo eligibleSnapshot = new SnapshotInfo(new SnapshotId("name", "uuid"), Collections.singletonList("index"), + 0L, null, 1L, 1, Collections.emptyList(), true, Collections.singletonMap("policy", policyId)); + final SnapshotInfo ineligibleSnapshot = new SnapshotInfo(new SnapshotId("name2", "uuid2"), Collections.singletonList("index"), + System.currentTimeMillis(), null, System.currentTimeMillis() + 1, 1, + Collections.emptyList(), true, Collections.singletonMap("policy", policyId)); + + Set deleted = ConcurrentHashMap.newKeySet(); + Set deletedSnapshotsInHistory = ConcurrentHashMap.newKeySet(); + CountDownLatch deletionLatch = new CountDownLatch(1); + CountDownLatch historyLatch = new CountDownLatch(1); + + MockSnapshotRetentionTask retentionTask = new MockSnapshotRetentionTask(noOpClient, clusterService, + new SnapshotLifecycleTaskTests.VerifyingHistoryStore(noOpClient, ZoneOffset.UTC, + (historyItem) -> { + assertEquals(deletionSuccess, historyItem.isSuccess()); + if (historyItem.isSuccess() == false) { + assertThat(historyItem.getErrorDetails(), containsString("deletion_failed")); + } + assertEquals(policyId, historyItem.getPolicyId()); + assertEquals(repoId, historyItem.getRepository()); + assertEquals(DELETE_OPERATION, historyItem.getOperation()); + deletedSnapshotsInHistory.add(historyItem.getSnapshotName()); + historyLatch.countDown(); + }), + threadPool, + () -> { + List snaps = new ArrayList<>(2); + snaps.add(eligibleSnapshot); + snaps.add(ineligibleSnapshot); + logger.info("--> retrieving snapshots [{}]", snaps); + return Collections.singletonMap(repoId, snaps); + }, + (deletionPolicyId, repo, snapId, slmStats, listener) -> { + logger.info("--> deleting {} from repo {}", snapId, repo); + deleted.add(snapId); + if (deletionSuccess) { + listener.onResponse(new AcknowledgedResponse(true)); + } else { + listener.onFailure(new RuntimeException("deletion_failed")); + } + deletionLatch.countDown(); + }, + System::nanoTime); + + long time = System.currentTimeMillis(); + retentionTask.triggered(new SchedulerEngine.Event(SnapshotRetentionService.SLM_RETENTION_JOB_ID, time, time)); + + deletionLatch.await(10, TimeUnit.SECONDS); + + assertThat("something should have been deleted", deleted, not(empty())); + assertThat("one snapshot should have been deleted", deleted, hasSize(1)); + assertThat(deleted, contains(eligibleSnapshot.snapshotId())); + + boolean historySuccess = historyLatch.await(10, TimeUnit.SECONDS); + assertThat("expected history entries for 1 snapshot deletions", historySuccess, equalTo(true)); + assertThat(deletedSnapshotsInHistory, contains(eligibleSnapshot.snapshotId().getName())); + + threadPool.shutdownNow(); + threadPool.awaitTermination(10, TimeUnit.SECONDS); + } + } + + public void testSuccessfulTimeBoundedDeletion() throws Exception { + timeBoundedDeletion(true); + } + + public void testFailureTimeBoundedDeletion() throws Exception { + timeBoundedDeletion(false); + } + + private void timeBoundedDeletion(final boolean deletionSuccess) throws Exception { + try (ThreadPool threadPool = new TestThreadPool("slm-test"); + ClusterService clusterService = ClusterServiceUtils.createClusterService(threadPool); + Client noOpClient = new NoOpClient("slm-test")) { + + final String policyId = "policy"; + final String repoId = "repo"; + SnapshotLifecyclePolicy policy = new SnapshotLifecyclePolicy(policyId, "snap", "1 * * * * ?", + repoId, null, new SnapshotRetentionConfiguration(null, null, 1)); + + ClusterState state = createState(policy); + state = ClusterState.builder(state) + .metaData(MetaData.builder(state.metaData()) + .transientSettings(Settings.builder() + .put(LifecycleSettings.SLM_RETENTION_DURATION, "500ms") + .build())).build(); + ClusterServiceUtils.setState(clusterService, state); + + final SnapshotInfo snap1 = new SnapshotInfo(new SnapshotId("name1", "uuid1"), Collections.singletonList("index"), + 0L, null, 1L, 1, Collections.emptyList(), true, Collections.singletonMap("policy", policyId)); + final SnapshotInfo snap2 = new SnapshotInfo(new SnapshotId("name2", "uuid2"), Collections.singletonList("index"), + 1L, null, 2L, 1, Collections.emptyList(), true, Collections.singletonMap("policy", policyId)); + final SnapshotInfo snap3 = new SnapshotInfo(new SnapshotId("name3", "uuid3"), Collections.singletonList("index"), + 2L, null, 3L, 1, Collections.emptyList(), true, Collections.singletonMap("policy", policyId)); + final SnapshotInfo snap4 = new SnapshotInfo(new SnapshotId("name4", "uuid4"), Collections.singletonList("index"), + 3L, null, 4L, 1, Collections.emptyList(), true, Collections.singletonMap("policy", policyId)); + final SnapshotInfo snap5 = new SnapshotInfo(new SnapshotId("name5", "uuid5"), Collections.singletonList("index"), + 4L, null, 5L, 1, Collections.emptyList(), true, Collections.singletonMap("policy", policyId)); + + final Set deleted = ConcurrentHashMap.newKeySet(); + // We're expected two deletions before they hit the "taken too long" test, so have a latch of 2 + CountDownLatch deletionLatch = new CountDownLatch(2); + CountDownLatch historyLatch = new CountDownLatch(2); + Set deletedSnapshotsInHistory = ConcurrentHashMap.newKeySet(); + AtomicLong nanos = new AtomicLong(System.nanoTime()); + MockSnapshotRetentionTask retentionTask = new MockSnapshotRetentionTask(noOpClient, clusterService, + new SnapshotLifecycleTaskTests.VerifyingHistoryStore(noOpClient, ZoneOffset.UTC, + (historyItem) -> { + assertEquals(deletionSuccess, historyItem.isSuccess()); + if (historyItem.isSuccess() == false) { + assertThat(historyItem.getErrorDetails(), containsString("deletion_failed")); + } + assertEquals(policyId, historyItem.getPolicyId()); + assertEquals(repoId, historyItem.getRepository()); + assertEquals(DELETE_OPERATION, historyItem.getOperation()); + deletedSnapshotsInHistory.add(historyItem.getSnapshotName()); + historyLatch.countDown(); + }), + threadPool, + () -> { + List snaps = Arrays.asList(snap1, snap2, snap3, snap4, snap5); + logger.info("--> retrieving snapshots [{}]", snaps); + return Collections.singletonMap(repoId, snaps); + }, + (deletionPolicyId, repo, snapId, slmStats, listener) -> { + logger.info("--> deleting {}", snapId); + // Don't pause until snapshot 2 + if (snapId.equals(snap2.snapshotId())) { + logger.info("--> pausing for 501ms while deleting snap2 to simulate deletion past a threshold"); + nanos.addAndGet(TimeValue.timeValueMillis(501).nanos()); + } + deleted.add(snapId); + if (deletionSuccess) { + listener.onResponse(new AcknowledgedResponse(true)); + } else { + listener.onFailure(new RuntimeException("deletion_failed")); + } + deletionLatch.countDown(); + }, + nanos::get); + + long time = System.currentTimeMillis(); + retentionTask.triggered(new SchedulerEngine.Event(SnapshotRetentionService.SLM_RETENTION_JOB_ID, time, time)); + + boolean success = deletionLatch.await(10, TimeUnit.SECONDS); + + assertThat("expected 2 snapshot deletions within 10 seconds, deleted: " + deleted, success, equalTo(true)); + + assertNotNull("something should have been deleted", deleted); + assertThat("two snapshots should have been deleted", deleted.size(), equalTo(2)); + assertThat(deleted, containsInAnyOrder(snap1.snapshotId(), snap2.snapshotId())); + + boolean historySuccess = historyLatch.await(10, TimeUnit.SECONDS); + assertThat("expected history entries for 2 snapshot deletions", historySuccess, equalTo(true)); + assertThat(deletedSnapshotsInHistory, containsInAnyOrder(snap1.snapshotId().getName(), snap2.snapshotId().getName())); + + threadPool.shutdownNow(); + threadPool.awaitTermination(10, TimeUnit.SECONDS); + } + } + + public void testOkToDeleteSnapshots() { + final Snapshot snapshot = new Snapshot("repo", new SnapshotId("name", "uuid")); + + SnapshotsInProgress inProgress = new SnapshotsInProgress( + new SnapshotsInProgress.Entry( + snapshot, true, false, SnapshotsInProgress.State.INIT, + Collections.singletonList(new IndexId("name", "id")), 0, 0, + ImmutableOpenMap.builder().build(), Collections.emptyMap())); + ClusterState state = ClusterState.builder(new ClusterName("cluster")) + .putCustom(SnapshotsInProgress.TYPE, inProgress) + .build(); + + assertThat(SnapshotRetentionTask.okayToDeleteSnapshots(state), equalTo(false)); + + SnapshotDeletionsInProgress delInProgress = new SnapshotDeletionsInProgress( + Collections.singletonList(new SnapshotDeletionsInProgress.Entry(snapshot, 0, 0))); + state = ClusterState.builder(new ClusterName("cluster")) + .putCustom(SnapshotDeletionsInProgress.TYPE, delInProgress) + .build(); + + assertThat(SnapshotRetentionTask.okayToDeleteSnapshots(state), equalTo(false)); + + RepositoryCleanupInProgress cleanupInProgress = new RepositoryCleanupInProgress(new RepositoryCleanupInProgress.Entry("repo", 0)); + state = ClusterState.builder(new ClusterName("cluster")) + .putCustom(RepositoryCleanupInProgress.TYPE, cleanupInProgress) + .build(); + + assertThat(SnapshotRetentionTask.okayToDeleteSnapshots(state), equalTo(false)); + + RestoreInProgress restoreInProgress = mock(RestoreInProgress.class); + state = ClusterState.builder(new ClusterName("cluster")) + .putCustom(RestoreInProgress.TYPE, restoreInProgress) + .build(); + + assertThat(SnapshotRetentionTask.okayToDeleteSnapshots(state), equalTo(false)); + } + + public void testSkipWhileStopping() throws Exception { + doTestSkipDuringMode(OperationMode.STOPPING); + } + + public void testSkipWhileStopped() throws Exception { + doTestSkipDuringMode(OperationMode.STOPPED); + } + + private void doTestSkipDuringMode(OperationMode mode) throws Exception { + try (ThreadPool threadPool = new TestThreadPool("slm-test"); + ClusterService clusterService = ClusterServiceUtils.createClusterService(threadPool); + Client noOpClient = new NoOpClient("slm-test")) { + final String policyId = "policy"; + final String repoId = "repo"; + SnapshotLifecyclePolicy policy = new SnapshotLifecyclePolicy(policyId, "snap", "1 * * * * ?", + repoId, null, new SnapshotRetentionConfiguration(TimeValue.timeValueDays(30), null, null)); + + ClusterState state = createState(mode, policy); + ClusterServiceUtils.setState(clusterService, state); + + SnapshotRetentionTask task = new MockSnapshotRetentionTask(noOpClient, clusterService, + new SnapshotLifecycleTaskTests.VerifyingHistoryStore(noOpClient, ZoneOffset.UTC, + (historyItem) -> fail("should never write history")), + threadPool, + () -> { + fail("should not retrieve snapshots"); + return null; + }, + (a, b, c, d, e) -> fail("should not delete snapshots"), + System::nanoTime); + + long time = System.currentTimeMillis(); + task.triggered(new SchedulerEngine.Event(SnapshotRetentionService.SLM_RETENTION_JOB_ID, time, time)); + + threadPool.shutdownNow(); + threadPool.awaitTermination(10, TimeUnit.SECONDS); + } + } + + public ClusterState createState(SnapshotLifecyclePolicy... policies) { + return createState(OperationMode.RUNNING, policies); + } + + public ClusterState createState(OperationMode mode, SnapshotLifecyclePolicy... policies) { + Map policyMetadataMap = Arrays.stream(policies) + .map(policy -> SnapshotLifecyclePolicyMetadata.builder() + .setPolicy(policy) + .setHeaders(Collections.emptyMap()) + .setModifiedDate(randomNonNegativeLong()) + .setVersion(randomNonNegativeLong()) + .build()) + .collect(Collectors.toMap(pm -> pm.getPolicy().getId(), pm -> pm)); + + MetaData metaData = MetaData.builder() + .putCustom(SnapshotLifecycleMetadata.TYPE, + new SnapshotLifecycleMetadata(policyMetadataMap, mode, new SnapshotLifecycleStats())) + .build(); + return ClusterState.builder(new ClusterName("cluster")) + .metaData(metaData) + .build(); + } + + private static class MockSnapshotRetentionTask extends SnapshotRetentionTask { + private final Supplier>> snapshotRetriever; + private final DeleteSnapshotMock deleteRunner; + + MockSnapshotRetentionTask(Client client, + ClusterService clusterService, + SnapshotHistoryStore historyStore, + ThreadPool threadPool, + Supplier>> snapshotRetriever, + DeleteSnapshotMock deleteRunner, + LongSupplier nanoSupplier) { + super(client, clusterService, nanoSupplier, historyStore, threadPool); + this.snapshotRetriever = snapshotRetriever; + this.deleteRunner = deleteRunner; + } + + @Override + void getAllSuccessfulSnapshots(Collection repositories, + ActionListener>> listener, + Consumer errorHandler) { + listener.onResponse(this.snapshotRetriever.get()); + } + + @Override + void deleteSnapshot(String policyId, String repo, SnapshotId snapshot, SnapshotLifecycleStats slmStats, + ActionListener listener) { + deleteRunner.apply(policyId, repo, snapshot, slmStats, listener); + } + } + + @FunctionalInterface + interface DeleteSnapshotMock { + void apply(String policyId, String repo, SnapshotId snapshot, SnapshotLifecycleStats slmStats, + ActionListener listener); + } +} diff --git a/x-pack/plugin/src/test/resources/rest-api-spec/api/slm.get_stats.json b/x-pack/plugin/src/test/resources/rest-api-spec/api/slm.get_stats.json new file mode 100644 index 0000000000000..233d302baee0b --- /dev/null +++ b/x-pack/plugin/src/test/resources/rest-api-spec/api/slm.get_stats.json @@ -0,0 +1,19 @@ +{ + "slm.get_stats":{ + "documentation":{ + "url":"https://www.elastic.co/guide/en/elasticsearch/reference/current/slm-api.html" + }, + "stability":"stable", + "url":{ + "paths":[ + { + "path":"/_slm/stats", + "methods":[ + "GET" + ] + } + ] + }, + "params":{} + } +}