step by step

This commit is contained in:
Tal Levy 2018-03-22 22:46:17 -07:00
parent fe22c08207
commit d63cd8c9c3
40 changed files with 4170 additions and 5628 deletions

View File

@ -6,13 +6,11 @@
package org.elasticsearch.xpack.core.indexlifecycle;
import org.apache.logging.log4j.Logger;
import org.elasticsearch.action.ActionListener;
import org.elasticsearch.action.admin.indices.settings.put.UpdateSettingsRequest;
import org.elasticsearch.action.admin.indices.settings.put.UpdateSettingsResponse;
import org.elasticsearch.action.support.ActiveShardCount;
import org.elasticsearch.client.Client;
import org.elasticsearch.cluster.ClusterState;
import org.elasticsearch.cluster.metadata.IndexMetaData;
import org.elasticsearch.cluster.metadata.MetaData;
import org.elasticsearch.cluster.routing.ShardRouting;
import org.elasticsearch.cluster.routing.allocation.RoutingAllocation;
import org.elasticsearch.cluster.routing.allocation.decider.AllocationDeciders;
@ -30,14 +28,15 @@ import org.elasticsearch.common.xcontent.ConstructingObjectParser;
import org.elasticsearch.common.xcontent.XContentBuilder;
import org.elasticsearch.common.xcontent.XContentParser;
import org.elasticsearch.index.Index;
import org.elasticsearch.index.IndexNotFoundException;
import org.elasticsearch.threadpool.ThreadPool;
import java.io.IOException;
import java.util.Arrays;
import java.util.Collections;
import java.util.List;
import java.util.Map;
import java.util.Objects;
import java.util.function.BiConsumer;
import java.util.function.LongSupplier;
public class AllocateAction implements LifecycleAction {
@ -145,78 +144,62 @@ public class AllocateAction implements LifecycleAction {
}).forEach(e -> newSettingsBuilder.put(settingPrefix + e.getKey(), e.getValue()));
}
void execute(Index index, BiConsumer<Settings, Listener> settingsUpdater, ClusterState clusterState, ClusterSettings clusterSettings,
Listener listener) {
// We only want to make progress if all shards are active so check that
// first
if (ActiveShardCount.ALL.enoughShardsActive(clusterState, index.getName()) == false) {
logger.debug("[{}] lifecycle action for index [{}] cannot make progress because not all shards are active", NAME,
index.getName());
listener.onSuccess(false);
return;
}
IndexMetaData idxMeta = clusterState.metaData().index(index);
if (idxMeta == null) {
listener.onFailure(
new IndexNotFoundException("Index not found when executing " + NAME + " lifecycle action.", index.getName()));
return;
}
Settings existingSettings = idxMeta.getSettings();
Settings.Builder newSettings = Settings.builder();
addMissingAttrs(include, IndexMetaData.INDEX_ROUTING_INCLUDE_GROUP_SETTING.getKey(), existingSettings, newSettings);
addMissingAttrs(exclude, IndexMetaData.INDEX_ROUTING_EXCLUDE_GROUP_SETTING.getKey(), existingSettings, newSettings);
addMissingAttrs(require, IndexMetaData.INDEX_ROUTING_REQUIRE_GROUP_SETTING.getKey(), existingSettings, newSettings);
Settings newAllocationIncludes = newSettings.build();
if (newAllocationIncludes.isEmpty()) {
public static ConditionalWaitStep getAllocationCheck(AllocationDeciders allocationDeciders, String phase, String index) {
return new ConditionalWaitStep("wait_allocation", NAME,
phase, index, (clusterState) -> {
// We only want to make progress if all shards are active so check that first
if (ActiveShardCount.ALL.enoughShardsActive(clusterState, index) == false) {
logger.debug("[{}] lifecycle action for index [{}] cannot make progress because not all shards are active", NAME,
index);
return false;
}
// All the allocation attributes are already set so just need to
// check if the allocation has happened
RoutingAllocation allocation = new RoutingAllocation(allocationDeciders, clusterState.getRoutingNodes(), clusterState, null,
System.nanoTime());
System.nanoTime());
int allocationPendingShards = 0;
List<ShardRouting> allShards = clusterState.getRoutingTable().allShards(index.getName());
List<ShardRouting> allShards = clusterState.getRoutingTable().allShards(index);
for (ShardRouting shardRouting : allShards) {
assert shardRouting.active() : "Shard not active, found " + shardRouting.state() + "for shard with id: "
+ shardRouting.shardId();
+ shardRouting.shardId();
String currentNodeId = shardRouting.currentNodeId();
boolean canRemainOnCurrentNode = allocationDeciders.canRemain(shardRouting,
clusterState.getRoutingNodes().node(currentNodeId), allocation).type() == Decision.Type.YES;
clusterState.getRoutingNodes().node(currentNodeId), allocation).type() == Decision.Type.YES;
if (canRemainOnCurrentNode == false) {
allocationPendingShards++;
}
}
if (allocationPendingShards > 0) {
logger.debug("[{}] lifecycle action for index [{}] waiting for [{}] shards "
+ "to be allocated to nodes matching the given filters", NAME, index.getName(), allocationPendingShards);
listener.onSuccess(false);
+ "to be allocated to nodes matching the given filters", NAME, index, allocationPendingShards);
return false;
} else {
logger.debug("[{}] lifecycle action for index [{}] complete", NAME, index.getName());
listener.onSuccess(true);
logger.debug("[{}] lifecycle action for index [{}] complete", NAME, index);
return true;
}
} else {
// We have some allocation attributes to set
settingsUpdater.accept(newAllocationIncludes, listener);
}
});
}
@Override
public void execute(Index index, Client client, ClusterService clusterService, Listener listener) {
ClusterState clusterState = clusterService.state();
BiConsumer<Settings, Listener> settingsUpdater = (s, l) -> {
public List<Step> toSteps(String phase, Index index, Client client, ThreadPool threadPool, LongSupplier nowSupplier) {
ClusterStateUpdateStep updateAllocationSettings = new ClusterStateUpdateStep(
"update_allocation", NAME, phase, index.getName(), (clusterState) -> {
IndexMetaData idxMeta = clusterState.metaData().index(index);
if (idxMeta == null) {
return clusterState;
}
Settings existingSettings = idxMeta.getSettings();
Settings.Builder newSettings = Settings.builder();
addMissingAttrs(include, IndexMetaData.INDEX_ROUTING_INCLUDE_GROUP_SETTING.getKey(), existingSettings, newSettings);
addMissingAttrs(exclude, IndexMetaData.INDEX_ROUTING_EXCLUDE_GROUP_SETTING.getKey(), existingSettings, newSettings);
addMissingAttrs(require, IndexMetaData.INDEX_ROUTING_REQUIRE_GROUP_SETTING.getKey(), existingSettings, newSettings);
return ClusterState.builder(clusterState)
.metaData(MetaData.builder(clusterState.metaData())
.updateSettings(newSettings.build(), index.getName())).build();
});
client.admin().indices().updateSettings(new UpdateSettingsRequest(s, index.getName()),
new ActionListener<UpdateSettingsResponse>() {
@Override
public void onResponse(UpdateSettingsResponse updateSettingsResponse) {
l.onSuccess(false);
}
@Override
public void onFailure(Exception e) {
l.onFailure(e);
}
});
};
execute(index, settingsUpdater, clusterState, clusterService.getClusterSettings(), listener);
return Arrays.asList(updateAllocationSettings, getAllocationCheck(allocationDeciders, phase, index.getName()));
}
@Override

View File

@ -0,0 +1,58 @@
/*
* 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.action.ActionListener;
import org.elasticsearch.action.ActionRequestBuilder;
import org.elasticsearch.action.ActionResponse;
import org.elasticsearch.cluster.ClusterState;
import org.elasticsearch.xpack.core.indexlifecycle.Step;
import org.elasticsearch.xpack.core.indexlifecycle.StepResult;
import java.util.concurrent.CompletableFuture;
import java.util.function.Function;
public class ClientStep<RequestBuilder extends ActionRequestBuilder, Response extends ActionResponse> extends Step {
private final RequestBuilder requestBuilder;
private final Function<ClusterState, Boolean> checkComplete;
private final Function<Response, Boolean> checkSuccess;
private Exception returnedException;
private boolean returnedSuccess;
public ClientStep(String name, String action, String phase, String index, RequestBuilder requestBuilder,
Function<ClusterState, Boolean> checkComplete, Function<Response, Boolean> checkSuccess) {
super(name, action, phase, index);
this.requestBuilder = requestBuilder;
this.checkComplete = checkComplete;
this.checkSuccess = checkSuccess;
this.returnedException = null;
this.returnedSuccess = false;
}
@Override
public StepResult execute(ClusterState currentState) {
if (checkComplete.apply(currentState)) {
return new StepResult("client-complete", null, currentState, true, true);
} else {
requestBuilder.execute(new ActionListener<Response>() {
@Override
public void onResponse(Response r) {
if (checkSuccess.apply(r)) {
returnedSuccess = true;
}
// IndexLifecycleService.triggerPolicies()
}
@Override
public void onFailure(Exception e) {
returnedException = e;
}
});
return new StepResult("client-in-progress", null, currentState, true, false);
}
}
}

View File

@ -0,0 +1,39 @@
/*
* 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.xpack.core.indexlifecycle.Step;
import org.elasticsearch.xpack.core.indexlifecycle.StepResult;
import java.util.function.Function;
public class ClusterStateUpdateStep extends Step {
private final String name;
private final String index;
private final String phase;
private final String action;
private final Function<ClusterState, ClusterState> updateTask;
public ClusterStateUpdateStep(String name, String index, String phase, String action, Function<ClusterState, ClusterState> updateTask) {
super(name, action, phase, index);
this.name = name;
this.index = index;
this.phase = phase;
this.action = action;
this.updateTask = updateTask;
}
public StepResult execute(ClusterState clusterState) {
ClusterState updated = null;
try {
updated = updateTask.apply(clusterState);
return new StepResult("done!", null, updated, true, true);
} catch (Exception e) {
return new StepResult("something went wrong", e, updated, true, true);
}
}
}

View File

@ -0,0 +1,27 @@
/*
* 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.xpack.core.indexlifecycle.Step;
import org.elasticsearch.xpack.core.indexlifecycle.StepResult;
import java.util.function.Function;
public class ConditionalWaitStep extends Step {
private final Function<ClusterState, Boolean> condition;
public ConditionalWaitStep(String name, String index, String phase, String action, Function<ClusterState, Boolean> condition) {
super(name, action, phase, index);
this.condition = condition;
}
@Override
public StepResult execute(ClusterState currentState) {
boolean isComplete = condition.apply(currentState);
return new StepResult(String.valueOf(isComplete), null, currentState, true, isComplete);
}
}

View File

@ -6,8 +6,8 @@
package org.elasticsearch.xpack.core.indexlifecycle;
import org.apache.logging.log4j.Logger;
import org.elasticsearch.action.ActionListener;
import org.elasticsearch.action.admin.indices.delete.DeleteIndexRequest;
import org.elasticsearch.action.admin.indices.delete.DeleteIndexRequestBuilder;
import org.elasticsearch.action.admin.indices.delete.DeleteIndexResponse;
import org.elasticsearch.client.Client;
import org.elasticsearch.cluster.service.ClusterService;
@ -19,8 +19,12 @@ import org.elasticsearch.common.xcontent.ObjectParser;
import org.elasticsearch.common.xcontent.XContentBuilder;
import org.elasticsearch.common.xcontent.XContentParser;
import org.elasticsearch.index.Index;
import org.elasticsearch.threadpool.ThreadPool;
import java.io.IOException;
import java.util.Collections;
import java.util.List;
import java.util.function.LongSupplier;
/**
* A {@link LifecycleAction} which deletes the index.
@ -63,18 +67,11 @@ public class DeleteAction implements LifecycleAction {
}
@Override
public void execute(Index index, Client client, ClusterService clusterService, Listener listener) {
client.admin().indices().delete(new DeleteIndexRequest(index.getName()), new ActionListener<DeleteIndexResponse>() {
@Override
public void onResponse(DeleteIndexResponse deleteIndexResponse) {
listener.onSuccess(true);
}
@Override
public void onFailure(Exception e) {
listener.onFailure(e);
}
});
public List<Step> toSteps(String phase, Index index, Client client, ThreadPool threadPool, LongSupplier nowSupplier) {
String indexName = index.getName();
return Collections.singletonList(new ClientStep<DeleteIndexRequestBuilder, DeleteIndexResponse>( "delete",
NAME, phase, indexName, client.admin().indices().prepareDelete(indexName),
clusterState -> clusterState.metaData().hasIndex(indexName), response -> true));
}
@Override

View File

@ -8,10 +8,16 @@ package org.elasticsearch.xpack.core.indexlifecycle;
import org.elasticsearch.action.ActionListener;
import org.elasticsearch.action.ActionResponse;
import org.elasticsearch.action.admin.indices.forcemerge.ForceMergeRequest;
import org.elasticsearch.action.admin.indices.forcemerge.ForceMergeRequestBuilder;
import org.elasticsearch.action.admin.indices.forcemerge.ForceMergeResponse;
import org.elasticsearch.action.admin.indices.segments.IndicesSegmentResponse;
import org.elasticsearch.action.admin.indices.segments.IndicesSegmentsRequest;
import org.elasticsearch.action.admin.indices.segments.IndicesSegmentsRequestBuilder;
import org.elasticsearch.action.admin.indices.settings.put.UpdateSettingsRequest;
import org.elasticsearch.client.Client;
import org.elasticsearch.cluster.ClusterState;
import org.elasticsearch.cluster.metadata.IndexMetaData;
import org.elasticsearch.cluster.metadata.MetaData;
import org.elasticsearch.cluster.service.ClusterService;
import org.elasticsearch.common.ParseField;
import org.elasticsearch.common.Strings;
@ -23,11 +29,14 @@ import org.elasticsearch.common.xcontent.XContentBuilder;
import org.elasticsearch.common.xcontent.XContentParser;
import org.elasticsearch.index.Index;
import org.elasticsearch.rest.RestStatus;
import org.elasticsearch.threadpool.ThreadPool;
import java.io.IOException;
import java.util.Arrays;
import java.util.List;
import java.util.Objects;
import java.util.function.Consumer;
import java.util.function.LongSupplier;
import java.util.stream.StreamSupport;
/**
@ -87,107 +96,37 @@ public class ForceMergeAction implements LifecycleAction {
return builder;
}
/**
* Helper method to check if a force-merge is necessary based on {@code maxNumSegments} and then calls
* the next {@code action}.
*
* @param index The specific index to check the segments of
* @param client The client to execute the transport actions
* @param listener The listener to call onFailure on if an exception occurs when executing the {@link IndicesSegmentsRequest}
* @param nextAction The next action to execute if there are too many segments and force-merge is appropriate
* @param skipToAction The next action to execute if there aren't too many segments
*/
void checkSegments(Index index, Client client, Listener listener, Consumer<ActionResponse> nextAction,
Consumer<ActionResponse> skipToAction) {
client.admin().indices().segments(new IndicesSegmentsRequest(index.getName()), ActionListener.wrap(r -> {
boolean hasTooManySegments = StreamSupport.stream(r.getIndices().get(index.getName()).spliterator(), false)
.anyMatch(iss -> Arrays.stream(iss.getShards()).anyMatch(p -> p.getSegments().size() > maxNumSegments));
if (nextAction != null && hasTooManySegments && RestStatus.OK.equals(r.getStatus())) {
nextAction.accept(r);
} else {
skipToAction.accept(r);
}
}, listener::onFailure));
}
/**
* Helper method to execute the force-merge
*
* @param index The specific index to force-merge
* @param client The client to execute the transport actions
* @param listener The listener to call onFailure on if an exception occurs when executing the {@link ForceMergeRequest}
* @param nextAction The next action to execute if the force-merge is successful
*/
void forceMerge(Index index, Client client, Listener listener, Consumer<ActionResponse> nextAction) {
ForceMergeRequest forceMergeRequest = new ForceMergeRequest(index.getName()).maxNumSegments(maxNumSegments);
client.admin().indices().forceMerge(forceMergeRequest, ActionListener.wrap(r -> {
if (RestStatus.OK.equals(r.getStatus())) {
nextAction.accept(r);
}
}, listener::onFailure));
}
/**
* Helper method to prepare the index for force-merging by making it read-only
*
* @param index The specific index to set as read-only
* @param client The client to execute the transport actions
* @param listener The listener to call onFailure on if an exception occurs when executing the {@link UpdateSettingsRequest}
* @param nextAction The next action to execute if updating the setting is successful
*/
void updateBlockWriteSettingToReadOnly(Index index, Client client, Listener listener, Consumer<ActionResponse> nextAction) {
UpdateSettingsRequest updateSettingsRequest = new UpdateSettingsRequest(Settings.builder()
.put(IndexMetaData.SETTING_BLOCKS_WRITE, true).build(), index.getName());
client.admin().indices().updateSettings(updateSettingsRequest, ActionListener.wrap(response -> {
nextAction.accept(response);
}, listener::onFailure));
}
/**
* Helper method to return the index back to read-write mode since force-merging was successful
*
* @param index The specific index to set back as read-write
* @param client The client to execute the transport actions
* @param listener The listener to return a final response to for this {@link ForceMergeAction}.
*/
void updateBlockWriteSettingToReadWrite(Index index, Client client, Listener listener) {
UpdateSettingsRequest updateSettingsRequest = new UpdateSettingsRequest(Settings.builder()
.put(IndexMetaData.SETTING_BLOCKS_WRITE, false).build(), index.getName());
client.admin().indices().updateSettings(updateSettingsRequest, ActionListener.wrap(
response -> listener.onSuccess(true), listener::onFailure));
}
@Override
public void execute(Index index, Client client, ClusterService clusterService, Listener listener) {
boolean isReadOnly = clusterService.state().metaData().indices().get(index.getName()).getSettings()
.getAsBoolean(IndexMetaData.SETTING_BLOCKS_WRITE, false);
if (isReadOnly) {
// index is already read-only, so just check if a force-merge is necessary and set back
// to read-write whether a force-merge is necessary or not.
checkSegments(index, client, listener, r1 ->
forceMerge(index, client, listener,
// after a successful force-merge, return the index to read-write
r2 -> updateBlockWriteSettingToReadWrite(index, client, listener)),
r3 -> updateBlockWriteSettingToReadWrite(index, client, listener));
} else {
// first check if a force-merge is appropriate
checkSegments(index, client, listener,
// if appropriate, set the index to read-only
r1 -> updateBlockWriteSettingToReadOnly(index, client, listener,
// once the index is read-only, run a force-merge on it
r2 -> forceMerge(index, client, listener,
// after a successful force-merge, return the index to read-write
r3 -> updateBlockWriteSettingToReadWrite(index, client, listener))),
r4 -> {
if (isReadOnly) {
updateBlockWriteSettingToReadWrite(index, client, listener);
} else {
listener.onSuccess(true);
}
});
}
public List<Step> toSteps(String phase, Index index, Client client, ThreadPool threadPool, LongSupplier nowSupplier) {
ClusterStateUpdateStep readOnlyStep = new ClusterStateUpdateStep(
"read_only", NAME, phase, index.getName(), (currentState) -> {
Settings readOnlySettings = Settings.builder().put(IndexMetaData.SETTING_BLOCKS_WRITE, true).build();
return ClusterState.builder(currentState).metaData(MetaData.builder(currentState.metaData())
.updateSettings(readOnlySettings, index.getName())).build();
});
ClientStep<IndicesSegmentsRequestBuilder, IndicesSegmentResponse> segmentCount = new ClientStep<>( "segment_count",
NAME, phase, index.getName(),
client.admin().indices().prepareSegments(index.getName()),
currentState -> false, response -> {
// check if has too many segments
return StreamSupport.stream(response.getIndices().get(index.getName()).spliterator(), false)
.anyMatch(iss -> Arrays.stream(iss.getShards()).anyMatch(p -> p.getSegments().size() > maxNumSegments));
});
ClientStep forceMerge = new ClientStep<ForceMergeRequestBuilder, ForceMergeResponse>( "force_merge",
NAME, phase, index.getName(),
client.admin().indices().prepareForceMerge(index.getName()).setMaxNumSegments(maxNumSegments),
currentState -> false, response -> RestStatus.OK.equals(response.getStatus()));
ClusterStateUpdateStep readWriteStep = new ClusterStateUpdateStep(
"read_only", NAME, phase, index.getName(), (currentState) -> {
Settings readOnlySettings = Settings.builder().put(IndexMetaData.SETTING_BLOCKS_WRITE, false).build();
return ClusterState.builder(currentState).metaData(MetaData.builder(currentState.metaData())
.updateSettings(readOnlySettings, index.getName())).build();
});
return Arrays.asList(readOnlyStep, segmentCount, forceMerge, readWriteStep);
}
@Override

View File

@ -1,108 +0,0 @@
/*
* 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;
/**
* Provides the context to a {@link LifecyclePolicy} for a particular target.
* This context provides the state of the lifecycle target (hereafter referred
* to as the target) as well as allow operations to be performed on the target.
*/
public interface IndexLifecycleContext {
/**
* Sets the phase for the target and calls the provided callback. Note that
* this will also set the action name to an empty {@link String}.
*
* @param phase
* the name of the phase to be set.
* @param listener
* a {@link Listener} to call after the operation.
*/
void setPhase(String phase, Listener listener);
/**
* Sets the action for the target and calls the provided callback.
*
* @param action
* the name of the action to be set.
* @param listener
* a {@link Listener} to call after the operation.
*/
void setAction(String action, Listener listener);
/**
* @return the current {@link LifecycleAction} name for the target.
*/
String getAction();
/**
* @return the start time of the current {@link LifecycleAction}.
*/
long getActionTime();
/**
* @return the current {@link Phase} name for the target.
*/
String getPhase();
/**
* @return the start time of the current {@link Phase}.
*/
long getPhaseTime();
/**
* @return the name of the target.
*/
String getLifecycleTarget();
/**
* @return the current index context's replica count.
*/
int getNumberOfReplicas();
/**
* Determines whether the target is able to move to the provided
* {@link Phase}
*
* @param phase
* the {@link Phase} to test
* @return <code>true</code> iff the target is ready to move to the
* {@link Phase}.
*/
boolean canExecute(Phase phase);
/**
* Executes the provided {@link LifecycleAction} passing the relevant target
* state to it.
*
* @param action
* the {@link LifecycleAction} to execute.
* @param listener
* a {@link LifecycleAction.Listener} to pass to the
* {@link LifecycleAction}.
*/
void executeAction(LifecycleAction action, LifecycleAction.Listener listener);
/**
* A callback for use when setting phase or action names.
*/
interface Listener {
/**
* Called if the call to set the action/phase name was successful.
*/
void onSuccess();
/**
* Called if there was an exception when setting the action or phase
* name.
*
* @param e
* the exception that caused the failure
*/
void onFailure(Exception e);
}
}

View File

@ -10,60 +10,19 @@ import org.elasticsearch.cluster.service.ClusterService;
import org.elasticsearch.common.io.stream.NamedWriteable;
import org.elasticsearch.common.xcontent.ToXContentObject;
import org.elasticsearch.index.Index;
import org.elasticsearch.threadpool.ThreadPool;
import java.util.List;
import java.util.function.LongSupplier;
/**
* Executes an action on an index related to its lifecycle.
*/
public interface LifecycleAction extends ToXContentObject, NamedWriteable {
/**
* Checks the current state of the {@link LifecycleAction} and progresses
* the action towards completion . Note that a {@link LifecycleAction} may
* require multiple calls to this method before it is complete. Equally this
* method may do nothing if it needs to wait for something to complete
* before proceeding.
*
* @param index
* the {@link Index} on which to perform the action.
* @param client
* the {@link Client} to use for making changes to the index.
* @param listener
* a {@link Listener} to call when this call completes.
*/
void execute(Index index, Client client, ClusterService clusterService, Listener listener);
List<Step> toSteps(String phase, Index index, Client client, ThreadPool threadPool, LongSupplier nowSupplier);
default boolean indexSurvives() {
return true;
}
/**
* A callback for when a {@link LifecycleAction} finishes executing
*/
interface Listener {
/**
* Called if the call to
* {@link LifecycleAction#execute(Index, Client, ClusterService, Listener)}
* was successful
*
* @param completed
* <code>true</code> iff the {@link LifecycleAction} is now
* complete and requires no more calls to
* {@link LifecycleAction#execute(Index, Client, ClusterService, Listener)
* execute(Index, Client, Listener)}.
*/
void onSuccess(boolean completed);
/**
* Called if there was an exception during
* {@link LifecycleAction#execute(Index, Client, ClusterService, Listener)}.
* Note that even the call to
* {@link LifecycleAction#execute(Index, Client, ClusterService, Listener)}
* may be retried even after this method is called.
*
* @param e
* the exception that caused the failure
*/
void onFailure(Exception e);
}
}

View File

@ -6,26 +6,33 @@
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.xpack.core.indexlifecycle.IndexLifecycleContext.Listener;
import java.io.IOException;
import java.util.Collections;
import java.util.List;
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;
/**
@ -78,7 +85,8 @@ public class LifecyclePolicy extends AbstractDiffable<LifecyclePolicy>
}
this.name = name;
this.phases = phases;
this.type.validate(phases.values());
// TODO(talevy): return validation
//this.type.validate(phases.values());
}
/**
@ -136,48 +144,49 @@ public class LifecyclePolicy extends AbstractDiffable<LifecyclePolicy>
return builder;
}
/**
* Checks the current state and executes the appropriate {@link Phase}.
*
* @param context
* the {@link IndexLifecycleContext} to use to execute the
* {@link LifecyclePolicy}.
*/
public void execute(IndexLifecycleContext context) {
String currentPhaseName = context.getPhase();
boolean currentPhaseActionsComplete = context.getAction().equals(Phase.PHASE_COMPLETED);
String indexName = context.getLifecycleTarget();
Phase currentPhase = phases.get(currentPhaseName);
if (Strings.isNullOrEmpty(currentPhaseName) || currentPhaseActionsComplete) {
Phase nextPhase = type.nextPhase(phases, currentPhase);
// We only want to execute the phase if the conditions for executing are met (e.g. the index is old enough)
if (nextPhase != null && context.canExecute(nextPhase)) {
String nextPhaseName = nextPhase.getName();
// Set the phase on the context to this phase so we know where we are next time we execute
context.setPhase(nextPhaseName, new Listener() {
@Override
public void onSuccess() {
logger.info("Successfully initialised phase [" + nextPhaseName + "] for index [" + indexName + "]");
// We might as well execute the phase now rather than waiting for execute to be called again
nextPhase.execute(context, type.getActionProvider(context, nextPhase));
}
@Override
public void onFailure(Exception e) {
logger.error("Failed to initialised phase [" + nextPhaseName + "] for index [" + indexName + "]", e);
}
});
}
} else {
// If we have already seen this index and the action is not PHASE_COMPLETED then we just need to execute the current phase again
if (currentPhase == null) {
throw new IllegalStateException("Current phase [" + currentPhaseName + "] not found in lifecycle ["
+ getName() + "] for index [" + indexName + "]");
public StepResult execute(List<Step> steps, ClusterState currentState, IndexMetaData indexMetaData, Client client, LongSupplier nowSupplier) {
StepResult lastStepResult = null;
ClusterState updatedState = currentState;
for (int i = getNextStepIdx(steps, indexMetaData); i < steps.size(); i++) {
lastStepResult = steps.get(i).execute(updatedState);
if (lastStepResult.isComplete() && lastStepResult.indexSurvived()) {
if (i < steps.size() - 1) {
Step nextStep = steps.get(i + 1);
long now = nowSupplier.getAsLong();
// fetch details about next step to run and update the cluster state with this information
Settings newLifecyclePhaseSettings = Settings.builder()
.put(LifecycleSettings.LIFECYCLE_PHASE, nextStep.getPhase())
.put(LifecycleSettings.LIFECYCLE_PHASE_TIME, now)
.put(LifecycleSettings.LIFECYCLE_ACTION_TIME, now)
.put(LifecycleSettings.LIFECYCLE_ACTION, nextStep.getAction())
.put(LifecycleSettings.LIFECYCLE_STEP_TIME, now)
.put(LifecycleSettings.LIFECYCLE_STEP, nextStep.getName())
.build();
updatedState = ClusterState.builder(lastStepResult.getClusterState())
.metaData(MetaData.builder(lastStepResult.getClusterState().metaData())
.updateSettings(newLifecyclePhaseSettings)).build();
lastStepResult = new StepResult(lastStepResult, updatedState);
}
} else {
currentPhase.execute(context, type.getActionProvider(context, currentPhase));
break;
}
}
return lastStepResult;
}
private int getNextStepIdx(List<Step> steps, IndexMetaData indexMetaData) {
String step = indexMetaData.getSettings().get(LifecycleSettings.LIFECYCLE_STEP);
if (step == null) {
return 0;
}
for (int i = 0; i < steps.size(); i++) {
if (steps.get(i).getName().equals(step)) {
return i;
}
}
return steps.size();
}
@Override

View File

@ -16,9 +16,11 @@ public class LifecycleSettings {
public static final String LIFECYCLE_NAME = "index.lifecycle.name";
public static final String LIFECYCLE_PHASE = "index.lifecycle.phase";
public static final String LIFECYCLE_ACTION = "index.lifecycle.action";
public static final String LIFECYCLE_STEP = "index.lifecycle.step";
public static final String LIFECYCLE_INDEX_CREATION_DATE = "index.lifecycle.date";
public static final String LIFECYCLE_PHASE_TIME = "index.lifecycle.phase_time";
public static final String LIFECYCLE_ACTION_TIME = "index.lifecycle.action_time";
public static final String LIFECYCLE_STEP_TIME = "index.lifecycle.step_time";
// NORELEASE: we should probably change the default to something other than three seconds for initial release
public static final Setting<TimeValue> LIFECYCLE_POLL_INTERVAL_SETTING = Setting.positiveTimeSetting(LIFECYCLE_POLL_INTERVAL,
@ -29,10 +31,14 @@ public class LifecycleSettings {
Setting.Property.Dynamic, Setting.Property.IndexScope);
public static final Setting<String> LIFECYCLE_ACTION_SETTING = Setting.simpleString(LIFECYCLE_ACTION,
Setting.Property.Dynamic, Setting.Property.IndexScope);
public static final Setting<String> LIFECYCLE_STEP_SETTING = Setting.simpleString(LIFECYCLE_ACTION,
Setting.Property.Dynamic, Setting.Property.IndexScope);
public static final Setting<Long> LIFECYCLE_INDEX_CREATION_DATE_SETTING = Setting.longSetting(LIFECYCLE_INDEX_CREATION_DATE,
-1L, -1L, Setting.Property.Dynamic, Setting.Property.IndexScope);
public static final Setting<Long> LIFECYCLE_PHASE_TIME_SETTING = Setting.longSetting(LIFECYCLE_PHASE_TIME,
-1L, -1L, Setting.Property.Dynamic, Setting.Property.IndexScope);
public static final Setting<Long> LIFECYCLE_ACTION_TIME_SETTING = Setting.longSetting(LIFECYCLE_ACTION_TIME,
-1L, -1L, Setting.Property.Dynamic, Setting.Property.IndexScope);
public static final Setting<Long> LIFECYCLE_STEP_TIME_SETTING = Setting.longSetting(LIFECYCLE_ACTION_TIME,
-1L, -1L, Setting.Property.Dynamic, Setting.Property.IndexScope);
}

View File

@ -47,14 +47,4 @@ public interface LifecycleType extends NamedWriteable {
* @return the {@link LifecyclePolicy} type.
*/
String getType();
/**
* @param context
* the index lifecycle context for this phase at the time of
* execution
* @param phase
* the current phase for which to provide an action provider
* @return the action provider
*/
LifecyclePolicy.NextActionProvider getActionProvider(IndexLifecycleContext context, Phase phase);
}

View File

@ -6,6 +6,7 @@
package org.elasticsearch.xpack.core.indexlifecycle;
import org.apache.logging.log4j.Logger;
import org.elasticsearch.client.Client;
import org.elasticsearch.common.ParseField;
import org.elasticsearch.common.Strings;
import org.elasticsearch.common.io.stream.StreamInput;
@ -18,13 +19,17 @@ import org.elasticsearch.common.xcontent.ObjectParser.ValueType;
import org.elasticsearch.common.xcontent.ToXContentObject;
import org.elasticsearch.common.xcontent.XContentBuilder;
import org.elasticsearch.common.xcontent.XContentParser;
import org.elasticsearch.xpack.core.indexlifecycle.IndexLifecycleContext.Listener;
import org.elasticsearch.index.Index;
import org.elasticsearch.threadpool.ThreadPool;
import java.io.IOException;
import java.util.List;
import java.util.Map;
import java.util.Objects;
import java.util.TreeMap;
import java.util.function.LongSupplier;
import java.util.stream.Collectors;
import java.util.stream.Stream;
import static org.elasticsearch.xpack.core.indexlifecycle.ObjectParserUtils.convertListToMapValues;
@ -131,126 +136,12 @@ public class Phase implements ToXContentObject, Writeable {
return actions;
}
/**
* Checks the current state and executes the appropriate
* {@link LifecycleAction}.
*
* @param context
* the {@link IndexLifecycleContext} to use to execute the
* {@link Phase}.
* @param nextActionProvider
* the next action provider
*/
protected void execute(IndexLifecycleContext context, LifecyclePolicy.NextActionProvider nextActionProvider) {
String currentActionName = context.getAction();
String indexName = context.getLifecycleTarget();
if (Strings.isNullOrEmpty(currentActionName)) {
// This is is the first time this phase has been called so get the first action and execute it.
String firstActionName;
LifecycleAction firstAction;
if (actions.isEmpty()) {
// There are no actions in this phase so use the PHASE_COMPLETE action name.
firstAction = null;
firstActionName = PHASE_COMPLETED;
} else {
firstAction = nextActionProvider.next(null);
firstActionName = firstAction.getWriteableName();
}
// Set the action on the context to this first action so we know where we are next time we execute
context.setAction(firstActionName, new Listener() {
@Override
public void onSuccess() {
logger.info("Successfully initialised action [" + firstActionName + "] for index [" + indexName + "]");
// Now execute the action unless its PHASE_COMPLETED
if (firstActionName.equals(PHASE_COMPLETED) == false) {
executeAction(context, indexName, firstAction, nextActionProvider);
}
}
@Override
public void onFailure(Exception e) {
// Something went wrong so log the error and hopefully it will succeed next time execute
// is called. NORELEASE can we do better here?
logger.error("Failed to initialised action [" + firstActionName + "] for index [" + indexName + "]", e);
}
});
} else if (currentActionName.equals(PHASE_COMPLETED) == false) {
// We have an action name and its not PHASE COMPLETED so we need to execute the action
// First find the action in the actions map.
if (actions.containsKey(currentActionName) == false) {
throw new IllegalStateException("Current action [" + currentActionName + "] not found in phase ["
+ getName() + "] for index [" + indexName + "]");
}
LifecycleAction currentAction = actions.get(currentActionName);
// then execute the action
executeAction(context, indexName, currentAction, nextActionProvider);
}
}
private void executeAction(IndexLifecycleContext context, String indexName, LifecycleAction action,
LifecyclePolicy.NextActionProvider nextActionProvider) {
String actionName = action.getWriteableName();
context.executeAction(action, new LifecycleAction.Listener() {
@Override
public void onSuccess(boolean completed) {
if (completed) {
// Since we completed the current action move to the next
// action if the index survives this action
if (action.indexSurvives()) {
logger.info("Action [" + actionName + "] for index [" + indexName + "] complete, moving to next action");
moveToNextAction(context, indexName, action, nextActionProvider);
} else {
logger.info("Action [" + actionName + "] for index [" + indexName + "] complete");
}
} else {
logger.info("Action [" + actionName + "] for index [" + indexName + "] executed sucessfully but is not yet complete");
}
}
@Override
public void onFailure(Exception e) {
logger.info("Action [" + actionName + "] for index [" + indexName + "] failed", e);
}
});
}
private void moveToNextAction(IndexLifecycleContext context, String indexName, LifecycleAction currentAction,
LifecyclePolicy.NextActionProvider nextActionProvider) {
LifecycleAction nextAction = nextActionProvider.next(currentAction);
if (nextAction != null) {
context.setAction(nextAction.getWriteableName(), new Listener() {
@Override
public void onSuccess() {
logger.info("Successfully initialised action [" + nextAction.getWriteableName() + "] in phase [" + getName()
+ "] for index [" + indexName + "]");
// We might as well execute the new action now rather than waiting for execute to be called again
execute(context, nextActionProvider);
}
@Override
public void onFailure(Exception e) {
logger.error("Failed to initialised action [" + nextAction.getWriteableName() + "] in phase [" + getName()
+ "] for index [" + indexName + "]", e);
}
});
} else {
// There is no next action so set the action to PHASE_COMPLETED
context.setAction(Phase.PHASE_COMPLETED, new Listener() {
@Override
public void onSuccess() {
logger.info("Successfully completed phase [" + getName() + "] for index [" + indexName + "]");
}
@Override
public void onFailure(Exception e) {
logger.error("Failed to complete phase [" + getName() + "] for index [" + indexName + "]", e);
}
});
}
public List<Step> toSteps(Index index, long indexLifecycleCreationDate, Client client, ThreadPool threadPool, LongSupplier nowSupplier) {
// TODO(talevy) phase needs to know indexLifecycleStartTime
PhaseAfterStep phaseAfterStep = new PhaseAfterStep(threadPool, indexLifecycleCreationDate, nowSupplier, after,
"phase_start", index.getName(), getName(), null);
return Stream.concat(Stream.of(phaseAfterStep), actions.values().stream()
.flatMap(a -> a.toSteps(name, index, client, threadPool, nowSupplier).stream())).collect(Collectors.toList());
}
@Override

View File

@ -0,0 +1,49 @@
/*
* 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.common.unit.TimeValue;
import org.elasticsearch.threadpool.ThreadPool;
import org.elasticsearch.xpack.core.indexlifecycle.Step;
import org.elasticsearch.xpack.core.indexlifecycle.StepResult;
import java.util.concurrent.CompletableFuture;
import java.util.function.BooleanSupplier;
import java.util.function.LongSupplier;
public class PhaseAfterStep extends Step {
private final CompletableFuture<StepResult> timeUp;
private final ThreadPool threadPool;
private final long indexCreationDate;
private final LongSupplier nowSupplier;
private final TimeValue after;
public PhaseAfterStep(ThreadPool threadPool, long indexCreationDate, LongSupplier nowSupplier, TimeValue after, String name,
String index, String phase, String action) {
super(name, action, phase, index);
this.threadPool = threadPool;
this.indexCreationDate = indexCreationDate;
this.nowSupplier = nowSupplier;
this.timeUp = new CompletableFuture<>();
this.after = after;
}
public StepResult execute(ClusterState currentState) {
LongSupplier elapsed = () -> nowSupplier.getAsLong() - indexCreationDate;
BooleanSupplier isReady = () -> after.getSeconds() <= elapsed.getAsLong();
if (isReady.getAsBoolean()) {
return new StepResult("phase starting", null, currentState, true, true);
} else {
threadPool.schedule(TimeValue.timeValueSeconds(elapsed.getAsLong()), ThreadPool.Names.GENERIC, () -> {
if (after.getSeconds() <= elapsed.getAsLong()) {
// IndexLifecycleService.triggerPolicies()
}
});
return new StepResult("phase-check-rescheduled", null, currentState, true, false);
}
}
}

View File

@ -10,7 +10,9 @@ import org.elasticsearch.action.admin.indices.settings.put.UpdateSettingsRequest
import org.elasticsearch.action.admin.indices.settings.put.UpdateSettingsResponse;
import org.elasticsearch.action.support.ActiveShardCount;
import org.elasticsearch.client.Client;
import org.elasticsearch.cluster.ClusterState;
import org.elasticsearch.cluster.metadata.IndexMetaData;
import org.elasticsearch.cluster.metadata.MetaData;
import org.elasticsearch.cluster.service.ClusterService;
import org.elasticsearch.common.ParseField;
import org.elasticsearch.common.Strings;
@ -21,9 +23,13 @@ import org.elasticsearch.common.xcontent.ConstructingObjectParser;
import org.elasticsearch.common.xcontent.XContentBuilder;
import org.elasticsearch.common.xcontent.XContentParser;
import org.elasticsearch.index.Index;
import org.elasticsearch.threadpool.ThreadPool;
import java.io.IOException;
import java.util.Arrays;
import java.util.List;
import java.util.Objects;
import java.util.function.LongSupplier;
import static org.elasticsearch.common.xcontent.ConstructingObjectParser.constructorArg;
@ -77,28 +83,14 @@ public class ReplicasAction implements LifecycleAction {
}
@Override
public void execute(Index index, Client client, ClusterService clusterService, Listener listener) {
IndexMetaData idxMeta = clusterService.state().metaData().getIndexSafe(index);
int currentNumberReplicas = idxMeta.getNumberOfReplicas();
if (currentNumberReplicas == numberOfReplicas) {
boolean isAllocationCompleted = ActiveShardCount.ALL.enoughShardsActive(clusterService.state(), index.getName());
listener.onSuccess(isAllocationCompleted);
} else {
UpdateSettingsRequest updateSettingsRequest = new UpdateSettingsRequest(index.getName())
.settings(Settings.builder().put(IndexMetaData.SETTING_NUMBER_OF_REPLICAS, numberOfReplicas));
client.admin().indices().updateSettings(updateSettingsRequest, new ActionListener<UpdateSettingsResponse>() {
@Override
public void onResponse(UpdateSettingsResponse response) {
listener.onSuccess(false);
}
@Override
public void onFailure(Exception e) {
listener.onFailure(e);
}
});
}
public List<Step> toSteps(String phase, Index index, Client client, ThreadPool threadPool, LongSupplier nowSupplier) {
ClusterStateUpdateStep updateAllocationSettings = new ClusterStateUpdateStep(
"update_replica_count", NAME, phase, index.getName(), (currentState) ->
ClusterState.builder(currentState).metaData(MetaData.builder(currentState.metaData())
.updateNumberOfReplicas(numberOfReplicas, index.getName())).build());
ConditionalWaitStep isReplicatedCheck = new ConditionalWaitStep("wait_replicas_allocated", NAME,
phase, index.getName(), (currentState) -> ActiveShardCount.ALL.enoughShardsActive(currentState, index.getName()) );
return Arrays.asList(updateAllocationSettings, isReplicatedCheck);
}
public int getNumberOfReplicas() {

View File

@ -6,7 +6,6 @@
package org.elasticsearch.xpack.core.indexlifecycle;
import org.apache.logging.log4j.Logger;
import org.elasticsearch.ExceptionsHelper;
import org.elasticsearch.action.ActionListener;
import org.elasticsearch.action.admin.indices.rollover.RolloverRequest;
import org.elasticsearch.action.admin.indices.rollover.RolloverResponse;
@ -24,9 +23,13 @@ import org.elasticsearch.common.xcontent.ObjectParser.ValueType;
import org.elasticsearch.common.xcontent.XContentBuilder;
import org.elasticsearch.common.xcontent.XContentParser;
import org.elasticsearch.index.Index;
import org.elasticsearch.threadpool.ThreadPool;
import java.io.IOException;
import java.util.Collections;
import java.util.List;
import java.util.Objects;
import java.util.function.LongSupplier;
/**
* A {@link LifecycleAction} which deletes the index.
@ -150,32 +153,38 @@ public class RolloverAction implements LifecycleAction {
}
@Override
public void execute(Index index, Client client, ClusterService clusterService, Listener listener) {
if (clusterService.state().getMetaData().index(index.getName()).getAliases().containsKey(alias)) {
RolloverRequest rolloverRequest = new RolloverRequest(alias, null);
if (maxAge != null) {
rolloverRequest.addMaxIndexAgeCondition(maxAge);
}
if (maxSize != null) {
rolloverRequest.addMaxIndexSizeCondition(maxSize);
}
if (maxDocs != null) {
rolloverRequest.addMaxIndexDocsCondition(maxDocs);
}
client.admin().indices().rolloverIndex(rolloverRequest, new ActionListener<RolloverResponse>() {
@Override
public void onResponse(RolloverResponse rolloverResponse) {
listener.onSuccess(rolloverResponse.isRolledOver());
}
public List<Step> toSteps(String phase, Index index, Client client, ThreadPool threadPool, LongSupplier nowSupplier) {
return Collections.emptyList();
// ConditionalWaitStep wait = new ConditionalWaitStep(clusterService, "wait_for_rollover", index.getName(), phase, action, (clusterState) -> {
// // TODO(talevy): actually, needs to RolloverRequest with dryrun to get the appropriate data; clusterState is not enough...
// // can potentially reduce down to original problem with RolloverRequest...1minute...RolloverRequest...1minute... probably ok?
// if (clusterService.state().getMetaData().index(index.getName()).getAliases().containsKey(alias)) {
// RolloverRequest rolloverRequest = new RolloverRequest(alias, null);
// if (maxAge != null) {
// rolloverRequest.addMaxIndexAgeCondition(maxAge);
// }
// if (maxSize != null) {
// rolloverRequest.addMaxIndexSizeCondition(maxSize);
// }
// if (maxDocs != null) {
// rolloverRequest.addMaxIndexDocsCondition(maxDocs);
// }
// client.admin().indices().rolloverIndex(rolloverRequest, new ActionListener<RolloverResponse>() {
// @Override
// public void onResponse(RolloverResponse rolloverResponse) {
// return rolloverResponse.isRolledOver();
// }
//
// @Override
// public void onFailure(Exception e) {
// listener.onFailure(e);
// }
// });
// } else {
// listener.onSuccess(true);
// }
// });
@Override
public void onFailure(Exception e) {
listener.onFailure(e);
}
});
} else {
listener.onSuccess(true);
}
}
@Override

View File

@ -10,27 +10,39 @@ import org.elasticsearch.action.ActionListener;
import org.elasticsearch.action.admin.indices.alias.Alias;
import org.elasticsearch.action.admin.indices.alias.IndicesAliasesRequest;
import org.elasticsearch.action.admin.indices.create.CreateIndexRequest;
import org.elasticsearch.action.admin.indices.create.CreateIndexRequestBuilder;
import org.elasticsearch.action.admin.indices.settings.put.UpdateSettingsRequest;
import org.elasticsearch.action.admin.indices.shrink.ResizeRequest;
import org.elasticsearch.action.admin.indices.shrink.ResizeRequestBuilder;
import org.elasticsearch.action.admin.indices.shrink.ResizeResponse;
import org.elasticsearch.action.support.ActiveShardCount;
import org.elasticsearch.client.Client;
import org.elasticsearch.cluster.ClusterState;
import org.elasticsearch.cluster.metadata.AliasMetaData;
import org.elasticsearch.cluster.metadata.IndexMetaData;
import org.elasticsearch.cluster.metadata.MetaData;
import org.elasticsearch.cluster.routing.allocation.decider.AllocationDeciders;
import org.elasticsearch.cluster.routing.allocation.decider.FilterAllocationDecider;
import org.elasticsearch.cluster.service.ClusterService;
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.ClusterSettings;
import org.elasticsearch.common.settings.Settings;
import org.elasticsearch.common.xcontent.ConstructingObjectParser;
import org.elasticsearch.common.xcontent.XContentBuilder;
import org.elasticsearch.common.xcontent.XContentParser;
import org.elasticsearch.index.Index;
import org.elasticsearch.threadpool.ThreadPool;
import java.io.IOException;
import java.util.Arrays;
import java.util.Collections;
import java.util.List;
import java.util.Objects;
import java.util.function.LongSupplier;
/**
* A {@link LifecycleAction} which shrinks the index.
@ -49,6 +61,7 @@ public class ShrinkAction implements LifecycleAction {
}
private int numberOfShards;
private AllocationDeciders allocationDeciders;
public static ShrinkAction parse(XContentParser parser) throws IOException {
return PARSER.parse(parser, new CreateIndexRequest());
@ -59,6 +72,9 @@ public class ShrinkAction implements LifecycleAction {
throw new IllegalArgumentException("[" + NUMBER_OF_SHARDS_FIELD.getPreferredName() + "] must be greater than 0");
}
this.numberOfShards = numberOfShards;
FilterAllocationDecider decider = new FilterAllocationDecider(Settings.EMPTY,
new ClusterSettings(Settings.EMPTY, ClusterSettings.BUILT_IN_CLUSTER_SETTINGS));
this.allocationDeciders = new AllocationDeciders(Settings.EMPTY, Collections.singletonList(decider));
}
public ShrinkAction(StreamInput in) throws IOException {
@ -87,103 +103,85 @@ public class ShrinkAction implements LifecycleAction {
return builder;
}
/**
* Executes the Shrink Action.
*
* This function first checks whether the target shrunk index exists already, if it does not, then
* it will set the index to read-only and issue a resize request.
*
* Since the shrink response is not returned after a successful shrunk operation, we must poll to see if
* all the shards of the newly shrunk index are initialized. If so, then we can return the index to read-write
* and tell the listener that we have completed the action.
*
* @param index
* the {@link Index} on which to perform the action.
* @param client
* the {@link Client} to use for making changes to the index.
* @param clusterService
* the {@link ClusterService} to retrieve the current cluster state from.
* @param listener
* the {@link LifecycleAction.Listener} to return completion or failure responses to.
*/
@Override
public void execute(Index index, Client client, ClusterService clusterService, Listener listener) {
String targetIndexName = SHRUNK_INDEX_NAME_PREFIX + index.getName();
ClusterState clusterState = clusterService.state();
IndexMetaData indexMetaData = clusterState.metaData().index(index.getName());
String sourceIndexName = IndexMetaData.INDEX_SHRINK_SOURCE_NAME.get(indexMetaData.getSettings());
boolean isShrunkIndex = index.getName().equals(SHRUNK_INDEX_NAME_PREFIX + sourceIndexName);
IndexMetaData shrunkIndexMetaData = clusterState.metaData().index(targetIndexName);
if (isShrunkIndex) {
// We are currently managing the shrunken index. This means all previous operations were successful and
// the original index is deleted. It is important to add an alias from the original index name to the shrunken
// index so that previous actions will still succeed.
boolean aliasAlreadyExists = indexMetaData.getAliases().values().contains(AliasMetaData.builder(sourceIndexName).build());
boolean sourceIndexDeleted = clusterState.metaData().hasIndex(sourceIndexName) == false;
if (sourceIndexDeleted && aliasAlreadyExists) {
listener.onSuccess(true);
} else {
IndicesAliasesRequest aliasesRequest = new IndicesAliasesRequest()
.addAliasAction(IndicesAliasesRequest.AliasActions.removeIndex().index(sourceIndexName))
.addAliasAction(IndicesAliasesRequest.AliasActions.add().index(index.getName()).alias(sourceIndexName));
client.admin().indices().aliases(aliasesRequest, ActionListener.wrap(response -> {
listener.onSuccess(true);
}, listener::onFailure));
public List<Step> toSteps(String phase, Index index, Client client, ThreadPool threadPool, LongSupplier nowSupplier) {
String shrunkenIndexName = SHRUNK_INDEX_NAME_PREFIX + index.getName();
// TODO(talevy): magical node.name to allocate to
String nodeName = "MAGIC";
ClusterStateUpdateStep updateAllocationToOneNode = new ClusterStateUpdateStep(
"move_to_single_node", NAME, phase, index.getName(), (clusterState) -> {
IndexMetaData idxMeta = clusterState.metaData().index(index);
if (idxMeta == null) {
return clusterState;
}
} else if (shrunkIndexMetaData == null) {
// Shrunken index is not present yet, it is time to issue to shrink request
ResizeRequest resizeRequest = new ResizeRequest(targetIndexName, index.getName());
resizeRequest.getTargetIndexRequest().settings(Settings.builder()
.put(IndexMetaData.SETTING_NUMBER_OF_SHARDS, numberOfShards)
.put(IndexMetaData.SETTING_NUMBER_OF_REPLICAS, indexMetaData.getNumberOfReplicas())
.put(LifecycleSettings.LIFECYCLE_INDEX_CREATION_DATE, indexMetaData.getCreationDate())
.build());
indexMetaData.getAliases().values().spliterator().forEachRemaining(aliasMetaDataObjectCursor -> {
resizeRequest.getTargetIndexRequest().alias(new Alias(aliasMetaDataObjectCursor.value.alias()));
});
UpdateSettingsRequest updateSettingsRequest = new UpdateSettingsRequest(Settings.builder()
.put(IndexMetaData.SETTING_BLOCKS_WRITE, true).build(), index.getName());
client.admin().indices().updateSettings(updateSettingsRequest, ActionListener.wrap(r -> {
client.admin().indices().resizeIndex(resizeRequest, ActionListener.wrap(
resizeResponse -> {
if (resizeResponse.isAcknowledged()) {
listener.onSuccess(false);
} else {
listener.onFailure(new IllegalStateException("Shrink request failed to be acknowledged"));
}
}, listener::onFailure));
}, listener::onFailure));
} else if (index.getName().equals(IndexMetaData.INDEX_SHRINK_SOURCE_NAME.get(shrunkIndexMetaData.getSettings())) == false) {
// The target shrunken index exists, but it was not shrunk from our managed index. This means
// some external actions were done to create this index, and so we cannot progress with the shrink
// action until this is resolved.
listener.onFailure(new IllegalStateException("Cannot shrink index [" + index.getName() + "] because target " +
"index [" + targetIndexName + "] already exists."));
} else if (ActiveShardCount.ALL.enoughShardsActive(clusterService.state(), targetIndexName)) {
if (indexMetaData.getSettings().get(LifecycleSettings.LIFECYCLE_NAME)
.equals(shrunkIndexMetaData.getSettings().get(LifecycleSettings.LIFECYCLE_NAME))) {
// Since both the shrunken and original indices co-exist, do nothing and wait until
// the final step of the shrink action is completed and this original index is deleted.
listener.onSuccess(false);
} else {
// Since all shards of the shrunken index are active, it is safe to continue forward
// and begin swapping the indices by inheriting the lifecycle management to the new shrunken index.
UpdateSettingsRequest updateSettingsRequest = new UpdateSettingsRequest(Settings.builder()
.put(LifecycleSettings.LIFECYCLE_NAME, indexMetaData.getSettings().get(LifecycleSettings.LIFECYCLE_NAME))
.put(LifecycleSettings.LIFECYCLE_PHASE, indexMetaData.getSettings().get(LifecycleSettings.LIFECYCLE_PHASE))
.put(LifecycleSettings.LIFECYCLE_ACTION, indexMetaData.getSettings().get(LifecycleSettings.LIFECYCLE_ACTION))
.build(), targetIndexName);
client.admin().indices().updateSettings(updateSettingsRequest,
ActionListener.wrap(r -> listener.onSuccess(false), listener::onFailure));
Settings.Builder newSettings = Settings.builder()
.put(IndexMetaData.INDEX_ROUTING_INCLUDE_GROUP_PREFIX, "")
.put(IndexMetaData.INDEX_ROUTING_EXCLUDE_GROUP_PREFIX, "")
.put(IndexMetaData.INDEX_ROUTING_REQUIRE_GROUP_SETTING.getKey() + "_name", nodeName);
return ClusterState.builder(clusterState)
.metaData(MetaData.builder(clusterState.metaData())
.updateSettings(newSettings.build(), index.getName())).build();
});
// resizeRequest.getTargetIndexRequest().settings(Settings.builder()
// .put(IndexMetaData.SETTING_NUMBER_OF_SHARDS, numberOfShards)
// .put(IndexMetaData.SETTING_NUMBER_OF_REPLICAS, indexMetaData.getNumberOfReplicas())
// .put(LifecycleSettings.LIFECYCLE_INDEX_CREATION_DATE, indexMetaData.getCreationDate())
// .build());
// indexMetaData.getAliases().values().spliterator().forEachRemaining(aliasMetaDataObjectCursor -> {
// resizeRequest.getTargetIndexRequest().alias(new Alias(aliasMetaDataObjectCursor.value.alias()));
// });
// TODO(talevy): needs access to original index metadata, not just Index
int numReplicas = -1;
long lifecycleDate = -1L;
Settings targetIndexSettings = Settings.builder()
.put(IndexMetaData.SETTING_NUMBER_OF_SHARDS, numberOfShards)
.put(IndexMetaData.SETTING_NUMBER_OF_REPLICAS, numReplicas)
.put(LifecycleSettings.LIFECYCLE_INDEX_CREATION_DATE, lifecycleDate)
.build();
CreateIndexRequest targetIndexRequest = new CreateIndexRequest(shrunkenIndexName, targetIndexSettings);
// TODO(talevy): need access to indexmetadata
// indexMetaData.getAliases().values().spliterator().forEachRemaining(aliasMetaDataObjectCursor -> {
// resizeRequest.getTargetIndexRequest().alias(new Alias(aliasMetaDataObjectCursor.value.alias()));
// });
ClientStep<ResizeRequestBuilder, ResizeResponse> shrinkStep = new ClientStep<>( "segment_count",
NAME, phase, index.getName(),
client.admin().indices().prepareResizeIndex(index.getName(), shrunkenIndexName).setTargetIndex(targetIndexRequest),
currentState -> {
// check that shrunken index was already created, if so, no need to both client
IndexMetaData shrunkMetaData = currentState.metaData().index(shrunkenIndexName);
boolean isSuccessful = shrunkMetaData != null && shrunkenIndexName.equals(IndexMetaData.INDEX_SHRINK_SOURCE_NAME
.get(shrunkMetaData.getSettings()));
}, ResizeResponse::isAcknowledged);
ConditionalWaitStep shrunkenIndexIsAllocated = new ConditionalWaitStep("wait_replicas_allocated", NAME,
phase, index.getName(), (currentState) -> ActiveShardCount.ALL.enoughShardsActive(currentState, index.getName()) );
ClusterStateUpdateStep deleteAndUpdateAliases = new ClusterStateUpdateStep(
"delete_this_index_set_aliases_on_shrunken", NAME, phase, index.getName(), (clusterState) -> {
IndexMetaData idxMeta = clusterState.metaData().index(index);
if (idxMeta == null) {
return clusterState;
}
} else {
// We are here because both the shrunken and original indices exist, but the shrunken index is not
// fully active yet. This means that we wait for another poll iteration of execute to check the
// state again.
logger.debug("index [" + index.getName() + "] has been shrunk to shrunken-index [" + targetIndexName + "], but" +
"shrunken index is not fully active yet");
listener.onSuccess(false);
}
// TODO(talevy): expose - MetadataDeleteIndexService.deleteIndices(clusterState, Set.of(index.getName()))
// also, looks like deletes are special CS tasks
// AckedClusterStateUpdateTask, Priority.URGENT
// 1. delete index
// 2. assign alias to shrunken index
// 3. assign index.lifecycle settings to shrunken index
return clusterState;
});
return Arrays.asList(updateAllocationToOneNode,
AllocateAction.getAllocationCheck(allocationDeciders, phase, index.getName()),
shrinkStep, shrunkenIndexIsAllocated, deleteAndUpdateAliases);
}
@Override

View File

@ -0,0 +1,43 @@
/*
* 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;
/**
* A {@link LifecycleAction} which deletes the index.
*/
public abstract class Step {
private final String name;
private final String action;
private final String phase;
private final String index;
public Step(String name, String action, String phase, String index) {
this.name = name;
this.action = action;
this.phase = phase;
this.index = index;
}
public String getName() {
return name;
}
public String getAction() {
return action;
}
public String getPhase() {
return phase;
}
public String getIndex() {
return index;
}
public abstract StepResult execute(ClusterState currentState);
}

View File

@ -0,0 +1,49 @@
/*
* 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;
public class StepResult {
private final String message;
private final Exception exception;
private final ClusterState clusterState;
private final boolean complete;
private final boolean indexSurvived;
public StepResult(String message, Exception exception, ClusterState clusterState, boolean indexSurvived, boolean complete) {
this.message = message;
this.exception = exception;
this.clusterState = clusterState;
this.indexSurvived = indexSurvived;
this.complete = complete;
}
public StepResult(StepResult result, ClusterState newClusterState) {
this(result.getMessage(), result.getException(), newClusterState, result.indexSurvived(), result.isComplete());
}
public String getMessage() {
return message;
}
public Exception getException() {
return exception;
}
public ClusterState getClusterState() {
return clusterState;
}
public boolean indexSurvived() {
return indexSurvived;
}
public boolean isComplete() {
return complete;
}
}

View File

@ -91,103 +91,103 @@ public class TimeseriesLifecycleType implements LifecycleType {
return null;
}
/**
* This action provider returns an ordering for the actions within each of the four timeseries phases.
* Hot Phase:
* The Hot Phase only supports the {@link RolloverAction} and so that is the only action to order
* Warm Phase:
* The Warm Phase executes the supported actions in a slightly complicated order for the sake of
* optimization. Assuming the {@link ReplicasAction} is specified, it will run first or last depending
* on whether it increases, decreases, or keeps the existing replica count. If number-of-replicas is
* kept the same, or reduced, then {@link ReplicasAction} is executed first, otherwise, it is last.
* So the ordering looks something like this:
* - {@link ReplicasAction} (if action.number_of_replicas lte idxMeta.number_of_replicas)
* - {@link AllocateAction}
* - {@link ShrinkAction}
* - {@link ForceMergeAction}
* - {@link ReplicasAction} (if action.number_of_replicas gt idxMeta.number_of_replicas)
*
* NORELEASE: there may exist further optimizations to this when {@link ShrinkAction} is specified.
*
* @param context the index lifecycle context for this phase at the time of execution
* @param phase the current phase for which to provide an action provider
* @return the {@link LifecyclePolicy.NextActionProvider} for {@code phase}.
*/
@Override
public LifecyclePolicy.NextActionProvider getActionProvider(IndexLifecycleContext context, Phase phase) {
Map<String, LifecycleAction> actions = phase.getActions();
switch (phase.getName()) {
case "hot":
// The hot-phase only has one action, either start with it, or return null. Simple as that!
return (action) -> (action == null) ? actions.getOrDefault(RolloverAction.NAME, null) : null;
case "warm":
return (action) -> {
ReplicasAction replicasAction = (ReplicasAction) actions.getOrDefault(ReplicasAction.NAME, null);
boolean replicaActionFirst = replicasAction != null
&& replicasAction.getNumberOfReplicas() <= context.getNumberOfReplicas();
if (action == null) {
if (replicaActionFirst) {
return replicasAction;
}
return Stream.of(AllocateAction.NAME, ShrinkAction.NAME, ForceMergeAction.NAME, ReplicasAction.NAME)
.map(a -> actions.getOrDefault(a, null))
.filter(Objects::nonNull).findFirst().orElse(null);
} else if (action instanceof ReplicasAction) {
if (replicaActionFirst) {
return Stream.of(AllocateAction.NAME, ShrinkAction.NAME, ForceMergeAction.NAME)
.map(a -> actions.getOrDefault(a, null))
.filter(Objects::nonNull).findFirst().orElse(null);
}
} else if (action instanceof AllocateAction) {
return Stream.of(ShrinkAction.NAME, ForceMergeAction.NAME, ReplicasAction.NAME)
.map(a -> actions.getOrDefault(a, null))
.filter(Objects::nonNull).findFirst().orElse(null);
} else if (action instanceof ShrinkAction) {
return Stream.of(ForceMergeAction.NAME, ReplicasAction.NAME)
.map(a -> actions.getOrDefault(a, null))
.filter(Objects::nonNull).findFirst().orElse(null);
} else if (action instanceof ForceMergeAction) {
if (replicaActionFirst == false) {
return replicasAction;
}
}
return null;
};
case "cold":
return (action) -> {
ReplicasAction replicasAction = (ReplicasAction) actions.getOrDefault(ReplicasAction.NAME, null);
LifecycleAction allocateAction = actions.getOrDefault(AllocateAction.NAME, null);
boolean replicaActionFirst = replicasAction != null
&& replicasAction.getNumberOfReplicas() <= context.getNumberOfReplicas();
if (action == null) {
if (replicaActionFirst) {
return replicasAction;
} else if (allocateAction != null) {
return allocateAction;
}
return replicasAction;
} else if (action instanceof ReplicasAction) {
if (replicaActionFirst) {
return allocateAction;
}
} else if (action instanceof AllocateAction) {
if (replicaActionFirst == false) {
return replicasAction;
}
}
return null;
};
case "delete":
return (action) -> {
if (action == null) {
return actions.getOrDefault(DeleteAction.NAME, null);
}
return null;
};
default:
throw new IllegalArgumentException("phase [" + phase.getName() + "] is invalid for policy [timeseries]");
}
}
// /**
// * This action provider returns an ordering for the actions within each of the four timeseries phases.
// * Hot Phase:
// * The Hot Phase only supports the {@link RolloverAction} and so that is the only action to order
// * Warm Phase:
// * The Warm Phase executes the supported actions in a slightly complicated order for the sake of
// * optimization. Assuming the {@link ReplicasAction} is specified, it will run first or last depending
// * on whether it increases, decreases, or keeps the existing replica count. If number-of-replicas is
// * kept the same, or reduced, then {@link ReplicasAction} is executed first, otherwise, it is last.
// * So the ordering looks something like this:
// * - {@link ReplicasAction} (if action.number_of_replicas lte idxMeta.number_of_replicas)
// * - {@link AllocateAction}
// * - {@link ShrinkAction}
// * - {@link ForceMergeAction}
// * - {@link ReplicasAction} (if action.number_of_replicas gt idxMeta.number_of_replicas)
// *
// * NORELEASE: there may exist further optimizations to this when {@link ShrinkAction} is specified.
// *
// * @param context the index lifecycle context for this phase at the time of execution
// * @param phase the current phase for which to provide an action provider
// * @return the {@link LifecyclePolicy.NextActionProvider} for {@code phase}.
// */
// @Override
// public LifecyclePolicy.NextActionProvider getActionProvider(IndexLifecycleContext context, Phase phase) {
// Map<String, LifecycleAction> actions = phase.getActions();
// switch (phase.getName()) {
// case "hot":
// // The hot-phase only has one action, either start with it, or return null. Simple as that!
// return (action) -> (action == null) ? actions.getOrDefault(RolloverAction.NAME, null) : null;
// case "warm":
// return (action) -> {
// ReplicasAction replicasAction = (ReplicasAction) actions.getOrDefault(ReplicasAction.NAME, null);
// boolean replicaActionFirst = replicasAction != null
// && replicasAction.getNumberOfReplicas() <= context.getNumberOfReplicas();
// if (action == null) {
// if (replicaActionFirst) {
// return replicasAction;
// }
// return Stream.of(AllocateAction.NAME, ShrinkAction.NAME, ForceMergeAction.NAME, ReplicasAction.NAME)
// .map(a -> actions.getOrDefault(a, null))
// .filter(Objects::nonNull).findFirst().orElse(null);
// } else if (action instanceof ReplicasAction) {
// if (replicaActionFirst) {
// return Stream.of(AllocateAction.NAME, ShrinkAction.NAME, ForceMergeAction.NAME)
// .map(a -> actions.getOrDefault(a, null))
// .filter(Objects::nonNull).findFirst().orElse(null);
// }
// } else if (action instanceof AllocateAction) {
// return Stream.of(ShrinkAction.NAME, ForceMergeAction.NAME, ReplicasAction.NAME)
// .map(a -> actions.getOrDefault(a, null))
// .filter(Objects::nonNull).findFirst().orElse(null);
// } else if (action instanceof ShrinkAction) {
// return Stream.of(ForceMergeAction.NAME, ReplicasAction.NAME)
// .map(a -> actions.getOrDefault(a, null))
// .filter(Objects::nonNull).findFirst().orElse(null);
// } else if (action instanceof ForceMergeAction) {
// if (replicaActionFirst == false) {
// return replicasAction;
// }
// }
// return null;
// };
// case "cold":
// return (action) -> {
// ReplicasAction replicasAction = (ReplicasAction) actions.getOrDefault(ReplicasAction.NAME, null);
// LifecycleAction allocateAction = actions.getOrDefault(AllocateAction.NAME, null);
// boolean replicaActionFirst = replicasAction != null
// && replicasAction.getNumberOfReplicas() <= context.getNumberOfReplicas();
// if (action == null) {
// if (replicaActionFirst) {
// return replicasAction;
// } else if (allocateAction != null) {
// return allocateAction;
// }
// return replicasAction;
// } else if (action instanceof ReplicasAction) {
// if (replicaActionFirst) {
// return allocateAction;
// }
// } else if (action instanceof AllocateAction) {
// if (replicaActionFirst == false) {
// return replicasAction;
// }
// }
// return null;
// };
// case "delete":
// return (action) -> {
// if (action == null) {
// return actions.getOrDefault(DeleteAction.NAME, null);
// }
// return null;
// };
// default:
// throw new IllegalArgumentException("phase [" + phase.getName() + "] is invalid for policy [timeseries]");
// }
// }
@Override
public void validate(Collection<Phase> phases) {

View File

@ -31,7 +31,6 @@ import org.elasticsearch.index.IndexNotFoundException;
import org.elasticsearch.index.shard.ShardId;
import org.elasticsearch.node.Node;
import org.elasticsearch.test.AbstractSerializingTestCase;
import org.elasticsearch.xpack.core.indexlifecycle.LifecycleAction.Listener;
import java.util.Collections;
import java.util.HashMap;
@ -113,368 +112,368 @@ public class AllocateActionTests extends AbstractSerializingTestCase<AllocateAct
+ "must contain attributes for action " + AllocateAction.NAME, exception.getMessage());
}
public void testExecuteNoExistingSettings() throws Exception {
Map<String, String> includes = randomMap(1, 5);
Map<String, String> excludes = randomMap(1, 5);
Map<String, String> requires = randomMap(1, 5);
Settings.Builder existingSettings = Settings.builder().put("index.version.created", Version.CURRENT.id);
Settings.Builder expectedSettings = Settings.builder();
includes.forEach((k, v) -> expectedSettings.put(IndexMetaData.INDEX_ROUTING_INCLUDE_GROUP_SETTING.getKey() + k, v));
excludes.forEach((k, v) -> expectedSettings.put(IndexMetaData.INDEX_ROUTING_EXCLUDE_GROUP_SETTING.getKey() + k, v));
requires.forEach((k, v) -> expectedSettings.put(IndexMetaData.INDEX_ROUTING_REQUIRE_GROUP_SETTING.getKey() + k, v));
AllocateAction action = new AllocateAction(includes, excludes, requires);
assertSettingsUpdate(action, existingSettings, expectedSettings.build());
}
public void testExecuteSettingsUnassignedShards() throws Exception {
Map<String, String> includes = randomMap(1, 5);
Map<String, String> excludes = randomMap(1, 5);
Map<String, String> requires = randomMap(1, 5);
Settings.Builder existingSettings = Settings.builder().put("index.version.created", Version.CURRENT.id);
Settings.Builder expectedSettings = Settings.builder();
includes.forEach((k, v) -> expectedSettings.put(IndexMetaData.INDEX_ROUTING_INCLUDE_GROUP_SETTING.getKey() + k, v));
excludes.forEach((k, v) -> expectedSettings.put(IndexMetaData.INDEX_ROUTING_EXCLUDE_GROUP_SETTING.getKey() + k, v));
requires.forEach((k, v) -> expectedSettings.put(IndexMetaData.INDEX_ROUTING_REQUIRE_GROUP_SETTING.getKey() + k, v));
AllocateAction action = new AllocateAction(includes, excludes, requires);
assertSettingsUpdate(action, existingSettings, expectedSettings.build());
}
public void testExecuteSomeExistingSettings() throws Exception {
Map<String, String> includes = randomMap(1, 5);
Map<String, String> excludes = randomMap(1, 5);
Map<String, String> requires = randomMap(1, 5);
Settings.Builder existingSettings = Settings.builder().put("index.version.created", Version.CURRENT.id);
Settings.Builder expectedSettings = Settings.builder();
includes.forEach((k, v) -> {
if (randomBoolean()) {
existingSettings.put(IndexMetaData.INDEX_ROUTING_INCLUDE_GROUP_SETTING.getKey() + k, v);
} else {
expectedSettings.put(IndexMetaData.INDEX_ROUTING_INCLUDE_GROUP_SETTING.getKey() + k, v);
}
});
excludes.forEach((k, v) -> {
if (randomBoolean()) {
existingSettings.put(IndexMetaData.INDEX_ROUTING_EXCLUDE_GROUP_SETTING.getKey() + k, v);
} else {
expectedSettings.put(IndexMetaData.INDEX_ROUTING_EXCLUDE_GROUP_SETTING.getKey() + k, v);
}
});
requires.forEach((k, v) -> {
if (randomBoolean()) {
existingSettings.put(IndexMetaData.INDEX_ROUTING_REQUIRE_GROUP_SETTING.getKey() + k, v);
} else {
expectedSettings.put(IndexMetaData.INDEX_ROUTING_REQUIRE_GROUP_SETTING.getKey() + k, v);
}
});
// make sure there is at least one setting that is missing
if (expectedSettings.keys().isEmpty()) {
String key = randomAlphaOfLengthBetween(1, 20);
String value = randomAlphaOfLengthBetween(1, 20);
includes.put(key, value);
expectedSettings.put(IndexMetaData.INDEX_ROUTING_INCLUDE_GROUP_SETTING.getKey() + key, value);
}
AllocateAction action = new AllocateAction(includes, excludes, requires);
assertSettingsUpdate(action, existingSettings, expectedSettings.build());
}
public void testExecuteSomeExistingSettingsDifferentValue() throws Exception {
Map<String, String> includes = randomMap(1, 5);
Map<String, String> excludes = randomMap(1, 5);
Map<String, String> requires = randomMap(1, 5);
Settings.Builder existingSettings = Settings.builder().put("index.version.created", Version.CURRENT.id);
Settings.Builder expectedSettings = Settings.builder();
includes.forEach((k, v) -> {
if (randomBoolean()) {
existingSettings.put(IndexMetaData.INDEX_ROUTING_INCLUDE_GROUP_SETTING.getKey() + k, v);
} else {
expectedSettings.put(IndexMetaData.INDEX_ROUTING_INCLUDE_GROUP_SETTING.getKey() + k, v);
existingSettings.put(IndexMetaData.INDEX_ROUTING_INCLUDE_GROUP_SETTING.getKey() + k, v + randomAlphaOfLength(4));
}
});
excludes.forEach((k, v) -> {
if (randomBoolean()) {
existingSettings.put(IndexMetaData.INDEX_ROUTING_EXCLUDE_GROUP_SETTING.getKey() + k, v);
} else {
expectedSettings.put(IndexMetaData.INDEX_ROUTING_EXCLUDE_GROUP_SETTING.getKey() + k, v);
existingSettings.put(IndexMetaData.INDEX_ROUTING_EXCLUDE_GROUP_SETTING.getKey() + k, v + randomAlphaOfLength(4));
}
});
requires.forEach((k, v) -> {
if (randomBoolean()) {
existingSettings.put(IndexMetaData.INDEX_ROUTING_REQUIRE_GROUP_SETTING.getKey() + k, v);
} else {
expectedSettings.put(IndexMetaData.INDEX_ROUTING_REQUIRE_GROUP_SETTING.getKey() + k, v);
existingSettings.put(IndexMetaData.INDEX_ROUTING_REQUIRE_GROUP_SETTING.getKey() + k, v + randomAlphaOfLength(4));
}
});
// make sure there is at least one setting that is different
if (expectedSettings.keys().isEmpty()) {
String key = randomAlphaOfLengthBetween(1, 20);
String value = randomAlphaOfLengthBetween(1, 20);
includes.put(key, value);
expectedSettings.put(IndexMetaData.INDEX_ROUTING_INCLUDE_GROUP_SETTING.getKey() + key, value);
existingSettings.put(IndexMetaData.INDEX_ROUTING_INCLUDE_GROUP_SETTING.getKey() + key, value + randomAlphaOfLength(4));
}
AllocateAction action = new AllocateAction(includes, excludes, requires);
assertSettingsUpdate(action, existingSettings, expectedSettings.build());
}
public void testExecuteUpdateSettingsFail() throws Exception {
Settings expectedSettings = Settings.builder().put(IndexMetaData.INDEX_ROUTING_INCLUDE_GROUP_SETTING.getKey() + "box_type", "foo")
.put(IndexMetaData.INDEX_ROUTING_EXCLUDE_GROUP_SETTING.getKey() + "box_type", "bar")
.put(IndexMetaData.INDEX_ROUTING_REQUIRE_GROUP_SETTING.getKey() + "box_type", "baz").build();
IndexMetaData indexMetadata = IndexMetaData.builder(randomAlphaOfLengthBetween(1, 20))
.settings(Settings.builder().put("index.version.created", Version.CURRENT.id)).numberOfShards(randomIntBetween(1, 5))
.numberOfReplicas(randomIntBetween(0, 5)).build();
Index index = indexMetadata.getIndex();
ImmutableOpenMap.Builder<String, IndexMetaData> indices = ImmutableOpenMap.<String, IndexMetaData> builder().fPut(index.getName(),
indexMetadata);
ClusterState clusterState = ClusterState.builder(ClusterState.EMPTY_STATE).metaData(MetaData.builder().indices(indices.build()))
.routingTable(RoutingTable.builder()
.add(IndexRoutingTable.builder(index).addShard(
TestShardRouting.newShardRouting(new ShardId(index, 0), "node1", true, ShardRoutingState.STARTED)))
.build())
.build();
Exception exception = new RuntimeException();
BiConsumer<Settings, Listener> settingsUpdater = (s, l) -> {
assertEquals(expectedSettings, s);
l.onFailure(exception);
};
Map<String, String> includes = new HashMap<>();
includes.put("box_type", "foo");
Map<String, String> excludes = new HashMap<>();
excludes.put("box_type", "bar");
Map<String, String> requires = new HashMap<>();
requires.put("box_type", "baz");
AllocateAction action = new AllocateAction(includes, excludes, requires);
RuntimeException thrownException = expectActionFailure(index, clusterState, null, action, settingsUpdater, RuntimeException.class);
assertSame(exception, thrownException);
}
public void testExecuteAllocateComplete() throws Exception {
Index index = new Index(randomAlphaOfLengthBetween(1, 20), randomAlphaOfLengthBetween(1, 20));
Map<String, String> includes = randomMap(1, 5);
Map<String, String> excludes = randomMap(1, 5);
Map<String, String> requires = randomMap(1, 5);
Settings.Builder existingSettings = Settings.builder().put(IndexMetaData.SETTING_VERSION_CREATED, Version.CURRENT.id)
.put(IndexMetaData.SETTING_INDEX_UUID, index.getUUID());
Settings.Builder expectedSettings = Settings.builder();
Settings.Builder node1Settings = Settings.builder();
Settings.Builder node2Settings = Settings.builder();
includes.forEach((k, v) -> {
existingSettings.put(IndexMetaData.INDEX_ROUTING_INCLUDE_GROUP_SETTING.getKey() + k, v);
expectedSettings.put(IndexMetaData.INDEX_ROUTING_INCLUDE_GROUP_SETTING.getKey() + k, v);
node1Settings.put(Node.NODE_ATTRIBUTES.getKey() + k, v);
});
excludes.forEach((k, v) -> {
existingSettings.put(IndexMetaData.INDEX_ROUTING_EXCLUDE_GROUP_SETTING.getKey() + k, v);
expectedSettings.put(IndexMetaData.INDEX_ROUTING_EXCLUDE_GROUP_SETTING.getKey() + k, v);
});
requires.forEach((k, v) -> {
existingSettings.put(IndexMetaData.INDEX_ROUTING_REQUIRE_GROUP_SETTING.getKey() + k, v);
expectedSettings.put(IndexMetaData.INDEX_ROUTING_REQUIRE_GROUP_SETTING.getKey() + k, v);
node1Settings.put(Node.NODE_ATTRIBUTES.getKey() + k, v);
});
IndexRoutingTable.Builder indexRoutingTable = IndexRoutingTable.builder(index)
.addShard(TestShardRouting.newShardRouting(new ShardId(index, 0), "node1", true, ShardRoutingState.STARTED));
AllocateAction action = new AllocateAction(includes, excludes, requires);
assertAllocateStatus(index, 1, 0, action, existingSettings, node1Settings, node2Settings, indexRoutingTable,
true);
}
public void testExecuteAllocateNotComplete() throws Exception {
Index index = new Index(randomAlphaOfLengthBetween(1, 20), randomAlphaOfLengthBetween(1, 20));
Map<String, String> includes = randomMap(1, 5);
Map<String, String> excludes = randomMap(1, 5);
Map<String, String> requires = randomMap(1, 5);
Settings.Builder existingSettings = Settings.builder().put(IndexMetaData.SETTING_VERSION_CREATED, Version.CURRENT.id)
.put(IndexMetaData.SETTING_INDEX_UUID, index.getUUID());
Settings.Builder expectedSettings = Settings.builder();
Settings.Builder node1Settings = Settings.builder();
Settings.Builder node2Settings = Settings.builder();
includes.forEach((k, v) -> {
existingSettings.put(IndexMetaData.INDEX_ROUTING_INCLUDE_GROUP_SETTING.getKey() + k, v);
expectedSettings.put(IndexMetaData.INDEX_ROUTING_INCLUDE_GROUP_SETTING.getKey() + k, v);
node1Settings.put(Node.NODE_ATTRIBUTES.getKey() + k, v);
});
excludes.forEach((k, v) -> {
existingSettings.put(IndexMetaData.INDEX_ROUTING_EXCLUDE_GROUP_SETTING.getKey() + k, v);
expectedSettings.put(IndexMetaData.INDEX_ROUTING_EXCLUDE_GROUP_SETTING.getKey() + k, v);
});
requires.forEach((k, v) -> {
existingSettings.put(IndexMetaData.INDEX_ROUTING_REQUIRE_GROUP_SETTING.getKey() + k, v);
expectedSettings.put(IndexMetaData.INDEX_ROUTING_REQUIRE_GROUP_SETTING.getKey() + k, v);
node1Settings.put(Node.NODE_ATTRIBUTES.getKey() + k, v);
});
IndexRoutingTable.Builder indexRoutingTable = IndexRoutingTable.builder(index)
.addShard(TestShardRouting.newShardRouting(new ShardId(index, 0), "node1", true, ShardRoutingState.STARTED))
.addShard(TestShardRouting.newShardRouting(new ShardId(index, 1), "node2", true, ShardRoutingState.STARTED));
AllocateAction action = new AllocateAction(includes, excludes, requires);
assertAllocateStatus(index, 2, 0, action, existingSettings, node1Settings, node2Settings, indexRoutingTable,
false);
}
public void testExecuteAllocateUnassigned() throws Exception {
Index index = new Index(randomAlphaOfLengthBetween(1, 20), randomAlphaOfLengthBetween(1, 20));
Map<String, String> includes = randomMap(1, 5);
Map<String, String> excludes = randomMap(1, 5);
Map<String, String> requires = randomMap(1, 5);
Settings.Builder existingSettings = Settings.builder().put(IndexMetaData.SETTING_VERSION_CREATED, Version.CURRENT.id)
.put(IndexMetaData.SETTING_INDEX_UUID, index.getUUID());
Settings.Builder expectedSettings = Settings.builder();
Settings.Builder node1Settings = Settings.builder();
Settings.Builder node2Settings = Settings.builder();
includes.forEach((k, v) -> {
existingSettings.put(IndexMetaData.INDEX_ROUTING_INCLUDE_GROUP_SETTING.getKey() + k, v);
expectedSettings.put(IndexMetaData.INDEX_ROUTING_INCLUDE_GROUP_SETTING.getKey() + k, v);
node1Settings.put(Node.NODE_ATTRIBUTES.getKey() + k, v);
});
excludes.forEach((k, v) -> {
existingSettings.put(IndexMetaData.INDEX_ROUTING_EXCLUDE_GROUP_SETTING.getKey() + k, v);
expectedSettings.put(IndexMetaData.INDEX_ROUTING_EXCLUDE_GROUP_SETTING.getKey() + k, v);
});
requires.forEach((k, v) -> {
existingSettings.put(IndexMetaData.INDEX_ROUTING_REQUIRE_GROUP_SETTING.getKey() + k, v);
expectedSettings.put(IndexMetaData.INDEX_ROUTING_REQUIRE_GROUP_SETTING.getKey() + k, v);
node1Settings.put(Node.NODE_ATTRIBUTES.getKey() + k, v);
});
IndexRoutingTable.Builder indexRoutingTable = IndexRoutingTable.builder(index)
.addShard(TestShardRouting.newShardRouting(new ShardId(index, 0), "node1", true, ShardRoutingState.STARTED))
.addShard(TestShardRouting.newShardRouting(new ShardId(index, 1), null, null, true, ShardRoutingState.UNASSIGNED,
new UnassignedInfo(randomFrom(Reason.values()), "the shard is intentionally unassigned")));
AllocateAction action = new AllocateAction(includes, excludes, requires);
assertAllocateStatus(index, 2, 0, action, existingSettings, node1Settings, node2Settings, indexRoutingTable,
false);
}
public void testExecuteIndexMissing() throws Exception {
Index index = new Index(randomAlphaOfLengthBetween(1, 20), randomAlphaOfLengthBetween(1, 20));
ClusterState clusterState = ClusterState.builder(ClusterState.EMPTY_STATE).build();
BiConsumer<Settings, Listener> settingsUpdater = (s, l) -> {
throw new AssertionError("Unexpected settings update");
};
Map<String, String> includes = new HashMap<>();
includes.put("box_type", "foo");
Map<String, String> excludes = new HashMap<>();
excludes.put("box_type", "bar");
Map<String, String> requires = new HashMap<>();
requires.put("box_type", "baz");
AllocateAction action = new AllocateAction(includes, excludes, requires);
IndexNotFoundException thrownException = expectActionFailure(index, clusterState, null, action, settingsUpdater,
IndexNotFoundException.class);
assertEquals("Index not found when executing " + AllocateAction.NAME + " lifecycle action.", thrownException.getMessage());
assertEquals(index.getName(), thrownException.getIndex().getName());
}
private void assertSettingsUpdate(AllocateAction action, Settings.Builder existingSettings, Settings expectedSettings) {
IndexMetaData indexMetadata = IndexMetaData.builder(randomAlphaOfLengthBetween(1, 20)).settings(existingSettings)
.numberOfShards(randomIntBetween(1, 5)).numberOfReplicas(randomIntBetween(0, 5)).build();
Index index = indexMetadata.getIndex();
ImmutableOpenMap.Builder<String, IndexMetaData> indices = ImmutableOpenMap.<String, IndexMetaData> builder().fPut(index.getName(),
indexMetadata);
ClusterState clusterState = ClusterState.builder(ClusterState.EMPTY_STATE).metaData(MetaData.builder().indices(indices.build()))
.routingTable(RoutingTable.builder()
.add(IndexRoutingTable.builder(index).addShard(
TestShardRouting.newShardRouting(new ShardId(index, 0), "node1", true, ShardRoutingState.STARTED)))
.build())
.build();
BiConsumer<Settings, Listener> settingsUpdater = (s, l) -> {
assertEquals(expectedSettings, s);
l.onSuccess(false);
};
assertActionStatus(index, clusterState, null, action, settingsUpdater, false);
}
private void assertAllocateStatus(Index index, int shards, int replicas, AllocateAction action, Settings.Builder existingSettings,
Settings.Builder node1Settings, Settings.Builder node2Settings, IndexRoutingTable.Builder indexRoutingTable,
boolean expectComplete) {
IndexMetaData indexMetadata = IndexMetaData.builder(index.getName()).settings(existingSettings).numberOfShards(shards)
.numberOfReplicas(replicas).build();
ImmutableOpenMap.Builder<String, IndexMetaData> indices = ImmutableOpenMap.<String, IndexMetaData> builder().fPut(index.getName(),
indexMetadata);
ClusterState clusterState = ClusterState.builder(ClusterState.EMPTY_STATE).metaData(MetaData.builder().indices(indices.build()))
.nodes(DiscoveryNodes.builder()
.add(DiscoveryNode.createLocal(node1Settings.build(), new TransportAddress(TransportAddress.META_ADDRESS, 9200),
"node1"))
.add(DiscoveryNode.createLocal(node2Settings.build(), new TransportAddress(TransportAddress.META_ADDRESS, 9201),
"node2")))
.routingTable(RoutingTable.builder().add(indexRoutingTable).build()).build();
ClusterSettings clusterSettings = new ClusterSettings(Settings.EMPTY,
Sets.newHashSet(FilterAllocationDecider.CLUSTER_ROUTING_INCLUDE_GROUP_SETTING,
FilterAllocationDecider.CLUSTER_ROUTING_EXCLUDE_GROUP_SETTING,
FilterAllocationDecider.CLUSTER_ROUTING_REQUIRE_GROUP_SETTING));
BiConsumer<Settings, Listener> settingsUpdater = (s, l) -> {
throw new AssertionError("Unexpected settings update");
};
assertActionStatus(index, clusterState, clusterSettings, action, settingsUpdater, expectComplete);
}
private void assertActionStatus(Index index, ClusterState clusterState, ClusterSettings clusterSettings, AllocateAction action,
BiConsumer<Settings, Listener> settingsUpdater, boolean expectComplete) {
SetOnce<Boolean> actionCompleted = new SetOnce<>();
action.execute(index, settingsUpdater, clusterState, clusterSettings, new Listener() {
@Override
public void onSuccess(boolean completed) {
actionCompleted.set(completed);
}
@Override
public void onFailure(Exception e) {
throw new AssertionError("Unexpected method call");
}
});
assertEquals(expectComplete, actionCompleted.get());
}
private <E> E expectActionFailure(Index index, ClusterState clusterState, ClusterSettings clusterSettings, AllocateAction action,
BiConsumer<Settings, Listener> settingsUpdater, Class<E> expectedExceptionType) {
SetOnce<E> exceptionThrown = new SetOnce<>();
action.execute(index, settingsUpdater, clusterState, clusterSettings, new Listener() {
@Override
public void onSuccess(boolean completed) {
throw new AssertionError("Unexpected method call");
}
@SuppressWarnings("unchecked")
@Override
public void onFailure(Exception e) {
assertThat(e, instanceOf(expectedExceptionType));
exceptionThrown.set((E) e);
}
});
return exceptionThrown.get();
}
// public void testExecuteNoExistingSettings() throws Exception {
// Map<String, String> includes = randomMap(1, 5);
// Map<String, String> excludes = randomMap(1, 5);
// Map<String, String> requires = randomMap(1, 5);
// Settings.Builder existingSettings = Settings.builder().put("index.version.created", Version.CURRENT.id);
// Settings.Builder expectedSettings = Settings.builder();
// includes.forEach((k, v) -> expectedSettings.put(IndexMetaData.INDEX_ROUTING_INCLUDE_GROUP_SETTING.getKey() + k, v));
// excludes.forEach((k, v) -> expectedSettings.put(IndexMetaData.INDEX_ROUTING_EXCLUDE_GROUP_SETTING.getKey() + k, v));
// requires.forEach((k, v) -> expectedSettings.put(IndexMetaData.INDEX_ROUTING_REQUIRE_GROUP_SETTING.getKey() + k, v));
// AllocateAction action = new AllocateAction(includes, excludes, requires);
//
// assertSettingsUpdate(action, existingSettings, expectedSettings.build());
// }
//
// public void testExecuteSettingsUnassignedShards() throws Exception {
// Map<String, String> includes = randomMap(1, 5);
// Map<String, String> excludes = randomMap(1, 5);
// Map<String, String> requires = randomMap(1, 5);
// Settings.Builder existingSettings = Settings.builder().put("index.version.created", Version.CURRENT.id);
// Settings.Builder expectedSettings = Settings.builder();
// includes.forEach((k, v) -> expectedSettings.put(IndexMetaData.INDEX_ROUTING_INCLUDE_GROUP_SETTING.getKey() + k, v));
// excludes.forEach((k, v) -> expectedSettings.put(IndexMetaData.INDEX_ROUTING_EXCLUDE_GROUP_SETTING.getKey() + k, v));
// requires.forEach((k, v) -> expectedSettings.put(IndexMetaData.INDEX_ROUTING_REQUIRE_GROUP_SETTING.getKey() + k, v));
// AllocateAction action = new AllocateAction(includes, excludes, requires);
//
// assertSettingsUpdate(action, existingSettings, expectedSettings.build());
// }
//
// public void testExecuteSomeExistingSettings() throws Exception {
// Map<String, String> includes = randomMap(1, 5);
// Map<String, String> excludes = randomMap(1, 5);
// Map<String, String> requires = randomMap(1, 5);
// Settings.Builder existingSettings = Settings.builder().put("index.version.created", Version.CURRENT.id);
// Settings.Builder expectedSettings = Settings.builder();
// includes.forEach((k, v) -> {
// if (randomBoolean()) {
// existingSettings.put(IndexMetaData.INDEX_ROUTING_INCLUDE_GROUP_SETTING.getKey() + k, v);
// } else {
// expectedSettings.put(IndexMetaData.INDEX_ROUTING_INCLUDE_GROUP_SETTING.getKey() + k, v);
// }
// });
// excludes.forEach((k, v) -> {
// if (randomBoolean()) {
// existingSettings.put(IndexMetaData.INDEX_ROUTING_EXCLUDE_GROUP_SETTING.getKey() + k, v);
// } else {
// expectedSettings.put(IndexMetaData.INDEX_ROUTING_EXCLUDE_GROUP_SETTING.getKey() + k, v);
// }
// });
// requires.forEach((k, v) -> {
// if (randomBoolean()) {
// existingSettings.put(IndexMetaData.INDEX_ROUTING_REQUIRE_GROUP_SETTING.getKey() + k, v);
// } else {
// expectedSettings.put(IndexMetaData.INDEX_ROUTING_REQUIRE_GROUP_SETTING.getKey() + k, v);
// }
// });
//
// // make sure there is at least one setting that is missing
// if (expectedSettings.keys().isEmpty()) {
// String key = randomAlphaOfLengthBetween(1, 20);
// String value = randomAlphaOfLengthBetween(1, 20);
// includes.put(key, value);
// expectedSettings.put(IndexMetaData.INDEX_ROUTING_INCLUDE_GROUP_SETTING.getKey() + key, value);
// }
// AllocateAction action = new AllocateAction(includes, excludes, requires);
//
// assertSettingsUpdate(action, existingSettings, expectedSettings.build());
// }
//
// public void testExecuteSomeExistingSettingsDifferentValue() throws Exception {
// Map<String, String> includes = randomMap(1, 5);
// Map<String, String> excludes = randomMap(1, 5);
// Map<String, String> requires = randomMap(1, 5);
// Settings.Builder existingSettings = Settings.builder().put("index.version.created", Version.CURRENT.id);
// Settings.Builder expectedSettings = Settings.builder();
// includes.forEach((k, v) -> {
// if (randomBoolean()) {
// existingSettings.put(IndexMetaData.INDEX_ROUTING_INCLUDE_GROUP_SETTING.getKey() + k, v);
// } else {
// expectedSettings.put(IndexMetaData.INDEX_ROUTING_INCLUDE_GROUP_SETTING.getKey() + k, v);
// existingSettings.put(IndexMetaData.INDEX_ROUTING_INCLUDE_GROUP_SETTING.getKey() + k, v + randomAlphaOfLength(4));
// }
// });
// excludes.forEach((k, v) -> {
// if (randomBoolean()) {
// existingSettings.put(IndexMetaData.INDEX_ROUTING_EXCLUDE_GROUP_SETTING.getKey() + k, v);
// } else {
// expectedSettings.put(IndexMetaData.INDEX_ROUTING_EXCLUDE_GROUP_SETTING.getKey() + k, v);
// existingSettings.put(IndexMetaData.INDEX_ROUTING_EXCLUDE_GROUP_SETTING.getKey() + k, v + randomAlphaOfLength(4));
// }
// });
// requires.forEach((k, v) -> {
// if (randomBoolean()) {
// existingSettings.put(IndexMetaData.INDEX_ROUTING_REQUIRE_GROUP_SETTING.getKey() + k, v);
// } else {
// expectedSettings.put(IndexMetaData.INDEX_ROUTING_REQUIRE_GROUP_SETTING.getKey() + k, v);
// existingSettings.put(IndexMetaData.INDEX_ROUTING_REQUIRE_GROUP_SETTING.getKey() + k, v + randomAlphaOfLength(4));
// }
// });
//
// // make sure there is at least one setting that is different
// if (expectedSettings.keys().isEmpty()) {
// String key = randomAlphaOfLengthBetween(1, 20);
// String value = randomAlphaOfLengthBetween(1, 20);
// includes.put(key, value);
// expectedSettings.put(IndexMetaData.INDEX_ROUTING_INCLUDE_GROUP_SETTING.getKey() + key, value);
// existingSettings.put(IndexMetaData.INDEX_ROUTING_INCLUDE_GROUP_SETTING.getKey() + key, value + randomAlphaOfLength(4));
// }
// AllocateAction action = new AllocateAction(includes, excludes, requires);
//
// assertSettingsUpdate(action, existingSettings, expectedSettings.build());
// }
//
// public void testExecuteUpdateSettingsFail() throws Exception {
// Settings expectedSettings = Settings.builder().put(IndexMetaData.INDEX_ROUTING_INCLUDE_GROUP_SETTING.getKey() + "box_type", "foo")
// .put(IndexMetaData.INDEX_ROUTING_EXCLUDE_GROUP_SETTING.getKey() + "box_type", "bar")
// .put(IndexMetaData.INDEX_ROUTING_REQUIRE_GROUP_SETTING.getKey() + "box_type", "baz").build();
// IndexMetaData indexMetadata = IndexMetaData.builder(randomAlphaOfLengthBetween(1, 20))
// .settings(Settings.builder().put("index.version.created", Version.CURRENT.id)).numberOfShards(randomIntBetween(1, 5))
// .numberOfReplicas(randomIntBetween(0, 5)).build();
// Index index = indexMetadata.getIndex();
// ImmutableOpenMap.Builder<String, IndexMetaData> indices = ImmutableOpenMap.<String, IndexMetaData> builder().fPut(index.getName(),
// indexMetadata);
// ClusterState clusterState = ClusterState.builder(ClusterState.EMPTY_STATE).metaData(MetaData.builder().indices(indices.build()))
// .routingTable(RoutingTable.builder()
// .add(IndexRoutingTable.builder(index).addShard(
// TestShardRouting.newShardRouting(new ShardId(index, 0), "node1", true, ShardRoutingState.STARTED)))
// .build())
// .build();
// Exception exception = new RuntimeException();
//
// BiConsumer<Settings, Listener> settingsUpdater = (s, l) -> {
// assertEquals(expectedSettings, s);
// l.onFailure(exception);
// };
//
// Map<String, String> includes = new HashMap<>();
// includes.put("box_type", "foo");
// Map<String, String> excludes = new HashMap<>();
// excludes.put("box_type", "bar");
// Map<String, String> requires = new HashMap<>();
// requires.put("box_type", "baz");
//
// AllocateAction action = new AllocateAction(includes, excludes, requires);
//
// RuntimeException thrownException = expectActionFailure(index, clusterState, null, action, settingsUpdater, RuntimeException.class);
// assertSame(exception, thrownException);
//
// }
//
// public void testExecuteAllocateComplete() throws Exception {
// Index index = new Index(randomAlphaOfLengthBetween(1, 20), randomAlphaOfLengthBetween(1, 20));
// Map<String, String> includes = randomMap(1, 5);
// Map<String, String> excludes = randomMap(1, 5);
// Map<String, String> requires = randomMap(1, 5);
// Settings.Builder existingSettings = Settings.builder().put(IndexMetaData.SETTING_VERSION_CREATED, Version.CURRENT.id)
// .put(IndexMetaData.SETTING_INDEX_UUID, index.getUUID());
// Settings.Builder expectedSettings = Settings.builder();
// Settings.Builder node1Settings = Settings.builder();
// Settings.Builder node2Settings = Settings.builder();
// includes.forEach((k, v) -> {
// existingSettings.put(IndexMetaData.INDEX_ROUTING_INCLUDE_GROUP_SETTING.getKey() + k, v);
// expectedSettings.put(IndexMetaData.INDEX_ROUTING_INCLUDE_GROUP_SETTING.getKey() + k, v);
// node1Settings.put(Node.NODE_ATTRIBUTES.getKey() + k, v);
// });
// excludes.forEach((k, v) -> {
// existingSettings.put(IndexMetaData.INDEX_ROUTING_EXCLUDE_GROUP_SETTING.getKey() + k, v);
// expectedSettings.put(IndexMetaData.INDEX_ROUTING_EXCLUDE_GROUP_SETTING.getKey() + k, v);
// });
// requires.forEach((k, v) -> {
// existingSettings.put(IndexMetaData.INDEX_ROUTING_REQUIRE_GROUP_SETTING.getKey() + k, v);
// expectedSettings.put(IndexMetaData.INDEX_ROUTING_REQUIRE_GROUP_SETTING.getKey() + k, v);
// node1Settings.put(Node.NODE_ATTRIBUTES.getKey() + k, v);
// });
//
// IndexRoutingTable.Builder indexRoutingTable = IndexRoutingTable.builder(index)
// .addShard(TestShardRouting.newShardRouting(new ShardId(index, 0), "node1", true, ShardRoutingState.STARTED));
//
// AllocateAction action = new AllocateAction(includes, excludes, requires);
// assertAllocateStatus(index, 1, 0, action, existingSettings, node1Settings, node2Settings, indexRoutingTable,
// true);
// }
//
// public void testExecuteAllocateNotComplete() throws Exception {
// Index index = new Index(randomAlphaOfLengthBetween(1, 20), randomAlphaOfLengthBetween(1, 20));
// Map<String, String> includes = randomMap(1, 5);
// Map<String, String> excludes = randomMap(1, 5);
// Map<String, String> requires = randomMap(1, 5);
// Settings.Builder existingSettings = Settings.builder().put(IndexMetaData.SETTING_VERSION_CREATED, Version.CURRENT.id)
// .put(IndexMetaData.SETTING_INDEX_UUID, index.getUUID());
// Settings.Builder expectedSettings = Settings.builder();
// Settings.Builder node1Settings = Settings.builder();
// Settings.Builder node2Settings = Settings.builder();
// includes.forEach((k, v) -> {
// existingSettings.put(IndexMetaData.INDEX_ROUTING_INCLUDE_GROUP_SETTING.getKey() + k, v);
// expectedSettings.put(IndexMetaData.INDEX_ROUTING_INCLUDE_GROUP_SETTING.getKey() + k, v);
// node1Settings.put(Node.NODE_ATTRIBUTES.getKey() + k, v);
// });
// excludes.forEach((k, v) -> {
// existingSettings.put(IndexMetaData.INDEX_ROUTING_EXCLUDE_GROUP_SETTING.getKey() + k, v);
// expectedSettings.put(IndexMetaData.INDEX_ROUTING_EXCLUDE_GROUP_SETTING.getKey() + k, v);
// });
// requires.forEach((k, v) -> {
// existingSettings.put(IndexMetaData.INDEX_ROUTING_REQUIRE_GROUP_SETTING.getKey() + k, v);
// expectedSettings.put(IndexMetaData.INDEX_ROUTING_REQUIRE_GROUP_SETTING.getKey() + k, v);
// node1Settings.put(Node.NODE_ATTRIBUTES.getKey() + k, v);
// });
//
// IndexRoutingTable.Builder indexRoutingTable = IndexRoutingTable.builder(index)
// .addShard(TestShardRouting.newShardRouting(new ShardId(index, 0), "node1", true, ShardRoutingState.STARTED))
// .addShard(TestShardRouting.newShardRouting(new ShardId(index, 1), "node2", true, ShardRoutingState.STARTED));
//
// AllocateAction action = new AllocateAction(includes, excludes, requires);
// assertAllocateStatus(index, 2, 0, action, existingSettings, node1Settings, node2Settings, indexRoutingTable,
// false);
// }
//
// public void testExecuteAllocateUnassigned() throws Exception {
// Index index = new Index(randomAlphaOfLengthBetween(1, 20), randomAlphaOfLengthBetween(1, 20));
// Map<String, String> includes = randomMap(1, 5);
// Map<String, String> excludes = randomMap(1, 5);
// Map<String, String> requires = randomMap(1, 5);
// Settings.Builder existingSettings = Settings.builder().put(IndexMetaData.SETTING_VERSION_CREATED, Version.CURRENT.id)
// .put(IndexMetaData.SETTING_INDEX_UUID, index.getUUID());
// Settings.Builder expectedSettings = Settings.builder();
// Settings.Builder node1Settings = Settings.builder();
// Settings.Builder node2Settings = Settings.builder();
// includes.forEach((k, v) -> {
// existingSettings.put(IndexMetaData.INDEX_ROUTING_INCLUDE_GROUP_SETTING.getKey() + k, v);
// expectedSettings.put(IndexMetaData.INDEX_ROUTING_INCLUDE_GROUP_SETTING.getKey() + k, v);
// node1Settings.put(Node.NODE_ATTRIBUTES.getKey() + k, v);
// });
// excludes.forEach((k, v) -> {
// existingSettings.put(IndexMetaData.INDEX_ROUTING_EXCLUDE_GROUP_SETTING.getKey() + k, v);
// expectedSettings.put(IndexMetaData.INDEX_ROUTING_EXCLUDE_GROUP_SETTING.getKey() + k, v);
// });
// requires.forEach((k, v) -> {
// existingSettings.put(IndexMetaData.INDEX_ROUTING_REQUIRE_GROUP_SETTING.getKey() + k, v);
// expectedSettings.put(IndexMetaData.INDEX_ROUTING_REQUIRE_GROUP_SETTING.getKey() + k, v);
// node1Settings.put(Node.NODE_ATTRIBUTES.getKey() + k, v);
// });
//
// IndexRoutingTable.Builder indexRoutingTable = IndexRoutingTable.builder(index)
// .addShard(TestShardRouting.newShardRouting(new ShardId(index, 0), "node1", true, ShardRoutingState.STARTED))
// .addShard(TestShardRouting.newShardRouting(new ShardId(index, 1), null, null, true, ShardRoutingState.UNASSIGNED,
// new UnassignedInfo(randomFrom(Reason.values()), "the shard is intentionally unassigned")));
//
// AllocateAction action = new AllocateAction(includes, excludes, requires);
// assertAllocateStatus(index, 2, 0, action, existingSettings, node1Settings, node2Settings, indexRoutingTable,
// false);
// }
//
// public void testExecuteIndexMissing() throws Exception {
// Index index = new Index(randomAlphaOfLengthBetween(1, 20), randomAlphaOfLengthBetween(1, 20));
// ClusterState clusterState = ClusterState.builder(ClusterState.EMPTY_STATE).build();
//
// BiConsumer<Settings, Listener> settingsUpdater = (s, l) -> {
// throw new AssertionError("Unexpected settings update");
// };
//
// Map<String, String> includes = new HashMap<>();
// includes.put("box_type", "foo");
// Map<String, String> excludes = new HashMap<>();
// excludes.put("box_type", "bar");
// Map<String, String> requires = new HashMap<>();
// requires.put("box_type", "baz");
//
// AllocateAction action = new AllocateAction(includes, excludes, requires);
//
// IndexNotFoundException thrownException = expectActionFailure(index, clusterState, null, action, settingsUpdater,
// IndexNotFoundException.class);
// assertEquals("Index not found when executing " + AllocateAction.NAME + " lifecycle action.", thrownException.getMessage());
// assertEquals(index.getName(), thrownException.getIndex().getName());
// }
//
// private void assertSettingsUpdate(AllocateAction action, Settings.Builder existingSettings, Settings expectedSettings) {
// IndexMetaData indexMetadata = IndexMetaData.builder(randomAlphaOfLengthBetween(1, 20)).settings(existingSettings)
// .numberOfShards(randomIntBetween(1, 5)).numberOfReplicas(randomIntBetween(0, 5)).build();
// Index index = indexMetadata.getIndex();
// ImmutableOpenMap.Builder<String, IndexMetaData> indices = ImmutableOpenMap.<String, IndexMetaData> builder().fPut(index.getName(),
// indexMetadata);
// ClusterState clusterState = ClusterState.builder(ClusterState.EMPTY_STATE).metaData(MetaData.builder().indices(indices.build()))
// .routingTable(RoutingTable.builder()
// .add(IndexRoutingTable.builder(index).addShard(
// TestShardRouting.newShardRouting(new ShardId(index, 0), "node1", true, ShardRoutingState.STARTED)))
// .build())
// .build();
//
// BiConsumer<Settings, Listener> settingsUpdater = (s, l) -> {
// assertEquals(expectedSettings, s);
// l.onSuccess(false);
// };
// assertActionStatus(index, clusterState, null, action, settingsUpdater, false);
// }
//
// private void assertAllocateStatus(Index index, int shards, int replicas, AllocateAction action, Settings.Builder existingSettings,
// Settings.Builder node1Settings, Settings.Builder node2Settings, IndexRoutingTable.Builder indexRoutingTable,
// boolean expectComplete) {
// IndexMetaData indexMetadata = IndexMetaData.builder(index.getName()).settings(existingSettings).numberOfShards(shards)
// .numberOfReplicas(replicas).build();
// ImmutableOpenMap.Builder<String, IndexMetaData> indices = ImmutableOpenMap.<String, IndexMetaData> builder().fPut(index.getName(),
// indexMetadata);
//
// ClusterState clusterState = ClusterState.builder(ClusterState.EMPTY_STATE).metaData(MetaData.builder().indices(indices.build()))
// .nodes(DiscoveryNodes.builder()
// .add(DiscoveryNode.createLocal(node1Settings.build(), new TransportAddress(TransportAddress.META_ADDRESS, 9200),
// "node1"))
// .add(DiscoveryNode.createLocal(node2Settings.build(), new TransportAddress(TransportAddress.META_ADDRESS, 9201),
// "node2")))
// .routingTable(RoutingTable.builder().add(indexRoutingTable).build()).build();
// ClusterSettings clusterSettings = new ClusterSettings(Settings.EMPTY,
// Sets.newHashSet(FilterAllocationDecider.CLUSTER_ROUTING_INCLUDE_GROUP_SETTING,
// FilterAllocationDecider.CLUSTER_ROUTING_EXCLUDE_GROUP_SETTING,
// FilterAllocationDecider.CLUSTER_ROUTING_REQUIRE_GROUP_SETTING));
//
// BiConsumer<Settings, Listener> settingsUpdater = (s, l) -> {
// throw new AssertionError("Unexpected settings update");
// };
//
// assertActionStatus(index, clusterState, clusterSettings, action, settingsUpdater, expectComplete);
// }
//
// private void assertActionStatus(Index index, ClusterState clusterState, ClusterSettings clusterSettings, AllocateAction action,
// BiConsumer<Settings, Listener> settingsUpdater, boolean expectComplete) {
//
// SetOnce<Boolean> actionCompleted = new SetOnce<>();
// action.execute(index, settingsUpdater, clusterState, clusterSettings, new Listener() {
//
// @Override
// public void onSuccess(boolean completed) {
// actionCompleted.set(completed);
// }
//
// @Override
// public void onFailure(Exception e) {
// throw new AssertionError("Unexpected method call");
// }
// });
//
// assertEquals(expectComplete, actionCompleted.get());
// }
//
// private <E> E expectActionFailure(Index index, ClusterState clusterState, ClusterSettings clusterSettings, AllocateAction action,
// BiConsumer<Settings, Listener> settingsUpdater, Class<E> expectedExceptionType) {
//
// SetOnce<E> exceptionThrown = new SetOnce<>();
// action.execute(index, settingsUpdater, clusterState, clusterSettings, new Listener() {
//
// @Override
// public void onSuccess(boolean completed) {
// throw new AssertionError("Unexpected method call");
// }
//
// @SuppressWarnings("unchecked")
// @Override
// public void onFailure(Exception e) {
// assertThat(e, instanceOf(expectedExceptionType));
// exceptionThrown.set((E) e);
// }
// });
//
// return exceptionThrown.get();
// }
private Map<String, String> randomMap(int minEntries, int maxEntries) {
Map<String, String> map = new HashMap<>();

View File

@ -16,7 +16,6 @@ import org.elasticsearch.common.io.stream.Writeable.Reader;
import org.elasticsearch.common.xcontent.XContentParser;
import org.elasticsearch.index.Index;
import org.elasticsearch.test.AbstractSerializingTestCase;
import org.elasticsearch.xpack.core.indexlifecycle.LifecycleAction.Listener;
import org.mockito.Mockito;
import org.mockito.invocation.InvocationOnMock;
import org.mockito.stubbing.Answer;
@ -40,101 +39,101 @@ public class DeleteActionTests extends AbstractSerializingTestCase<DeleteAction>
return DeleteAction::new;
}
public void testExecute() throws Exception {
Index index = new Index(randomAlphaOfLengthBetween(1, 20), randomAlphaOfLengthBetween(1, 20));
Client client = Mockito.mock(Client.class);
AdminClient adminClient = Mockito.mock(AdminClient.class);
IndicesAdminClient indicesClient = Mockito.mock(IndicesAdminClient.class);
Mockito.when(client.admin()).thenReturn(adminClient);
Mockito.when(adminClient.indices()).thenReturn(indicesClient);
Mockito.doAnswer(new Answer<Void>() {
@Override
public Void answer(InvocationOnMock invocation) throws Throwable {
DeleteIndexRequest request = (DeleteIndexRequest) invocation.getArguments()[0];
@SuppressWarnings("unchecked")
ActionListener<DeleteIndexResponse> listener = (ActionListener<DeleteIndexResponse>) invocation.getArguments()[1];
assertNotNull(request);
assertEquals(1, request.indices().length);
assertEquals(index.getName(), request.indices()[0]);
listener.onResponse(null);
return null;
}
}).when(indicesClient).delete(Mockito.any(), Mockito.any());
SetOnce<Boolean> actionCompleted = new SetOnce<>();
DeleteAction action = new DeleteAction();
action.execute(index, client, null, new Listener() {
@Override
public void onSuccess(boolean completed) {
actionCompleted.set(completed);
}
@Override
public void onFailure(Exception e) {
throw new AssertionError("Unexpected method call", e);
}
});
assertEquals(true, actionCompleted.get());
Mockito.verify(client, Mockito.only()).admin();
Mockito.verify(adminClient, Mockito.only()).indices();
Mockito.verify(indicesClient, Mockito.only()).delete(Mockito.any(), Mockito.any());
}
public void testExecuteFailure() throws Exception {
Index index = new Index(randomAlphaOfLengthBetween(1, 20), randomAlphaOfLengthBetween(1, 20));
Exception exception = new RuntimeException();
Client client = Mockito.mock(Client.class);
AdminClient adminClient = Mockito.mock(AdminClient.class);
IndicesAdminClient indicesClient = Mockito.mock(IndicesAdminClient.class);
Mockito.when(client.admin()).thenReturn(adminClient);
Mockito.when(adminClient.indices()).thenReturn(indicesClient);
Mockito.doAnswer(new Answer<Void>() {
@Override
public Void answer(InvocationOnMock invocation) throws Throwable {
DeleteIndexRequest request = (DeleteIndexRequest) invocation.getArguments()[0];
@SuppressWarnings("unchecked")
ActionListener<DeleteIndexResponse> listener = (ActionListener<DeleteIndexResponse>) invocation.getArguments()[1];
assertNotNull(request);
assertEquals(1, request.indices().length);
assertEquals(index.getName(), request.indices()[0]);
listener.onFailure(exception);
;
return null;
}
}).when(indicesClient).delete(Mockito.any(), Mockito.any());
SetOnce<Boolean> exceptionThrown = new SetOnce<>();
DeleteAction action = new DeleteAction();
action.execute(index, client, null, new Listener() {
@Override
public void onSuccess(boolean completed) {
throw new AssertionError("Unexpected method call");
}
@Override
public void onFailure(Exception e) {
assertEquals(exception, e);
exceptionThrown.set(true);
}
});
assertEquals(true, exceptionThrown.get());
Mockito.verify(client, Mockito.only()).admin();
Mockito.verify(adminClient, Mockito.only()).indices();
Mockito.verify(indicesClient, Mockito.only()).delete(Mockito.any(), Mockito.any());
}
// public void testExecute() throws Exception {
// Index index = new Index(randomAlphaOfLengthBetween(1, 20), randomAlphaOfLengthBetween(1, 20));
//
// Client client = Mockito.mock(Client.class);
// AdminClient adminClient = Mockito.mock(AdminClient.class);
// IndicesAdminClient indicesClient = Mockito.mock(IndicesAdminClient.class);
//
// Mockito.when(client.admin()).thenReturn(adminClient);
// Mockito.when(adminClient.indices()).thenReturn(indicesClient);
// Mockito.doAnswer(new Answer<Void>() {
//
// @Override
// public Void answer(InvocationOnMock invocation) throws Throwable {
// DeleteIndexRequest request = (DeleteIndexRequest) invocation.getArguments()[0];
// @SuppressWarnings("unchecked")
// ActionListener<DeleteIndexResponse> listener = (ActionListener<DeleteIndexResponse>) invocation.getArguments()[1];
// assertNotNull(request);
// assertEquals(1, request.indices().length);
// assertEquals(index.getName(), request.indices()[0]);
// listener.onResponse(null);
// return null;
// }
//
// }).when(indicesClient).delete(Mockito.any(), Mockito.any());
//
// SetOnce<Boolean> actionCompleted = new SetOnce<>();
// DeleteAction action = new DeleteAction();
// action.execute(index, client, null, new Listener() {
//
// @Override
// public void onSuccess(boolean completed) {
// actionCompleted.set(completed);
// }
//
// @Override
// public void onFailure(Exception e) {
// throw new AssertionError("Unexpected method call", e);
// }
// });
//
// assertEquals(true, actionCompleted.get());
//
// Mockito.verify(client, Mockito.only()).admin();
// Mockito.verify(adminClient, Mockito.only()).indices();
// Mockito.verify(indicesClient, Mockito.only()).delete(Mockito.any(), Mockito.any());
// }
//
// public void testExecuteFailure() throws Exception {
// Index index = new Index(randomAlphaOfLengthBetween(1, 20), randomAlphaOfLengthBetween(1, 20));
// Exception exception = new RuntimeException();
//
// Client client = Mockito.mock(Client.class);
// AdminClient adminClient = Mockito.mock(AdminClient.class);
// IndicesAdminClient indicesClient = Mockito.mock(IndicesAdminClient.class);
//
// Mockito.when(client.admin()).thenReturn(adminClient);
// Mockito.when(adminClient.indices()).thenReturn(indicesClient);
// Mockito.doAnswer(new Answer<Void>() {
//
// @Override
// public Void answer(InvocationOnMock invocation) throws Throwable {
// DeleteIndexRequest request = (DeleteIndexRequest) invocation.getArguments()[0];
// @SuppressWarnings("unchecked")
// ActionListener<DeleteIndexResponse> listener = (ActionListener<DeleteIndexResponse>) invocation.getArguments()[1];
// assertNotNull(request);
// assertEquals(1, request.indices().length);
// assertEquals(index.getName(), request.indices()[0]);
// listener.onFailure(exception);
// ;
// return null;
// }
//
// }).when(indicesClient).delete(Mockito.any(), Mockito.any());
//
// SetOnce<Boolean> exceptionThrown = new SetOnce<>();
// DeleteAction action = new DeleteAction();
// action.execute(index, client, null, new Listener() {
//
// @Override
// public void onSuccess(boolean completed) {
// throw new AssertionError("Unexpected method call");
// }
//
// @Override
// public void onFailure(Exception e) {
// assertEquals(exception, e);
// exceptionThrown.set(true);
// }
// });
//
// assertEquals(true, exceptionThrown.get());
//
// Mockito.verify(client, Mockito.only()).admin();
// Mockito.verify(adminClient, Mockito.only()).indices();
// Mockito.verify(indicesClient, Mockito.only()).delete(Mockito.any(), Mockito.any());
// }
}

View File

@ -34,7 +34,6 @@ import org.elasticsearch.index.Index;
import org.elasticsearch.index.engine.Segment;
import org.elasticsearch.rest.RestStatus;
import org.elasticsearch.test.AbstractSerializingTestCase;
import org.elasticsearch.xpack.core.indexlifecycle.LifecycleAction.Listener;
import org.mockito.InOrder;
import org.mockito.Mockito;
@ -71,7 +70,7 @@ public class ForceMergeActionTests extends AbstractSerializingTestCase<ForceMerg
}
public void testMissingMaxNumSegments() throws IOException {
BytesReference emptyObject = JsonXContent.contentBuilder().startObject().endObject().bytes();
BytesReference emptyObject = BytesReference.bytes(JsonXContent.contentBuilder().startObject().endObject());
XContentParser parser = XContentHelper.createParser(null, DeprecationHandler.THROW_UNSUPPORTED_OPERATION,
emptyObject, XContentType.JSON);
Exception e = expectThrows(IllegalArgumentException.class, () -> ForceMergeAction.parse(parser));
@ -83,332 +82,332 @@ public class ForceMergeActionTests extends AbstractSerializingTestCase<ForceMerg
assertThat(r.getMessage(), equalTo("[max_num_segments] must be a positive integer"));
}
public void testExecuteSuccessfully() throws Exception {
Index index = new Index(randomAlphaOfLengthBetween(1, 20), randomAlphaOfLengthBetween(1, 20));
ClusterService clusterService = Mockito.mock(ClusterService.class);
IndexMetaData indexMetadata = IndexMetaData.builder(index.getName())
.settings(settings(Version.CURRENT))
.numberOfShards(randomIntBetween(1, 5)).numberOfReplicas(randomIntBetween(0, 5)).build();
ImmutableOpenMap.Builder<String, IndexMetaData> indices = ImmutableOpenMap.<String, IndexMetaData> builder().fPut(index.getName(),
indexMetadata);
ClusterState clusterState = ClusterState.builder(ClusterState.EMPTY_STATE).metaData(MetaData.builder().indices(indices.build()))
.build();
Mockito.when(clusterService.state()).thenReturn(clusterState);
Client client = Mockito.mock(Client.class);
AdminClient adminClient = Mockito.mock(AdminClient.class);
IndicesAdminClient indicesClient = Mockito.mock(IndicesAdminClient.class);
UpdateSettingsResponse updateSettingsResponse = Mockito.mock(UpdateSettingsResponse.class);
IndicesSegmentResponse indicesSegmentResponse = Mockito.mock(IndicesSegmentResponse.class);
IndexSegments indexSegments = Mockito.mock(IndexSegments.class);
IndexShardSegments indexShardSegments = Mockito.mock(IndexShardSegments.class);
Map<Integer, IndexShardSegments> indexShards = Collections.singletonMap(0, indexShardSegments);
ShardSegments shardSegmentsOne = Mockito.mock(ShardSegments.class);
ShardSegments[] shardSegmentsArray = new ShardSegments[] { shardSegmentsOne };
Spliterator<IndexShardSegments> iss = indexShards.values().spliterator();
List<Segment> segments = Arrays.asList(null, null);
ForceMergeResponse forceMergeResponse = Mockito.mock(ForceMergeResponse.class);
Mockito.when(forceMergeResponse.getStatus()).thenReturn(RestStatus.OK);
Mockito.when(indicesSegmentResponse.getStatus()).thenReturn(RestStatus.OK);
Mockito.when(indicesSegmentResponse.getIndices()).thenReturn(Collections.singletonMap(index.getName(), indexSegments));
Mockito.when(indexSegments.spliterator()).thenReturn(iss);
Mockito.when(indexShardSegments.getShards()).thenReturn(shardSegmentsArray);
Mockito.when(shardSegmentsOne.getSegments()).thenReturn(segments);
int maxNumSegments = 1;
Mockito.when(client.admin()).thenReturn(adminClient);
Mockito.when(adminClient.indices()).thenReturn(indicesClient);
Mockito.doAnswer(invocationOnMock -> {
@SuppressWarnings("unchecked")
ActionListener<IndicesSegmentResponse> listener = (ActionListener<IndicesSegmentResponse>) invocationOnMock.getArguments()[1];
listener.onResponse(indicesSegmentResponse);
return null;
}).when(indicesClient).segments(any(), any());
Mockito.doAnswer(invocationOnMock -> {
@SuppressWarnings("unchecked")
ActionListener<UpdateSettingsResponse> listener = (ActionListener<UpdateSettingsResponse>) invocationOnMock.getArguments()[1];
listener.onResponse(updateSettingsResponse);
return null;
}).when(indicesClient).updateSettings(any(), any());
Mockito.doAnswer(invocationOnMock -> {
ForceMergeRequest request = (ForceMergeRequest) invocationOnMock.getArguments()[0];
assertThat(request.maxNumSegments(), equalTo(maxNumSegments));
@SuppressWarnings("unchecked")
ActionListener<ForceMergeResponse> listener = (ActionListener<ForceMergeResponse>) invocationOnMock.getArguments()[1];
listener.onResponse(forceMergeResponse);
return null;
}).when(indicesClient).forceMerge(any(), any());
SetOnce<Boolean> actionCompleted = new SetOnce<>();
ForceMergeAction action = new ForceMergeAction(maxNumSegments);
action.execute(index, client, clusterService, new Listener() {
@Override
public void onSuccess(boolean completed) {
actionCompleted.set(completed);
}
@Override
public void onFailure(Exception e) {
throw new AssertionError("Unexpected method call", e);
}
});
assertEquals(true, actionCompleted.get());
Mockito.verify(client, Mockito.atLeast(1)).admin();
Mockito.verify(adminClient, Mockito.atLeast(1)).indices();
Mockito.verify(indicesClient, Mockito.atLeast(1)).segments(any(), any());
Mockito.verify(indicesClient, Mockito.atLeast(1)).updateSettings(any(), any());
Mockito.verify(indicesClient, Mockito.atLeast(1)).forceMerge(any(), any());
Mockito.verify(clusterService, Mockito.atLeast(1)).state();
}
public void testExecuteWhenReadOnlyAlready() {
Index index = new Index(randomAlphaOfLengthBetween(1, 20), randomAlphaOfLengthBetween(1, 20));
boolean isReadOnlyAlready = true;
ClusterService clusterService = Mockito.mock(ClusterService.class);
IndexMetaData indexMetadata = IndexMetaData.builder(index.getName())
.settings(settings(Version.CURRENT).put(IndexMetaData.SETTING_BLOCKS_WRITE, isReadOnlyAlready))
.numberOfShards(randomIntBetween(1, 5)).numberOfReplicas(randomIntBetween(0, 5)).build();
ImmutableOpenMap.Builder<String, IndexMetaData> indices = ImmutableOpenMap.<String, IndexMetaData> builder().fPut(index.getName(),
indexMetadata);
ClusterState clusterState = ClusterState.builder(ClusterState.EMPTY_STATE).metaData(MetaData.builder().indices(indices.build()))
.build();
Mockito.when(clusterService.state()).thenReturn(clusterState);
Client client = Mockito.mock(Client.class);
AdminClient adminClient = Mockito.mock(AdminClient.class);
IndicesAdminClient indicesClient = Mockito.mock(IndicesAdminClient.class);
IndicesSegmentResponse indicesSegmentResponse = Mockito.mock(IndicesSegmentResponse.class);
IndexSegments indexSegments = Mockito.mock(IndexSegments.class);
IndexShardSegments indexShardSegments = Mockito.mock(IndexShardSegments.class);
Map<Integer, IndexShardSegments> indexShards = Collections.singletonMap(0, indexShardSegments);
ShardSegments shardSegmentsOne = Mockito.mock(ShardSegments.class);
ShardSegments[] shardSegmentsArray = new ShardSegments[] { shardSegmentsOne };
Spliterator<IndexShardSegments> iss = indexShards.values().spliterator();
List<Segment> segments = Arrays.asList(null, null);
Mockito.when(indicesSegmentResponse.getStatus()).thenReturn(RestStatus.OK);
Mockito.when(indicesSegmentResponse.getIndices()).thenReturn(Collections.singletonMap(index.getName(), indexSegments));
Mockito.when(indexSegments.spliterator()).thenReturn(iss);
Mockito.when(indexShardSegments.getShards()).thenReturn(shardSegmentsArray);
Mockito.when(shardSegmentsOne.getSegments()).thenReturn(segments);
int maxNumSegments = Integer.MAX_VALUE;
Mockito.when(client.admin()).thenReturn(adminClient);
Mockito.when(adminClient.indices()).thenReturn(indicesClient);
Mockito.doAnswer(invocationOnMock -> {
@SuppressWarnings("unchecked")
ActionListener<IndicesSegmentResponse> listener = (ActionListener<IndicesSegmentResponse>) invocationOnMock.getArguments()[1];
listener.onResponse(indicesSegmentResponse);
return null;
}).when(indicesClient).segments(any(), any());
SetOnce<Boolean> actionCompleted = new SetOnce<>();
ForceMergeAction action = new ForceMergeAction(maxNumSegments);
action.execute(index, client, clusterService, new Listener() {
@Override
public void onSuccess(boolean completed) {
actionCompleted.set(completed);
}
@Override
public void onFailure(Exception e) {
throw new AssertionError("Unexpected method call", e);
}
});
assertEquals(null, actionCompleted.get());
InOrder inOrder = Mockito.inOrder(clusterService, client, adminClient, indicesClient);
inOrder.verify(clusterService).state();
inOrder.verify(client).admin();
inOrder.verify(adminClient).indices();
inOrder.verify(indicesClient).segments(any(), any());
inOrder.verify(indicesClient).updateSettings(any(), any());
Mockito.verify(indicesClient, Mockito.never()).forceMerge(any(), any());
}
public void testExecuteWithNoNeedToForceMerge() {
Index index = new Index(randomAlphaOfLengthBetween(1, 20), randomAlphaOfLengthBetween(1, 20));
boolean isReadOnlyAlready = false;
ClusterService clusterService = Mockito.mock(ClusterService.class);
IndexMetaData indexMetadata = IndexMetaData.builder(index.getName())
.settings(settings(Version.CURRENT).put(IndexMetaData.SETTING_BLOCKS_WRITE, isReadOnlyAlready))
.numberOfShards(randomIntBetween(1, 5)).numberOfReplicas(randomIntBetween(0, 5)).build();
ImmutableOpenMap.Builder<String, IndexMetaData> indices = ImmutableOpenMap.<String, IndexMetaData> builder().fPut(index.getName(),
indexMetadata);
ClusterState clusterState = ClusterState.builder(ClusterState.EMPTY_STATE).metaData(MetaData.builder().indices(indices.build()))
.build();
Mockito.when(clusterService.state()).thenReturn(clusterState);
Client client = Mockito.mock(Client.class);
AdminClient adminClient = Mockito.mock(AdminClient.class);
IndicesAdminClient indicesClient = Mockito.mock(IndicesAdminClient.class);
IndicesSegmentResponse indicesSegmentResponse = Mockito.mock(IndicesSegmentResponse.class);
IndexSegments indexSegments = Mockito.mock(IndexSegments.class);
IndexShardSegments indexShardSegments = Mockito.mock(IndexShardSegments.class);
Map<Integer, IndexShardSegments> indexShards = Collections.singletonMap(0, indexShardSegments);
ShardSegments shardSegmentsOne = Mockito.mock(ShardSegments.class);
ShardSegments[] shardSegmentsArray = new ShardSegments[] { shardSegmentsOne };
Spliterator<IndexShardSegments> iss = indexShards.values().spliterator();
List<Segment> segments = Arrays.asList(null, null);
Mockito.when(indicesSegmentResponse.getStatus()).thenReturn(RestStatus.OK);
Mockito.when(indicesSegmentResponse.getIndices()).thenReturn(Collections.singletonMap(index.getName(), indexSegments));
Mockito.when(indexSegments.spliterator()).thenReturn(iss);
Mockito.when(indexShardSegments.getShards()).thenReturn(shardSegmentsArray);
Mockito.when(shardSegmentsOne.getSegments()).thenReturn(segments);
int maxNumSegments = Integer.MAX_VALUE;
Mockito.when(client.admin()).thenReturn(adminClient);
Mockito.when(adminClient.indices()).thenReturn(indicesClient);
Mockito.doAnswer(invocationOnMock -> {
@SuppressWarnings("unchecked")
ActionListener<IndicesSegmentResponse> listener = (ActionListener<IndicesSegmentResponse>) invocationOnMock.getArguments()[1];
listener.onResponse(indicesSegmentResponse);
return null;
}).when(indicesClient).segments(any(), any());
SetOnce<Boolean> actionCompleted = new SetOnce<>();
ForceMergeAction action = new ForceMergeAction(maxNumSegments);
action.execute(index, client, clusterService, new Listener() {
@Override
public void onSuccess(boolean completed) {
actionCompleted.set(completed);
}
@Override
public void onFailure(Exception e) {
throw new AssertionError("Unexpected method call", e);
}
});
assertEquals(true, actionCompleted.get());
InOrder inOrder = Mockito.inOrder(clusterService, client, adminClient, indicesClient);
inOrder.verify(clusterService).state();
inOrder.verify(client).admin();
inOrder.verify(adminClient).indices();
inOrder.verify(indicesClient).segments(any(), any());
Mockito.verify(indicesClient, Mockito.never()).updateSettings(any(), any());
Mockito.verify(indicesClient, Mockito.never()).forceMerge(any(), any());
}
public void testCheckSegmentsNeedsMerging() {
Index index = new Index(randomAlphaOfLengthBetween(1, 20), randomAlphaOfLengthBetween(1, 20));
Client client = Mockito.mock(Client.class);
AdminClient adminClient = Mockito.mock(AdminClient.class);
IndicesAdminClient indicesClient = Mockito.mock(IndicesAdminClient.class);
IndicesSegmentResponse indicesSegmentResponse = Mockito.mock(IndicesSegmentResponse.class);
IndexSegments indexSegments = Mockito.mock(IndexSegments.class);
IndexShardSegments indexShardSegments = Mockito.mock(IndexShardSegments.class);
Map<Integer, IndexShardSegments> indexShards = Collections.singletonMap(0, indexShardSegments);
ShardSegments shardSegmentsOne = Mockito.mock(ShardSegments.class);
ShardSegments[] shardSegmentsArray = new ShardSegments[] { shardSegmentsOne };
Spliterator<IndexShardSegments> iss = indexShards.values().spliterator();
List<Segment> segments = Arrays.asList(null, null);
Mockito.when(indicesSegmentResponse.getStatus()).thenReturn(RestStatus.OK);
Mockito.when(indicesSegmentResponse.getIndices()).thenReturn(Collections.singletonMap(index.getName(), indexSegments));
Mockito.when(indexSegments.spliterator()).thenReturn(iss);
Mockito.when(indexShardSegments.getShards()).thenReturn(shardSegmentsArray);
Mockito.when(shardSegmentsOne.getSegments()).thenReturn(segments);
int maxNumSegments = 1;
Mockito.when(client.admin()).thenReturn(adminClient);
Mockito.when(adminClient.indices()).thenReturn(indicesClient);
Mockito.doAnswer(invocationOnMock -> {
@SuppressWarnings("unchecked")
ActionListener<IndicesSegmentResponse> listener = (ActionListener<IndicesSegmentResponse>) invocationOnMock.getArguments()[1];
listener.onResponse(indicesSegmentResponse);
return null;
}).when(indicesClient).segments(any(), any());
SetOnce<Boolean> nextActionCalled = new SetOnce<>();
ForceMergeAction action = new ForceMergeAction(maxNumSegments);
action.checkSegments(index, client, new Listener() {
@Override
public void onSuccess(boolean completed) {
throw new AssertionError("Unexpected method call to onSuccess");
}
@Override
public void onFailure(Exception e) {
throw new AssertionError("Unexpected method call", e);
}
}, r -> nextActionCalled.set(true), r2 -> {throw new AssertionError("unexpected call to action");});
assertEquals(true, nextActionCalled.get());
Mockito.verify(client, Mockito.only()).admin();
Mockito.verify(adminClient, Mockito.only()).indices();
Mockito.verify(indicesClient, Mockito.only()).segments(any(), any());
}
public void testCheckSegmentsNoNeedToMerge() {
Index index = new Index(randomAlphaOfLengthBetween(1, 20), randomAlphaOfLengthBetween(1, 20));
Client client = Mockito.mock(Client.class);
AdminClient adminClient = Mockito.mock(AdminClient.class);
IndicesAdminClient indicesClient = Mockito.mock(IndicesAdminClient.class);
IndicesSegmentResponse indicesSegmentResponse = Mockito.mock(IndicesSegmentResponse.class);
IndexSegments indexSegments = Mockito.mock(IndexSegments.class);
IndexShardSegments indexShardSegments = Mockito.mock(IndexShardSegments.class);
Map<Integer, IndexShardSegments> indexShards = Collections.singletonMap(0, indexShardSegments);
ShardSegments shardSegmentsOne = Mockito.mock(ShardSegments.class);
ShardSegments[] shardSegmentsArray = new ShardSegments[] { shardSegmentsOne };
Spliterator<IndexShardSegments> iss = indexShards.values().spliterator();
List<Segment> segments = Arrays.asList(null, null);
Mockito.when(indicesSegmentResponse.getStatus()).thenReturn(RestStatus.OK);
Mockito.when(indicesSegmentResponse.getIndices()).thenReturn(Collections.singletonMap(index.getName(), indexSegments));
Mockito.when(indexSegments.spliterator()).thenReturn(iss);
Mockito.when(indexShardSegments.getShards()).thenReturn(shardSegmentsArray);
Mockito.when(shardSegmentsOne.getSegments()).thenReturn(segments);
int maxNumSegments = randomIntBetween(2, Integer.MAX_VALUE);
Mockito.when(client.admin()).thenReturn(adminClient);
Mockito.when(adminClient.indices()).thenReturn(indicesClient);
Mockito.doAnswer(invocationOnMock -> {
@SuppressWarnings("unchecked")
ActionListener<IndicesSegmentResponse> listener = (ActionListener<IndicesSegmentResponse>) invocationOnMock.getArguments()[1];
listener.onResponse(indicesSegmentResponse);
return null;
}).when(indicesClient).segments(any(), any());
SetOnce<Boolean> skipActionCalled = new SetOnce<>();
ForceMergeAction action = new ForceMergeAction(maxNumSegments);
action.checkSegments(index, client, new Listener() {
@Override
public void onSuccess(boolean completed) {
throw new AssertionError("Unexpected method call to onSuccess");
}
@Override
public void onFailure(Exception e) {
throw new AssertionError("Unexpected method call", e);
}
}, r -> { throw new AssertionError("next action should not be called"); },
r2 -> skipActionCalled.set(true));
assertTrue(skipActionCalled.get());
Mockito.verify(client, Mockito.only()).admin();
Mockito.verify(adminClient, Mockito.only()).indices();
Mockito.verify(indicesClient, Mockito.only()).segments(any(), any());
}
// public void testExecuteSuccessfully() throws Exception {
// Index index = new Index(randomAlphaOfLengthBetween(1, 20), randomAlphaOfLengthBetween(1, 20));
//
// ClusterService clusterService = Mockito.mock(ClusterService.class);
// IndexMetaData indexMetadata = IndexMetaData.builder(index.getName())
// .settings(settings(Version.CURRENT))
// .numberOfShards(randomIntBetween(1, 5)).numberOfReplicas(randomIntBetween(0, 5)).build();
// ImmutableOpenMap.Builder<String, IndexMetaData> indices = ImmutableOpenMap.<String, IndexMetaData> builder().fPut(index.getName(),
// indexMetadata);
// ClusterState clusterState = ClusterState.builder(ClusterState.EMPTY_STATE).metaData(MetaData.builder().indices(indices.build()))
// .build();
// Mockito.when(clusterService.state()).thenReturn(clusterState);
// Client client = Mockito.mock(Client.class);
// AdminClient adminClient = Mockito.mock(AdminClient.class);
// IndicesAdminClient indicesClient = Mockito.mock(IndicesAdminClient.class);
// UpdateSettingsResponse updateSettingsResponse = Mockito.mock(UpdateSettingsResponse.class);
// IndicesSegmentResponse indicesSegmentResponse = Mockito.mock(IndicesSegmentResponse.class);
// IndexSegments indexSegments = Mockito.mock(IndexSegments.class);
// IndexShardSegments indexShardSegments = Mockito.mock(IndexShardSegments.class);
// Map<Integer, IndexShardSegments> indexShards = Collections.singletonMap(0, indexShardSegments);
// ShardSegments shardSegmentsOne = Mockito.mock(ShardSegments.class);
// ShardSegments[] shardSegmentsArray = new ShardSegments[] { shardSegmentsOne };
// Spliterator<IndexShardSegments> iss = indexShards.values().spliterator();
// List<Segment> segments = Arrays.asList(null, null);
// ForceMergeResponse forceMergeResponse = Mockito.mock(ForceMergeResponse.class);
// Mockito.when(forceMergeResponse.getStatus()).thenReturn(RestStatus.OK);
// Mockito.when(indicesSegmentResponse.getStatus()).thenReturn(RestStatus.OK);
// Mockito.when(indicesSegmentResponse.getIndices()).thenReturn(Collections.singletonMap(index.getName(), indexSegments));
// Mockito.when(indexSegments.spliterator()).thenReturn(iss);
// Mockito.when(indexShardSegments.getShards()).thenReturn(shardSegmentsArray);
// Mockito.when(shardSegmentsOne.getSegments()).thenReturn(segments);
// int maxNumSegments = 1;
//
// Mockito.when(client.admin()).thenReturn(adminClient);
// Mockito.when(adminClient.indices()).thenReturn(indicesClient);
//
// Mockito.doAnswer(invocationOnMock -> {
// @SuppressWarnings("unchecked")
// ActionListener<IndicesSegmentResponse> listener = (ActionListener<IndicesSegmentResponse>) invocationOnMock.getArguments()[1];
// listener.onResponse(indicesSegmentResponse);
// return null;
// }).when(indicesClient).segments(any(), any());
//
// Mockito.doAnswer(invocationOnMock -> {
// @SuppressWarnings("unchecked")
// ActionListener<UpdateSettingsResponse> listener = (ActionListener<UpdateSettingsResponse>) invocationOnMock.getArguments()[1];
// listener.onResponse(updateSettingsResponse);
// return null;
// }).when(indicesClient).updateSettings(any(), any());
//
// Mockito.doAnswer(invocationOnMock -> {
// ForceMergeRequest request = (ForceMergeRequest) invocationOnMock.getArguments()[0];
// assertThat(request.maxNumSegments(), equalTo(maxNumSegments));
// @SuppressWarnings("unchecked")
// ActionListener<ForceMergeResponse> listener = (ActionListener<ForceMergeResponse>) invocationOnMock.getArguments()[1];
// listener.onResponse(forceMergeResponse);
// return null;
// }).when(indicesClient).forceMerge(any(), any());
//
// SetOnce<Boolean> actionCompleted = new SetOnce<>();
// ForceMergeAction action = new ForceMergeAction(maxNumSegments);
// action.execute(index, client, clusterService, new Listener() {
//
// @Override
// public void onSuccess(boolean completed) {
// actionCompleted.set(completed);
// }
//
// @Override
// public void onFailure(Exception e) {
// throw new AssertionError("Unexpected method call", e);
// }
// });
//
// assertEquals(true, actionCompleted.get());
//
// Mockito.verify(client, Mockito.atLeast(1)).admin();
// Mockito.verify(adminClient, Mockito.atLeast(1)).indices();
// Mockito.verify(indicesClient, Mockito.atLeast(1)).segments(any(), any());
// Mockito.verify(indicesClient, Mockito.atLeast(1)).updateSettings(any(), any());
// Mockito.verify(indicesClient, Mockito.atLeast(1)).forceMerge(any(), any());
// Mockito.verify(clusterService, Mockito.atLeast(1)).state();
// }
//
// public void testExecuteWhenReadOnlyAlready() {
// Index index = new Index(randomAlphaOfLengthBetween(1, 20), randomAlphaOfLengthBetween(1, 20));
// boolean isReadOnlyAlready = true;
//
// ClusterService clusterService = Mockito.mock(ClusterService.class);
// IndexMetaData indexMetadata = IndexMetaData.builder(index.getName())
// .settings(settings(Version.CURRENT).put(IndexMetaData.SETTING_BLOCKS_WRITE, isReadOnlyAlready))
// .numberOfShards(randomIntBetween(1, 5)).numberOfReplicas(randomIntBetween(0, 5)).build();
// ImmutableOpenMap.Builder<String, IndexMetaData> indices = ImmutableOpenMap.<String, IndexMetaData> builder().fPut(index.getName(),
// indexMetadata);
// ClusterState clusterState = ClusterState.builder(ClusterState.EMPTY_STATE).metaData(MetaData.builder().indices(indices.build()))
// .build();
// Mockito.when(clusterService.state()).thenReturn(clusterState);
//
// Client client = Mockito.mock(Client.class);
// AdminClient adminClient = Mockito.mock(AdminClient.class);
// IndicesAdminClient indicesClient = Mockito.mock(IndicesAdminClient.class);
// IndicesSegmentResponse indicesSegmentResponse = Mockito.mock(IndicesSegmentResponse.class);
// IndexSegments indexSegments = Mockito.mock(IndexSegments.class);
// IndexShardSegments indexShardSegments = Mockito.mock(IndexShardSegments.class);
// Map<Integer, IndexShardSegments> indexShards = Collections.singletonMap(0, indexShardSegments);
// ShardSegments shardSegmentsOne = Mockito.mock(ShardSegments.class);
// ShardSegments[] shardSegmentsArray = new ShardSegments[] { shardSegmentsOne };
// Spliterator<IndexShardSegments> iss = indexShards.values().spliterator();
// List<Segment> segments = Arrays.asList(null, null);
//
// Mockito.when(indicesSegmentResponse.getStatus()).thenReturn(RestStatus.OK);
// Mockito.when(indicesSegmentResponse.getIndices()).thenReturn(Collections.singletonMap(index.getName(), indexSegments));
// Mockito.when(indexSegments.spliterator()).thenReturn(iss);
// Mockito.when(indexShardSegments.getShards()).thenReturn(shardSegmentsArray);
// Mockito.when(shardSegmentsOne.getSegments()).thenReturn(segments);
// int maxNumSegments = Integer.MAX_VALUE;
//
// Mockito.when(client.admin()).thenReturn(adminClient);
// Mockito.when(adminClient.indices()).thenReturn(indicesClient);
//
//
// Mockito.doAnswer(invocationOnMock -> {
// @SuppressWarnings("unchecked")
// ActionListener<IndicesSegmentResponse> listener = (ActionListener<IndicesSegmentResponse>) invocationOnMock.getArguments()[1];
// listener.onResponse(indicesSegmentResponse);
// return null;
// }).when(indicesClient).segments(any(), any());
//
//
// SetOnce<Boolean> actionCompleted = new SetOnce<>();
// ForceMergeAction action = new ForceMergeAction(maxNumSegments);
// action.execute(index, client, clusterService, new Listener() {
//
// @Override
// public void onSuccess(boolean completed) {
// actionCompleted.set(completed);
// }
//
// @Override
// public void onFailure(Exception e) {
// throw new AssertionError("Unexpected method call", e);
// }
// });
//
// assertEquals(null, actionCompleted.get());
//
// InOrder inOrder = Mockito.inOrder(clusterService, client, adminClient, indicesClient);
// inOrder.verify(clusterService).state();
// inOrder.verify(client).admin();
// inOrder.verify(adminClient).indices();
// inOrder.verify(indicesClient).segments(any(), any());
// inOrder.verify(indicesClient).updateSettings(any(), any());
// Mockito.verify(indicesClient, Mockito.never()).forceMerge(any(), any());
// }
//
// public void testExecuteWithNoNeedToForceMerge() {
// Index index = new Index(randomAlphaOfLengthBetween(1, 20), randomAlphaOfLengthBetween(1, 20));
// boolean isReadOnlyAlready = false;
//
// ClusterService clusterService = Mockito.mock(ClusterService.class);
// IndexMetaData indexMetadata = IndexMetaData.builder(index.getName())
// .settings(settings(Version.CURRENT).put(IndexMetaData.SETTING_BLOCKS_WRITE, isReadOnlyAlready))
// .numberOfShards(randomIntBetween(1, 5)).numberOfReplicas(randomIntBetween(0, 5)).build();
// ImmutableOpenMap.Builder<String, IndexMetaData> indices = ImmutableOpenMap.<String, IndexMetaData> builder().fPut(index.getName(),
// indexMetadata);
// ClusterState clusterState = ClusterState.builder(ClusterState.EMPTY_STATE).metaData(MetaData.builder().indices(indices.build()))
// .build();
// Mockito.when(clusterService.state()).thenReturn(clusterState);
//
// Client client = Mockito.mock(Client.class);
// AdminClient adminClient = Mockito.mock(AdminClient.class);
// IndicesAdminClient indicesClient = Mockito.mock(IndicesAdminClient.class);
// IndicesSegmentResponse indicesSegmentResponse = Mockito.mock(IndicesSegmentResponse.class);
// IndexSegments indexSegments = Mockito.mock(IndexSegments.class);
// IndexShardSegments indexShardSegments = Mockito.mock(IndexShardSegments.class);
// Map<Integer, IndexShardSegments> indexShards = Collections.singletonMap(0, indexShardSegments);
// ShardSegments shardSegmentsOne = Mockito.mock(ShardSegments.class);
// ShardSegments[] shardSegmentsArray = new ShardSegments[] { shardSegmentsOne };
// Spliterator<IndexShardSegments> iss = indexShards.values().spliterator();
// List<Segment> segments = Arrays.asList(null, null);
//
// Mockito.when(indicesSegmentResponse.getStatus()).thenReturn(RestStatus.OK);
// Mockito.when(indicesSegmentResponse.getIndices()).thenReturn(Collections.singletonMap(index.getName(), indexSegments));
// Mockito.when(indexSegments.spliterator()).thenReturn(iss);
// Mockito.when(indexShardSegments.getShards()).thenReturn(shardSegmentsArray);
// Mockito.when(shardSegmentsOne.getSegments()).thenReturn(segments);
// int maxNumSegments = Integer.MAX_VALUE;
//
// Mockito.when(client.admin()).thenReturn(adminClient);
// Mockito.when(adminClient.indices()).thenReturn(indicesClient);
//
//
// Mockito.doAnswer(invocationOnMock -> {
// @SuppressWarnings("unchecked")
// ActionListener<IndicesSegmentResponse> listener = (ActionListener<IndicesSegmentResponse>) invocationOnMock.getArguments()[1];
// listener.onResponse(indicesSegmentResponse);
// return null;
// }).when(indicesClient).segments(any(), any());
//
//
// SetOnce<Boolean> actionCompleted = new SetOnce<>();
// ForceMergeAction action = new ForceMergeAction(maxNumSegments);
// action.execute(index, client, clusterService, new Listener() {
//
// @Override
// public void onSuccess(boolean completed) {
// actionCompleted.set(completed);
// }
//
// @Override
// public void onFailure(Exception e) {
// throw new AssertionError("Unexpected method call", e);
// }
// });
//
// assertEquals(true, actionCompleted.get());
//
// InOrder inOrder = Mockito.inOrder(clusterService, client, adminClient, indicesClient);
// inOrder.verify(clusterService).state();
// inOrder.verify(client).admin();
// inOrder.verify(adminClient).indices();
// inOrder.verify(indicesClient).segments(any(), any());
// Mockito.verify(indicesClient, Mockito.never()).updateSettings(any(), any());
// Mockito.verify(indicesClient, Mockito.never()).forceMerge(any(), any());
// }
//
// public void testCheckSegmentsNeedsMerging() {
// Index index = new Index(randomAlphaOfLengthBetween(1, 20), randomAlphaOfLengthBetween(1, 20));
// Client client = Mockito.mock(Client.class);
// AdminClient adminClient = Mockito.mock(AdminClient.class);
// IndicesAdminClient indicesClient = Mockito.mock(IndicesAdminClient.class);
// IndicesSegmentResponse indicesSegmentResponse = Mockito.mock(IndicesSegmentResponse.class);
// IndexSegments indexSegments = Mockito.mock(IndexSegments.class);
// IndexShardSegments indexShardSegments = Mockito.mock(IndexShardSegments.class);
// Map<Integer, IndexShardSegments> indexShards = Collections.singletonMap(0, indexShardSegments);
// ShardSegments shardSegmentsOne = Mockito.mock(ShardSegments.class);
// ShardSegments[] shardSegmentsArray = new ShardSegments[] { shardSegmentsOne };
// Spliterator<IndexShardSegments> iss = indexShards.values().spliterator();
// List<Segment> segments = Arrays.asList(null, null);
// Mockito.when(indicesSegmentResponse.getStatus()).thenReturn(RestStatus.OK);
// Mockito.when(indicesSegmentResponse.getIndices()).thenReturn(Collections.singletonMap(index.getName(), indexSegments));
// Mockito.when(indexSegments.spliterator()).thenReturn(iss);
// Mockito.when(indexShardSegments.getShards()).thenReturn(shardSegmentsArray);
// Mockito.when(shardSegmentsOne.getSegments()).thenReturn(segments);
// int maxNumSegments = 1;
//
// Mockito.when(client.admin()).thenReturn(adminClient);
// Mockito.when(adminClient.indices()).thenReturn(indicesClient);
//
// Mockito.doAnswer(invocationOnMock -> {
// @SuppressWarnings("unchecked")
// ActionListener<IndicesSegmentResponse> listener = (ActionListener<IndicesSegmentResponse>) invocationOnMock.getArguments()[1];
// listener.onResponse(indicesSegmentResponse);
// return null;
// }).when(indicesClient).segments(any(), any());
//
// SetOnce<Boolean> nextActionCalled = new SetOnce<>();
// ForceMergeAction action = new ForceMergeAction(maxNumSegments);
// action.checkSegments(index, client, new Listener() {
//
// @Override
// public void onSuccess(boolean completed) {
// throw new AssertionError("Unexpected method call to onSuccess");
// }
//
// @Override
// public void onFailure(Exception e) {
// throw new AssertionError("Unexpected method call", e);
// }
// }, r -> nextActionCalled.set(true), r2 -> {throw new AssertionError("unexpected call to action");});
//
// assertEquals(true, nextActionCalled.get());
// Mockito.verify(client, Mockito.only()).admin();
// Mockito.verify(adminClient, Mockito.only()).indices();
// Mockito.verify(indicesClient, Mockito.only()).segments(any(), any());
// }
//
// public void testCheckSegmentsNoNeedToMerge() {
// Index index = new Index(randomAlphaOfLengthBetween(1, 20), randomAlphaOfLengthBetween(1, 20));
// Client client = Mockito.mock(Client.class);
// AdminClient adminClient = Mockito.mock(AdminClient.class);
// IndicesAdminClient indicesClient = Mockito.mock(IndicesAdminClient.class);
// IndicesSegmentResponse indicesSegmentResponse = Mockito.mock(IndicesSegmentResponse.class);
// IndexSegments indexSegments = Mockito.mock(IndexSegments.class);
// IndexShardSegments indexShardSegments = Mockito.mock(IndexShardSegments.class);
// Map<Integer, IndexShardSegments> indexShards = Collections.singletonMap(0, indexShardSegments);
// ShardSegments shardSegmentsOne = Mockito.mock(ShardSegments.class);
// ShardSegments[] shardSegmentsArray = new ShardSegments[] { shardSegmentsOne };
// Spliterator<IndexShardSegments> iss = indexShards.values().spliterator();
// List<Segment> segments = Arrays.asList(null, null);
// Mockito.when(indicesSegmentResponse.getStatus()).thenReturn(RestStatus.OK);
// Mockito.when(indicesSegmentResponse.getIndices()).thenReturn(Collections.singletonMap(index.getName(), indexSegments));
// Mockito.when(indexSegments.spliterator()).thenReturn(iss);
// Mockito.when(indexShardSegments.getShards()).thenReturn(shardSegmentsArray);
// Mockito.when(shardSegmentsOne.getSegments()).thenReturn(segments);
// int maxNumSegments = randomIntBetween(2, Integer.MAX_VALUE);
//
// Mockito.when(client.admin()).thenReturn(adminClient);
// Mockito.when(adminClient.indices()).thenReturn(indicesClient);
//
// Mockito.doAnswer(invocationOnMock -> {
// @SuppressWarnings("unchecked")
// ActionListener<IndicesSegmentResponse> listener = (ActionListener<IndicesSegmentResponse>) invocationOnMock.getArguments()[1];
// listener.onResponse(indicesSegmentResponse);
// return null;
// }).when(indicesClient).segments(any(), any());
//
// SetOnce<Boolean> skipActionCalled = new SetOnce<>();
// ForceMergeAction action = new ForceMergeAction(maxNumSegments);
// action.checkSegments(index, client, new Listener() {
//
// @Override
// public void onSuccess(boolean completed) {
// throw new AssertionError("Unexpected method call to onSuccess");
// }
//
// @Override
// public void onFailure(Exception e) {
// throw new AssertionError("Unexpected method call", e);
// }
// }, r -> { throw new AssertionError("next action should not be called"); },
// r2 -> skipActionCalled.set(true));
//
// assertTrue(skipActionCalled.get());
// Mockito.verify(client, Mockito.only()).admin();
// Mockito.verify(adminClient, Mockito.only()).indices();
// Mockito.verify(indicesClient, Mockito.only()).segments(any(), any());
// }
}

View File

@ -16,10 +16,14 @@ import org.elasticsearch.common.xcontent.ConstructingObjectParser;
import org.elasticsearch.common.xcontent.XContentBuilder;
import org.elasticsearch.common.xcontent.XContentParser;
import org.elasticsearch.index.Index;
import org.elasticsearch.threadpool.ThreadPool;
import java.io.IOException;
import java.util.ArrayList;
import java.util.List;
import java.util.Objects;
import java.util.concurrent.atomic.AtomicLong;
import java.util.function.LongSupplier;
public class MockAction implements LifecycleAction {
public static final ParseField COMPLETED_FIELD = new ParseField("completed");
@ -29,9 +33,10 @@ public class MockAction implements LifecycleAction {
private final AtomicLong executedCount;
private Exception exceptionToThrow = null;
private boolean completeOnExecute = false;
private final List<Step> steps;
private static final ConstructingObjectParser<MockAction, Void> PARSER = new ConstructingObjectParser<>(NAME,
a -> new MockAction((Boolean) a[0], (Long) a[1]));
a -> new MockAction(null, (Boolean) a[0], (Long) a[1]));
static {
PARSER.declareBoolean(ConstructingObjectParser.optionalConstructorArg(), COMPLETED_FIELD);
PARSER.declareLong(ConstructingObjectParser.constructorArg(), EXECUTED_COUNT_FIELD);
@ -41,11 +46,12 @@ public class MockAction implements LifecycleAction {
return PARSER.apply(parser, null);
}
public MockAction() {
this(null, 0);
public MockAction(List<Step> steps) {
this(steps, null, 0);
}
MockAction(Boolean completed, long executedCount) {
MockAction(List<Step> steps, Boolean completed, long executedCount) {
this.steps = steps;
if (completed != null) {
this.completed.set(completed);
}
@ -53,6 +59,12 @@ public class MockAction implements LifecycleAction {
}
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);
@ -76,6 +88,11 @@ public class MockAction implements LifecycleAction {
return NAME;
}
@Override
public List<Step> toSteps(String phase, Index index, Client client, ThreadPool threadPool, LongSupplier nowSupplier) {
return steps;
}
public void setCompleteOnExecute(boolean completeOnExecute) {
this.completeOnExecute = completeOnExecute;
}
@ -102,19 +119,6 @@ public class MockAction implements LifecycleAction {
out.writeLong(executedCount.get());
}
@Override
public void execute(Index index, Client client, ClusterService clusterService, Listener listener) {
executedCount.incrementAndGet();
if (exceptionToThrow == null) {
if (completeOnExecute) {
completed.set(true);
}
listener.onSuccess(completeOnExecute);
} else {
listener.onFailure(exceptionToThrow);
}
}
@Override
public int hashCode() {
return Objects.hash(completed.get(), executedCount.get());
@ -137,5 +141,4 @@ public class MockAction implements LifecycleAction {
public String toString() {
return Strings.toString(this);
}
}

View File

@ -16,7 +16,7 @@ public class MockActionTests extends AbstractSerializingTestCase<MockAction> {
@Override
protected MockAction createTestInstance() {
return new MockAction(randomBoolean() ? null : randomBoolean(), randomLong());
return new MockAction(null, randomBoolean() ? null : randomBoolean(), randomLong());
}
@Override
@ -43,145 +43,145 @@ public class MockActionTests extends AbstractSerializingTestCase<MockAction> {
default:
throw new AssertionError("Illegal randomisation branch");
}
return new MockAction(completed, executedCount);
return new MockAction(null, completed, executedCount);
}
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());
}
// 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());
// }
}

View File

@ -1,98 +0,0 @@
/*
* 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 java.util.function.LongSupplier;
public abstract class MockIndexLifecycleContext implements IndexLifecycleContext {
private final String targetName;
private String phase;
private String action;
private Exception exceptionToThrow;
private int numberOfReplicas;
private LongSupplier nowSupplier;
private long phaseTime;
private long actionTime;
public MockIndexLifecycleContext(String targetName, String initialPhase, String initialAction, int numberOfReplicas,
LongSupplier nowSupplier) {
this.targetName = targetName;
this.phase = initialPhase;
this.action = initialAction;
this.numberOfReplicas = numberOfReplicas;
this.nowSupplier = nowSupplier;
this.phaseTime = -1L;
this.actionTime = -1L;
}
public void failOnSetters(Exception exceptionToThrow) {
this.exceptionToThrow = exceptionToThrow;
}
@Override
public void setPhase(String phase, Listener listener) {
if (exceptionToThrow != null) {
listener.onFailure(exceptionToThrow);
return;
}
this.phase = phase;
this.phaseTime = nowSupplier.getAsLong();
this.action = "";
this.actionTime = -1L;
listener.onSuccess();
}
@Override
public void setAction(String action, Listener listener) {
if (exceptionToThrow != null) {
listener.onFailure(exceptionToThrow);
return;
}
this.action = action;
this.actionTime = nowSupplier.getAsLong();
listener.onSuccess();
}
@Override
public String getAction() {
return action;
}
@Override
public long getActionTime() {
return actionTime;
}
@Override
public String getPhase() {
return phase;
}
@Override
public long getPhaseTime() {
return phaseTime;
}
@Override
public String getLifecycleTarget() {
return targetName;
}
@Override
public int getNumberOfReplicas() {
return numberOfReplicas;
}
@Override
public abstract boolean canExecute(Phase phase);
@Override
public void executeAction(LifecycleAction action, LifecycleAction.Listener listener) {
action.execute(null, null, null, listener);
}
}

View File

@ -1,256 +0,0 @@
/*
* 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.apache.lucene.util.SetOnce;
import org.elasticsearch.test.ESTestCase;
import org.elasticsearch.xpack.core.indexlifecycle.IndexLifecycleContext.Listener;
public class MockIndexLifecycleContextTests extends ESTestCase {
public void testSetPhase() {
String targetName = randomAlphaOfLengthBetween(1, 20);
String newPhase = randomAlphaOfLengthBetween(1, 20);
long now = randomNonNegativeLong();
MockIndexLifecycleContext context = new MockIndexLifecycleContext(targetName,
randomAlphaOfLengthBetween(1, 20), randomAlphaOfLengthBetween(1, 20), randomIntBetween(0, 10), () -> now) {
@Override
public boolean canExecute(Phase phase) {
throw new AssertionError("canExecute should not have been called.");
}
};
// Use setOnce so it throws an error if we call the listener multiple
// times
SetOnce<Boolean> listenerCalled = new SetOnce<>();
context.setPhase(newPhase, new Listener() {
@Override
public void onSuccess() {
listenerCalled.set(true);
}
@Override
public void onFailure(Exception e) {
throw new AssertionError("Unexpected Error", e);
}
});
assertEquals(true, listenerCalled.get());
assertEquals(newPhase, context.getPhase());
assertEquals("", context.getAction());
assertEquals(now, context.getPhaseTime());
assertEquals(-1L, context.getActionTime());
assertEquals(targetName, context.getLifecycleTarget());
}
public void testGetPhase() {
String phase = randomAlphaOfLengthBetween(1, 20);
MockIndexLifecycleContext context = new MockIndexLifecycleContext(randomAlphaOfLengthBetween(1, 20), phase,
randomAlphaOfLengthBetween(1, 20), randomIntBetween(0, 10), () -> 0L) {
@Override
public boolean canExecute(Phase phase) {
throw new AssertionError("canExecute should not have been called.");
}
};
assertEquals(phase, context.getPhase());
}
public void testGetReplicas() {
int replicas = randomIntBetween(1, 10);
MockIndexLifecycleContext context = new MockIndexLifecycleContext(randomAlphaOfLengthBetween(1, 20),
randomAlphaOfLengthBetween(1, 20), randomAlphaOfLengthBetween(1, 20), replicas, () -> 0L) {
@Override
public boolean canExecute(Phase phase) {
throw new AssertionError("canExecute should not have been called.");
}
};
assertEquals(replicas, context.getNumberOfReplicas());
}
public void testSetAction() {
String targetName = randomAlphaOfLengthBetween(1, 20);
String phase = randomAlphaOfLengthBetween(1, 20);
String newAction = randomAlphaOfLengthBetween(1, 20);
long now = randomNonNegativeLong();
MockIndexLifecycleContext context = new MockIndexLifecycleContext(targetName, phase,
randomAlphaOfLengthBetween(1, 20), randomIntBetween(0, 10), () -> now) {
@Override
public boolean canExecute(Phase phase) {
throw new AssertionError("canExecute should not have been called.");
}
};
// Use setOnce so it throws an error if we call the listener multiple
// times
SetOnce<Boolean> listenerCalled = new SetOnce<>();
context.setAction(newAction, new Listener() {
@Override
public void onSuccess() {
listenerCalled.set(true);
}
@Override
public void onFailure(Exception e) {
throw new AssertionError("Unexpected Error", e);
}
});
assertEquals(true, listenerCalled.get());
assertEquals(newAction, context.getAction());
assertEquals(phase, context.getPhase());
assertEquals(-1L, context.getPhaseTime()); // no setPhase was called to set this yet
assertEquals(now, context.getActionTime());
assertEquals(targetName, context.getLifecycleTarget());
}
public void testGetAction() {
String action = randomAlphaOfLengthBetween(1, 20);
MockIndexLifecycleContext context = new MockIndexLifecycleContext(randomAlphaOfLengthBetween(1, 20),
randomAlphaOfLengthBetween(1, 20), action, randomIntBetween(0, 10), () -> 0L) {
@Override
public boolean canExecute(Phase phase) {
throw new AssertionError("canExecute should not have been called.");
}
};
assertEquals(action, context.getAction());
}
public void testGetLifecycleTarget() {
String target = randomAlphaOfLengthBetween(1, 20);
MockIndexLifecycleContext context = new MockIndexLifecycleContext(target, randomAlphaOfLengthBetween(1, 20),
randomAlphaOfLengthBetween(1, 20), randomIntBetween(0, 10), () -> 0L) {
@Override
public boolean canExecute(Phase phase) {
throw new AssertionError("canExecute should not have been called.");
}
};
assertEquals(target, context.getLifecycleTarget());
}
public void testExecuteAction() {
MockIndexLifecycleContext context = new MockIndexLifecycleContext(randomAlphaOfLengthBetween(1, 20),
randomAlphaOfLengthBetween(1, 20), randomAlphaOfLengthBetween(1, 20), randomIntBetween(0, 10), () -> 0L) {
@Override
public boolean canExecute(Phase phase) {
throw new AssertionError("canExecute should not have been called.");
}
};
MockAction action = new MockAction();
action.setCompleteOnExecute(true);
assertFalse(action.wasCompleted());
assertEquals(0L, action.getExecutedCount());
SetOnce<Boolean> listenerCalled = new SetOnce<>();
context.executeAction(action, 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 testFailOnPhaseSetter() {
String phase = randomAlphaOfLengthBetween(1, 20);
String action = randomAlphaOfLengthBetween(1, 20);
MockIndexLifecycleContext context = new MockIndexLifecycleContext(randomAlphaOfLengthBetween(1, 20),
phase, action, randomIntBetween(0, 10), () -> 0L) {
@Override
public boolean canExecute(Phase phase) {
throw new AssertionError("canExecute should not have been called.");
}
};
RuntimeException exception = new RuntimeException();
context.failOnSetters(exception);
SetOnce<Exception> listenerCalled = new SetOnce<>();
context.setPhase(randomAlphaOfLengthBetween(1, 20), new Listener() {
@Override
public void onSuccess() {
throw new AssertionError("Unexpected method call");
}
@Override
public void onFailure(Exception e) {
listenerCalled.set(e);
}
});
assertSame(exception, listenerCalled.get());
assertEquals(phase, context.getPhase());
assertEquals(action, context.getAction());
}
public void testFailOnActionSetter() {
String phase = randomAlphaOfLengthBetween(1, 20);
String action = randomAlphaOfLengthBetween(1, 20);
MockIndexLifecycleContext context = new MockIndexLifecycleContext(randomAlphaOfLengthBetween(1, 20),
phase, action, randomIntBetween(0, 10), () -> 0L) {
@Override
public boolean canExecute(Phase phase) {
throw new AssertionError("canExecute should not have been called.");
}
};
RuntimeException exception = new RuntimeException();
context.failOnSetters(exception);
SetOnce<Exception> listenerCalled = new SetOnce<>();
context.setAction(randomAlphaOfLengthBetween(1, 20), new Listener() {
@Override
public void onSuccess() {
throw new AssertionError("Unexpected method call");
}
@Override
public void onFailure(Exception e) {
listenerCalled.set(e);
}
});
assertSame(exception, listenerCalled.get());
assertEquals(phase, context.getPhase());
assertEquals(action, context.getAction());
}
}

View File

@ -29,7 +29,7 @@ import org.elasticsearch.common.xcontent.XContentParser;
import org.elasticsearch.index.Index;
import org.elasticsearch.index.shard.ShardId;
import org.elasticsearch.test.AbstractSerializingTestCase;
import org.elasticsearch.xpack.core.indexlifecycle.LifecycleAction.Listener;
import org.mockito.Mockito;
import org.mockito.invocation.InvocationOnMock;
import org.mockito.stubbing.Answer;
@ -64,310 +64,310 @@ public class ReplicasActionTests extends AbstractSerializingTestCase<ReplicasAct
assertEquals("[" + ReplicasAction.NUMBER_OF_REPLICAS_FIELD.getPreferredName() + "] must be >= 0", exception.getMessage());
}
public void testExecuteDifferentReplicaCount() {
int existingNumReplicas = randomIntBetween(0, 10);
int newNumReplicas = randomValueOtherThan(existingNumReplicas, () -> randomIntBetween(0, 10));
ReplicasAction action = new ReplicasAction(newNumReplicas);
IndexMetaData indexMetadata = IndexMetaData.builder(randomAlphaOfLengthBetween(1, 20))
.settings(Settings.builder().put("index.version.created", Version.CURRENT.id)).numberOfShards(randomIntBetween(1, 5))
.numberOfReplicas(existingNumReplicas).build();
Index index = indexMetadata.getIndex();
ImmutableOpenMap.Builder<String, IndexMetaData> indices = ImmutableOpenMap.<String, IndexMetaData> builder().fPut(index.getName(),
indexMetadata);
ClusterState clusterstate = ClusterState.builder(ClusterState.EMPTY_STATE).metaData(MetaData.builder().indices(indices.build()))
.build();
Client client = Mockito.mock(Client.class);
AdminClient adminClient = Mockito.mock(AdminClient.class);
IndicesAdminClient indicesClient = Mockito.mock(IndicesAdminClient.class);
ClusterService clusterService = Mockito.mock(ClusterService.class);
Mockito.when(client.admin()).thenReturn(adminClient);
Mockito.when(adminClient.indices()).thenReturn(indicesClient);
Mockito.doAnswer(new Answer<Void>() {
@Override
public Void answer(InvocationOnMock invocation) throws Throwable {
UpdateSettingsRequest request = (UpdateSettingsRequest) invocation.getArguments()[0];
@SuppressWarnings("unchecked")
ActionListener<UpdateSettingsResponse> listener = (ActionListener<UpdateSettingsResponse>) invocation.getArguments()[1];
Settings expectedSettings = Settings.builder().put(IndexMetaData.SETTING_NUMBER_OF_REPLICAS, newNumReplicas).build();
UpdateSettingsTestHelper.assertSettingsRequest(request, expectedSettings, index.getName());
listener.onResponse(UpdateSettingsTestHelper.createMockResponse(true));
return null;
}
}).when(indicesClient).updateSettings(Mockito.any(), Mockito.any());
Mockito.when(clusterService.state()).thenReturn(clusterstate);
SetOnce<Boolean> actionCompleted = new SetOnce<>();
action.execute(index, client, clusterService, new Listener() {
@Override
public void onSuccess(boolean completed) {
actionCompleted.set(completed);
}
@Override
public void onFailure(Exception e) {
throw new AssertionError("Unexpected method call", e);
}
});
assertEquals(false, actionCompleted.get());
Mockito.verify(client, Mockito.only()).admin();
Mockito.verify(adminClient, Mockito.only()).indices();
Mockito.verify(indicesClient, Mockito.only()).updateSettings(Mockito.any(), Mockito.any());
Mockito.verify(clusterService, Mockito.only()).state();
}
public void testExecuteUpdateReplicaCountFailure() {
int existingNumReplicas = randomIntBetween(0, 10);
int newNumReplicas = randomValueOtherThan(existingNumReplicas, () -> randomIntBetween(0, 10));
ReplicasAction action = new ReplicasAction(newNumReplicas);
Exception exception = new RuntimeException();
IndexMetaData indexMetadata = IndexMetaData.builder(randomAlphaOfLengthBetween(1, 20))
.settings(Settings.builder().put("index.version.created", Version.CURRENT.id)).numberOfShards(randomIntBetween(1, 5))
.numberOfReplicas(existingNumReplicas).build();
Index index = indexMetadata.getIndex();
ImmutableOpenMap.Builder<String, IndexMetaData> indices = ImmutableOpenMap.<String, IndexMetaData> builder().fPut(index.getName(),
indexMetadata);
ClusterState clusterstate = ClusterState.builder(ClusterState.EMPTY_STATE).metaData(MetaData.builder().indices(indices.build()))
.build();
Client client = Mockito.mock(Client.class);
AdminClient adminClient = Mockito.mock(AdminClient.class);
IndicesAdminClient indicesClient = Mockito.mock(IndicesAdminClient.class);
ClusterService clusterService = Mockito.mock(ClusterService.class);
Mockito.when(client.admin()).thenReturn(adminClient);
Mockito.when(adminClient.indices()).thenReturn(indicesClient);
Mockito.doAnswer(new Answer<Void>() {
@Override
public Void answer(InvocationOnMock invocation) throws Throwable {
UpdateSettingsRequest request = (UpdateSettingsRequest) invocation.getArguments()[0];
@SuppressWarnings("unchecked")
ActionListener<UpdateSettingsResponse> listener = (ActionListener<UpdateSettingsResponse>) invocation.getArguments()[1];
Settings expectedSettings = Settings.builder().put(IndexMetaData.SETTING_NUMBER_OF_REPLICAS, newNumReplicas).build();
UpdateSettingsTestHelper.assertSettingsRequest(request, expectedSettings, index.getName());
listener.onFailure(exception);
return null;
}
}).when(indicesClient).updateSettings(Mockito.any(), Mockito.any());
Mockito.when(clusterService.state()).thenReturn(clusterstate);
SetOnce<Boolean> exceptionThrown = new SetOnce<>();
action.execute(index, client, clusterService, new Listener() {
@Override
public void onSuccess(boolean completed) {
throw new AssertionError("Unexpected method call");
}
@Override
public void onFailure(Exception e) {
exceptionThrown.set(true);
}
});
assertEquals(true, exceptionThrown.get());
Mockito.verify(client, Mockito.only()).admin();
Mockito.verify(adminClient, Mockito.only()).indices();
Mockito.verify(indicesClient, Mockito.only()).updateSettings(Mockito.any(), Mockito.any());
Mockito.verify(clusterService, Mockito.only()).state();
}
public void testExecuteAllocationNotComplete() {
ReplicasAction action = new ReplicasAction(randomIntBetween(1, 10));
int numberOfShards = randomIntBetween(1, 5);
int numberOfReplicas = action.getNumberOfReplicas();
IndexMetaData indexMetadata = IndexMetaData.builder(randomAlphaOfLengthBetween(1, 20))
.settings(Settings.builder().put("index.version.created", Version.CURRENT.id)).numberOfShards(numberOfShards)
.numberOfReplicas(numberOfReplicas).build();
Index index = indexMetadata.getIndex();
ImmutableOpenMap.Builder<String, IndexMetaData> indices = ImmutableOpenMap.<String, IndexMetaData> builder().fPut(index.getName(),
indexMetadata);
IndexRoutingTable.Builder indexRoutingTable = IndexRoutingTable.builder(index);
for (int shard = 0; shard < numberOfShards; shard++) {
for (int replica = 0; replica < numberOfReplicas + 1; replica++) {
ShardRoutingState state;
if (replica == 0) {
state = ShardRoutingState.STARTED;
} else if ((replica == numberOfReplicas) || randomBoolean()) {
state = randomFrom(ShardRoutingState.UNASSIGNED, ShardRoutingState.INITIALIZING);
} else {
state = ShardRoutingState.STARTED;
}
String nodeId = "node" + replica;
if (ShardRoutingState.UNASSIGNED.equals(state)) {
nodeId = null;
}
indexRoutingTable.addShard(TestShardRouting.newShardRouting(new ShardId(index, shard), nodeId, replica == 0, state));
}
}
ClusterState clusterstate = ClusterState.builder(ClusterState.EMPTY_STATE).metaData(MetaData.builder().indices(indices.build()))
.routingTable(RoutingTable.builder().add(indexRoutingTable).build())
.build();
Client client = Mockito.mock(Client.class);
AdminClient adminClient = Mockito.mock(AdminClient.class);
IndicesAdminClient indicesClient = Mockito.mock(IndicesAdminClient.class);
ClusterService clusterService = Mockito.mock(ClusterService.class);
Mockito.when(client.admin()).thenReturn(adminClient);
Mockito.when(adminClient.indices()).thenReturn(indicesClient);
Mockito.when(clusterService.state()).thenReturn(clusterstate);
SetOnce<Boolean> actionCompleted = new SetOnce<>();
action.execute(index, client, clusterService, new Listener() {
@Override
public void onSuccess(boolean completed) {
actionCompleted.set(completed);
}
@Override
public void onFailure(Exception e) {
throw new AssertionError("Unexpected method call", e);
}
});
assertEquals(false, actionCompleted.get());
Mockito.verify(clusterService, Mockito.times(2)).state();
Mockito.verify(client, Mockito.never()).admin();
Mockito.verify(adminClient, Mockito.never()).indices();
Mockito.verify(indicesClient, Mockito.never()).updateSettings(Mockito.any(), Mockito.any());
}
public void testExecuteAllocationUnassignedPrimaries() {
ReplicasAction action = createTestInstance();
int numberOfShards = randomIntBetween(1, 5);
int numberOfReplicas = action.getNumberOfReplicas();
IndexMetaData indexMetadata = IndexMetaData.builder(randomAlphaOfLengthBetween(1, 20))
.settings(Settings.builder().put("index.version.created", Version.CURRENT.id)).numberOfShards(numberOfShards)
.numberOfReplicas(numberOfReplicas).build();
Index index = indexMetadata.getIndex();
ImmutableOpenMap.Builder<String, IndexMetaData> indices = ImmutableOpenMap.<String, IndexMetaData> builder().fPut(index.getName(),
indexMetadata);
IndexRoutingTable.Builder indexRoutingTable = IndexRoutingTable.builder(index);
for (int shard = 0; shard < numberOfShards; shard++) {
boolean unassignedPrimary = shard == 0 || randomBoolean();
for (int replica = 0; replica < numberOfReplicas + 1; replica++) {
ShardRoutingState state;
if (unassignedPrimary) {
state = ShardRoutingState.UNASSIGNED;
} else if (replica == 0) {
state = ShardRoutingState.STARTED;
} else if ((replica == numberOfReplicas) || randomBoolean()) {
state = randomFrom(ShardRoutingState.UNASSIGNED, ShardRoutingState.INITIALIZING);
} else {
state = ShardRoutingState.STARTED;
}
String nodeId = "node" + replica;
if (ShardRoutingState.UNASSIGNED.equals(state)) {
nodeId = null;
}
indexRoutingTable.addShard(TestShardRouting.newShardRouting(new ShardId(index, shard), nodeId, replica == 0, state));
}
}
ClusterState clusterstate = ClusterState.builder(ClusterState.EMPTY_STATE).metaData(MetaData.builder().indices(indices.build()))
.routingTable(RoutingTable.builder().add(indexRoutingTable).build()).build();
Client client = Mockito.mock(Client.class);
AdminClient adminClient = Mockito.mock(AdminClient.class);
IndicesAdminClient indicesClient = Mockito.mock(IndicesAdminClient.class);
ClusterService clusterService = Mockito.mock(ClusterService.class);
Mockito.when(client.admin()).thenReturn(adminClient);
Mockito.when(adminClient.indices()).thenReturn(indicesClient);
Mockito.when(clusterService.state()).thenReturn(clusterstate);
SetOnce<Boolean> actionCompleted = new SetOnce<>();
action.execute(index, client, clusterService, new Listener() {
@Override
public void onSuccess(boolean completed) {
actionCompleted.set(completed);
}
@Override
public void onFailure(Exception e) {
throw new AssertionError("Unexpected method call", e);
}
});
assertEquals(false, actionCompleted.get());
Mockito.verify(clusterService, Mockito.times(2)).state();
Mockito.verify(client, Mockito.never()).admin();
Mockito.verify(adminClient, Mockito.never()).indices();
Mockito.verify(indicesClient, Mockito.never()).updateSettings(Mockito.any(), Mockito.any());
}
public void testExecuteAllocationComplete() {
ReplicasAction action = createTestInstance();
int numberOfShards = randomIntBetween(1, 5);
int numberOfReplicas = action.getNumberOfReplicas();
IndexMetaData indexMetadata = IndexMetaData.builder(randomAlphaOfLengthBetween(1, 20))
.settings(Settings.builder().put("index.version.created", Version.CURRENT.id)).numberOfShards(numberOfShards)
.numberOfReplicas(numberOfReplicas).build();
Index index = indexMetadata.getIndex();
ImmutableOpenMap.Builder<String, IndexMetaData> indices = ImmutableOpenMap.<String, IndexMetaData> builder().fPut(index.getName(),
indexMetadata);
IndexRoutingTable.Builder indexRoutingTable = IndexRoutingTable.builder(index);
for (int shard = 0; shard < numberOfShards; shard++) {
for (int replica = 0; replica < numberOfReplicas + 1; replica++) {
ShardRoutingState state = ShardRoutingState.STARTED;
String nodeId = "node" + replica;
indexRoutingTable.addShard(TestShardRouting.newShardRouting(new ShardId(index, shard), nodeId, replica == 0, state));
}
}
ClusterState clusterstate = ClusterState.builder(ClusterState.EMPTY_STATE).metaData(MetaData.builder().indices(indices.build()))
.routingTable(RoutingTable.builder().add(indexRoutingTable).build()).build();
Client client = Mockito.mock(Client.class);
AdminClient adminClient = Mockito.mock(AdminClient.class);
IndicesAdminClient indicesClient = Mockito.mock(IndicesAdminClient.class);
ClusterService clusterService = Mockito.mock(ClusterService.class);
Mockito.when(client.admin()).thenReturn(adminClient);
Mockito.when(adminClient.indices()).thenReturn(indicesClient);
Mockito.when(clusterService.state()).thenReturn(clusterstate);
SetOnce<Boolean> actionCompleted = new SetOnce<>();
action.execute(index, client, clusterService, new Listener() {
@Override
public void onSuccess(boolean completed) {
actionCompleted.set(completed);
}
@Override
public void onFailure(Exception e) {
throw new AssertionError("Unexpected method call", e);
}
});
assertEquals(true, actionCompleted.get());
Mockito.verify(clusterService, Mockito.times(2)).state();
Mockito.verify(client, Mockito.never()).admin();
Mockito.verify(adminClient, Mockito.never()).indices();
Mockito.verify(indicesClient, Mockito.never()).updateSettings(Mockito.any(), Mockito.any());
}
// public void testExecuteDifferentReplicaCount() {
// int existingNumReplicas = randomIntBetween(0, 10);
// int newNumReplicas = randomValueOtherThan(existingNumReplicas, () -> randomIntBetween(0, 10));
//
// ReplicasAction action = new ReplicasAction(newNumReplicas);
//
// IndexMetaData indexMetadata = IndexMetaData.builder(randomAlphaOfLengthBetween(1, 20))
// .settings(Settings.builder().put("index.version.created", Version.CURRENT.id)).numberOfShards(randomIntBetween(1, 5))
// .numberOfReplicas(existingNumReplicas).build();
// Index index = indexMetadata.getIndex();
// ImmutableOpenMap.Builder<String, IndexMetaData> indices = ImmutableOpenMap.<String, IndexMetaData> builder().fPut(index.getName(),
// indexMetadata);
// ClusterState clusterstate = ClusterState.builder(ClusterState.EMPTY_STATE).metaData(MetaData.builder().indices(indices.build()))
// .build();
//
// Client client = Mockito.mock(Client.class);
// AdminClient adminClient = Mockito.mock(AdminClient.class);
// IndicesAdminClient indicesClient = Mockito.mock(IndicesAdminClient.class);
// ClusterService clusterService = Mockito.mock(ClusterService.class);
//
// Mockito.when(client.admin()).thenReturn(adminClient);
// Mockito.when(adminClient.indices()).thenReturn(indicesClient);
// Mockito.doAnswer(new Answer<Void>() {
//
// @Override
// public Void answer(InvocationOnMock invocation) throws Throwable {
// UpdateSettingsRequest request = (UpdateSettingsRequest) invocation.getArguments()[0];
// @SuppressWarnings("unchecked")
// ActionListener<UpdateSettingsResponse> listener = (ActionListener<UpdateSettingsResponse>) invocation.getArguments()[1];
// Settings expectedSettings = Settings.builder().put(IndexMetaData.SETTING_NUMBER_OF_REPLICAS, newNumReplicas).build();
// UpdateSettingsTestHelper.assertSettingsRequest(request, expectedSettings, index.getName());
// listener.onResponse(UpdateSettingsTestHelper.createMockResponse(true));
// return null;
// }
//
// }).when(indicesClient).updateSettings(Mockito.any(), Mockito.any());
// Mockito.when(clusterService.state()).thenReturn(clusterstate);
//
// SetOnce<Boolean> actionCompleted = new SetOnce<>();
// action.execute(index, client, clusterService, new Listener() {
//
// @Override
// public void onSuccess(boolean completed) {
// actionCompleted.set(completed);
// }
//
// @Override
// public void onFailure(Exception e) {
// throw new AssertionError("Unexpected method call", e);
// }
// });
//
// assertEquals(false, actionCompleted.get());
//
// Mockito.verify(client, Mockito.only()).admin();
// Mockito.verify(adminClient, Mockito.only()).indices();
// Mockito.verify(indicesClient, Mockito.only()).updateSettings(Mockito.any(), Mockito.any());
// Mockito.verify(clusterService, Mockito.only()).state();
// }
//
// public void testExecuteUpdateReplicaCountFailure() {
// int existingNumReplicas = randomIntBetween(0, 10);
// int newNumReplicas = randomValueOtherThan(existingNumReplicas, () -> randomIntBetween(0, 10));
//
// ReplicasAction action = new ReplicasAction(newNumReplicas);
// Exception exception = new RuntimeException();
//
// IndexMetaData indexMetadata = IndexMetaData.builder(randomAlphaOfLengthBetween(1, 20))
// .settings(Settings.builder().put("index.version.created", Version.CURRENT.id)).numberOfShards(randomIntBetween(1, 5))
// .numberOfReplicas(existingNumReplicas).build();
// Index index = indexMetadata.getIndex();
// ImmutableOpenMap.Builder<String, IndexMetaData> indices = ImmutableOpenMap.<String, IndexMetaData> builder().fPut(index.getName(),
// indexMetadata);
// ClusterState clusterstate = ClusterState.builder(ClusterState.EMPTY_STATE).metaData(MetaData.builder().indices(indices.build()))
// .build();
//
// Client client = Mockito.mock(Client.class);
// AdminClient adminClient = Mockito.mock(AdminClient.class);
// IndicesAdminClient indicesClient = Mockito.mock(IndicesAdminClient.class);
// ClusterService clusterService = Mockito.mock(ClusterService.class);
//
// Mockito.when(client.admin()).thenReturn(adminClient);
// Mockito.when(adminClient.indices()).thenReturn(indicesClient);
// Mockito.doAnswer(new Answer<Void>() {
//
// @Override
// public Void answer(InvocationOnMock invocation) throws Throwable {
// UpdateSettingsRequest request = (UpdateSettingsRequest) invocation.getArguments()[0];
// @SuppressWarnings("unchecked")
// ActionListener<UpdateSettingsResponse> listener = (ActionListener<UpdateSettingsResponse>) invocation.getArguments()[1];
// Settings expectedSettings = Settings.builder().put(IndexMetaData.SETTING_NUMBER_OF_REPLICAS, newNumReplicas).build();
// UpdateSettingsTestHelper.assertSettingsRequest(request, expectedSettings, index.getName());
// listener.onFailure(exception);
// return null;
// }
//
// }).when(indicesClient).updateSettings(Mockito.any(), Mockito.any());
// Mockito.when(clusterService.state()).thenReturn(clusterstate);
//
// SetOnce<Boolean> exceptionThrown = new SetOnce<>();
// action.execute(index, client, clusterService, new Listener() {
//
// @Override
// public void onSuccess(boolean completed) {
// throw new AssertionError("Unexpected method call");
// }
//
// @Override
// public void onFailure(Exception e) {
// exceptionThrown.set(true);
// }
// });
//
// assertEquals(true, exceptionThrown.get());
//
// Mockito.verify(client, Mockito.only()).admin();
// Mockito.verify(adminClient, Mockito.only()).indices();
// Mockito.verify(indicesClient, Mockito.only()).updateSettings(Mockito.any(), Mockito.any());
// Mockito.verify(clusterService, Mockito.only()).state();
// }
//
// public void testExecuteAllocationNotComplete() {
//
// ReplicasAction action = new ReplicasAction(randomIntBetween(1, 10));
//
// int numberOfShards = randomIntBetween(1, 5);
// int numberOfReplicas = action.getNumberOfReplicas();
// IndexMetaData indexMetadata = IndexMetaData.builder(randomAlphaOfLengthBetween(1, 20))
// .settings(Settings.builder().put("index.version.created", Version.CURRENT.id)).numberOfShards(numberOfShards)
// .numberOfReplicas(numberOfReplicas).build();
// Index index = indexMetadata.getIndex();
// ImmutableOpenMap.Builder<String, IndexMetaData> indices = ImmutableOpenMap.<String, IndexMetaData> builder().fPut(index.getName(),
// indexMetadata);
// IndexRoutingTable.Builder indexRoutingTable = IndexRoutingTable.builder(index);
// for (int shard = 0; shard < numberOfShards; shard++) {
// for (int replica = 0; replica < numberOfReplicas + 1; replica++) {
// ShardRoutingState state;
// if (replica == 0) {
// state = ShardRoutingState.STARTED;
// } else if ((replica == numberOfReplicas) || randomBoolean()) {
// state = randomFrom(ShardRoutingState.UNASSIGNED, ShardRoutingState.INITIALIZING);
// } else {
// state = ShardRoutingState.STARTED;
// }
// String nodeId = "node" + replica;
// if (ShardRoutingState.UNASSIGNED.equals(state)) {
// nodeId = null;
// }
// indexRoutingTable.addShard(TestShardRouting.newShardRouting(new ShardId(index, shard), nodeId, replica == 0, state));
// }
// }
// ClusterState clusterstate = ClusterState.builder(ClusterState.EMPTY_STATE).metaData(MetaData.builder().indices(indices.build()))
// .routingTable(RoutingTable.builder().add(indexRoutingTable).build())
// .build();
//
// Client client = Mockito.mock(Client.class);
// AdminClient adminClient = Mockito.mock(AdminClient.class);
// IndicesAdminClient indicesClient = Mockito.mock(IndicesAdminClient.class);
// ClusterService clusterService = Mockito.mock(ClusterService.class);
//
// Mockito.when(client.admin()).thenReturn(adminClient);
// Mockito.when(adminClient.indices()).thenReturn(indicesClient);
// Mockito.when(clusterService.state()).thenReturn(clusterstate);
//
// SetOnce<Boolean> actionCompleted = new SetOnce<>();
// action.execute(index, client, clusterService, new Listener() {
//
// @Override
// public void onSuccess(boolean completed) {
// actionCompleted.set(completed);
// }
//
// @Override
// public void onFailure(Exception e) {
// throw new AssertionError("Unexpected method call", e);
// }
// });
//
// assertEquals(false, actionCompleted.get());
//
// Mockito.verify(clusterService, Mockito.times(2)).state();
// Mockito.verify(client, Mockito.never()).admin();
// Mockito.verify(adminClient, Mockito.never()).indices();
// Mockito.verify(indicesClient, Mockito.never()).updateSettings(Mockito.any(), Mockito.any());
// }
//
// public void testExecuteAllocationUnassignedPrimaries() {
//
// ReplicasAction action = createTestInstance();
//
// int numberOfShards = randomIntBetween(1, 5);
// int numberOfReplicas = action.getNumberOfReplicas();
// IndexMetaData indexMetadata = IndexMetaData.builder(randomAlphaOfLengthBetween(1, 20))
// .settings(Settings.builder().put("index.version.created", Version.CURRENT.id)).numberOfShards(numberOfShards)
// .numberOfReplicas(numberOfReplicas).build();
// Index index = indexMetadata.getIndex();
// ImmutableOpenMap.Builder<String, IndexMetaData> indices = ImmutableOpenMap.<String, IndexMetaData> builder().fPut(index.getName(),
// indexMetadata);
// IndexRoutingTable.Builder indexRoutingTable = IndexRoutingTable.builder(index);
// for (int shard = 0; shard < numberOfShards; shard++) {
// boolean unassignedPrimary = shard == 0 || randomBoolean();
// for (int replica = 0; replica < numberOfReplicas + 1; replica++) {
// ShardRoutingState state;
// if (unassignedPrimary) {
// state = ShardRoutingState.UNASSIGNED;
// } else if (replica == 0) {
// state = ShardRoutingState.STARTED;
// } else if ((replica == numberOfReplicas) || randomBoolean()) {
// state = randomFrom(ShardRoutingState.UNASSIGNED, ShardRoutingState.INITIALIZING);
// } else {
// state = ShardRoutingState.STARTED;
// }
// String nodeId = "node" + replica;
// if (ShardRoutingState.UNASSIGNED.equals(state)) {
// nodeId = null;
// }
// indexRoutingTable.addShard(TestShardRouting.newShardRouting(new ShardId(index, shard), nodeId, replica == 0, state));
// }
// }
// ClusterState clusterstate = ClusterState.builder(ClusterState.EMPTY_STATE).metaData(MetaData.builder().indices(indices.build()))
// .routingTable(RoutingTable.builder().add(indexRoutingTable).build()).build();
//
// Client client = Mockito.mock(Client.class);
// AdminClient adminClient = Mockito.mock(AdminClient.class);
// IndicesAdminClient indicesClient = Mockito.mock(IndicesAdminClient.class);
// ClusterService clusterService = Mockito.mock(ClusterService.class);
//
// Mockito.when(client.admin()).thenReturn(adminClient);
// Mockito.when(adminClient.indices()).thenReturn(indicesClient);
// Mockito.when(clusterService.state()).thenReturn(clusterstate);
//
// SetOnce<Boolean> actionCompleted = new SetOnce<>();
// action.execute(index, client, clusterService, new Listener() {
//
// @Override
// public void onSuccess(boolean completed) {
// actionCompleted.set(completed);
// }
//
// @Override
// public void onFailure(Exception e) {
// throw new AssertionError("Unexpected method call", e);
// }
// });
//
// assertEquals(false, actionCompleted.get());
//
// Mockito.verify(clusterService, Mockito.times(2)).state();
// Mockito.verify(client, Mockito.never()).admin();
// Mockito.verify(adminClient, Mockito.never()).indices();
// Mockito.verify(indicesClient, Mockito.never()).updateSettings(Mockito.any(), Mockito.any());
// }
//
// public void testExecuteAllocationComplete() {
//
// ReplicasAction action = createTestInstance();
//
// int numberOfShards = randomIntBetween(1, 5);
// int numberOfReplicas = action.getNumberOfReplicas();
// IndexMetaData indexMetadata = IndexMetaData.builder(randomAlphaOfLengthBetween(1, 20))
// .settings(Settings.builder().put("index.version.created", Version.CURRENT.id)).numberOfShards(numberOfShards)
// .numberOfReplicas(numberOfReplicas).build();
// Index index = indexMetadata.getIndex();
// ImmutableOpenMap.Builder<String, IndexMetaData> indices = ImmutableOpenMap.<String, IndexMetaData> builder().fPut(index.getName(),
// indexMetadata);
// IndexRoutingTable.Builder indexRoutingTable = IndexRoutingTable.builder(index);
// for (int shard = 0; shard < numberOfShards; shard++) {
// for (int replica = 0; replica < numberOfReplicas + 1; replica++) {
// ShardRoutingState state = ShardRoutingState.STARTED;
// String nodeId = "node" + replica;
// indexRoutingTable.addShard(TestShardRouting.newShardRouting(new ShardId(index, shard), nodeId, replica == 0, state));
// }
// }
// ClusterState clusterstate = ClusterState.builder(ClusterState.EMPTY_STATE).metaData(MetaData.builder().indices(indices.build()))
// .routingTable(RoutingTable.builder().add(indexRoutingTable).build()).build();
//
// Client client = Mockito.mock(Client.class);
// AdminClient adminClient = Mockito.mock(AdminClient.class);
// IndicesAdminClient indicesClient = Mockito.mock(IndicesAdminClient.class);
// ClusterService clusterService = Mockito.mock(ClusterService.class);
//
// Mockito.when(client.admin()).thenReturn(adminClient);
// Mockito.when(adminClient.indices()).thenReturn(indicesClient);
// Mockito.when(clusterService.state()).thenReturn(clusterstate);
//
// SetOnce<Boolean> actionCompleted = new SetOnce<>();
// action.execute(index, client, clusterService, new Listener() {
//
// @Override
// public void onSuccess(boolean completed) {
// actionCompleted.set(completed);
// }
//
// @Override
// public void onFailure(Exception e) {
// throw new AssertionError("Unexpected method call", e);
// }
// });
//
// assertEquals(true, actionCompleted.get());
//
// Mockito.verify(clusterService, Mockito.times(2)).state();
// Mockito.verify(client, Mockito.never()).admin();
// Mockito.verify(adminClient, Mockito.never()).indices();
// Mockito.verify(indicesClient, Mockito.never()).updateSettings(Mockito.any(), Mockito.any());
// }
}

View File

@ -32,8 +32,8 @@ import org.elasticsearch.common.unit.TimeValue;
import org.elasticsearch.common.xcontent.XContentParser;
import org.elasticsearch.index.Index;
import org.elasticsearch.test.AbstractSerializingTestCase;
import org.elasticsearch.xpack.core.indexlifecycle.LifecycleAction.Listener;
import org.elasticsearch.xpack.core.indexlifecycle.RolloverAction;
import org.mockito.Mockito;
import org.mockito.invocation.InvocationOnMock;
import org.mockito.stubbing.Answer;
@ -105,247 +105,247 @@ public class RolloverActionTests extends AbstractSerializingTestCase<RolloverAct
assertEquals(RolloverAction.ALIAS_FIELD.getPreferredName() + " must be not be null", exception.getMessage());
}
public void testExecute() throws Exception {
Index index = new Index(randomAlphaOfLengthBetween(1, 20), randomAlphaOfLengthBetween(1, 20));
RolloverAction action = createTestInstance();
IndexMetaData indexMetadata = IndexMetaData.builder(index.getName())
.settings(Settings.builder().put("index.version.created", Version.CURRENT.id))
.putAlias(AliasMetaData.builder(action.getAlias()))
.numberOfShards(randomIntBetween(1, 5)).numberOfReplicas(randomIntBetween(0, 5)).build();
ImmutableOpenMap.Builder<String, IndexMetaData> indices = ImmutableOpenMap.<String, IndexMetaData> builder().fPut(index.getName(),
indexMetadata);
ClusterState clusterstate = ClusterState.builder(ClusterState.EMPTY_STATE).metaData(MetaData.builder().indices(indices.build()))
.build();
Client client = Mockito.mock(Client.class);
AdminClient adminClient = Mockito.mock(AdminClient.class);
IndicesAdminClient indicesClient = Mockito.mock(IndicesAdminClient.class);
ClusterService clusterService = Mockito.mock(ClusterService.class);
Mockito.when(client.admin()).thenReturn(adminClient);
Mockito.when(adminClient.indices()).thenReturn(indicesClient);
Mockito.doAnswer(new Answer<Void>() {
@Override
public Void answer(InvocationOnMock invocation) throws Throwable {
RolloverRequest request = (RolloverRequest) invocation.getArguments()[0];
@SuppressWarnings("unchecked")
ActionListener<RolloverResponse> listener = (ActionListener<RolloverResponse>) invocation.getArguments()[1];
Set<Condition<?>> expectedConditions = new HashSet<>();
if (action.getMaxAge() != null) {
expectedConditions.add(new MaxAgeCondition(action.getMaxAge()));
}
if (action.getMaxSize() != null) {
expectedConditions.add(new MaxSizeCondition(action.getMaxSize()));
}
if (action.getMaxDocs() != null) {
expectedConditions.add(new MaxDocsCondition(action.getMaxDocs()));
}
RolloverIndexTestHelper.assertRolloverIndexRequest(request, action.getAlias(), expectedConditions);
listener.onResponse(RolloverIndexTestHelper.createMockResponse(request, true));
return null;
}
}).when(indicesClient).rolloverIndex(Mockito.any(), Mockito.any());
Mockito.when(clusterService.state()).thenReturn(clusterstate);
SetOnce<Boolean> actionCompleted = new SetOnce<>();
action.execute(index, client, clusterService, new Listener() {
@Override
public void onSuccess(boolean completed) {
actionCompleted.set(completed);
}
@Override
public void onFailure(Exception e) {
throw new AssertionError("Unexpected method call", e);
}
});
assertEquals(true, actionCompleted.get());
Mockito.verify(client, Mockito.only()).admin();
Mockito.verify(adminClient, Mockito.only()).indices();
Mockito.verify(indicesClient, Mockito.only()).rolloverIndex(Mockito.any(), Mockito.any());
Mockito.verify(clusterService, Mockito.only()).state();
}
public void testExecuteNotComplete() throws Exception {
Index index = new Index(randomAlphaOfLengthBetween(1, 20), randomAlphaOfLengthBetween(1, 20));
RolloverAction action = createTestInstance();
IndexMetaData indexMetadata = IndexMetaData.builder(index.getName())
.settings(Settings.builder().put("index.version.created", Version.CURRENT.id))
.putAlias(AliasMetaData.builder(action.getAlias())).numberOfShards(randomIntBetween(1, 5))
.numberOfReplicas(randomIntBetween(0, 5)).build();
ImmutableOpenMap.Builder<String, IndexMetaData> indices = ImmutableOpenMap.<String, IndexMetaData> builder().fPut(index.getName(),
indexMetadata);
ClusterState clusterstate = ClusterState.builder(ClusterState.EMPTY_STATE).metaData(MetaData.builder().indices(indices.build()))
.build();
Client client = Mockito.mock(Client.class);
AdminClient adminClient = Mockito.mock(AdminClient.class);
IndicesAdminClient indicesClient = Mockito.mock(IndicesAdminClient.class);
ClusterService clusterService = Mockito.mock(ClusterService.class);
Mockito.when(client.admin()).thenReturn(adminClient);
Mockito.when(adminClient.indices()).thenReturn(indicesClient);
Mockito.doAnswer(new Answer<Void>() {
@Override
public Void answer(InvocationOnMock invocation) throws Throwable {
RolloverRequest request = (RolloverRequest) invocation.getArguments()[0];
@SuppressWarnings("unchecked")
ActionListener<RolloverResponse> listener = (ActionListener<RolloverResponse>) invocation.getArguments()[1];
Set<Condition<?>> expectedConditions = new HashSet<>();
if (action.getMaxAge() != null) {
expectedConditions.add(new MaxAgeCondition(action.getMaxAge()));
}
if (action.getMaxSize() != null) {
expectedConditions.add(new MaxSizeCondition(action.getMaxSize()));
}
if (action.getMaxDocs() != null) {
expectedConditions.add(new MaxDocsCondition(action.getMaxDocs()));
}
RolloverIndexTestHelper.assertRolloverIndexRequest(request, action.getAlias(), expectedConditions);
listener.onResponse(RolloverIndexTestHelper.createMockResponse(request, false));
return null;
}
}).when(indicesClient).rolloverIndex(Mockito.any(), Mockito.any());
Mockito.when(clusterService.state()).thenReturn(clusterstate);
SetOnce<Boolean> actionCompleted = new SetOnce<>();
action.execute(index, client, clusterService, new Listener() {
@Override
public void onSuccess(boolean completed) {
actionCompleted.set(completed);
}
@Override
public void onFailure(Exception e) {
throw new AssertionError("Unexpected method call", e);
}
});
assertEquals(false, actionCompleted.get());
Mockito.verify(client, Mockito.only()).admin();
Mockito.verify(adminClient, Mockito.only()).indices();
Mockito.verify(indicesClient, Mockito.only()).rolloverIndex(Mockito.any(), Mockito.any());
Mockito.verify(clusterService, Mockito.only()).state();
}
public void testExecuteAlreadyCompleted() throws Exception {
Index index = new Index(randomAlphaOfLengthBetween(1, 20), randomAlphaOfLengthBetween(1, 20));
RolloverAction action = createTestInstance();
IndexMetaData indexMetadata = IndexMetaData.builder(index.getName())
.settings(Settings.builder().put("index.version.created", Version.CURRENT.id)).numberOfShards(randomIntBetween(1, 5))
.numberOfReplicas(randomIntBetween(0, 5)).build();
ImmutableOpenMap.Builder<String, IndexMetaData> indices = ImmutableOpenMap.<String, IndexMetaData> builder().fPut(index.getName(),
indexMetadata);
ClusterState clusterstate = ClusterState.builder(ClusterState.EMPTY_STATE).metaData(MetaData.builder().indices(indices.build()))
.build();
Client client = Mockito.mock(Client.class);
AdminClient adminClient = Mockito.mock(AdminClient.class);
IndicesAdminClient indicesClient = Mockito.mock(IndicesAdminClient.class);
ClusterService clusterService = Mockito.mock(ClusterService.class);
Mockito.when(clusterService.state()).thenReturn(clusterstate);
SetOnce<Boolean> actionCompleted = new SetOnce<>();
action.execute(index, client, clusterService, new Listener() {
@Override
public void onSuccess(boolean completed) {
actionCompleted.set(completed);
}
@Override
public void onFailure(Exception e) {
throw new AssertionError("Unexpected method call", e);
}
});
assertEquals(true, actionCompleted.get());
Mockito.verify(clusterService, Mockito.only()).state();
Mockito.verifyZeroInteractions(client, adminClient, indicesClient);
}
public void testExecuteFailure() throws Exception {
Index index = new Index(randomAlphaOfLengthBetween(1, 20), randomAlphaOfLengthBetween(1, 20));
Exception exception = new RuntimeException();
RolloverAction action = createTestInstance();
IndexMetaData indexMetadata = IndexMetaData.builder(index.getName())
.settings(Settings.builder().put("index.version.created", Version.CURRENT.id))
.putAlias(AliasMetaData.builder(action.getAlias())).numberOfShards(randomIntBetween(1, 5))
.numberOfReplicas(randomIntBetween(0, 5)).build();
ImmutableOpenMap.Builder<String, IndexMetaData> indices = ImmutableOpenMap.<String, IndexMetaData> builder().fPut(index.getName(),
indexMetadata);
ClusterState clusterstate = ClusterState.builder(ClusterState.EMPTY_STATE).metaData(MetaData.builder().indices(indices.build()))
.build();
Client client = Mockito.mock(Client.class);
AdminClient adminClient = Mockito.mock(AdminClient.class);
IndicesAdminClient indicesClient = Mockito.mock(IndicesAdminClient.class);
ClusterService clusterService = Mockito.mock(ClusterService.class);
Mockito.when(client.admin()).thenReturn(adminClient);
Mockito.when(adminClient.indices()).thenReturn(indicesClient);
Mockito.doAnswer(new Answer<Void>() {
@Override
public Void answer(InvocationOnMock invocation) throws Throwable {
RolloverRequest request = (RolloverRequest) invocation.getArguments()[0];
@SuppressWarnings("unchecked")
ActionListener<RolloverResponse> listener = (ActionListener<RolloverResponse>) invocation.getArguments()[1];
Set<Condition<?>> expectedConditions = new HashSet<>();
if (action.getMaxAge() != null) {
expectedConditions.add(new MaxAgeCondition(action.getMaxAge()));
}
if (action.getMaxSize() != null) {
expectedConditions.add(new MaxSizeCondition(action.getMaxSize()));
}
if (action.getMaxDocs() != null) {
expectedConditions.add(new MaxDocsCondition(action.getMaxDocs()));
}
RolloverIndexTestHelper.assertRolloverIndexRequest(request, action.getAlias(), expectedConditions);
listener.onFailure(exception);
return null;
}
}).when(indicesClient).rolloverIndex(Mockito.any(), Mockito.any());
Mockito.when(clusterService.state()).thenReturn(clusterstate);
SetOnce<Boolean> exceptionThrown = new SetOnce<>();
action.execute(index, client, clusterService, new Listener() {
@Override
public void onSuccess(boolean completed) {
throw new AssertionError("Unexpected method call");
}
@Override
public void onFailure(Exception e) {
assertEquals(exception, e);
exceptionThrown.set(true);
}
});
assertEquals(true, exceptionThrown.get());
Mockito.verify(client, Mockito.only()).admin();
Mockito.verify(adminClient, Mockito.only()).indices();
Mockito.verify(indicesClient, Mockito.only()).rolloverIndex(Mockito.any(), Mockito.any());
}
// public void testExecute() throws Exception {
// Index index = new Index(randomAlphaOfLengthBetween(1, 20), randomAlphaOfLengthBetween(1, 20));
//
// RolloverAction action = createTestInstance();
// IndexMetaData indexMetadata = IndexMetaData.builder(index.getName())
// .settings(Settings.builder().put("index.version.created", Version.CURRENT.id))
// .putAlias(AliasMetaData.builder(action.getAlias()))
// .numberOfShards(randomIntBetween(1, 5)).numberOfReplicas(randomIntBetween(0, 5)).build();
// ImmutableOpenMap.Builder<String, IndexMetaData> indices = ImmutableOpenMap.<String, IndexMetaData> builder().fPut(index.getName(),
// indexMetadata);
// ClusterState clusterstate = ClusterState.builder(ClusterState.EMPTY_STATE).metaData(MetaData.builder().indices(indices.build()))
// .build();
//
// Client client = Mockito.mock(Client.class);
// AdminClient adminClient = Mockito.mock(AdminClient.class);
// IndicesAdminClient indicesClient = Mockito.mock(IndicesAdminClient.class);
// ClusterService clusterService = Mockito.mock(ClusterService.class);
//
// Mockito.when(client.admin()).thenReturn(adminClient);
// Mockito.when(adminClient.indices()).thenReturn(indicesClient);
// Mockito.doAnswer(new Answer<Void>() {
//
// @Override
// public Void answer(InvocationOnMock invocation) throws Throwable {
// RolloverRequest request = (RolloverRequest) invocation.getArguments()[0];
// @SuppressWarnings("unchecked")
// ActionListener<RolloverResponse> listener = (ActionListener<RolloverResponse>) invocation.getArguments()[1];
// Set<Condition<?>> expectedConditions = new HashSet<>();
// if (action.getMaxAge() != null) {
// expectedConditions.add(new MaxAgeCondition(action.getMaxAge()));
// }
// if (action.getMaxSize() != null) {
// expectedConditions.add(new MaxSizeCondition(action.getMaxSize()));
// }
// if (action.getMaxDocs() != null) {
// expectedConditions.add(new MaxDocsCondition(action.getMaxDocs()));
// }
// RolloverIndexTestHelper.assertRolloverIndexRequest(request, action.getAlias(), expectedConditions);
// listener.onResponse(RolloverIndexTestHelper.createMockResponse(request, true));
// return null;
// }
//
// }).when(indicesClient).rolloverIndex(Mockito.any(), Mockito.any());
// Mockito.when(clusterService.state()).thenReturn(clusterstate);
//
// SetOnce<Boolean> actionCompleted = new SetOnce<>();
// action.execute(index, client, clusterService, new Listener() {
//
// @Override
// public void onSuccess(boolean completed) {
// actionCompleted.set(completed);
// }
//
// @Override
// public void onFailure(Exception e) {
// throw new AssertionError("Unexpected method call", e);
// }
// });
//
// assertEquals(true, actionCompleted.get());
//
// Mockito.verify(client, Mockito.only()).admin();
// Mockito.verify(adminClient, Mockito.only()).indices();
// Mockito.verify(indicesClient, Mockito.only()).rolloverIndex(Mockito.any(), Mockito.any());
// Mockito.verify(clusterService, Mockito.only()).state();
// }
//
// public void testExecuteNotComplete() throws Exception {
// Index index = new Index(randomAlphaOfLengthBetween(1, 20), randomAlphaOfLengthBetween(1, 20));
//
// RolloverAction action = createTestInstance();
// IndexMetaData indexMetadata = IndexMetaData.builder(index.getName())
// .settings(Settings.builder().put("index.version.created", Version.CURRENT.id))
// .putAlias(AliasMetaData.builder(action.getAlias())).numberOfShards(randomIntBetween(1, 5))
// .numberOfReplicas(randomIntBetween(0, 5)).build();
// ImmutableOpenMap.Builder<String, IndexMetaData> indices = ImmutableOpenMap.<String, IndexMetaData> builder().fPut(index.getName(),
// indexMetadata);
// ClusterState clusterstate = ClusterState.builder(ClusterState.EMPTY_STATE).metaData(MetaData.builder().indices(indices.build()))
// .build();
//
// Client client = Mockito.mock(Client.class);
// AdminClient adminClient = Mockito.mock(AdminClient.class);
// IndicesAdminClient indicesClient = Mockito.mock(IndicesAdminClient.class);
// ClusterService clusterService = Mockito.mock(ClusterService.class);
//
// Mockito.when(client.admin()).thenReturn(adminClient);
// Mockito.when(adminClient.indices()).thenReturn(indicesClient);
// Mockito.doAnswer(new Answer<Void>() {
//
// @Override
// public Void answer(InvocationOnMock invocation) throws Throwable {
// RolloverRequest request = (RolloverRequest) invocation.getArguments()[0];
// @SuppressWarnings("unchecked")
// ActionListener<RolloverResponse> listener = (ActionListener<RolloverResponse>) invocation.getArguments()[1];
// Set<Condition<?>> expectedConditions = new HashSet<>();
// if (action.getMaxAge() != null) {
// expectedConditions.add(new MaxAgeCondition(action.getMaxAge()));
// }
// if (action.getMaxSize() != null) {
// expectedConditions.add(new MaxSizeCondition(action.getMaxSize()));
// }
// if (action.getMaxDocs() != null) {
// expectedConditions.add(new MaxDocsCondition(action.getMaxDocs()));
// }
// RolloverIndexTestHelper.assertRolloverIndexRequest(request, action.getAlias(), expectedConditions);
// listener.onResponse(RolloverIndexTestHelper.createMockResponse(request, false));
// return null;
// }
//
// }).when(indicesClient).rolloverIndex(Mockito.any(), Mockito.any());
// Mockito.when(clusterService.state()).thenReturn(clusterstate);
//
// SetOnce<Boolean> actionCompleted = new SetOnce<>();
// action.execute(index, client, clusterService, new Listener() {
//
// @Override
// public void onSuccess(boolean completed) {
// actionCompleted.set(completed);
// }
//
// @Override
// public void onFailure(Exception e) {
// throw new AssertionError("Unexpected method call", e);
// }
// });
//
// assertEquals(false, actionCompleted.get());
//
// Mockito.verify(client, Mockito.only()).admin();
// Mockito.verify(adminClient, Mockito.only()).indices();
// Mockito.verify(indicesClient, Mockito.only()).rolloverIndex(Mockito.any(), Mockito.any());
// Mockito.verify(clusterService, Mockito.only()).state();
// }
//
// public void testExecuteAlreadyCompleted() throws Exception {
// Index index = new Index(randomAlphaOfLengthBetween(1, 20), randomAlphaOfLengthBetween(1, 20));
//
// RolloverAction action = createTestInstance();
//
// IndexMetaData indexMetadata = IndexMetaData.builder(index.getName())
// .settings(Settings.builder().put("index.version.created", Version.CURRENT.id)).numberOfShards(randomIntBetween(1, 5))
// .numberOfReplicas(randomIntBetween(0, 5)).build();
// ImmutableOpenMap.Builder<String, IndexMetaData> indices = ImmutableOpenMap.<String, IndexMetaData> builder().fPut(index.getName(),
// indexMetadata);
// ClusterState clusterstate = ClusterState.builder(ClusterState.EMPTY_STATE).metaData(MetaData.builder().indices(indices.build()))
// .build();
//
// Client client = Mockito.mock(Client.class);
// AdminClient adminClient = Mockito.mock(AdminClient.class);
// IndicesAdminClient indicesClient = Mockito.mock(IndicesAdminClient.class);
// ClusterService clusterService = Mockito.mock(ClusterService.class);
//
// Mockito.when(clusterService.state()).thenReturn(clusterstate);
//
// SetOnce<Boolean> actionCompleted = new SetOnce<>();
// action.execute(index, client, clusterService, new Listener() {
//
// @Override
// public void onSuccess(boolean completed) {
// actionCompleted.set(completed);
// }
//
// @Override
// public void onFailure(Exception e) {
// throw new AssertionError("Unexpected method call", e);
// }
// });
//
// assertEquals(true, actionCompleted.get());
//
// Mockito.verify(clusterService, Mockito.only()).state();
// Mockito.verifyZeroInteractions(client, adminClient, indicesClient);
// }
//
// public void testExecuteFailure() throws Exception {
// Index index = new Index(randomAlphaOfLengthBetween(1, 20), randomAlphaOfLengthBetween(1, 20));
// Exception exception = new RuntimeException();
//
// RolloverAction action = createTestInstance();
//
// IndexMetaData indexMetadata = IndexMetaData.builder(index.getName())
// .settings(Settings.builder().put("index.version.created", Version.CURRENT.id))
// .putAlias(AliasMetaData.builder(action.getAlias())).numberOfShards(randomIntBetween(1, 5))
// .numberOfReplicas(randomIntBetween(0, 5)).build();
// ImmutableOpenMap.Builder<String, IndexMetaData> indices = ImmutableOpenMap.<String, IndexMetaData> builder().fPut(index.getName(),
// indexMetadata);
// ClusterState clusterstate = ClusterState.builder(ClusterState.EMPTY_STATE).metaData(MetaData.builder().indices(indices.build()))
// .build();
//
// Client client = Mockito.mock(Client.class);
// AdminClient adminClient = Mockito.mock(AdminClient.class);
// IndicesAdminClient indicesClient = Mockito.mock(IndicesAdminClient.class);
// ClusterService clusterService = Mockito.mock(ClusterService.class);
//
// Mockito.when(client.admin()).thenReturn(adminClient);
// Mockito.when(adminClient.indices()).thenReturn(indicesClient);
// Mockito.doAnswer(new Answer<Void>() {
//
// @Override
// public Void answer(InvocationOnMock invocation) throws Throwable {
// RolloverRequest request = (RolloverRequest) invocation.getArguments()[0];
// @SuppressWarnings("unchecked")
// ActionListener<RolloverResponse> listener = (ActionListener<RolloverResponse>) invocation.getArguments()[1];
// Set<Condition<?>> expectedConditions = new HashSet<>();
// if (action.getMaxAge() != null) {
// expectedConditions.add(new MaxAgeCondition(action.getMaxAge()));
// }
// if (action.getMaxSize() != null) {
// expectedConditions.add(new MaxSizeCondition(action.getMaxSize()));
// }
// if (action.getMaxDocs() != null) {
// expectedConditions.add(new MaxDocsCondition(action.getMaxDocs()));
// }
// RolloverIndexTestHelper.assertRolloverIndexRequest(request, action.getAlias(), expectedConditions);
// listener.onFailure(exception);
// return null;
// }
//
// }).when(indicesClient).rolloverIndex(Mockito.any(), Mockito.any());
// Mockito.when(clusterService.state()).thenReturn(clusterstate);
//
// SetOnce<Boolean> exceptionThrown = new SetOnce<>();
// action.execute(index, client, clusterService, new Listener() {
//
// @Override
// public void onSuccess(boolean completed) {
// throw new AssertionError("Unexpected method call");
// }
//
// @Override
// public void onFailure(Exception e) {
// assertEquals(exception, e);
// exceptionThrown.set(true);
// }
// });
//
// assertEquals(true, exceptionThrown.get());
//
// Mockito.verify(client, Mockito.only()).admin();
// Mockito.verify(adminClient, Mockito.only()).indices();
// Mockito.verify(indicesClient, Mockito.only()).rolloverIndex(Mockito.any(), Mockito.any());
// }
}

View File

@ -41,9 +41,7 @@ import org.elasticsearch.common.xcontent.XContentParser;
import org.elasticsearch.index.Index;
import org.elasticsearch.index.shard.ShardId;
import org.elasticsearch.test.AbstractSerializingTestCase;
import org.elasticsearch.xpack.core.indexlifecycle.LifecycleAction;
import org.elasticsearch.xpack.core.indexlifecycle.LifecycleSettings;
import org.elasticsearch.xpack.core.indexlifecycle.ShrinkAction;
import org.mockito.Mockito;
import java.io.IOException;
@ -79,480 +77,480 @@ public class ShrinkActionTests extends AbstractSerializingTestCase<ShrinkAction>
assertThat(e.getMessage(), equalTo("[number_of_shards] must be greater than 0"));
}
public void testExecuteSuccessfullyCompleted() {
String originalIndexName = randomAlphaOfLengthBetween(1, 20);
Index index = new Index("shrunk-" + originalIndexName, randomAlphaOfLengthBetween(1, 20));
ClusterService clusterService = Mockito.mock(ClusterService.class);
IndexMetaData originalIndexMetaData = IndexMetaData.builder(originalIndexName)
.settings(settings(Version.CURRENT)).numberOfReplicas(0).numberOfShards(1).build();
IndexMetaData indexMetadata = IndexMetaData.builder(index.getName())
.settings(settings(Version.CURRENT).put(IndexMetaData.INDEX_SHRINK_SOURCE_NAME_KEY, originalIndexName))
.numberOfShards(randomIntBetween(1, 5)).numberOfReplicas(randomIntBetween(0, 5)).build();
ImmutableOpenMap.Builder<String, IndexMetaData> indices = ImmutableOpenMap.<String, IndexMetaData> builder().fPut(index.getName(),
indexMetadata).fPut(originalIndexName, originalIndexMetaData);
ClusterState clusterState = ClusterState.builder(ClusterState.EMPTY_STATE).metaData(MetaData.builder().indices(indices.build()))
.build();
Mockito.when(clusterService.state()).thenReturn(clusterState);
Client client = Mockito.mock(Client.class);
AdminClient adminClient = Mockito.mock(AdminClient.class);
IndicesAdminClient indicesClient = Mockito.mock(IndicesAdminClient.class);
Mockito.when(client.admin()).thenReturn(adminClient);
Mockito.when(adminClient.indices()).thenReturn(indicesClient);
Mockito.doAnswer(invocationOnMock -> {
IndicesAliasesRequest request = (IndicesAliasesRequest) invocationOnMock.getArguments()[0];
@SuppressWarnings("unchecked")
ActionListener<IndicesAliasesResponse> listener = (ActionListener<IndicesAliasesResponse>) invocationOnMock.getArguments()[1];
IndicesAliasesResponse response = IndicesAliasesAction.INSTANCE.newResponse();
response.readFrom(StreamInput.wrap(new byte[] { 1 }));
assertThat(request.getAliasActions().size(), equalTo(2));
assertThat(request.getAliasActions().get(0).actionType(), equalTo(IndicesAliasesRequest.AliasActions.Type.REMOVE_INDEX));
assertThat(request.getAliasActions().get(0).indices(), equalTo(new String[] { originalIndexName }));
assertThat(request.getAliasActions().get(1).actionType(), equalTo(IndicesAliasesRequest.AliasActions.Type.ADD));
assertThat(request.getAliasActions().get(1).indices(), equalTo(new String[] { index.getName() }));
assertThat(request.getAliasActions().get(1).aliases(), equalTo(new String[] { originalIndexName }));
listener.onResponse(response);
return null;
}).when(indicesClient).aliases(any(), any());
SetOnce<Boolean> actionCompleted = new SetOnce<>();
ShrinkAction action = new ShrinkAction(randomIntBetween(1, 10));
action.execute(index, client, clusterService, new LifecycleAction.Listener() {
@Override
public void onSuccess(boolean completed) {
actionCompleted.set(completed);
}
@Override
public void onFailure(Exception e) {
throw new AssertionError("Unexpected method call", e);
}
});
assertTrue(actionCompleted.get());
Mockito.verify(client, Mockito.only()).admin();
Mockito.verify(adminClient, Mockito.only()).indices();
Mockito.verify(indicesClient, Mockito.only()).aliases(any(), any());
}
public void testExecuteAlreadyCompletedAndRunAgain() {
String originalIndexName = randomAlphaOfLengthBetween(1, 20);
Index index = new Index("shrunk-" + originalIndexName, randomAlphaOfLengthBetween(1, 20));
ClusterService clusterService = Mockito.mock(ClusterService.class);
IndexMetaData indexMetadata = IndexMetaData.builder(index.getName())
.putAlias(AliasMetaData.builder(originalIndexName).build())
.settings(settings(Version.CURRENT).put(IndexMetaData.INDEX_SHRINK_SOURCE_NAME_KEY, originalIndexName))
.numberOfShards(randomIntBetween(1, 5)).numberOfReplicas(randomIntBetween(0, 5)).build();
ImmutableOpenMap.Builder<String, IndexMetaData> indices = ImmutableOpenMap.<String, IndexMetaData> builder().fPut(index.getName(),
indexMetadata);
ClusterState clusterState = ClusterState.builder(ClusterState.EMPTY_STATE).metaData(MetaData.builder().indices(indices.build()))
.build();
Mockito.when(clusterService.state()).thenReturn(clusterState);
Client client = Mockito.mock(Client.class);
SetOnce<Boolean> actionCompleted = new SetOnce<>();
ShrinkAction action = new ShrinkAction(randomIntBetween(1, 10));
action.execute(index, client, clusterService, new LifecycleAction.Listener() {
@Override
public void onSuccess(boolean completed) {
actionCompleted.set(completed);
}
@Override
public void onFailure(Exception e) {
throw new AssertionError("Unexpected method call", e);
}
});
assertTrue(actionCompleted.get());
Mockito.verify(client, Mockito.never()).admin();
}
public void testExecuteOriginalIndexAliasFailure() {
String originalIndexName = randomAlphaOfLengthBetween(1, 20);
Index index = new Index("shrunk-" + originalIndexName, randomAlphaOfLengthBetween(1, 20));
Index targetIndex = new Index(randomAlphaOfLengthBetween(1, 20), randomAlphaOfLengthBetween(1, 20));
ClusterService clusterService = Mockito.mock(ClusterService.class);
IndexMetaData indexMetadata = IndexMetaData.builder(index.getName())
.settings(settings(Version.CURRENT).put(IndexMetaData.INDEX_SHRINK_SOURCE_NAME_KEY, originalIndexName))
.numberOfShards(randomIntBetween(1, 5)).numberOfReplicas(randomIntBetween(0, 5)).build();
ImmutableOpenMap.Builder<String, IndexMetaData> indices = ImmutableOpenMap.<String, IndexMetaData> builder().fPut(index.getName(),
indexMetadata);
ClusterState clusterState = ClusterState.builder(ClusterState.EMPTY_STATE).metaData(MetaData.builder().indices(indices.build()))
.routingTable(RoutingTable.builder()
.add(IndexRoutingTable.builder(targetIndex).addShard(
TestShardRouting.newShardRouting(new ShardId(index, 0), "node1", true,
ShardRoutingState.STARTED)))
.build())
.build();
Mockito.when(clusterService.state()).thenReturn(clusterState);
Client client = Mockito.mock(Client.class);
AdminClient adminClient = Mockito.mock(AdminClient.class);
IndicesAdminClient indicesClient = Mockito.mock(IndicesAdminClient.class);
Mockito.when(client.admin()).thenReturn(adminClient);
Mockito.when(adminClient.indices()).thenReturn(indicesClient);
Mockito.doAnswer(invocationOnMock -> {
@SuppressWarnings("unchecked")
ActionListener<IndicesAliasesResponse> listener = (ActionListener<IndicesAliasesResponse>) invocationOnMock.getArguments()[1];
listener.onFailure(new RuntimeException("failed"));
return null;
}).when(indicesClient).aliases(any(), any());
SetOnce<Exception> onFailureException = new SetOnce<>();
ShrinkAction action = new ShrinkAction(randomIntBetween(1, 10));
action.execute(index, client, clusterService, new LifecycleAction.Listener() {
@Override
public void onSuccess(boolean completed) {
throw new AssertionError("Unexpected method call");
}
@Override
public void onFailure(Exception e) {
onFailureException.set(e);
}
});
assertThat(onFailureException.get().getMessage(), equalTo("failed"));
Mockito.verify(client, Mockito.only()).admin();
Mockito.verify(adminClient, Mockito.only()).indices();
Mockito.verify(indicesClient, Mockito.only()).aliases(any(), any());
}
public void testExecuteWithIssuedResizeRequest() {
Index index = new Index(randomAlphaOfLengthBetween(1, 20), randomAlphaOfLengthBetween(1, 20));
Index targetIndex = new Index("shrunk-" + index.getName(), randomAlphaOfLengthBetween(1, 20));
int numberOfShards = randomIntBetween(1, 5);
int numberOfReplicas = randomIntBetween(1, 5);
long creationDate = randomNonNegativeLong();
ClusterService clusterService = Mockito.mock(ClusterService.class);
IndexMetaData indexMetadata = IndexMetaData.builder(index.getName())
.settings(settings(Version.CURRENT))
.putAlias(AliasMetaData.builder("my_alias"))
.creationDate(creationDate)
.numberOfShards(randomIntBetween(1, 5)).numberOfReplicas(numberOfReplicas).build();
ImmutableOpenMap.Builder<String, IndexMetaData> indices = ImmutableOpenMap.<String, IndexMetaData> builder().fPut(index.getName(),
indexMetadata);
ClusterState clusterState = ClusterState.builder(ClusterState.EMPTY_STATE).metaData(MetaData.builder().indices(indices.build()))
.routingTable(RoutingTable.builder()
.add(IndexRoutingTable.builder(targetIndex).addShard(
TestShardRouting.newShardRouting(new ShardId(index, 0), "node1", true,
ShardRoutingState.STARTED)))
.build())
.build();
Mockito.when(clusterService.state()).thenReturn(clusterState);
Client client = Mockito.mock(Client.class);
AdminClient adminClient = Mockito.mock(AdminClient.class);
IndicesAdminClient indicesClient = Mockito.mock(IndicesAdminClient.class);
Mockito.when(client.admin()).thenReturn(adminClient);
Mockito.when(adminClient.indices()).thenReturn(indicesClient);
Settings expectedSettings = Settings.builder().put(IndexMetaData.SETTING_BLOCKS_WRITE, true).build();
Mockito.doAnswer(invocationOnMock -> {
UpdateSettingsRequest request = (UpdateSettingsRequest) invocationOnMock.getArguments()[0];
@SuppressWarnings("unchecked")
ActionListener<UpdateSettingsResponse> listener = (ActionListener<UpdateSettingsResponse>) invocationOnMock.getArguments()[1];
UpdateSettingsTestHelper.assertSettingsRequest(request, expectedSettings, index.getName());
listener.onResponse(UpdateSettingsTestHelper.createMockResponse(true));
return null;
}).when(indicesClient).updateSettings(any(), any());
Mockito.doAnswer(invocationOnMock -> {
ResizeRequest request = (ResizeRequest) invocationOnMock.getArguments()[0];
@SuppressWarnings("unchecked")
ActionListener<AcknowledgedResponse> listener = (ActionListener<AcknowledgedResponse>) invocationOnMock.getArguments()[1];
assertThat(request.getSourceIndex(), equalTo(index.getName()));
assertThat(request.getTargetIndexRequest().aliases(), equalTo(Collections.singleton(new Alias("my_alias"))));
assertThat(request.getTargetIndexRequest().settings(), equalTo(Settings.builder()
.put(IndexMetaData.SETTING_NUMBER_OF_SHARDS, numberOfShards)
.put(IndexMetaData.SETTING_NUMBER_OF_REPLICAS, numberOfReplicas)
.put(LifecycleSettings.LIFECYCLE_INDEX_CREATION_DATE, creationDate).build()));
assertThat(request.getTargetIndexRequest().index(), equalTo(targetIndex.getName()));
ResizeResponse resizeResponse = ResizeAction.INSTANCE.newResponse();
resizeResponse.readFrom(StreamInput.wrap(new byte[] { 1, 1, 1, 1, 1 }));
listener.onResponse(resizeResponse);
return null;
}).when(indicesClient).resizeIndex(any(), any());
SetOnce<Boolean> actionCompleted = new SetOnce<>();
ShrinkAction action = new ShrinkAction(numberOfShards);
action.execute(index, client, clusterService, new LifecycleAction.Listener() {
@Override
public void onSuccess(boolean completed) {
actionCompleted.set(completed);
}
@Override
public void onFailure(Exception e) {
throw new AssertionError("Unexpected method call", e);
}
});
assertFalse(actionCompleted.get());
Mockito.verify(client, Mockito.atLeast(1)).admin();
Mockito.verify(adminClient, Mockito.atLeast(1)).indices();
Mockito.verify(indicesClient, Mockito.atLeast(1)).updateSettings(any(), any());
Mockito.verify(indicesClient, Mockito.atLeast(1)).resizeIndex(any(), any());
}
public void testExecuteWithIssuedResizeRequestFailure() {
Index index = new Index(randomAlphaOfLengthBetween(1, 20), randomAlphaOfLengthBetween(1, 20));
Index targetIndex = new Index(randomAlphaOfLengthBetween(1, 20), randomAlphaOfLengthBetween(1, 20));
ClusterService clusterService = Mockito.mock(ClusterService.class);
IndexMetaData indexMetadata = IndexMetaData.builder(index.getName())
.settings(settings(Version.CURRENT))
.numberOfShards(randomIntBetween(1, 5)).numberOfReplicas(randomIntBetween(0, 5)).build();
ImmutableOpenMap.Builder<String, IndexMetaData> indices = ImmutableOpenMap.<String, IndexMetaData> builder().fPut(index.getName(),
indexMetadata);
ClusterState clusterState = ClusterState.builder(ClusterState.EMPTY_STATE).metaData(MetaData.builder().indices(indices.build()))
.routingTable(RoutingTable.builder()
.add(IndexRoutingTable.builder(targetIndex).addShard(
TestShardRouting.newShardRouting(new ShardId(index, 0), "node1", true,
ShardRoutingState.STARTED)))
.build())
.build();
Mockito.when(clusterService.state()).thenReturn(clusterState);
Client client = Mockito.mock(Client.class);
AdminClient adminClient = Mockito.mock(AdminClient.class);
IndicesAdminClient indicesClient = Mockito.mock(IndicesAdminClient.class);
Mockito.when(client.admin()).thenReturn(adminClient);
Mockito.when(adminClient.indices()).thenReturn(indicesClient);
Mockito.doAnswer(invocationOnMock -> {
@SuppressWarnings("unchecked")
ActionListener<UpdateSettingsResponse> listener = (ActionListener<UpdateSettingsResponse>) invocationOnMock.getArguments()[1];
listener.onResponse(UpdateSettingsTestHelper.createMockResponse(true));
return null;
}).when(indicesClient).updateSettings(any(), any());
Mockito.doAnswer(invocationOnMock -> {
@SuppressWarnings("unchecked")
ActionListener<AcknowledgedResponse> listener = (ActionListener<AcknowledgedResponse>) invocationOnMock.getArguments()[1];
ResizeResponse resizeResponse = ResizeAction.INSTANCE.newResponse();
resizeResponse.readFrom(StreamInput.wrap(new byte[] { 0, 1, 1, 1, 1 }));
listener.onResponse(resizeResponse);
return null;
}).when(indicesClient).resizeIndex(any(), any());
SetOnce<Exception> exceptionReturned = new SetOnce<>();
ShrinkAction action = new ShrinkAction(randomIntBetween(1, 10));
action.execute(index, client, clusterService, new LifecycleAction.Listener() {
@Override
public void onSuccess(boolean completed) {
throw new AssertionError("Unexpected method call to onSuccess");
}
@Override
public void onFailure(Exception e) {
exceptionReturned.set(e);
}
});
assertThat(exceptionReturned.get().getMessage(), equalTo("Shrink request failed to be acknowledged"));
Mockito.verify(client, Mockito.atLeast(1)).admin();
Mockito.verify(adminClient, Mockito.atLeast(1)).indices();
Mockito.verify(indicesClient, Mockito.atLeast(1)).updateSettings(any(), any());
Mockito.verify(indicesClient, Mockito.atLeast(1)).resizeIndex(any(), any());
}
public void testExecuteWithAllShardsAllocatedAndShrunkenIndexSetting() {
Index index = new Index(randomAlphaOfLengthBetween(1, 20), randomAlphaOfLengthBetween(1, 20));
Index targetIndex = new Index("shrunk-" + index.getName(), randomAlphaOfLengthBetween(1, 20));
ClusterService clusterService = Mockito.mock(ClusterService.class);
IndexMetaData indexMetadata = IndexMetaData.builder(index.getName())
.settings(settings(Version.CURRENT).put(LifecycleSettings.LIFECYCLE_PHASE, "phase1")
.put(LifecycleSettings.LIFECYCLE_ACTION, "action1").put(LifecycleSettings.LIFECYCLE_NAME, "test"))
.numberOfShards(randomIntBetween(1, 5)).numberOfReplicas(randomIntBetween(0, 5)).build();
IndexMetaData targetIndexMetaData = IndexMetaData.builder(targetIndex.getName())
.settings(settings(Version.CURRENT).put(IndexMetaData.INDEX_SHRINK_SOURCE_NAME_KEY, index.getName()))
.numberOfShards(randomIntBetween(1, 3)).numberOfReplicas(indexMetadata.getNumberOfReplicas()).build();
ImmutableOpenMap.Builder<String, IndexMetaData> indices = ImmutableOpenMap.<String, IndexMetaData> builder()
.fPut(index.getName(), indexMetadata).fPut(targetIndex.getName(), targetIndexMetaData);
ClusterState clusterState = ClusterState.builder(ClusterState.EMPTY_STATE).metaData(MetaData.builder().indices(indices.build()))
.routingTable(RoutingTable.builder()
.add(IndexRoutingTable.builder(targetIndex).addShard(
TestShardRouting.newShardRouting(new ShardId(index, 0), "node1", true,
ShardRoutingState.STARTED)))
.build())
.build();
Mockito.when(clusterService.state()).thenReturn(clusterState);
Client client = Mockito.mock(Client.class);
AdminClient adminClient = Mockito.mock(AdminClient.class);
IndicesAdminClient indicesClient = Mockito.mock(IndicesAdminClient.class);
Mockito.when(client.admin()).thenReturn(adminClient);
Mockito.when(adminClient.indices()).thenReturn(indicesClient);
Settings expectedSettings = Settings.builder().put(LifecycleSettings.LIFECYCLE_NAME, "test")
.put(LifecycleSettings.LIFECYCLE_PHASE, "phase1").put(LifecycleSettings.LIFECYCLE_ACTION, "action1").build();
Mockito.doAnswer(invocationOnMock -> {
UpdateSettingsRequest request = (UpdateSettingsRequest) invocationOnMock.getArguments()[0];
@SuppressWarnings("unchecked")
ActionListener<UpdateSettingsResponse> listener = (ActionListener<UpdateSettingsResponse>) invocationOnMock.getArguments()[1];
UpdateSettingsTestHelper.assertSettingsRequest(request, expectedSettings, targetIndex.getName());
listener.onResponse(UpdateSettingsTestHelper.createMockResponse(true));
return null;
}).when(indicesClient).updateSettings(any(), any());
Mockito.doAnswer(invocationOnMock -> {
DeleteIndexRequest request = (DeleteIndexRequest) invocationOnMock.getArguments()[0];
@SuppressWarnings("unchecked")
ActionListener<DeleteIndexResponse> listener = (ActionListener<DeleteIndexResponse>) invocationOnMock.getArguments()[1];
assertNotNull(request);
assertEquals(1, request.indices().length);
assertEquals(index.getName(), request.indices()[0]);
listener.onResponse(null);
return null;
}).when(indicesClient).delete(any(), any());
SetOnce<Boolean> actionCompleted = new SetOnce<>();
ShrinkAction action = new ShrinkAction(randomIntBetween(1, 10));
action.execute(index, client, clusterService, new LifecycleAction.Listener() {
@Override
public void onSuccess(boolean completed) {
actionCompleted.set(completed);
}
@Override
public void onFailure(Exception e) {
throw new AssertionError("Unexpected method call", e);
}
});
assertFalse(actionCompleted.get());
Mockito.verify(client, Mockito.atLeast(1)).admin();
Mockito.verify(adminClient, Mockito.atLeast(1)).indices();
Mockito.verify(indicesClient, Mockito.atLeast(1)).updateSettings(any(), any());
}
public void testExecuteWithAllShardsAllocatedAndShrunkenIndexConfigured() {
String lifecycleName = randomAlphaOfLengthBetween(5, 10);
Index index = new Index(randomAlphaOfLengthBetween(1, 20), randomAlphaOfLengthBetween(1, 20));
Index targetIndex = new Index("shrunk-" + index.getName(), randomAlphaOfLengthBetween(1, 20));
ClusterService clusterService = Mockito.mock(ClusterService.class);
IndexMetaData indexMetadata = IndexMetaData.builder(index.getName())
.settings(settings(Version.CURRENT).put(LifecycleSettings.LIFECYCLE_PHASE, "phase1")
.put(LifecycleSettings.LIFECYCLE_ACTION, "action1").put(LifecycleSettings.LIFECYCLE_NAME, lifecycleName))
.numberOfShards(randomIntBetween(1, 5)).numberOfReplicas(randomIntBetween(0, 5)).build();
IndexMetaData targetIndexMetaData = IndexMetaData.builder(targetIndex.getName())
.settings(settings(Version.CURRENT)
.put(IndexMetaData.INDEX_SHRINK_SOURCE_NAME_KEY, index.getName())
.put(LifecycleSettings.LIFECYCLE_NAME_SETTING.getKey(), lifecycleName))
.numberOfShards(randomIntBetween(1, 3)).numberOfReplicas(indexMetadata.getNumberOfReplicas()).build();
ImmutableOpenMap.Builder<String, IndexMetaData> indices = ImmutableOpenMap.<String, IndexMetaData> builder()
.fPut(index.getName(), indexMetadata).fPut(targetIndex.getName(), targetIndexMetaData);
ClusterState clusterState = ClusterState.builder(ClusterState.EMPTY_STATE).metaData(MetaData.builder().indices(indices.build()))
.routingTable(RoutingTable.builder()
.add(IndexRoutingTable.builder(targetIndex).addShard(
TestShardRouting.newShardRouting(new ShardId(index, 0), "node1", true,
ShardRoutingState.STARTED)))
.build())
.build();
Mockito.when(clusterService.state()).thenReturn(clusterState);
Client client = Mockito.mock(Client.class);
SetOnce<Boolean> actionCompleted = new SetOnce<>();
ShrinkAction action = new ShrinkAction(randomIntBetween(1, 10));
action.execute(index, client, clusterService, new LifecycleAction.Listener() {
@Override
public void onSuccess(boolean completed) {
actionCompleted.set(completed);
}
@Override
public void onFailure(Exception e) {
throw new AssertionError("Unexpected method call", e);
}
});
assertFalse(actionCompleted.get());
Mockito.verifyZeroInteractions(client);
}
public void testExecuteWaitingOnAllShardsActive() {
Index index = new Index(randomAlphaOfLengthBetween(1, 20), randomAlphaOfLengthBetween(1, 20));
Index targetIndex = new Index("shrunk-" + index.getName(), randomAlphaOfLengthBetween(1, 20));
ClusterService clusterService = Mockito.mock(ClusterService.class);
IndexMetaData indexMetadata = IndexMetaData.builder(index.getName())
.settings(settings(Version.CURRENT))
.numberOfShards(randomIntBetween(1, 5)).numberOfReplicas(randomIntBetween(0, 5)).build();
IndexMetaData targetIndexMetadata = IndexMetaData.builder(targetIndex.getName())
.settings(settings(Version.CURRENT).put(IndexMetaData.INDEX_SHRINK_SOURCE_NAME_KEY, index.getName()))
.numberOfShards(1).numberOfReplicas(randomIntBetween(0, 5)).build();
ImmutableOpenMap.Builder<String, IndexMetaData> indices = ImmutableOpenMap.<String, IndexMetaData> builder().fPut(index.getName(),
indexMetadata).fPut(targetIndex.getName(), targetIndexMetadata);
ClusterState clusterState = ClusterState.builder(ClusterState.EMPTY_STATE).metaData(MetaData.builder().indices(indices.build()))
.routingTable(RoutingTable.builder()
.add(IndexRoutingTable.builder(targetIndex).addShard(
TestShardRouting.newShardRouting(new ShardId(index, 0), "node1", true,
ShardRoutingState.INITIALIZING)))
.build())
.build();
Mockito.when(clusterService.state()).thenReturn(clusterState);
Client client = Mockito.mock(Client.class);
SetOnce<Boolean> actionCompleted = new SetOnce<>();
ShrinkAction action = new ShrinkAction(randomIntBetween(1, 10));
action.execute(index, client, clusterService, new LifecycleAction.Listener() {
@Override
public void onSuccess(boolean completed) {
actionCompleted.set(completed);
}
@Override
public void onFailure(Exception e) {
throw new AssertionError("Unexpected method call", e);
}
});
assertFalse(actionCompleted.get());
}
public void testExecuteIndexAlreadyExists() {
Index index = new Index(randomAlphaOfLengthBetween(1, 20), randomAlphaOfLengthBetween(1, 20));
Index targetIndex = new Index("shrunk-" + index.getName(), randomAlphaOfLengthBetween(1, 20));
ClusterService clusterService = Mockito.mock(ClusterService.class);
IndexMetaData indexMetadata = IndexMetaData.builder(index.getName())
.settings(settings(Version.CURRENT))
.numberOfShards(randomIntBetween(1, 5)).numberOfReplicas(randomIntBetween(0, 5)).build();
IndexMetaData targetIndexMetadata = IndexMetaData.builder(targetIndex.getName())
.settings(settings(Version.CURRENT))
.numberOfShards(1).numberOfReplicas(randomIntBetween(0, 5)).build();
ImmutableOpenMap.Builder<String, IndexMetaData> indices = ImmutableOpenMap.<String, IndexMetaData> builder().fPut(index.getName(),
indexMetadata).fPut(targetIndex.getName(), targetIndexMetadata);
ClusterState clusterState = ClusterState.builder(ClusterState.EMPTY_STATE)
.metaData(MetaData.builder().indices(indices.build())).build();
Mockito.when(clusterService.state()).thenReturn(clusterState);
Client client = Mockito.mock(Client.class);
SetOnce<Exception> actionFailed = new SetOnce<>();
ShrinkAction action = new ShrinkAction(randomIntBetween(1, 10));
action.execute(index, client, clusterService, new LifecycleAction.Listener() {
@Override
public void onSuccess(boolean completed) {
throw new AssertionError("Unexpected method call to onSuccess");
}
@Override
public void onFailure(Exception e) {
actionFailed.set(e);
}
});
assertThat(actionFailed.get().getMessage(), equalTo("Cannot shrink index [" + index.getName() + "]" +
" because target index [" + targetIndex.getName() + "] already exists."));
}
// public void testExecuteSuccessfullyCompleted() {
// String originalIndexName = randomAlphaOfLengthBetween(1, 20);
// Index index = new Index("shrunk-" + originalIndexName, randomAlphaOfLengthBetween(1, 20));
// ClusterService clusterService = Mockito.mock(ClusterService.class);
// IndexMetaData originalIndexMetaData = IndexMetaData.builder(originalIndexName)
// .settings(settings(Version.CURRENT)).numberOfReplicas(0).numberOfShards(1).build();
// IndexMetaData indexMetadata = IndexMetaData.builder(index.getName())
// .settings(settings(Version.CURRENT).put(IndexMetaData.INDEX_SHRINK_SOURCE_NAME_KEY, originalIndexName))
// .numberOfShards(randomIntBetween(1, 5)).numberOfReplicas(randomIntBetween(0, 5)).build();
// ImmutableOpenMap.Builder<String, IndexMetaData> indices = ImmutableOpenMap.<String, IndexMetaData> builder().fPut(index.getName(),
// indexMetadata).fPut(originalIndexName, originalIndexMetaData);
// ClusterState clusterState = ClusterState.builder(ClusterState.EMPTY_STATE).metaData(MetaData.builder().indices(indices.build()))
// .build();
// Mockito.when(clusterService.state()).thenReturn(clusterState);
// Client client = Mockito.mock(Client.class);
// AdminClient adminClient = Mockito.mock(AdminClient.class);
// IndicesAdminClient indicesClient = Mockito.mock(IndicesAdminClient.class);
// Mockito.when(client.admin()).thenReturn(adminClient);
// Mockito.when(adminClient.indices()).thenReturn(indicesClient);
//
// Mockito.doAnswer(invocationOnMock -> {
// IndicesAliasesRequest request = (IndicesAliasesRequest) invocationOnMock.getArguments()[0];
// @SuppressWarnings("unchecked")
// ActionListener<IndicesAliasesResponse> listener = (ActionListener<IndicesAliasesResponse>) invocationOnMock.getArguments()[1];
// IndicesAliasesResponse response = IndicesAliasesAction.INSTANCE.newResponse();
// response.readFrom(StreamInput.wrap(new byte[] { 1 }));
//
// assertThat(request.getAliasActions().size(), equalTo(2));
// assertThat(request.getAliasActions().get(0).actionType(), equalTo(IndicesAliasesRequest.AliasActions.Type.REMOVE_INDEX));
// assertThat(request.getAliasActions().get(0).indices(), equalTo(new String[] { originalIndexName }));
// assertThat(request.getAliasActions().get(1).actionType(), equalTo(IndicesAliasesRequest.AliasActions.Type.ADD));
// assertThat(request.getAliasActions().get(1).indices(), equalTo(new String[] { index.getName() }));
// assertThat(request.getAliasActions().get(1).aliases(), equalTo(new String[] { originalIndexName }));
//
// listener.onResponse(response);
// return null;
// }).when(indicesClient).aliases(any(), any());
//
// SetOnce<Boolean> actionCompleted = new SetOnce<>();
// ShrinkAction action = new ShrinkAction(randomIntBetween(1, 10));
//
// action.execute(index, client, clusterService, new LifecycleAction.Listener() {
// @Override
// public void onSuccess(boolean completed) {
// actionCompleted.set(completed);
// }
//
// @Override
// public void onFailure(Exception e) {
// throw new AssertionError("Unexpected method call", e);
// }
// });
//
// assertTrue(actionCompleted.get());
// Mockito.verify(client, Mockito.only()).admin();
// Mockito.verify(adminClient, Mockito.only()).indices();
// Mockito.verify(indicesClient, Mockito.only()).aliases(any(), any());
// }
//
// public void testExecuteAlreadyCompletedAndRunAgain() {
// String originalIndexName = randomAlphaOfLengthBetween(1, 20);
// Index index = new Index("shrunk-" + originalIndexName, randomAlphaOfLengthBetween(1, 20));
// ClusterService clusterService = Mockito.mock(ClusterService.class);
// IndexMetaData indexMetadata = IndexMetaData.builder(index.getName())
// .putAlias(AliasMetaData.builder(originalIndexName).build())
// .settings(settings(Version.CURRENT).put(IndexMetaData.INDEX_SHRINK_SOURCE_NAME_KEY, originalIndexName))
// .numberOfShards(randomIntBetween(1, 5)).numberOfReplicas(randomIntBetween(0, 5)).build();
// ImmutableOpenMap.Builder<String, IndexMetaData> indices = ImmutableOpenMap.<String, IndexMetaData> builder().fPut(index.getName(),
// indexMetadata);
// ClusterState clusterState = ClusterState.builder(ClusterState.EMPTY_STATE).metaData(MetaData.builder().indices(indices.build()))
// .build();
// Mockito.when(clusterService.state()).thenReturn(clusterState);
// Client client = Mockito.mock(Client.class);
// SetOnce<Boolean> actionCompleted = new SetOnce<>();
// ShrinkAction action = new ShrinkAction(randomIntBetween(1, 10));
//
// action.execute(index, client, clusterService, new LifecycleAction.Listener() {
// @Override
// public void onSuccess(boolean completed) {
// actionCompleted.set(completed);
// }
//
// @Override
// public void onFailure(Exception e) {
// throw new AssertionError("Unexpected method call", e);
// }
// });
//
// assertTrue(actionCompleted.get());
// Mockito.verify(client, Mockito.never()).admin();
// }
//
// public void testExecuteOriginalIndexAliasFailure() {
// String originalIndexName = randomAlphaOfLengthBetween(1, 20);
// Index index = new Index("shrunk-" + originalIndexName, randomAlphaOfLengthBetween(1, 20));
// Index targetIndex = new Index(randomAlphaOfLengthBetween(1, 20), randomAlphaOfLengthBetween(1, 20));
// ClusterService clusterService = Mockito.mock(ClusterService.class);
// IndexMetaData indexMetadata = IndexMetaData.builder(index.getName())
// .settings(settings(Version.CURRENT).put(IndexMetaData.INDEX_SHRINK_SOURCE_NAME_KEY, originalIndexName))
// .numberOfShards(randomIntBetween(1, 5)).numberOfReplicas(randomIntBetween(0, 5)).build();
// ImmutableOpenMap.Builder<String, IndexMetaData> indices = ImmutableOpenMap.<String, IndexMetaData> builder().fPut(index.getName(),
// indexMetadata);
// ClusterState clusterState = ClusterState.builder(ClusterState.EMPTY_STATE).metaData(MetaData.builder().indices(indices.build()))
// .routingTable(RoutingTable.builder()
// .add(IndexRoutingTable.builder(targetIndex).addShard(
// TestShardRouting.newShardRouting(new ShardId(index, 0), "node1", true,
// ShardRoutingState.STARTED)))
// .build())
// .build();
// Mockito.when(clusterService.state()).thenReturn(clusterState);
// Client client = Mockito.mock(Client.class);
// AdminClient adminClient = Mockito.mock(AdminClient.class);
// IndicesAdminClient indicesClient = Mockito.mock(IndicesAdminClient.class);
// Mockito.when(client.admin()).thenReturn(adminClient);
// Mockito.when(adminClient.indices()).thenReturn(indicesClient);
//
// Mockito.doAnswer(invocationOnMock -> {
// @SuppressWarnings("unchecked")
// ActionListener<IndicesAliasesResponse> listener = (ActionListener<IndicesAliasesResponse>) invocationOnMock.getArguments()[1];
// listener.onFailure(new RuntimeException("failed"));
// return null;
// }).when(indicesClient).aliases(any(), any());
//
// SetOnce<Exception> onFailureException = new SetOnce<>();
// ShrinkAction action = new ShrinkAction(randomIntBetween(1, 10));
//
// action.execute(index, client, clusterService, new LifecycleAction.Listener() {
// @Override
// public void onSuccess(boolean completed) {
// throw new AssertionError("Unexpected method call");
// }
//
// @Override
// public void onFailure(Exception e) {
// onFailureException.set(e);
// }
// });
//
// assertThat(onFailureException.get().getMessage(), equalTo("failed"));
//
// Mockito.verify(client, Mockito.only()).admin();
// Mockito.verify(adminClient, Mockito.only()).indices();
// Mockito.verify(indicesClient, Mockito.only()).aliases(any(), any());
// }
//
// public void testExecuteWithIssuedResizeRequest() {
// Index index = new Index(randomAlphaOfLengthBetween(1, 20), randomAlphaOfLengthBetween(1, 20));
// Index targetIndex = new Index("shrunk-" + index.getName(), randomAlphaOfLengthBetween(1, 20));
// int numberOfShards = randomIntBetween(1, 5);
// int numberOfReplicas = randomIntBetween(1, 5);
// long creationDate = randomNonNegativeLong();
// ClusterService clusterService = Mockito.mock(ClusterService.class);
// IndexMetaData indexMetadata = IndexMetaData.builder(index.getName())
// .settings(settings(Version.CURRENT))
// .putAlias(AliasMetaData.builder("my_alias"))
// .creationDate(creationDate)
// .numberOfShards(randomIntBetween(1, 5)).numberOfReplicas(numberOfReplicas).build();
// ImmutableOpenMap.Builder<String, IndexMetaData> indices = ImmutableOpenMap.<String, IndexMetaData> builder().fPut(index.getName(),
// indexMetadata);
// ClusterState clusterState = ClusterState.builder(ClusterState.EMPTY_STATE).metaData(MetaData.builder().indices(indices.build()))
// .routingTable(RoutingTable.builder()
// .add(IndexRoutingTable.builder(targetIndex).addShard(
// TestShardRouting.newShardRouting(new ShardId(index, 0), "node1", true,
// ShardRoutingState.STARTED)))
// .build())
// .build();
// Mockito.when(clusterService.state()).thenReturn(clusterState);
// Client client = Mockito.mock(Client.class);
// AdminClient adminClient = Mockito.mock(AdminClient.class);
// IndicesAdminClient indicesClient = Mockito.mock(IndicesAdminClient.class);
// Mockito.when(client.admin()).thenReturn(adminClient);
// Mockito.when(adminClient.indices()).thenReturn(indicesClient);
//
// Settings expectedSettings = Settings.builder().put(IndexMetaData.SETTING_BLOCKS_WRITE, true).build();
//
// Mockito.doAnswer(invocationOnMock -> {
// UpdateSettingsRequest request = (UpdateSettingsRequest) invocationOnMock.getArguments()[0];
// @SuppressWarnings("unchecked")
// ActionListener<UpdateSettingsResponse> listener = (ActionListener<UpdateSettingsResponse>) invocationOnMock.getArguments()[1];
// UpdateSettingsTestHelper.assertSettingsRequest(request, expectedSettings, index.getName());
// listener.onResponse(UpdateSettingsTestHelper.createMockResponse(true));
// return null;
// }).when(indicesClient).updateSettings(any(), any());
//
// Mockito.doAnswer(invocationOnMock -> {
// ResizeRequest request = (ResizeRequest) invocationOnMock.getArguments()[0];
// @SuppressWarnings("unchecked")
// ActionListener<AcknowledgedResponse> listener = (ActionListener<AcknowledgedResponse>) invocationOnMock.getArguments()[1];
// assertThat(request.getSourceIndex(), equalTo(index.getName()));
// assertThat(request.getTargetIndexRequest().aliases(), equalTo(Collections.singleton(new Alias("my_alias"))));
// assertThat(request.getTargetIndexRequest().settings(), equalTo(Settings.builder()
// .put(IndexMetaData.SETTING_NUMBER_OF_SHARDS, numberOfShards)
// .put(IndexMetaData.SETTING_NUMBER_OF_REPLICAS, numberOfReplicas)
// .put(LifecycleSettings.LIFECYCLE_INDEX_CREATION_DATE, creationDate).build()));
// assertThat(request.getTargetIndexRequest().index(), equalTo(targetIndex.getName()));
// ResizeResponse resizeResponse = ResizeAction.INSTANCE.newResponse();
// resizeResponse.readFrom(StreamInput.wrap(new byte[] { 1, 1, 1, 1, 1 }));
// listener.onResponse(resizeResponse);
// return null;
// }).when(indicesClient).resizeIndex(any(), any());
//
// SetOnce<Boolean> actionCompleted = new SetOnce<>();
// ShrinkAction action = new ShrinkAction(numberOfShards);
//
// action.execute(index, client, clusterService, new LifecycleAction.Listener() {
// @Override
// public void onSuccess(boolean completed) {
// actionCompleted.set(completed);
// }
//
// @Override
// public void onFailure(Exception e) {
// throw new AssertionError("Unexpected method call", e);
// }
// });
//
// assertFalse(actionCompleted.get());
//
// Mockito.verify(client, Mockito.atLeast(1)).admin();
// Mockito.verify(adminClient, Mockito.atLeast(1)).indices();
// Mockito.verify(indicesClient, Mockito.atLeast(1)).updateSettings(any(), any());
// Mockito.verify(indicesClient, Mockito.atLeast(1)).resizeIndex(any(), any());
// }
//
// public void testExecuteWithIssuedResizeRequestFailure() {
// Index index = new Index(randomAlphaOfLengthBetween(1, 20), randomAlphaOfLengthBetween(1, 20));
// Index targetIndex = new Index(randomAlphaOfLengthBetween(1, 20), randomAlphaOfLengthBetween(1, 20));
// ClusterService clusterService = Mockito.mock(ClusterService.class);
// IndexMetaData indexMetadata = IndexMetaData.builder(index.getName())
// .settings(settings(Version.CURRENT))
// .numberOfShards(randomIntBetween(1, 5)).numberOfReplicas(randomIntBetween(0, 5)).build();
// ImmutableOpenMap.Builder<String, IndexMetaData> indices = ImmutableOpenMap.<String, IndexMetaData> builder().fPut(index.getName(),
// indexMetadata);
// ClusterState clusterState = ClusterState.builder(ClusterState.EMPTY_STATE).metaData(MetaData.builder().indices(indices.build()))
// .routingTable(RoutingTable.builder()
// .add(IndexRoutingTable.builder(targetIndex).addShard(
// TestShardRouting.newShardRouting(new ShardId(index, 0), "node1", true,
// ShardRoutingState.STARTED)))
// .build())
// .build();
// Mockito.when(clusterService.state()).thenReturn(clusterState);
// Client client = Mockito.mock(Client.class);
// AdminClient adminClient = Mockito.mock(AdminClient.class);
// IndicesAdminClient indicesClient = Mockito.mock(IndicesAdminClient.class);
// Mockito.when(client.admin()).thenReturn(adminClient);
// Mockito.when(adminClient.indices()).thenReturn(indicesClient);
//
// Mockito.doAnswer(invocationOnMock -> {
// @SuppressWarnings("unchecked")
// ActionListener<UpdateSettingsResponse> listener = (ActionListener<UpdateSettingsResponse>) invocationOnMock.getArguments()[1];
// listener.onResponse(UpdateSettingsTestHelper.createMockResponse(true));
// return null;
// }).when(indicesClient).updateSettings(any(), any());
//
// Mockito.doAnswer(invocationOnMock -> {
// @SuppressWarnings("unchecked")
// ActionListener<AcknowledgedResponse> listener = (ActionListener<AcknowledgedResponse>) invocationOnMock.getArguments()[1];
// ResizeResponse resizeResponse = ResizeAction.INSTANCE.newResponse();
// resizeResponse.readFrom(StreamInput.wrap(new byte[] { 0, 1, 1, 1, 1 }));
// listener.onResponse(resizeResponse);
// return null;
// }).when(indicesClient).resizeIndex(any(), any());
//
// SetOnce<Exception> exceptionReturned = new SetOnce<>();
// ShrinkAction action = new ShrinkAction(randomIntBetween(1, 10));
//
// action.execute(index, client, clusterService, new LifecycleAction.Listener() {
// @Override
// public void onSuccess(boolean completed) {
// throw new AssertionError("Unexpected method call to onSuccess");
// }
//
// @Override
// public void onFailure(Exception e) {
// exceptionReturned.set(e);
// }
// });
//
// assertThat(exceptionReturned.get().getMessage(), equalTo("Shrink request failed to be acknowledged"));
//
// Mockito.verify(client, Mockito.atLeast(1)).admin();
// Mockito.verify(adminClient, Mockito.atLeast(1)).indices();
// Mockito.verify(indicesClient, Mockito.atLeast(1)).updateSettings(any(), any());
// Mockito.verify(indicesClient, Mockito.atLeast(1)).resizeIndex(any(), any());
// }
//
// public void testExecuteWithAllShardsAllocatedAndShrunkenIndexSetting() {
// Index index = new Index(randomAlphaOfLengthBetween(1, 20), randomAlphaOfLengthBetween(1, 20));
// Index targetIndex = new Index("shrunk-" + index.getName(), randomAlphaOfLengthBetween(1, 20));
// ClusterService clusterService = Mockito.mock(ClusterService.class);
// IndexMetaData indexMetadata = IndexMetaData.builder(index.getName())
// .settings(settings(Version.CURRENT).put(LifecycleSettings.LIFECYCLE_PHASE, "phase1")
// .put(LifecycleSettings.LIFECYCLE_ACTION, "action1").put(LifecycleSettings.LIFECYCLE_NAME, "test"))
// .numberOfShards(randomIntBetween(1, 5)).numberOfReplicas(randomIntBetween(0, 5)).build();
// IndexMetaData targetIndexMetaData = IndexMetaData.builder(targetIndex.getName())
// .settings(settings(Version.CURRENT).put(IndexMetaData.INDEX_SHRINK_SOURCE_NAME_KEY, index.getName()))
// .numberOfShards(randomIntBetween(1, 3)).numberOfReplicas(indexMetadata.getNumberOfReplicas()).build();
// ImmutableOpenMap.Builder<String, IndexMetaData> indices = ImmutableOpenMap.<String, IndexMetaData> builder()
// .fPut(index.getName(), indexMetadata).fPut(targetIndex.getName(), targetIndexMetaData);
// ClusterState clusterState = ClusterState.builder(ClusterState.EMPTY_STATE).metaData(MetaData.builder().indices(indices.build()))
// .routingTable(RoutingTable.builder()
// .add(IndexRoutingTable.builder(targetIndex).addShard(
// TestShardRouting.newShardRouting(new ShardId(index, 0), "node1", true,
// ShardRoutingState.STARTED)))
// .build())
// .build();
// Mockito.when(clusterService.state()).thenReturn(clusterState);
// Client client = Mockito.mock(Client.class);
// AdminClient adminClient = Mockito.mock(AdminClient.class);
// IndicesAdminClient indicesClient = Mockito.mock(IndicesAdminClient.class);
// Mockito.when(client.admin()).thenReturn(adminClient);
// Mockito.when(adminClient.indices()).thenReturn(indicesClient);
//
// Settings expectedSettings = Settings.builder().put(LifecycleSettings.LIFECYCLE_NAME, "test")
// .put(LifecycleSettings.LIFECYCLE_PHASE, "phase1").put(LifecycleSettings.LIFECYCLE_ACTION, "action1").build();
//
// Mockito.doAnswer(invocationOnMock -> {
// UpdateSettingsRequest request = (UpdateSettingsRequest) invocationOnMock.getArguments()[0];
// @SuppressWarnings("unchecked")
// ActionListener<UpdateSettingsResponse> listener = (ActionListener<UpdateSettingsResponse>) invocationOnMock.getArguments()[1];
// UpdateSettingsTestHelper.assertSettingsRequest(request, expectedSettings, targetIndex.getName());
// listener.onResponse(UpdateSettingsTestHelper.createMockResponse(true));
// return null;
// }).when(indicesClient).updateSettings(any(), any());
//
// Mockito.doAnswer(invocationOnMock -> {
// DeleteIndexRequest request = (DeleteIndexRequest) invocationOnMock.getArguments()[0];
// @SuppressWarnings("unchecked")
// ActionListener<DeleteIndexResponse> listener = (ActionListener<DeleteIndexResponse>) invocationOnMock.getArguments()[1];
// assertNotNull(request);
// assertEquals(1, request.indices().length);
// assertEquals(index.getName(), request.indices()[0]);
// listener.onResponse(null);
// return null;
// }).when(indicesClient).delete(any(), any());
//
// SetOnce<Boolean> actionCompleted = new SetOnce<>();
// ShrinkAction action = new ShrinkAction(randomIntBetween(1, 10));
//
// action.execute(index, client, clusterService, new LifecycleAction.Listener() {
// @Override
// public void onSuccess(boolean completed) {
// actionCompleted.set(completed);
// }
//
// @Override
// public void onFailure(Exception e) {
// throw new AssertionError("Unexpected method call", e);
// }
// });
//
// assertFalse(actionCompleted.get());
// Mockito.verify(client, Mockito.atLeast(1)).admin();
// Mockito.verify(adminClient, Mockito.atLeast(1)).indices();
// Mockito.verify(indicesClient, Mockito.atLeast(1)).updateSettings(any(), any());
// }
//
// public void testExecuteWithAllShardsAllocatedAndShrunkenIndexConfigured() {
// String lifecycleName = randomAlphaOfLengthBetween(5, 10);
// Index index = new Index(randomAlphaOfLengthBetween(1, 20), randomAlphaOfLengthBetween(1, 20));
// Index targetIndex = new Index("shrunk-" + index.getName(), randomAlphaOfLengthBetween(1, 20));
// ClusterService clusterService = Mockito.mock(ClusterService.class);
// IndexMetaData indexMetadata = IndexMetaData.builder(index.getName())
// .settings(settings(Version.CURRENT).put(LifecycleSettings.LIFECYCLE_PHASE, "phase1")
// .put(LifecycleSettings.LIFECYCLE_ACTION, "action1").put(LifecycleSettings.LIFECYCLE_NAME, lifecycleName))
// .numberOfShards(randomIntBetween(1, 5)).numberOfReplicas(randomIntBetween(0, 5)).build();
// IndexMetaData targetIndexMetaData = IndexMetaData.builder(targetIndex.getName())
// .settings(settings(Version.CURRENT)
// .put(IndexMetaData.INDEX_SHRINK_SOURCE_NAME_KEY, index.getName())
// .put(LifecycleSettings.LIFECYCLE_NAME_SETTING.getKey(), lifecycleName))
// .numberOfShards(randomIntBetween(1, 3)).numberOfReplicas(indexMetadata.getNumberOfReplicas()).build();
// ImmutableOpenMap.Builder<String, IndexMetaData> indices = ImmutableOpenMap.<String, IndexMetaData> builder()
// .fPut(index.getName(), indexMetadata).fPut(targetIndex.getName(), targetIndexMetaData);
// ClusterState clusterState = ClusterState.builder(ClusterState.EMPTY_STATE).metaData(MetaData.builder().indices(indices.build()))
// .routingTable(RoutingTable.builder()
// .add(IndexRoutingTable.builder(targetIndex).addShard(
// TestShardRouting.newShardRouting(new ShardId(index, 0), "node1", true,
// ShardRoutingState.STARTED)))
// .build())
// .build();
// Mockito.when(clusterService.state()).thenReturn(clusterState);
// Client client = Mockito.mock(Client.class);
//
// SetOnce<Boolean> actionCompleted = new SetOnce<>();
// ShrinkAction action = new ShrinkAction(randomIntBetween(1, 10));
//
// action.execute(index, client, clusterService, new LifecycleAction.Listener() {
// @Override
// public void onSuccess(boolean completed) {
// actionCompleted.set(completed);
// }
//
// @Override
// public void onFailure(Exception e) {
// throw new AssertionError("Unexpected method call", e);
// }
// });
//
// assertFalse(actionCompleted.get());
// Mockito.verifyZeroInteractions(client);
// }
//
// public void testExecuteWaitingOnAllShardsActive() {
// Index index = new Index(randomAlphaOfLengthBetween(1, 20), randomAlphaOfLengthBetween(1, 20));
// Index targetIndex = new Index("shrunk-" + index.getName(), randomAlphaOfLengthBetween(1, 20));
// ClusterService clusterService = Mockito.mock(ClusterService.class);
// IndexMetaData indexMetadata = IndexMetaData.builder(index.getName())
// .settings(settings(Version.CURRENT))
// .numberOfShards(randomIntBetween(1, 5)).numberOfReplicas(randomIntBetween(0, 5)).build();
// IndexMetaData targetIndexMetadata = IndexMetaData.builder(targetIndex.getName())
// .settings(settings(Version.CURRENT).put(IndexMetaData.INDEX_SHRINK_SOURCE_NAME_KEY, index.getName()))
// .numberOfShards(1).numberOfReplicas(randomIntBetween(0, 5)).build();
// ImmutableOpenMap.Builder<String, IndexMetaData> indices = ImmutableOpenMap.<String, IndexMetaData> builder().fPut(index.getName(),
// indexMetadata).fPut(targetIndex.getName(), targetIndexMetadata);
// ClusterState clusterState = ClusterState.builder(ClusterState.EMPTY_STATE).metaData(MetaData.builder().indices(indices.build()))
// .routingTable(RoutingTable.builder()
// .add(IndexRoutingTable.builder(targetIndex).addShard(
// TestShardRouting.newShardRouting(new ShardId(index, 0), "node1", true,
// ShardRoutingState.INITIALIZING)))
// .build())
// .build();
// Mockito.when(clusterService.state()).thenReturn(clusterState);
// Client client = Mockito.mock(Client.class);
// SetOnce<Boolean> actionCompleted = new SetOnce<>();
// ShrinkAction action = new ShrinkAction(randomIntBetween(1, 10));
//
// action.execute(index, client, clusterService, new LifecycleAction.Listener() {
// @Override
// public void onSuccess(boolean completed) {
// actionCompleted.set(completed);
// }
//
// @Override
// public void onFailure(Exception e) {
// throw new AssertionError("Unexpected method call", e);
// }
// });
//
// assertFalse(actionCompleted.get());
// }
//
// public void testExecuteIndexAlreadyExists() {
// Index index = new Index(randomAlphaOfLengthBetween(1, 20), randomAlphaOfLengthBetween(1, 20));
// Index targetIndex = new Index("shrunk-" + index.getName(), randomAlphaOfLengthBetween(1, 20));
// ClusterService clusterService = Mockito.mock(ClusterService.class);
// IndexMetaData indexMetadata = IndexMetaData.builder(index.getName())
// .settings(settings(Version.CURRENT))
// .numberOfShards(randomIntBetween(1, 5)).numberOfReplicas(randomIntBetween(0, 5)).build();
// IndexMetaData targetIndexMetadata = IndexMetaData.builder(targetIndex.getName())
// .settings(settings(Version.CURRENT))
// .numberOfShards(1).numberOfReplicas(randomIntBetween(0, 5)).build();
// ImmutableOpenMap.Builder<String, IndexMetaData> indices = ImmutableOpenMap.<String, IndexMetaData> builder().fPut(index.getName(),
// indexMetadata).fPut(targetIndex.getName(), targetIndexMetadata);
// ClusterState clusterState = ClusterState.builder(ClusterState.EMPTY_STATE)
// .metaData(MetaData.builder().indices(indices.build())).build();
// Mockito.when(clusterService.state()).thenReturn(clusterState);
// Client client = Mockito.mock(Client.class);
//
// SetOnce<Exception> actionFailed = new SetOnce<>();
// ShrinkAction action = new ShrinkAction(randomIntBetween(1, 10));
//
// action.execute(index, client, clusterService, new LifecycleAction.Listener() {
// @Override
// public void onSuccess(boolean completed) {
// throw new AssertionError("Unexpected method call to onSuccess");
// }
//
// @Override
// public void onFailure(Exception e) {
// actionFailed.set(e);
// }
// });
//
// assertThat(actionFailed.get().getMessage(), equalTo("Cannot shrink index [" + index.getName() + "]" +
// " because target index [" + targetIndex.getName() + "] already exists."));
// }
}

View File

@ -0,0 +1,15 @@
/*
* Copyright Elasticsearch B.V. and/or licensed to Elasticsearch B.V. under one
* or more contributor license agreements. Licensed under the Elastic License;
* you may not use this file except in compliance with the Elastic License.
*/
package org.elasticsearch.xpack.core.indexlifecycle;
import org.elasticsearch.test.ESTestCase;
public class StepTests extends ESTestCase {
public void test() {
}
}

View File

@ -36,10 +36,10 @@ 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 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) {

View File

@ -47,357 +47,361 @@ public class TimeseriesLifecycleTypeTests extends ESTestCase {
private static final ShrinkAction TEST_SHRINK_ACTION = new ShrinkAction(1);
private static final LongSupplier TEST_NOW_SUPPLIER = () -> 0L;
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 testStub() {
}
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 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 + "]");
// }
// }
}

View File

@ -105,7 +105,9 @@ public class IndexLifecycle extends Plugin implements ActionPlugin {
LifecycleSettings.LIFECYCLE_INDEX_CREATION_DATE_SETTING,
LifecycleSettings.LIFECYCLE_PHASE_TIME_SETTING,
LifecycleSettings.LIFECYCLE_ACTION_TIME_SETTING,
LifecycleSettings.LIFECYCLE_ACTION_SETTING);
LifecycleSettings.LIFECYCLE_ACTION_SETTING,
LifecycleSettings.LIFECYCLE_STEP_TIME_SETTING,
LifecycleSettings.LIFECYCLE_STEP_SETTING);
}
@Override

View File

@ -27,14 +27,18 @@ import org.elasticsearch.common.unit.TimeValue;
import org.elasticsearch.threadpool.ThreadPool;
import org.elasticsearch.xpack.core.indexlifecycle.LifecyclePolicy;
import org.elasticsearch.xpack.core.indexlifecycle.LifecycleSettings;
import org.elasticsearch.xpack.core.indexlifecycle.Step;
import org.elasticsearch.xpack.core.indexlifecycle.StepResult;
import org.elasticsearch.xpack.core.scheduler.SchedulerEngine;
import java.io.Closeable;
import java.time.Clock;
import java.util.Collections;
import java.util.List;
import java.util.SortedMap;
import java.util.concurrent.CompletableFuture;
import java.util.function.LongSupplier;
import java.util.stream.Collectors;
/**
* A service which runs the {@link LifecyclePolicy}s associated with indexes.
@ -112,38 +116,49 @@ public class IndexLifecycleService extends AbstractComponent
scheduler.get().add(scheduledJob);
}
public synchronized void triggerPolicies() {
IndexLifecycleMetadata indexLifecycleMetadata = clusterService.state().metaData().custom(IndexLifecycleMetadata.TYPE);
SortedMap<String, LifecyclePolicy> policies = indexLifecycleMetadata.getPolicies();
// loop through all indices in cluster state and filter for ones that are
// managed by the Index Lifecycle Service they have a index.lifecycle.name setting
// associated to a policy
ClusterState clusterState = clusterService.state();
clusterState.metaData().indices().valuesIt().forEachRemaining((idxMeta) -> {
String policyName = LifecycleSettings.LIFECYCLE_NAME_SETTING.get(idxMeta.getSettings());
if (Strings.isNullOrEmpty(policyName) == false) {
LifecyclePolicy policy = policies.get(policyName);
// fetch step
// check whether complete
// if complete, launch next task
clusterService.submitStateUpdateTask("index-lifecycle-" + policyName, new ClusterStateUpdateTask() {
@Override
public ClusterState execute(ClusterState currentState) throws Exception {
// ensure date is set
currentState = putLifecycleDate(currentState, idxMeta);
long lifecycleDate = currentState.metaData().settings()
.getAsLong(LifecycleSettings.LIFECYCLE_INDEX_CREATION_DATE, -1L);
// TODO(talevy): make real
List<Step> steps = policy.getPhases().values().stream()
.flatMap(p -> p.toSteps(idxMeta.getIndex(), lifecycleDate, client, threadPool, nowSupplier).stream())
.collect(Collectors.toList());
StepResult result = policy.execute(steps, currentState, idxMeta, client, nowSupplier);
return result.getClusterState();
}
@Override
public void onFailure(String source, Exception e) {
}
});
}
});
}
@Override
public void triggered(SchedulerEngine.Event event) {
if (event.getJobName().equals(IndexLifecycle.NAME)) {
logger.info("Job triggered: " + event.getJobName() + ", " + event.getScheduledTime() + ", " + event.getTriggeredTime());
IndexLifecycleMetadata indexLifecycleMetadata = clusterService.state().metaData().custom(IndexLifecycleMetadata.TYPE);
SortedMap<String, LifecyclePolicy> policies = indexLifecycleMetadata.getPolicies();
// loop through all indices in cluster state and filter for ones that are
// managed by the Index Lifecycle Service they have a index.lifecycle.name setting
// associated to a policy
clusterService.state().metaData().indices().valuesIt().forEachRemaining((idxMeta) -> {
String policyName = LifecycleSettings.LIFECYCLE_NAME_SETTING.get(idxMeta.getSettings());
if (Strings.isNullOrEmpty(policyName) == false) {
// ensure that all managed indices have `index.lifecycle.date` set
// and then execute their respective lifecycle policies.
putLifecycleDate(idxMeta).thenRun(() -> executePolicy(idxMeta, policies, policyName));
}
});
}
}
private void executePolicy(IndexMetaData idxMeta, SortedMap<String, LifecyclePolicy> policies, String policyName) {
logger.info("Checking index for next action: " + idxMeta.getIndex().getName() + " (" + policyName + ")");
LifecyclePolicy policy = policies.get(policyName);
if (policy == null) {
logger.error("Unknown lifecycle policy [{}] for index [{}]", policyName, idxMeta.getIndex().getName());
} else {
try {
policy.execute(new InternalIndexLifecycleContext(idxMeta.getIndex(), client, clusterService, nowSupplier));
} catch (Exception e) {
logger.error(new FormattedMessage("Failed to execute lifecycle policy [{}] for index [{}]", policyName,
idxMeta.getIndex().getName()), e);
}
triggerPolicies();
}
}
@ -166,29 +181,17 @@ public class IndexLifecycleService extends AbstractComponent
}));
}
private CompletableFuture<Void> putLifecycleDate(IndexMetaData idxMeta) {
CompletableFuture<Void> completableFuture = new CompletableFuture<>();
if (idxMeta.getSettings().hasValue(LifecycleSettings.LIFECYCLE_INDEX_CREATION_DATE_SETTING.getKey())) {
completableFuture.complete(null);
private ClusterState putLifecycleDate(ClusterState clusterState, IndexMetaData idxMeta) {
if (idxMeta.getSettings().hasValue(LifecycleSettings.LIFECYCLE_INDEX_CREATION_DATE)) {
return clusterState;
} else {
UpdateSettingsRequest updateSettingsRequest = new UpdateSettingsRequest(Settings.builder()
.put(LifecycleSettings.LIFECYCLE_INDEX_CREATION_DATE_SETTING.getKey(), idxMeta.getCreationDate()).build(),
idxMeta.getIndex().getName());
client.admin().indices().updateSettings(updateSettingsRequest, new ActionListener<UpdateSettingsResponse>() {
@Override
public void onResponse(UpdateSettingsResponse updateSettingsResponse) {
completableFuture.complete(null);
}
@Override
public void onFailure(Exception e) {
logger.error("unable to update " + LifecycleSettings.LIFECYCLE_INDEX_CREATION_DATE + " setting on indices", e);
completableFuture.completeExceptionally(e);
}
});
ClusterState.Builder builder = new ClusterState.Builder(clusterState);
MetaData.Builder metadataBuilder = MetaData.builder(clusterState.metaData());
Settings settings = Settings.builder()
.put(LifecycleSettings.LIFECYCLE_INDEX_CREATION_DATE_SETTING.getKey(), idxMeta.getCreationDate()).build();
metadataBuilder.updateSettings(settings, idxMeta.getIndex().getName());
return builder.metaData(metadataBuilder.build()).build();
}
return completableFuture;
}
@Override

View File

@ -1,134 +0,0 @@
/*
* 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.indexlifecycle;
import org.elasticsearch.action.ActionListener;
import org.elasticsearch.action.admin.indices.settings.put.UpdateSettingsRequest;
import org.elasticsearch.action.admin.indices.settings.put.UpdateSettingsResponse;
import org.elasticsearch.client.Client;
import org.elasticsearch.cluster.metadata.IndexMetaData;
import org.elasticsearch.cluster.service.ClusterService;
import org.elasticsearch.common.settings.Settings;
import org.elasticsearch.index.Index;
import org.elasticsearch.xpack.core.indexlifecycle.IndexLifecycleContext;
import org.elasticsearch.xpack.core.indexlifecycle.LifecycleAction;
import org.elasticsearch.xpack.core.indexlifecycle.LifecycleSettings;
import org.elasticsearch.xpack.core.indexlifecycle.Phase;
import java.util.function.LongSupplier;
/**
* An Implementation of {@link IndexLifecycleContext} which writes lifecycle
* state to index settings.
*/
public class InternalIndexLifecycleContext implements IndexLifecycleContext {
private Client client;
private Index index;
private LongSupplier nowSupplier;
private ClusterService clusterService;
/**
* @param index
* the {@link Index} for this context.
* @param client
* the {@link Client} to use when modifying the index settings.
* @param nowSupplier
* a {@link LongSupplier} to provide the current timestamp when
* required.
*/
public InternalIndexLifecycleContext(Index index, Client client, ClusterService clusterService, LongSupplier nowSupplier) {
this.index = index;
this.client = client;
this.clusterService = clusterService;
this.nowSupplier = nowSupplier;
}
private IndexMetaData getIdxMetaData() {
return clusterService.state().metaData().index(index.getName());
}
@Override
public void setPhase(String phase, Listener listener) {
Settings newLifecyclePhaseSettings = Settings.builder()
.put(LifecycleSettings.LIFECYCLE_PHASE, phase)
.put(LifecycleSettings.LIFECYCLE_PHASE_TIME, nowSupplier.getAsLong())
.put(LifecycleSettings.LIFECYCLE_ACTION_TIME, -1L)
.put(LifecycleSettings.LIFECYCLE_ACTION, "").build();
writeSettings(index.getName(), newLifecyclePhaseSettings, listener);
}
@Override
public String getPhase() {
return LifecycleSettings.LIFECYCLE_PHASE_SETTING.get(getIdxMetaData().getSettings());
}
@Override
public long getPhaseTime() {
return LifecycleSettings.LIFECYCLE_PHASE_TIME_SETTING.get(getIdxMetaData().getSettings());
}
@Override
public void setAction(String action, Listener listener) {
Settings newLifecycleActionSettings = Settings.builder()
.put(LifecycleSettings.LIFECYCLE_ACTION, action)
.put(LifecycleSettings.LIFECYCLE_ACTION_TIME, nowSupplier.getAsLong())
.build();
writeSettings(index.getName(), newLifecycleActionSettings, listener);
}
@Override
public String getAction() {
return LifecycleSettings.LIFECYCLE_ACTION_SETTING.get(getIdxMetaData().getSettings());
}
@Override
public long getActionTime() {
return LifecycleSettings.LIFECYCLE_ACTION_TIME_SETTING.get(getIdxMetaData().getSettings());
}
@Override
public String getLifecycleTarget() {
return index.getName();
}
@Override
public int getNumberOfReplicas() {
return getIdxMetaData().getNumberOfReplicas();
}
@Override
public boolean canExecute(Phase phase) {
long now = nowSupplier.getAsLong();
long initialIndexCreated = LifecycleSettings.LIFECYCLE_INDEX_CREATION_DATE_SETTING.get(getIdxMetaData().getSettings());
return (initialIndexCreated + phase.getAfter().millis()) <= now;
}
@Override
public void executeAction(LifecycleAction action, LifecycleAction.Listener listener) {
action.execute(index, client, clusterService, listener);
}
private void writeSettings(String index, Settings settings, Listener listener) {
client.admin().indices().updateSettings(new UpdateSettingsRequest(settings, index), new ActionListener<UpdateSettingsResponse>() {
@Override
public void onResponse(UpdateSettingsResponse response) {
if (response.isAcknowledged()) {
listener.onSuccess();
} else {
listener.onFailure(null);
}
}
@Override
public void onFailure(Exception e) {
listener.onFailure(e);
}
});
}
}

View File

@ -0,0 +1,15 @@
/*
* 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.indexlifecycle;
import org.elasticsearch.test.ESTestCase;
public class ClusterStateUpdateStepTests extends ESTestCase {
public void test() {
}
}

View File

@ -278,7 +278,7 @@ public class IndexLifecycleServiceTests extends ESTestCase {
@SuppressWarnings("unchecked")
public void testTriggeredWithMatchingPolicy() {
String policyName = randomAlphaOfLengthBetween(1, 20);
MockAction mockAction = new MockAction();
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));
@ -352,7 +352,7 @@ public class IndexLifecycleServiceTests extends ESTestCase {
@SuppressWarnings("unchecked")
public void testTriggeredWithDateSettingAlreadyPresent() {
String policyName = randomAlphaOfLengthBetween(1, 20);
MockAction mockAction = new MockAction();
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));
@ -409,7 +409,7 @@ public class IndexLifecycleServiceTests extends ESTestCase {
*/
public void testTriggeredUnknownPolicyNameSet() {
String policyName = randomAlphaOfLengthBetween(1, 20);
MockAction mockAction = new MockAction();
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));
@ -453,7 +453,7 @@ public class IndexLifecycleServiceTests extends ESTestCase {
*/
public void testTriggeredNoPolicyNameSet() {
String policyName = randomAlphaOfLengthBetween(1, 20);
MockAction mockAction = new MockAction();
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));

View File

@ -1,641 +0,0 @@
/*
* 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.indexlifecycle;
import org.apache.lucene.util.SetOnce;
import org.elasticsearch.Version;
import org.elasticsearch.action.ActionListener;
import org.elasticsearch.action.admin.indices.settings.put.UpdateSettingsRequest;
import org.elasticsearch.action.admin.indices.settings.put.UpdateSettingsResponse;
import org.elasticsearch.action.admin.indices.settings.put.UpdateSettingsTestHelper;
import org.elasticsearch.client.AdminClient;
import org.elasticsearch.client.Client;
import org.elasticsearch.client.IndicesAdminClient;
import org.elasticsearch.cluster.ClusterName;
import org.elasticsearch.cluster.ClusterState;
import org.elasticsearch.cluster.metadata.IndexMetaData;
import org.elasticsearch.cluster.metadata.MetaData;
import org.elasticsearch.cluster.service.ClusterService;
import org.elasticsearch.common.collect.ImmutableOpenMap;
import org.elasticsearch.common.settings.Settings;
import org.elasticsearch.common.unit.TimeValue;
import org.elasticsearch.index.Index;
import org.elasticsearch.test.ESTestCase;
import org.elasticsearch.xpack.core.indexlifecycle.IndexLifecycleContext.Listener;
import org.elasticsearch.xpack.core.indexlifecycle.LifecycleAction;
import org.elasticsearch.xpack.core.indexlifecycle.LifecycleSettings;
import org.elasticsearch.xpack.core.indexlifecycle.MockAction;
import org.elasticsearch.xpack.core.indexlifecycle.Phase;
import org.mockito.Mockito;
import org.mockito.invocation.InvocationOnMock;
import org.mockito.stubbing.Answer;
import java.util.Collections;
import java.util.function.LongSupplier;
public class InternalIndexLifecycleContextTests extends ESTestCase {
private static final Index TEST_INDEX = new Index("test", "test");
private ClusterState getClusterState(IndexMetaData indexMetaData) {
ImmutableOpenMap.Builder<String, IndexMetaData> indices = ImmutableOpenMap.<String, IndexMetaData> builder()
.fPut(indexMetaData.getIndex().getName(), indexMetaData);
MetaData metaData = MetaData.builder().indices(indices.build())
.persistentSettings(settings(Version.CURRENT).build()).build();
return ClusterState.builder(ClusterName.DEFAULT).metaData(metaData).build();
}
public void testSetPhase() {
long oldNow = randomNonNegativeLong();
long now = randomNonNegativeLong();
LongSupplier nowSupplier = () -> now;
long creationDate = randomNonNegativeLong();
String oldPhase = randomAlphaOfLengthBetween(1, 5);
String newPhase = randomAlphaOfLengthBetween(6, 10);
Settings expectedSettings = Settings.builder().put(LifecycleSettings.LIFECYCLE_PHASE, newPhase)
.put(LifecycleSettings.LIFECYCLE_PHASE_TIME, now)
.put(LifecycleSettings.LIFECYCLE_ACTION_TIME, -1L)
.put(LifecycleSettings.LIFECYCLE_ACTION, "").build();
IndexMetaData idxMeta = IndexMetaData.builder(TEST_INDEX.getName())
.settings(Settings.builder().put("index.version.created", 7000001L).put("index.creation_date", creationDate)
.put(LifecycleSettings.LIFECYCLE_PHASE, oldPhase)
.put(LifecycleSettings.LIFECYCLE_PHASE_TIME, oldNow)
.put(LifecycleSettings.LIFECYCLE_ACTION_TIME, oldNow)
.put(LifecycleSettings.LIFECYCLE_ACTION, randomAlphaOfLengthBetween(1, 20)).build())
.numberOfShards(randomIntBetween(1, 5)).numberOfReplicas(randomIntBetween(0, 5)).build();
ClusterState clusterState = getClusterState(idxMeta);
ClusterState updatedClusterState = getClusterState(IndexMetaData.builder(idxMeta)
.settings(Settings.builder().put(idxMeta.getSettings()).put(expectedSettings)).build());
ClusterService clusterService = Mockito.mock(ClusterService.class);
Mockito.when(clusterService.state()).thenReturn(clusterState, updatedClusterState);
Client client = Mockito.mock(Client.class);
AdminClient adminClient = Mockito.mock(AdminClient.class);
IndicesAdminClient indicesClient = Mockito.mock(IndicesAdminClient.class);
Mockito.when(client.admin()).thenReturn(adminClient);
Mockito.when(adminClient.indices()).thenReturn(indicesClient);
Mockito.doAnswer(new Answer<Void>() {
@Override
public Void answer(InvocationOnMock invocation) throws Throwable {
UpdateSettingsRequest request = (UpdateSettingsRequest) invocation.getArguments()[0];
@SuppressWarnings("unchecked")
ActionListener<UpdateSettingsResponse> listener = (ActionListener<UpdateSettingsResponse>) invocation.getArguments()[1];
UpdateSettingsTestHelper.assertSettingsRequest(request, expectedSettings, TEST_INDEX.getName());
listener.onResponse(UpdateSettingsTestHelper.createMockResponse(true));
return null;
}
}).when(indicesClient).updateSettings(Mockito.any(), Mockito.any());
InternalIndexLifecycleContext context = new InternalIndexLifecycleContext(TEST_INDEX, client, clusterService, nowSupplier);
// Use setOnce so it throws an error if we call the listener multiple
// times
SetOnce<Boolean> listenerCalled = new SetOnce<>();
assertEquals(oldPhase, context.getPhase());
context.setPhase(newPhase, new Listener() {
@Override
public void onSuccess() {
listenerCalled.set(true);
}
@Override
public void onFailure(Exception e) {
throw new AssertionError("Unexpected Error", e);
}
});
assertEquals(true, listenerCalled.get());
assertEquals(newPhase, context.getPhase());
Mockito.verify(client, Mockito.only()).admin();
Mockito.verify(adminClient, Mockito.only()).indices();
Mockito.verify(indicesClient, Mockito.only()).updateSettings(Mockito.any(), Mockito.any());
}
public void testSetPhaseNotAcknowledged() {
long now = randomNonNegativeLong();
LongSupplier nowSupplier = () -> now;
long creationDate = randomNonNegativeLong();
String newPhase = randomAlphaOfLengthBetween(1, 20);
IndexMetaData idxMeta = IndexMetaData.builder(TEST_INDEX.getName())
.settings(Settings.builder().put("index.version.created", 7000001L).put("index.creation_date", creationDate)
.put(LifecycleSettings.LIFECYCLE_PHASE, randomAlphaOfLengthBetween(1, 20))
.put(LifecycleSettings.LIFECYCLE_PHASE_TIME, now)
.put(LifecycleSettings.LIFECYCLE_ACTION, randomAlphaOfLengthBetween(1, 20)).build())
.numberOfShards(randomIntBetween(1, 5)).numberOfReplicas(randomIntBetween(0, 5)).build();
ClusterState clusterState = getClusterState(idxMeta);
ClusterService clusterService = Mockito.mock(ClusterService.class);
Mockito.when(clusterService.state()).thenReturn(clusterState);
Client client = Mockito.mock(Client.class);
AdminClient adminClient = Mockito.mock(AdminClient.class);
IndicesAdminClient indicesClient = Mockito.mock(IndicesAdminClient.class);
Mockito.when(client.admin()).thenReturn(adminClient);
Mockito.when(adminClient.indices()).thenReturn(indicesClient);
Mockito.doAnswer(new Answer<Void>() {
@Override
public Void answer(InvocationOnMock invocation) throws Throwable {
UpdateSettingsRequest request = (UpdateSettingsRequest) invocation.getArguments()[0];
@SuppressWarnings("unchecked")
ActionListener<UpdateSettingsResponse> listener = (ActionListener<UpdateSettingsResponse>) invocation.getArguments()[1];
Settings expectedSettings = Settings.builder().put(LifecycleSettings.LIFECYCLE_PHASE, newPhase)
.put(LifecycleSettings.LIFECYCLE_PHASE_TIME, now)
.put(LifecycleSettings.LIFECYCLE_ACTION_TIME, -1L)
.put(LifecycleSettings.LIFECYCLE_ACTION, "").build();
UpdateSettingsTestHelper.assertSettingsRequest(request, expectedSettings, TEST_INDEX.getName());
listener.onResponse(UpdateSettingsTestHelper.createMockResponse(false));
return null;
}
}).when(indicesClient).updateSettings(Mockito.any(), Mockito.any());
InternalIndexLifecycleContext context = new InternalIndexLifecycleContext(TEST_INDEX, client, clusterService, nowSupplier);
// Use setOnce so it throws an error if we call the listener multiple
// times
SetOnce<Boolean> listenerCalled = new SetOnce<>();
context.setPhase(newPhase, new Listener() {
@Override
public void onSuccess() {
throw new AssertionError("Unexpected Error");
}
@Override
public void onFailure(Exception e) {
assertNull(e);
listenerCalled.set(true);
}
});
assertEquals(true, listenerCalled.get());
Mockito.verify(client, Mockito.only()).admin();
Mockito.verify(adminClient, Mockito.only()).indices();
Mockito.verify(indicesClient, Mockito.only()).updateSettings(Mockito.any(), Mockito.any());
}
public void testSetPhaseFailure() {
long now = randomNonNegativeLong();
LongSupplier nowSupplier = () -> now;
long creationDate = randomNonNegativeLong();
String newPhase = randomAlphaOfLengthBetween(1, 20);
IndexMetaData idxMeta = IndexMetaData.builder(TEST_INDEX.getName())
.settings(Settings.builder().put("index.version.created", 7000001L).put("index.creation_date", creationDate)
.put(LifecycleSettings.LIFECYCLE_PHASE, randomAlphaOfLengthBetween(1, 20))
.put(LifecycleSettings.LIFECYCLE_PHASE_TIME, now)
.put(LifecycleSettings.LIFECYCLE_ACTION, randomAlphaOfLengthBetween(1, 20)).build())
.numberOfShards(randomIntBetween(1, 5)).numberOfReplicas(randomIntBetween(0, 5)).build();
ClusterState clusterState = getClusterState(idxMeta);
ClusterService clusterService = Mockito.mock(ClusterService.class);
Mockito.when(clusterService.state()).thenReturn(clusterState);
Exception exception = new RuntimeException();
Client client = Mockito.mock(Client.class);
AdminClient adminClient = Mockito.mock(AdminClient.class);
IndicesAdminClient indicesClient = Mockito.mock(IndicesAdminClient.class);
Mockito.when(client.admin()).thenReturn(adminClient);
Mockito.when(adminClient.indices()).thenReturn(indicesClient);
Mockito.doAnswer(new Answer<Void>() {
@Override
public Void answer(InvocationOnMock invocation) throws Throwable {
UpdateSettingsRequest request = (UpdateSettingsRequest) invocation.getArguments()[0];
@SuppressWarnings("unchecked")
ActionListener<UpdateSettingsResponse> listener = (ActionListener<UpdateSettingsResponse>) invocation.getArguments()[1];
Settings expectedSettings = Settings.builder().put(LifecycleSettings.LIFECYCLE_PHASE, newPhase)
.put(LifecycleSettings.LIFECYCLE_ACTION_TIME, -1L).put(LifecycleSettings.LIFECYCLE_PHASE_TIME, now)
.put(LifecycleSettings.LIFECYCLE_ACTION, "").build();
UpdateSettingsTestHelper.assertSettingsRequest(request, expectedSettings, TEST_INDEX.getName());
listener.onFailure(exception);
return null;
}
}).when(indicesClient).updateSettings(Mockito.any(), Mockito.any());
InternalIndexLifecycleContext context = new InternalIndexLifecycleContext(TEST_INDEX, client, clusterService, nowSupplier);
// Use setOnce so it throws an error if we call the listener multiple
// times
SetOnce<Boolean> listenerCalled = new SetOnce<>();
context.setPhase(newPhase, new Listener() {
@Override
public void onSuccess() {
throw new AssertionError("Unexpected Error");
}
@Override
public void onFailure(Exception e) {
assertSame(exception, e);
listenerCalled.set(true);
}
});
assertEquals(true, listenerCalled.get());
Mockito.verify(client, Mockito.only()).admin();
Mockito.verify(adminClient, Mockito.only()).indices();
Mockito.verify(indicesClient, Mockito.only()).updateSettings(Mockito.any(), Mockito.any());
}
public void testGetPhase() {
long creationDate = randomNonNegativeLong();
String phase = randomAlphaOfLengthBetween(1, 20);
IndexMetaData idxMeta = IndexMetaData.builder(TEST_INDEX.getName())
.settings(Settings.builder().put("index.version.created", 7000001L).put("index.creation_date", creationDate)
.put(LifecycleSettings.LIFECYCLE_PHASE, phase).build())
.numberOfShards(randomIntBetween(1, 5)).numberOfReplicas(randomIntBetween(0, 5)).build();
ClusterState clusterState = getClusterState(idxMeta);
ClusterService clusterService = Mockito.mock(ClusterService.class);
Mockito.when(clusterService.state()).thenReturn(clusterState);
InternalIndexLifecycleContext context = new InternalIndexLifecycleContext(TEST_INDEX, null, clusterService, () -> {
throw new AssertionError("nowSupplier should not be called");
});
assertEquals(phase, context.getPhase());
}
public void testGetPhaseTime() {
long creationDate = randomNonNegativeLong();
long phaseTime = randomNonNegativeLong();
IndexMetaData idxMeta = IndexMetaData.builder(TEST_INDEX.getName())
.settings(Settings.builder().put("index.version.created", 7000001L).put("index.creation_date", creationDate)
.put(LifecycleSettings.LIFECYCLE_PHASE_TIME, phaseTime).build())
.numberOfShards(randomIntBetween(1, 5)).numberOfReplicas(randomIntBetween(0, 5)).build();
ClusterState clusterState = getClusterState(idxMeta);
ClusterService clusterService = Mockito.mock(ClusterService.class);
Mockito.when(clusterService.state()).thenReturn(clusterState);
InternalIndexLifecycleContext context = new InternalIndexLifecycleContext(TEST_INDEX, null, clusterService, () -> {
throw new AssertionError("nowSupplier should not be called");
});
assertEquals(phaseTime, context.getPhaseTime());
}
public void testGetReplicas() {
int replicas = randomIntBetween(0, 5);
IndexMetaData idxMeta = IndexMetaData.builder(TEST_INDEX.getName())
.settings(Settings.builder().put("index.version.created", 7000001L).build())
.numberOfShards(randomIntBetween(1, 5)).numberOfReplicas(replicas).build();
ClusterState clusterState = getClusterState(idxMeta);
ClusterService clusterService = Mockito.mock(ClusterService.class);
Mockito.when(clusterService.state()).thenReturn(clusterState);
InternalIndexLifecycleContext context = new InternalIndexLifecycleContext(TEST_INDEX, null, clusterService, () -> {
throw new AssertionError("nowSupplier should not be called");
});
assertEquals(replicas, context.getNumberOfReplicas());
}
public void testSetAction() {
long now = randomNonNegativeLong();
LongSupplier nowSupplier = () -> now;
long creationDate = randomNonNegativeLong();
String oldAction = randomAlphaOfLengthBetween(1, 5);
String newAction = randomAlphaOfLengthBetween(6, 10);
Settings expectedSettings = Settings.builder().put(LifecycleSettings.LIFECYCLE_ACTION, newAction)
.put(LifecycleSettings.LIFECYCLE_ACTION_TIME, now).build();
IndexMetaData idxMeta = IndexMetaData.builder(TEST_INDEX.getName())
.settings(Settings.builder().put("index.version.created", 7000001L).put("index.creation_date", creationDate)
.put(LifecycleSettings.LIFECYCLE_ACTION_TIME, now).put(LifecycleSettings.LIFECYCLE_ACTION, oldAction).build())
.numberOfShards(randomIntBetween(1, 5)).numberOfReplicas(randomIntBetween(0, 5)).build();
ClusterState clusterState = getClusterState(idxMeta);
ClusterState updatedClusterState = getClusterState(IndexMetaData.builder(idxMeta)
.settings(Settings.builder().put(idxMeta.getSettings()).put(expectedSettings)).build());
ClusterService clusterService = Mockito.mock(ClusterService.class);
Mockito.when(clusterService.state()).thenReturn(clusterState, updatedClusterState);
Client client = Mockito.mock(Client.class);
AdminClient adminClient = Mockito.mock(AdminClient.class);
IndicesAdminClient indicesClient = Mockito.mock(IndicesAdminClient.class);
Mockito.when(client.admin()).thenReturn(adminClient);
Mockito.when(adminClient.indices()).thenReturn(indicesClient);
Mockito.doAnswer(new Answer<Void>() {
@Override
public Void answer(InvocationOnMock invocation) throws Throwable {
UpdateSettingsRequest request = (UpdateSettingsRequest) invocation.getArguments()[0];
@SuppressWarnings("unchecked")
ActionListener<UpdateSettingsResponse> listener = (ActionListener<UpdateSettingsResponse>) invocation.getArguments()[1];
UpdateSettingsTestHelper.assertSettingsRequest(request, expectedSettings, TEST_INDEX.getName());
listener.onResponse(UpdateSettingsTestHelper.createMockResponse(true));
return null;
}
}).when(indicesClient).updateSettings(Mockito.any(), Mockito.any());
InternalIndexLifecycleContext context = new InternalIndexLifecycleContext(TEST_INDEX, client, clusterService, nowSupplier);
// Use setOnce so it throws an error if we call the listener multiple
// times
SetOnce<Boolean> listenerCalled = new SetOnce<>();
assertEquals(oldAction, context.getAction());
context.setAction(newAction, new Listener() {
@Override
public void onSuccess() {
listenerCalled.set(true);
}
@Override
public void onFailure(Exception e) {
throw new AssertionError("Unexpected Error", e);
}
});
assertEquals(true, listenerCalled.get());
assertEquals(newAction, context.getAction());
Mockito.verify(client, Mockito.only()).admin();
Mockito.verify(adminClient, Mockito.only()).indices();
Mockito.verify(indicesClient, Mockito.only()).updateSettings(Mockito.any(), Mockito.any());
}
public void testSetActionNotAcknowledged() {
long now = randomNonNegativeLong();
LongSupplier nowSupplier = () -> now;
long creationDate = randomNonNegativeLong();
String newAction = randomAlphaOfLengthBetween(1, 20);
IndexMetaData idxMeta = IndexMetaData.builder(TEST_INDEX.getName())
.settings(Settings.builder().put("index.version.created", 7000001L).put("index.creation_date", creationDate)
.put(LifecycleSettings.LIFECYCLE_ACTION_TIME, now)
.put(LifecycleSettings.LIFECYCLE_ACTION, randomAlphaOfLengthBetween(1, 20)).build())
.numberOfShards(randomIntBetween(1, 5)).numberOfReplicas(randomIntBetween(0, 5)).build();
ClusterState clusterState = getClusterState(idxMeta);
ClusterService clusterService = Mockito.mock(ClusterService.class);
Mockito.when(clusterService.state()).thenReturn(clusterState);
Client client = Mockito.mock(Client.class);
AdminClient adminClient = Mockito.mock(AdminClient.class);
IndicesAdminClient indicesClient = Mockito.mock(IndicesAdminClient.class);
Mockito.when(client.admin()).thenReturn(adminClient);
Mockito.when(adminClient.indices()).thenReturn(indicesClient);
Mockito.doAnswer(new Answer<Void>() {
@Override
public Void answer(InvocationOnMock invocation) throws Throwable {
UpdateSettingsRequest request = (UpdateSettingsRequest) invocation.getArguments()[0];
@SuppressWarnings("unchecked")
ActionListener<UpdateSettingsResponse> listener = (ActionListener<UpdateSettingsResponse>) invocation.getArguments()[1];
Settings expectedSettings = Settings.builder().put(LifecycleSettings.LIFECYCLE_ACTION, newAction)
.put(LifecycleSettings.LIFECYCLE_ACTION_TIME, now).build();
UpdateSettingsTestHelper.assertSettingsRequest(request, expectedSettings, TEST_INDEX.getName());
listener.onResponse(UpdateSettingsTestHelper.createMockResponse(false));
return null;
}
}).when(indicesClient).updateSettings(Mockito.any(), Mockito.any());
InternalIndexLifecycleContext context = new InternalIndexLifecycleContext(TEST_INDEX, client, clusterService, nowSupplier);
// Use setOnce so it throws an error if we call the listener multiple
// times
SetOnce<Boolean> listenerCalled = new SetOnce<>();
context.setAction(newAction, new Listener() {
@Override
public void onSuccess() {
throw new AssertionError("Unexpected Error");
}
@Override
public void onFailure(Exception e) {
assertNull(e);
listenerCalled.set(true);
}
});
assertEquals(true, listenerCalled.get());
Mockito.verify(client, Mockito.only()).admin();
Mockito.verify(adminClient, Mockito.only()).indices();
Mockito.verify(indicesClient, Mockito.only()).updateSettings(Mockito.any(), Mockito.any());
}
public void testSetActionFailure() {
long now = randomNonNegativeLong();
LongSupplier nowSupplier = () -> now;
long creationDate = randomNonNegativeLong();
String newAction = randomAlphaOfLengthBetween(1, 20);
IndexMetaData idxMeta = IndexMetaData.builder(TEST_INDEX.getName())
.settings(Settings.builder().put("index.version.created", 7000001L).put("index.creation_date", creationDate)
.put(LifecycleSettings.LIFECYCLE_ACTION_TIME, now)
.put(LifecycleSettings.LIFECYCLE_ACTION, randomAlphaOfLengthBetween(1, 20)).build())
.numberOfShards(randomIntBetween(1, 5)).numberOfReplicas(randomIntBetween(0, 5)).build();
ClusterState clusterState = getClusterState(idxMeta);
ClusterService clusterService = Mockito.mock(ClusterService.class);
Mockito.when(clusterService.state()).thenReturn(clusterState);
Exception exception = new RuntimeException();
Client client = Mockito.mock(Client.class);
AdminClient adminClient = Mockito.mock(AdminClient.class);
IndicesAdminClient indicesClient = Mockito.mock(IndicesAdminClient.class);
Mockito.when(client.admin()).thenReturn(adminClient);
Mockito.when(adminClient.indices()).thenReturn(indicesClient);
Mockito.doAnswer(new Answer<Void>() {
@Override
public Void answer(InvocationOnMock invocation) throws Throwable {
UpdateSettingsRequest request = (UpdateSettingsRequest) invocation.getArguments()[0];
@SuppressWarnings("unchecked")
ActionListener<UpdateSettingsResponse> listener = (ActionListener<UpdateSettingsResponse>) invocation.getArguments()[1];
Settings expectedSettings = Settings.builder().put(LifecycleSettings.LIFECYCLE_ACTION, newAction)
.put(LifecycleSettings.LIFECYCLE_ACTION_TIME, now).build();
UpdateSettingsTestHelper.assertSettingsRequest(request, expectedSettings, TEST_INDEX.getName());
listener.onFailure(exception);
return null;
}
}).when(indicesClient).updateSettings(Mockito.any(), Mockito.any());
InternalIndexLifecycleContext context = new InternalIndexLifecycleContext(TEST_INDEX, client, clusterService, nowSupplier);
// Use setOnce so it throws an error if we call the listener multiple
// times
SetOnce<Boolean> listenerCalled = new SetOnce<>();
context.setAction(newAction, new Listener() {
@Override
public void onSuccess() {
throw new AssertionError("Unexpected Error");
}
@Override
public void onFailure(Exception e) {
assertSame(exception, e);
listenerCalled.set(true);
}
});
assertEquals(true, listenerCalled.get());
Mockito.verify(client, Mockito.only()).admin();
Mockito.verify(adminClient, Mockito.only()).indices();
Mockito.verify(indicesClient, Mockito.only()).updateSettings(Mockito.any(), Mockito.any());
}
public void testGetAction() {
long creationDate = randomNonNegativeLong();
String action = randomAlphaOfLengthBetween(1, 20);
IndexMetaData idxMeta = IndexMetaData.builder("test")
.settings(Settings.builder().put("index.version.created", 7000001L).put("index.creation_date", creationDate)
.put(LifecycleSettings.LIFECYCLE_ACTION, action).build())
.numberOfShards(randomIntBetween(1, 5)).numberOfReplicas(randomIntBetween(0, 5)).build();
ClusterState clusterState = getClusterState(idxMeta);
ClusterService clusterService = Mockito.mock(ClusterService.class);
Mockito.when(clusterService.state()).thenReturn(clusterState);
InternalIndexLifecycleContext context = new InternalIndexLifecycleContext(TEST_INDEX, null, clusterService, () -> {
throw new AssertionError("nowSupplier should not be called");
});
assertEquals(action, context.getAction());
}
public void testGetActionTime() {
long creationDate = randomNonNegativeLong();
long actionTime = randomNonNegativeLong();
IndexMetaData idxMeta = IndexMetaData.builder(TEST_INDEX.getName())
.settings(Settings.builder().put("index.version.created", 7000001L).put("index.creation_date", creationDate)
.put(LifecycleSettings.LIFECYCLE_ACTION_TIME, actionTime).build())
.numberOfShards(randomIntBetween(1, 5)).numberOfReplicas(randomIntBetween(0, 5)).build();
ClusterState clusterState = getClusterState(idxMeta);
ClusterService clusterService = Mockito.mock(ClusterService.class);
Mockito.when(clusterService.state()).thenReturn(clusterState);
InternalIndexLifecycleContext context = new InternalIndexLifecycleContext(TEST_INDEX, null, clusterService, () -> {
throw new AssertionError("nowSupplier should not be called");
});
assertEquals(actionTime, context.getActionTime());
}
public void testGetLifecycleTarget() {
long creationDate = randomNonNegativeLong();
Index index = new Index(randomAlphaOfLengthBetween(1, 20), randomAlphaOfLengthBetween(1, 20));
IndexMetaData idxMeta = IndexMetaData.builder(index.getName())
.settings(Settings.builder().put("index.version.created", 7000001L).put("index.creation_date", creationDate).build())
.numberOfShards(randomIntBetween(1, 5)).numberOfReplicas(randomIntBetween(0, 5)).build();
ClusterState clusterState = getClusterState(idxMeta);
ClusterService clusterService = Mockito.mock(ClusterService.class);
Mockito.when(clusterService.state()).thenReturn(clusterState);
InternalIndexLifecycleContext context = new InternalIndexLifecycleContext(index, null, clusterService, () -> {
throw new AssertionError("nowSupplier should not be called");
});
assertEquals(index.getName(), context.getLifecycleTarget());
}
public void testCanExecuteBeforeTrigger() {
TimeValue after = TimeValue.timeValueSeconds(randomIntBetween(0, 100000));
long creationDate = randomNonNegativeLong();
long now = random().longs(creationDate, creationDate + after.millis()).iterator().nextLong();
IndexMetaData idxMeta = IndexMetaData.builder(TEST_INDEX.getName())
.settings(Settings.builder().put("index.version.created", 7000001L)
.put(LifecycleSettings.LIFECYCLE_INDEX_CREATION_DATE, creationDate).build())
.numberOfShards(randomIntBetween(1, 5)).numberOfReplicas(randomIntBetween(0, 5)).build();
ClusterState clusterState = getClusterState(idxMeta);
ClusterService clusterService = Mockito.mock(ClusterService.class);
Mockito.when(clusterService.state()).thenReturn(clusterState);
InternalIndexLifecycleContext context = new InternalIndexLifecycleContext(TEST_INDEX, null, clusterService, () -> now);
Phase phase = new Phase("test_phase", after, Collections.emptyMap());
assertFalse(context.canExecute(phase));
}
public void testCanExecuteOnTrigger() {
TimeValue after = TimeValue.timeValueSeconds(randomIntBetween(0, 100000));
long creationDate = randomNonNegativeLong();
long now = creationDate + after.millis();
IndexMetaData idxMeta = IndexMetaData.builder(TEST_INDEX.getName())
.settings(Settings.builder().put("index.version.created", 7000001L)
.put(LifecycleSettings.LIFECYCLE_INDEX_CREATION_DATE, creationDate).build())
.numberOfShards(randomIntBetween(1, 5)).numberOfReplicas(randomIntBetween(0, 5)).build();
ClusterState clusterState = getClusterState(idxMeta);
ClusterService clusterService = Mockito.mock(ClusterService.class);
Mockito.when(clusterService.state()).thenReturn(clusterState);
InternalIndexLifecycleContext context = new InternalIndexLifecycleContext(TEST_INDEX, null, clusterService, () -> now);
Phase phase = new Phase("test_phase", after, Collections.emptyMap());
assertTrue(context.canExecute(phase));
}
public void testCanExecuteAfterTrigger() {
TimeValue after = TimeValue.timeValueSeconds(randomIntBetween(0, 100000));
long creationDate = randomNonNegativeLong();
long now = random().longs(creationDate + after.millis(), Long.MAX_VALUE).iterator().nextLong();
IndexMetaData idxMeta = IndexMetaData.builder(TEST_INDEX.getName())
.settings(Settings.builder().put("index.version.created", 7000001L)
.put(LifecycleSettings.LIFECYCLE_INDEX_CREATION_DATE, creationDate).build())
.numberOfShards(randomIntBetween(1, 5)).numberOfReplicas(randomIntBetween(0, 5)).build();
ClusterState clusterState = getClusterState(idxMeta);
ClusterService clusterService = Mockito.mock(ClusterService.class);
Mockito.when(clusterService.state()).thenReturn(clusterState);
InternalIndexLifecycleContext context = new InternalIndexLifecycleContext(TEST_INDEX, null, clusterService, () -> now);
Phase phase = new Phase("test_phase", after, Collections.emptyMap());
assertTrue(context.canExecute(phase));
}
public void testExecuteAction() {
IndexMetaData idxMeta = IndexMetaData.builder(TEST_INDEX.getName())
.settings(Settings.builder().put("index.version.created", 7000001L).put("index.creation_date", 0L).build())
.numberOfShards(randomIntBetween(1, 5)).numberOfReplicas(randomIntBetween(0, 5)).build();
ClusterState clusterState = getClusterState(idxMeta);
ClusterService clusterService = Mockito.mock(ClusterService.class);
Mockito.when(clusterService.state()).thenReturn(clusterState);
InternalIndexLifecycleContext context = new InternalIndexLifecycleContext(TEST_INDEX, null, clusterService, () -> {
throw new AssertionError("nowSupplier should not be called");
});
MockAction action = new MockAction();
action.setCompleteOnExecute(true);
assertFalse(action.wasCompleted());
assertEquals(0L, action.getExecutedCount());
SetOnce<Boolean> listenerCalled = new SetOnce<>();
context.executeAction(action, 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());
}
}

View File

@ -5,24 +5,35 @@
*/
package org.elasticsearch.xpack.indexlifecycle;
import org.elasticsearch.Version;
import org.elasticsearch.client.Client;
import org.elasticsearch.cluster.ClusterModule;
import org.elasticsearch.cluster.ClusterState;
import org.elasticsearch.cluster.metadata.IndexMetaData;
import org.elasticsearch.cluster.metadata.MetaData;
import org.elasticsearch.common.ParseField;
import org.elasticsearch.common.io.stream.NamedWriteableRegistry;
import org.elasticsearch.common.io.stream.Writeable.Reader;
import org.elasticsearch.common.unit.TimeValue;
import org.elasticsearch.common.xcontent.NamedXContentRegistry;
import org.elasticsearch.common.xcontent.XContentParser;
import org.elasticsearch.index.Index;
import org.elasticsearch.test.AbstractSerializingTestCase;
import org.elasticsearch.threadpool.TestThreadPool;
import org.elasticsearch.threadpool.ThreadPool;
import org.elasticsearch.xpack.core.indexlifecycle.ClusterStateUpdateStep;
import org.elasticsearch.xpack.core.indexlifecycle.ConditionalWaitStep;
import org.elasticsearch.xpack.core.indexlifecycle.DeleteAction;
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.MockIndexLifecycleContext;
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.StepResult;
import org.elasticsearch.xpack.core.indexlifecycle.TimeseriesLifecycleType;
import org.elasticsearch.xpack.core.indexlifecycle.TestLifecycleType;
import org.junit.Before;
import java.io.IOException;
import java.util.ArrayList;
@ -32,6 +43,9 @@ import java.util.HashMap;
import java.util.LinkedHashMap;
import java.util.List;
import java.util.Map;
import java.util.function.LongSupplier;
import static org.mockito.Mockito.mock;
public class LifecyclePolicyTests extends AbstractSerializingTestCase<LifecyclePolicy> {
@ -112,347 +126,36 @@ public class LifecyclePolicyTests extends AbstractSerializingTestCase<LifecycleP
assertSame(TimeseriesLifecycleType.INSTANCE, policy.getType());
}
@Before
public void setupPolicy() {
public void testSteps() throws Exception {
ThreadPool threadPool = new TestThreadPool("test");
LongSupplier nowSupplier = () -> 0L;
Client client = mock(Client.class);
Step phaseAfterStep = new PhaseAfterStep(threadPool, 0L, nowSupplier,
TimeValue.timeValueSeconds(0L), "name-0", "index", "phase", "mock_action");
Step updateStep = new ClusterStateUpdateStep("name-1", "index", "phase", "mock_action",
(state) -> state);
Step waitStep = new ConditionalWaitStep("name-2", "index", "phase", "mock_action",
(state) -> true);
indexName = randomAlphaOfLengthBetween(1, 20);
lifecycleName = randomAlphaOfLengthBetween(1, 20);
Map<String, Phase> phases = new LinkedHashMap<>();
firstAction = new MockAction();
firstAction = new MockAction(Arrays.asList(phaseAfterStep, updateStep, waitStep));
Map<String, LifecycleAction> actions = Collections.singletonMap(MockAction.NAME, firstAction);
TimeValue after = TimeValue.timeValueSeconds(0);
firstPhase = new Phase("first_phase", after, actions);
firstPhase = new Phase("phase", null, actions);
phases.put(firstPhase.getName(), firstPhase);
secondAction = new MockAction();
actions = Collections.singletonMap(MockAction.NAME, secondAction);
after = TimeValue.timeValueSeconds(10);
secondPhase = new Phase("second_phase", after, actions);
phases.put(secondPhase.getName(), secondPhase);
thirdAction = new MockAction();
actions = Collections.singletonMap(MockAction.NAME, thirdAction);
after = TimeValue.timeValueSeconds(20);
thirdPhase = new Phase("third_phase", after, actions);
phases.put(thirdPhase.getName(), thirdPhase);
policy = new LifecyclePolicy(TestLifecycleType.INSTANCE, lifecycleName, phases);
policy = new LifecyclePolicy(TestLifecycleType.INSTANCE, lifecycleName, null);
List<Step> phaseSteps = firstPhase.toSteps(new Index(indexName, indexName), 0L,
client, threadPool, nowSupplier);
ClusterState clusterState = ClusterState.builder(ClusterState.EMPTY_STATE)
.metaData(MetaData.builder().put(
IndexMetaData.builder("index")
.settings(settings(Version.CURRENT))
.numberOfShards(1).numberOfReplicas(1))).build();
StepResult result = policy.execute(phaseSteps, clusterState, clusterState.metaData().index("index"), client, nowSupplier);
threadPool.shutdown();
}
public void testExecuteNewIndexBeforeTrigger() throws Exception {
MockIndexLifecycleContext context = new MockIndexLifecycleContext(indexName, "", "", 0, () -> 0L) {
@Override
public boolean canExecute(Phase phase) {
if (phase == firstPhase) {
return false;
} else {
throw new AssertionError("canExecute should not have been called on this phase: " + phase.getName());
}
}
};
policy.execute(context);
assertEquals(indexName, context.getLifecycleTarget());
assertEquals("", context.getPhase());
assertEquals("", context.getAction());
assertFalse(firstAction.wasCompleted());
assertEquals(0L, firstAction.getExecutedCount());
assertFalse(secondAction.wasCompleted());
assertEquals(0L, secondAction.getExecutedCount());
assertFalse(thirdAction.wasCompleted());
assertEquals(0L, thirdAction.getExecutedCount());
}
public void testExecuteNewIndexAfterTrigger() throws Exception {
MockIndexLifecycleContext context = new MockIndexLifecycleContext(indexName, "", "", 0, () -> 0L) {
@Override
public boolean canExecute(Phase phase) {
if (phase == firstPhase) {
return true;
} else {
throw new AssertionError("canExecute should not have been called on this phase: " + phase.getName());
}
}
};
policy.execute(context);
assertEquals(indexName, context.getLifecycleTarget());
assertEquals(firstPhase.getName(), context.getPhase());
assertEquals(MockAction.NAME, context.getAction());
assertFalse(firstAction.wasCompleted());
assertEquals(1L, firstAction.getExecutedCount());
assertFalse(secondAction.wasCompleted());
assertEquals(0L, secondAction.getExecutedCount());
assertFalse(thirdAction.wasCompleted());
assertEquals(0L, thirdAction.getExecutedCount());
}
public void testExecuteNewIndexAfterTriggerFailure() throws Exception {
MockIndexLifecycleContext context = new MockIndexLifecycleContext(indexName, "", "", 0, () -> 0L) {
@Override
public boolean canExecute(Phase phase) {
if (phase == firstPhase) {
return true;
} else {
throw new AssertionError("canExecute should not have been called on this phase: " + phase.getName());
}
}
};
RuntimeException exception = new RuntimeException();
context.failOnSetters(exception);
policy.execute(context);
assertEquals(indexName, context.getLifecycleTarget());
assertEquals("", context.getPhase());
assertEquals("", context.getAction());
assertFalse(firstAction.wasCompleted());
assertEquals(0L, firstAction.getExecutedCount());
assertFalse(secondAction.wasCompleted());
assertEquals(0L, secondAction.getExecutedCount());
assertFalse(thirdAction.wasCompleted());
assertEquals(0L, thirdAction.getExecutedCount());
}
public void testExecuteFirstPhase() throws Exception {
MockIndexLifecycleContext context = new MockIndexLifecycleContext(indexName, firstPhase.getName(), "", 0, () -> 0L) {
@Override
public boolean canExecute(Phase phase) {
throw new AssertionError("canExecute should not have been called.");
}
};
policy.execute(context);
assertEquals(indexName, context.getLifecycleTarget());
assertEquals(firstPhase.getName(), context.getPhase());
assertEquals(MockAction.NAME, context.getAction());
assertFalse(firstAction.wasCompleted());
assertEquals(1L, firstAction.getExecutedCount());
assertFalse(secondAction.wasCompleted());
assertEquals(0L, secondAction.getExecutedCount());
assertFalse(thirdAction.wasCompleted());
assertEquals(0L, thirdAction.getExecutedCount());
}
public void testExecuteSecondPhase() throws Exception {
MockIndexLifecycleContext context = new MockIndexLifecycleContext(indexName, secondPhase.getName(), "", 0, () -> 0L) {
@Override
public boolean canExecute(Phase phase) {
throw new AssertionError("canExecute should not have been called.");
}
};
policy.execute(context);
assertEquals(indexName, context.getLifecycleTarget());
assertEquals(secondPhase.getName(), context.getPhase());
assertEquals(MockAction.NAME, context.getAction());
assertFalse(firstAction.wasCompleted());
assertEquals(0L, firstAction.getExecutedCount());
assertFalse(secondAction.wasCompleted());
assertEquals(1L, secondAction.getExecutedCount());
assertFalse(thirdAction.wasCompleted());
assertEquals(0L, thirdAction.getExecutedCount());
}
public void testExecuteThirdPhase() throws Exception {
MockIndexLifecycleContext context = new MockIndexLifecycleContext(indexName, thirdPhase.getName(), "", 0, () -> 0L) {
@Override
public boolean canExecute(Phase phase) {
throw new AssertionError("canExecute should not have been called.");
}
};
policy.execute(context);
assertEquals(indexName, context.getLifecycleTarget());
assertEquals(thirdPhase.getName(), context.getPhase());
assertEquals(MockAction.NAME, context.getAction());
assertFalse(firstAction.wasCompleted());
assertEquals(0L, firstAction.getExecutedCount());
assertFalse(secondAction.wasCompleted());
assertEquals(0L, secondAction.getExecutedCount());
assertFalse(thirdAction.wasCompleted());
assertEquals(1L, thirdAction.getExecutedCount());
}
public void testExecuteMissingPhase() throws Exception {
MockIndexLifecycleContext context = new MockIndexLifecycleContext(indexName, "does_not_exist", "", 0, () -> 0L) {
@Override
public boolean canExecute(Phase phase) {
throw new AssertionError("canExecute should not have been called.");
}
};
IllegalStateException exception = expectThrows(IllegalStateException.class, () -> policy.execute(context));
assertEquals(
"Current phase [" + "does_not_exist" + "] not found in lifecycle [" + lifecycleName + "] for index [" + indexName + "]",
exception.getMessage());
assertEquals(indexName, context.getLifecycleTarget());
assertEquals("does_not_exist", context.getPhase());
assertEquals("", context.getAction());
assertFalse(firstAction.wasCompleted());
assertEquals(0L, firstAction.getExecutedCount());
assertFalse(secondAction.wasCompleted());
assertEquals(0L, secondAction.getExecutedCount());
assertFalse(thirdAction.wasCompleted());
assertEquals(0L, thirdAction.getExecutedCount());
}
public void testExecuteFirstPhaseCompletedBeforeTrigger() throws Exception {
MockIndexLifecycleContext context = new MockIndexLifecycleContext(indexName, firstPhase.getName(), Phase.PHASE_COMPLETED,
0, () -> 0L) {
@Override
public boolean canExecute(Phase phase) {
if (phase == secondPhase) {
return false;
} else {
throw new AssertionError("canExecute should not have been called on this phase: " + phase.getName());
}
}
};
policy.execute(context);
assertEquals(indexName, context.getLifecycleTarget());
assertEquals(firstPhase.getName(), context.getPhase());
assertEquals(Phase.PHASE_COMPLETED, context.getAction());
assertFalse(firstAction.wasCompleted());
assertEquals(0L, thirdAction.getExecutedCount());
assertFalse(thirdAction.wasCompleted());
assertEquals(0L, secondAction.getExecutedCount());
assertFalse(thirdAction.wasCompleted());
assertEquals(0L, thirdAction.getExecutedCount());
}
public void testExecuteFirstPhaseCompletedAfterTrigger() throws Exception {
MockIndexLifecycleContext context = new MockIndexLifecycleContext(indexName, firstPhase.getName(), Phase.PHASE_COMPLETED,
0, () -> 0L) {
@Override
public boolean canExecute(Phase phase) {
if (phase == secondPhase) {
return true;
} else {
throw new AssertionError("canExecute should not have been called on this phase: " + phase.getName());
}
}
};
policy.execute(context);
assertEquals(indexName, context.getLifecycleTarget());
assertEquals(secondPhase.getName(), context.getPhase());
assertEquals(MockAction.NAME, context.getAction());
assertFalse(firstAction.wasCompleted());
assertEquals(0L, firstAction.getExecutedCount());
assertFalse(secondAction.wasCompleted());
assertEquals(1L, secondAction.getExecutedCount());
assertFalse(thirdAction.wasCompleted());
assertEquals(0L, thirdAction.getExecutedCount());
}
public void testExecuteSecondPhaseCompletedBeforeTrigger() throws Exception {
MockIndexLifecycleContext context = new MockIndexLifecycleContext(indexName, secondPhase.getName(), Phase.PHASE_COMPLETED,
0, () -> 0L) {
@Override
public boolean canExecute(Phase phase) {
if (phase == thirdPhase) {
return false;
} else {
throw new AssertionError("canExecute should not have been called on this phase: " + phase.getName());
}
}
};
policy.execute(context);
assertEquals(indexName, context.getLifecycleTarget());
assertEquals(secondPhase.getName(), context.getPhase());
assertEquals(Phase.PHASE_COMPLETED, context.getAction());
assertFalse(firstAction.wasCompleted());
assertEquals(0L, firstAction.getExecutedCount());
assertFalse(secondAction.wasCompleted());
assertEquals(0L, secondAction.getExecutedCount());
assertFalse(thirdAction.wasCompleted());
assertEquals(0L, thirdAction.getExecutedCount());
}
public void testExecuteSecondPhaseCompletedAfterTrigger() throws Exception {
MockIndexLifecycleContext context = new MockIndexLifecycleContext(indexName, secondPhase.getName(), Phase.PHASE_COMPLETED,
0, () -> 0L) {
@Override
public boolean canExecute(Phase phase) {
if (phase == thirdPhase) {
return true;
} else {
throw new AssertionError("canExecute should not have been called on this phase: " + phase.getName());
}
}
};
policy.execute(context);
assertEquals(indexName, context.getLifecycleTarget());
assertEquals(thirdPhase.getName(), context.getPhase());
assertEquals(MockAction.NAME, context.getAction());
assertFalse(firstAction.wasCompleted());
assertEquals(0L, firstAction.getExecutedCount());
assertFalse(secondAction.wasCompleted());
assertEquals(0L, secondAction.getExecutedCount());
assertFalse(thirdAction.wasCompleted());
assertEquals(1L, thirdAction.getExecutedCount());
}
public void testExecuteThirdPhaseCompleted() throws Exception {
MockIndexLifecycleContext context = new MockIndexLifecycleContext(indexName, thirdPhase.getName(), Phase.PHASE_COMPLETED,
0, () -> 0L) {
@Override
public boolean canExecute(Phase phase) {
throw new AssertionError("canExecute should not have been called");
}
@Override
public int getNumberOfReplicas() {
return 0;
}
};
policy.execute(context);
assertEquals(indexName, context.getLifecycleTarget());
assertEquals(thirdPhase.getName(), context.getPhase());
assertEquals(Phase.PHASE_COMPLETED, context.getAction());
assertFalse(firstAction.wasCompleted());
assertEquals(0L, firstAction.getExecutedCount());
assertFalse(secondAction.wasCompleted());
assertEquals(0L, secondAction.getExecutedCount());
assertFalse(thirdAction.wasCompleted());
assertEquals(0L, thirdAction.getExecutedCount());
}
}