diff --git a/docs/changelog/121370.yaml b/docs/changelog/121370.yaml new file mode 100644 index 0000000000000..cfa67bf5b2644 --- /dev/null +++ b/docs/changelog/121370.yaml @@ -0,0 +1,5 @@ +pr: 121370 +summary: Improve SLM Health Indicator to cover missing snapshot +area: ILM+SLM +type: enhancement +issues: [] diff --git a/libs/core/src/main/java/org/elasticsearch/core/TimeValue.java b/libs/core/src/main/java/org/elasticsearch/core/TimeValue.java index 89c2494cd128c..a957552528831 100644 --- a/libs/core/src/main/java/org/elasticsearch/core/TimeValue.java +++ b/libs/core/src/main/java/org/elasticsearch/core/TimeValue.java @@ -23,6 +23,7 @@ public class TimeValue implements Comparable { public static final TimeValue MAX_VALUE = new TimeValue(Long.MAX_VALUE, TimeUnit.NANOSECONDS); public static final TimeValue THIRTY_SECONDS = new TimeValue(30, TimeUnit.SECONDS); public static final TimeValue ONE_MINUTE = new TimeValue(1, TimeUnit.MINUTES); + public static final TimeValue ONE_HOUR = new TimeValue(1, TimeUnit.HOURS); private static final long C0 = 1L; private static final long C1 = C0 * 1000L; diff --git a/server/src/main/java/org/elasticsearch/TransportVersions.java b/server/src/main/java/org/elasticsearch/TransportVersions.java index 1a4bb7fde5ef6..af04d18f66866 100644 --- a/server/src/main/java/org/elasticsearch/TransportVersions.java +++ b/server/src/main/java/org/elasticsearch/TransportVersions.java @@ -187,6 +187,7 @@ static TransportVersion def(int id) { public static final TransportVersion ESQL_PROFILE_ASYNC_NANOS = def(9_007_00_0); public static final TransportVersion ESQL_LOOKUP_JOIN_SOURCE_TEXT = def(9_008_0_00); public static final TransportVersion REMOVE_ALL_APPLICABLE_SELECTOR = def(9_009_0_00); + public static final TransportVersion SLM_UNHEALTHY_IF_NO_SNAPSHOT_WITHIN = def(9_010_0_00); /* * STOP! READ THIS FIRST! No, really, diff --git a/test/framework/src/main/java/org/elasticsearch/test/ESTestCase.java b/test/framework/src/main/java/org/elasticsearch/test/ESTestCase.java index 227d7ca3046f8..0f69d0d37143c 100644 --- a/test/framework/src/main/java/org/elasticsearch/test/ESTestCase.java +++ b/test/framework/src/main/java/org/elasticsearch/test/ESTestCase.java @@ -1459,7 +1459,8 @@ public static void assertBusy(CheckedRunnable codeBlock) throws Excep } /** - * Runs the code block for the provided interval, waiting for no assertions to trip. + * Runs the code block for the provided interval, waiting for no assertions to trip. Retries on AssertionError + * with exponential backoff until provided time runs out */ public static void assertBusy(CheckedRunnable codeBlock, long maxWaitTime, TimeUnit unit) throws Exception { long maxTimeInMillis = TimeUnit.MILLISECONDS.convert(maxWaitTime, unit); 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 23bf21004040a..7290710d6d042 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 @@ -7,6 +7,7 @@ package org.elasticsearch.xpack.core.slm; +import org.elasticsearch.TransportVersions; import org.elasticsearch.action.ActionRequestValidationException; import org.elasticsearch.action.admin.cluster.snapshots.create.CreateSnapshotRequest; import org.elasticsearch.cluster.SimpleDiffable; @@ -52,12 +53,14 @@ public class SnapshotLifecyclePolicy implements SimpleDiffable configuration; private final SnapshotRetentionConfiguration retentionPolicy; private final boolean isCronSchedule; + private final TimeValue unhealthyIfNoSnapshotWithin; 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 ParseField UNHEALTHY_IF_NO_SNAPSHOT_WITHIN = new ParseField("unhealthy_if_no_snapshot_within"); private static final String METADATA_FIELD_NAME = "metadata"; @SuppressWarnings("unchecked") @@ -70,7 +73,8 @@ public class SnapshotLifecyclePolicy implements SimpleDiffable config = (Map) a[3]; SnapshotRetentionConfiguration retention = (SnapshotRetentionConfiguration) a[4]; - return new SnapshotLifecyclePolicy(id, name, schedule, repo, config, retention); + TimeValue unhealthyIfNoSnapshotWithin = (TimeValue) a[5]; + return new SnapshotLifecyclePolicy(id, name, schedule, repo, config, retention, unhealthyIfNoSnapshotWithin); } ); @@ -80,6 +84,11 @@ public class SnapshotLifecyclePolicy implements SimpleDiffable p.map(), CONFIG); PARSER.declareObject(ConstructingObjectParser.optionalConstructorArg(), SnapshotRetentionConfiguration::parse, RETENTION); + PARSER.declareString( + ConstructingObjectParser.optionalConstructorArg(), + value -> TimeValue.parseTimeValue(value, UNHEALTHY_IF_NO_SNAPSHOT_WITHIN.getPreferredName()), + UNHEALTHY_IF_NO_SNAPSHOT_WITHIN + ); } public SnapshotLifecyclePolicy( @@ -89,6 +98,18 @@ public SnapshotLifecyclePolicy( final String repository, @Nullable final Map configuration, @Nullable final SnapshotRetentionConfiguration retentionPolicy + ) { + this(id, name, schedule, repository, configuration, retentionPolicy, null); + } + + public SnapshotLifecyclePolicy( + final String id, + final String name, + final String schedule, + final String repository, + @Nullable final Map configuration, + @Nullable final SnapshotRetentionConfiguration retentionPolicy, + @Nullable final TimeValue unhealthyIfNoSnapshotWithin ) { this.id = Objects.requireNonNull(id, "policy id is required"); this.name = Objects.requireNonNull(name, "policy snapshot name is required"); @@ -96,6 +117,7 @@ public SnapshotLifecyclePolicy( this.repository = Objects.requireNonNull(repository, "policy snapshot repository is required"); this.configuration = configuration; this.retentionPolicy = retentionPolicy; + this.unhealthyIfNoSnapshotWithin = unhealthyIfNoSnapshotWithin; this.isCronSchedule = isCronSchedule(schedule); } @@ -106,6 +128,9 @@ public SnapshotLifecyclePolicy(StreamInput in) throws IOException { this.repository = in.readString(); this.configuration = in.readGenericMap(); this.retentionPolicy = in.readOptionalWriteable(SnapshotRetentionConfiguration::new); + this.unhealthyIfNoSnapshotWithin = in.getTransportVersion().onOrAfter(TransportVersions.SLM_UNHEALTHY_IF_NO_SNAPSHOT_WITHIN) + ? in.readOptionalTimeValue() + : null; this.isCronSchedule = isCronSchedule(schedule); } @@ -135,6 +160,11 @@ public SnapshotRetentionConfiguration getRetentionPolicy() { return this.retentionPolicy; } + @Nullable + public TimeValue getUnhealthyIfNoSnapshotWithin() { + return this.unhealthyIfNoSnapshotWithin; + } + boolean isCronSchedule() { return this.isCronSchedule; } @@ -236,6 +266,7 @@ public ActionRequestValidationException validate() { // Schedule validation // n.b. there's more validation beyond this in SnapshotLifecycleService#validateMinimumInterval + boolean canValidateUnhealthyIfNoSnapshotWithin = false; // true if schedule is syntactically valid if (Strings.hasText(schedule) == false) { err.addValidationError("invalid schedule [" + schedule + "]: must not be empty"); } else { @@ -244,13 +275,35 @@ public ActionRequestValidationException validate() { if (intervalTimeValue.millis() == 0) { err.addValidationError("invalid schedule [" + schedule + "]: time unit must be at least 1 millisecond"); } + canValidateUnhealthyIfNoSnapshotWithin = true; } catch (IllegalArgumentException e1) { if (isCronSchedule(schedule) == false) { err.addValidationError("invalid schedule [" + schedule + "]: must be a valid cron expression or time unit"); + } else { + canValidateUnhealthyIfNoSnapshotWithin = true; } } } + // validate unhealthyIfNoSnapshotWithin if schedule is syntactically valid + if (canValidateUnhealthyIfNoSnapshotWithin) { + TimeValue snapshotInterval = calculateNextInterval(Clock.systemUTC()); + if (unhealthyIfNoSnapshotWithin != null + && snapshotInterval.duration() > 0 + && unhealthyIfNoSnapshotWithin.compareTo(snapshotInterval) < 0) { + err.addValidationError( + "invalid unhealthy_if_no_snapshot_within [" + + unhealthyIfNoSnapshotWithin.getStringRep() + + "]: " + + "time is too short, expecting at least more than the interval between snapshots [" + + snapshotInterval.toHumanReadableString(2) + + "] for schedule [" + + schedule + + "]" + ); + } + } + if (configuration != null && configuration.containsKey(METADATA_FIELD_NAME)) { if (configuration.get(METADATA_FIELD_NAME) instanceof Map == false) { err.addValidationError( @@ -297,7 +350,7 @@ public ActionRequestValidationException validate() { err.addValidationError("invalid repository name [" + repository + "]: cannot be empty"); } - return err.validationErrors().size() == 0 ? null : err; + return err.validationErrors().isEmpty() ? null : err; } private Map addPolicyNameToMetadata(final Map metadata) { @@ -339,6 +392,9 @@ public void writeTo(StreamOutput out) throws IOException { out.writeString(this.repository); out.writeGenericMap(this.configuration); out.writeOptionalWriteable(this.retentionPolicy); + if (out.getTransportVersion().onOrAfter(TransportVersions.SLM_UNHEALTHY_IF_NO_SNAPSHOT_WITHIN)) { + out.writeOptionalTimeValue(this.unhealthyIfNoSnapshotWithin); + } } @Override @@ -353,13 +409,16 @@ public XContentBuilder toXContent(XContentBuilder builder, Params params) throws if (this.retentionPolicy != null) { builder.field(RETENTION.getPreferredName(), this.retentionPolicy); } + if (this.unhealthyIfNoSnapshotWithin != null) { + builder.field(UNHEALTHY_IF_NO_SNAPSHOT_WITHIN.getPreferredName(), this.unhealthyIfNoSnapshotWithin); + } builder.endObject(); return builder; } @Override public int hashCode() { - return Objects.hash(id, name, schedule, repository, configuration, retentionPolicy); + return Objects.hash(id, name, schedule, repository, configuration, retentionPolicy, unhealthyIfNoSnapshotWithin); } @Override @@ -377,7 +436,8 @@ public boolean equals(Object obj) { && Objects.equals(schedule, other.schedule) && Objects.equals(repository, other.repository) && Objects.equals(configuration, other.configuration) - && Objects.equals(retentionPolicy, other.retentionPolicy); + && Objects.equals(retentionPolicy, other.retentionPolicy) + && Objects.equals(unhealthyIfNoSnapshotWithin, other.unhealthyIfNoSnapshotWithin); } @Override diff --git a/x-pack/plugin/slm/src/internalClusterTest/java/org/elasticsearch/xpack/slm/SLMHealthBlockedSnapshotIT.java b/x-pack/plugin/slm/src/internalClusterTest/java/org/elasticsearch/xpack/slm/SLMHealthBlockedSnapshotIT.java new file mode 100644 index 0000000000000..08eb3f2140ed0 --- /dev/null +++ b/x-pack/plugin/slm/src/internalClusterTest/java/org/elasticsearch/xpack/slm/SLMHealthBlockedSnapshotIT.java @@ -0,0 +1,328 @@ +/* + * Copyright Elasticsearch B.V. and/or licensed to Elasticsearch B.V. under one + * or more contributor license agreements. Licensed under the Elastic License + * 2.0; you may not use this file except in compliance with the Elastic License + * 2.0. + */ + +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.action.admin.indices.refresh.RefreshRequest; +import org.elasticsearch.action.index.IndexRequestBuilder; +import org.elasticsearch.cluster.SnapshotsInProgress; +import org.elasticsearch.cluster.metadata.RepositoryMetadata; +import org.elasticsearch.cluster.service.ClusterService; +import org.elasticsearch.common.settings.Settings; +import org.elasticsearch.common.util.BigArrays; +import org.elasticsearch.core.TimeValue; +import org.elasticsearch.datastreams.DataStreamsPlugin; +import org.elasticsearch.env.Environment; +import org.elasticsearch.health.Diagnosis; +import org.elasticsearch.health.GetHealthAction; +import org.elasticsearch.health.HealthIndicatorImpact; +import org.elasticsearch.health.HealthIndicatorResult; +import org.elasticsearch.health.HealthStatus; +import org.elasticsearch.index.snapshots.blobstore.BlobStoreIndexShardSnapshot; +import org.elasticsearch.indices.recovery.RecoverySettings; +import org.elasticsearch.plugins.Plugin; +import org.elasticsearch.plugins.RepositoryPlugin; +import org.elasticsearch.repositories.RepositoriesMetrics; +import org.elasticsearch.repositories.Repository; +import org.elasticsearch.repositories.SnapshotShardContext; +import org.elasticsearch.repositories.fs.FsRepository; +import org.elasticsearch.snapshots.AbstractSnapshotIntegTestCase; +import org.elasticsearch.snapshots.SnapshotMissingException; +import org.elasticsearch.snapshots.mockstore.MockRepository; +import org.elasticsearch.test.ESIntegTestCase; +import org.elasticsearch.test.transport.MockTransportService; +import org.elasticsearch.xcontent.NamedXContentRegistry; +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.SnapshotRetentionConfiguration; +import org.elasticsearch.xpack.core.slm.action.ExecuteSnapshotLifecycleAction; +import org.elasticsearch.xpack.core.slm.action.PutSnapshotLifecycleAction; +import org.elasticsearch.xpack.ilm.IndexLifecycle; + +import java.io.IOException; +import java.util.ArrayList; +import java.util.Arrays; +import java.util.Collection; +import java.util.Collections; +import java.util.HashMap; +import java.util.List; +import java.util.Map; +import java.util.concurrent.CountDownLatch; +import java.util.concurrent.ExecutionException; +import java.util.concurrent.TimeUnit; +import java.util.concurrent.atomic.AtomicBoolean; + +import static org.hamcrest.Matchers.equalTo; +import static org.hamcrest.Matchers.notNullValue; + +@ESIntegTestCase.ClusterScope(scope = ESIntegTestCase.Scope.TEST, numDataNodes = 0) +public class SLMHealthBlockedSnapshotIT extends AbstractSnapshotIntegTestCase { + + // never auto-trigger, instead we will manually trigger in test for better control + private static final String NEVER_EXECUTE_CRON_SCHEDULE = "* * * 31 FEB ? *"; + + @Override + protected Collection> nodePlugins() { + return Arrays.asList( + MockRepository.Plugin.class, + MockTransportService.TestPlugin.class, + LocalStateCompositeXPackPlugin.class, + IndexLifecycle.class, + SnapshotLifecycle.class, + DataStreamsPlugin.class, + TestDelayedRepoPlugin.class + ); + } + + @Override + protected Settings nodeSettings(int nodeOrdinal, Settings otherSettings) { + return Settings.builder() + .put(super.nodeSettings(nodeOrdinal, otherSettings)) + .put(LifecycleSettings.LIFECYCLE_HISTORY_INDEX_ENABLED, false) + .put(LifecycleSettings.SLM_MINIMUM_INTERVAL, TimeValue.timeValueSeconds(1L)) // use a small value to allow frequent snapshot + .build(); + } + + public static class TestDelayedRepoPlugin extends Plugin implements RepositoryPlugin { + + // Use static vars since instantiated by plugin system + private static final AtomicBoolean doDelay = new AtomicBoolean(true); + private static final CountDownLatch delayedRepoLatch = new CountDownLatch(1); + + static void enable() { + doDelay.set(true); + } + + static void disable() { + doDelay.set(false); + } + + static void removeDelay() { + delayedRepoLatch.countDown(); + } + + @Override + public Map getRepositories( + Environment env, + NamedXContentRegistry namedXContentRegistry, + ClusterService clusterService, + BigArrays bigArrays, + RecoverySettings recoverySettings, + RepositoriesMetrics repositoriesMetrics + ) { + return Map.of( + TestDelayedRepo.TYPE, + metadata -> new TestDelayedRepo(metadata, env, namedXContentRegistry, clusterService, bigArrays, recoverySettings, () -> { + if (doDelay.get()) { + try { + assertTrue(delayedRepoLatch.await(1, TimeUnit.MINUTES)); + } catch (InterruptedException e) { + throw new RuntimeException(e); + } + } + }) + ); + } + } + + static class TestDelayedRepo extends FsRepository { + private static final String TYPE = "delayed"; + private final Runnable delayFn; + + protected TestDelayedRepo( + RepositoryMetadata metadata, + Environment env, + NamedXContentRegistry namedXContentRegistry, + ClusterService clusterService, + BigArrays bigArrays, + RecoverySettings recoverySettings, + Runnable delayFn + ) { + super(metadata, env, namedXContentRegistry, clusterService, bigArrays, recoverySettings); + this.delayFn = delayFn; + } + + @Override + protected void snapshotFile(SnapshotShardContext context, BlobStoreIndexShardSnapshot.FileInfo fileInfo) throws IOException { + delayFn.run(); + super.snapshotFile(context, fileInfo); + } + } + + public void testSlmHealthYellowWithBlockedSnapshot() throws Exception { + final String repoName = "test-repo"; + + internalCluster().startMasterOnlyNodes(1); + final String masterNode = internalCluster().getMasterName(); + final String dataNode = internalCluster().startDataOnlyNode(); + ensureStableCluster(2); + + createRepository(repoName, TestDelayedRepo.TYPE); + + String idxName = "test-index"; + String policyHealthy = "policy-health"; + String policyHealthyBelowThreshold = "policy-health-below-threshold"; + String policyUnhealthy = "policy-unhealthy"; + + List policyNames = List.of(policyHealthy, policyHealthyBelowThreshold, policyUnhealthy); + List policyNamesUnhealthy = List.of(policyUnhealthy); + + createRandomIndex(idxName); + putSnapshotPolicy(policyHealthy, "snap", NEVER_EXECUTE_CRON_SCHEDULE, repoName, idxName, null); + // 1hr unhealthyIfNoSnapshotWithin should not be exceeded during test period, so policy is healthy + putSnapshotPolicy(policyHealthyBelowThreshold, "snap", NEVER_EXECUTE_CRON_SCHEDULE, repoName, idxName, TimeValue.ONE_HOUR); + // zero unhealthyIfNoSnapshotWithin will always be exceeded, so policy is always unhealthy + putSnapshotPolicy(policyUnhealthy, "snap", NEVER_EXECUTE_CRON_SCHEDULE, repoName, idxName, TimeValue.ZERO); + + ensureGreen(); + + // allow snapshots to run + TestDelayedRepoPlugin.disable(); + + // create a successful snapshot, so there's baseline time to check against missing snapshot threshold + List firstSnapshots = executePolicies(masterNode, policyNames); + waitForSnapshotsAndClusterState(repoName, firstSnapshots); + + // block snapshot execution, create second set of snapshots, assert YELLOW health + TestDelayedRepoPlugin.enable(); + List secondSnapshots = executePolicies(masterNode, policyNames); + assertSlmYellowMissingSnapshot(policyNamesUnhealthy); + + // resume snapshot execution + TestDelayedRepoPlugin.removeDelay(); + waitForSnapshotsAndClusterState(repoName, secondSnapshots); + + // increase policy unhealthy threshold, assert GREEN health + putSnapshotPolicy(policyUnhealthy, "snap", NEVER_EXECUTE_CRON_SCHEDULE, repoName, idxName, TimeValue.ONE_HOUR); + assertBusy(() -> { + GetHealthAction.Request getHealthRequest = new GetHealthAction.Request(true, 1000); + GetHealthAction.Response health = admin().cluster().execute(GetHealthAction.INSTANCE, getHealthRequest).get(); + assertThat(health.getStatus(), equalTo(HealthStatus.GREEN)); + }); + } + + private void createRandomIndex(String idxName) throws InterruptedException { + createIndex(idxName); + + logger.info("--> indexing some data"); + final int numdocs = randomIntBetween(10, 100); + IndexRequestBuilder[] builders = new IndexRequestBuilder[numdocs]; + for (int i = 0; i < builders.length; i++) { + builders[i] = prepareIndex(idxName).setId(Integer.toString(i)).setSource("field1", "bar " + i); + } + indexRandom(true, builders); + indicesAdmin().refresh(new RefreshRequest(idxName)).actionGet(); + } + + private void putSnapshotPolicy( + String policyName, + String snapshotNamePattern, + String schedule, + String repoId, + String indexPattern, + TimeValue unhealthyIfNoSnapshotWithin + ) throws ExecutionException, InterruptedException { + Map snapConfig = new HashMap<>(); + snapConfig.put("indices", Collections.singletonList(indexPattern)); + snapConfig.put("ignore_unavailable", false); + snapConfig.put("partial", true); + + SnapshotLifecyclePolicy policy = new SnapshotLifecyclePolicy( + policyName, + snapshotNamePattern, + schedule, + repoId, + snapConfig, + SnapshotRetentionConfiguration.EMPTY, + unhealthyIfNoSnapshotWithin + ); + + PutSnapshotLifecycleAction.Request putLifecycle = new PutSnapshotLifecycleAction.Request( + TEST_REQUEST_TIMEOUT, + TEST_REQUEST_TIMEOUT, + policyName, + policy + ); + + client().execute(PutSnapshotLifecycleAction.INSTANCE, putLifecycle).get(); + } + + private void assertSlmYellowMissingSnapshot(List unhealthyPolicies) throws Exception { + assertBusy(() -> { + GetHealthAction.Request getHealthRequest = new GetHealthAction.Request(true, 1000); + GetHealthAction.Response health = admin().cluster().execute(GetHealthAction.INSTANCE, getHealthRequest).get(); + assertThat(health.getStatus(), equalTo(HealthStatus.YELLOW)); + HealthIndicatorResult slmIndicator = health.findIndicator(SlmHealthIndicatorService.NAME); + assertThat(slmIndicator.status(), equalTo(HealthStatus.YELLOW)); + assertThat(slmIndicator.impacts().size(), equalTo(1)); + assertThat(slmIndicator.impacts().getFirst().id(), equalTo(SlmHealthIndicatorService.MISSING_SNAPSHOT_IMPACT_ID)); + List missingSnapshotPolicies = slmIndicator.impacts() + .stream() + .filter(impact -> SlmHealthIndicatorService.MISSING_SNAPSHOT_IMPACT_ID.equals(impact.id())) + .toList(); + assertThat(missingSnapshotPolicies.size(), equalTo(unhealthyPolicies.size())); + + // validate affected policy names + assertThat(slmIndicator.diagnosisList().size(), equalTo(1)); + Diagnosis diagnosis = slmIndicator.diagnosisList().getFirst(); + List resources = diagnosis.affectedResources(); + assertThat(resources, notNullValue()); + assertThat(resources.size(), equalTo(1)); + assertThat(resources.getFirst().getValues(), equalTo(unhealthyPolicies)); + }); + } + + private List executePolicies(String node, List policies) throws Exception { + List snapshots = new ArrayList<>(); + for (String policyName : policies) { + snapshots.add(executePolicy(node, policyName)); + } + return snapshots; + } + + /** + * Execute the given policy and return the generated snapshot name + */ + private String executePolicy(String node, String policyId) throws ExecutionException, InterruptedException { + ExecuteSnapshotLifecycleAction.Request executeReq = new ExecuteSnapshotLifecycleAction.Request( + TEST_REQUEST_TIMEOUT, + TEST_REQUEST_TIMEOUT, + policyId + ); + ExecuteSnapshotLifecycleAction.Response resp = client(node).execute(ExecuteSnapshotLifecycleAction.INSTANCE, executeReq).get(); + return resp.getSnapshotName(); + } + + private void waitForSnapshotsAndClusterState(String repo, List snapshots) throws Exception { + for (String snapshot : snapshots) { + waitForSnapshot(repo, snapshot); + } + assertBusy(() -> assertTrue(SnapshotsInProgress.get(internalCluster().clusterService().state()).isEmpty())); + } + + private void waitForSnapshot(String repo, String snapshotName) throws Exception { + assertBusy(() -> { + try { + SnapshotsStatusResponse s = getSnapshotStatus(repo, snapshotName); + 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: {}", snapshotName, status.getState()); + assertThat(status.getState(), equalTo(SnapshotsInProgress.State.SUCCESS)); + } catch (SnapshotMissingException e) { + fail("expected a snapshot with name " + snapshotName + " but it does not exist"); + } + }); + } + + private SnapshotsStatusResponse getSnapshotStatus(String repo, String snapshotName) { + return clusterAdmin().prepareSnapshotStatus(TEST_REQUEST_TIMEOUT, repo).setSnapshots(snapshotName).get(); + } + +} diff --git a/x-pack/plugin/slm/src/main/java/org/elasticsearch/xpack/slm/SlmHealthIndicatorService.java b/x-pack/plugin/slm/src/main/java/org/elasticsearch/xpack/slm/SlmHealthIndicatorService.java index 6076214833704..829c9d034b389 100644 --- a/x-pack/plugin/slm/src/main/java/org/elasticsearch/xpack/slm/SlmHealthIndicatorService.java +++ b/x-pack/plugin/slm/src/main/java/org/elasticsearch/xpack/slm/SlmHealthIndicatorService.java @@ -9,7 +9,9 @@ import org.elasticsearch.cluster.ClusterState; import org.elasticsearch.cluster.service.ClusterService; +import org.elasticsearch.common.ReferenceDocs; import org.elasticsearch.common.time.DateFormatter; +import org.elasticsearch.core.TimeValue; import org.elasticsearch.health.Diagnosis; import org.elasticsearch.health.HealthIndicatorDetails; import org.elasticsearch.health.HealthIndicatorImpact; @@ -23,6 +25,7 @@ import org.elasticsearch.xpack.core.slm.SnapshotLifecycleMetadata; import org.elasticsearch.xpack.core.slm.SnapshotLifecyclePolicyMetadata; +import java.time.Instant; import java.time.ZoneOffset; import java.util.Collection; import java.util.Collections; @@ -65,6 +68,7 @@ public final class SlmHealthIndicatorService implements HealthIndicatorService { public static final String DIAGNOSIS_CHECK_RECENTLY_FAILED_SNAPSHOTS_ID = "check_recent_snapshot_failures"; public static final String DIAGNOSIS_CHECK_RECENTLY_FAILED_SNAPSHOTS_HELP_URL = "https://ela.st/fix-recent-snapshot-failures"; + public static final String DIAGNOSIS_CONTACT_SUPPORT_ID = "contact_support"; // Visible for testing static Diagnosis.Definition checkRecentlyFailedSnapshots(String causeText, String actionText) { @@ -77,8 +81,20 @@ static Diagnosis.Definition checkRecentlyFailedSnapshots(String causeText, Strin ); } + // Visible for testing + static Diagnosis.Definition contactSupport(String causeText, String actionText) { + return new Diagnosis.Definition( + NAME, + DIAGNOSIS_CONTACT_SUPPORT_ID, + causeText, + actionText, + ReferenceDocs.CONTACT_SUPPORT.toString() + ); + } + public static final String AUTOMATION_DISABLED_IMPACT_ID = "automation_disabled"; public static final String STALE_SNAPSHOTS_IMPACT_ID = "stale_snapshots"; + public static final String MISSING_SNAPSHOT_IMPACT_ID = "missing_snapshot"; private final ClusterService clusterService; private volatile long failedSnapshotWarnThreshold; @@ -130,70 +146,33 @@ public HealthIndicatorResult calculate(boolean verbose, int maxAffectedResources verbose ? List.of(SLM_NOT_RUNNING) : List.of() ); } else { - List unhealthyPolicies = slmMetadata.getSnapshotConfigurations() - .values() - .stream() + Collection snapshotConfigs = slmMetadata.getSnapshotConfigurations().values(); + + List failingSnapshotPolicies = snapshotConfigs.stream() .filter(metadata -> snapshotFailuresExceedWarningCount(failedSnapshotWarnThreshold, metadata)) .sorted(Comparator.comparing(SnapshotLifecyclePolicyMetadata::getId)) .toList(); - - if (unhealthyPolicies.size() > 0) { - List impacts = Collections.singletonList( - new HealthIndicatorImpact( - NAME, - STALE_SNAPSHOTS_IMPACT_ID, - 2, - "Some automated snapshots have not had a successful execution recently. Indices restored from affected " - + "snapshots may not contain recent changes.", - List.of(ImpactArea.BACKUP) - ) + if (failingSnapshotPolicies.isEmpty() == false) { + return createFailingSnapshotsIndicator( + failingSnapshotPolicies, + verbose, + maxAffectedResourcesCount, + slmMetadata, + currentMode ); + } - String unhealthyPolicyCauses = unhealthyPolicies.stream() - .map( - policy -> "- [" - + policy.getId() - + "] had [" - + policy.getInvocationsSinceLastSuccess() - + "] repeated failures without successful execution" - + (policy.getLastSuccess() != null && policy.getLastSuccess().getSnapshotStartTimestamp() != null - ? " since [" + FORMATTER.formatMillis(policy.getLastSuccess().getSnapshotStartTimestamp()) + "]" - : "") - ) - .collect(Collectors.joining("\n")); - String cause = (unhealthyPolicies.size() > 1 - ? "Several automated snapshot policies are unhealthy:\n" - : "An automated snapshot policy is unhealthy:\n") + unhealthyPolicyCauses; - - String unhealthyPolicyActions = unhealthyPolicies.stream() - .map(policy -> "- GET /_slm/policy/" + policy.getId() + "?human") - .collect(Collectors.joining("\n")); - String action = "Check the snapshot lifecycle " - + (unhealthyPolicies.size() > 1 ? "policies" : "policy") - + " for detailed failure info:\n" - + unhealthyPolicyActions; - - return createIndicator( - YELLOW, - "Encountered [" + unhealthyPolicies.size() + "] unhealthy snapshot lifecycle management policies.", - createDetails(verbose, unhealthyPolicies, slmMetadata, currentMode), - impacts, - verbose - ? List.of( - new Diagnosis( - checkRecentlyFailedSnapshots(cause, action), - List.of( - new Diagnosis.Resource( - Diagnosis.Resource.Type.SLM_POLICY, - unhealthyPolicies.stream() - .map(SnapshotLifecyclePolicyMetadata::getId) - .limit(Math.min(unhealthyPolicies.size(), maxAffectedResourcesCount)) - .toList() - ) - ) - ) - ) - : List.of() + List missingSnapshotPolicies = snapshotConfigs.stream() + .filter(SlmHealthIndicatorService::missingSnapshotTimeExceeded) + .sorted(Comparator.comparing(SnapshotLifecyclePolicyMetadata::getId)) + .toList(); + if (missingSnapshotPolicies.isEmpty() == false) { + return createMissingSnapshotIndicator( + missingSnapshotPolicies, + verbose, + maxAffectedResourcesCount, + slmMetadata, + currentMode ); } @@ -207,6 +186,21 @@ public HealthIndicatorResult calculate(boolean verbose, int maxAffectedResources } } + static boolean missingSnapshotTimeExceeded(SnapshotLifecyclePolicyMetadata policyMetadata) { + TimeValue unhealthyIfNoSnapshotWithin = policyMetadata.getPolicy().getUnhealthyIfNoSnapshotWithin(); + if (unhealthyIfNoSnapshotWithin == null) { + return false; + } + Long startTime = getMissingSnapshotPeriodStartTime(policyMetadata); + if (startTime != null) { + // time since last successful snapshot is exceeded + long now = Instant.now().toEpochMilli(); + long threshold = unhealthyIfNoSnapshotWithin.getMillis(); + return now - startTime > threshold; + } + return false; + } + static boolean snapshotFailuresExceedWarningCount(long failedSnapshotWarnThreshold, SnapshotLifecyclePolicyMetadata policyMetadata) { SnapshotInvocationRecord lastFailure = policyMetadata.getLastFailure(); if (lastFailure == null) { @@ -236,7 +230,7 @@ private static HealthIndicatorDetails createDetails( Map details = new LinkedHashMap<>(); details.put("slm_status", mode); details.put("policies", metadata.getSnapshotConfigurations().size()); - if (unhealthyPolicies.size() > 0) { + if (unhealthyPolicies.isEmpty() == false) { details.put( "unhealthy_policies", Map.of( @@ -255,4 +249,150 @@ private static HealthIndicatorDetails createDetails( } return new SimpleHealthIndicatorDetails(details); } + + private HealthIndicatorResult createFailingSnapshotsIndicator( + List unhealthyPolicies, + boolean verbose, + int maxAffectedResourcesCount, + SnapshotLifecycleMetadata slmMetadata, + OperationMode currentMode + ) { + List impacts = Collections.singletonList( + new HealthIndicatorImpact( + NAME, + STALE_SNAPSHOTS_IMPACT_ID, + 2, + "Some automated snapshots have not had a successful execution recently. Indices restored from affected " + + "snapshots may not contain recent changes.", + List.of(ImpactArea.BACKUP) + ) + ); + + String unhealthyPolicyCauses = unhealthyPolicies.stream() + .map( + policy -> "- [" + + policy.getId() + + "] had [" + + policy.getInvocationsSinceLastSuccess() + + "] repeated failures without successful execution" + + (policy.getLastSuccess() != null && policy.getLastSuccess().getSnapshotStartTimestamp() != null + ? " since [" + FORMATTER.formatMillis(policy.getLastSuccess().getSnapshotStartTimestamp()) + "]" + : "") + ) + .collect(Collectors.joining("\n")); + String cause = (unhealthyPolicies.size() > 1 + ? "Several automated snapshot policies are unhealthy:\n" + : "An automated snapshot policy is unhealthy:\n") + unhealthyPolicyCauses; + + String unhealthyPolicyActions = unhealthyPolicies.stream() + .map(policy -> "- GET /_slm/policy/" + policy.getId() + "?human") + .collect(Collectors.joining("\n")); + String action = "Check the snapshot lifecycle " + + (unhealthyPolicies.size() > 1 ? "policies" : "policy") + + " for detailed failure info:\n" + + unhealthyPolicyActions; + + return createIndicator( + YELLOW, + "Encountered [" + unhealthyPolicies.size() + "] unhealthy snapshot lifecycle management policies", + createDetails(verbose, unhealthyPolicies, slmMetadata, currentMode), + impacts, + verbose + ? List.of( + new Diagnosis( + checkRecentlyFailedSnapshots(cause, action), + List.of( + new Diagnosis.Resource( + Diagnosis.Resource.Type.SLM_POLICY, + unhealthyPolicies.stream() + .map(SnapshotLifecyclePolicyMetadata::getId) + .limit(Math.min(unhealthyPolicies.size(), maxAffectedResourcesCount)) + .toList() + ) + ) + ) + ) + : List.of() + ); + } + + private HealthIndicatorResult createMissingSnapshotIndicator( + List unhealthyPolicies, + boolean verbose, + int maxAffectedResourcesCount, + SnapshotLifecycleMetadata slmMetadata, + OperationMode currentMode + ) { + List impacts = Collections.singletonList( + new HealthIndicatorImpact( + NAME, + MISSING_SNAPSHOT_IMPACT_ID, + 2, + "Some snapshot lifecycle policies have not had a snapshot for some time", + List.of(ImpactArea.BACKUP) + ) + ); + + String unhealthyPolicyCauses = unhealthyPolicies.stream().map(policy -> { + Long missingStartTime = getMissingSnapshotPeriodStartTime(policy); + TimeValue unhealthyIfNoSnapshotWithin = policy.getPolicy().getUnhealthyIfNoSnapshotWithin(); + // missingStartTime and unhealthyIfNoSnapshotWithin should be non-null due to above filtering with missingSnapshotTimeExceeded + assert missingStartTime != null; + assert unhealthyIfNoSnapshotWithin != null; + return "- [" + + policy.getId() + + "] has not had a snapshot for " + + (unhealthyIfNoSnapshotWithin != null ? unhealthyIfNoSnapshotWithin.toHumanReadableString(2) : "some time") + + (missingStartTime != null ? ", since [" + FORMATTER.formatMillis(missingStartTime) + "]" : ""); + }).collect(Collectors.joining("\n")); + String cause = (unhealthyPolicies.size() > 1 + ? "Several automated snapshot policies are unhealthy:\n" + : "An automated snapshot policy is unhealthy:\n") + unhealthyPolicyCauses; + + String unhealthyPolicyActions = unhealthyPolicies.stream() + .map(policy -> "- GET /_slm/policy/" + policy.getId() + "?human") + .collect(Collectors.joining("\n")); + String action = "Check the snapshot lifecycle " + + (unhealthyPolicies.size() > 1 ? "policies" : "policy") + + " for detailed failure info:\n" + + unhealthyPolicyActions; + + return createIndicator( + YELLOW, + "Encountered [" + unhealthyPolicies.size() + "] unhealthy snapshot lifecycle management policies", + createDetails(verbose, unhealthyPolicies, slmMetadata, currentMode), + impacts, + verbose + ? List.of( + new Diagnosis( + contactSupport(cause, action), + List.of( + new Diagnosis.Resource( + Diagnosis.Resource.Type.SLM_POLICY, + unhealthyPolicies.stream() + .map(SnapshotLifecyclePolicyMetadata::getId) + .limit(Math.min(unhealthyPolicies.size(), maxAffectedResourcesCount)) + .toList() + ) + ) + ) + ) + : List.of() + ); + } + + private static Long getMissingSnapshotPeriodStartTime(SnapshotLifecyclePolicyMetadata policy) { + if (policy.getLastSuccess() != null) { + // prefer snapshotStartTimestamp over snapshotFinishTimestamp in case of a very long-running snapshot + // that started a long time ago + SnapshotInvocationRecord lastSuccess = policy.getLastSuccess(); + return lastSuccess.getSnapshotStartTimestamp() != null + ? lastSuccess.getSnapshotStartTimestamp() + : lastSuccess.getSnapshotFinishTimestamp(); + } + // TODO: handle first snapshot (i.e. no prior success of failure), maybe record the policy first trigger timestamp + + // SLM has not been triggered yet + return null; + } } diff --git a/x-pack/plugin/slm/src/test/java/org/elasticsearch/xpack/slm/SlmHealthIndicatorServiceTests.java b/x-pack/plugin/slm/src/test/java/org/elasticsearch/xpack/slm/SlmHealthIndicatorServiceTests.java index 9b0d20308cf76..b0946662a24de 100644 --- a/x-pack/plugin/slm/src/test/java/org/elasticsearch/xpack/slm/SlmHealthIndicatorServiceTests.java +++ b/x-pack/plugin/slm/src/test/java/org/elasticsearch/xpack/slm/SlmHealthIndicatorServiceTests.java @@ -15,6 +15,7 @@ import org.elasticsearch.common.settings.Settings; import org.elasticsearch.common.time.DateFormatter; import org.elasticsearch.core.Nullable; +import org.elasticsearch.core.TimeValue; import org.elasticsearch.health.Diagnosis; import org.elasticsearch.health.Diagnosis.Resource.Type; import org.elasticsearch.health.HealthIndicatorDetails; @@ -30,7 +31,9 @@ import org.elasticsearch.xpack.core.slm.SnapshotLifecyclePolicy; import org.elasticsearch.xpack.core.slm.SnapshotLifecyclePolicyMetadata; +import java.time.Instant; import java.time.ZoneOffset; +import java.time.temporal.ChronoUnit; import java.util.Collections; import java.util.List; import java.util.Map; @@ -145,10 +148,11 @@ public void testIsGreenWhenPoliciesHaveFailedForLessThanWarningThreshold() { long window = TimeUnit.HOURS.toMillis(24) - 5000L; // Just under 24 hours. var clusterState = createClusterStateWith( new SnapshotLifecycleMetadata( - createSlmPolicyWithInvocations( + createSlmPolicy( snapshotInvocation(randomBoolean() ? null : execTime, execTime + 1000L), snapshotInvocation(null, execTime + window + 1000L), - randomLongBetween(0, 4) + randomLongBetween(0, 4), + null ), RUNNING, null @@ -182,7 +186,9 @@ public void testIsYellowWhenPoliciesHaveFailedForMoreThanWarningThreshold() { Map.of( "test-policy", SnapshotLifecyclePolicyMetadata.builder() - .setPolicy(new SnapshotLifecyclePolicy("test-policy", "", "", "test-repository", null, null)) + .setPolicy( + new SnapshotLifecyclePolicy("test-policy", "", "", "test-repository", null, null, null) + ) .setVersion(1L) .setModifiedDate(System.currentTimeMillis()) .setLastSuccess(snapshotInvocation(execTime, execTime + 1000L)) @@ -198,6 +204,7 @@ public void testIsYellowWhenPoliciesHaveFailedForMoreThanWarningThreshold() { "", "test-repository", null, + null, null ) ) @@ -216,6 +223,7 @@ public void testIsYellowWhenPoliciesHaveFailedForMoreThanWarningThreshold() { "", "test-repository", null, + null, null ) ) @@ -239,7 +247,7 @@ public void testIsYellowWhenPoliciesHaveFailedForMoreThanWarningThreshold() { new HealthIndicatorResult( NAME, YELLOW, - "Encountered [3] unhealthy snapshot lifecycle management policies.", + "Encountered [3] unhealthy snapshot lifecycle management policies", new SimpleHealthIndicatorDetails( Map.of( "slm_status", @@ -306,9 +314,141 @@ public void testIsYellowWhenPoliciesHaveFailedForMoreThanWarningThreshold() { ); } + public void testIsYellowWhenPoliciesExceedsUnhealthyIfNoSnapshotWithin() { + long tenMinutesAgo = Instant.now().minus(10, ChronoUnit.MINUTES).toEpochMilli(); + long fiveMinutesAgo = Instant.now().minus(5, ChronoUnit.MINUTES).toEpochMilli(); + + TimeValue threshold = TimeValue.ONE_MINUTE; + + var clusterState = createClusterStateWith( + new SnapshotLifecycleMetadata( + Map.of( + "test-policy-no-time-configured", + SnapshotLifecyclePolicyMetadata.builder() + .setPolicy( + new SnapshotLifecyclePolicy("test-policy-no-time-configured", "test", "", "test-repository", null, null, null) + ) + .setVersion(1L) + .setModifiedDate(System.currentTimeMillis()) + .setLastSuccess(snapshotInvocation(tenMinutesAgo, fiveMinutesAgo)) + .build(), + "test-policy-does-not-exceed-time", + SnapshotLifecyclePolicyMetadata.builder() + .setPolicy( + new SnapshotLifecyclePolicy( + "test-policy-does-not-exceeds-time", + "test", + "", + "test-repository", + null, + null, + new TimeValue(1, TimeUnit.HOURS) + ) + ) + .setVersion(1L) + .setModifiedDate(System.currentTimeMillis()) + .setLastSuccess(snapshotInvocation(tenMinutesAgo, fiveMinutesAgo)) + .build(), + "test-policy-exceeds-time", + SnapshotLifecyclePolicyMetadata.builder() + .setPolicy( + new SnapshotLifecyclePolicy("test-policy-exceeds-time", "test", "", "test-repository", null, null, threshold) + ) + .setVersion(1L) + .setModifiedDate(System.currentTimeMillis()) + .setLastSuccess(snapshotInvocation(tenMinutesAgo, fiveMinutesAgo)) + .build(), + "test-policy-exceeds-time-without-success-start-time", + SnapshotLifecyclePolicyMetadata.builder() + .setPolicy( + new SnapshotLifecyclePolicy( + "test-policy-exceeds-time-without-success-start-time", + "test", + "", + "test-repository", + null, + null, + threshold + ) + ) + .setVersion(1L) + .setModifiedDate(System.currentTimeMillis()) + .setLastSuccess(snapshotInvocation(null, fiveMinutesAgo)) + .build() + // TODO: first snapshot + ), + RUNNING, + null + ) + ); + SlmHealthIndicatorService service = createSlmHealthIndicatorService(clusterState); + + HealthIndicatorResult calculate = service.calculate(true, HealthInfo.EMPTY_HEALTH_INFO); + assertThat( + calculate, + equalTo( + new HealthIndicatorResult( + NAME, + YELLOW, + "Encountered [2] unhealthy snapshot lifecycle management policies", + new SimpleHealthIndicatorDetails( + Map.of( + "slm_status", + RUNNING, + "policies", + 4, + "unhealthy_policies", + Map.of( + "count", + 2, + "invocations_since_last_success", + Map.of("test-policy-exceeds-time", 0L, "test-policy-exceeds-time-without-success-start-time", 0L) + ) + ) + ), + Collections.singletonList( + new HealthIndicatorImpact( + NAME, + SlmHealthIndicatorService.MISSING_SNAPSHOT_IMPACT_ID, + 2, + "Some snapshot lifecycle policies have not had a snapshot for some time", + List.of(ImpactArea.BACKUP) + ) + ), + List.of( + new Diagnosis( + SlmHealthIndicatorService.contactSupport( + "Several automated snapshot policies are unhealthy:\n" + + "- [test-policy-exceeds-time] has not had a snapshot for " + + threshold.toHumanReadableString(2) + + ", since [" + + FORMATTER.formatMillis(tenMinutesAgo) + + "]\n" + + "- [test-policy-exceeds-time-without-success-start-time] has not had a snapshot for " + + threshold.toHumanReadableString(2) + + ", since [" + + FORMATTER.formatMillis(fiveMinutesAgo) + + "]", + "Check the snapshot lifecycle policies for detailed failure info:\n" + + "- GET /_slm/policy/test-policy-exceeds-time?human\n" + + "- GET /_slm/policy/test-policy-exceeds-time-without-success-start-time?human" + ), + List.of( + new Diagnosis.Resource( + Type.SLM_POLICY, + List.of("test-policy-exceeds-time", "test-policy-exceeds-time-without-success-start-time") + ) + ) + ) + ) + ) + ) + ); + } + public void testSnapshotPolicyExceedsWarningThresholdPredicate() { SnapshotLifecyclePolicyMetadata slmPolicyMetadata = SnapshotLifecyclePolicyMetadata.builder() - .setPolicy(new SnapshotLifecyclePolicy("id", "test-policy", "", "test-repository", null, null)) + .setPolicy(new SnapshotLifecyclePolicy("id", "test-policy", "", "test-repository", null, null, null)) .setVersion(1L) .setModifiedDate(System.currentTimeMillis()) .build(); @@ -318,7 +458,7 @@ public void testSnapshotPolicyExceedsWarningThresholdPredicate() { assertThat(SlmHealthIndicatorService.snapshotFailuresExceedWarningCount(1L, slmPolicyMetadata), is(false)); slmPolicyMetadata = SnapshotLifecyclePolicyMetadata.builder() - .setPolicy(new SnapshotLifecyclePolicy("id", "test-policy", "", "test-repository", null, null)) + .setPolicy(new SnapshotLifecyclePolicy("id", "test-policy", "", "test-repository", null, null, null)) .setVersion(1L) .setModifiedDate(System.currentTimeMillis()) .setLastSuccess(snapshotInvocation(1000L, 2000L)) @@ -330,7 +470,7 @@ public void testSnapshotPolicyExceedsWarningThresholdPredicate() { assertThat(SlmHealthIndicatorService.snapshotFailuresExceedWarningCount(1L, slmPolicyMetadata), is(false)); slmPolicyMetadata = SnapshotLifecyclePolicyMetadata.builder() - .setPolicy(new SnapshotLifecyclePolicy("id", "test-policy", "", "test-repository", null, null)) + .setPolicy(new SnapshotLifecyclePolicy("id", "test-policy", "", "test-repository", null, null, null)) .setVersion(1L) .setModifiedDate(System.currentTimeMillis()) .setLastSuccess(snapshotInvocation(1000L, 2000L)) @@ -343,7 +483,7 @@ public void testSnapshotPolicyExceedsWarningThresholdPredicate() { assertThat(SlmHealthIndicatorService.snapshotFailuresExceedWarningCount(1L, slmPolicyMetadata), is(true)); slmPolicyMetadata = SnapshotLifecyclePolicyMetadata.builder() - .setPolicy(new SnapshotLifecyclePolicy("id", "test-policy", "", "test-repository", null, null)) + .setPolicy(new SnapshotLifecyclePolicy("id", "test-policy", "", "test-repository", null, null, null)) .setVersion(1L) .setModifiedDate(System.currentTimeMillis()) .setLastSuccess(snapshotInvocation(8000L, 9000L)) @@ -356,6 +496,61 @@ public void testSnapshotPolicyExceedsWarningThresholdPredicate() { assertThat(SlmHealthIndicatorService.snapshotFailuresExceedWarningCount(1L, slmPolicyMetadata), is(false)); } + public void testSnapshotPolicyMissingSnapshotTimeExceededPredicate() { + long tenMinutesAgo = Instant.now().minus(10, ChronoUnit.MINUTES).toEpochMilli(); + long fiveMinutesAgo = Instant.now().minus(5, ChronoUnit.MINUTES).toEpochMilli(); + // null unhealthyIfNoSnapshotWithin + { + SnapshotLifecyclePolicyMetadata slmPolicyMetadata = SnapshotLifecyclePolicyMetadata.builder() + .setPolicy(new SnapshotLifecyclePolicy("id", "test-policy", "", "test-repository", null, null, null)) + .setVersion(1L) + .setModifiedDate(System.currentTimeMillis()) + .setLastSuccess(new SnapshotInvocationRecord("test-snapshot", tenMinutesAgo, fiveMinutesAgo, null)) + .build(); + assertThat(SlmHealthIndicatorService.missingSnapshotTimeExceeded(slmPolicyMetadata), is(false)); + } + // does not exceed unhealthyIfNoSnapshotWithin + { + SnapshotLifecyclePolicyMetadata slmPolicyMetadata = SnapshotLifecyclePolicyMetadata.builder() + .setPolicy(new SnapshotLifecyclePolicy("id", "test-policy", "", "test-repository", null, null, TimeValue.MAX_VALUE)) + .setVersion(1L) + .setModifiedDate(System.currentTimeMillis()) + .setLastSuccess(new SnapshotInvocationRecord("test-snapshot", tenMinutesAgo, fiveMinutesAgo, null)) + .build(); + assertThat(SlmHealthIndicatorService.missingSnapshotTimeExceeded(slmPolicyMetadata), is(false)); + } + // exceed unhealthyIfNoSnapshotWithin + { + SnapshotLifecyclePolicyMetadata slmPolicyMetadata = SnapshotLifecyclePolicyMetadata.builder() + .setPolicy(new SnapshotLifecyclePolicy("id", "test-policy", "", "test-repository", null, null, TimeValue.ONE_MINUTE)) + .setVersion(1L) + .setModifiedDate(System.currentTimeMillis()) + .setLastSuccess(new SnapshotInvocationRecord("test-snapshot", tenMinutesAgo, fiveMinutesAgo, null)) + .build(); + assertThat(SlmHealthIndicatorService.missingSnapshotTimeExceeded(slmPolicyMetadata), is(true)); + } + // first snapshot, does not exceed unhealthyIfNoSnapshotWithin + { + SnapshotLifecyclePolicyMetadata slmPolicyMetadata = SnapshotLifecyclePolicyMetadata.builder() + .setPolicy(new SnapshotLifecyclePolicy("id", "test-policy", "", "test-repository", null, null, TimeValue.MAX_VALUE)) + .setVersion(1L) + .setModifiedDate(System.currentTimeMillis()) + // TODO: set first trigger time + .build(); + assertThat(SlmHealthIndicatorService.missingSnapshotTimeExceeded(slmPolicyMetadata), is(false)); + } + // first snapshot, exceed unhealthyIfNoSnapshotWithin + { + SnapshotLifecyclePolicyMetadata slmPolicyMetadata = SnapshotLifecyclePolicyMetadata.builder() + .setPolicy(new SnapshotLifecyclePolicy("id", "test-policy", "", "test-repository", null, null, TimeValue.ONE_MINUTE)) + .setVersion(1L) + .setModifiedDate(System.currentTimeMillis()) + // TODO: set first trigger time + .build(); + assertThat(SlmHealthIndicatorService.missingSnapshotTimeExceeded(slmPolicyMetadata), is(false)); + } + } + public void testSkippingFieldsWhenVerboseIsFalse() { var status = randomFrom(STOPPED, STOPPING); var clusterState = createClusterStateWith(new SnapshotLifecycleMetadata(createSlmPolicy(), status, null)); @@ -404,18 +599,21 @@ private static ClusterState createClusterStateWith(SnapshotLifecycleMetadata met } private static Map createSlmPolicy() { - return createSlmPolicyWithInvocations(null, null, 0L); + return createSlmPolicy(null, null, 0L, null); } - private static Map createSlmPolicyWithInvocations( + private static Map createSlmPolicy( SnapshotInvocationRecord lastSuccess, SnapshotInvocationRecord lastFailure, - long invocationsSinceLastSuccess + long invocationsSinceLastSuccess, + TimeValue unhealthyIfNoSnapshotWithin ) { return Map.of( "test-policy", SnapshotLifecyclePolicyMetadata.builder() - .setPolicy(new SnapshotLifecyclePolicy("policy-id", "test-policy", "", "test-repository", null, null)) + .setPolicy( + new SnapshotLifecyclePolicy("policy-id", "test-policy", "", "test-repository", null, null, unhealthyIfNoSnapshotWithin) + ) .setVersion(1L) .setModifiedDate(System.currentTimeMillis()) .setLastSuccess(lastSuccess) diff --git a/x-pack/plugin/slm/src/test/java/org/elasticsearch/xpack/slm/SnapshotLifecyclePolicyTests.java b/x-pack/plugin/slm/src/test/java/org/elasticsearch/xpack/slm/SnapshotLifecyclePolicyTests.java index 0ab3e99e1efc9..90397dbf6d0d7 100644 --- a/x-pack/plugin/slm/src/test/java/org/elasticsearch/xpack/slm/SnapshotLifecyclePolicyTests.java +++ b/x-pack/plugin/slm/src/test/java/org/elasticsearch/xpack/slm/SnapshotLifecyclePolicyTests.java @@ -13,6 +13,7 @@ import org.elasticsearch.common.io.stream.Writeable; import org.elasticsearch.core.TimeValue; import org.elasticsearch.test.AbstractXContentSerializingTestCase; +import org.elasticsearch.test.ESTestCase; import org.elasticsearch.xcontent.XContentParser; import org.elasticsearch.xpack.core.slm.SnapshotLifecyclePolicy; import org.elasticsearch.xpack.core.slm.SnapshotLifecyclePolicyMetadataTests; @@ -47,14 +48,15 @@ public void testToRequest() { schedule, "repo", Collections.emptyMap(), - SnapshotRetentionConfiguration.EMPTY + SnapshotRetentionConfiguration.EMPTY, + null ); CreateSnapshotRequest request = p.toRequest(TEST_REQUEST_TIMEOUT); CreateSnapshotRequest expected = new CreateSnapshotRequest(TEST_REQUEST_TIMEOUT).userMetadata( Collections.singletonMap("policy", "id") ); - p = new SnapshotLifecyclePolicy("id", "name", schedule, "repo", null, null); + p = new SnapshotLifecyclePolicy("id", "name", schedule, "repo", null, null, null); request = p.toRequest(TEST_REQUEST_TIMEOUT); expected.waitForCompletion(true).snapshot(request.snapshot()).repository("repo").uuid(request.uuid()); assertEquals(expected, request); @@ -67,7 +69,8 @@ public void testNextExecutionTimeSchedule() { "0 1 2 3 4 ? 2049", "repo", Collections.emptyMap(), - SnapshotRetentionConfiguration.EMPTY + SnapshotRetentionConfiguration.EMPTY, + null ); assertThat(p.calculateNextExecution(-1, Clock.systemUTC()), equalTo(2501028060000L)); } @@ -79,7 +82,8 @@ public void testNextExecutionTimeInterval() { "30m", "repo", Collections.emptyMap(), - SnapshotRetentionConfiguration.EMPTY + SnapshotRetentionConfiguration.EMPTY, + null ); { @@ -144,7 +148,8 @@ public void testCalculateNextIntervalInterval() { "30m", "repo", Collections.emptyMap(), - SnapshotRetentionConfiguration.EMPTY + SnapshotRetentionConfiguration.EMPTY, + null ); assertThat(p.calculateNextInterval(Clock.systemUTC()), equalTo(TimeValue.timeValueMinutes(30))); } @@ -156,7 +161,8 @@ public void testCalculateNextIntervalInterval() { schedule, "repo", Collections.emptyMap(), - SnapshotRetentionConfiguration.EMPTY + SnapshotRetentionConfiguration.EMPTY, + null ); assertThat(p.calculateNextInterval(Clock.systemUTC()), equalTo(TimeValue.parseTimeValue(schedule, "schedule"))); } @@ -170,7 +176,8 @@ public void testCalculateNextIntervalSchedule() { "0 0/5 * * * ?", "repo", Collections.emptyMap(), - SnapshotRetentionConfiguration.EMPTY + SnapshotRetentionConfiguration.EMPTY, + null ); assertThat(p.calculateNextInterval(Clock.systemUTC()), equalTo(TimeValue.timeValueMinutes(5))); } @@ -182,7 +189,8 @@ public void testCalculateNextIntervalSchedule() { "0 1 2 3 4 ? 2099", "repo", Collections.emptyMap(), - SnapshotRetentionConfiguration.EMPTY + SnapshotRetentionConfiguration.EMPTY, + null ); assertThat(p.calculateNextInterval(Clock.systemUTC()), equalTo(TimeValue.MINUS_ONE)); } @@ -194,7 +202,8 @@ public void testCalculateNextIntervalSchedule() { "* * * 31 FEB ? *", "repo", Collections.emptyMap(), - SnapshotRetentionConfiguration.EMPTY + SnapshotRetentionConfiguration.EMPTY, + null ); assertThat(p.calculateNextInterval(Clock.systemUTC()), equalTo(TimeValue.MINUS_ONE)); } @@ -208,7 +217,8 @@ public void testValidation() { "* * * * * L", " ", Collections.emptyMap(), - SnapshotRetentionConfiguration.EMPTY + SnapshotRetentionConfiguration.EMPTY, + null ); ValidationException e = policy.validate(); @@ -232,7 +242,8 @@ public void testValidation() { " ", "repo", Collections.emptyMap(), - SnapshotRetentionConfiguration.EMPTY + SnapshotRetentionConfiguration.EMPTY, + null ); ValidationException e = policy.validate(); @@ -253,7 +264,8 @@ public void testValidation() { "0d", "repo", Collections.emptyMap(), - SnapshotRetentionConfiguration.EMPTY + SnapshotRetentionConfiguration.EMPTY, + null ); ValidationException e = policy.validate(); @@ -267,7 +279,8 @@ public void testValidation() { "999micros", "repo", Collections.emptyMap(), - SnapshotRetentionConfiguration.EMPTY + SnapshotRetentionConfiguration.EMPTY, + null ); ValidationException e = policy.validate(); @@ -281,7 +294,8 @@ public void testValidation() { "0 0/30 * * * ?", "repo", Collections.emptyMap(), - SnapshotRetentionConfiguration.EMPTY + SnapshotRetentionConfiguration.EMPTY, + null ); ValidationException e = policy.validate(); assertThat(e, nullValue()); @@ -294,7 +308,8 @@ public void testValidation() { "30m", "repo", Collections.emptyMap(), - SnapshotRetentionConfiguration.EMPTY + SnapshotRetentionConfiguration.EMPTY, + TimeValue.parseTimeValue("1h", "unhealthyIfNoSnapshotWithin") ); ValidationException e = policy.validate(); assertThat(e, nullValue()); @@ -307,12 +322,68 @@ public void testValidation() { "1ms", "repo", Collections.emptyMap(), - SnapshotRetentionConfiguration.EMPTY + SnapshotRetentionConfiguration.EMPTY, + null ); ValidationException e = policy.validate(); assertThat(e, nullValue()); } + { + SnapshotLifecyclePolicy policy = new SnapshotLifecyclePolicy( + "my_policy", + "my_snap", + "15m", + "repo", + Collections.emptyMap(), + SnapshotRetentionConfiguration.EMPTY, + TimeValue.ONE_MINUTE + ); + + ValidationException e = policy.validate(); + assertThat( + e.validationErrors(), + containsInAnyOrder( + "invalid unhealthy_if_no_snapshot_within [1m]: time is too short, " + + "expecting at least more than the interval between snapshots [15m] for schedule [15m]" + ) + ); + } + { + SnapshotLifecyclePolicy policy = new SnapshotLifecyclePolicy( + "my_policy", + "my_snap", + "0 0 1 * * ?", // every day + "repo", + Collections.emptyMap(), + SnapshotRetentionConfiguration.EMPTY, + TimeValue.parseTimeValue("2d", "unhealthyIfNoSnapshotWithin") + ); + + ValidationException e = policy.validate(); + assertThat(e, nullValue()); + } + { + SnapshotLifecyclePolicy policy = new SnapshotLifecyclePolicy( + "my_policy", + "my_snap", + "0 0 1 * * ?", // every day + "repo", + Collections.emptyMap(), + SnapshotRetentionConfiguration.EMPTY, + TimeValue.ONE_MINUTE + ); + + ValidationException e = policy.validate(); + assertThat( + e.validationErrors(), + containsInAnyOrder( + "invalid unhealthy_if_no_snapshot_within [1m]: time is too short, " + + "expecting at least more than the interval between snapshots [1d] for schedule [0 0 1 * * ?]" + ) + ); + + } } public void testMetadataValidation() { @@ -327,7 +398,8 @@ public void testMetadataValidation() { "1 * * * * ?", "myrepo", configuration, - SnapshotRetentionConfiguration.EMPTY + SnapshotRetentionConfiguration.EMPTY, + null ); ValidationException e = policy.validate(); assertThat( @@ -348,7 +420,8 @@ public void testMetadataValidation() { "1 * * * * ?", "myrepo", configuration, - SnapshotRetentionConfiguration.EMPTY + SnapshotRetentionConfiguration.EMPTY, + null ); ValidationException e = policy.validate(); assertThat( @@ -378,7 +451,8 @@ public void testMetadataValidation() { "1 * * * * ?", "myrepo", configuration, - SnapshotRetentionConfiguration.EMPTY + SnapshotRetentionConfiguration.EMPTY, + null ); ValidationException e = policy.validate(); assertThat( @@ -401,68 +475,37 @@ protected SnapshotLifecyclePolicy createTestInstance() { @Override 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.getRetentionPolicy() - ); - case 1: - return new SnapshotLifecyclePolicy( - instance.getId(), - instance.getName() + randomAlphaOfLength(2), - instance.getSchedule(), - instance.getRepository(), - instance.getConfig(), - instance.getRetentionPolicy() - ); - case 2: - return new SnapshotLifecyclePolicy( - instance.getId(), - instance.getName(), - randomValueOtherThan(instance.getSchedule(), SnapshotLifecyclePolicyMetadataTests::randomSchedule), - instance.getRepository(), - instance.getConfig(), - instance.getRetentionPolicy() - ); - case 3: - return new SnapshotLifecyclePolicy( - instance.getId(), - instance.getName(), - instance.getSchedule(), - instance.getRepository() + randomAlphaOfLength(2), - instance.getConfig(), - instance.getRetentionPolicy() - ); - case 4: + String id = instance.getId(); + String name = instance.getName(); + String schedule = instance.getSchedule(); + String repository = instance.getRepository(); + Map config = instance.getConfig(); + SnapshotRetentionConfiguration retentionPolicy = instance.getRetentionPolicy(); + TimeValue unhealthyIfNoSnapshotWithin = instance.getUnhealthyIfNoSnapshotWithin(); + + switch (between(0, 6)) { + case 0 -> id += randomAlphaOfLength(2); + case 1 -> name += randomAlphaOfLength(2); + case 2 -> schedule = randomValueOtherThan(instance.getSchedule(), SnapshotLifecyclePolicyMetadataTests::randomSchedule); + case 3 -> repository += randomAlphaOfLength(2); + case 4 -> { Map newConfig = new HashMap<>(); for (int i = 0; i < randomIntBetween(2, 5); i++) { newConfig.put(randomAlphaOfLength(3), randomAlphaOfLength(3)); } - return new SnapshotLifecyclePolicy( - instance.getId(), - instance.getName() + randomAlphaOfLength(2), - instance.getSchedule(), - instance.getRepository(), - 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"); + config = newConfig; + } + case 5 -> retentionPolicy = randomValueOtherThan( + instance.getRetentionPolicy(), + SnapshotLifecyclePolicyMetadataTests::randomRetention + ); + case 6 -> unhealthyIfNoSnapshotWithin = randomValueOtherThan( + instance.getUnhealthyIfNoSnapshotWithin(), + ESTestCase::randomTimeValue + ); + default -> throw new AssertionError("failure, got illegal switch case"); } + return new SnapshotLifecyclePolicy(id, name, schedule, repository, config, retentionPolicy, unhealthyIfNoSnapshotWithin); } @Override