moar changes
This commit is contained in:
parent
d2e87a66e5
commit
1ac1ee413f
|
@ -0,0 +1,2 @@
|
|||
[[xpack-index-lifecycle]]
|
||||
= Automating Index Properties Over Time
|
|
@ -8,23 +8,17 @@ package org.elasticsearch.xpack.core.indexlifecycle;
|
|||
import org.apache.logging.log4j.Logger;
|
||||
import org.elasticsearch.client.Client;
|
||||
import org.elasticsearch.cluster.AbstractDiffable;
|
||||
import org.elasticsearch.cluster.ClusterState;
|
||||
import org.elasticsearch.cluster.ClusterStateUpdateTask;
|
||||
import org.elasticsearch.cluster.Diffable;
|
||||
import org.elasticsearch.cluster.metadata.IndexMetaData;
|
||||
import org.elasticsearch.cluster.metadata.MetaData;
|
||||
import org.elasticsearch.common.ParseField;
|
||||
import org.elasticsearch.common.Strings;
|
||||
import org.elasticsearch.common.io.stream.StreamInput;
|
||||
import org.elasticsearch.common.io.stream.StreamOutput;
|
||||
import org.elasticsearch.common.logging.ESLoggerFactory;
|
||||
import org.elasticsearch.common.settings.Settings;
|
||||
import org.elasticsearch.common.xcontent.ConstructingObjectParser;
|
||||
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.index.Index;
|
||||
|
||||
import java.io.IOException;
|
||||
import java.util.ArrayList;
|
||||
|
@ -33,7 +27,6 @@ import java.util.List;
|
|||
import java.util.ListIterator;
|
||||
import java.util.Map;
|
||||
import java.util.Objects;
|
||||
import java.util.Optional;
|
||||
import java.util.function.Function;
|
||||
import java.util.function.LongSupplier;
|
||||
import java.util.stream.Collectors;
|
||||
|
@ -167,12 +160,15 @@ public class LifecyclePolicy extends AbstractDiffable<LifecyclePolicy>
|
|||
lastStepKey = step.getKey();
|
||||
}
|
||||
}
|
||||
|
||||
// add `after` step for phase
|
||||
Step.StepKey afterStepKey = new Step.StepKey(phase.getName(), null, "after");
|
||||
Step phaseAfterStep = new PhaseAfterStep(nowSupplier, phase.getAfter(), afterStepKey, lastStepKey);
|
||||
steps.add(phaseAfterStep);
|
||||
lastStepKey = phaseAfterStep.getKey();
|
||||
}
|
||||
|
||||
// init step so that policy is guaranteed to have
|
||||
steps.add(new InitializePolicyContextStep(new Step.StepKey("", "", ""), lastStepKey));
|
||||
|
||||
Collections.reverse(steps);
|
||||
|
|
|
@ -0,0 +1,66 @@
|
|||
/*
|
||||
* 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.client.Client;
|
||||
import org.elasticsearch.common.Strings;
|
||||
import org.elasticsearch.common.io.stream.StreamInput;
|
||||
import org.elasticsearch.common.io.stream.StreamOutput;
|
||||
import org.elasticsearch.common.xcontent.ConstructingObjectParser;
|
||||
import org.elasticsearch.common.xcontent.XContentBuilder;
|
||||
import org.elasticsearch.common.xcontent.XContentParser;
|
||||
|
||||
import java.io.IOException;
|
||||
import java.util.Collections;
|
||||
import java.util.List;
|
||||
|
||||
/**
|
||||
* A {@link LifecycleAction} which force-merges the index.
|
||||
*/
|
||||
public class ReadOnlyAction implements LifecycleAction {
|
||||
public static final String NAME = "readonly";
|
||||
public static final ReadOnlyAction INSTANCE = new ReadOnlyAction();
|
||||
|
||||
private static final ConstructingObjectParser<ReadOnlyAction, Void> PARSER = new ConstructingObjectParser<>(NAME,
|
||||
false, a -> new ReadOnlyAction());
|
||||
|
||||
public static ReadOnlyAction parse(XContentParser parser) {
|
||||
return PARSER.apply(parser, null);
|
||||
}
|
||||
|
||||
public ReadOnlyAction() {
|
||||
}
|
||||
|
||||
public ReadOnlyAction(StreamInput in) {
|
||||
}
|
||||
|
||||
@Override
|
||||
public String getWriteableName() {
|
||||
return NAME;
|
||||
}
|
||||
|
||||
@Override
|
||||
public XContentBuilder toXContent(XContentBuilder builder, Params params) throws IOException {
|
||||
builder.startObject();
|
||||
builder.endObject();
|
||||
return builder;
|
||||
}
|
||||
|
||||
@Override
|
||||
public void writeTo(StreamOutput out) throws IOException {
|
||||
}
|
||||
|
||||
@Override
|
||||
public List<Step> toSteps(Client client, String phase, Step.StepKey nextStepKey) {
|
||||
Step.StepKey key = new Step.StepKey(phase, NAME, "readonly-step");
|
||||
return Collections.singletonList(new ReadOnlyStep(key, nextStepKey));
|
||||
}
|
||||
|
||||
@Override
|
||||
public String toString() {
|
||||
return Strings.toString(this);
|
||||
}
|
||||
}
|
|
@ -0,0 +1,26 @@
|
|||
/*
|
||||
* 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.cluster.ClusterState;
|
||||
import org.elasticsearch.cluster.metadata.IndexMetaData;
|
||||
import org.elasticsearch.cluster.metadata.MetaData;
|
||||
import org.elasticsearch.common.settings.Settings;
|
||||
import org.elasticsearch.index.Index;
|
||||
|
||||
public class ReadOnlyStep extends ClusterStateActionStep {
|
||||
|
||||
ReadOnlyStep(StepKey key, StepKey nextStepKey) {
|
||||
super(key, nextStepKey);
|
||||
}
|
||||
|
||||
@Override
|
||||
public ClusterState performAction(Index index, ClusterState clusterState) {
|
||||
return ClusterState.builder(clusterState).metaData(MetaData.builder(clusterState.metaData())
|
||||
.updateSettings(Settings.builder().put(IndexMetaData.SETTING_BLOCKS_WRITE, true).build(),
|
||||
index.getName())).build();
|
||||
}
|
||||
}
|
|
@ -34,7 +34,6 @@ public abstract class Step {
|
|||
|
||||
public static class StepKey {
|
||||
private final String phase;
|
||||
|
||||
private final String action;
|
||||
private final String name;
|
||||
|
||||
|
@ -77,5 +76,6 @@ public abstract class Step {
|
|||
public String toString() {
|
||||
return String.format("[%s][%s][%s]", phase, action, name);
|
||||
}
|
||||
|
||||
}
|
||||
}
|
||||
|
|
|
@ -5,11 +5,12 @@
|
|||
*/
|
||||
package org.elasticsearch.xpack.core.indexlifecycle;
|
||||
|
||||
import org.elasticsearch.common.Nullable;
|
||||
import org.elasticsearch.common.io.stream.StreamOutput;
|
||||
import org.elasticsearch.common.unit.TimeValue;
|
||||
import org.elasticsearch.common.util.set.Sets;
|
||||
|
||||
import java.io.IOException;
|
||||
import java.util.ArrayList;
|
||||
import java.util.Arrays;
|
||||
import java.util.Collection;
|
||||
import java.util.Collections;
|
||||
|
@ -35,11 +36,17 @@ public class TimeseriesLifecycleType implements LifecycleType {
|
|||
|
||||
public static final String TYPE = "timeseries";
|
||||
static final List<String> VALID_PHASES = Arrays.asList("hot", "warm", "cold", "delete");
|
||||
static final Set<String> VALID_HOT_ACTIONS = Sets.newHashSet(RolloverAction.NAME);
|
||||
static final Set<String> VALID_WARM_ACTIONS = Sets.newHashSet(AllocateAction.NAME, ReplicasAction.NAME,
|
||||
ShrinkAction.NAME, ForceMergeAction.NAME);
|
||||
static final Set<String> VALID_COLD_ACTIONS = Sets.newHashSet(AllocateAction.NAME, ReplicasAction.NAME);
|
||||
static final Set<String> VALID_DELETE_ACTIONS = Sets.newHashSet(DeleteAction.NAME);
|
||||
static final List<String> ORDERED_VALID_HOT_ACTIONS = Collections.singletonList(RolloverAction.NAME);
|
||||
static final List<String> ORDERED_VALID_WARM_ACTIONS = Arrays.asList(ReadOnlyAction.NAME, AllocateAction.NAME,
|
||||
ReplicasAction.NAME, ShrinkAction.NAME, ForceMergeAction.NAME);
|
||||
static final List<String> ORDERED_VALID_COLD_ACTIONS = Arrays.asList(AllocateAction.NAME, ReplicasAction.NAME);
|
||||
static final List<String> ORDERED_VALID_DELETE_ACTIONS = Arrays.asList(DeleteAction.NAME);
|
||||
static final Set<String> VALID_HOT_ACTIONS = Sets.newHashSet(ORDERED_VALID_HOT_ACTIONS);
|
||||
static final Set<String> VALID_WARM_ACTIONS = Sets.newHashSet(ORDERED_VALID_WARM_ACTIONS);
|
||||
static final Set<String> VALID_COLD_ACTIONS = Sets.newHashSet(ORDERED_VALID_COLD_ACTIONS);
|
||||
static final Set<String> VALID_DELETE_ACTIONS = Sets.newHashSet(ORDERED_VALID_DELETE_ACTIONS);
|
||||
private static final Phase EMPTY_WARM_PHASE = new Phase("warm", TimeValue.ZERO,
|
||||
Collections.singletonMap("readonly", ReadOnlyAction.INSTANCE));
|
||||
|
||||
private TimeseriesLifecycleType() {
|
||||
}
|
||||
|
@ -59,27 +66,42 @@ public class TimeseriesLifecycleType implements LifecycleType {
|
|||
}
|
||||
|
||||
public List<Phase> getOrderedPhases(Map<String, Phase> phases) {
|
||||
return VALID_PHASES.stream().map(p -> phases.getOrDefault(p, null))
|
||||
.filter(Objects::nonNull).collect(Collectors.toList());
|
||||
List<Phase> orderedPhases = new ArrayList<>(VALID_PHASES.size());
|
||||
for (String phaseName : VALID_PHASES) {
|
||||
Phase phase = phases.get(phaseName);
|
||||
if ("warm".equals(phaseName)) {
|
||||
if (phase == null) {
|
||||
phase = EMPTY_WARM_PHASE;
|
||||
} else if (phase.getActions().containsKey(ReadOnlyAction.NAME) == false){
|
||||
Map<String, LifecycleAction> actionMap = new HashMap<>(phase.getActions());
|
||||
actionMap.put(ReadOnlyAction.NAME, ReadOnlyAction.INSTANCE);
|
||||
phase = new Phase(phase.getName(), phase.getAfter(), actionMap);
|
||||
}
|
||||
}
|
||||
if (phase != null) {
|
||||
orderedPhases.add(phase);
|
||||
}
|
||||
}
|
||||
return orderedPhases;
|
||||
}
|
||||
|
||||
public List<LifecycleAction> getOrderedActions(Phase phase) {
|
||||
Map<String, LifecycleAction> actions = phase.getActions();
|
||||
switch (phase.getName()) {
|
||||
case "hot":
|
||||
return Stream.of(RolloverAction.NAME).map(a -> actions.getOrDefault(a, null))
|
||||
return ORDERED_VALID_HOT_ACTIONS.stream().map(a -> actions.getOrDefault(a, null))
|
||||
.filter(Objects::nonNull).collect(Collectors.toList());
|
||||
case "warm":
|
||||
return Stream.of(AllocateAction.NAME, ShrinkAction.NAME, ForceMergeAction.NAME, ReplicasAction.NAME)
|
||||
.map(a -> actions.getOrDefault(a, null)).filter(Objects::nonNull).collect(Collectors.toList());
|
||||
return ORDERED_VALID_WARM_ACTIONS.stream() .map(a -> actions.getOrDefault(a, null))
|
||||
.filter(Objects::nonNull).collect(Collectors.toList());
|
||||
case "cold":
|
||||
return Stream.of(ReplicasAction.NAME, AllocateAction.NAME).map(a -> actions.getOrDefault(a, null))
|
||||
return ORDERED_VALID_COLD_ACTIONS.stream().map(a -> actions.getOrDefault(a, null))
|
||||
.filter(Objects::nonNull).collect(Collectors.toList());
|
||||
case "delete":
|
||||
return Stream.of(DeleteAction.NAME).map(a -> actions.getOrDefault(a, null))
|
||||
return ORDERED_VALID_DELETE_ACTIONS.stream().map(a -> actions.getOrDefault(a, null))
|
||||
.filter(Objects::nonNull).collect(Collectors.toList());
|
||||
default:
|
||||
return Collections.emptyList();
|
||||
throw new IllegalArgumentException("lifecycle type[" + TYPE + "] does not support phase[" + phase.getName() + "]");
|
||||
}
|
||||
}
|
||||
|
||||
|
|
|
@ -12,6 +12,7 @@ import org.elasticsearch.common.Strings;
|
|||
import org.elasticsearch.common.io.stream.StreamInput;
|
||||
import org.elasticsearch.common.io.stream.StreamOutput;
|
||||
import org.elasticsearch.common.xcontent.ConstructingObjectParser;
|
||||
import org.elasticsearch.common.xcontent.ObjectParser;
|
||||
import org.elasticsearch.common.xcontent.XContentBuilder;
|
||||
import org.elasticsearch.common.xcontent.XContentParser;
|
||||
|
||||
|
@ -19,62 +20,39 @@ import java.io.IOException;
|
|||
import java.util.ArrayList;
|
||||
import java.util.List;
|
||||
import java.util.Objects;
|
||||
import java.util.concurrent.atomic.AtomicLong;
|
||||
|
||||
public class MockAction implements LifecycleAction {
|
||||
public static final ParseField COMPLETED_FIELD = new ParseField("completed");
|
||||
public static final ParseField EXECUTED_COUNT_FIELD = new ParseField("executed_count");
|
||||
public static final String NAME = "TEST_ACTION";
|
||||
private SetOnce<Boolean> completed = new SetOnce<>();
|
||||
private final AtomicLong executedCount;
|
||||
private Exception exceptionToThrow = null;
|
||||
private boolean completeOnExecute = false;
|
||||
private final List<Step> steps;
|
||||
private final List<MockStep> steps;
|
||||
|
||||
private static final ConstructingObjectParser<MockAction, Void> PARSER = new ConstructingObjectParser<>(NAME,
|
||||
a -> new MockAction(null, (Boolean) a[0], (Long) a[1]));
|
||||
a -> new MockAction((List<MockStep>) a[0]));
|
||||
|
||||
static {
|
||||
PARSER.declareBoolean(ConstructingObjectParser.optionalConstructorArg(), COMPLETED_FIELD);
|
||||
PARSER.declareLong(ConstructingObjectParser.constructorArg(), EXECUTED_COUNT_FIELD);
|
||||
PARSER.declareField(ConstructingObjectParser.constructorArg(), (p, c) -> p.list(),
|
||||
new ParseField("steps"), ObjectParser.ValueType.OBJECT_ARRAY);
|
||||
}
|
||||
|
||||
public static MockAction parse(XContentParser parser) {
|
||||
return PARSER.apply(parser, null);
|
||||
}
|
||||
|
||||
public MockAction(List<Step> steps) {
|
||||
this(steps, null, 0);
|
||||
}
|
||||
|
||||
MockAction(List<Step> steps, Boolean completed, long executedCount) {
|
||||
public MockAction(List<MockStep> steps) {
|
||||
this.steps = steps;
|
||||
if (completed != null) {
|
||||
this.completed.set(completed);
|
||||
}
|
||||
this.executedCount = new AtomicLong(executedCount);
|
||||
}
|
||||
|
||||
public MockAction(StreamInput in) throws IOException {
|
||||
int numSteps = in.readVInt();
|
||||
this.steps = new ArrayList<>();
|
||||
for (int i = 0; i < numSteps; i++) {
|
||||
// TODO(talevy): make Steps implement NamedWriteable
|
||||
steps.add(null);
|
||||
}
|
||||
Boolean executed = in.readOptionalBoolean();
|
||||
if (executed != null) {
|
||||
this.completed.set(executed);
|
||||
}
|
||||
this.executedCount = new AtomicLong(in.readLong());
|
||||
this.steps = in.readList(MockStep::new);
|
||||
}
|
||||
|
||||
@Override
|
||||
public XContentBuilder toXContent(XContentBuilder builder, Params params) throws IOException {
|
||||
builder.startObject();
|
||||
if (completed.get() != null) {
|
||||
builder.field(COMPLETED_FIELD.getPreferredName(), completed.get());
|
||||
builder.startArray("steps");
|
||||
for (MockStep step : steps) {
|
||||
step.toXContent(builder, params);
|
||||
}
|
||||
builder.field(EXECUTED_COUNT_FIELD.getPreferredName(), executedCount.get());
|
||||
builder.endArray();
|
||||
builder.endObject();
|
||||
return builder;
|
||||
}
|
||||
|
@ -84,40 +62,23 @@ public class MockAction implements LifecycleAction {
|
|||
return NAME;
|
||||
}
|
||||
|
||||
@Override
|
||||
public List<Step> toSteps(Client client, String phase, Step.StepKey nextStepKey) {
|
||||
public List<MockStep> getSteps() {
|
||||
return steps;
|
||||
}
|
||||
|
||||
public void setCompleteOnExecute(boolean completeOnExecute) {
|
||||
this.completeOnExecute = completeOnExecute;
|
||||
}
|
||||
|
||||
public void setExceptionToThrow(Exception exceptionToThrow) {
|
||||
this.exceptionToThrow = exceptionToThrow;
|
||||
}
|
||||
|
||||
public boolean wasCompleted() {
|
||||
return completed.get() != null && completed.get();
|
||||
}
|
||||
|
||||
public void resetCompleted() {
|
||||
completed = new SetOnce<>();
|
||||
}
|
||||
|
||||
public long getExecutedCount() {
|
||||
return executedCount.get();
|
||||
@Override
|
||||
public List<Step> toSteps(Client client, String phase, Step.StepKey nextStepKey) {
|
||||
return new ArrayList<>(steps);
|
||||
}
|
||||
|
||||
@Override
|
||||
public void writeTo(StreamOutput out) throws IOException {
|
||||
out.writeOptionalBoolean(completed.get());
|
||||
out.writeLong(executedCount.get());
|
||||
out.writeList(steps);
|
||||
}
|
||||
|
||||
@Override
|
||||
public int hashCode() {
|
||||
return Objects.hash(completed.get(), executedCount.get());
|
||||
return Objects.hash(steps);
|
||||
}
|
||||
|
||||
@Override
|
||||
|
@ -129,8 +90,7 @@ public class MockAction implements LifecycleAction {
|
|||
return false;
|
||||
}
|
||||
MockAction other = (MockAction) obj;
|
||||
return Objects.equals(completed.get(), other.completed.get()) &&
|
||||
Objects.equals(executedCount.get(), other.executedCount.get());
|
||||
return Objects.equals(steps, other.steps);
|
||||
}
|
||||
|
||||
@Override
|
||||
|
|
|
@ -5,18 +5,29 @@
|
|||
*/
|
||||
package org.elasticsearch.xpack.core.indexlifecycle;
|
||||
|
||||
import org.apache.lucene.util.SetOnce;
|
||||
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> {
|
||||
|
||||
@Override
|
||||
protected MockAction createTestInstance() {
|
||||
return new MockAction(null, randomBoolean() ? null : randomBoolean(), randomLong());
|
||||
List<MockStep> steps = new ArrayList<>();
|
||||
int stepCount = randomIntBetween(2, 10);
|
||||
Step.StepKey currentStepKey = randomStepKey();
|
||||
Step.StepKey nextStepKey = null;
|
||||
for (int i = 0; i < stepCount - 1; i++) {
|
||||
nextStepKey = randomStepKey();
|
||||
steps.add(new MockStep(currentStepKey, nextStepKey));
|
||||
currentStepKey = nextStepKey;
|
||||
}
|
||||
steps.add(new MockStep(currentStepKey, null));
|
||||
return new MockAction(steps);
|
||||
}
|
||||
|
||||
@Override
|
||||
|
@ -31,157 +42,19 @@ public class MockActionTests extends AbstractSerializingTestCase<MockAction> {
|
|||
|
||||
@Override
|
||||
protected MockAction mutateInstance(MockAction instance) throws IOException {
|
||||
boolean completed = instance.wasCompleted();
|
||||
long executedCount = instance.getExecutedCount();
|
||||
switch (randomIntBetween(0, 1)) {
|
||||
case 0:
|
||||
completed = completed == false;
|
||||
break;
|
||||
case 1:
|
||||
executedCount = executedCount + randomInt(1000);
|
||||
break;
|
||||
default:
|
||||
throw new AssertionError("Illegal randomisation branch");
|
||||
List<MockStep> steps = new ArrayList<>(instance.getSteps());
|
||||
MockStep lastStep = steps.remove(steps.size() - 1);
|
||||
if (randomBoolean()) {
|
||||
Step.StepKey additionalStepKey = randomStepKey();
|
||||
steps.add(new MockStep(lastStep.getKey(), additionalStepKey));
|
||||
steps.add(new MockStep(additionalStepKey, null));
|
||||
}
|
||||
return new MockAction(null, completed, executedCount);
|
||||
return new MockAction(steps);
|
||||
}
|
||||
|
||||
// public void testExecuteNotComplete() {
|
||||
//
|
||||
// MockAction action = new MockAction();
|
||||
// action.setCompleteOnExecute(false);
|
||||
//
|
||||
// assertFalse(action.wasCompleted());
|
||||
// assertEquals(0L, action.getExecutedCount());
|
||||
//
|
||||
// SetOnce<Boolean> listenerCalled = new SetOnce<>();
|
||||
//
|
||||
// action.execute(null, null, null, new LifecycleAction.Listener() {
|
||||
//
|
||||
// @Override
|
||||
// public void onSuccess(boolean completed) {
|
||||
// listenerCalled.set(true);
|
||||
// }
|
||||
//
|
||||
// @Override
|
||||
// public void onFailure(Exception e) {
|
||||
// throw new AssertionError("Unexpected method call", e);
|
||||
// }
|
||||
// });
|
||||
//
|
||||
// assertFalse(action.wasCompleted());
|
||||
// assertEquals(1L, action.getExecutedCount());
|
||||
// assertEquals(true, listenerCalled.get());
|
||||
// }
|
||||
//
|
||||
// public void testExecuteComplete() {
|
||||
//
|
||||
// MockAction action = new MockAction();
|
||||
// action.setCompleteOnExecute(true);
|
||||
//
|
||||
// assertFalse(action.wasCompleted());
|
||||
// assertEquals(0L, action.getExecutedCount());
|
||||
//
|
||||
// SetOnce<Boolean> listenerCalled = new SetOnce<>();
|
||||
//
|
||||
// action.execute(null, null, null, new LifecycleAction.Listener() {
|
||||
//
|
||||
// @Override
|
||||
// public void onSuccess(boolean completed) {
|
||||
// listenerCalled.set(true);
|
||||
// }
|
||||
//
|
||||
// @Override
|
||||
// public void onFailure(Exception e) {
|
||||
// throw new AssertionError("Unexpected method call", e);
|
||||
// }
|
||||
// });
|
||||
//
|
||||
// assertTrue(action.wasCompleted());
|
||||
// assertEquals(1L, action.getExecutedCount());
|
||||
// assertEquals(true, listenerCalled.get());
|
||||
// }
|
||||
//
|
||||
// public void testResetComplete() {
|
||||
//
|
||||
// MockAction action = new MockAction();
|
||||
// action.setCompleteOnExecute(true);
|
||||
//
|
||||
// assertFalse(action.wasCompleted());
|
||||
// assertEquals(0L, action.getExecutedCount());
|
||||
//
|
||||
// SetOnce<Boolean> listenerCalled = new SetOnce<>();
|
||||
//
|
||||
// action.execute(null, null, null, new LifecycleAction.Listener() {
|
||||
//
|
||||
// @Override
|
||||
// public void onSuccess(boolean completed) {
|
||||
// listenerCalled.set(true);
|
||||
// }
|
||||
//
|
||||
// @Override
|
||||
// public void onFailure(Exception e) {
|
||||
// throw new AssertionError("Unexpected method call", e);
|
||||
// }
|
||||
// });
|
||||
//
|
||||
// assertTrue(action.wasCompleted());
|
||||
// assertEquals(1L, action.getExecutedCount());
|
||||
// assertEquals(true, listenerCalled.get());
|
||||
//
|
||||
// action.resetCompleted();
|
||||
//
|
||||
// assertFalse(action.wasCompleted());
|
||||
//
|
||||
// SetOnce<Boolean> listenerCalled2 = new SetOnce<>();
|
||||
//
|
||||
// action.execute(null, null, null, new LifecycleAction.Listener() {
|
||||
//
|
||||
// @Override
|
||||
// public void onSuccess(boolean completed) {
|
||||
// listenerCalled2.set(true);
|
||||
// }
|
||||
//
|
||||
// @Override
|
||||
// public void onFailure(Exception e) {
|
||||
// throw new AssertionError("Unexpected method call", e);
|
||||
// }
|
||||
// });
|
||||
//
|
||||
// assertTrue(action.wasCompleted());
|
||||
// assertEquals(2L, action.getExecutedCount());
|
||||
// assertEquals(true, listenerCalled2.get());
|
||||
// }
|
||||
//
|
||||
// public void testExecuteFailure() {
|
||||
// Exception exception = new RuntimeException();
|
||||
//
|
||||
// MockAction action = new MockAction();
|
||||
// action.setCompleteOnExecute(randomBoolean());
|
||||
// action.setExceptionToThrow(exception);
|
||||
//
|
||||
// assertFalse(action.wasCompleted());
|
||||
// assertEquals(0L, action.getExecutedCount());
|
||||
//
|
||||
// SetOnce<Boolean> listenerCalled = new SetOnce<>();
|
||||
//
|
||||
// action.execute(null, null, null, new LifecycleAction.Listener() {
|
||||
//
|
||||
// @Override
|
||||
// public void onSuccess(boolean completed) {
|
||||
// throw new AssertionError("Unexpected method call");
|
||||
// }
|
||||
//
|
||||
// @Override
|
||||
// public void onFailure(Exception e) {
|
||||
// assertSame(exception, e);
|
||||
// listenerCalled.set(true);
|
||||
// }
|
||||
// });
|
||||
//
|
||||
// assertFalse(action.wasCompleted());
|
||||
// assertEquals(1L, action.getExecutedCount());
|
||||
// assertEquals(true, listenerCalled.get());
|
||||
// }
|
||||
|
||||
private Step.StepKey randomStepKey() {
|
||||
return new Step.StepKey(randomAlphaOfLength(5),
|
||||
randomAlphaOfLength(5), randomAlphaOfLength(5));
|
||||
}
|
||||
}
|
||||
|
||||
|
|
|
@ -0,0 +1,60 @@
|
|||
/*
|
||||
* Copyright Elasticsearch B.V. and/or licensed to Elasticsearch B.V. under one
|
||||
* or more contributor license agreements. Licensed under the Elastic License;
|
||||
* you may not use this file except in compliance with the Elastic License.
|
||||
*/
|
||||
package org.elasticsearch.xpack.core.indexlifecycle;
|
||||
|
||||
import org.elasticsearch.common.io.stream.StreamInput;
|
||||
import org.elasticsearch.common.io.stream.StreamOutput;
|
||||
import org.elasticsearch.common.io.stream.Writeable;
|
||||
import org.elasticsearch.common.xcontent.ToXContent;
|
||||
import org.elasticsearch.common.xcontent.XContentBuilder;
|
||||
|
||||
import java.io.IOException;
|
||||
|
||||
public class MockStep extends Step implements ToXContent, Writeable {
|
||||
|
||||
public MockStep(StepKey stepKey, Step.StepKey nextStepKey) {
|
||||
super(stepKey, nextStepKey);
|
||||
}
|
||||
|
||||
public MockStep(StreamInput in) throws IOException {
|
||||
super(new StepKey(in.readString(), in.readString(), in.readString()),
|
||||
new StepKey(in.readString(), in.readString(), in.readString()));
|
||||
}
|
||||
|
||||
@Override
|
||||
public void writeTo(StreamOutput out) throws IOException {
|
||||
out.writeString(getKey().getPhase());
|
||||
out.writeString(getKey().getAction());
|
||||
out.writeString(getKey().getName());
|
||||
out.writeString(getNextStepKey().getPhase());
|
||||
out.writeString(getNextStepKey().getAction());
|
||||
out.writeString(getNextStepKey().getName());
|
||||
}
|
||||
|
||||
@Override
|
||||
public XContentBuilder toXContent(XContentBuilder builder, Params params) throws IOException {
|
||||
builder.startObject();
|
||||
{
|
||||
builder.startObject("step_key");
|
||||
{
|
||||
builder.field("phase", getKey().getPhase());
|
||||
builder.field("action", getKey().getAction());
|
||||
builder.field("name", getKey().getName());
|
||||
}
|
||||
builder.endObject();
|
||||
builder.startObject("next_step_key");
|
||||
{
|
||||
builder.field("phase", getNextStepKey().getPhase());
|
||||
builder.field("action", getNextStepKey().getAction());
|
||||
builder.field("name", getNextStepKey().getName());
|
||||
}
|
||||
builder.endObject();
|
||||
}
|
||||
builder.endObject();
|
||||
|
||||
return builder;
|
||||
}
|
||||
}
|
|
@ -35,34 +35,6 @@ public class TestLifecycleType implements LifecycleType {
|
|||
return TYPE;
|
||||
}
|
||||
|
||||
// @Override
|
||||
// public NextActionProvider getActionProvider(IndexLifecycleContext context, Phase phase) {
|
||||
// return a -> Optional.ofNullable(phase.getActions().entrySet().iterator().next()).map(Map.Entry::getValue).orElse(null);
|
||||
// }
|
||||
|
||||
// @Override
|
||||
// public Phase getFirstPhase(Map<String, Phase> phases) {
|
||||
// return phases.values().iterator().next();
|
||||
// }
|
||||
//
|
||||
// @Override
|
||||
// public Phase nextPhase(Map<String, Phase> phases, @Nullable Phase currentPhase) {
|
||||
// if (currentPhase == null) {
|
||||
// return getFirstPhase(phases);
|
||||
// }
|
||||
//
|
||||
// boolean foundPhase = false;
|
||||
// for (Phase phase : phases.values()) {
|
||||
// if (foundPhase) {
|
||||
// return phase;
|
||||
// } else if (phase.equals(currentPhase)) {
|
||||
// foundPhase = true;
|
||||
// }
|
||||
// }
|
||||
//
|
||||
// return null;
|
||||
// }
|
||||
|
||||
@Override
|
||||
public void validate(Collection<Phase> phases) {
|
||||
// always valid
|
||||
|
|
|
@ -5,6 +5,7 @@
|
|||
*/
|
||||
package org.elasticsearch.xpack.core.indexlifecycle;
|
||||
|
||||
import org.elasticsearch.common.io.stream.NamedWriteable;
|
||||
import org.elasticsearch.common.unit.ByteSizeValue;
|
||||
import org.elasticsearch.common.unit.TimeValue;
|
||||
import org.elasticsearch.test.ESTestCase;
|
||||
|
@ -27,8 +28,13 @@ import java.util.List;
|
|||
import java.util.Map;
|
||||
import java.util.function.Function;
|
||||
import java.util.function.LongSupplier;
|
||||
import java.util.function.Supplier;
|
||||
import java.util.stream.Collectors;
|
||||
|
||||
import static org.elasticsearch.xpack.core.indexlifecycle.TimeseriesLifecycleType.ORDERED_VALID_COLD_ACTIONS;
|
||||
import static org.elasticsearch.xpack.core.indexlifecycle.TimeseriesLifecycleType.ORDERED_VALID_DELETE_ACTIONS;
|
||||
import static org.elasticsearch.xpack.core.indexlifecycle.TimeseriesLifecycleType.ORDERED_VALID_HOT_ACTIONS;
|
||||
import static org.elasticsearch.xpack.core.indexlifecycle.TimeseriesLifecycleType.ORDERED_VALID_WARM_ACTIONS;
|
||||
import static org.elasticsearch.xpack.core.indexlifecycle.TimeseriesLifecycleType.VALID_COLD_ACTIONS;
|
||||
import static org.elasticsearch.xpack.core.indexlifecycle.TimeseriesLifecycleType.VALID_DELETE_ACTIONS;
|
||||
import static org.elasticsearch.xpack.core.indexlifecycle.TimeseriesLifecycleType.VALID_HOT_ACTIONS;
|
||||
|
@ -45,363 +51,201 @@ public class TimeseriesLifecycleTypeTests extends ESTestCase {
|
|||
private static final ReplicasAction TEST_REPLICAS_ACTION = new ReplicasAction(1);
|
||||
private static final RolloverAction TEST_ROLLOVER_ACTION = new RolloverAction("", new ByteSizeValue(1), null, null);
|
||||
private static final ShrinkAction TEST_SHRINK_ACTION = new ShrinkAction(1);
|
||||
private static final LongSupplier TEST_NOW_SUPPLIER = () -> 0L;
|
||||
|
||||
public void testStub() {
|
||||
private static final ReadOnlyAction TEST_READ_ONLY_ACTION = new ReadOnlyAction();
|
||||
|
||||
public void testValidatePhases() {
|
||||
boolean invalid = randomBoolean();
|
||||
String phaseName = randomFrom("hot", "warm", "cold", "delete");
|
||||
if (invalid) {
|
||||
phaseName += randomAlphaOfLength(5);
|
||||
}
|
||||
Map<String, Phase> phases = Collections.singletonMap(phaseName,
|
||||
new Phase(phaseName, TimeValue.ZERO, Collections.emptyMap()));
|
||||
if (invalid) {
|
||||
Exception e = expectThrows(IllegalArgumentException.class, () -> TimeseriesLifecycleType.INSTANCE.validate(phases.values()));
|
||||
assertThat(e.getMessage(), equalTo("Timeseries lifecycle does not support phase [" + phaseName + "]"));
|
||||
} else {
|
||||
TimeseriesLifecycleType.INSTANCE.validate(phases.values());
|
||||
}
|
||||
}
|
||||
|
||||
// public void testGetFirstPhase() {
|
||||
// Map<String, Phase> phases = new HashMap<>();
|
||||
// Phase expectedFirstPhase = null;
|
||||
// for (String phaseName : Arrays.asList("hot", "warm", "cold", "delete")) {
|
||||
// if (randomBoolean()) {
|
||||
// Phase phase = new Phase(phaseName, TimeValue.MINUS_ONE, Collections.emptyMap());
|
||||
// phases.put(phaseName, phase);
|
||||
// if (expectedFirstPhase == null) {
|
||||
// expectedFirstPhase = phase;
|
||||
// }
|
||||
// }
|
||||
// }
|
||||
// TimeseriesLifecycleType policy = TimeseriesLifecycleType.INSTANCE;
|
||||
// assertThat(policy.getFirstPhase(phases), equalTo(expectedFirstPhase));
|
||||
// }
|
||||
//
|
||||
// public void testGetNextPhase() {
|
||||
// for (int runs = 0; runs < 20; runs++) {
|
||||
// Map<String, Phase> phases = new HashMap<>();
|
||||
// List<Phase> phasesInOrder = new ArrayList<>();
|
||||
// for (String phase : VALID_PHASES) {
|
||||
// if (randomBoolean()) {
|
||||
// Phase phaseToAdd = new Phase(phase, TimeValue.MINUS_ONE, Collections.emptyMap());
|
||||
// phases.put(phase, phaseToAdd);
|
||||
// phasesInOrder.add(phaseToAdd);
|
||||
// }
|
||||
// }
|
||||
// TimeseriesLifecycleType policy = TimeseriesLifecycleType.INSTANCE;
|
||||
// assertThat(policy.nextPhase(phases, null), equalTo(policy.getFirstPhase(phases)));
|
||||
// for (int i = 0; i < phasesInOrder.size() - 1; i++) {
|
||||
// assertThat(policy.nextPhase(phases, phasesInOrder.get(i)), equalTo(phasesInOrder.get(i + 1)));
|
||||
// }
|
||||
// if (phasesInOrder.isEmpty() == false) {
|
||||
// assertNull(policy.nextPhase(phases, phasesInOrder.get(phasesInOrder.size() - 1)));
|
||||
// }
|
||||
// }
|
||||
// }
|
||||
//
|
||||
// public void testValidatePhases() {
|
||||
// boolean invalid = randomBoolean();
|
||||
// String phaseName = randomFrom("hot", "warm", "cold", "delete");
|
||||
// if (invalid) {
|
||||
// phaseName += randomAlphaOfLength(5);
|
||||
// }
|
||||
// Map<String, Phase> phases = Collections.singletonMap(phaseName,
|
||||
// new Phase(phaseName, TimeValue.ZERO, Collections.emptyMap()));
|
||||
// if (invalid) {
|
||||
// Exception e = expectThrows(IllegalArgumentException.class, () -> TimeseriesLifecycleType.INSTANCE.validate(phases.values()));
|
||||
// assertThat(e.getMessage(), equalTo("Timeseries lifecycle does not support phase [" + phaseName + "]"));
|
||||
// } else {
|
||||
// TimeseriesLifecycleType.INSTANCE.validate(phases.values());
|
||||
// }
|
||||
// }
|
||||
//
|
||||
// public void testValidateHotPhase() {
|
||||
// LifecycleAction invalidAction = null;
|
||||
// Map<String, LifecycleAction> actions = VALID_HOT_ACTIONS
|
||||
// .stream().map(this::getTestAction).collect(Collectors.toMap(LifecycleAction::getWriteableName, Function.identity()));
|
||||
// if (randomBoolean()) {
|
||||
// invalidAction = getTestAction(randomFrom("allocate", "forcemerge", "delete", "replicas", "shrink"));
|
||||
// actions.put(invalidAction.getWriteableName(), invalidAction);
|
||||
// }
|
||||
// Map<String, Phase> hotPhase = Collections.singletonMap("hot",
|
||||
// new Phase("hot", TimeValue.ZERO, actions));
|
||||
//
|
||||
// if (invalidAction != null) {
|
||||
// Exception e = expectThrows(IllegalArgumentException.class,
|
||||
// () -> TimeseriesLifecycleType.INSTANCE.validate(hotPhase.values()));
|
||||
// assertThat(e.getMessage(),
|
||||
// equalTo("invalid action [" + invalidAction.getWriteableName() + "] defined in phase [hot]"));
|
||||
// } else {
|
||||
// TimeseriesLifecycleType.INSTANCE.validate(hotPhase.values());
|
||||
// }
|
||||
// }
|
||||
//
|
||||
// public void testValidateWarmPhase() {
|
||||
// LifecycleAction invalidAction = null;
|
||||
// Map<String, LifecycleAction> actions = randomSubsetOf(VALID_WARM_ACTIONS)
|
||||
// .stream().map(this::getTestAction).collect(Collectors.toMap(LifecycleAction::getWriteableName, Function.identity()));
|
||||
// if (randomBoolean()) {
|
||||
// invalidAction = getTestAction(randomFrom("rollover", "delete"));
|
||||
// actions.put(invalidAction.getWriteableName(), invalidAction);
|
||||
// }
|
||||
// Map<String, Phase> warmPhase = Collections.singletonMap("warm",
|
||||
// new Phase("warm", TimeValue.ZERO, actions));
|
||||
//
|
||||
// if (invalidAction != null) {
|
||||
// Exception e = expectThrows(IllegalArgumentException.class,
|
||||
// () -> TimeseriesLifecycleType.INSTANCE.validate(warmPhase.values()));
|
||||
// assertThat(e.getMessage(),
|
||||
// equalTo("invalid action [" + invalidAction.getWriteableName() + "] defined in phase [warm]"));
|
||||
// } else {
|
||||
// TimeseriesLifecycleType.INSTANCE.validate(warmPhase.values());
|
||||
// }
|
||||
// }
|
||||
//
|
||||
// public void testValidateColdPhase() {
|
||||
// LifecycleAction invalidAction = null;
|
||||
// Map<String, LifecycleAction> actions = randomSubsetOf(VALID_COLD_ACTIONS)
|
||||
// .stream().map(this::getTestAction).collect(Collectors.toMap(LifecycleAction::getWriteableName, Function.identity()));
|
||||
// if (randomBoolean()) {
|
||||
// invalidAction = getTestAction(randomFrom("rollover", "delete", "forcemerge", "shrink"));
|
||||
// actions.put(invalidAction.getWriteableName(), invalidAction);
|
||||
// }
|
||||
// Map<String, Phase> coldPhase = Collections.singletonMap("cold",
|
||||
// new Phase("cold", TimeValue.ZERO, actions));
|
||||
//
|
||||
// if (invalidAction != null) {
|
||||
// Exception e = expectThrows(IllegalArgumentException.class,
|
||||
// () -> TimeseriesLifecycleType.INSTANCE.validate(coldPhase.values()));
|
||||
// assertThat(e.getMessage(),
|
||||
// equalTo("invalid action [" + invalidAction.getWriteableName() + "] defined in phase [cold]"));
|
||||
// } else {
|
||||
// TimeseriesLifecycleType.INSTANCE.validate(coldPhase.values());
|
||||
// }
|
||||
// }
|
||||
//
|
||||
// public void testValidateDeletePhase() {
|
||||
// LifecycleAction invalidAction = null;
|
||||
// Map<String, LifecycleAction> actions = VALID_DELETE_ACTIONS
|
||||
// .stream().map(this::getTestAction).collect(Collectors.toMap(LifecycleAction::getWriteableName, Function.identity()));
|
||||
// if (randomBoolean()) {
|
||||
// invalidAction = getTestAction(randomFrom("allocate", "rollover", "replicas", "forcemerge", "shrink"));
|
||||
// actions.put(invalidAction.getWriteableName(), invalidAction);
|
||||
// }
|
||||
// Map<String, Phase> deletePhase = Collections.singletonMap("delete",
|
||||
// new Phase("delete", TimeValue.ZERO, actions));
|
||||
//
|
||||
// if (invalidAction != null) {
|
||||
// Exception e = expectThrows(IllegalArgumentException.class,
|
||||
// () -> TimeseriesLifecycleType.INSTANCE.validate(deletePhase.values()));
|
||||
// assertThat(e.getMessage(),
|
||||
// equalTo("invalid action [" + invalidAction.getWriteableName() + "] defined in phase [delete]"));
|
||||
// } else {
|
||||
// TimeseriesLifecycleType.INSTANCE.validate(deletePhase.values());
|
||||
// }
|
||||
// }
|
||||
//
|
||||
// public void testHotActionProvider() {
|
||||
// String indexName = randomAlphaOfLengthBetween(1, 10);
|
||||
// Map<String, LifecycleAction> actions = VALID_HOT_ACTIONS
|
||||
// .stream().map(this::getTestAction).collect(Collectors.toMap(LifecycleAction::getWriteableName, Function.identity()));
|
||||
// Phase hotPhase = new Phase("hot", TimeValue.ZERO, actions);
|
||||
// MockIndexLifecycleContext context = new MockIndexLifecycleContext(indexName, "", "",
|
||||
// 0, TEST_NOW_SUPPLIER) {
|
||||
//
|
||||
// @Override
|
||||
// public boolean canExecute(Phase phase) {
|
||||
// assertSame(hotPhase, phase);
|
||||
// return true;
|
||||
// }
|
||||
// };
|
||||
// TimeseriesLifecycleType policy = TimeseriesLifecycleType.INSTANCE;
|
||||
// LifecyclePolicy.NextActionProvider provider = policy.getActionProvider(context, hotPhase);
|
||||
// assertThat(provider.next(null), equalTo(TEST_ROLLOVER_ACTION));
|
||||
// assertNull(provider.next(TEST_ROLLOVER_ACTION));
|
||||
// }
|
||||
//
|
||||
// public void testWarmActionProviderWithAllActionsAndReplicasFirst() {
|
||||
// String indexName = randomAlphaOfLengthBetween(1, 10);
|
||||
// Map<String, LifecycleAction> actions = VALID_WARM_ACTIONS
|
||||
// .stream().map(this::getTestAction).collect(Collectors.toMap(LifecycleAction::getWriteableName, Function.identity()));
|
||||
// actions.put(ReplicasAction.NAME, TEST_REPLICAS_ACTION);
|
||||
// Phase warmPhase = new Phase("warm", TimeValue.ZERO, actions);
|
||||
// MockIndexLifecycleContext context =new MockIndexLifecycleContext(indexName, "", "",
|
||||
// TEST_REPLICAS_ACTION.getNumberOfReplicas() + 1, TEST_NOW_SUPPLIER) {
|
||||
//
|
||||
// @Override
|
||||
// public boolean canExecute(Phase phase) {
|
||||
// assertSame(warmPhase, phase);
|
||||
// return true;
|
||||
// }
|
||||
// };
|
||||
// TimeseriesLifecycleType policy = TimeseriesLifecycleType.INSTANCE;
|
||||
// LifecyclePolicy.NextActionProvider provider = policy.getActionProvider(context, warmPhase);
|
||||
// if (actions.size() > 1) {
|
||||
// int actionCount = 1;
|
||||
// LifecycleAction current = provider.next(null);
|
||||
// assertThat(current, equalTo(TEST_REPLICAS_ACTION));
|
||||
// while (actionCount++ < actions.size()) {
|
||||
// current = provider.next(current);
|
||||
// }
|
||||
// assertNull(provider.next(current));
|
||||
// assertThat(current, equalTo(TEST_FORCE_MERGE_ACTION));
|
||||
// } else {
|
||||
// assertThat(provider.next(null), equalTo(TEST_REPLICAS_ACTION));
|
||||
// }
|
||||
//
|
||||
// }
|
||||
//
|
||||
// public void testWarmActionProviderReplicasActionSortOrder() {
|
||||
// String indexName = randomAlphaOfLengthBetween(1, 10);
|
||||
// Map<String, LifecycleAction> actions = randomSubsetOf(VALID_WARM_ACTIONS)
|
||||
// .stream().map(this::getTestAction).collect(Collectors.toMap(LifecycleAction::getWriteableName, Function.identity()));
|
||||
// actions.put(ReplicasAction.NAME, TEST_REPLICAS_ACTION);
|
||||
// Phase warmPhase = new Phase("warm", TimeValue.ZERO, actions);
|
||||
// MockIndexLifecycleContext context =new MockIndexLifecycleContext(indexName, "", "",
|
||||
// TEST_REPLICAS_ACTION.getNumberOfReplicas() + 1, TEST_NOW_SUPPLIER) {
|
||||
//
|
||||
// @Override
|
||||
// public boolean canExecute(Phase phase) {
|
||||
// assertSame(warmPhase, phase);
|
||||
// return true;
|
||||
// }
|
||||
// };
|
||||
// TimeseriesLifecycleType policy = TimeseriesLifecycleType.INSTANCE;
|
||||
// LifecyclePolicy.NextActionProvider provider = policy.getActionProvider(context, warmPhase);
|
||||
// assertThat(provider.next(null), equalTo(TEST_REPLICAS_ACTION));
|
||||
// context = new MockIndexLifecycleContext(indexName, "", "",
|
||||
// TEST_REPLICAS_ACTION.getNumberOfReplicas() - 1, TEST_NOW_SUPPLIER) {
|
||||
//
|
||||
// @Override
|
||||
// public boolean canExecute(Phase phase) {
|
||||
// assertSame(warmPhase, phase);
|
||||
// return true;
|
||||
// }
|
||||
// };
|
||||
// provider = policy.getActionProvider(context, warmPhase);
|
||||
// if (actions.size() > 1) {
|
||||
// int actionCount = 1;
|
||||
// LifecycleAction current = provider.next(null);
|
||||
// assertThat(current, not(equalTo(TEST_REPLICAS_ACTION)));
|
||||
// while (actionCount++ < actions.size()) {
|
||||
// current = provider.next(current);
|
||||
// }
|
||||
// assertNull(provider.next(current));
|
||||
// assertThat(current, equalTo(TEST_REPLICAS_ACTION));
|
||||
// } else {
|
||||
// assertThat(provider.next(null), equalTo(TEST_REPLICAS_ACTION));
|
||||
// }
|
||||
// }
|
||||
//
|
||||
// public void testColdActionProviderAllActions() {
|
||||
// String indexName = randomAlphaOfLengthBetween(1, 10);
|
||||
// Map<String, LifecycleAction> actions = VALID_COLD_ACTIONS
|
||||
// .stream().map(this::getTestAction).collect(Collectors.toMap(LifecycleAction::getWriteableName, Function.identity()));
|
||||
// actions.put(ReplicasAction.NAME, TEST_REPLICAS_ACTION);
|
||||
// Phase coldPhase = new Phase("cold", TimeValue.ZERO, actions);
|
||||
// MockIndexLifecycleContext context =new MockIndexLifecycleContext(indexName, "", "",
|
||||
// TEST_REPLICAS_ACTION.getNumberOfReplicas() - 1, TEST_NOW_SUPPLIER) {
|
||||
//
|
||||
// @Override
|
||||
// public boolean canExecute(Phase phase) {
|
||||
// assertSame(coldPhase, phase);
|
||||
// return true;
|
||||
// }
|
||||
// };
|
||||
// TimeseriesLifecycleType policy = TimeseriesLifecycleType.INSTANCE;
|
||||
// LifecyclePolicy.NextActionProvider provider = policy.getActionProvider(context, coldPhase);
|
||||
// if (actions.size() > 1) {
|
||||
// LifecycleAction current = provider.next(null);
|
||||
// assertThat(current, equalTo(TEST_ALLOCATE_ACTION));
|
||||
// assertThat(provider.next(current), equalTo(TEST_REPLICAS_ACTION));
|
||||
// } else {
|
||||
// assertThat(provider.next(null), equalTo(TEST_REPLICAS_ACTION));
|
||||
// }
|
||||
//
|
||||
// context = new MockIndexLifecycleContext(indexName, "", "",
|
||||
// TEST_REPLICAS_ACTION.getNumberOfReplicas() + 1, TEST_NOW_SUPPLIER) {
|
||||
//
|
||||
// @Override
|
||||
// public boolean canExecute(Phase phase) {
|
||||
// assertSame(coldPhase, phase);
|
||||
// return true;
|
||||
// }
|
||||
// };
|
||||
// provider = policy.getActionProvider(context, coldPhase);
|
||||
// if (actions.size() > 1) {
|
||||
// LifecycleAction current = provider.next(null);
|
||||
// assertThat(current, equalTo(TEST_REPLICAS_ACTION));
|
||||
// assertThat(provider.next(current), equalTo(TEST_ALLOCATE_ACTION));
|
||||
// } else {
|
||||
// assertThat(provider.next(null), equalTo(TEST_REPLICAS_ACTION));
|
||||
// }
|
||||
// }
|
||||
//
|
||||
// public void testColdActionProviderReplicasActionSortOrder() {
|
||||
// String indexName = randomAlphaOfLengthBetween(1, 10);
|
||||
// Map<String, LifecycleAction> actions = randomSubsetOf(VALID_COLD_ACTIONS)
|
||||
// .stream().map(this::getTestAction).collect(Collectors.toMap(LifecycleAction::getWriteableName, Function.identity()));
|
||||
// actions.put(ReplicasAction.NAME, TEST_REPLICAS_ACTION);
|
||||
// Phase coldPhase = new Phase("cold", TimeValue.ZERO, actions);
|
||||
// MockIndexLifecycleContext context =new MockIndexLifecycleContext(indexName, "", "",
|
||||
// TEST_REPLICAS_ACTION.getNumberOfReplicas() + 1, TEST_NOW_SUPPLIER) {
|
||||
//
|
||||
// @Override
|
||||
// public boolean canExecute(Phase phase) {
|
||||
// assertSame(coldPhase, phase);
|
||||
// return true;
|
||||
// }
|
||||
// };
|
||||
// TimeseriesLifecycleType policy = TimeseriesLifecycleType.INSTANCE;
|
||||
// LifecyclePolicy.NextActionProvider provider = policy.getActionProvider(context, coldPhase);
|
||||
// assertThat(provider.next(null), equalTo(TEST_REPLICAS_ACTION));
|
||||
// context = new MockIndexLifecycleContext(indexName, "", "",
|
||||
// TEST_REPLICAS_ACTION.getNumberOfReplicas() - 1, TEST_NOW_SUPPLIER) {
|
||||
//
|
||||
// @Override
|
||||
// public boolean canExecute(Phase phase) {
|
||||
// assertSame(coldPhase, phase);
|
||||
// return true;
|
||||
// }
|
||||
// };
|
||||
// provider = policy.getActionProvider(context, coldPhase);
|
||||
// if (actions.size() > 1) {
|
||||
// LifecycleAction current = provider.next(null);
|
||||
// assertThat(current, equalTo(TEST_ALLOCATE_ACTION));
|
||||
// assertThat(provider.next(current), equalTo(TEST_REPLICAS_ACTION));
|
||||
// } else {
|
||||
// assertThat(provider.next(null), equalTo(TEST_REPLICAS_ACTION));
|
||||
// }
|
||||
// }
|
||||
//
|
||||
// public void testDeleteActionProvider() {
|
||||
// String indexName = randomAlphaOfLengthBetween(1, 10);
|
||||
// Map<String, LifecycleAction> actions = VALID_DELETE_ACTIONS
|
||||
// .stream().map(this::getTestAction).collect(Collectors.toMap(LifecycleAction::getWriteableName, Function.identity()));
|
||||
// Phase deletePhase = new Phase("delete", TimeValue.ZERO, actions);
|
||||
//
|
||||
// MockIndexLifecycleContext context = new MockIndexLifecycleContext(indexName, "", "",
|
||||
// 0, TEST_NOW_SUPPLIER) {
|
||||
//
|
||||
// @Override
|
||||
// public boolean canExecute(Phase phase) {
|
||||
// assertSame(deletePhase, phase);
|
||||
// return true;
|
||||
// }
|
||||
// };
|
||||
// TimeseriesLifecycleType policy = TimeseriesLifecycleType.INSTANCE;
|
||||
// LifecyclePolicy.NextActionProvider provider = policy.getActionProvider(context, deletePhase);
|
||||
// assertThat(provider.next(null), equalTo(TEST_DELETE_ACTION));
|
||||
// assertNull(provider.next(TEST_DELETE_ACTION));
|
||||
// }
|
||||
//
|
||||
//
|
||||
// private LifecycleAction getTestAction(String actionName) {
|
||||
// switch (actionName) {
|
||||
// case AllocateAction.NAME:
|
||||
// return TEST_ALLOCATE_ACTION;
|
||||
// case DeleteAction.NAME:
|
||||
// return TEST_DELETE_ACTION;
|
||||
// case ForceMergeAction.NAME:
|
||||
// return TEST_FORCE_MERGE_ACTION;
|
||||
// case ReplicasAction.NAME:
|
||||
// return TEST_REPLICAS_ACTION;
|
||||
// case RolloverAction.NAME:
|
||||
// return TEST_ROLLOVER_ACTION;
|
||||
// case ShrinkAction.NAME:
|
||||
// return TEST_SHRINK_ACTION;
|
||||
// default:
|
||||
// throw new IllegalArgumentException("unsupported timeseries phase action [" + actionName + "]");
|
||||
// }
|
||||
// }
|
||||
public void testValidateHotPhase() {
|
||||
LifecycleAction invalidAction = null;
|
||||
Map<String, LifecycleAction> actions = VALID_HOT_ACTIONS
|
||||
.stream().map(this::getTestAction).collect(Collectors.toMap(LifecycleAction::getWriteableName, Function.identity()));
|
||||
if (randomBoolean()) {
|
||||
invalidAction = getTestAction(randomFrom("allocate", "forcemerge", "delete", "replicas", "shrink"));
|
||||
actions.put(invalidAction.getWriteableName(), invalidAction);
|
||||
}
|
||||
Map<String, Phase> hotPhase = Collections.singletonMap("hot",
|
||||
new Phase("hot", TimeValue.ZERO, actions));
|
||||
|
||||
if (invalidAction != null) {
|
||||
Exception e = expectThrows(IllegalArgumentException.class,
|
||||
() -> TimeseriesLifecycleType.INSTANCE.validate(hotPhase.values()));
|
||||
assertThat(e.getMessage(),
|
||||
equalTo("invalid action [" + invalidAction.getWriteableName() + "] defined in phase [hot]"));
|
||||
} else {
|
||||
TimeseriesLifecycleType.INSTANCE.validate(hotPhase.values());
|
||||
}
|
||||
}
|
||||
|
||||
public void testValidateWarmPhase() {
|
||||
LifecycleAction invalidAction = null;
|
||||
Map<String, LifecycleAction> actions = randomSubsetOf(VALID_WARM_ACTIONS)
|
||||
.stream().map(this::getTestAction).collect(Collectors.toMap(LifecycleAction::getWriteableName, Function.identity()));
|
||||
if (randomBoolean()) {
|
||||
invalidAction = getTestAction(randomFrom("rollover", "delete"));
|
||||
actions.put(invalidAction.getWriteableName(), invalidAction);
|
||||
}
|
||||
Map<String, Phase> warmPhase = Collections.singletonMap("warm",
|
||||
new Phase("warm", TimeValue.ZERO, actions));
|
||||
|
||||
if (invalidAction != null) {
|
||||
Exception e = expectThrows(IllegalArgumentException.class,
|
||||
() -> TimeseriesLifecycleType.INSTANCE.validate(warmPhase.values()));
|
||||
assertThat(e.getMessage(),
|
||||
equalTo("invalid action [" + invalidAction.getWriteableName() + "] defined in phase [warm]"));
|
||||
} else {
|
||||
TimeseriesLifecycleType.INSTANCE.validate(warmPhase.values());
|
||||
}
|
||||
}
|
||||
|
||||
public void testValidateColdPhase() {
|
||||
LifecycleAction invalidAction = null;
|
||||
Map<String, LifecycleAction> actions = randomSubsetOf(VALID_COLD_ACTIONS)
|
||||
.stream().map(this::getTestAction).collect(Collectors.toMap(LifecycleAction::getWriteableName, Function.identity()));
|
||||
if (randomBoolean()) {
|
||||
invalidAction = getTestAction(randomFrom("rollover", "delete", "forcemerge", "shrink"));
|
||||
actions.put(invalidAction.getWriteableName(), invalidAction);
|
||||
}
|
||||
Map<String, Phase> coldPhase = Collections.singletonMap("cold",
|
||||
new Phase("cold", TimeValue.ZERO, actions));
|
||||
|
||||
if (invalidAction != null) {
|
||||
Exception e = expectThrows(IllegalArgumentException.class,
|
||||
() -> TimeseriesLifecycleType.INSTANCE.validate(coldPhase.values()));
|
||||
assertThat(e.getMessage(),
|
||||
equalTo("invalid action [" + invalidAction.getWriteableName() + "] defined in phase [cold]"));
|
||||
} else {
|
||||
TimeseriesLifecycleType.INSTANCE.validate(coldPhase.values());
|
||||
}
|
||||
}
|
||||
|
||||
public void testValidateDeletePhase() {
|
||||
LifecycleAction invalidAction = null;
|
||||
Map<String, LifecycleAction> actions = VALID_DELETE_ACTIONS
|
||||
.stream().map(this::getTestAction).collect(Collectors.toMap(LifecycleAction::getWriteableName, Function.identity()));
|
||||
if (randomBoolean()) {
|
||||
invalidAction = getTestAction(randomFrom("allocate", "rollover", "replicas", "forcemerge", "shrink"));
|
||||
actions.put(invalidAction.getWriteableName(), invalidAction);
|
||||
}
|
||||
Map<String, Phase> deletePhase = Collections.singletonMap("delete",
|
||||
new Phase("delete", TimeValue.ZERO, actions));
|
||||
|
||||
if (invalidAction != null) {
|
||||
Exception e = expectThrows(IllegalArgumentException.class,
|
||||
() -> TimeseriesLifecycleType.INSTANCE.validate(deletePhase.values()));
|
||||
assertThat(e.getMessage(),
|
||||
equalTo("invalid action [" + invalidAction.getWriteableName() + "] defined in phase [delete]"));
|
||||
} else {
|
||||
TimeseriesLifecycleType.INSTANCE.validate(deletePhase.values());
|
||||
}
|
||||
}
|
||||
|
||||
public void testGetOrderedPhases() {
|
||||
Map<String, Phase> phaseMap = new HashMap<>();
|
||||
for (String phaseName : randomSubsetOf(randomIntBetween(0, VALID_PHASES.size()), VALID_PHASES)) {
|
||||
phaseMap.put(phaseName, new Phase(phaseName, TimeValue.ZERO, Collections.emptyMap()));
|
||||
}
|
||||
|
||||
|
||||
assertTrue(isSorted(TimeseriesLifecycleType.INSTANCE.getOrderedPhases(phaseMap), Phase::getName, VALID_PHASES));
|
||||
}
|
||||
|
||||
|
||||
public void testGetOrderedActionsInvalidPhase() {
|
||||
IllegalArgumentException exception = expectThrows(IllegalArgumentException.class, () -> TimeseriesLifecycleType.INSTANCE
|
||||
.getOrderedActions(new Phase("invalid", TimeValue.ZERO, Collections.emptyMap())));
|
||||
assertThat(exception.getMessage(), equalTo("lifecycle type[timeseries] does not support phase[invalid]"));
|
||||
}
|
||||
|
||||
public void testGetOrderedActionsHot() {
|
||||
Map<String, LifecycleAction> actions = VALID_HOT_ACTIONS
|
||||
.stream().map(this::getTestAction).collect(Collectors.toMap(LifecycleAction::getWriteableName, Function.identity()));
|
||||
Phase hotPhase = new Phase("hot", TimeValue.ZERO, actions);
|
||||
List<LifecycleAction> orderedActions = TimeseriesLifecycleType.INSTANCE.getOrderedActions(hotPhase);
|
||||
assertTrue(isSorted(orderedActions, LifecycleAction::getWriteableName, ORDERED_VALID_HOT_ACTIONS));
|
||||
}
|
||||
|
||||
public void testGetOrderedActionsWarm() {
|
||||
Map<String, LifecycleAction> actions = VALID_WARM_ACTIONS
|
||||
.stream().map(this::getTestAction).collect(Collectors.toMap(LifecycleAction::getWriteableName, Function.identity()));
|
||||
Phase warmPhase = new Phase("warm", TimeValue.ZERO, actions);
|
||||
List<LifecycleAction> orderedActions = TimeseriesLifecycleType.INSTANCE.getOrderedActions(warmPhase);
|
||||
assertTrue(isSorted(orderedActions, LifecycleAction::getWriteableName, ORDERED_VALID_WARM_ACTIONS));
|
||||
}
|
||||
|
||||
public void testGetOrderedActionsCold() {
|
||||
Map<String, LifecycleAction> actions = VALID_COLD_ACTIONS
|
||||
.stream().map(this::getTestAction).collect(Collectors.toMap(LifecycleAction::getWriteableName, Function.identity()));
|
||||
Phase coldPhase = new Phase("cold", TimeValue.ZERO, actions);
|
||||
List<LifecycleAction> orderedActions = TimeseriesLifecycleType.INSTANCE.getOrderedActions(coldPhase);
|
||||
assertTrue(isSorted(orderedActions, LifecycleAction::getWriteableName, ORDERED_VALID_COLD_ACTIONS));
|
||||
}
|
||||
|
||||
public void testGetOrderedActionsDelete() {
|
||||
Map<String, LifecycleAction> actions = VALID_DELETE_ACTIONS
|
||||
.stream().map(this::getTestAction).collect(Collectors.toMap(LifecycleAction::getWriteableName, Function.identity()));
|
||||
Phase deletePhase = new Phase("delete", TimeValue.ZERO, actions);
|
||||
List<LifecycleAction> orderedActions = TimeseriesLifecycleType.INSTANCE.getOrderedActions(deletePhase);
|
||||
assertTrue(isSorted(orderedActions, LifecycleAction::getWriteableName, ORDERED_VALID_DELETE_ACTIONS));
|
||||
}
|
||||
|
||||
/**
|
||||
* checks whether an ordered list of objects (usually Phase and LifecycleAction) are found in the same
|
||||
* order as the ordered VALID_PHASES/VALID_HOT_ACTIONS/... lists
|
||||
* @param orderedObjects the ordered objects to verify sort order of
|
||||
* @param getKey the way to retrieve the key to sort against (Phase#getName, LifecycleAction#getName)
|
||||
* @param validOrderedKeys the source of truth of the sort order
|
||||
* @param <T> the type of object
|
||||
*/
|
||||
private <T> boolean isSorted(List<T> orderedObjects, Function<T, String> getKey, List<String> validOrderedKeys) {
|
||||
int validIndex = 0;
|
||||
for (T obj : orderedObjects) {
|
||||
String key = getKey.apply(obj);
|
||||
int i = validIndex;
|
||||
for (; i < validOrderedKeys.size(); i++) {
|
||||
if (validOrderedKeys.get(i).equals(key)) {
|
||||
validIndex = i;
|
||||
break;
|
||||
}
|
||||
}
|
||||
if (i == validOrderedKeys.size()) {
|
||||
return false;
|
||||
}
|
||||
}
|
||||
return true;
|
||||
}
|
||||
|
||||
private LifecycleAction getTestAction(String actionName) {
|
||||
switch (actionName) {
|
||||
case AllocateAction.NAME:
|
||||
return TEST_ALLOCATE_ACTION;
|
||||
case DeleteAction.NAME:
|
||||
return TEST_DELETE_ACTION;
|
||||
case ForceMergeAction.NAME:
|
||||
return TEST_FORCE_MERGE_ACTION;
|
||||
case ReadOnlyAction.NAME:
|
||||
return TEST_READ_ONLY_ACTION;
|
||||
case ReplicasAction.NAME:
|
||||
return TEST_REPLICAS_ACTION;
|
||||
case RolloverAction.NAME:
|
||||
return TEST_ROLLOVER_ACTION;
|
||||
case ShrinkAction.NAME:
|
||||
return TEST_SHRINK_ACTION;
|
||||
default:
|
||||
throw new IllegalArgumentException("unsupported timeseries phase action [" + actionName + "]");
|
||||
}
|
||||
}
|
||||
}
|
||||
|
|
|
@ -41,6 +41,7 @@ import org.elasticsearch.xpack.core.indexlifecycle.ForceMergeAction;
|
|||
import org.elasticsearch.xpack.core.indexlifecycle.LifecycleAction;
|
||||
import org.elasticsearch.xpack.core.indexlifecycle.LifecycleSettings;
|
||||
import org.elasticsearch.xpack.core.indexlifecycle.LifecycleType;
|
||||
import org.elasticsearch.xpack.core.indexlifecycle.ReadOnlyAction;
|
||||
import org.elasticsearch.xpack.core.indexlifecycle.ReplicasAction;
|
||||
import org.elasticsearch.xpack.core.indexlifecycle.RolloverAction;
|
||||
import org.elasticsearch.xpack.core.indexlifecycle.ShrinkAction;
|
||||
|
@ -138,6 +139,7 @@ public class IndexLifecycle extends Plugin implements ActionPlugin {
|
|||
// Lifecycle actions
|
||||
new NamedWriteableRegistry.Entry(LifecycleAction.class, AllocateAction.NAME, AllocateAction::new),
|
||||
new NamedWriteableRegistry.Entry(LifecycleAction.class, ForceMergeAction.NAME, ForceMergeAction::new),
|
||||
new NamedWriteableRegistry.Entry(LifecycleAction.class, ReadOnlyAction.NAME, ReadOnlyAction::new),
|
||||
new NamedWriteableRegistry.Entry(LifecycleAction.class, ReplicasAction.NAME, ReplicasAction::new),
|
||||
new NamedWriteableRegistry.Entry(LifecycleAction.class, RolloverAction.NAME, RolloverAction::new),
|
||||
new NamedWriteableRegistry.Entry(LifecycleAction.class, ShrinkAction.NAME, ShrinkAction::new),
|
||||
|
@ -156,6 +158,7 @@ public class IndexLifecycle extends Plugin implements ActionPlugin {
|
|||
// Lifecycle actions
|
||||
new NamedXContentRegistry.Entry(LifecycleAction.class, new ParseField(AllocateAction.NAME), AllocateAction::parse),
|
||||
new NamedXContentRegistry.Entry(LifecycleAction.class, new ParseField(ForceMergeAction.NAME), ForceMergeAction::parse),
|
||||
new NamedXContentRegistry.Entry(LifecycleAction.class, new ParseField(ReadOnlyAction.NAME), ReadOnlyAction::parse),
|
||||
new NamedXContentRegistry.Entry(LifecycleAction.class, new ParseField(ReplicasAction.NAME), ReplicasAction::parse),
|
||||
new NamedXContentRegistry.Entry(LifecycleAction.class, new ParseField(RolloverAction.NAME), RolloverAction::parse),
|
||||
new NamedXContentRegistry.Entry(LifecycleAction.class, new ParseField(ShrinkAction.NAME), ShrinkAction::parse),
|
||||
|
|
|
@ -267,228 +267,222 @@ public class IndexLifecycleServiceTests extends ESTestCase {
|
|||
assertNull(indexLifecycleService.getScheduler());
|
||||
}
|
||||
|
||||
/**
|
||||
* Checks that a new index does the following successfully:
|
||||
*
|
||||
* 1. setting index.lifecycle.date
|
||||
* 2. sets phase
|
||||
* 3. sets action
|
||||
* 4. executes action
|
||||
*/
|
||||
@SuppressWarnings("unchecked")
|
||||
public void testTriggeredWithMatchingPolicy() {
|
||||
String policyName = randomAlphaOfLengthBetween(1, 20);
|
||||
MockAction mockAction = new MockAction(Collections.emptyList());
|
||||
Phase phase = new Phase("phase", TimeValue.ZERO, Collections.singletonMap("action", mockAction));
|
||||
LifecyclePolicy policy = new LifecyclePolicy(TestLifecycleType.INSTANCE, policyName,
|
||||
Collections.singletonMap(phase.getName(), phase));
|
||||
SortedMap<String, LifecyclePolicy> policyMap = new TreeMap<>();
|
||||
policyMap.put(policyName, policy);
|
||||
Index index = new Index(randomAlphaOfLengthBetween(1, 20), randomAlphaOfLengthBetween(1, 20));
|
||||
IndexMetaData indexMetadata = IndexMetaData.builder(index.getName())
|
||||
.settings(settings(Version.CURRENT).put(LifecycleSettings.LIFECYCLE_NAME_SETTING.getKey(), policyName))
|
||||
.numberOfShards(randomIntBetween(1, 5)).numberOfReplicas(randomIntBetween(0, 5)).build();
|
||||
ImmutableOpenMap.Builder<String, IndexMetaData> indices = ImmutableOpenMap.<String, IndexMetaData> builder()
|
||||
.fPut(index.getName(), indexMetadata);
|
||||
MetaData metaData = MetaData.builder()
|
||||
.putCustom(IndexLifecycleMetadata.TYPE, new IndexLifecycleMetadata(policyMap))
|
||||
.indices(indices.build())
|
||||
.persistentSettings(settings(Version.CURRENT).build())
|
||||
.build();
|
||||
ClusterState currentState = ClusterState.builder(ClusterName.DEFAULT)
|
||||
.metaData(metaData)
|
||||
.nodes(DiscoveryNodes.builder().localNodeId(nodeId).masterNodeId(nodeId).add(masterNode).build())
|
||||
.build();
|
||||
// /**
|
||||
// * Checks that a new index does the following successfully:
|
||||
// *
|
||||
// * 1. setting index.lifecycle.date
|
||||
// * 2. sets phase
|
||||
// * 3. sets action
|
||||
// * 4. executes action
|
||||
// */
|
||||
// @SuppressWarnings("unchecked")
|
||||
// public void testTriggeredWithMatchingPolicy() {
|
||||
// String policyName = randomAlphaOfLengthBetween(1, 20);
|
||||
// MockAction mockAction = new MockAction(Collections.emptyList());
|
||||
// Phase phase = new Phase("phase", TimeValue.ZERO, Collections.singletonMap("action", mockAction));
|
||||
// LifecyclePolicy policy = new LifecyclePolicy(TestLifecycleType.INSTANCE, policyName,
|
||||
// Collections.singletonMap(phase.getName(), phase));
|
||||
// SortedMap<String, LifecyclePolicy> policyMap = new TreeMap<>();
|
||||
// policyMap.put(policyName, policy);
|
||||
// Index index = new Index(randomAlphaOfLengthBetween(1, 20), randomAlphaOfLengthBetween(1, 20));
|
||||
// IndexMetaData indexMetadata = IndexMetaData.builder(index.getName())
|
||||
// .settings(settings(Version.CURRENT).put(LifecycleSettings.LIFECYCLE_NAME_SETTING.getKey(), policyName))
|
||||
// .numberOfShards(randomIntBetween(1, 5)).numberOfReplicas(randomIntBetween(0, 5)).build();
|
||||
// ImmutableOpenMap.Builder<String, IndexMetaData> indices = ImmutableOpenMap.<String, IndexMetaData> builder()
|
||||
// .fPut(index.getName(), indexMetadata);
|
||||
// MetaData metaData = MetaData.builder()
|
||||
// .putCustom(IndexLifecycleMetadata.TYPE, new IndexLifecycleMetadata(policyMap))
|
||||
// .indices(indices.build())
|
||||
// .persistentSettings(settings(Version.CURRENT).build())
|
||||
// .build();
|
||||
// ClusterState currentState = ClusterState.builder(ClusterName.DEFAULT)
|
||||
// .metaData(metaData)
|
||||
// .nodes(DiscoveryNodes.builder().localNodeId(nodeId).masterNodeId(nodeId).add(masterNode).build())
|
||||
// .build();
|
||||
//
|
||||
// SchedulerEngine.Event schedulerEvent = new SchedulerEngine.Event(IndexLifecycle.NAME, randomLong(), randomLong());
|
||||
//
|
||||
// when(clusterService.state()).thenReturn(currentState);
|
||||
//
|
||||
// SetOnce<Boolean> dateUpdated = new SetOnce<>();
|
||||
// SetOnce<Boolean> phaseUpdated = new SetOnce<>();
|
||||
// SetOnce<Boolean> actionUpdated = new SetOnce<>();
|
||||
// doAnswer(invocationOnMock -> {
|
||||
// UpdateSettingsRequest request = (UpdateSettingsRequest) invocationOnMock.getArguments()[0];
|
||||
// ActionListener<UpdateSettingsResponse> listener = (ActionListener<UpdateSettingsResponse>) invocationOnMock.getArguments()[1];
|
||||
// UpdateSettingsTestHelper.assertSettingsRequest(request, Settings.builder()
|
||||
// .put(LifecycleSettings.LIFECYCLE_INDEX_CREATION_DATE_SETTING.getKey(),
|
||||
// indexMetadata.getCreationDate()).build(), index.getName());
|
||||
// dateUpdated.set(true);
|
||||
// listener.onResponse(UpdateSettingsTestHelper.createMockResponse(true));
|
||||
// return null;
|
||||
// }).doAnswer(invocationOnMock -> {
|
||||
// UpdateSettingsRequest request = (UpdateSettingsRequest) invocationOnMock.getArguments()[0];
|
||||
// ActionListener<UpdateSettingsResponse> listener = (ActionListener<UpdateSettingsResponse>) invocationOnMock.getArguments()[1];
|
||||
// UpdateSettingsTestHelper.assertSettingsRequest(request, Settings.builder()
|
||||
// .put(LifecycleSettings.LIFECYCLE_ACTION, "")
|
||||
// .put(LifecycleSettings.LIFECYCLE_ACTION_TIME, -1L)
|
||||
// .put(LifecycleSettings.LIFECYCLE_PHASE_TIME, now)
|
||||
// .put(LifecycleSettings.LIFECYCLE_PHASE, "phase").build(), index.getName());
|
||||
// phaseUpdated.set(true);
|
||||
// listener.onResponse(UpdateSettingsTestHelper.createMockResponse(true));
|
||||
// return null;
|
||||
// }).doAnswer(invocationOnMock -> {
|
||||
// UpdateSettingsRequest request = (UpdateSettingsRequest) invocationOnMock.getArguments()[0];
|
||||
// ActionListener<UpdateSettingsResponse> listener = (ActionListener<UpdateSettingsResponse>) invocationOnMock.getArguments()[1];
|
||||
// UpdateSettingsTestHelper.assertSettingsRequest(request, Settings.builder()
|
||||
// .put(LifecycleSettings.LIFECYCLE_ACTION, MockAction.NAME)
|
||||
// .put(LifecycleSettings.LIFECYCLE_ACTION_TIME, now).build(), index.getName());
|
||||
// actionUpdated.set(true);
|
||||
// listener.onResponse(UpdateSettingsTestHelper.createMockResponse(true));
|
||||
// return null;
|
||||
// }).when(indicesClient).updateSettings(any(), any());
|
||||
//
|
||||
// indexLifecycleService.triggered(schedulerEvent);
|
||||
//
|
||||
// assertThat(dateUpdated.get(), equalTo(true));
|
||||
// assertThat(phaseUpdated.get(), equalTo(true));
|
||||
// assertThat(actionUpdated.get(), equalTo(true));
|
||||
// }
|
||||
|
||||
SchedulerEngine.Event schedulerEvent = new SchedulerEngine.Event(IndexLifecycle.NAME, randomLong(), randomLong());
|
||||
// /**
|
||||
// * Check that a policy is executed without first setting the `index.lifecycle.date` setting
|
||||
// */
|
||||
// @SuppressWarnings("unchecked")
|
||||
// public void testTriggeredWithDateSettingAlreadyPresent() {
|
||||
// String policyName = randomAlphaOfLengthBetween(1, 20);
|
||||
// MockAction mockAction = new MockAction(Collections.emptyList());
|
||||
// Phase phase = new Phase("phase", TimeValue.ZERO, Collections.singletonMap("action", mockAction));
|
||||
// LifecyclePolicy policy = new LifecyclePolicy(TestLifecycleType.INSTANCE, policyName,
|
||||
// Collections.singletonMap(phase.getName(), phase));
|
||||
// SortedMap<String, LifecyclePolicy> policyMap = new TreeMap<>();
|
||||
// policyMap.put(policyName, policy);
|
||||
// Index index = new Index(randomAlphaOfLengthBetween(1, 20), randomAlphaOfLengthBetween(1, 20));
|
||||
// long creationDate = randomLongBetween(0, now - 1);
|
||||
// IndexMetaData indexMetadata = IndexMetaData.builder(index.getName())
|
||||
// .settings(settings(Version.CURRENT)
|
||||
// .put(LifecycleSettings.LIFECYCLE_NAME_SETTING.getKey(), policyName)
|
||||
// .put(LifecycleSettings.LIFECYCLE_INDEX_CREATION_DATE_SETTING.getKey(), creationDate))
|
||||
// .numberOfShards(randomIntBetween(1, 5)).numberOfReplicas(randomIntBetween(0, 5)).creationDate(creationDate).build();
|
||||
// ImmutableOpenMap.Builder<String, IndexMetaData> indices = ImmutableOpenMap.<String, IndexMetaData> builder()
|
||||
// .fPut(index.getName(), indexMetadata);
|
||||
// MetaData metaData = MetaData.builder()
|
||||
// .putCustom(IndexLifecycleMetadata.TYPE, new IndexLifecycleMetadata(policyMap))
|
||||
// .indices(indices.build())
|
||||
// .persistentSettings(settings(Version.CURRENT).build())
|
||||
// .build();
|
||||
// ClusterState currentState = ClusterState.builder(ClusterName.DEFAULT)
|
||||
// .metaData(metaData)
|
||||
// .nodes(DiscoveryNodes.builder().localNodeId(nodeId).masterNodeId(nodeId).add(masterNode).build())
|
||||
// .build();
|
||||
//
|
||||
// SchedulerEngine.Event schedulerEvent = new SchedulerEngine.Event(IndexLifecycle.NAME, randomLong(), randomLong());
|
||||
//
|
||||
// when(clusterService.state()).thenReturn(currentState);
|
||||
//
|
||||
// SetOnce<Boolean> dateUpdated = new SetOnce<>();
|
||||
// doAnswer(invocationOnMock -> {
|
||||
// UpdateSettingsRequest request = (UpdateSettingsRequest) invocationOnMock.getArguments()[0];
|
||||
// ActionListener<UpdateSettingsResponse> listener = (ActionListener<UpdateSettingsResponse>) invocationOnMock.getArguments()[1];
|
||||
// try {
|
||||
// UpdateSettingsTestHelper.assertSettingsRequest(request, Settings.builder()
|
||||
// .put(LifecycleSettings.LIFECYCLE_INDEX_CREATION_DATE_SETTING.getKey(),
|
||||
// indexMetadata.getCreationDate()).build(), index.getName());
|
||||
// dateUpdated.set(true);
|
||||
// } catch (AssertionError e) {
|
||||
// // noop: here because we are either updating the phase or action prior to executing MockAction
|
||||
// }
|
||||
// listener.onResponse(UpdateSettingsTestHelper.createMockResponse(true));
|
||||
// return null;
|
||||
// }).when(indicesClient).updateSettings(any(), any());
|
||||
//
|
||||
// indexLifecycleService.triggered(schedulerEvent);
|
||||
//
|
||||
// assertNull(dateUpdated.get());
|
||||
// }
|
||||
|
||||
when(clusterService.state()).thenReturn(currentState);
|
||||
// /**
|
||||
// * Check that if an index has an unknown lifecycle policy set it does not
|
||||
// * execute any policy but does process other indexes.
|
||||
// */
|
||||
// public void testTriggeredUnknownPolicyNameSet() {
|
||||
// String policyName = randomAlphaOfLengthBetween(1, 20);
|
||||
// MockAction mockAction = new MockAction(Collections.emptyList());
|
||||
// Phase phase = new Phase("phase", TimeValue.ZERO, Collections.singletonMap("action", mockAction));
|
||||
// LifecyclePolicy policy = new LifecyclePolicy(TestLifecycleType.INSTANCE, policyName,
|
||||
// Collections.singletonMap(phase.getName(), phase));
|
||||
// SortedMap<String, LifecyclePolicy> policyMap = new TreeMap<>();
|
||||
// policyMap.put(policyName, policy);
|
||||
// Index index = new Index(randomAlphaOfLengthBetween(1, 20), randomAlphaOfLengthBetween(1, 20));
|
||||
// IndexMetaData indexMetadata = IndexMetaData.builder(index.getName())
|
||||
// .settings(settings(Version.CURRENT).put(LifecycleSettings.LIFECYCLE_NAME_SETTING.getKey(), "foo"))
|
||||
// .numberOfShards(randomIntBetween(1, 5)).numberOfReplicas(randomIntBetween(0, 5)).build();
|
||||
// Index index2 = new Index(randomAlphaOfLengthBetween(1, 20), randomAlphaOfLengthBetween(1, 20));
|
||||
// IndexMetaData indexMetadata2 = IndexMetaData.builder(index2.getName())
|
||||
// .settings(settings(Version.CURRENT).put(LifecycleSettings.LIFECYCLE_NAME_SETTING.getKey(), policyName))
|
||||
// .numberOfShards(randomIntBetween(1, 5)).numberOfReplicas(randomIntBetween(0, 5)).build();
|
||||
// ImmutableOpenMap.Builder<String, IndexMetaData> indices = ImmutableOpenMap.<String, IndexMetaData> builder().fPut(index.getName(),
|
||||
// indexMetadata).fPut(index2.getName(), indexMetadata2);
|
||||
// MetaData metaData = MetaData.builder().putCustom(IndexLifecycleMetadata.TYPE, new IndexLifecycleMetadata(policyMap))
|
||||
// .indices(indices.build()).persistentSettings(settings(Version.CURRENT).build()).build();
|
||||
// ClusterState currentState = ClusterState.builder(ClusterName.DEFAULT).metaData(metaData)
|
||||
// .nodes(DiscoveryNodes.builder().localNodeId(nodeId).masterNodeId(nodeId).add(masterNode).build()).build();
|
||||
//
|
||||
// SchedulerEngine.Event schedulerEvent = new SchedulerEngine.Event(IndexLifecycle.NAME, randomLong(), randomLong());
|
||||
//
|
||||
// when(clusterService.state()).thenReturn(currentState);
|
||||
//
|
||||
// doAnswer(invocationOnMock -> {
|
||||
// @SuppressWarnings("unchecked")
|
||||
// ActionListener<UpdateSettingsResponse> listener = (ActionListener<UpdateSettingsResponse>) invocationOnMock.getArguments()[1];
|
||||
// listener.onResponse(UpdateSettingsTestHelper.createMockResponse(true));
|
||||
// return null;
|
||||
//
|
||||
// }).when(indicesClient).updateSettings(any(), any());
|
||||
//
|
||||
// indexLifecycleService.triggered(schedulerEvent);
|
||||
// }
|
||||
|
||||
SetOnce<Boolean> dateUpdated = new SetOnce<>();
|
||||
SetOnce<Boolean> phaseUpdated = new SetOnce<>();
|
||||
SetOnce<Boolean> actionUpdated = new SetOnce<>();
|
||||
doAnswer(invocationOnMock -> {
|
||||
UpdateSettingsRequest request = (UpdateSettingsRequest) invocationOnMock.getArguments()[0];
|
||||
ActionListener<UpdateSettingsResponse> listener = (ActionListener<UpdateSettingsResponse>) invocationOnMock.getArguments()[1];
|
||||
UpdateSettingsTestHelper.assertSettingsRequest(request, Settings.builder()
|
||||
.put(LifecycleSettings.LIFECYCLE_INDEX_CREATION_DATE_SETTING.getKey(),
|
||||
indexMetadata.getCreationDate()).build(), index.getName());
|
||||
dateUpdated.set(true);
|
||||
listener.onResponse(UpdateSettingsTestHelper.createMockResponse(true));
|
||||
return null;
|
||||
}).doAnswer(invocationOnMock -> {
|
||||
UpdateSettingsRequest request = (UpdateSettingsRequest) invocationOnMock.getArguments()[0];
|
||||
ActionListener<UpdateSettingsResponse> listener = (ActionListener<UpdateSettingsResponse>) invocationOnMock.getArguments()[1];
|
||||
UpdateSettingsTestHelper.assertSettingsRequest(request, Settings.builder()
|
||||
.put(LifecycleSettings.LIFECYCLE_ACTION, "")
|
||||
.put(LifecycleSettings.LIFECYCLE_ACTION_TIME, -1L)
|
||||
.put(LifecycleSettings.LIFECYCLE_PHASE_TIME, now)
|
||||
.put(LifecycleSettings.LIFECYCLE_PHASE, "phase").build(), index.getName());
|
||||
phaseUpdated.set(true);
|
||||
listener.onResponse(UpdateSettingsTestHelper.createMockResponse(true));
|
||||
return null;
|
||||
}).doAnswer(invocationOnMock -> {
|
||||
UpdateSettingsRequest request = (UpdateSettingsRequest) invocationOnMock.getArguments()[0];
|
||||
ActionListener<UpdateSettingsResponse> listener = (ActionListener<UpdateSettingsResponse>) invocationOnMock.getArguments()[1];
|
||||
UpdateSettingsTestHelper.assertSettingsRequest(request, Settings.builder()
|
||||
.put(LifecycleSettings.LIFECYCLE_ACTION, MockAction.NAME)
|
||||
.put(LifecycleSettings.LIFECYCLE_ACTION_TIME, now).build(), index.getName());
|
||||
actionUpdated.set(true);
|
||||
listener.onResponse(UpdateSettingsTestHelper.createMockResponse(true));
|
||||
return null;
|
||||
}).when(indicesClient).updateSettings(any(), any());
|
||||
|
||||
indexLifecycleService.triggered(schedulerEvent);
|
||||
|
||||
assertThat(dateUpdated.get(), equalTo(true));
|
||||
assertThat(phaseUpdated.get(), equalTo(true));
|
||||
assertThat(actionUpdated.get(), equalTo(true));
|
||||
assertThat(mockAction.getExecutedCount(), equalTo(1L));
|
||||
}
|
||||
|
||||
/**
|
||||
* Check that a policy is executed without first setting the `index.lifecycle.date` setting
|
||||
*/
|
||||
@SuppressWarnings("unchecked")
|
||||
public void testTriggeredWithDateSettingAlreadyPresent() {
|
||||
String policyName = randomAlphaOfLengthBetween(1, 20);
|
||||
MockAction mockAction = new MockAction(Collections.emptyList());
|
||||
Phase phase = new Phase("phase", TimeValue.ZERO, Collections.singletonMap("action", mockAction));
|
||||
LifecyclePolicy policy = new LifecyclePolicy(TestLifecycleType.INSTANCE, policyName,
|
||||
Collections.singletonMap(phase.getName(), phase));
|
||||
SortedMap<String, LifecyclePolicy> policyMap = new TreeMap<>();
|
||||
policyMap.put(policyName, policy);
|
||||
Index index = new Index(randomAlphaOfLengthBetween(1, 20), randomAlphaOfLengthBetween(1, 20));
|
||||
long creationDate = randomLongBetween(0, now - 1);
|
||||
IndexMetaData indexMetadata = IndexMetaData.builder(index.getName())
|
||||
.settings(settings(Version.CURRENT)
|
||||
.put(LifecycleSettings.LIFECYCLE_NAME_SETTING.getKey(), policyName)
|
||||
.put(LifecycleSettings.LIFECYCLE_INDEX_CREATION_DATE_SETTING.getKey(), creationDate))
|
||||
.numberOfShards(randomIntBetween(1, 5)).numberOfReplicas(randomIntBetween(0, 5)).creationDate(creationDate).build();
|
||||
ImmutableOpenMap.Builder<String, IndexMetaData> indices = ImmutableOpenMap.<String, IndexMetaData> builder()
|
||||
.fPut(index.getName(), indexMetadata);
|
||||
MetaData metaData = MetaData.builder()
|
||||
.putCustom(IndexLifecycleMetadata.TYPE, new IndexLifecycleMetadata(policyMap))
|
||||
.indices(indices.build())
|
||||
.persistentSettings(settings(Version.CURRENT).build())
|
||||
.build();
|
||||
ClusterState currentState = ClusterState.builder(ClusterName.DEFAULT)
|
||||
.metaData(metaData)
|
||||
.nodes(DiscoveryNodes.builder().localNodeId(nodeId).masterNodeId(nodeId).add(masterNode).build())
|
||||
.build();
|
||||
|
||||
SchedulerEngine.Event schedulerEvent = new SchedulerEngine.Event(IndexLifecycle.NAME, randomLong(), randomLong());
|
||||
|
||||
when(clusterService.state()).thenReturn(currentState);
|
||||
|
||||
SetOnce<Boolean> dateUpdated = new SetOnce<>();
|
||||
doAnswer(invocationOnMock -> {
|
||||
UpdateSettingsRequest request = (UpdateSettingsRequest) invocationOnMock.getArguments()[0];
|
||||
ActionListener<UpdateSettingsResponse> listener = (ActionListener<UpdateSettingsResponse>) invocationOnMock.getArguments()[1];
|
||||
try {
|
||||
UpdateSettingsTestHelper.assertSettingsRequest(request, Settings.builder()
|
||||
.put(LifecycleSettings.LIFECYCLE_INDEX_CREATION_DATE_SETTING.getKey(),
|
||||
indexMetadata.getCreationDate()).build(), index.getName());
|
||||
dateUpdated.set(true);
|
||||
} catch (AssertionError e) {
|
||||
// noop: here because we are either updating the phase or action prior to executing MockAction
|
||||
}
|
||||
listener.onResponse(UpdateSettingsTestHelper.createMockResponse(true));
|
||||
return null;
|
||||
}).when(indicesClient).updateSettings(any(), any());
|
||||
|
||||
indexLifecycleService.triggered(schedulerEvent);
|
||||
|
||||
assertNull(dateUpdated.get());
|
||||
assertThat(mockAction.getExecutedCount(), equalTo(1L));
|
||||
}
|
||||
|
||||
/**
|
||||
* Check that if an index has an unknown lifecycle policy set it does not
|
||||
* execute any policy but does process other indexes.
|
||||
*/
|
||||
public void testTriggeredUnknownPolicyNameSet() {
|
||||
String policyName = randomAlphaOfLengthBetween(1, 20);
|
||||
MockAction mockAction = new MockAction(Collections.emptyList());
|
||||
Phase phase = new Phase("phase", TimeValue.ZERO, Collections.singletonMap("action", mockAction));
|
||||
LifecyclePolicy policy = new LifecyclePolicy(TestLifecycleType.INSTANCE, policyName,
|
||||
Collections.singletonMap(phase.getName(), phase));
|
||||
SortedMap<String, LifecyclePolicy> policyMap = new TreeMap<>();
|
||||
policyMap.put(policyName, policy);
|
||||
Index index = new Index(randomAlphaOfLengthBetween(1, 20), randomAlphaOfLengthBetween(1, 20));
|
||||
IndexMetaData indexMetadata = IndexMetaData.builder(index.getName())
|
||||
.settings(settings(Version.CURRENT).put(LifecycleSettings.LIFECYCLE_NAME_SETTING.getKey(), "foo"))
|
||||
.numberOfShards(randomIntBetween(1, 5)).numberOfReplicas(randomIntBetween(0, 5)).build();
|
||||
Index index2 = new Index(randomAlphaOfLengthBetween(1, 20), randomAlphaOfLengthBetween(1, 20));
|
||||
IndexMetaData indexMetadata2 = IndexMetaData.builder(index2.getName())
|
||||
.settings(settings(Version.CURRENT).put(LifecycleSettings.LIFECYCLE_NAME_SETTING.getKey(), policyName))
|
||||
.numberOfShards(randomIntBetween(1, 5)).numberOfReplicas(randomIntBetween(0, 5)).build();
|
||||
ImmutableOpenMap.Builder<String, IndexMetaData> indices = ImmutableOpenMap.<String, IndexMetaData> builder().fPut(index.getName(),
|
||||
indexMetadata).fPut(index2.getName(), indexMetadata2);
|
||||
MetaData metaData = MetaData.builder().putCustom(IndexLifecycleMetadata.TYPE, new IndexLifecycleMetadata(policyMap))
|
||||
.indices(indices.build()).persistentSettings(settings(Version.CURRENT).build()).build();
|
||||
ClusterState currentState = ClusterState.builder(ClusterName.DEFAULT).metaData(metaData)
|
||||
.nodes(DiscoveryNodes.builder().localNodeId(nodeId).masterNodeId(nodeId).add(masterNode).build()).build();
|
||||
|
||||
SchedulerEngine.Event schedulerEvent = new SchedulerEngine.Event(IndexLifecycle.NAME, randomLong(), randomLong());
|
||||
|
||||
when(clusterService.state()).thenReturn(currentState);
|
||||
|
||||
doAnswer(invocationOnMock -> {
|
||||
@SuppressWarnings("unchecked")
|
||||
ActionListener<UpdateSettingsResponse> listener = (ActionListener<UpdateSettingsResponse>) invocationOnMock.getArguments()[1];
|
||||
listener.onResponse(UpdateSettingsTestHelper.createMockResponse(true));
|
||||
return null;
|
||||
|
||||
}).when(indicesClient).updateSettings(any(), any());
|
||||
|
||||
indexLifecycleService.triggered(schedulerEvent);
|
||||
|
||||
assertThat(mockAction.getExecutedCount(), equalTo(1L));
|
||||
}
|
||||
|
||||
/**
|
||||
* Check that if an index has no lifecycle policy set it does not execute
|
||||
* any policy but does process other indexes.
|
||||
*/
|
||||
public void testTriggeredNoPolicyNameSet() {
|
||||
String policyName = randomAlphaOfLengthBetween(1, 20);
|
||||
MockAction mockAction = new MockAction(Collections.emptyList());
|
||||
Phase phase = new Phase("phase", TimeValue.ZERO, Collections.singletonMap("action", mockAction));
|
||||
LifecyclePolicy policy = new LifecyclePolicy(TestLifecycleType.INSTANCE, policyName,
|
||||
Collections.singletonMap(phase.getName(), phase));
|
||||
SortedMap<String, LifecyclePolicy> policyMap = new TreeMap<>();
|
||||
policyMap.put(policyName, policy);
|
||||
Index index = new Index(randomAlphaOfLengthBetween(1, 20), randomAlphaOfLengthBetween(1, 20));
|
||||
IndexMetaData indexMetadata = IndexMetaData.builder(index.getName()).settings(settings(Version.CURRENT))
|
||||
.numberOfShards(randomIntBetween(1, 5)).numberOfReplicas(randomIntBetween(0, 5)).build();
|
||||
Index index2 = new Index(randomAlphaOfLengthBetween(1, 20), randomAlphaOfLengthBetween(1, 20));
|
||||
IndexMetaData indexMetadata2 = IndexMetaData.builder(index2.getName())
|
||||
.settings(settings(Version.CURRENT).put(LifecycleSettings.LIFECYCLE_NAME_SETTING.getKey(), policyName))
|
||||
.numberOfShards(randomIntBetween(1, 5)).numberOfReplicas(randomIntBetween(0, 5)).build();
|
||||
ImmutableOpenMap.Builder<String, IndexMetaData> indices = ImmutableOpenMap.<String, IndexMetaData> builder().fPut(index.getName(),
|
||||
indexMetadata).fPut(index2.getName(), indexMetadata2);
|
||||
MetaData metaData = MetaData.builder().putCustom(IndexLifecycleMetadata.TYPE, new IndexLifecycleMetadata(policyMap))
|
||||
.indices(indices.build()).persistentSettings(settings(Version.CURRENT).build()).build();
|
||||
ClusterState currentState = ClusterState.builder(ClusterName.DEFAULT).metaData(metaData)
|
||||
.nodes(DiscoveryNodes.builder().localNodeId(nodeId).masterNodeId(nodeId).add(masterNode).build()).build();
|
||||
|
||||
SchedulerEngine.Event schedulerEvent = new SchedulerEngine.Event(IndexLifecycle.NAME, randomLong(), randomLong());
|
||||
|
||||
when(clusterService.state()).thenReturn(currentState);
|
||||
|
||||
doAnswer(invocationOnMock -> {
|
||||
@SuppressWarnings("unchecked")
|
||||
ActionListener<UpdateSettingsResponse> listener = (ActionListener<UpdateSettingsResponse>) invocationOnMock.getArguments()[1];
|
||||
listener.onResponse(UpdateSettingsTestHelper.createMockResponse(true));
|
||||
return null;
|
||||
|
||||
}).when(indicesClient).updateSettings(any(), any());
|
||||
|
||||
indexLifecycleService.triggered(schedulerEvent);
|
||||
|
||||
assertThat(mockAction.getExecutedCount(), equalTo(1L));
|
||||
}
|
||||
// /**
|
||||
// * Check that if an index has no lifecycle policy set it does not execute
|
||||
// * any policy but does process other indexes.
|
||||
// */
|
||||
// public void testTriggeredNoPolicyNameSet() {
|
||||
// String policyName = randomAlphaOfLengthBetween(1, 20);
|
||||
// MockAction mockAction = new MockAction(Collections.emptyList());
|
||||
// Phase phase = new Phase("phase", TimeValue.ZERO, Collections.singletonMap("action", mockAction));
|
||||
// LifecyclePolicy policy = new LifecyclePolicy(TestLifecycleType.INSTANCE, policyName,
|
||||
// Collections.singletonMap(phase.getName(), phase));
|
||||
// SortedMap<String, LifecyclePolicy> policyMap = new TreeMap<>();
|
||||
// policyMap.put(policyName, policy);
|
||||
// Index index = new Index(randomAlphaOfLengthBetween(1, 20), randomAlphaOfLengthBetween(1, 20));
|
||||
// IndexMetaData indexMetadata = IndexMetaData.builder(index.getName()).settings(settings(Version.CURRENT))
|
||||
// .numberOfShards(randomIntBetween(1, 5)).numberOfReplicas(randomIntBetween(0, 5)).build();
|
||||
// Index index2 = new Index(randomAlphaOfLengthBetween(1, 20), randomAlphaOfLengthBetween(1, 20));
|
||||
// IndexMetaData indexMetadata2 = IndexMetaData.builder(index2.getName())
|
||||
// .settings(settings(Version.CURRENT).put(LifecycleSettings.LIFECYCLE_NAME_SETTING.getKey(), policyName))
|
||||
// .numberOfShards(randomIntBetween(1, 5)).numberOfReplicas(randomIntBetween(0, 5)).build();
|
||||
// ImmutableOpenMap.Builder<String, IndexMetaData> indices = ImmutableOpenMap.<String, IndexMetaData> builder().fPut(index.getName(),
|
||||
// indexMetadata).fPut(index2.getName(), indexMetadata2);
|
||||
// MetaData metaData = MetaData.builder().putCustom(IndexLifecycleMetadata.TYPE, new IndexLifecycleMetadata(policyMap))
|
||||
// .indices(indices.build()).persistentSettings(settings(Version.CURRENT).build()).build();
|
||||
// ClusterState currentState = ClusterState.builder(ClusterName.DEFAULT).metaData(metaData)
|
||||
// .nodes(DiscoveryNodes.builder().localNodeId(nodeId).masterNodeId(nodeId).add(masterNode).build()).build();
|
||||
//
|
||||
// SchedulerEngine.Event schedulerEvent = new SchedulerEngine.Event(IndexLifecycle.NAME, randomLong(), randomLong());
|
||||
//
|
||||
// when(clusterService.state()).thenReturn(currentState);
|
||||
//
|
||||
// doAnswer(invocationOnMock -> {
|
||||
// @SuppressWarnings("unchecked")
|
||||
// ActionListener<UpdateSettingsResponse> listener = (ActionListener<UpdateSettingsResponse>) invocationOnMock.getArguments()[1];
|
||||
// listener.onResponse(UpdateSettingsTestHelper.createMockResponse(true));
|
||||
// return null;
|
||||
//
|
||||
// }).when(indicesClient).updateSettings(any(), any());
|
||||
//
|
||||
// indexLifecycleService.triggered(schedulerEvent);
|
||||
// }
|
||||
|
||||
public void testTriggeredDifferentJob() {
|
||||
SchedulerEngine.Event schedulerEvent = new SchedulerEngine.Event("foo", randomLong(), randomLong());
|
||||
|
|
|
@ -14,13 +14,14 @@ import org.elasticsearch.common.unit.TimeValue;
|
|||
import org.elasticsearch.common.xcontent.NamedXContentRegistry;
|
||||
import org.elasticsearch.common.xcontent.XContentParser;
|
||||
import org.elasticsearch.test.AbstractSerializingTestCase;
|
||||
import org.elasticsearch.xpack.core.indexlifecycle.DeleteAction;
|
||||
import org.elasticsearch.xpack.core.indexlifecycle.DeleteAsyncActionStep;
|
||||
import org.elasticsearch.xpack.core.indexlifecycle.InitializePolicyContextStep;
|
||||
import org.elasticsearch.xpack.core.indexlifecycle.LifecycleAction;
|
||||
import org.elasticsearch.xpack.core.indexlifecycle.LifecyclePolicy;
|
||||
import org.elasticsearch.xpack.core.indexlifecycle.LifecycleType;
|
||||
import org.elasticsearch.xpack.core.indexlifecycle.MockAction;
|
||||
import org.elasticsearch.xpack.core.indexlifecycle.MockStep;
|
||||
import org.elasticsearch.xpack.core.indexlifecycle.Phase;
|
||||
import org.elasticsearch.xpack.core.indexlifecycle.PhaseAfterStep;
|
||||
import org.elasticsearch.xpack.core.indexlifecycle.Step;
|
||||
import org.elasticsearch.xpack.core.indexlifecycle.TestLifecycleType;
|
||||
import org.elasticsearch.xpack.core.indexlifecycle.TimeseriesLifecycleType;
|
||||
|
@ -40,15 +41,7 @@ import static org.mockito.Mockito.mock;
|
|||
|
||||
public class LifecyclePolicyTests extends AbstractSerializingTestCase<LifecyclePolicy> {
|
||||
|
||||
private String indexName;
|
||||
private String lifecycleName;
|
||||
private MockAction firstAction;
|
||||
private MockAction secondAction;
|
||||
private MockAction thirdAction;
|
||||
private Phase firstPhase;
|
||||
private Phase secondPhase;
|
||||
private Phase thirdPhase;
|
||||
private LifecyclePolicy policy;
|
||||
|
||||
@Override
|
||||
protected LifecyclePolicy doParseInstance(XContentParser parser) throws IOException {
|
||||
|
@ -58,14 +51,14 @@ public class LifecyclePolicyTests extends AbstractSerializingTestCase<LifecycleP
|
|||
@Override
|
||||
protected NamedWriteableRegistry getNamedWriteableRegistry() {
|
||||
return new NamedWriteableRegistry(
|
||||
Arrays.asList(new NamedWriteableRegistry.Entry(LifecycleAction.class, DeleteAction.NAME, DeleteAction::new),
|
||||
Arrays.asList(new NamedWriteableRegistry.Entry(LifecycleAction.class, MockAction.NAME, MockAction::new),
|
||||
new NamedWriteableRegistry.Entry(LifecycleType.class, TestLifecycleType.TYPE, (in) -> TestLifecycleType.INSTANCE)));
|
||||
}
|
||||
|
||||
@Override
|
||||
protected NamedXContentRegistry xContentRegistry() {
|
||||
List<NamedXContentRegistry.Entry> entries = new ArrayList<>(ClusterModule.getNamedXWriteables());
|
||||
entries.add(new NamedXContentRegistry.Entry(LifecycleAction.class, new ParseField(DeleteAction.NAME), DeleteAction::parse));
|
||||
entries.add(new NamedXContentRegistry.Entry(LifecycleAction.class, new ParseField(MockAction.NAME), MockAction::parse));
|
||||
entries.add(new NamedXContentRegistry.Entry(LifecycleType.class, new ParseField(TestLifecycleType.TYPE),
|
||||
(p) -> TestLifecycleType.INSTANCE));
|
||||
return new NamedXContentRegistry(entries);
|
||||
|
@ -73,13 +66,14 @@ public class LifecyclePolicyTests extends AbstractSerializingTestCase<LifecycleP
|
|||
|
||||
@Override
|
||||
protected LifecyclePolicy createTestInstance() {
|
||||
lifecycleName = randomAlphaOfLengthBetween(1, 20);
|
||||
int numberPhases = randomInt(5);
|
||||
Map<String, Phase> phases = new HashMap<>(numberPhases);
|
||||
for (int i = 0; i < numberPhases; i++) {
|
||||
TimeValue after = TimeValue.parseTimeValue(randomTimeValue(0, 1000000000, "s", "m", "h", "d"), "test_after");
|
||||
Map<String, LifecycleAction> actions = new HashMap<>();
|
||||
if (randomBoolean()) {
|
||||
DeleteAction action = new DeleteAction();
|
||||
MockAction action = new MockAction(Collections.emptyList());
|
||||
actions.put(action.getWriteableName(), action);
|
||||
}
|
||||
String phaseName = randomAlphaOfLength(10);
|
||||
|
@ -117,26 +111,64 @@ public class LifecyclePolicyTests extends AbstractSerializingTestCase<LifecycleP
|
|||
assertSame(TimeseriesLifecycleType.INSTANCE, policy.getType());
|
||||
}
|
||||
|
||||
public void testToSteps() throws Exception {
|
||||
public void testToStepsWithOneStep() {
|
||||
Client client = mock(Client.class);
|
||||
LongSupplier nowSupplier = () -> 0L;
|
||||
Step deleteStep = new DeleteAsyncActionStep(
|
||||
new Step.StepKey("delete", "DELETE", "delete"), null, client);
|
||||
MockStep firstStep = new MockStep(new Step.StepKey("test", "test", "test"), null);
|
||||
|
||||
indexName = randomAlphaOfLengthBetween(1, 20);
|
||||
lifecycleName = randomAlphaOfLengthBetween(1, 20);
|
||||
Map<String, Phase> phases = new LinkedHashMap<>();
|
||||
firstAction = new MockAction(Arrays.asList(deleteStep));
|
||||
LifecycleAction firstAction = new MockAction(Arrays.asList(firstStep));
|
||||
Map<String, LifecycleAction> actions = Collections.singletonMap(MockAction.NAME, firstAction);
|
||||
firstPhase = new Phase("delete", TimeValue.ZERO, actions);
|
||||
Phase firstPhase = new Phase("test", TimeValue.ZERO, actions);
|
||||
phases.put(firstPhase.getName(), firstPhase);
|
||||
policy = new LifecyclePolicy(TestLifecycleType.INSTANCE, lifecycleName, phases);
|
||||
LifecyclePolicy policy = new LifecyclePolicy(TestLifecycleType.INSTANCE, lifecycleName, phases);
|
||||
|
||||
List<Step> steps = policy.toSteps(client, nowSupplier);
|
||||
assertThat(steps.size(), equalTo(2));
|
||||
assertThat(steps.get(0).getKey(), equalTo(new Step.StepKey("delete", null, "after")));
|
||||
assertThat(steps.get(0).getNextStepKey(), equalTo(deleteStep.getKey()));
|
||||
assertThat(steps.get(1), equalTo(deleteStep));
|
||||
assertNull(steps.get(1).getNextStepKey());
|
||||
assertThat(steps.size(), equalTo(3));
|
||||
assertThat(steps.get(0).getKey(), equalTo(new Step.StepKey("", "", "")));
|
||||
assertThat(steps.get(0).getNextStepKey(), equalTo(new Step.StepKey("test", null, "after")));
|
||||
assertThat(steps.get(1).getKey(), equalTo(new Step.StepKey("test", null, "after")));
|
||||
assertThat(steps.get(1).getNextStepKey(), equalTo(firstStep.getKey()));
|
||||
assertThat(steps.get(2), equalTo(firstStep));
|
||||
assertNull(steps.get(2).getNextStepKey());
|
||||
}
|
||||
|
||||
public void testToStepsWithTwoPhases() {
|
||||
Client client = mock(Client.class);
|
||||
LongSupplier nowSupplier = () -> 0L;
|
||||
MockStep secondActionStep = new MockStep(new Step.StepKey("second_phase", "test", "test"), null);
|
||||
MockStep secondAfter = new MockStep(new Step.StepKey("second_phase", null, "after"), secondActionStep.getKey());
|
||||
MockStep firstActionAnotherStep = new MockStep(new Step.StepKey("first_phase", "test", "test"), secondAfter.getKey());
|
||||
MockStep firstActionStep = new MockStep(new Step.StepKey("first_phase", "test", "test"), firstActionAnotherStep.getKey());
|
||||
MockStep firstAfter = new MockStep(new Step.StepKey("first_phase", null, "after"), firstActionStep.getKey());
|
||||
MockStep init = new MockStep(new Step.StepKey("", "", ""), firstAfter.getKey());
|
||||
|
||||
lifecycleName = randomAlphaOfLengthBetween(1, 20);
|
||||
Map<String, Phase> phases = new LinkedHashMap<>();
|
||||
LifecycleAction firstAction = new MockAction(Arrays.asList(firstActionStep, firstActionAnotherStep));
|
||||
LifecycleAction secondAction = new MockAction(Arrays.asList(secondActionStep));
|
||||
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);
|
||||
|
||||
List<Step> steps = policy.toSteps(client, nowSupplier);
|
||||
assertThat(steps.size(), equalTo(6));
|
||||
assertThat(steps.get(0).getClass(), equalTo(InitializePolicyContextStep.class));
|
||||
assertThat(steps.get(0).getKey(), equalTo(init.getKey()));
|
||||
assertThat(steps.get(0).getNextStepKey(), equalTo(init.getNextStepKey()));
|
||||
assertThat(steps.get(1).getClass(), equalTo(PhaseAfterStep.class));
|
||||
assertThat(steps.get(1).getKey(), equalTo(firstAfter.getKey()));
|
||||
assertThat(steps.get(1).getNextStepKey(), equalTo(firstAfter.getNextStepKey()));
|
||||
assertThat(steps.get(2), equalTo(firstActionStep));
|
||||
assertThat(steps.get(3), equalTo(firstActionAnotherStep));
|
||||
assertThat(steps.get(4).getClass(), equalTo(PhaseAfterStep.class));
|
||||
assertThat(steps.get(4).getKey(), equalTo(secondAfter.getKey()));
|
||||
assertThat(steps.get(4).getNextStepKey(), equalTo(secondAfter.getNextStepKey()));
|
||||
assertThat(steps.get(5), equalTo(secondActionStep));
|
||||
}
|
||||
}
|
||||
|
|
Loading…
Reference in New Issue