Adds concept of a safe action (#31971)

* Adds concept of a safe action

A safe action is one that does not have unwanted side effects if the
configuration of the action is change in the policy while and index is
executing the action.

This commit formalises this concept with the only current unsafe action
being ShrinkAction. It also adds testing around this and add a method
to LifecyclePolicy which returns whether the action for the provided
StepKey is safe.

* Makes IndexLifecycleRunners checks use the safe indications instead of
hardcoding shrink action

* Fixes test failure
This commit is contained in:
Colin Goodheart-Smithe 2018-07-12 11:40:38 +01:00 committed by GitHub
parent 4192f387b4
commit 89e8f9e240
No known key found for this signature in database
GPG Key ID: 4AEE18F83AFDEB23
22 changed files with 338 additions and 96 deletions

View File

@ -112,6 +112,10 @@ public class AllocateAction implements LifecycleAction {
return builder;
}
@Override
public boolean isSafeAction() {
return true;
}
@Override
public List<Step> toSteps(Client client, String phase, StepKey nextStepKey) {

View File

@ -51,6 +51,11 @@ public class DeleteAction implements LifecycleAction {
return builder;
}
@Override
public boolean isSafeAction() {
return true;
}
@Override
public List<Step> toSteps(Client client, String phase, Step.StepKey nextStepKey) {
Step.StepKey deleteStepKey = new Step.StepKey(phase, NAME, DeleteStep.NAME);

View File

@ -83,6 +83,11 @@ public class ForceMergeAction implements LifecycleAction {
return NAME;
}
@Override
public boolean isSafeAction() {
return true;
}
@Override
public XContentBuilder toXContent(XContentBuilder builder, Params params) throws IOException {
builder.startObject();

View File

@ -28,4 +28,11 @@ public interface LifecycleAction extends ToXContentObject, NamedWriteable {
* @return an ordered list of steps that represent the execution plan of the action
*/
List<Step> toSteps(Client client, String phase, @Nullable Step.StepKey nextStepKey);
/**
* @return true if this action is considered safe. An action is not safe if
* it will produce unwanted side effects or will get stuck when the
* action configuration is changed while an index is in this action
*/
boolean isSafeAction();
}

View File

@ -19,6 +19,7 @@ import org.elasticsearch.common.xcontent.ObjectParser.ValueType;
import org.elasticsearch.common.xcontent.ToXContentObject;
import org.elasticsearch.common.xcontent.XContentBuilder;
import org.elasticsearch.common.xcontent.XContentParser;
import org.elasticsearch.xpack.core.indexlifecycle.Step.StepKey;
import java.io.IOException;
import java.util.ArrayList;
@ -226,6 +227,24 @@ public class LifecyclePolicy extends AbstractDiffable<LifecyclePolicy>
return steps;
}
public boolean isActionSafe(StepKey stepKey) {
if ("new".equals(stepKey.getPhase())) {
return true;
}
Phase phase = phases.get(stepKey.getPhase());
if (phase != null) {
LifecycleAction action = phase.getActions().get(stepKey.getAction());
if (action != null) {
return action.isSafeAction();
} else {
throw new IllegalArgumentException("Action [" + stepKey.getAction() + "] in phase [" + stepKey.getPhase()
+ "] does not exist in policy [" + name + "]");
}
} else {
throw new IllegalArgumentException("Phase [" + stepKey.getPhase() + "] does not exist in policy [" + name + "]");
}
}
@Override
public int hashCode() {
return Objects.hash(name, phases);

View File

@ -54,6 +54,11 @@ public class ReadOnlyAction implements LifecycleAction {
public void writeTo(StreamOutput out) throws IOException {
}
@Override
public boolean isSafeAction() {
return true;
}
@Override
public List<Step> toSteps(Client client, String phase, Step.StepKey nextStepKey) {
Step.StepKey key = new Step.StepKey(phase, NAME, NAME);

View File

@ -73,6 +73,11 @@ public class ReplicasAction implements LifecycleAction {
return builder;
}
@Override
public boolean isSafeAction() {
return true;
}
@Override
public List<Step> toSteps(Client client, String phase, Step.StepKey nextStepKey) {
StepKey updateReplicasKey = new StepKey(phase, NAME, UpdateSettingsStep.NAME);

View File

@ -125,6 +125,11 @@ public class RolloverAction implements LifecycleAction {
return builder;
}
@Override
public boolean isSafeAction() {
return true;
}
@Override
public List<Step> toSteps(Client client, String phase, Step.StepKey nextStepKey) {
StepKey updateDateStepKey = new StepKey(phase, NAME, UpdateRolloverLifecycleDateStep.NAME);

View File

@ -75,6 +75,11 @@ public class ShrinkAction implements LifecycleAction {
return builder;
}
@Override
public boolean isSafeAction() {
return false;
}
@Override
public List<Step> toSteps(Client client, String phase, Step.StepKey nextStepKey) {
StepKey setSingleNodeKey = new StepKey(phase, NAME, SetSingleNodeAllocateStep.NAME);

View File

@ -0,0 +1,23 @@
/*
* 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.indexlifecycle;
import org.elasticsearch.test.AbstractSerializingTestCase;
public abstract class AbstractActionTestCase<T extends LifecycleAction> extends AbstractSerializingTestCase<T> {
public abstract void testToSteps();
protected boolean isSafeAction() {
return true;
}
public final void testIsSafeAction() {
LifecycleAction action = createTestInstance();
assertEquals(isSafeAction(), action.isSafeAction());
}
}

View File

@ -9,7 +9,6 @@ import org.elasticsearch.cluster.metadata.IndexMetaData;
import org.elasticsearch.common.io.stream.Writeable.Reader;
import org.elasticsearch.common.settings.Settings;
import org.elasticsearch.common.xcontent.XContentParser;
import org.elasticsearch.test.AbstractSerializingTestCase;
import org.elasticsearch.xpack.core.indexlifecycle.Step.StepKey;
import java.util.Collections;
@ -19,7 +18,7 @@ import java.util.Map;
import static org.hamcrest.Matchers.equalTo;
public class AllocateActionTests extends AbstractSerializingTestCase<AllocateAction> {
public class AllocateActionTests extends AbstractActionTestCase<AllocateAction> {
@Override
protected AllocateAction doParseInstance(XContentParser parser) {

View File

@ -7,13 +7,12 @@ package org.elasticsearch.xpack.core.indexlifecycle;
import org.elasticsearch.common.io.stream.Writeable.Reader;
import org.elasticsearch.common.xcontent.XContentParser;
import org.elasticsearch.test.AbstractSerializingTestCase;
import org.elasticsearch.xpack.core.indexlifecycle.Step.StepKey;
import java.io.IOException;
import java.util.List;
public class DeleteActionTests extends AbstractSerializingTestCase<DeleteAction> {
public class DeleteActionTests extends AbstractActionTestCase<DeleteAction> {
@Override
protected DeleteAction doParseInstance(XContentParser parser) throws IOException {

View File

@ -13,7 +13,6 @@ import org.elasticsearch.common.xcontent.XContentHelper;
import org.elasticsearch.common.xcontent.XContentParser;
import org.elasticsearch.common.xcontent.XContentType;
import org.elasticsearch.common.xcontent.json.JsonXContent;
import org.elasticsearch.test.AbstractSerializingTestCase;
import org.elasticsearch.xpack.core.indexlifecycle.Step.StepKey;
import java.io.IOException;
@ -21,7 +20,7 @@ import java.util.List;
import static org.hamcrest.Matchers.equalTo;
public class ForceMergeActionTests extends AbstractSerializingTestCase<ForceMergeAction> {
public class ForceMergeActionTests extends AbstractActionTestCase<ForceMergeAction> {
@Override
protected ForceMergeAction doParseInstance(XContentParser parser) {

View File

@ -186,4 +186,30 @@ public class LifecyclePolicyTests extends AbstractSerializingTestCase<LifecycleP
assertThat(steps.get(5), equalTo(secondActionStep));
assertSame(steps.get(6), TerminalPolicyStep.INSTANCE);
}
public void testIsActionSafe() {
Map<String, Phase> phases = new LinkedHashMap<>();
LifecycleAction firstAction = new MockAction(Collections.emptyList(), true);
LifecycleAction secondAction = new MockAction(Collections.emptyList(), false);
Map<String, LifecycleAction> firstActions = Collections.singletonMap(MockAction.NAME, firstAction);
Map<String, LifecycleAction> secondActions = Collections.singletonMap(MockAction.NAME, secondAction);
Phase firstPhase = new Phase("first_phase", TimeValue.ZERO, firstActions);
Phase secondPhase = new Phase("second_phase", TimeValue.ZERO, secondActions);
phases.put(firstPhase.getName(), firstPhase);
phases.put(secondPhase.getName(), secondPhase);
LifecyclePolicy policy = new LifecyclePolicy(TestLifecycleType.INSTANCE, lifecycleName, phases);
assertTrue(policy.isActionSafe(new StepKey("first_phase", MockAction.NAME, randomAlphaOfLength(10))));
assertFalse(policy.isActionSafe(new StepKey("second_phase", MockAction.NAME, randomAlphaOfLength(10))));
IllegalArgumentException exception = expectThrows(IllegalArgumentException.class,
() -> policy.isActionSafe(new StepKey("non_existant_phase", MockAction.NAME, randomAlphaOfLength(10))));
assertEquals("Phase [non_existant_phase] does not exist in policy [" + policy.getName() + "]", exception.getMessage());
exception = expectThrows(IllegalArgumentException.class,
() -> policy.isActionSafe(new StepKey("first_phase", "non_existant_action", randomAlphaOfLength(10))));
assertEquals("Action [non_existant_action] in phase [first_phase] does not exist in policy [" + policy.getName() + "]",
exception.getMessage());
}
}

View File

@ -24,21 +24,28 @@ public class MockAction implements LifecycleAction {
private List<Step> steps;
private static final ObjectParser<MockAction, Void> PARSER = new ObjectParser<>(NAME, MockAction::new);
private final boolean safe;
public static MockAction parse(XContentParser parser) {
return PARSER.apply(parser, null);
}
public MockAction() {
this.steps = Collections.emptyList();
this(Collections.emptyList());
}
public MockAction(List<Step> steps) {
this(steps, true);
}
public MockAction(List<Step> steps, boolean safe) {
this.steps = steps;
this.safe = safe;
}
public MockAction(StreamInput in) throws IOException {
this.steps = in.readList(MockStep::new);
this.safe = in.readBoolean();
}
@Override
@ -57,6 +64,11 @@ public class MockAction implements LifecycleAction {
return steps;
}
@Override
public boolean isSafeAction() {
return safe;
}
@Override
public List<Step> toSteps(Client client, String phase, Step.StepKey nextStepKey) {
return new ArrayList<>(steps);
@ -65,11 +77,12 @@ public class MockAction implements LifecycleAction {
@Override
public void writeTo(StreamOutput out) throws IOException {
out.writeList(steps.stream().map(MockStep::new).collect(Collectors.toList()));
out.writeBoolean(safe);
}
@Override
public int hashCode() {
return Objects.hash(steps);
return Objects.hash(steps, safe);
}
@Override
@ -81,6 +94,7 @@ public class MockAction implements LifecycleAction {
return false;
}
MockAction other = (MockAction) obj;
return Objects.equals(steps, other.steps);
return Objects.equals(steps, other.steps) &&
Objects.equals(safe, other.safe);
}
}

View File

@ -7,13 +7,12 @@ package org.elasticsearch.xpack.core.indexlifecycle;
import org.elasticsearch.common.io.stream.Writeable.Reader;
import org.elasticsearch.common.xcontent.XContentParser;
import org.elasticsearch.test.AbstractSerializingTestCase;
import java.io.IOException;
import java.util.ArrayList;
import java.util.List;
public class MockActionTests extends AbstractSerializingTestCase<MockAction> {
public class MockActionTests extends AbstractActionTestCase<MockAction> {
@Override
protected MockAction createTestInstance() {
@ -32,7 +31,10 @@ public class MockActionTests extends AbstractSerializingTestCase<MockAction> {
@Override
protected MockAction mutateInstance(MockAction instance) throws IOException {
List<Step> steps = new ArrayList<>(instance.getSteps());
List<Step> steps = instance.getSteps();
boolean safe = instance.isSafeAction();
if (randomBoolean()) {
steps = new ArrayList<>(steps);
if (steps.size() > 0) {
Step lastStep = steps.remove(steps.size() - 1);
if (randomBoolean()) {
@ -43,12 +45,26 @@ public class MockActionTests extends AbstractSerializingTestCase<MockAction> {
} else {
steps.add(new MockStep(randomStepKey(), null));
}
return new MockAction(steps);
} else {
safe = safe == false;
}
return new MockAction(steps, safe);
}
private static Step.StepKey randomStepKey() {
return new Step.StepKey(randomAlphaOfLength(5),
randomAlphaOfLength(5), randomAlphaOfLength(5));
}
@Override
public void testToSteps() {
int numSteps = randomIntBetween(1, 10);
List<Step> steps = new ArrayList<>(numSteps);
for (int i = 0; i < numSteps; i++) {
steps.add(new MockStep(randomStepKey(), randomStepKey()));
}
MockAction action = new MockAction(steps);
assertEquals(action.getSteps(), action.toSteps(null, null, null));
}
}

View File

@ -8,14 +8,13 @@ package org.elasticsearch.xpack.core.indexlifecycle;
import org.elasticsearch.cluster.metadata.IndexMetaData;
import org.elasticsearch.common.io.stream.Writeable.Reader;
import org.elasticsearch.common.xcontent.XContentParser;
import org.elasticsearch.test.AbstractSerializingTestCase;
import org.elasticsearch.xpack.core.indexlifecycle.Step.StepKey;
import java.util.List;
import static org.hamcrest.Matchers.equalTo;
public class ReadOnlyActionTests extends AbstractSerializingTestCase<ReadOnlyAction> {
public class ReadOnlyActionTests extends AbstractActionTestCase<ReadOnlyAction> {
@Override
protected ReadOnlyAction doParseInstance(XContentParser parser) {

View File

@ -8,13 +8,12 @@ package org.elasticsearch.xpack.core.indexlifecycle;
import org.elasticsearch.cluster.metadata.IndexMetaData;
import org.elasticsearch.common.io.stream.Writeable.Reader;
import org.elasticsearch.common.xcontent.XContentParser;
import org.elasticsearch.test.AbstractSerializingTestCase;
import org.elasticsearch.xpack.core.indexlifecycle.Step.StepKey;
import java.io.IOException;
import java.util.List;
public class ReplicasActionTests extends AbstractSerializingTestCase<ReplicasAction> {
public class ReplicasActionTests extends AbstractActionTestCase<ReplicasAction> {
@Override
protected ReplicasAction doParseInstance(XContentParser parser) {

View File

@ -10,13 +10,12 @@ import org.elasticsearch.common.unit.ByteSizeUnit;
import org.elasticsearch.common.unit.ByteSizeValue;
import org.elasticsearch.common.unit.TimeValue;
import org.elasticsearch.common.xcontent.XContentParser;
import org.elasticsearch.test.AbstractSerializingTestCase;
import org.elasticsearch.xpack.core.indexlifecycle.Step.StepKey;
import java.io.IOException;
import java.util.List;
public class RolloverActionTests extends AbstractSerializingTestCase<RolloverAction> {
public class RolloverActionTests extends AbstractActionTestCase<RolloverAction> {
@Override
protected RolloverAction doParseInstance(XContentParser parser) throws IOException {

View File

@ -7,7 +7,6 @@ package org.elasticsearch.xpack.core.indexlifecycle;
import org.elasticsearch.common.io.stream.Writeable.Reader;
import org.elasticsearch.common.xcontent.XContentParser;
import org.elasticsearch.test.AbstractSerializingTestCase;
import org.elasticsearch.xpack.core.indexlifecycle.Step.StepKey;
import java.io.IOException;
@ -15,7 +14,7 @@ import java.util.List;
import static org.hamcrest.Matchers.equalTo;
public class ShrinkActionTests extends AbstractSerializingTestCase<ShrinkAction> {
public class ShrinkActionTests extends AbstractActionTestCase<ShrinkAction> {
@Override
protected ShrinkAction doParseInstance(XContentParser parser) throws IOException {
@ -80,4 +79,9 @@ public class ShrinkActionTests extends AbstractSerializingTestCase<ShrinkAction>
assertThat(steps.get(5).getNextStepKey(), equalTo(nextStepKey));
assertThat(((ShrunkenIndexCheckStep) steps.get(5)).getShrunkIndexPrefix(), equalTo(ShrinkAction.SHRUNKEN_INDEX_PREFIX));
}
@Override
protected boolean isSafeAction() {
return false;
}
}

View File

@ -9,7 +9,6 @@ import com.carrotsearch.hppc.cursors.ObjectCursor;
import org.apache.logging.log4j.Logger;
import org.elasticsearch.ElasticsearchException;
import org.elasticsearch.action.admin.indices.shrink.ShrinkAction;
import org.elasticsearch.cluster.ClusterState;
import org.elasticsearch.cluster.metadata.IndexMetaData;
import org.elasticsearch.cluster.metadata.MetaData;
@ -28,6 +27,7 @@ import org.elasticsearch.xpack.core.indexlifecycle.AsyncWaitStep;
import org.elasticsearch.xpack.core.indexlifecycle.ClusterStateActionStep;
import org.elasticsearch.xpack.core.indexlifecycle.ClusterStateWaitStep;
import org.elasticsearch.xpack.core.indexlifecycle.ErrorStep;
import org.elasticsearch.xpack.core.indexlifecycle.IndexLifecycleMetadata;
import org.elasticsearch.xpack.core.indexlifecycle.LifecyclePolicy;
import org.elasticsearch.xpack.core.indexlifecycle.LifecycleSettings;
import org.elasticsearch.xpack.core.indexlifecycle.RolloverAction;
@ -37,6 +37,7 @@ import org.elasticsearch.xpack.core.indexlifecycle.TerminalPolicyStep;
import java.io.IOException;
import java.util.List;
import java.util.Map;
import java.util.function.LongSupplier;
public class IndexLifecycleRunner {
@ -286,6 +287,8 @@ public class IndexLifecycleRunner {
public static ClusterState setPolicyForIndexes(final String newPolicyName, final Index[] indices, ClusterState currentState,
LifecyclePolicy newPolicy, List<String> failedIndexes) {
Map<String, LifecyclePolicy> policiesMap = ((IndexLifecycleMetadata) currentState.metaData().custom(IndexLifecycleMetadata.TYPE))
.getPolicies();
MetaData.Builder newMetadata = MetaData.builder(currentState.getMetaData());
boolean clusterStateChanged = false;
for (Index index : indices) {
@ -294,8 +297,8 @@ public class IndexLifecycleRunner {
// Index doesn't exist so fail it
failedIndexes.add(index.getName());
} else {
IndexMetaData.Builder newIdxMetadata = IndexLifecycleRunner.setPolicyForIndex(newPolicyName, newPolicy, failedIndexes,
index, indexMetadata);
IndexMetaData.Builder newIdxMetadata = IndexLifecycleRunner.setPolicyForIndex(newPolicyName, newPolicy, policiesMap,
failedIndexes, index, indexMetadata);
if (newIdxMetadata != null) {
newMetadata.put(newIdxMetadata);
clusterStateChanged = true;
@ -312,12 +315,15 @@ public class IndexLifecycleRunner {
}
private static IndexMetaData.Builder setPolicyForIndex(final String newPolicyName, LifecyclePolicy newPolicy,
List<String> failedIndexes,
Index index, IndexMetaData indexMetadata) {
Map<String, LifecyclePolicy> policiesMap, List<String> failedIndexes, Index index, IndexMetaData indexMetadata) {
Settings idxSettings = indexMetadata.getSettings();
Settings.Builder newSettings = Settings.builder().put(idxSettings);
String currentPolicy = LifecycleSettings.LIFECYCLE_NAME_SETTING.get(idxSettings);
String currentPolicyName = LifecycleSettings.LIFECYCLE_NAME_SETTING.get(idxSettings);
StepKey currentStepKey = IndexLifecycleRunner.getCurrentStepKey(idxSettings);
LifecyclePolicy currentPolicy = null;
if (Strings.hasLength(currentPolicyName)) {
currentPolicy = policiesMap.get(currentPolicyName);
}
if (canSetPolicy(currentStepKey, currentPolicy, newPolicy)) {
newSettings.put(LifecycleSettings.LIFECYCLE_NAME_SETTING.getKey(), newPolicyName);
@ -329,14 +335,14 @@ public class IndexLifecycleRunner {
}
}
private static boolean canSetPolicy(StepKey currentStepKey, String currentPolicyName, LifecyclePolicy newPolicy) {
if (Strings.hasLength(currentPolicyName)) {
if (ShrinkAction.NAME.equals(currentStepKey.getAction())) {
private static boolean canSetPolicy(StepKey currentStepKey, LifecyclePolicy currentPolicy, LifecyclePolicy newPolicy) {
if (currentPolicy != null) {
if (currentPolicy.isActionSafe(currentStepKey)) {
return true;
} else {
// Index is in the shrink action so fail it
// NORELEASE also need to check if the shrink action has changed between oldPolicy and newPolicy
return false;
} else {
return true;
}
} else {
// Index not previously managed by ILM so safe to change policy
@ -360,13 +366,19 @@ public class IndexLifecycleRunner {
* the current {@link ClusterState}
*/
public static boolean canUpdatePolicy(String policyName, LifecyclePolicy newPolicy, ClusterState currentState) {
Map<String, LifecyclePolicy> policiesMap = ((IndexLifecycleMetadata) currentState.metaData().custom(IndexLifecycleMetadata.TYPE))
.getPolicies();
for (ObjectCursor<IndexMetaData> cursor : currentState.getMetaData().indices().values()) {
IndexMetaData idxMetadata = cursor.value;
Settings idxSettings = idxMetadata.getSettings();
String currentPolicyName = LifecycleSettings.LIFECYCLE_NAME_SETTING.get(idxSettings);
LifecyclePolicy currentPolicy = null;
if (Strings.hasLength(currentPolicyName)) {
currentPolicy = policiesMap.get(currentPolicyName);
}
if (policyName.equals(currentPolicyName)) {
StepKey currentStepKey = IndexLifecycleRunner.getCurrentStepKey(idxSettings);
if (canSetPolicy(currentStepKey, policyName, newPolicy) == false) {
if (canSetPolicy(currentStepKey, currentPolicy, newPolicy) == false) {
return false;
}
}
@ -375,6 +387,8 @@ public class IndexLifecycleRunner {
}
public static ClusterState removePolicyForIndexes(final Index[] indices, ClusterState currentState, List<String> failedIndexes) {
Map<String, LifecyclePolicy> policiesMap = ((IndexLifecycleMetadata) currentState.metaData().custom(IndexLifecycleMetadata.TYPE))
.getPolicies();
MetaData.Builder newMetadata = MetaData.builder(currentState.getMetaData());
boolean clusterStateChanged = false;
for (Index index : indices) {
@ -383,7 +397,8 @@ public class IndexLifecycleRunner {
// Index doesn't exist so fail it
failedIndexes.add(index.getName());
} else {
IndexMetaData.Builder newIdxMetadata = IndexLifecycleRunner.removePolicyForIndex(index, indexMetadata, failedIndexes);
IndexMetaData.Builder newIdxMetadata = IndexLifecycleRunner.removePolicyForIndex(index, indexMetadata, policiesMap,
failedIndexes);
if (newIdxMetadata != null) {
newMetadata.put(newIdxMetadata);
clusterStateChanged = true;
@ -399,11 +414,16 @@ public class IndexLifecycleRunner {
}
}
private static IndexMetaData.Builder removePolicyForIndex(Index index, IndexMetaData indexMetadata, List<String> failedIndexes) {
private static IndexMetaData.Builder removePolicyForIndex(Index index, IndexMetaData indexMetadata,
Map<String, LifecyclePolicy> policiesMap, List<String> failedIndexes) {
Settings idxSettings = indexMetadata.getSettings();
Settings.Builder newSettings = Settings.builder().put(idxSettings);
String currentPolicy = LifecycleSettings.LIFECYCLE_NAME_SETTING.get(idxSettings);
String currentPolicyName = LifecycleSettings.LIFECYCLE_NAME_SETTING.get(idxSettings);
StepKey currentStepKey = IndexLifecycleRunner.getCurrentStepKey(idxSettings);
LifecyclePolicy currentPolicy = null;
if (Strings.hasLength(currentPolicyName)) {
currentPolicy = policiesMap.get(currentPolicyName);
}
if (canRemovePolicy(currentStepKey, currentPolicy)) {
newSettings.remove(LifecycleSettings.LIFECYCLE_NAME_SETTING.getKey());
@ -425,11 +445,10 @@ public class IndexLifecycleRunner {
}
}
private static boolean canRemovePolicy(StepKey currentStepKey, String currentPolicyName) {
if (Strings.hasLength(currentPolicyName)) {
// Can't remove policy if the index is currently in the Shrink
// action
return ShrinkAction.NAME.equals(currentStepKey.getAction()) == false;
private static boolean canRemovePolicy(StepKey currentStepKey, LifecyclePolicy currentPolicy) {
if (currentPolicy != null) {
// Can't remove policy if the index is currently in an unsafe action
return currentPolicy.isActionSafe(currentStepKey);
} else {
// Index not previously managed by ILM
return true;

View File

@ -16,6 +16,7 @@ import org.elasticsearch.cluster.service.ClusterService;
import org.elasticsearch.common.bytes.BytesReference;
import org.elasticsearch.common.settings.Settings;
import org.elasticsearch.common.settings.Settings.Builder;
import org.elasticsearch.common.unit.TimeValue;
import org.elasticsearch.common.xcontent.ToXContent;
import org.elasticsearch.common.xcontent.ToXContentObject;
import org.elasticsearch.common.xcontent.XContentBuilder;
@ -28,10 +29,15 @@ import org.elasticsearch.xpack.core.indexlifecycle.AsyncWaitStep;
import org.elasticsearch.xpack.core.indexlifecycle.ClusterStateActionStep;
import org.elasticsearch.xpack.core.indexlifecycle.ClusterStateWaitStep;
import org.elasticsearch.xpack.core.indexlifecycle.ErrorStep;
import org.elasticsearch.xpack.core.indexlifecycle.IndexLifecycleMetadata;
import org.elasticsearch.xpack.core.indexlifecycle.LifecycleAction;
import org.elasticsearch.xpack.core.indexlifecycle.LifecyclePolicy;
import org.elasticsearch.xpack.core.indexlifecycle.LifecyclePolicyMetadata;
import org.elasticsearch.xpack.core.indexlifecycle.LifecycleSettings;
import org.elasticsearch.xpack.core.indexlifecycle.MockAction;
import org.elasticsearch.xpack.core.indexlifecycle.MockStep;
import org.elasticsearch.xpack.core.indexlifecycle.OperationMode;
import org.elasticsearch.xpack.core.indexlifecycle.Phase;
import org.elasticsearch.xpack.core.indexlifecycle.RandomStepInfo;
import org.elasticsearch.xpack.core.indexlifecycle.RolloverAction;
import org.elasticsearch.xpack.core.indexlifecycle.Step;
@ -49,6 +55,8 @@ import java.util.List;
import java.util.Map;
import java.util.Objects;
import java.util.SortedMap;
import java.util.function.Function;
import java.util.stream.Collectors;
import static org.hamcrest.Matchers.equalTo;
import static org.mockito.Mockito.mock;
@ -531,7 +539,7 @@ public class IndexLifecycleRunnerTests extends ESTestCase {
StepKey nextStep = new StepKey("next_phase", "next_action", "next_step");
long now = randomNonNegativeLong();
ClusterState clusterState = buildClusterState(indexName, Settings.builder());
ClusterState clusterState = buildClusterState(indexName, Settings.builder(), Collections.emptyList());
Index index = clusterState.metaData().index(indexName).getIndex();
ClusterState newClusterState = IndexLifecycleRunner.moveClusterStateToNextStep(index, clusterState, currentStep, nextStep,
() -> now);
@ -544,7 +552,7 @@ public class IndexLifecycleRunnerTests extends ESTestCase {
indexSettingsBuilder.put(LifecycleSettings.LIFECYCLE_STEP_INFO, randomAlphaOfLength(20));
}
clusterState = buildClusterState(indexName,
indexSettingsBuilder);
indexSettingsBuilder, Collections.emptyList());
index = clusterState.metaData().index(indexName).getIndex();
newClusterState = IndexLifecycleRunner.moveClusterStateToNextStep(index, clusterState, currentStep, nextStep, () -> now);
assertClusterStateOnNextStep(clusterState, index, currentStep, nextStep, newClusterState, now);
@ -556,7 +564,7 @@ public class IndexLifecycleRunnerTests extends ESTestCase {
StepKey nextStep = new StepKey("current_phase", "next_action", "next_step");
long now = randomNonNegativeLong();
ClusterState clusterState = buildClusterState(indexName, Settings.builder());
ClusterState clusterState = buildClusterState(indexName, Settings.builder(), Collections.emptyList());
Index index = clusterState.metaData().index(indexName).getIndex();
ClusterState newClusterState = IndexLifecycleRunner.moveClusterStateToNextStep(index, clusterState, currentStep, nextStep,
() -> now);
@ -569,7 +577,7 @@ public class IndexLifecycleRunnerTests extends ESTestCase {
indexSettingsBuilder.put(LifecycleSettings.LIFECYCLE_STEP_INFO, randomAlphaOfLength(20));
}
clusterState = buildClusterState(indexName,
indexSettingsBuilder);
indexSettingsBuilder, Collections.emptyList());
index = clusterState.metaData().index(indexName).getIndex();
newClusterState = IndexLifecycleRunner.moveClusterStateToNextStep(index, clusterState, currentStep, nextStep, () -> now);
assertClusterStateOnNextStep(clusterState, index, currentStep, nextStep, newClusterState, now);
@ -581,7 +589,7 @@ public class IndexLifecycleRunnerTests extends ESTestCase {
StepKey nextStep = new StepKey("current_phase", "current_action", "next_step");
long now = randomNonNegativeLong();
ClusterState clusterState = buildClusterState(indexName, Settings.builder());
ClusterState clusterState = buildClusterState(indexName, Settings.builder(), Collections.emptyList());
Index index = clusterState.metaData().index(indexName).getIndex();
ClusterState newClusterState = IndexLifecycleRunner.moveClusterStateToNextStep(index, clusterState, currentStep, nextStep,
() -> now);
@ -594,7 +602,7 @@ public class IndexLifecycleRunnerTests extends ESTestCase {
indexSettingsBuilder.put(LifecycleSettings.LIFECYCLE_STEP_INFO, randomAlphaOfLength(20));
}
clusterState = buildClusterState(indexName,
indexSettingsBuilder);
indexSettingsBuilder, Collections.emptyList());
index = clusterState.metaData().index(indexName).getIndex();
newClusterState = IndexLifecycleRunner.moveClusterStateToNextStep(index, clusterState, currentStep, nextStep, () -> now);
assertClusterStateOnNextStep(clusterState, index, currentStep, nextStep, newClusterState, now);
@ -613,7 +621,7 @@ public class IndexLifecycleRunnerTests extends ESTestCase {
.put(LifecycleSettings.LIFECYCLE_PHASE, currentStepKey.getPhase())
.put(LifecycleSettings.LIFECYCLE_ACTION, currentStepKey.getAction())
.put(LifecycleSettings.LIFECYCLE_STEP, currentStepKey.getName());
ClusterState clusterState = buildClusterState(indexName, indexSettingsBuilder);
ClusterState clusterState = buildClusterState(indexName, indexSettingsBuilder, Collections.emptyList());
Index index = clusterState.metaData().index(indexName).getIndex();
ClusterState newClusterState = IndexLifecycleRunner.moveClusterStateToStep(indexName, clusterState, currentStepKey,
nextStepKey, () -> now, stepRegistry);
@ -633,7 +641,7 @@ public class IndexLifecycleRunnerTests extends ESTestCase {
.put(LifecycleSettings.LIFECYCLE_PHASE, currentStepKey.getPhase())
.put(LifecycleSettings.LIFECYCLE_ACTION, currentStepKey.getAction())
.put(LifecycleSettings.LIFECYCLE_STEP, currentStepKey.getName());
ClusterState clusterState = buildClusterState(indexName, indexSettingsBuilder);
ClusterState clusterState = buildClusterState(indexName, indexSettingsBuilder, Collections.emptyList());
IllegalArgumentException exception = expectThrows(IllegalArgumentException.class,
() -> IndexLifecycleRunner.moveClusterStateToStep(indexName, clusterState, currentStepKey,
nextStepKey, () -> now, stepRegistry));
@ -654,7 +662,7 @@ public class IndexLifecycleRunnerTests extends ESTestCase {
.put(LifecycleSettings.LIFECYCLE_PHASE, currentStepKey.getPhase())
.put(LifecycleSettings.LIFECYCLE_ACTION, currentStepKey.getAction())
.put(LifecycleSettings.LIFECYCLE_STEP, currentStepKey.getName());
ClusterState clusterState = buildClusterState(indexName, indexSettingsBuilder);
ClusterState clusterState = buildClusterState(indexName, indexSettingsBuilder, Collections.emptyList());
IllegalArgumentException exception = expectThrows(IllegalArgumentException.class,
() -> IndexLifecycleRunner.moveClusterStateToStep(indexName, clusterState, notCurrentStepKey,
nextStepKey, () -> now, stepRegistry));
@ -675,7 +683,7 @@ public class IndexLifecycleRunnerTests extends ESTestCase {
.put(LifecycleSettings.LIFECYCLE_PHASE, currentStepKey.getPhase())
.put(LifecycleSettings.LIFECYCLE_ACTION, currentStepKey.getAction())
.put(LifecycleSettings.LIFECYCLE_STEP, currentStepKey.getName());
ClusterState clusterState = buildClusterState(indexName, indexSettingsBuilder);
ClusterState clusterState = buildClusterState(indexName, indexSettingsBuilder, Collections.emptyList());
IllegalArgumentException exception = expectThrows(IllegalArgumentException.class,
() -> IndexLifecycleRunner.moveClusterStateToStep(indexName, clusterState, currentStepKey,
nextStepKey, () -> now, stepRegistry));
@ -692,7 +700,8 @@ public class IndexLifecycleRunnerTests extends ESTestCase {
ClusterState clusterState = buildClusterState(indexName,
Settings.builder().put(LifecycleSettings.LIFECYCLE_PHASE, currentStep.getPhase())
.put(LifecycleSettings.LIFECYCLE_ACTION, currentStep.getAction())
.put(LifecycleSettings.LIFECYCLE_STEP, currentStep.getName()));
.put(LifecycleSettings.LIFECYCLE_STEP, currentStep.getName()),
Collections.emptyList());
Index index = clusterState.metaData().index(indexName).getIndex();
ClusterState newClusterState = IndexLifecycleRunner.moveClusterStateToErrorStep(index, clusterState, currentStep, cause, () -> now);
@ -720,7 +729,7 @@ public class IndexLifecycleRunnerTests extends ESTestCase {
.put(LifecycleSettings.LIFECYCLE_ACTION, errorStepKey.getAction())
.put(LifecycleSettings.LIFECYCLE_FAILED_STEP, failedStepKey.getName())
.put(LifecycleSettings.LIFECYCLE_STEP, errorStepKey.getName());
ClusterState clusterState = buildClusterState(indexName, indexSettingsBuilder);
ClusterState clusterState = buildClusterState(indexName, indexSettingsBuilder, Collections.emptyList());
Index index = clusterState.metaData().index(indexName).getIndex();
IndexLifecycleRunner runner = new IndexLifecycleRunner(policyRegistry, null, () -> now);
ClusterState nextClusterState = runner.moveClusterStateToFailedStep(clusterState, indices);
@ -731,7 +740,7 @@ public class IndexLifecycleRunnerTests extends ESTestCase {
public void testMoveClusterStateToFailedStepIndexNotFound() {
String existingIndexName = "my_index";
String invalidIndexName = "does_not_exist";
ClusterState clusterState = buildClusterState(existingIndexName, Settings.builder());
ClusterState clusterState = buildClusterState(existingIndexName, Settings.builder(), Collections.emptyList());
IndexLifecycleRunner runner = new IndexLifecycleRunner(null, null, () -> 0L);
IllegalArgumentException exception = expectThrows(IllegalArgumentException.class,
() -> runner.moveClusterStateToFailedStep(clusterState, new String[] { invalidIndexName }));
@ -753,7 +762,7 @@ public class IndexLifecycleRunnerTests extends ESTestCase {
.put(LifecycleSettings.LIFECYCLE_ACTION, errorStepKey.getAction())
.put(LifecycleSettings.LIFECYCLE_FAILED_STEP, failedStepKey.getName())
.put(LifecycleSettings.LIFECYCLE_STEP, errorStepKey.getName());
ClusterState clusterState = buildClusterState(indexName, indexSettingsBuilder);
ClusterState clusterState = buildClusterState(indexName, indexSettingsBuilder, Collections.emptyList());
IndexLifecycleRunner runner = new IndexLifecycleRunner(policyRegistry, null, () -> now);
IllegalArgumentException exception = expectThrows(IllegalArgumentException.class,
() -> runner.moveClusterStateToFailedStep(clusterState, indices));
@ -773,7 +782,7 @@ public class IndexLifecycleRunnerTests extends ESTestCase {
.put(LifecycleSettings.LIFECYCLE_PHASE, failedStepKey.getPhase())
.put(LifecycleSettings.LIFECYCLE_ACTION, failedStepKey.getAction())
.put(LifecycleSettings.LIFECYCLE_STEP, failedStepKey.getName());
ClusterState clusterState = buildClusterState(indexName, indexSettingsBuilder);
ClusterState clusterState = buildClusterState(indexName, indexSettingsBuilder, Collections.emptyList());
IndexLifecycleRunner runner = new IndexLifecycleRunner(policyRegistry, null, () -> now);
IllegalArgumentException exception = expectThrows(IllegalArgumentException.class,
() -> runner.moveClusterStateToFailedStep(clusterState, indices));
@ -789,7 +798,8 @@ public class IndexLifecycleRunnerTests extends ESTestCase {
ClusterState clusterState = buildClusterState(indexName,
Settings.builder().put(LifecycleSettings.LIFECYCLE_PHASE, currentStep.getPhase())
.put(LifecycleSettings.LIFECYCLE_ACTION, currentStep.getAction())
.put(LifecycleSettings.LIFECYCLE_STEP, currentStep.getName()));
.put(LifecycleSettings.LIFECYCLE_STEP, currentStep.getName()),
Collections.emptyList());
Index index = clusterState.metaData().index(indexName).getIndex();
ClusterState newClusterState = IndexLifecycleRunner.addStepInfoToClusterState(index, clusterState, stepInfo);
assertClusterStateStepInfo(clusterState, index, currentStep, newClusterState, stepInfo);
@ -804,7 +814,8 @@ public class IndexLifecycleRunnerTests extends ESTestCase {
.put(LifecycleSettings.LIFECYCLE_PHASE, randomAlphaOfLength(5))
.put(LifecycleSettings.LIFECYCLE_ACTION, randomAlphaOfLength(5))
.put(LifecycleSettings.LIFECYCLE_STEP, randomAlphaOfLength(5))
.put(LifecycleSettings.LIFECYCLE_SKIP, true));
.put(LifecycleSettings.LIFECYCLE_SKIP, true),
Collections.emptyList());
Step step = mock(randomFrom(TerminalPolicyStep.class, ClusterStateActionStep.class,
ClusterStateWaitStep.class, AsyncActionStep.class, AsyncWaitStep.class));
PolicyStepsRegistry stepRegistry = createOneStepPolicyStepRegistry(policy, step);
@ -814,12 +825,19 @@ public class IndexLifecycleRunnerTests extends ESTestCase {
Mockito.verifyZeroInteractions(clusterService);
}
private ClusterState buildClusterState(String indexName, Settings.Builder indexSettingsBuilder) {
private ClusterState buildClusterState(String indexName, Settings.Builder indexSettingsBuilder,
List<LifecyclePolicyMetadata> lifecyclePolicyMetadatas) {
Settings indexSettings = indexSettingsBuilder.put(IndexMetaData.SETTING_NUMBER_OF_SHARDS, 1)
.put(IndexMetaData.SETTING_NUMBER_OF_REPLICAS, 0).put(IndexMetaData.SETTING_VERSION_CREATED, Version.CURRENT).build();
IndexMetaData indexMetadata = IndexMetaData.builder(indexName).settings(indexSettings)
.build();
MetaData metadata = MetaData.builder().put(indexMetadata, true).build();
Map<String, LifecyclePolicyMetadata> lifecyclePolicyMetadatasMap = lifecyclePolicyMetadatas.stream()
.collect(Collectors.toMap(LifecyclePolicyMetadata::getName, Function.identity()));
IndexLifecycleMetadata indexLifecycleMetadata = new IndexLifecycleMetadata(lifecyclePolicyMetadatasMap, OperationMode.RUNNING);
MetaData metadata = MetaData.builder().put(indexMetadata, true).putCustom(IndexLifecycleMetadata.TYPE, indexLifecycleMetadata)
.build();
return ClusterState.builder(new ClusterName("my_cluster")).metaData(metadata).build();
}
@ -829,12 +847,15 @@ public class IndexLifecycleRunnerTests extends ESTestCase {
String oldPolicyName = "old_policy";
String newPolicyName = "new_policy";
LifecyclePolicy newPolicy = new LifecyclePolicy(TestLifecycleType.INSTANCE, newPolicyName, Collections.emptyMap());
StepKey currentStep = AbstractStepTestCase.randomStepKey();
StepKey currentStep = new StepKey(randomAlphaOfLength(10), MockAction.NAME, randomAlphaOfLength(10));
LifecyclePolicy oldPolicy = createPolicy(oldPolicyName, currentStep, null);
Settings.Builder indexSettingsBuilder = Settings.builder().put(LifecycleSettings.LIFECYCLE_NAME, oldPolicyName)
.put(LifecycleSettings.LIFECYCLE_PHASE, currentStep.getPhase())
.put(LifecycleSettings.LIFECYCLE_ACTION, currentStep.getAction())
.put(LifecycleSettings.LIFECYCLE_STEP, currentStep.getName()).put(LifecycleSettings.LIFECYCLE_SKIP, true);
ClusterState clusterState = buildClusterState(indexName, indexSettingsBuilder);
List<LifecyclePolicyMetadata> policyMetadatas = new ArrayList<>();
policyMetadatas.add(new LifecyclePolicyMetadata(oldPolicy, Collections.emptyMap()));
ClusterState clusterState = buildClusterState(indexName, indexSettingsBuilder, policyMetadatas);
Index index = clusterState.metaData().index(indexName).getIndex();
Index[] indices = new Index[] { index };
List<String> failedIndexes = new ArrayList<>();
@ -853,7 +874,7 @@ public class IndexLifecycleRunnerTests extends ESTestCase {
LifecyclePolicy newPolicy = new LifecyclePolicy(TestLifecycleType.INSTANCE, newPolicyName, Collections.emptyMap());
StepKey currentStep = new StepKey("", "", "");
Settings.Builder indexSettingsBuilder = Settings.builder();
ClusterState clusterState = buildClusterState(indexName, indexSettingsBuilder);
ClusterState clusterState = buildClusterState(indexName, indexSettingsBuilder, Collections.emptyList());
Index index = clusterState.metaData().index(indexName).getIndex();
Index[] indices = new Index[] { index };
List<String> failedIndexes = new ArrayList<>();
@ -869,13 +890,16 @@ public class IndexLifecycleRunnerTests extends ESTestCase {
String indexName = randomAlphaOfLength(10);
String oldPolicyName = "old_policy";
String newPolicyName = "new_policy";
LifecyclePolicy oldPolicy = new LifecyclePolicy(TestLifecycleType.INSTANCE, oldPolicyName, Collections.emptyMap());
LifecyclePolicy newPolicy = new LifecyclePolicy(TestLifecycleType.INSTANCE, newPolicyName, Collections.emptyMap());
StepKey currentStep = AbstractStepTestCase.randomStepKey();
Settings.Builder indexSettingsBuilder = Settings.builder().put(LifecycleSettings.LIFECYCLE_NAME, oldPolicyName)
.put(LifecycleSettings.LIFECYCLE_PHASE, currentStep.getPhase())
.put(LifecycleSettings.LIFECYCLE_ACTION, currentStep.getAction())
.put(LifecycleSettings.LIFECYCLE_STEP, currentStep.getName()).put(LifecycleSettings.LIFECYCLE_SKIP, true);
ClusterState clusterState = buildClusterState(indexName, indexSettingsBuilder);
List<LifecyclePolicyMetadata> policyMetadatas = new ArrayList<>();
policyMetadatas.add(new LifecyclePolicyMetadata(oldPolicy, Collections.emptyMap()));
ClusterState clusterState = buildClusterState(indexName, indexSettingsBuilder, policyMetadatas);
Index index = new Index("doesnt_exist", "im_not_here");
Index[] indices = new Index[] { index };
List<String> failedIndexes = new ArrayList<>();
@ -888,17 +912,20 @@ public class IndexLifecycleRunnerTests extends ESTestCase {
assertSame(clusterState, newClusterState);
}
public void testSetPolicyForIndexIndexInShrink() {
public void testSetPolicyForIndexIndexInUnsafe() {
String indexName = randomAlphaOfLength(10);
String oldPolicyName = "old_policy";
String newPolicyName = "new_policy";
LifecyclePolicy newPolicy = new LifecyclePolicy(TestLifecycleType.INSTANCE, newPolicyName, Collections.emptyMap());
StepKey currentStep = new StepKey(randomAlphaOfLength(10), ShrinkAction.NAME, randomAlphaOfLength(10));
StepKey currentStep = new StepKey(randomAlphaOfLength(10), MockAction.NAME, randomAlphaOfLength(10));
LifecyclePolicy oldPolicy = createPolicy(oldPolicyName, null, currentStep);
Settings.Builder indexSettingsBuilder = Settings.builder().put(LifecycleSettings.LIFECYCLE_NAME, oldPolicyName)
.put(LifecycleSettings.LIFECYCLE_PHASE, currentStep.getPhase())
.put(LifecycleSettings.LIFECYCLE_ACTION, currentStep.getAction())
.put(LifecycleSettings.LIFECYCLE_STEP, currentStep.getName()).put(LifecycleSettings.LIFECYCLE_SKIP, true);
ClusterState clusterState = buildClusterState(indexName, indexSettingsBuilder);
List<LifecyclePolicyMetadata> policyMetadatas = new ArrayList<>();
policyMetadatas.add(new LifecyclePolicyMetadata(oldPolicy, Collections.emptyMap()));
ClusterState clusterState = buildClusterState(indexName, indexSettingsBuilder, policyMetadatas);
Index index = clusterState.metaData().index(indexName).getIndex();
Index[] indices = new Index[] { index };
List<String> failedIndexes = new ArrayList<>();
@ -911,34 +938,64 @@ public class IndexLifecycleRunnerTests extends ESTestCase {
assertSame(clusterState, newClusterState);
}
private static LifecyclePolicy createPolicy(String policyName, StepKey safeStep, StepKey unsafeStep) {
Map<String, Phase> phases = new HashMap<>();
if (safeStep != null) {
assert MockAction.NAME.equals(safeStep.getAction()) : "The safe action needs to be MockAction.NAME";
assert unsafeStep == null
|| safeStep.getPhase().equals(unsafeStep.getPhase()) == false : "safe and unsafe actions must be in different phases";
Map<String, LifecycleAction> actions = new HashMap<>();
MockAction safeAction = new MockAction(Collections.emptyList(), true);
actions.put(safeAction.getWriteableName(), safeAction);
Phase phase = new Phase(safeStep.getPhase(), TimeValue.timeValueMillis(0), actions);
phases.put(phase.getName(), phase);
}
if (unsafeStep != null) {
assert MockAction.NAME.equals(unsafeStep.getAction()) : "The unsafe action needs to be MockAction.NAME";
Map<String, LifecycleAction> actions = new HashMap<>();
MockAction unsafeAction = new MockAction(Collections.emptyList(), false);
actions.put(unsafeAction.getWriteableName(), unsafeAction);
Phase phase = new Phase(unsafeStep.getPhase(), TimeValue.timeValueMillis(0), actions);
phases.put(phase.getName(), phase);
}
LifecyclePolicy oldPolicy = new LifecyclePolicy(TestLifecycleType.INSTANCE, policyName, phases);
return oldPolicy;
}
public void testCanUpdatePolicy() {
String indexName = randomAlphaOfLength(10);
String oldPolicyName = "old_policy";
String newPolicyName = "new_policy";
LifecyclePolicy newPolicy = new LifecyclePolicy(TestLifecycleType.INSTANCE, newPolicyName, Collections.emptyMap());
StepKey currentStep = AbstractStepTestCase.randomStepKey();
StepKey currentStep = new StepKey(randomAlphaOfLength(10), MockAction.NAME, randomAlphaOfLength(10));
LifecyclePolicy oldPolicy = createPolicy(oldPolicyName, currentStep, null);
Settings.Builder indexSettingsBuilder = Settings.builder().put(LifecycleSettings.LIFECYCLE_NAME, oldPolicyName)
.put(LifecycleSettings.LIFECYCLE_PHASE, currentStep.getPhase())
.put(LifecycleSettings.LIFECYCLE_ACTION, currentStep.getAction())
.put(LifecycleSettings.LIFECYCLE_STEP, currentStep.getName()).put(LifecycleSettings.LIFECYCLE_SKIP, true);
ClusterState clusterState = buildClusterState(indexName, indexSettingsBuilder);
List<LifecyclePolicyMetadata> policyMetadatas = new ArrayList<>();
policyMetadatas.add(new LifecyclePolicyMetadata(oldPolicy, Collections.emptyMap()));
ClusterState clusterState = buildClusterState(indexName, indexSettingsBuilder, policyMetadatas);
boolean canUpdatePolicy = IndexLifecycleRunner.canUpdatePolicy(oldPolicyName, newPolicy, clusterState);
assertTrue(canUpdatePolicy);
}
public void testCanUpdatePolicyIndexInShrink() {
public void testCanUpdatePolicyIndexInUnsafe() {
String indexName = randomAlphaOfLength(10);
String oldPolicyName = "old_policy";
String newPolicyName = "new_policy";
LifecyclePolicy newPolicy = new LifecyclePolicy(TestLifecycleType.INSTANCE, newPolicyName, Collections.emptyMap());
StepKey currentStep = new StepKey(randomAlphaOfLength(10), ShrinkAction.NAME, randomAlphaOfLength(10));
StepKey currentStep = new StepKey(randomAlphaOfLength(10), MockAction.NAME, randomAlphaOfLength(10));
LifecyclePolicy oldPolicy = createPolicy(oldPolicyName, null, currentStep);
Settings.Builder indexSettingsBuilder = Settings.builder().put(LifecycleSettings.LIFECYCLE_NAME, oldPolicyName)
.put(LifecycleSettings.LIFECYCLE_PHASE, currentStep.getPhase())
.put(LifecycleSettings.LIFECYCLE_ACTION, currentStep.getAction())
.put(LifecycleSettings.LIFECYCLE_STEP, currentStep.getName()).put(LifecycleSettings.LIFECYCLE_SKIP, true);
ClusterState clusterState = buildClusterState(indexName, indexSettingsBuilder);
List<LifecyclePolicyMetadata> policyMetadatas = new ArrayList<>();
policyMetadatas.add(new LifecyclePolicyMetadata(oldPolicy, Collections.emptyMap()));
ClusterState clusterState = buildClusterState(indexName, indexSettingsBuilder, policyMetadatas);
boolean canUpdatePolicy = IndexLifecycleRunner.canUpdatePolicy(oldPolicyName, newPolicy, clusterState);
@ -949,9 +1006,12 @@ public class IndexLifecycleRunnerTests extends ESTestCase {
String indexName = randomAlphaOfLength(10);
String oldPolicyName = "old_policy";
String newPolicyName = "new_policy";
LifecyclePolicy oldPolicy = new LifecyclePolicy(TestLifecycleType.INSTANCE, oldPolicyName, Collections.emptyMap());
LifecyclePolicy newPolicy = new LifecyclePolicy(TestLifecycleType.INSTANCE, newPolicyName, Collections.emptyMap());
Settings.Builder indexSettingsBuilder = Settings.builder();
ClusterState clusterState = buildClusterState(indexName, indexSettingsBuilder);
List<LifecyclePolicyMetadata> policyMetadatas = new ArrayList<>();
policyMetadatas.add(new LifecyclePolicyMetadata(oldPolicy, Collections.emptyMap()));
ClusterState clusterState = buildClusterState(indexName, indexSettingsBuilder, policyMetadatas);
boolean canUpdatePolicy = IndexLifecycleRunner.canUpdatePolicy(oldPolicyName, newPolicy, clusterState);
@ -962,13 +1022,16 @@ public class IndexLifecycleRunnerTests extends ESTestCase {
String indexName = randomAlphaOfLength(10);
String oldPolicyName = "old_policy";
String newPolicyName = "new_policy";
LifecyclePolicy oldPolicy = new LifecyclePolicy(TestLifecycleType.INSTANCE, oldPolicyName, Collections.emptyMap());
LifecyclePolicy newPolicy = new LifecyclePolicy(TestLifecycleType.INSTANCE, newPolicyName, Collections.emptyMap());
StepKey currentStep = new StepKey(randomAlphaOfLength(10), ShrinkAction.NAME, randomAlphaOfLength(10));
Settings.Builder indexSettingsBuilder = Settings.builder().put(LifecycleSettings.LIFECYCLE_NAME, "different_policy")
.put(LifecycleSettings.LIFECYCLE_PHASE, currentStep.getPhase())
.put(LifecycleSettings.LIFECYCLE_ACTION, currentStep.getAction())
.put(LifecycleSettings.LIFECYCLE_STEP, currentStep.getName()).put(LifecycleSettings.LIFECYCLE_SKIP, true);
ClusterState clusterState = buildClusterState(indexName, indexSettingsBuilder);
List<LifecyclePolicyMetadata> policyMetadatas = new ArrayList<>();
policyMetadatas.add(new LifecyclePolicyMetadata(oldPolicy, Collections.emptyMap()));
ClusterState clusterState = buildClusterState(indexName, indexSettingsBuilder, policyMetadatas);
boolean canUpdatePolicy = IndexLifecycleRunner.canUpdatePolicy(oldPolicyName, newPolicy, clusterState);
@ -981,7 +1044,7 @@ public class IndexLifecycleRunnerTests extends ESTestCase {
LifecyclePolicy newPolicy = new LifecyclePolicy(TestLifecycleType.INSTANCE, newPolicyName, Collections.emptyMap());
String index1Name = randomAlphaOfLength(10);
StepKey currentStep1 = AbstractStepTestCase.randomStepKey();
StepKey currentStep1 = new StepKey(randomAlphaOfLength(10), MockAction.NAME, randomAlphaOfLength(10));
Settings.Builder indexSettingsBuilder1 = Settings.builder().put(IndexMetaData.SETTING_NUMBER_OF_SHARDS, 1)
.put(IndexMetaData.SETTING_NUMBER_OF_REPLICAS, 0).put(IndexMetaData.SETTING_VERSION_CREATED, Version.CURRENT)
.put(LifecycleSettings.LIFECYCLE_NAME, oldPolicyName).put(LifecycleSettings.LIFECYCLE_PHASE, currentStep1.getPhase())
@ -990,7 +1053,7 @@ public class IndexLifecycleRunnerTests extends ESTestCase {
IndexMetaData indexMetadata1 = IndexMetaData.builder(index1Name).settings(indexSettingsBuilder1).build();
String index2Name = randomAlphaOfLength(10);
StepKey currentStep2 = AbstractStepTestCase.randomStepKey();
StepKey currentStep2 = currentStep1;
Settings.Builder indexSettingsBuilder2 = Settings.builder().put(IndexMetaData.SETTING_NUMBER_OF_SHARDS, 1)
.put(IndexMetaData.SETTING_NUMBER_OF_REPLICAS, 0).put(IndexMetaData.SETTING_VERSION_CREATED, Version.CURRENT)
.put(LifecycleSettings.LIFECYCLE_NAME, oldPolicyName).put(LifecycleSettings.LIFECYCLE_PHASE, currentStep2.getPhase())
@ -999,7 +1062,7 @@ public class IndexLifecycleRunnerTests extends ESTestCase {
IndexMetaData indexMetadata2 = IndexMetaData.builder(index2Name).settings(indexSettingsBuilder2).build();
String index3Name = randomAlphaOfLength(10);
StepKey currentStep3 = new StepKey(randomAlphaOfLength(10), ShrinkAction.NAME, randomAlphaOfLength(10));
StepKey currentStep3 = new StepKey(randomAlphaOfLength(10), MockAction.NAME, randomAlphaOfLength(10));
Settings.Builder indexSettingsBuilder3 = Settings.builder().put(IndexMetaData.SETTING_NUMBER_OF_SHARDS, 1)
.put(IndexMetaData.SETTING_NUMBER_OF_REPLICAS, 0).put(IndexMetaData.SETTING_VERSION_CREATED, Version.CURRENT)
.put(LifecycleSettings.LIFECYCLE_NAME, "different_policy").put(LifecycleSettings.LIFECYCLE_PHASE, currentStep3.getPhase())
@ -1012,8 +1075,15 @@ public class IndexLifecycleRunnerTests extends ESTestCase {
.put(IndexMetaData.SETTING_NUMBER_OF_REPLICAS, 0).put(IndexMetaData.SETTING_VERSION_CREATED, Version.CURRENT);
IndexMetaData indexMetadata4 = IndexMetaData.builder(index4Name).settings(indexSettingsBuilder4).build();
Map<String, LifecyclePolicyMetadata> lifecyclePolicyMetadatasMap = new HashMap<>();
lifecyclePolicyMetadatasMap.put(oldPolicyName,
new LifecyclePolicyMetadata(createPolicy(oldPolicyName, currentStep1, null), Collections.emptyMap()));
lifecyclePolicyMetadatasMap.put("different_policy",
new LifecyclePolicyMetadata(createPolicy("different_policy", null, currentStep3), Collections.emptyMap()));
IndexLifecycleMetadata indexLifecycleMetadata = new IndexLifecycleMetadata(lifecyclePolicyMetadatasMap, OperationMode.RUNNING);
MetaData metadata = MetaData.builder().put(indexMetadata1, true).put(indexMetadata2, true).put(indexMetadata3, true)
.put(indexMetadata4, true).build();
.put(indexMetadata4, true).putCustom(IndexLifecycleMetadata.TYPE, indexLifecycleMetadata).build();
ClusterState clusterState = ClusterState.builder(new ClusterName("my_cluster")).metaData(metadata).build();
boolean canUpdatePolicy = IndexLifecycleRunner.canUpdatePolicy(oldPolicyName, newPolicy, clusterState);
@ -1027,7 +1097,7 @@ public class IndexLifecycleRunnerTests extends ESTestCase {
LifecyclePolicy newPolicy = new LifecyclePolicy(TestLifecycleType.INSTANCE, newPolicyName, Collections.emptyMap());
String index1Name = randomAlphaOfLength(10);
StepKey currentStep1 = AbstractStepTestCase.randomStepKey();
StepKey currentStep1 = new StepKey(randomAlphaOfLength(10), MockAction.NAME, randomAlphaOfLength(10));
Settings.Builder indexSettingsBuilder1 = Settings.builder().put(IndexMetaData.SETTING_NUMBER_OF_SHARDS, 1)
.put(IndexMetaData.SETTING_NUMBER_OF_REPLICAS, 0).put(IndexMetaData.SETTING_VERSION_CREATED, Version.CURRENT)
.put(LifecycleSettings.LIFECYCLE_NAME, oldPolicyName).put(LifecycleSettings.LIFECYCLE_PHASE, currentStep1.getPhase())
@ -1036,7 +1106,7 @@ public class IndexLifecycleRunnerTests extends ESTestCase {
IndexMetaData indexMetadata1 = IndexMetaData.builder(index1Name).settings(indexSettingsBuilder1).build();
String index2Name = randomAlphaOfLength(10);
StepKey currentStep2 = new StepKey(randomAlphaOfLength(10), ShrinkAction.NAME, randomAlphaOfLength(10));
StepKey currentStep2 = new StepKey(randomAlphaOfLength(10), MockAction.NAME, randomAlphaOfLength(10));
Settings.Builder indexSettingsBuilder2 = Settings.builder().put(IndexMetaData.SETTING_NUMBER_OF_SHARDS, 1)
.put(IndexMetaData.SETTING_NUMBER_OF_REPLICAS, 0).put(IndexMetaData.SETTING_VERSION_CREATED, Version.CURRENT)
.put(LifecycleSettings.LIFECYCLE_NAME, oldPolicyName).put(LifecycleSettings.LIFECYCLE_PHASE, currentStep2.getPhase())
@ -1045,7 +1115,7 @@ public class IndexLifecycleRunnerTests extends ESTestCase {
IndexMetaData indexMetadata2 = IndexMetaData.builder(index2Name).settings(indexSettingsBuilder2).build();
String index3Name = randomAlphaOfLength(10);
StepKey currentStep3 = new StepKey(randomAlphaOfLength(10), ShrinkAction.NAME, randomAlphaOfLength(10));
StepKey currentStep3 = new StepKey(randomAlphaOfLength(10), MockAction.NAME, randomAlphaOfLength(10));
Settings.Builder indexSettingsBuilder3 = Settings.builder().put(IndexMetaData.SETTING_NUMBER_OF_SHARDS, 1)
.put(IndexMetaData.SETTING_NUMBER_OF_REPLICAS, 0).put(IndexMetaData.SETTING_VERSION_CREATED, Version.CURRENT)
.put(LifecycleSettings.LIFECYCLE_NAME, "different_policy").put(LifecycleSettings.LIFECYCLE_PHASE, currentStep3.getPhase())
@ -1058,8 +1128,15 @@ public class IndexLifecycleRunnerTests extends ESTestCase {
.put(IndexMetaData.SETTING_NUMBER_OF_REPLICAS, 0).put(IndexMetaData.SETTING_VERSION_CREATED, Version.CURRENT);
IndexMetaData indexMetadata4 = IndexMetaData.builder(index4Name).settings(indexSettingsBuilder4).build();
Map<String, LifecyclePolicyMetadata> lifecyclePolicyMetadatasMap = new HashMap<>();
lifecyclePolicyMetadatasMap.put(oldPolicyName,
new LifecyclePolicyMetadata(createPolicy(oldPolicyName, currentStep1, currentStep2), Collections.emptyMap()));
lifecyclePolicyMetadatasMap.put("different_policy",
new LifecyclePolicyMetadata(createPolicy("different_policy", null, currentStep3), Collections.emptyMap()));
IndexLifecycleMetadata indexLifecycleMetadata = new IndexLifecycleMetadata(lifecyclePolicyMetadatasMap, OperationMode.RUNNING);
MetaData metadata = MetaData.builder().put(indexMetadata1, true).put(indexMetadata2, true).put(indexMetadata3, true)
.put(indexMetadata4, true).build();
.put(indexMetadata4, true).putCustom(IndexLifecycleMetadata.TYPE, indexLifecycleMetadata).build();
ClusterState clusterState = ClusterState.builder(new ClusterName("my_cluster")).metaData(metadata).build();
boolean canUpdatePolicy = IndexLifecycleRunner.canUpdatePolicy(oldPolicyName, newPolicy, clusterState);
@ -1070,12 +1147,15 @@ public class IndexLifecycleRunnerTests extends ESTestCase {
public void testRemovePolicyForIndex() {
String indexName = randomAlphaOfLength(10);
String oldPolicyName = "old_policy";
StepKey currentStep = AbstractStepTestCase.randomStepKey();
StepKey currentStep = new StepKey(randomAlphaOfLength(10), MockAction.NAME, randomAlphaOfLength(10));
LifecyclePolicy oldPolicy = createPolicy(oldPolicyName, currentStep, null);
Settings.Builder indexSettingsBuilder = Settings.builder().put(LifecycleSettings.LIFECYCLE_NAME, oldPolicyName)
.put(LifecycleSettings.LIFECYCLE_PHASE, currentStep.getPhase())
.put(LifecycleSettings.LIFECYCLE_ACTION, currentStep.getAction())
.put(LifecycleSettings.LIFECYCLE_STEP, currentStep.getName()).put(LifecycleSettings.LIFECYCLE_SKIP, true);
ClusterState clusterState = buildClusterState(indexName, indexSettingsBuilder);
List<LifecyclePolicyMetadata> policyMetadatas = new ArrayList<>();
policyMetadatas.add(new LifecyclePolicyMetadata(oldPolicy, Collections.emptyMap()));
ClusterState clusterState = buildClusterState(indexName, indexSettingsBuilder, policyMetadatas);
Index index = clusterState.metaData().index(indexName).getIndex();
Index[] indices = new Index[] { index };
List<String> failedIndexes = new ArrayList<>();
@ -1089,7 +1169,7 @@ public class IndexLifecycleRunnerTests extends ESTestCase {
public void testRemovePolicyForIndexNoCurrentPolicy() {
String indexName = randomAlphaOfLength(10);
Settings.Builder indexSettingsBuilder = Settings.builder();
ClusterState clusterState = buildClusterState(indexName, indexSettingsBuilder);
ClusterState clusterState = buildClusterState(indexName, indexSettingsBuilder, Collections.emptyList());
Index index = clusterState.metaData().index(indexName).getIndex();
Index[] indices = new Index[] { index };
List<String> failedIndexes = new ArrayList<>();
@ -1103,12 +1183,15 @@ public class IndexLifecycleRunnerTests extends ESTestCase {
public void testRemovePolicyForIndexIndexDoesntExist() {
String indexName = randomAlphaOfLength(10);
String oldPolicyName = "old_policy";
LifecyclePolicy oldPolicy = new LifecyclePolicy(TestLifecycleType.INSTANCE, oldPolicyName, Collections.emptyMap());
StepKey currentStep = AbstractStepTestCase.randomStepKey();
Settings.Builder indexSettingsBuilder = Settings.builder().put(LifecycleSettings.LIFECYCLE_NAME, oldPolicyName)
.put(LifecycleSettings.LIFECYCLE_PHASE, currentStep.getPhase())
.put(LifecycleSettings.LIFECYCLE_ACTION, currentStep.getAction())
.put(LifecycleSettings.LIFECYCLE_STEP, currentStep.getName()).put(LifecycleSettings.LIFECYCLE_SKIP, true);
ClusterState clusterState = buildClusterState(indexName, indexSettingsBuilder);
List<LifecyclePolicyMetadata> policyMetadatas = new ArrayList<>();
policyMetadatas.add(new LifecyclePolicyMetadata(oldPolicy, Collections.emptyMap()));
ClusterState clusterState = buildClusterState(indexName, indexSettingsBuilder, policyMetadatas);
Index index = new Index("doesnt_exist", "im_not_here");
Index[] indices = new Index[] { index };
List<String> failedIndexes = new ArrayList<>();
@ -1120,15 +1203,18 @@ public class IndexLifecycleRunnerTests extends ESTestCase {
assertSame(clusterState, newClusterState);
}
public void testRemovePolicyForIndexIndexInShrink() {
public void testRemovePolicyForIndexIndexInUnsafe() {
String indexName = randomAlphaOfLength(10);
String oldPolicyName = "old_policy";
StepKey currentStep = new StepKey(randomAlphaOfLength(10), ShrinkAction.NAME, randomAlphaOfLength(10));
StepKey currentStep = new StepKey(randomAlphaOfLength(10), MockAction.NAME, randomAlphaOfLength(10));
LifecyclePolicy oldPolicy = createPolicy(oldPolicyName, null, currentStep);
Settings.Builder indexSettingsBuilder = Settings.builder().put(LifecycleSettings.LIFECYCLE_NAME, oldPolicyName)
.put(LifecycleSettings.LIFECYCLE_PHASE, currentStep.getPhase())
.put(LifecycleSettings.LIFECYCLE_ACTION, currentStep.getAction())
.put(LifecycleSettings.LIFECYCLE_STEP, currentStep.getName()).put(LifecycleSettings.LIFECYCLE_SKIP, true);
ClusterState clusterState = buildClusterState(indexName, indexSettingsBuilder);
List<LifecyclePolicyMetadata> policyMetadatas = new ArrayList<>();
policyMetadatas.add(new LifecyclePolicyMetadata(oldPolicy, Collections.emptyMap()));
ClusterState clusterState = buildClusterState(indexName, indexSettingsBuilder, policyMetadatas);
Index index = clusterState.metaData().index(indexName).getIndex();
Index[] indices = new Index[] { index };
List<String> failedIndexes = new ArrayList<>();