diff --git a/x-pack/plugin/core/src/main/java/org/elasticsearch/xpack/core/indexlifecycle/IndexLifecycleMetadata.java b/x-pack/plugin/core/src/main/java/org/elasticsearch/xpack/core/indexlifecycle/IndexLifecycleMetadata.java index feb5cf5306811..b7a90a2b6e346 100644 --- a/x-pack/plugin/core/src/main/java/org/elasticsearch/xpack/core/indexlifecycle/IndexLifecycleMetadata.java +++ b/x-pack/plugin/core/src/main/java/org/elasticsearch/xpack/core/indexlifecycle/IndexLifecycleMetadata.java @@ -34,6 +34,7 @@ public class IndexLifecycleMetadata implements XPackMetaDataCustom { public static final String TYPE = "index_lifecycle"; public static final ParseField POLICIES_FIELD = new ParseField("policies"); + public static final IndexLifecycleMetadata EMPTY = new IndexLifecycleMetadata(Collections.emptySortedMap()); @SuppressWarnings("unchecked") public static final ConstructingObjectParser PARSER = new ConstructingObjectParser<>( diff --git a/x-pack/plugin/index-lifecycle/src/main/java/org/elasticsearch/xpack/indexlifecycle/IndexLifecycle.java b/x-pack/plugin/index-lifecycle/src/main/java/org/elasticsearch/xpack/indexlifecycle/IndexLifecycle.java index 9d5123bf13516..ad8a2e55c46f0 100644 --- a/x-pack/plugin/index-lifecycle/src/main/java/org/elasticsearch/xpack/indexlifecycle/IndexLifecycle.java +++ b/x-pack/plugin/index-lifecycle/src/main/java/org/elasticsearch/xpack/indexlifecycle/IndexLifecycle.java @@ -49,7 +49,7 @@ import org.elasticsearch.xpack.indexlifecycle.action.RestPutLifecycleAction; import org.elasticsearch.xpack.indexlifecycle.action.RestRetryAction; import org.elasticsearch.xpack.indexlifecycle.action.TransportSetPolicyForIndexAction; -import org.elasticsearch.xpack.indexlifecycle.action.TransportDeleteLifcycleAction; +import org.elasticsearch.xpack.indexlifecycle.action.TransportDeleteLifecycleAction; import org.elasticsearch.xpack.indexlifecycle.action.TransportExplainLifecycleAction; import org.elasticsearch.xpack.indexlifecycle.action.TransportGetLifecycleAction; import org.elasticsearch.xpack.indexlifecycle.action.TransportMoveToStepAction; @@ -124,7 +124,7 @@ public Collection createComponents(Client client, ClusterService cluster return emptyList(); } indexLifecycleInitialisationService - .set(new IndexLifecycleService(settings, client, clusterService, getClock(), threadPool, System::currentTimeMillis)); + .set(new IndexLifecycleService(settings, client, clusterService, getClock(), System::currentTimeMillis)); return Collections.singletonList(indexLifecycleInitialisationService.get()); } @@ -164,7 +164,7 @@ public List getRestHandlers(Settings settings, RestController restC return Arrays.asList( new ActionHandler<>(PutLifecycleAction.INSTANCE, TransportPutLifecycleAction.class), new ActionHandler<>(GetLifecycleAction.INSTANCE, TransportGetLifecycleAction.class), - new ActionHandler<>(DeleteLifecycleAction.INSTANCE, TransportDeleteLifcycleAction.class), + new ActionHandler<>(DeleteLifecycleAction.INSTANCE, TransportDeleteLifecycleAction.class), new ActionHandler<>(ExplainLifecycleAction.INSTANCE, TransportExplainLifecycleAction.class), new ActionHandler<>(SetPolicyForIndexAction.INSTANCE, TransportSetPolicyForIndexAction.class), new ActionHandler<>(MoveToStepAction.INSTANCE, TransportMoveToStepAction.class), diff --git a/x-pack/plugin/index-lifecycle/src/main/java/org/elasticsearch/xpack/indexlifecycle/IndexLifecycleService.java b/x-pack/plugin/index-lifecycle/src/main/java/org/elasticsearch/xpack/indexlifecycle/IndexLifecycleService.java index ca575d68e31ff..1adf8166ea431 100644 --- a/x-pack/plugin/index-lifecycle/src/main/java/org/elasticsearch/xpack/indexlifecycle/IndexLifecycleService.java +++ b/x-pack/plugin/index-lifecycle/src/main/java/org/elasticsearch/xpack/indexlifecycle/IndexLifecycleService.java @@ -12,15 +12,12 @@ import org.elasticsearch.cluster.ClusterState; import org.elasticsearch.cluster.ClusterStateApplier; import org.elasticsearch.cluster.ClusterStateListener; -import org.elasticsearch.cluster.ClusterStateUpdateTask; -import org.elasticsearch.cluster.metadata.MetaData; import org.elasticsearch.cluster.service.ClusterService; import org.elasticsearch.common.Strings; import org.elasticsearch.common.component.AbstractComponent; import org.elasticsearch.common.logging.ESLoggerFactory; import org.elasticsearch.common.settings.Settings; import org.elasticsearch.common.unit.TimeValue; -import org.elasticsearch.threadpool.ThreadPool; import org.elasticsearch.xpack.core.indexlifecycle.IndexLifecycleMetadata; import org.elasticsearch.xpack.core.indexlifecycle.LifecyclePolicy; import org.elasticsearch.xpack.core.indexlifecycle.LifecycleSettings; @@ -29,7 +26,6 @@ import java.io.Closeable; import java.time.Clock; -import java.util.Collections; import java.util.function.LongSupplier; /** @@ -44,18 +40,15 @@ public class IndexLifecycleService extends AbstractComponent private final PolicyStepsRegistry policyRegistry; private Client client; private ClusterService clusterService; - private ThreadPool threadPool; private LongSupplier nowSupplier; private SchedulerEngine.Job scheduledJob; private IndexLifecycleRunner lifecycleRunner; - public IndexLifecycleService(Settings settings, Client client, ClusterService clusterService, Clock clock, - ThreadPool threadPool, LongSupplier nowSupplier) { + public IndexLifecycleService(Settings settings, Client client, ClusterService clusterService, Clock clock, LongSupplier nowSupplier) { super(settings); this.client = client; this.clusterService = clusterService; this.clock = clock; - this.threadPool = threadPool; this.nowSupplier = nowSupplier; this.scheduledJob = null; this.policyRegistry = new PolicyStepsRegistry(); @@ -91,8 +84,8 @@ public PolicyStepsRegistry getPolicyRegistry() { @Override public void clusterChanged(ClusterChangedEvent event) { - if (event.localNodeMaster()) { // only act if we are master, otherwise keep idle until elected - IndexLifecycleMetadata lifecycleMetadata = event.state().metaData().custom(IndexLifecycleMetadata.TYPE); + IndexLifecycleMetadata lifecycleMetadata = event.state().metaData().custom(IndexLifecycleMetadata.TYPE); + if (event.localNodeMaster() && lifecycleMetadata != null) { TimeValue pollInterval = LifecycleSettings.LIFECYCLE_POLL_INTERVAL_SETTING .get(event.state().getMetaData().settings()); TimeValue previousPollInterval = LifecycleSettings.LIFECYCLE_POLL_INTERVAL_SETTING @@ -100,10 +93,8 @@ public void clusterChanged(ClusterChangedEvent event) { boolean pollIntervalSettingChanged = !pollInterval.equals(previousPollInterval); - if (lifecycleMetadata == null) { // no lifecycle metadata, install initial empty metadata state - lifecycleMetadata = new IndexLifecycleMetadata(Collections.emptySortedMap()); - installMetadata(lifecycleMetadata); - } else if (scheduler.get() == null) { // metadata installed and scheduler should be kicked off. start your engines. + + if (scheduler.get() == null) { // metadata installed and scheduler should be kicked off. start your engines. scheduler.set(new SchedulerEngine(clock)); scheduler.get().register(this); scheduleJob(pollInterval); @@ -151,26 +142,7 @@ public void triggered(SchedulerEngine.Event event) { } } - private void installMetadata(IndexLifecycleMetadata lifecycleMetadata) { - threadPool.executor(ThreadPool.Names.GENERIC) - .execute(() -> clusterService.submitStateUpdateTask("install-index-lifecycle-metadata", new ClusterStateUpdateTask() { - @Override - public ClusterState execute(ClusterState currentState) throws Exception { - ClusterState.Builder builder = new ClusterState.Builder(currentState); - MetaData.Builder metadataBuilder = MetaData.builder(currentState.metaData()); - metadataBuilder.putCustom(IndexLifecycleMetadata.TYPE, lifecycleMetadata); - builder.metaData(metadataBuilder.build()); - return builder.build(); - } - - @Override - public void onFailure(String source, Exception e) { - logger.error("unable to install index lifecycle metadata", e); - } - })); - } - - public void triggerPolicies(ClusterState clusterState, boolean fromClusterStateChange) { + void triggerPolicies(ClusterState clusterState, boolean fromClusterStateChange) { // 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 diff --git a/x-pack/plugin/index-lifecycle/src/main/java/org/elasticsearch/xpack/indexlifecycle/PolicyStepsRegistry.java b/x-pack/plugin/index-lifecycle/src/main/java/org/elasticsearch/xpack/indexlifecycle/PolicyStepsRegistry.java index 3414a8e21ce9c..502a9a0783e7e 100644 --- a/x-pack/plugin/index-lifecycle/src/main/java/org/elasticsearch/xpack/indexlifecycle/PolicyStepsRegistry.java +++ b/x-pack/plugin/index-lifecycle/src/main/java/org/elasticsearch/xpack/indexlifecycle/PolicyStepsRegistry.java @@ -59,6 +59,8 @@ Map> getStepMap() { @SuppressWarnings({ "unchecked", "rawtypes" }) public void update(ClusterState currentState, Client client, LongSupplier nowSupplier) { IndexLifecycleMetadata meta = currentState.metaData().custom(IndexLifecycleMetadata.TYPE); + assert meta != null : "IndexLifecycleMetadata cannot be null when updating the policy steps registry"; + Diff> diff = DiffableUtils.diff(lifecyclePolicyMap, meta.getPolicyMetadatas(), DiffableUtils.getStringKeySerializer()); DiffableUtils.MapDiff> mapDiff = (DiffableUtils.MapDiff) diff; diff --git a/x-pack/plugin/index-lifecycle/src/main/java/org/elasticsearch/xpack/indexlifecycle/action/TransportDeleteLifcycleAction.java b/x-pack/plugin/index-lifecycle/src/main/java/org/elasticsearch/xpack/indexlifecycle/action/TransportDeleteLifecycleAction.java similarity index 88% rename from x-pack/plugin/index-lifecycle/src/main/java/org/elasticsearch/xpack/indexlifecycle/action/TransportDeleteLifcycleAction.java rename to x-pack/plugin/index-lifecycle/src/main/java/org/elasticsearch/xpack/indexlifecycle/action/TransportDeleteLifecycleAction.java index e1fa80107d88c..2defb5ff24718 100644 --- a/x-pack/plugin/index-lifecycle/src/main/java/org/elasticsearch/xpack/indexlifecycle/action/TransportDeleteLifcycleAction.java +++ b/x-pack/plugin/index-lifecycle/src/main/java/org/elasticsearch/xpack/indexlifecycle/action/TransportDeleteLifecycleAction.java @@ -32,11 +32,12 @@ import java.util.SortedMap; import java.util.TreeMap; -public class TransportDeleteLifcycleAction extends TransportMasterNodeAction { +public class TransportDeleteLifecycleAction extends TransportMasterNodeAction { @Inject - public TransportDeleteLifcycleAction(Settings settings, TransportService transportService, ClusterService clusterService, - ThreadPool threadPool, ActionFilters actionFilters, IndexNameExpressionResolver indexNameExpressionResolver) { + public TransportDeleteLifecycleAction(Settings settings, TransportService transportService, ClusterService clusterService, + ThreadPool threadPool, ActionFilters actionFilters, + IndexNameExpressionResolver indexNameExpressionResolver) { super(settings, DeleteLifecycleAction.NAME, transportService, clusterService, threadPool, actionFilters, indexNameExpressionResolver, Request::new); } @@ -74,7 +75,8 @@ public ClusterState execute(ClusterState currentState) { } ClusterState.Builder newState = ClusterState.builder(currentState); IndexLifecycleMetadata currentMetadata = currentState.metaData().custom(IndexLifecycleMetadata.TYPE); - if (currentMetadata.getPolicyMetadatas().containsKey(request.getPolicyName()) == false) { + if (currentMetadata == null + || currentMetadata.getPolicyMetadatas().containsKey(request.getPolicyName()) == false) { throw new ResourceNotFoundException("Lifecycle policy not found: {}", request.getPolicyName()); } SortedMap newPolicies = new TreeMap<>(currentMetadata.getPolicyMetadatas()); @@ -91,4 +93,4 @@ public ClusterState execute(ClusterState currentState) { protected ClusterBlockException checkBlock(Request request, ClusterState state) { return state.blocks().globalBlockedException(ClusterBlockLevel.METADATA_WRITE); } -} \ No newline at end of file +} diff --git a/x-pack/plugin/index-lifecycle/src/main/java/org/elasticsearch/xpack/indexlifecycle/action/TransportPutLifecycleAction.java b/x-pack/plugin/index-lifecycle/src/main/java/org/elasticsearch/xpack/indexlifecycle/action/TransportPutLifecycleAction.java index aff771ded2171..998252c071377 100644 --- a/x-pack/plugin/index-lifecycle/src/main/java/org/elasticsearch/xpack/indexlifecycle/action/TransportPutLifecycleAction.java +++ b/x-pack/plugin/index-lifecycle/src/main/java/org/elasticsearch/xpack/indexlifecycle/action/TransportPutLifecycleAction.java @@ -32,6 +32,10 @@ import java.util.TreeMap; import java.util.stream.Collectors; +/** + * This class is responsible for bootstrapping {@link IndexLifecycleMetadata} into the cluster-state, as well + * as adding the desired new policy to be inserted. + */ public class TransportPutLifecycleAction extends TransportMasterNodeAction { @Inject @@ -64,6 +68,9 @@ protected Response newResponse(boolean acknowledged) { public ClusterState execute(ClusterState currentState) throws Exception { ClusterState.Builder newState = ClusterState.builder(currentState); IndexLifecycleMetadata currentMetadata = currentState.metaData().custom(IndexLifecycleMetadata.TYPE); + if (currentMetadata == null) { // first time using index-lifecycle feature, bootstrap metadata + currentMetadata = IndexLifecycleMetadata.EMPTY; + } if (currentMetadata.getPolicyMetadatas().containsKey(request.getPolicy().getName())) { throw new ResourceAlreadyExistsException("Lifecycle policy already exists: {}", request.getPolicy().getName()); @@ -87,4 +94,4 @@ public ClusterState execute(ClusterState currentState) throws Exception { protected ClusterBlockException checkBlock(Request request, ClusterState state) { return state.blocks().globalBlockedException(ClusterBlockLevel.METADATA_WRITE); } -} \ No newline at end of file +} diff --git a/x-pack/plugin/index-lifecycle/src/main/java/org/elasticsearch/xpack/indexlifecycle/action/TransportSetPolicyForIndexAction.java b/x-pack/plugin/index-lifecycle/src/main/java/org/elasticsearch/xpack/indexlifecycle/action/TransportSetPolicyForIndexAction.java index f348fd798b546..cf5d976e6f704 100644 --- a/x-pack/plugin/index-lifecycle/src/main/java/org/elasticsearch/xpack/indexlifecycle/action/TransportSetPolicyForIndexAction.java +++ b/x-pack/plugin/index-lifecycle/src/main/java/org/elasticsearch/xpack/indexlifecycle/action/TransportSetPolicyForIndexAction.java @@ -68,6 +68,11 @@ protected void masterOperation(Request request, ClusterState state, ActionListen public ClusterState execute(ClusterState currentState) throws Exception { IndexLifecycleMetadata ilmMetadata = (IndexLifecycleMetadata) currentState.metaData() .custom(IndexLifecycleMetadata.TYPE); + + if (ilmMetadata == null) { + throw new ResourceNotFoundException("Policy does not exist [{}]", newPolicyName); + } + LifecyclePolicy newPolicy = ilmMetadata.getPolicies().get(newPolicyName); if (newPolicy == null) { diff --git a/x-pack/plugin/index-lifecycle/src/test/java/org/elasticsearch/xpack/indexlifecycle/IndexLifecycleServiceTests.java b/x-pack/plugin/index-lifecycle/src/test/java/org/elasticsearch/xpack/indexlifecycle/IndexLifecycleServiceTests.java index f57b93fe30ebe..e1d4795cefb5b 100644 --- a/x-pack/plugin/index-lifecycle/src/test/java/org/elasticsearch/xpack/indexlifecycle/IndexLifecycleServiceTests.java +++ b/x-pack/plugin/index-lifecycle/src/test/java/org/elasticsearch/xpack/indexlifecycle/IndexLifecycleServiceTests.java @@ -5,7 +5,6 @@ */ package org.elasticsearch.xpack.indexlifecycle; -import org.apache.lucene.util.SetOnce; import org.elasticsearch.Version; import org.elasticsearch.client.AdminClient; import org.elasticsearch.client.Client; @@ -22,7 +21,6 @@ import org.elasticsearch.common.transport.TransportAddress; import org.elasticsearch.common.unit.TimeValue; import org.elasticsearch.test.ESTestCase; -import org.elasticsearch.threadpool.ThreadPool; import org.elasticsearch.xpack.core.indexlifecycle.IndexLifecycleMetadata; import org.elasticsearch.xpack.core.indexlifecycle.LifecycleSettings; import org.elasticsearch.xpack.core.scheduler.SchedulerEngine; @@ -34,7 +32,6 @@ import java.time.Clock; import java.time.Instant; import java.time.ZoneId; -import java.util.Collections; import java.util.concurrent.ExecutorService; import static org.elasticsearch.node.Node.NODE_MASTER_SETTING; @@ -42,7 +39,6 @@ import static org.mockito.Matchers.any; import static org.mockito.Matchers.anyString; import static org.mockito.Mockito.doAnswer; -import static org.mockito.Mockito.doThrow; import static org.mockito.Mockito.mock; import static org.mockito.Mockito.never; import static org.mockito.Mockito.times; @@ -61,7 +57,6 @@ public class IndexLifecycleServiceTests extends ESTestCase { @Before public void prepareServices() { nodeId = randomAlphaOfLength(10); - ThreadPool threadPool = mock(ThreadPool.class); ExecutorService executorService = mock(ExecutorService.class); clusterService = mock(ClusterService.class); masterNode = DiscoveryNode.createLocal(settings(Version.CURRENT) @@ -71,7 +66,6 @@ public void prepareServices() { Clock clock = Clock.fixed(Instant.ofEpochMilli(now), ZoneId.of(randomFrom(ZoneId.getAvailableZoneIds()))); doAnswer(invocationOnMock -> null).when(clusterService).addListener(any()); - when(threadPool.executor(ThreadPool.Names.GENERIC)).thenReturn(executorService); doAnswer(invocationOnMock -> { Runnable runnable = (Runnable) invocationOnMock.getArguments()[0]; runnable.run(); @@ -84,20 +78,23 @@ public void prepareServices() { when(client.admin()).thenReturn(adminClient); when(adminClient.indices()).thenReturn(indicesClient); - indexLifecycleService = new IndexLifecycleService(Settings.EMPTY, client, clusterService, clock, - threadPool, () -> now); + indexLifecycleService = new IndexLifecycleService(Settings.EMPTY, client, clusterService, clock, () -> now); Mockito.verify(clusterService).addListener(indexLifecycleService); Mockito.verify(clusterService).addStateApplier(indexLifecycleService); } - public void testOnlyChangesStateOnMaster() throws Exception { - MetaData metaData = MetaData.builder() + public void testOnlyChangesStateOnMasterAndMetadataExists() { + boolean isMaster = randomBoolean(); + String localNodeId = isMaster ? nodeId : nodeId + "not_master"; + MetaData.Builder metaData = MetaData.builder() .persistentSettings(settings(Version.CURRENT) - .put(LifecycleSettings.LIFECYCLE_POLL_INTERVAL_SETTING.getKey(), TimeValue.timeValueSeconds(3)).build()) - .build(); + .put(LifecycleSettings.LIFECYCLE_POLL_INTERVAL_SETTING.getKey(), TimeValue.timeValueSeconds(3)).build()); + if (isMaster == false) { + metaData.putCustom(IndexLifecycleMetadata.TYPE, IndexLifecycleMetadata.EMPTY); + } ClusterState state = ClusterState.builder(ClusterName.DEFAULT) .metaData(metaData) - .nodes(DiscoveryNodes.builder().localNodeId(nodeId + "not").masterNodeId(nodeId).add(masterNode).build()) + .nodes(DiscoveryNodes.builder().localNodeId(localNodeId).masterNodeId(nodeId).add(masterNode).build()) .build(); ClusterChangedEvent event = new ClusterChangedEvent("_source", state, state); @@ -113,7 +110,7 @@ public void testElectUnElectMaster() throws Exception { MetaData metaData = MetaData.builder() .persistentSettings(settings(Version.CURRENT) .put(LifecycleSettings.LIFECYCLE_POLL_INTERVAL_SETTING.getKey(), TimeValue.timeValueSeconds(3)).build()) - .putCustom(IndexLifecycleMetadata.TYPE, new IndexLifecycleMetadata(Collections.emptySortedMap())) + .putCustom(IndexLifecycleMetadata.TYPE, IndexLifecycleMetadata.EMPTY) .build(); // First check that when the node has never been master the scheduler @@ -131,7 +128,7 @@ public void testElectUnElectMaster() throws Exception { Mockito.verifyNoMoreInteractions(clusterService); assertNull(indexLifecycleService.getScheduler()); assertNull(indexLifecycleService.getScheduledJob()); - + Mockito.reset(clusterService); state = ClusterState.builder(ClusterName.DEFAULT) .metaData(metaData) @@ -147,7 +144,7 @@ public void testElectUnElectMaster() throws Exception { assertNotNull(indexLifecycleService.getScheduler()); assertEquals(1, indexLifecycleService.getScheduler().jobCount()); assertNotNull(indexLifecycleService.getScheduledJob()); - + Mockito.reset(clusterService); state = ClusterState.builder(ClusterName.DEFAULT) .metaData(metaData) @@ -162,7 +159,7 @@ public void testElectUnElectMaster() throws Exception { assertNotNull(indexLifecycleService.getScheduler()); assertEquals(0, indexLifecycleService.getScheduler().jobCount()); assertNull(indexLifecycleService.getScheduledJob()); - + Mockito.reset(clusterService); state = ClusterState.builder(ClusterName.DEFAULT) .metaData(metaData) @@ -179,36 +176,6 @@ public void testElectUnElectMaster() throws Exception { assertNotNull(indexLifecycleService.getScheduledJob()); } - public void testServiceSetupOnFirstClusterChange() { - TimeValue pollInterval = TimeValue.timeValueSeconds(randomIntBetween(1, 59)); - MetaData metaData = MetaData.builder().persistentSettings(settings(Version.CURRENT) - .put(LifecycleSettings.LIFECYCLE_POLL_INTERVAL_SETTING.getKey(), pollInterval).build()) - .build(); - ClusterState state = ClusterState.builder(ClusterName.DEFAULT) - .metaData(metaData) - .nodes(DiscoveryNodes.builder().localNodeId(nodeId).masterNodeId(nodeId).add(masterNode).build()) - .build(); - ClusterChangedEvent event = new ClusterChangedEvent("_source", state, state); - final SetOnce installedEvent = new SetOnce<>(); - doAnswer(invocationOnMock -> { - ClusterStateUpdateTask updateTask = (ClusterStateUpdateTask) invocationOnMock.getArguments()[1]; - ClusterState newState = updateTask.execute(state); - IndexLifecycleMetadata indexLifecycleMetadata = newState.metaData().custom(IndexLifecycleMetadata.TYPE); - assertThat(indexLifecycleMetadata.getPolicyMetadatas(), equalTo(Collections.emptySortedMap())); - installedEvent.set(new ClusterChangedEvent(event.source(), newState, state)); - return null; - }).when(clusterService).submitStateUpdateTask(anyString(), any(ClusterStateUpdateTask.class)); - - indexLifecycleService.applyClusterState(event); - indexLifecycleService.clusterChanged(event); - - verify(clusterService, times(1)).addListener(any()); - verify(clusterService, times(1)).addStateApplier(any()); - verify(clusterService).submitStateUpdateTask(anyString(), any(ClusterStateUpdateTask.class)); - Mockito.verifyNoMoreInteractions(clusterService); - assertNull(indexLifecycleService.getScheduler()); - } - @After public void cleanup() throws IOException { indexLifecycleService.close(); @@ -217,7 +184,7 @@ public void cleanup() throws IOException { public void testSchedulerInitializationAndUpdate() { TimeValue pollInterval = TimeValue.timeValueSeconds(randomIntBetween(1, 59)); MetaData metaData = MetaData.builder() - .putCustom(IndexLifecycleMetadata.TYPE, new IndexLifecycleMetadata(Collections.emptySortedMap())) + .putCustom(IndexLifecycleMetadata.TYPE, IndexLifecycleMetadata.EMPTY) .persistentSettings(settings(Version.CURRENT).build()) .build(); MetaData updatedPollMetaData = MetaData.builder(metaData).persistentSettings(settings(Version.CURRENT) @@ -254,29 +221,6 @@ public void testSchedulerInitializationAndUpdate() { Mockito.verifyNoMoreInteractions(clusterService); } - public void testInstallMetadataFail() { - TimeValue pollInterval = TimeValue.timeValueSeconds(randomIntBetween(1, 59)); - MetaData metaData = MetaData.builder() .persistentSettings(settings(Version.CURRENT) - .put(LifecycleSettings.LIFECYCLE_POLL_INTERVAL_SETTING.getKey(), pollInterval).build()) - .build(); - ClusterState state = ClusterState.builder(ClusterName.DEFAULT) - .metaData(metaData) - .nodes(DiscoveryNodes.builder().localNodeId(nodeId).masterNodeId(nodeId).add(masterNode).build()) - .build(); - ClusterChangedEvent event = new ClusterChangedEvent("_source", state, state); - - doThrow(new RuntimeException("error")).when(clusterService).submitStateUpdateTask(anyString(), any(ClusterStateUpdateTask.class)); - - Exception exception = expectThrows(RuntimeException.class, () -> indexLifecycleService.clusterChanged(event)); - assertThat(exception.getMessage(), equalTo("error")); - - verify(clusterService, times(1)).addListener(any()); - verify(clusterService, times(1)).addStateApplier(any()); - verify(clusterService).submitStateUpdateTask(anyString(), any(ClusterStateUpdateTask.class)); - Mockito.verifyNoMoreInteractions(clusterService); - assertNull(indexLifecycleService.getScheduler()); - } - // /** // * Checks that a new index does the following successfully: // * diff --git a/x-pack/plugin/src/test/resources/rest-api-spec/test/index_lifecycle/10_basic.yml b/x-pack/plugin/src/test/resources/rest-api-spec/test/index_lifecycle/10_basic.yml index 22e9620fcb4f5..61435af9f5f82 100644 --- a/x-pack/plugin/src/test/resources/rest-api-spec/test/index_lifecycle/10_basic.yml +++ b/x-pack/plugin/src/test/resources/rest-api-spec/test/index_lifecycle/10_basic.yml @@ -6,6 +6,16 @@ setup: --- "Test Basic Policy CRUD": + - do: + catch: missing + xpack.index_lifecycle.get_lifecycle: + lifecycle: "my_timeseries_lifecycle" + + - do: + catch: missing + xpack.index_lifecycle.delete_lifecycle: + lifecycle: "my_timeseries_lifecycle" + - do: acknowlege: true xpack.index_lifecycle.put_lifecycle: