Skip to content

Commit 852106c

Browse files
authored
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
1 parent 445aca9 commit 852106c

File tree

7 files changed

+198
-57
lines changed

7 files changed

+198
-57
lines changed

x-pack/plugin/core/src/main/java/org/elasticsearch/xpack/core/ilm/LifecycleSettings.java

+3
Original file line numberDiff line numberDiff line change
@@ -20,6 +20,7 @@ public class LifecycleSettings {
2020

2121
public static final String SLM_HISTORY_INDEX_ENABLED = "slm.history_index_enabled";
2222
public static final String SLM_RETENTION_SCHEDULE = "slm.retention_schedule";
23+
public static final String SLM_RETENTION_DURATION = "slm.retention_duration";
2324

2425

2526
public static final Setting<TimeValue> LIFECYCLE_POLL_INTERVAL_SETTING = Setting.timeSetting(LIFECYCLE_POLL_INTERVAL,
@@ -42,4 +43,6 @@ public class LifecycleSettings {
4243
SLM_RETENTION_SCHEDULE + "]", e);
4344
}
4445
}, Setting.Property.Dynamic, Setting.Property.NodeScope);
46+
public static final Setting<TimeValue> SLM_RETENTION_DURATION_SETTING = Setting.timeSetting(SLM_RETENTION_DURATION,
47+
TimeValue.timeValueHours(1), TimeValue.timeValueMillis(500), Setting.Property.Dynamic, Setting.Property.NodeScope);
4548
}

x-pack/plugin/ilm/qa/multi-node/src/test/java/org/elasticsearch/xpack/ilm/TimeSeriesLifecycleActionsIT.java

+4-3
Original file line numberDiff line numberDiff line change
@@ -383,7 +383,8 @@ public void testDeleteDuringSnapshot() throws Exception {
383383
// index document so snapshot actually does something
384384
indexDocument();
385385
// start snapshot
386-
request = new Request("PUT", "/_snapshot/repo/snapshot");
386+
String snapName = "snapshot-" + randomAlphaOfLength(6).toLowerCase(Locale.ROOT);
387+
request = new Request("PUT", "/_snapshot/repo/" + snapName);
387388
request.addParameter("wait_for_completion", "false");
388389
request.setJsonEntity("{\"indices\": \"" + index + "\"}");
389390
assertOK(client().performRequest(request));
@@ -392,8 +393,8 @@ public void testDeleteDuringSnapshot() throws Exception {
392393
// assert that index was deleted
393394
assertBusy(() -> assertFalse(indexExists(index)), 2, TimeUnit.MINUTES);
394395
// assert that snapshot is still in progress and clean up
395-
assertThat(getSnapshotState("snapshot"), equalTo("SUCCESS"));
396-
assertOK(client().performRequest(new Request("DELETE", "/_snapshot/repo/snapshot")));
396+
assertThat(getSnapshotState(snapName), equalTo("SUCCESS"));
397+
assertOK(client().performRequest(new Request("DELETE", "/_snapshot/repo/" + snapName)));
397398
}
398399

399400
public void testReadOnly() throws Exception {

x-pack/plugin/ilm/qa/multi-node/src/test/java/org/elasticsearch/xpack/slm/SnapshotLifecycleIT.java

+3-2
Original file line numberDiff line numberDiff line change
@@ -36,6 +36,7 @@
3636
import java.util.HashMap;
3737
import java.util.List;
3838
import java.util.Map;
39+
import java.util.concurrent.TimeUnit;
3940

4041
import static org.elasticsearch.common.xcontent.XContentFactory.jsonBuilder;
4142
import static org.hamcrest.Matchers.containsString;
@@ -293,12 +294,13 @@ public void testBasicTimeBasedRetenion() throws Exception {
293294
assertBusy(() -> {
294295
// We expect a failed response because the snapshot should not exist
295296
try {
297+
logger.info("--> checking to see if snapshot has been deleted...");
296298
Response response = client().performRequest(new Request("GET", "/_snapshot/" + repoId + "/" + snapshotName));
297299
assertThat(EntityUtils.toString(response.getEntity()), containsString("snapshot_missing_exception"));
298300
} catch (ResponseException e) {
299301
assertThat(EntityUtils.toString(e.getResponse().getEntity()), containsString("snapshot_missing_exception"));
300302
}
301-
});
303+
}, 60, TimeUnit.SECONDS);
302304

303305
Request delReq = new Request("DELETE", "/_slm/policy/" + policyName);
304306
assertOK(client().performRequest(delReq));
@@ -424,7 +426,6 @@ private void inializeRepo(String repoName) throws IOException {
424426
.startObject("settings")
425427
.field("compress", randomBoolean())
426428
.field("location", System.getProperty("tests.path.repo"))
427-
.field("max_snapshot_bytes_per_sec", "256b")
428429
.endObject()
429430
.endObject()));
430431
assertOK(client().performRequest(request));

x-pack/plugin/ilm/src/main/java/org/elasticsearch/xpack/ilm/IndexLifecycle.java

+2-1
Original file line numberDiff line numberDiff line change
@@ -154,7 +154,8 @@ public Collection<Object> createComponents(Client client, ClusterService cluster
154154
snapshotHistoryStore.set(new SnapshotHistoryStore(settings, client, getClock().getZone()));
155155
snapshotLifecycleService.set(new SnapshotLifecycleService(settings,
156156
() -> new SnapshotLifecycleTask(client, clusterService, snapshotHistoryStore.get()), clusterService, getClock()));
157-
snapshotRetentionService.set(new SnapshotRetentionService(settings, () -> new SnapshotRetentionTask(client, clusterService),
157+
snapshotRetentionService.set(new SnapshotRetentionService(settings,
158+
() -> new SnapshotRetentionTask(client, clusterService, System::nanoTime),
158159
clusterService, getClock()));
159160
return Arrays.asList(indexLifecycleInitialisationService.get(), snapshotLifecycleService.get(), snapshotHistoryStore.get(),
160161
snapshotRetentionService.get());

x-pack/plugin/ilm/src/main/java/org/elasticsearch/xpack/slm/SnapshotRetentionTask.java

+70-36
Original file line numberDiff line numberDiff line change
@@ -18,8 +18,12 @@
1818
import org.elasticsearch.cluster.ClusterState;
1919
import org.elasticsearch.cluster.service.ClusterService;
2020
import org.elasticsearch.common.Strings;
21+
import org.elasticsearch.common.unit.TimeValue;
22+
import org.elasticsearch.snapshots.SnapshotId;
2123
import org.elasticsearch.snapshots.SnapshotInfo;
24+
import org.elasticsearch.snapshots.SnapshotState;
2225
import org.elasticsearch.xpack.core.ClientHelper;
26+
import org.elasticsearch.xpack.core.ilm.LifecycleSettings;
2327
import org.elasticsearch.xpack.core.scheduler.SchedulerEngine;
2428
import org.elasticsearch.xpack.core.slm.SnapshotLifecycleMetadata;
2529
import org.elasticsearch.xpack.core.slm.SnapshotLifecyclePolicy;
@@ -35,6 +39,7 @@
3539
import java.util.concurrent.CountDownLatch;
3640
import java.util.concurrent.atomic.AtomicBoolean;
3741
import java.util.function.Consumer;
42+
import java.util.function.LongSupplier;
3843
import java.util.stream.Collectors;
3944

4045
/**
@@ -50,10 +55,12 @@ public class SnapshotRetentionTask implements SchedulerEngine.Listener {
5055

5156
private final Client client;
5257
private final ClusterService clusterService;
58+
private final LongSupplier nowNanoSupplier;
5359

54-
public SnapshotRetentionTask(Client client, ClusterService clusterService) {
60+
public SnapshotRetentionTask(Client client, ClusterService clusterService, LongSupplier nowNanoSupplier) {
5561
this.client = new OriginSettingClient(client, ClientHelper.INDEX_LIFECYCLE_ORIGIN);
5662
this.clusterService = clusterService;
63+
this.nowNanoSupplier = nowNanoSupplier;
5764
}
5865

5966
@Override
@@ -64,6 +71,7 @@ public void triggered(SchedulerEngine.Event event) {
6471
try {
6572
logger.info("starting SLM retention snapshot cleanup task");
6673
final ClusterState state = clusterService.state();
74+
final TimeValue maxDeletionTime = LifecycleSettings.SLM_RETENTION_DURATION_SETTING.get(state.metaData().settings());
6775

6876
// Find all SLM policies that have retention enabled
6977
final Map<String, SnapshotLifecyclePolicy> policiesWithRetention = getAllPoliciesWithRetentionEnabled(state);
@@ -74,7 +82,7 @@ public void triggered(SchedulerEngine.Event event) {
7482
.map(SnapshotLifecyclePolicy::getRepository)
7583
.collect(Collectors.toSet());
7684

77-
getAllSnapshots(repositioriesToFetch, new ActionListener<>() {
85+
getAllSuccessfulSnapshots(repositioriesToFetch, new ActionListener<>() {
7886
@Override
7987
public void onResponse(Map<String, List<SnapshotInfo>> allSnapshots) {
8088
// Find all the snapshots that are past their retention date
@@ -85,7 +93,7 @@ public void onResponse(Map<String, List<SnapshotInfo>> allSnapshots) {
8593
.collect(Collectors.toList())));
8694

8795
// Finally, delete the snapshots that need to be deleted
88-
deleteSnapshots(snapshotsToBeDeleted);
96+
deleteSnapshots(snapshotsToBeDeleted, maxDeletionTime);
8997
}
9098

9199
@Override
@@ -160,8 +168,8 @@ static boolean snapshotEligibleForDeletion(SnapshotInfo snapshot, Map<String, Li
160168
return eligible;
161169
}
162170

163-
void getAllSnapshots(Collection<String> repositories, ActionListener<Map<String, List<SnapshotInfo>>> listener,
164-
Consumer<Exception> errorHandler) {
171+
void getAllSuccessfulSnapshots(Collection<String> repositories, ActionListener<Map<String, List<SnapshotInfo>>> listener,
172+
Consumer<Exception> errorHandler) {
165173
if (repositories.isEmpty()) {
166174
// Skip retrieving anything if there are no repositories to fetch
167175
listener.onResponse(Collections.emptyMap());
@@ -175,7 +183,11 @@ void getAllSnapshots(Collection<String> repositories, ActionListener<Map<String,
175183
public void onResponse(final GetSnapshotsResponse resp) {
176184
Map<String, List<SnapshotInfo>> snapshots = new HashMap<>();
177185
repositories.forEach(repo -> {
178-
snapshots.put(repo, resp.getSnapshots(repo));
186+
snapshots.put(repo,
187+
// Only return snapshots in the SUCCESS state
188+
resp.getSnapshots(repo).stream()
189+
.filter(info -> info.state() == SnapshotState.SUCCESS)
190+
.collect(Collectors.toList()));
179191
});
180192
listener.onResponse(snapshots);
181193
}
@@ -188,42 +200,64 @@ public void onFailure(Exception e) {
188200
});
189201
}
190202

191-
void deleteSnapshots(Map<String, List<SnapshotInfo>> snapshotsToDelete) {
192-
// TODO: make this more resilient and possibly only delete for a certain amount of time
203+
void deleteSnapshots(Map<String, List<SnapshotInfo>> snapshotsToDelete, TimeValue maximumTime) {
193204
int count = snapshotsToDelete.values().stream().mapToInt(List::size).sum();
194205
if (count == 0) {
195206
logger.debug("no snapshots are eligible for deletion");
196207
return;
197208
}
209+
198210
logger.info("starting snapshot retention deletion for [{}] snapshots", count);
199-
snapshotsToDelete.forEach((repo, snapshots) -> {
200-
snapshots.forEach(info -> {
201-
logger.info("[{}] snapshot retention deleting snapshot [{}]", repo, info.snapshotId());
202-
CountDownLatch latch = new CountDownLatch(1);
203-
client.admin().cluster().prepareDeleteSnapshot(repo, info.snapshotId().getName())
204-
.execute(new LatchedActionListener<>(new ActionListener<>() {
205-
@Override
206-
public void onResponse(AcknowledgedResponse acknowledgedResponse) {
207-
if (acknowledgedResponse.isAcknowledged()) {
208-
logger.debug("[{}] snapshot [{}] deleted successfully", repo, info.snapshotId());
209-
}
210-
}
211-
212-
@Override
213-
public void onFailure(Exception e) {
214-
logger.warn(new ParameterizedMessage("[{}] failed to delete snapshot [{}] for retention",
215-
repo, info.snapshotId()), e);
216-
}
217-
}, latch));
218-
try {
219-
// Deletes cannot occur simultaneously, so wait for this
220-
// deletion to complete before attempting the next one
221-
latch.await();
222-
} catch (InterruptedException e) {
223-
logger.error(new ParameterizedMessage("[{}] deletion of snapshot [{}] interrupted",
224-
repo, info.snapshotId()), e);
211+
long startTime = nowNanoSupplier.getAsLong();
212+
int deleted = 0;
213+
for (Map.Entry<String, List<SnapshotInfo>> entry : snapshotsToDelete.entrySet()) {
214+
String repo = entry.getKey();
215+
List<SnapshotInfo> snapshots = entry.getValue();
216+
for (SnapshotInfo info : snapshots) {
217+
deleteSnapshot(repo, info.snapshotId());
218+
deleted++;
219+
// Check whether we have exceeded the maximum time allowed to spend deleting
220+
// snapshots, if we have, short-circuit the rest of the deletions
221+
TimeValue elapsedDeletionTime = TimeValue.timeValueNanos(nowNanoSupplier.getAsLong() - startTime);
222+
logger.trace("elapsed time for deletion of [{}] snapshot: {}", info.snapshotId(), elapsedDeletionTime);
223+
if (elapsedDeletionTime.compareTo(maximumTime) > 0) {
224+
logger.info("maximum snapshot retention deletion time reached, time spent: [{}]," +
225+
" maximum allowed time: [{}], deleted {} out of {} snapshots scheduled for deletion",
226+
elapsedDeletionTime, maximumTime, deleted, count);
227+
return;
225228
}
226-
});
227-
});
229+
}
230+
}
231+
}
232+
233+
/**
234+
* Delete the given snapshot from the repository in blocking manner
235+
*/
236+
void deleteSnapshot(String repo, SnapshotId snapshot) {
237+
logger.info("[{}] snapshot retention deleting snapshot [{}]", repo, snapshot);
238+
CountDownLatch latch = new CountDownLatch(1);
239+
client.admin().cluster().prepareDeleteSnapshot(repo, snapshot.getName())
240+
.execute(new LatchedActionListener<>(new ActionListener<>() {
241+
@Override
242+
public void onResponse(AcknowledgedResponse acknowledgedResponse) {
243+
if (acknowledgedResponse.isAcknowledged()) {
244+
logger.debug("[{}] snapshot [{}] deleted successfully", repo, snapshot);
245+
}
246+
}
247+
248+
@Override
249+
public void onFailure(Exception e) {
250+
logger.warn(new ParameterizedMessage("[{}] failed to delete snapshot [{}] for retention",
251+
repo, snapshot), e);
252+
}
253+
}, latch));
254+
try {
255+
// Deletes cannot occur simultaneously, so wait for this
256+
// deletion to complete before attempting the next one
257+
latch.await();
258+
} catch (InterruptedException e) {
259+
logger.error(new ParameterizedMessage("[{}] deletion of snapshot [{}] interrupted",
260+
repo, snapshot), e);
261+
}
228262
}
229263
}

x-pack/plugin/ilm/src/test/java/org/elasticsearch/xpack/slm/SnapshotRetentionServiceTests.java

+1-1
Original file line numberDiff line numberDiff line change
@@ -67,7 +67,7 @@ public void testJobsAreScheduled() {
6767

6868
private static class FakeRetentionTask extends SnapshotRetentionTask {
6969
FakeRetentionTask() {
70-
super(mock(Client.class), null);
70+
super(mock(Client.class), null, System::nanoTime);
7171
}
7272

7373
@Override

0 commit comments

Comments
 (0)