From 7cb2d059ae7ecbf59eff037b40039c4f844a945b Mon Sep 17 00:00:00 2001 From: Armin Braun Date: Fri, 8 Oct 2021 14:59:18 +0200 Subject: [PATCH] Store DataTier Preference directly on IndexMetadata (#78668) The data tier preference is very expensive to parse out of the setting string repeatedly for large number of indices when using it in the data tier allocation decider. => as done with other index settings relevant to allocation, this commit moves the data tier preference to a field in `IndexMetadata`. The required moving the `DataTier` class itself to `server`. In a follow-up we can look into making the setting a list setting to remove the duplication around turning the string value into a list in various places. --- .../cluster/metadata/IndexMetadata.java | 31 ++++- .../cluster/routing/allocation}/DataTier.java | 100 +++++++++++++-- .../SearchableSnapshotsSettings.java | 23 ++++ .../cluster/metadata/IndexMetadataTests.java | 23 ++++ ...bstractFrozenAutoscalingIntegTestCase.java | 2 +- .../storage/ReactiveStorageIT.java | 9 +- .../ReactiveStorageDeciderService.java | 10 +- .../xpack/autoscaling/util/FrozenUtils.java | 13 +- .../FrozenShardsDeciderServiceTests.java | 2 +- .../FrozenStorageDeciderServiceTests.java | 2 +- .../ReactiveStorageDeciderDecisionTests.java | 6 +- .../ReactiveStorageDeciderServiceTests.java | 6 +- .../autoscaling/util/FrozenUtilsTests.java | 9 +- ....java => DataTierAllocationDeciderIT.java} | 38 +++--- .../allocation/DataTierAllocationDecider.java | 117 ++++-------------- .../mapper/DataTierFieldMapper.java | 4 +- .../core/DataTiersUsageTransportAction.java | 4 +- .../elasticsearch/xpack/core/XPackPlugin.java | 3 +- .../core/ilm/DataTierMigrationRoutedStep.java | 16 ++- .../xpack/core/ilm/MigrateAction.java | 9 +- .../xpack/core/ilm/MountSnapshotStep.java | 5 +- .../xpack/core/ilm/WaitForDataTierStep.java | 4 +- .../MountSearchableSnapshotRequest.java | 2 +- .../SearchableSnapshotsConstants.java | 26 ---- .../IndexMetadataConversionTests.java | 8 +- .../DataTierAllocationDeciderTests.java | 73 ++++++----- .../mapper/DataTierFieldTypeTests.java | 4 +- .../xpack/core/DataTierTests.java | 9 +- .../core/DataTiersFeatureSetUsageTests.java | 1 + .../DataTiersUsageTransportActionTests.java | 6 +- .../ilm/DataTierMigrationRoutedStepTests.java | 4 +- .../xpack/core/ilm/MigrateActionTests.java | 16 +-- .../ilm/SearchableSnapshotActionTests.java | 2 +- .../core/ilm/WaitForDataTierStepTests.java | 2 +- .../SearchableSnapshotsConstantsTests.java | 12 +- .../datastreams/DataTierDataStreamIT.java | 7 +- .../xpack/MigrateToDataTiersIT.java | 12 +- .../actions/SearchableSnapshotActionIT.java | 6 +- .../xpack/ilm/actions/ShrinkActionIT.java | 4 +- .../xpack/ilm/DataTiersMigrationsTests.java | 4 +- ...adataMigrateToDataTiersRoutingService.java | 4 +- ...MigrateToDataTiersRoutingServiceTests.java | 2 +- .../FrozenSearchableSnapshotsIntegTests.java | 11 +- .../SearchableSnapshotsIntegTests.java | 7 +- .../SearchableSnapshotsResizeIntegTests.java | 5 +- .../SearchableSnapshotDataTierIntegTests.java | 14 +-- ...ableSnapshotsBlobStoreCacheIntegTests.java | 4 +- ...tiallyCachedShardAllocationIntegTests.java | 2 +- .../SearchableSnapshots.java | 8 +- ...archableSnapshotsUsageTransportAction.java | 3 +- ...ransportMountSearchableSnapshotAction.java | 12 +- ...rtSearchableSnapshotCacheStoresAction.java | 2 +- .../SearchableSnapshotAllocator.java | 2 +- .../cache/shared/FrozenCacheService.java | 2 +- .../store/SearchableSnapshotDirectory.java | 2 +- .../SearchableSnapshotAllocatorTests.java | 2 +- ...SearchableSnapshotDirectoryStatsTests.java | 14 +-- .../store/input/FrozenIndexInputTests.java | 4 +- ...bleSnapshotIndexMetadataUpgraderTests.java | 3 +- 59 files changed, 390 insertions(+), 347 deletions(-) rename {x-pack/plugin/core/src/main/java/org/elasticsearch/xpack/core => server/src/main/java/org/elasticsearch/cluster/routing/allocation}/DataTier.java (64%) rename x-pack/plugin/core/src/internalClusterTest/java/org/elasticsearch/xpack/cluster/routing/allocation/{DataTierIT.java => DataTierAllocationDeciderIT.java} (89%) diff --git a/server/src/main/java/org/elasticsearch/cluster/metadata/IndexMetadata.java b/server/src/main/java/org/elasticsearch/cluster/metadata/IndexMetadata.java index 022771ccd9ca1..b39ab956f4610 100644 --- a/server/src/main/java/org/elasticsearch/cluster/metadata/IndexMetadata.java +++ b/server/src/main/java/org/elasticsearch/cluster/metadata/IndexMetadata.java @@ -24,6 +24,7 @@ import org.elasticsearch.cluster.block.ClusterBlockLevel; import org.elasticsearch.cluster.node.DiscoveryNodeFilters; import org.elasticsearch.cluster.routing.IndexRouting; +import org.elasticsearch.cluster.routing.allocation.DataTier; import org.elasticsearch.cluster.routing.allocation.IndexMetadataUpdater; import org.elasticsearch.cluster.routing.allocation.decider.DiskThresholdDecider; import org.elasticsearch.common.collect.ImmutableOpenIntMap; @@ -400,6 +401,9 @@ public static APIBlock readFrom(StreamInput input) throws IOException { private final boolean ignoreDiskWatermarks; + @Nullable // since we store null if DataTier.TIER_PREFERENCE_SETTING failed validation + private final List tierPreference; + private IndexMetadata( final Index index, final long version, @@ -429,7 +433,8 @@ private IndexMetadata( final IndexLongFieldRange timestampRange, final int priority, final long creationDate, - final boolean ignoreDiskWatermarks + final boolean ignoreDiskWatermarks, + @Nullable final List tierPreference ) { this.index = index; @@ -468,6 +473,7 @@ private IndexMetadata( this.priority = priority; this.creationDate = creationDate; this.ignoreDiskWatermarks = ignoreDiskWatermarks; + this.tierPreference = tierPreference; assert numberOfShards * routingFactor == routingNumShards : routingNumShards + " must be a multiple of " + numberOfShards; } @@ -574,6 +580,15 @@ public ImmutableOpenMap getAliases() { return this.aliases; } + public List getTierPreference() { + if (tierPreference == null) { + final List parsed = DataTier.parseTierList(DataTier.TIER_PREFERENCE_SETTING.get(settings)); + assert false : "the setting parsing should always throw if we didn't store a tier preference when building this instance"; + return parsed; + } + return tierPreference; + } + /** * Return the concrete mapping for this index or {@code null} if this index has no mappings at all. */ @@ -1311,6 +1326,17 @@ public IndexMetadata build() { final String uuid = settings.get(SETTING_INDEX_UUID, INDEX_UUID_NA_VALUE); + List tierPreference; + try { + tierPreference = DataTier.parseTierList(DataTier.TIER_PREFERENCE_SETTING.get(settings)); + } catch (Exception e) { + assert e instanceof IllegalArgumentException : e; + // BwC hack: the setting failed validation but it will be fixed in + // #IndexMetadataVerifier#convertSharedCacheTierPreference(IndexMetadata)} later so we just store a null + // to be able to build a temporary instance + tierPreference = null; + } + return new IndexMetadata( new Index(index, uuid), version, @@ -1340,7 +1366,8 @@ public IndexMetadata build() { timestampRange, IndexMetadata.INDEX_PRIORITY_SETTING.get(settings), settings.getAsLong(SETTING_CREATION_DATE, -1L), - DiskThresholdDecider.SETTING_IGNORE_DISK_WATERMARKS.get(settings) + DiskThresholdDecider.SETTING_IGNORE_DISK_WATERMARKS.get(settings), + tierPreference ); } diff --git a/x-pack/plugin/core/src/main/java/org/elasticsearch/xpack/core/DataTier.java b/server/src/main/java/org/elasticsearch/cluster/routing/allocation/DataTier.java similarity index 64% rename from x-pack/plugin/core/src/main/java/org/elasticsearch/xpack/core/DataTier.java rename to server/src/main/java/org/elasticsearch/cluster/routing/allocation/DataTier.java index 25ab37d16b51c..b605f6580a008 100644 --- a/x-pack/plugin/core/src/main/java/org/elasticsearch/xpack/core/DataTier.java +++ b/server/src/main/java/org/elasticsearch/cluster/routing/allocation/DataTier.java @@ -1,23 +1,31 @@ /* * Copyright Elasticsearch B.V. and/or licensed to Elasticsearch B.V. under one * or more contributor license agreements. Licensed under the Elastic License - * 2.0; you may not use this file except in compliance with the Elastic License - * 2.0. + * 2.0 and the Server Side Public License, v 1; you may not use this file except + * in compliance with, at your election, the Elastic License 2.0 or the Server + * Side Public License, v 1. */ -package org.elasticsearch.xpack.core; +package org.elasticsearch.cluster.routing.allocation; import org.apache.logging.log4j.LogManager; import org.apache.logging.log4j.Logger; import org.elasticsearch.cluster.metadata.IndexMetadata; import org.elasticsearch.cluster.node.DiscoveryNode; import org.elasticsearch.cluster.node.DiscoveryNodeRole; +import org.elasticsearch.common.Strings; +import org.elasticsearch.common.settings.Setting; import org.elasticsearch.common.settings.Settings; +import org.elasticsearch.index.IndexModule; import org.elasticsearch.index.shard.IndexSettingProvider; -import org.elasticsearch.xpack.cluster.routing.allocation.DataTierAllocationDecider; +import org.elasticsearch.snapshots.SearchableSnapshotsSettings; +import java.util.Collection; +import java.util.Iterator; import java.util.List; +import java.util.Map; import java.util.Set; +import java.util.function.Function; import java.util.stream.Collectors; /** @@ -25,9 +33,6 @@ * "hot", "warm", and "cold" tiers as node roles. In contains the * roles themselves as well as helpers for validation and determining if a node * has a tier configured. - * - * Related: - * {@link org.elasticsearch.xpack.cluster.routing.allocation.DataTierAllocationDecider} */ public class DataTier { @@ -39,6 +44,17 @@ public class DataTier { public static final Set ALL_DATA_TIERS = Set.of(DATA_CONTENT, DATA_HOT, DATA_WARM, DATA_COLD, DATA_FROZEN); + public static final String TIER_PREFERENCE = "index.routing.allocation.include._tier_preference"; + + public static final Setting TIER_PREFERENCE_SETTING = new Setting<>( + new Setting.SimpleKey(TIER_PREFERENCE), + DataTierSettingValidator::getDefaultTierPreference, + Function.identity(), + new DataTierSettingValidator(), + Setting.Property.Dynamic, + Setting.Property.IndexScope + ); + static { for (String tier : ALL_DATA_TIERS) { assert tier.equals(DATA_FROZEN) || tier.contains(DATA_FROZEN) == false @@ -59,8 +75,8 @@ public static boolean validTierName(String tierName) { /** * Based on the provided target tier it will return a comma separated list of preferred tiers. - * ie. if `data_cold` is the target tier, it will return `data_cold,data_warm,data_hot` - * This is usually used in conjunction with {@link DataTierAllocationDecider#TIER_PREFERENCE_SETTING} + * ie. if `data_cold` is the target tier, it will return `data_cold,data_warm,data_hot`. + * This is usually used in conjunction with {@link #TIER_PREFERENCE_SETTING}. */ public static String getPreferredTiersConfiguration(String targetTier) { int indexOfTargetTier = ORDERED_FROZEN_TO_HOT_TIERS.indexOf(targetTier); @@ -115,6 +131,15 @@ public static boolean isFrozenNode(final Set roles) { return roles.contains(DiscoveryNodeRole.DATA_FROZEN_NODE_ROLE) || roles.contains(DiscoveryNodeRole.DATA_ROLE); } + public static List parseTierList(String tiers) { + if (Strings.hasText(tiers) == false) { + // avoid parsing overhead in the null/empty string case + return List.of(); + } else { + return List.of(tiers.split(",")); + } + } + /** * This setting provider injects the setting allocating all newly created indices with * {@code index.routing.allocation.include._tier_preference: "data_hot"} for a data stream index @@ -128,9 +153,9 @@ public static class DefaultHotAllocationSettingProvider implements IndexSettingP @Override public Settings getAdditionalIndexSettings(String indexName, boolean isDataStreamIndex, Settings indexSettings) { Set settings = indexSettings.keySet(); - if (settings.contains(DataTierAllocationDecider.TIER_PREFERENCE)) { + if (settings.contains(TIER_PREFERENCE)) { // just a marker -- this null value will be removed or overridden by the template/request settings - return Settings.builder().putNull(DataTierAllocationDecider.TIER_PREFERENCE).build(); + return Settings.builder().putNull(TIER_PREFERENCE).build(); } else if (settings.stream().anyMatch(s -> s.startsWith(IndexMetadata.INDEX_ROUTING_REQUIRE_GROUP_PREFIX + ".")) || settings.stream().anyMatch(s -> s.startsWith(IndexMetadata.INDEX_ROUTING_EXCLUDE_GROUP_PREFIX + ".")) || settings.stream().anyMatch(s -> s.startsWith(IndexMetadata.INDEX_ROUTING_INCLUDE_GROUP_PREFIX + "."))) { @@ -142,11 +167,60 @@ public Settings getAdditionalIndexSettings(String indexName, boolean isDataStrea // tier if the index is part of a data stream, the "content" // tier if it is not. if (isDataStreamIndex) { - return Settings.builder().put(DataTierAllocationDecider.TIER_PREFERENCE, DATA_HOT).build(); + return Settings.builder().put(TIER_PREFERENCE, DATA_HOT).build(); + } else { + return Settings.builder().put(TIER_PREFERENCE, DATA_CONTENT).build(); + } + } + } + } + + private static final class DataTierSettingValidator implements Setting.Validator { + + private static final Collection> dependencies = List.of( + IndexModule.INDEX_STORE_TYPE_SETTING, + SearchableSnapshotsSettings.SNAPSHOT_PARTIAL_SETTING + ); + + public static String getDefaultTierPreference(Settings settings) { + if (SearchableSnapshotsSettings.isPartialSearchableSnapshotIndex(settings)) { + return DATA_FROZEN; + } else { + return ""; + } + } + + @Override + public void validate(String value) { + if (Strings.hasText(value)) { + for (String s : parseTierList(value)) { + if (validTierName(s) == false) { + throw new IllegalArgumentException( + "invalid tier names found in [" + value + "] allowed values are " + ALL_DATA_TIERS); + } + } + } + } + + @Override + public void validate(String value, Map, Object> settings, boolean exists) { + if (exists && value != null) { + if (SearchableSnapshotsSettings.isPartialSearchableSnapshotIndex(settings)) { + if (value.equals(DATA_FROZEN) == false) { + throw new IllegalArgumentException("only the [" + DATA_FROZEN + + "] tier preference may be used for partial searchable snapshots (got: [" + value + "])"); + } } else { - return Settings.builder().put(DataTierAllocationDecider.TIER_PREFERENCE, DATA_CONTENT).build(); + if (value.contains(DATA_FROZEN)) { + throw new IllegalArgumentException("[" + DATA_FROZEN + "] tier can only be used for partial searchable snapshots"); + } } } } + + @Override + public Iterator> settings() { + return dependencies.iterator(); + } } } diff --git a/server/src/main/java/org/elasticsearch/snapshots/SearchableSnapshotsSettings.java b/server/src/main/java/org/elasticsearch/snapshots/SearchableSnapshotsSettings.java index 46f8ac4816341..6dd99fdd7145b 100644 --- a/server/src/main/java/org/elasticsearch/snapshots/SearchableSnapshotsSettings.java +++ b/server/src/main/java/org/elasticsearch/snapshots/SearchableSnapshotsSettings.java @@ -8,14 +8,25 @@ package org.elasticsearch.snapshots; +import org.elasticsearch.common.settings.Setting; import org.elasticsearch.common.settings.Settings; +import java.util.Map; + import static org.elasticsearch.index.IndexModule.INDEX_STORE_TYPE_SETTING; public final class SearchableSnapshotsSettings { public static final String SEARCHABLE_SNAPSHOT_STORE_TYPE = "snapshot"; public static final String SEARCHABLE_SNAPSHOT_PARTIAL_SETTING_KEY = "index.store.snapshot.partial"; + + public static final Setting SNAPSHOT_PARTIAL_SETTING = Setting.boolSetting( + SEARCHABLE_SNAPSHOT_PARTIAL_SETTING_KEY, + false, + Setting.Property.IndexScope, + Setting.Property.PrivateIndex, + Setting.Property.NotCopyableOnResize + ); public static final String SEARCHABLE_SNAPSHOTS_REPOSITORY_NAME_SETTING_KEY = "index.store.snapshot.repository_name"; public static final String SEARCHABLE_SNAPSHOTS_REPOSITORY_UUID_SETTING_KEY = "index.store.snapshot.repository_uuid"; public static final String SEARCHABLE_SNAPSHOTS_SNAPSHOT_NAME_SETTING_KEY = "index.store.snapshot.snapshot_name"; @@ -31,4 +42,16 @@ public static boolean isSearchableSnapshotStore(Settings indexSettings) { public static boolean isPartialSearchableSnapshotIndex(Settings indexSettings) { return isSearchableSnapshotStore(indexSettings) && indexSettings.getAsBoolean(SEARCHABLE_SNAPSHOT_PARTIAL_SETTING_KEY, false); } + + /** + * Based on a map from setting to value, do the settings represent a partial searchable snapshot index? + * + * Both index.store.type and index.store.snapshot.partial must be supplied. + */ + public static boolean isPartialSearchableSnapshotIndex(Map, Object> indexSettings) { + assert indexSettings.containsKey(INDEX_STORE_TYPE_SETTING) : "must include store type in map"; + assert indexSettings.get(SNAPSHOT_PARTIAL_SETTING) != null : "partial setting must be non-null in map (has default value)"; + return SEARCHABLE_SNAPSHOT_STORE_TYPE.equals(indexSettings.get(INDEX_STORE_TYPE_SETTING)) + && (boolean) indexSettings.get(SNAPSHOT_PARTIAL_SETTING); + } } diff --git a/server/src/test/java/org/elasticsearch/cluster/metadata/IndexMetadataTests.java b/server/src/test/java/org/elasticsearch/cluster/metadata/IndexMetadataTests.java index 10c679224b875..b50d1a96930f7 100644 --- a/server/src/test/java/org/elasticsearch/cluster/metadata/IndexMetadataTests.java +++ b/server/src/test/java/org/elasticsearch/cluster/metadata/IndexMetadataTests.java @@ -14,6 +14,7 @@ import org.elasticsearch.action.admin.indices.rollover.MaxPrimaryShardSizeCondition; import org.elasticsearch.action.admin.indices.rollover.MaxSizeCondition; import org.elasticsearch.action.admin.indices.rollover.RolloverInfo; +import org.elasticsearch.cluster.routing.allocation.DataTier; import org.elasticsearch.common.Strings; import org.elasticsearch.common.bytes.BytesReference; import org.elasticsearch.common.collect.ImmutableOpenMap; @@ -388,4 +389,26 @@ public void testIsHidden() { assertTrue(indexMetadata.isHidden()); // preserved if settings unchanged } + public void testGetTierPreference() { + final Settings indexSettings = indexSettingsWithDataTier("data_warm,data_cold"); + final IndexMetadata indexMetadata = IndexMetadata.builder("myindex").settings(indexSettings).build(); + assertThat(indexMetadata.getTierPreference(), is(DataTier.parseTierList(DataTier.TIER_PREFERENCE_SETTING.get(indexSettings)))); + assertThat(indexMetadata.getTierPreference(), is(List.of(DataTier.DATA_WARM, DataTier.DATA_COLD))); + + } + + public void testBuildsWithBrokenTierPreference() { + final Settings indexSettings = indexSettingsWithDataTier("broken_tier"); + final IndexMetadata indexMetadata = IndexMetadata.builder("myindex").settings(indexSettings).build(); + expectThrows(IllegalArgumentException.class, indexMetadata::getTierPreference); + } + + private static Settings indexSettingsWithDataTier(String dataTier) { + return Settings.builder() + .put(IndexMetadata.SETTING_VERSION_CREATED, Version.CURRENT) + .put(IndexMetadata.SETTING_NUMBER_OF_SHARDS, 1) + .put(IndexMetadata.SETTING_NUMBER_OF_REPLICAS, 0) + .put(DataTier.TIER_PREFERENCE, dataTier) + .build(); + } } diff --git a/x-pack/plugin/autoscaling/src/internalClusterTest/java/org/elasticsearch/xpack/autoscaling/AbstractFrozenAutoscalingIntegTestCase.java b/x-pack/plugin/autoscaling/src/internalClusterTest/java/org/elasticsearch/xpack/autoscaling/AbstractFrozenAutoscalingIntegTestCase.java index f68d6370a999b..d814209abafda 100644 --- a/x-pack/plugin/autoscaling/src/internalClusterTest/java/org/elasticsearch/xpack/autoscaling/AbstractFrozenAutoscalingIntegTestCase.java +++ b/x-pack/plugin/autoscaling/src/internalClusterTest/java/org/elasticsearch/xpack/autoscaling/AbstractFrozenAutoscalingIntegTestCase.java @@ -10,6 +10,7 @@ import org.elasticsearch.action.admin.cluster.snapshots.restore.RestoreSnapshotResponse; import org.elasticsearch.cluster.node.DiscoveryNode; import org.elasticsearch.cluster.node.DiscoveryNodeRole; +import org.elasticsearch.cluster.routing.allocation.DataTier; import org.elasticsearch.common.Strings; import org.elasticsearch.common.settings.Settings; import org.elasticsearch.common.unit.ByteSizeUnit; @@ -22,7 +23,6 @@ import org.elasticsearch.xpack.autoscaling.action.PutAutoscalingPolicyAction; import org.elasticsearch.xpack.autoscaling.capacity.AutoscalingCapacity; import org.elasticsearch.xpack.autoscaling.shards.LocalStateAutoscalingAndSearchableSnapshots; -import org.elasticsearch.xpack.core.DataTier; import org.elasticsearch.xpack.core.searchablesnapshots.MountSearchableSnapshotAction; import org.elasticsearch.xpack.core.searchablesnapshots.MountSearchableSnapshotRequest; import org.elasticsearch.xpack.searchablesnapshots.cache.shared.FrozenCacheService; diff --git a/x-pack/plugin/autoscaling/src/internalClusterTest/java/org/elasticsearch/xpack/autoscaling/storage/ReactiveStorageIT.java b/x-pack/plugin/autoscaling/src/internalClusterTest/java/org/elasticsearch/xpack/autoscaling/storage/ReactiveStorageIT.java index 1216e153fc629..a1f7a5bb24841 100644 --- a/x-pack/plugin/autoscaling/src/internalClusterTest/java/org/elasticsearch/xpack/autoscaling/storage/ReactiveStorageIT.java +++ b/x-pack/plugin/autoscaling/src/internalClusterTest/java/org/elasticsearch/xpack/autoscaling/storage/ReactiveStorageIT.java @@ -16,6 +16,7 @@ import org.elasticsearch.cluster.InternalClusterInfoService; import org.elasticsearch.cluster.metadata.IndexMetadata; import org.elasticsearch.cluster.node.DiscoveryNodeRole; +import org.elasticsearch.cluster.routing.allocation.DataTier; import org.elasticsearch.common.settings.ClusterSettings; import org.elasticsearch.common.settings.Settings; import org.elasticsearch.node.Node; @@ -23,8 +24,6 @@ import org.elasticsearch.test.NodeRoles; import org.elasticsearch.xpack.autoscaling.action.GetAutoscalingCapacityAction; import org.elasticsearch.xpack.autoscaling.action.PutAutoscalingPolicyAction; -import org.elasticsearch.xpack.cluster.routing.allocation.DataTierAllocationDecider; -import org.elasticsearch.xpack.core.DataTier; import org.hamcrest.Matchers; import java.util.Arrays; @@ -118,7 +117,7 @@ private void testScaleFromEmptyWarm(boolean allocatable) throws Exception { .put(IndexMetadata.SETTING_NUMBER_OF_REPLICAS, 0) .put(IndexMetadata.SETTING_NUMBER_OF_SHARDS, 6) .put(INDEX_STORE_STATS_REFRESH_INTERVAL_SETTING.getKey(), "0ms") - .put(DataTierAllocationDecider.TIER_PREFERENCE, allocatable ? "data_hot" : "data_content") + .put(DataTier.TIER_PREFERENCE, allocatable ? "data_hot" : "data_content") .build() ).setWaitForActiveShards(allocatable ? ActiveShardCount.DEFAULT : ActiveShardCount.NONE) ); @@ -131,9 +130,7 @@ private void testScaleFromEmptyWarm(boolean allocatable) throws Exception { client().admin() .indices() .updateSettings( - new UpdateSettingsRequest(indexName).settings( - Settings.builder().put(DataTierAllocationDecider.TIER_PREFERENCE, "data_warm,data_hot") - ) + new UpdateSettingsRequest(indexName).settings(Settings.builder().put(DataTier.TIER_PREFERENCE, "data_warm,data_hot")) ) .actionGet() ); diff --git a/x-pack/plugin/autoscaling/src/main/java/org/elasticsearch/xpack/autoscaling/storage/ReactiveStorageDeciderService.java b/x-pack/plugin/autoscaling/src/main/java/org/elasticsearch/xpack/autoscaling/storage/ReactiveStorageDeciderService.java index 1e509cf9b9190..e4a268447874c 100644 --- a/x-pack/plugin/autoscaling/src/main/java/org/elasticsearch/xpack/autoscaling/storage/ReactiveStorageDeciderService.java +++ b/x-pack/plugin/autoscaling/src/main/java/org/elasticsearch/xpack/autoscaling/storage/ReactiveStorageDeciderService.java @@ -24,6 +24,7 @@ import org.elasticsearch.cluster.routing.RoutingTable; import org.elasticsearch.cluster.routing.ShardRouting; import org.elasticsearch.cluster.routing.ShardRoutingState; +import org.elasticsearch.cluster.routing.allocation.DataTier; import org.elasticsearch.cluster.routing.allocation.DiskThresholdSettings; import org.elasticsearch.cluster.routing.allocation.RoutingAllocation; import org.elasticsearch.cluster.routing.allocation.decider.AllocationDeciders; @@ -288,7 +289,7 @@ public boolean canRemainOnlyHighestTierPreference(ShardRouting shard, RoutingAll ) != Decision.NO; if (result && nodes.isEmpty() - && Strings.hasText(DataTierAllocationDecider.TIER_PREFERENCE_SETTING.get(indexMetadata(shard, allocation).getSettings()))) { + && Strings.hasText(DataTier.TIER_PREFERENCE_SETTING.get(indexMetadata(shard, allocation).getSettings()))) { // The data tier decider allows a shard to remain on a lower preference tier when no nodes exists on higher preference // tiers. // Here we ensure that if our policy governs the highest preference tier, we assume the shard needs to move to that tier @@ -391,10 +392,9 @@ private IndexMetadata indexMetadata(ShardRouting shard, RoutingAllocation alloca return allocation.metadata().getIndexSafe(shard.index()); } - private Optional highestPreferenceTier(String tierPreference, DiscoveryNodes nodes) { - String[] preferredTiers = DataTierAllocationDecider.parseTierList(tierPreference); - assert preferredTiers.length > 0; - return Optional.of(preferredTiers[0]); + private Optional highestPreferenceTier(List preferredTiers, DiscoveryNodes nodes) { + assert preferredTiers.isEmpty() == false; + return Optional.of(preferredTiers.get(0)); } public long maxShardSize() { diff --git a/x-pack/plugin/autoscaling/src/main/java/org/elasticsearch/xpack/autoscaling/util/FrozenUtils.java b/x-pack/plugin/autoscaling/src/main/java/org/elasticsearch/xpack/autoscaling/util/FrozenUtils.java index 6c090c50b703e..3946025c297fd 100644 --- a/x-pack/plugin/autoscaling/src/main/java/org/elasticsearch/xpack/autoscaling/util/FrozenUtils.java +++ b/x-pack/plugin/autoscaling/src/main/java/org/elasticsearch/xpack/autoscaling/util/FrozenUtils.java @@ -7,16 +7,17 @@ package org.elasticsearch.xpack.autoscaling.util; +import org.elasticsearch.cluster.routing.allocation.DataTier; import org.elasticsearch.common.settings.Settings; -import org.elasticsearch.xpack.cluster.routing.allocation.DataTierAllocationDecider; -import org.elasticsearch.xpack.core.DataTier; + +import java.util.List; public class FrozenUtils { public static boolean isFrozenIndex(Settings indexSettings) { - String tierPreference = DataTierAllocationDecider.TIER_PREFERENCE_SETTING.get(indexSettings); - String[] preferredTiers = DataTierAllocationDecider.parseTierList(tierPreference); - if (preferredTiers.length >= 1 && preferredTiers[0].equals(DataTier.DATA_FROZEN)) { - assert preferredTiers.length == 1 : "frozen tier preference must be frozen only"; + String tierPreference = DataTier.TIER_PREFERENCE_SETTING.get(indexSettings); + List preferredTiers = DataTier.parseTierList(tierPreference); + if (preferredTiers.isEmpty() == false && preferredTiers.get(0).equals(DataTier.DATA_FROZEN)) { + assert preferredTiers.size() == 1 : "frozen tier preference must be frozen only"; return true; } else { return false; diff --git a/x-pack/plugin/autoscaling/src/test/java/org/elasticsearch/xpack/autoscaling/shards/FrozenShardsDeciderServiceTests.java b/x-pack/plugin/autoscaling/src/test/java/org/elasticsearch/xpack/autoscaling/shards/FrozenShardsDeciderServiceTests.java index f9de618cd7da8..34f4ec007bbd9 100644 --- a/x-pack/plugin/autoscaling/src/test/java/org/elasticsearch/xpack/autoscaling/shards/FrozenShardsDeciderServiceTests.java +++ b/x-pack/plugin/autoscaling/src/test/java/org/elasticsearch/xpack/autoscaling/shards/FrozenShardsDeciderServiceTests.java @@ -11,6 +11,7 @@ import org.elasticsearch.cluster.ClusterState; import org.elasticsearch.cluster.metadata.IndexMetadata; import org.elasticsearch.cluster.metadata.Metadata; +import org.elasticsearch.cluster.routing.allocation.DataTier; import org.elasticsearch.common.settings.Settings; import org.elasticsearch.common.unit.ByteSizeUnit; import org.elasticsearch.common.unit.ByteSizeValue; @@ -18,7 +19,6 @@ import org.elasticsearch.xpack.autoscaling.capacity.AutoscalingDeciderContext; import org.elasticsearch.xpack.autoscaling.capacity.AutoscalingDeciderResult; import org.elasticsearch.xpack.autoscaling.util.FrozenUtilsTests; -import org.elasticsearch.xpack.core.DataTier; import java.util.Objects; diff --git a/x-pack/plugin/autoscaling/src/test/java/org/elasticsearch/xpack/autoscaling/storage/FrozenStorageDeciderServiceTests.java b/x-pack/plugin/autoscaling/src/test/java/org/elasticsearch/xpack/autoscaling/storage/FrozenStorageDeciderServiceTests.java index 4c2900710235c..1b8082db46776 100644 --- a/x-pack/plugin/autoscaling/src/test/java/org/elasticsearch/xpack/autoscaling/storage/FrozenStorageDeciderServiceTests.java +++ b/x-pack/plugin/autoscaling/src/test/java/org/elasticsearch/xpack/autoscaling/storage/FrozenStorageDeciderServiceTests.java @@ -13,6 +13,7 @@ import org.elasticsearch.cluster.ClusterState; import org.elasticsearch.cluster.metadata.IndexMetadata; import org.elasticsearch.cluster.metadata.Metadata; +import org.elasticsearch.cluster.routing.allocation.DataTier; import org.elasticsearch.common.collect.ImmutableOpenMap; import org.elasticsearch.common.settings.Settings; import org.elasticsearch.common.unit.ByteSizeValue; @@ -23,7 +24,6 @@ import org.elasticsearch.xpack.autoscaling.capacity.AutoscalingDeciderContext; import org.elasticsearch.xpack.autoscaling.capacity.AutoscalingDeciderResult; import org.elasticsearch.xpack.autoscaling.util.FrozenUtilsTests; -import org.elasticsearch.xpack.core.DataTier; import static org.hamcrest.Matchers.equalTo; import static org.hamcrest.Matchers.nullValue; diff --git a/x-pack/plugin/autoscaling/src/test/java/org/elasticsearch/xpack/autoscaling/storage/ReactiveStorageDeciderDecisionTests.java b/x-pack/plugin/autoscaling/src/test/java/org/elasticsearch/xpack/autoscaling/storage/ReactiveStorageDeciderDecisionTests.java index 11addd6a847cf..84ab30ff9f175 100644 --- a/x-pack/plugin/autoscaling/src/test/java/org/elasticsearch/xpack/autoscaling/storage/ReactiveStorageDeciderDecisionTests.java +++ b/x-pack/plugin/autoscaling/src/test/java/org/elasticsearch/xpack/autoscaling/storage/ReactiveStorageDeciderDecisionTests.java @@ -25,6 +25,7 @@ import org.elasticsearch.cluster.routing.RoutingTable; import org.elasticsearch.cluster.routing.ShardRouting; import org.elasticsearch.cluster.routing.ShardRoutingState; +import org.elasticsearch.cluster.routing.allocation.DataTier; import org.elasticsearch.cluster.routing.allocation.DiskThresholdSettings; import org.elasticsearch.cluster.routing.allocation.RoutingAllocation; import org.elasticsearch.cluster.routing.allocation.allocator.BalancedShardsAllocator; @@ -47,7 +48,6 @@ import org.elasticsearch.xpack.autoscaling.capacity.AutoscalingDeciderContext; import org.elasticsearch.xpack.autoscaling.capacity.AutoscalingDeciderResult; import org.elasticsearch.xpack.cluster.routing.allocation.DataTierAllocationDecider; -import org.elasticsearch.xpack.core.DataTier; import org.junit.Before; import java.util.Collection; @@ -290,7 +290,7 @@ private IndexMetadata moveToCold(IndexMetadata imd) { overrideSetting( imd, builder, - DataTierAllocationDecider.TIER_PREFERENCE_SETTING, + DataTier.TIER_PREFERENCE_SETTING, randomFrom(DataTier.DATA_COLD, DataTier.DATA_COLD + "," + DataTier.DATA_HOT) ); return IndexMetadata.builder(imd).settings(builder).build(); @@ -591,7 +591,7 @@ public Long getShardSize(ShardRouting shardRouting) { } private static ClusterState addRandomIndices(int minShards, int maxShardCopies, ClusterState state) { - String[] tierSettingNames = new String[] { DataTierAllocationDecider.TIER_PREFERENCE }; + String[] tierSettingNames = new String[] { DataTier.TIER_PREFERENCE }; int shards = randomIntBetween(minShards, 20); Metadata.Builder builder = Metadata.builder(); RoutingTable.Builder routingTableBuilder = RoutingTable.builder(); diff --git a/x-pack/plugin/autoscaling/src/test/java/org/elasticsearch/xpack/autoscaling/storage/ReactiveStorageDeciderServiceTests.java b/x-pack/plugin/autoscaling/src/test/java/org/elasticsearch/xpack/autoscaling/storage/ReactiveStorageDeciderServiceTests.java index 479503e678dc3..8b32b0dd44482 100644 --- a/x-pack/plugin/autoscaling/src/test/java/org/elasticsearch/xpack/autoscaling/storage/ReactiveStorageDeciderServiceTests.java +++ b/x-pack/plugin/autoscaling/src/test/java/org/elasticsearch/xpack/autoscaling/storage/ReactiveStorageDeciderServiceTests.java @@ -27,6 +27,7 @@ import org.elasticsearch.cluster.routing.ShardRouting; import org.elasticsearch.cluster.routing.ShardRoutingState; import org.elasticsearch.cluster.routing.TestShardRouting; +import org.elasticsearch.cluster.routing.allocation.DataTier; import org.elasticsearch.cluster.routing.allocation.DiskThresholdSettings; import org.elasticsearch.cluster.routing.allocation.RoutingAllocation; import org.elasticsearch.cluster.routing.allocation.decider.AllocationDecider; @@ -52,7 +53,6 @@ import org.elasticsearch.snapshots.SnapshotShardSizeInfo; import org.elasticsearch.xpack.autoscaling.AutoscalingTestCase; import org.elasticsearch.xpack.cluster.routing.allocation.DataTierAllocationDecider; -import org.elasticsearch.xpack.core.DataTier; import java.util.Arrays; import java.util.Collection; @@ -513,7 +513,7 @@ public Decision canRemain(ShardRouting shardRouting, RoutingNode node, RoutingAl public ClusterState addPreference(IndexMetadata indexMetadata, ClusterState clusterState, String preference) { IndexMetadata indexMetadataWithPreference = IndexMetadata.builder(indexMetadata) - .settings(Settings.builder().put(indexMetadata.getSettings()).put(DataTierAllocationDecider.TIER_PREFERENCE, preference)) + .settings(Settings.builder().put(indexMetadata.getSettings()).put(DataTier.TIER_PREFERENCE, preference)) .build(); return ClusterState.builder(clusterState) @@ -552,7 +552,7 @@ public void testNeedsThisTier() { Metadata.Builder metaBuilder = Metadata.builder(); Settings.Builder settings = settings(Version.CURRENT); if (randomBoolean()) { - settings.put(DataTierAllocationDecider.TIER_PREFERENCE, randomBoolean() ? DataTier.DATA_HOT : "data_hot,data_warm"); + settings.put(DataTier.TIER_PREFERENCE, randomBoolean() ? DataTier.DATA_HOT : "data_hot,data_warm"); } IndexMetadata indexMetadata = IndexMetadata.builder(randomAlphaOfLength(5)) .settings(settings) diff --git a/x-pack/plugin/autoscaling/src/test/java/org/elasticsearch/xpack/autoscaling/util/FrozenUtilsTests.java b/x-pack/plugin/autoscaling/src/test/java/org/elasticsearch/xpack/autoscaling/util/FrozenUtilsTests.java index f6b8f4d9be346..01e4dbc77d9cc 100644 --- a/x-pack/plugin/autoscaling/src/test/java/org/elasticsearch/xpack/autoscaling/util/FrozenUtilsTests.java +++ b/x-pack/plugin/autoscaling/src/test/java/org/elasticsearch/xpack/autoscaling/util/FrozenUtilsTests.java @@ -11,12 +11,11 @@ import org.elasticsearch.Version; import org.elasticsearch.cluster.metadata.IndexMetadata; +import org.elasticsearch.cluster.routing.allocation.DataTier; import org.elasticsearch.common.settings.Settings; import org.elasticsearch.index.IndexModule; +import org.elasticsearch.snapshots.SearchableSnapshotsSettings; import org.elasticsearch.xpack.autoscaling.AutoscalingTestCase; -import org.elasticsearch.xpack.cluster.routing.allocation.DataTierAllocationDecider; -import org.elasticsearch.xpack.core.DataTier; -import org.elasticsearch.xpack.core.searchablesnapshots.SearchableSnapshotsConstants; import java.util.Objects; @@ -42,11 +41,11 @@ public static String randomNonFrozenTierPreference() { public static Settings indexSettings(String tierPreference) { Settings.Builder settings = Settings.builder() .put(randomAlphaOfLength(10), randomLong()) - .put(DataTierAllocationDecider.TIER_PREFERENCE, tierPreference) + .put(DataTier.TIER_PREFERENCE, tierPreference) .put(IndexMetadata.SETTING_VERSION_CREATED, Version.CURRENT); // pass setting validator. if (Objects.equals(tierPreference, DataTier.DATA_FROZEN)) { - settings.put(SearchableSnapshotsConstants.SNAPSHOT_PARTIAL_SETTING.getKey(), true) + settings.put(SearchableSnapshotsSettings.SNAPSHOT_PARTIAL_SETTING.getKey(), true) .put(IndexModule.INDEX_STORE_TYPE_SETTING.getKey(), SEARCHABLE_SNAPSHOT_STORE_TYPE); } return settings.build(); diff --git a/x-pack/plugin/core/src/internalClusterTest/java/org/elasticsearch/xpack/cluster/routing/allocation/DataTierIT.java b/x-pack/plugin/core/src/internalClusterTest/java/org/elasticsearch/xpack/cluster/routing/allocation/DataTierAllocationDeciderIT.java similarity index 89% rename from x-pack/plugin/core/src/internalClusterTest/java/org/elasticsearch/xpack/cluster/routing/allocation/DataTierIT.java rename to x-pack/plugin/core/src/internalClusterTest/java/org/elasticsearch/xpack/cluster/routing/allocation/DataTierAllocationDeciderIT.java index daf632cc4e81f..f51817ea63544 100644 --- a/x-pack/plugin/core/src/internalClusterTest/java/org/elasticsearch/xpack/cluster/routing/allocation/DataTierIT.java +++ b/x-pack/plugin/core/src/internalClusterTest/java/org/elasticsearch/xpack/cluster/routing/allocation/DataTierAllocationDeciderIT.java @@ -17,7 +17,7 @@ import org.elasticsearch.common.settings.Settings; import org.elasticsearch.plugins.Plugin; import org.elasticsearch.test.ESIntegTestCase; -import org.elasticsearch.xpack.core.DataTier; +import org.elasticsearch.cluster.routing.allocation.DataTier; import org.elasticsearch.xpack.core.DataTiersFeatureSetUsage; import org.elasticsearch.xpack.core.action.XPackUsageRequestBuilder; import org.elasticsearch.xpack.core.action.XPackUsageResponse; @@ -31,7 +31,7 @@ import static org.hamcrest.Matchers.greaterThanOrEqualTo; @ESIntegTestCase.ClusterScope(scope = ESIntegTestCase.Scope.TEST, numDataNodes = 0, numClientNodes = 0) -public class DataTierIT extends ESIntegTestCase { +public class DataTierAllocationDeciderIT extends ESIntegTestCase { private static final String index = "myindex"; @Override @@ -47,7 +47,7 @@ public void testDefaultIndexAllocateToContent() { client().admin().indices().prepareCreate(index).setWaitForActiveShards(0).get(); Settings idxSettings = client().admin().indices().prepareGetIndex().addIndices(index).get().getSettings().get(index); - assertThat(DataTierAllocationDecider.TIER_PREFERENCE_SETTING.get(idxSettings), equalTo(DataTier.DATA_CONTENT)); + assertThat(DataTier.TIER_PREFERENCE_SETTING.get(idxSettings), equalTo(DataTier.DATA_CONTENT)); // index should be red assertThat(client().admin().cluster().prepareHealth(index).get().getIndices().get(index).getStatus(), @@ -73,11 +73,11 @@ public void testOverrideDefaultAllocation() { client().admin().indices().prepareCreate(index) .setWaitForActiveShards(0) .setSettings(Settings.builder() - .put(DataTierAllocationDecider.TIER_PREFERENCE, DataTier.DATA_WARM)) + .put(DataTier.TIER_PREFERENCE, DataTier.DATA_WARM)) .get(); Settings idxSettings = client().admin().indices().prepareGetIndex().addIndices(index).get().getSettings().get(index); - assertThat(idxSettings.get(DataTierAllocationDecider.TIER_PREFERENCE), equalTo(DataTier.DATA_WARM)); + assertThat(idxSettings.get(DataTier.TIER_PREFERENCE), equalTo(DataTier.DATA_WARM)); // index should be yellow logger.info("--> waiting for {} to be yellow", index); @@ -92,13 +92,13 @@ public void testRequestSettingOverridesAllocation() { client().admin().indices().prepareCreate(index) .setWaitForActiveShards(0) .setSettings(Settings.builder() - .putNull(DataTierAllocationDecider.TIER_PREFERENCE)) + .putNull(DataTier.TIER_PREFERENCE)) .get(); Settings idxSettings = client().admin().indices().prepareGetIndex().addIndices(index).get().getSettings().get(index); - assertThat(DataTierAllocationDecider.TIER_PREFERENCE_SETTING.get(idxSettings), equalTo("")); + assertThat(DataTier.TIER_PREFERENCE_SETTING.get(idxSettings), equalTo("")); // Even the key shouldn't exist if it has been nulled out - assertFalse(idxSettings.keySet().toString(), idxSettings.keySet().contains(DataTierAllocationDecider.TIER_PREFERENCE)); + assertFalse(idxSettings.keySet().toString(), idxSettings.keySet().contains(DataTier.TIER_PREFERENCE)); // index should be yellow logger.info("--> waiting for {} to be yellow", index); @@ -114,9 +114,9 @@ public void testRequestSettingOverridesAllocation() { .get(); idxSettings = client().admin().indices().prepareGetIndex().addIndices(index).get().getSettings().get(index); - assertThat(DataTierAllocationDecider.TIER_PREFERENCE_SETTING.get(idxSettings), equalTo("")); + assertThat(DataTier.TIER_PREFERENCE_SETTING.get(idxSettings), equalTo("")); // The key should not be put in place since it was overridden - assertFalse(idxSettings.keySet().contains(DataTierAllocationDecider.TIER_PREFERENCE)); + assertFalse(idxSettings.keySet().contains(DataTier.TIER_PREFERENCE)); assertThat(idxSettings.get(IndexMetadata.INDEX_ROUTING_REQUIRE_GROUP_PREFIX + ".box"), equalTo("cold")); // index should be yellow @@ -137,7 +137,7 @@ public void testShrinkStaysOnTier() { .setSettings(Settings.builder() .put(IndexMetadata.SETTING_NUMBER_OF_SHARDS, 2) .put(IndexMetadata.SETTING_NUMBER_OF_REPLICAS, 0) - .put(DataTierAllocationDecider.TIER_PREFERENCE, "data_warm")) + .put(DataTier.TIER_PREFERENCE, "data_warm")) .get(); client().admin().indices().prepareAddBlock(IndexMetadata.APIBlock.READ_ONLY, index).get(); @@ -153,7 +153,7 @@ public void testShrinkStaysOnTier() { Settings idxSettings = client().admin().indices().prepareGetIndex().addIndices(index + "-shrunk") .get().getSettings().get(index + "-shrunk"); // It should inherit the setting of its originator - assertThat(DataTierAllocationDecider.TIER_PREFERENCE_SETTING.get(idxSettings), equalTo(DataTier.DATA_WARM)); + assertThat(DataTier.TIER_PREFERENCE_SETTING.get(idxSettings), equalTo(DataTier.DATA_WARM)); // Required or else the test cleanup fails because it can't delete the indices client().admin().indices().prepareUpdateSettings(index, index + "-shrunk") @@ -177,7 +177,7 @@ public void testTemplateOverridesDefaults() { client().admin().indices().prepareCreate(index).setWaitForActiveShards(0).get(); Settings idxSettings = client().admin().indices().prepareGetIndex().addIndices(index).get().getSettings().get(index); - assertThat(idxSettings.keySet().contains(DataTierAllocationDecider.TIER_PREFERENCE), equalTo(false)); + assertThat(idxSettings.keySet().contains(DataTier.TIER_PREFERENCE), equalTo(false)); // index should be yellow ensureYellow(index); @@ -185,7 +185,7 @@ public void testTemplateOverridesDefaults() { client().admin().indices().prepareDelete(index).get(); t = new Template(Settings.builder() - .putNull(DataTierAllocationDecider.TIER_PREFERENCE) + .putNull(DataTier.TIER_PREFERENCE) .build(), null, null); ct = new ComposableIndexTemplate.Builder().indexPatterns(Collections.singletonList(index)) .template(t).build(); @@ -195,7 +195,7 @@ public void testTemplateOverridesDefaults() { client().admin().indices().prepareCreate(index).setWaitForActiveShards(0).get(); idxSettings = client().admin().indices().prepareGetIndex().addIndices(index).get().getSettings().get(index); - assertThat(idxSettings.keySet().contains(DataTierAllocationDecider.TIER_PREFERENCE), equalTo(false)); + assertThat(idxSettings.keySet().contains(DataTier.TIER_PREFERENCE), equalTo(false)); ensureYellow(index); } @@ -207,7 +207,7 @@ public void testDataTierTelemetry() { client().admin().indices().prepareCreate(index) .setSettings(Settings.builder() - .put(DataTierAllocationDecider.TIER_PREFERENCE, "data_hot") + .put(DataTier.TIER_PREFERENCE, "data_hot") .put("index.number_of_shards", 2) .put("index.number_of_replicas", 0)) .setWaitForActiveShards(0) @@ -253,7 +253,7 @@ public void testIllegalOnFrozen() { () -> createIndex(index, Settings.builder() .put("index.number_of_shards", 1) .put("index.number_of_replicas", 0) - .put(DataTierAllocationDecider.TIER_PREFERENCE, DataTier.DATA_FROZEN) + .put(DataTier.TIER_PREFERENCE, DataTier.DATA_FROZEN) .build())); assertThat(e.getMessage(), equalTo("[data_frozen] tier can only be used for partial searchable snapshots")); @@ -261,7 +261,7 @@ public void testIllegalOnFrozen() { createIndex(index, Settings.builder() .put("index.number_of_shards", 1) .put("index.number_of_replicas", 0) - .put(DataTierAllocationDecider.TIER_PREFERENCE, initialTier) + .put(DataTier.TIER_PREFERENCE, initialTier) .build()); IllegalArgumentException e2 = expectThrows(IllegalArgumentException.class, () -> updatePreference(DataTier.DATA_FROZEN)); @@ -272,7 +272,7 @@ public void testIllegalOnFrozen() { private void updatePreference(String tier) { client().admin().indices().updateSettings(new UpdateSettingsRequest(index) - .settings(Map.of(DataTierAllocationDecider.TIER_PREFERENCE, tier))).actionGet(); + .settings(Map.of(DataTier.TIER_PREFERENCE, tier))).actionGet(); } private DataTiersFeatureSetUsage getUsage() { diff --git a/x-pack/plugin/core/src/main/java/org/elasticsearch/xpack/cluster/routing/allocation/DataTierAllocationDecider.java b/x-pack/plugin/core/src/main/java/org/elasticsearch/xpack/cluster/routing/allocation/DataTierAllocationDecider.java index 648e7b44c86e3..9466d09a30d62 100644 --- a/x-pack/plugin/core/src/main/java/org/elasticsearch/xpack/cluster/routing/allocation/DataTierAllocationDecider.java +++ b/x-pack/plugin/core/src/main/java/org/elasticsearch/xpack/cluster/routing/allocation/DataTierAllocationDecider.java @@ -17,23 +17,11 @@ import org.elasticsearch.cluster.routing.allocation.decider.AllocationDecider; import org.elasticsearch.cluster.routing.allocation.decider.Decision; import org.elasticsearch.common.Strings; -import org.elasticsearch.common.settings.Setting; -import org.elasticsearch.common.settings.Setting.Property; -import org.elasticsearch.common.settings.Settings; -import org.elasticsearch.index.IndexModule; -import org.elasticsearch.snapshots.SearchableSnapshotsSettings; -import org.elasticsearch.xpack.core.DataTier; -import org.elasticsearch.xpack.core.searchablesnapshots.SearchableSnapshotsConstants; - -import java.util.Collection; -import java.util.Iterator; +import org.elasticsearch.cluster.routing.allocation.DataTier; + import java.util.List; -import java.util.Map; import java.util.Optional; import java.util.Set; -import java.util.function.Function; - -import static org.elasticsearch.xpack.core.DataTier.DATA_FROZEN; /** * The {@code DataTierAllocationDecider} is a custom allocation decider that behaves similar to the @@ -44,65 +32,6 @@ public class DataTierAllocationDecider extends AllocationDecider { public static final String NAME = "data_tier"; - public static final String TIER_PREFERENCE = "index.routing.allocation.include._tier_preference"; - - private static final DataTierValidator VALIDATOR = new DataTierValidator(); - public static final Setting TIER_PREFERENCE_SETTING = new Setting<>(new Setting.SimpleKey(TIER_PREFERENCE), - DataTierValidator::getDefaultTierPreference, Function.identity(), VALIDATOR, Property.Dynamic, Property.IndexScope); - - private static void validateTierSetting(String setting) { - if (Strings.hasText(setting)) { - for (String s : setting.split(",")) { - if (DataTier.validTierName(s) == false) { - throw new IllegalArgumentException( - "invalid tier names found in [" + setting + "] allowed values are " + DataTier.ALL_DATA_TIERS); - } - } - } - } - - private static class DataTierValidator implements Setting.Validator { - - private static final Collection> dependencies = List.of( - IndexModule.INDEX_STORE_TYPE_SETTING, - SearchableSnapshotsConstants.SNAPSHOT_PARTIAL_SETTING - ); - - public static String getDefaultTierPreference(Settings settings) { - if (SearchableSnapshotsSettings.isPartialSearchableSnapshotIndex(settings)) { - return DATA_FROZEN; - } else { - return ""; - } - } - - @Override - public void validate(String value) { - validateTierSetting(value); - } - - @Override - public void validate(String value, Map, Object> settings, boolean exists) { - if (exists && value != null) { - if (SearchableSnapshotsConstants.isPartialSearchableSnapshotIndex(settings)) { - if (value.equals(DATA_FROZEN) == false) { - throw new IllegalArgumentException("only the [" + DATA_FROZEN + - "] tier preference may be used for partial searchable snapshots (got: [" + value + "])"); - } - } else { - if (value.contains(DATA_FROZEN)) { - throw new IllegalArgumentException("[" + DATA_FROZEN + "] tier can only be used for partial searchable snapshots"); - } - } - } - } - - @Override - public Iterator> settings() { - return dependencies.iterator(); - } - } - public DataTierAllocationDecider() { } @@ -135,7 +64,7 @@ public Decision shouldFilter(IndexMetadata indexMd, Set roles } public interface PreferredTierFunction { - Optional apply(String tierPreference, DiscoveryNodes nodes); + Optional apply(List tierPreference, DiscoveryNodes nodes); } public Decision shouldFilter(IndexMetadata indexMd, Set roles, @@ -150,23 +79,36 @@ public Decision shouldFilter(IndexMetadata indexMd, Set roles private Decision shouldIndexPreferTier(IndexMetadata indexMetadata, Set roles, PreferredTierFunction preferredTierFunction, RoutingAllocation allocation) { - Settings indexSettings = indexMetadata.getSettings(); - String tierPreference = TIER_PREFERENCE_SETTING.get(indexSettings); + List tierPreference = indexMetadata.getTierPreference(); - if (Strings.hasText(tierPreference)) { + if (tierPreference.isEmpty() == false) { Optional tier = preferredTierFunction.apply(tierPreference, allocation.nodes()); if (tier.isPresent()) { String tierName = tier.get(); if (allocationAllowed(tierName, roles)) { + if (allocation.debugDecision() == false) { + return Decision.YES; + } return allocation.decision(Decision.YES, NAME, - "index has a preference for tiers [%s] and node has tier [%s]", tierPreference, tierName); + "index has a preference for tiers [%s] and node has tier [%s]", String.join(",", tierPreference), tierName); } else { - return allocation.decision(Decision.NO, NAME, - "index has a preference for tiers [%s] and node does not meet the required [%s] tier", tierPreference, tierName); + if (allocation.debugDecision() == false) { + return Decision.NO; + } + return allocation.decision( + Decision.NO, + NAME, + "index has a preference for tiers [%s] and node does not meet the required [%s] tier", + String.join(",", tierPreference), + tierName + ); } } else { + if (allocation.debugDecision() == false) { + return Decision.NO; + } return allocation.decision(Decision.NO, NAME, "index has a preference for tiers [%s], " + - "but no nodes for any of those tiers are available in the cluster", tierPreference); + "but no nodes for any of those tiers are available in the cluster", String.join(",", tierPreference)); } } return null; @@ -178,8 +120,8 @@ private Decision shouldIndexPreferTier(IndexMetadata indexMetadata, Set}. */ - public static Optional preferredAvailableTier(String prioritizedTiers, DiscoveryNodes nodes) { - for (String tier : parseTierList(prioritizedTiers)) { + public static Optional preferredAvailableTier(List prioritizedTiers, DiscoveryNodes nodes) { + for (String tier : prioritizedTiers) { if (tierNodesPresent(tier, nodes)) { return Optional.of(tier); } @@ -187,15 +129,6 @@ public static Optional preferredAvailableTier(String prioritizedTiers, D return Optional.empty(); } - public static String[] parseTierList(String tiers) { - if (Strings.hasText(tiers) == false) { - // avoid parsing overhead in the null/empty string case - return Strings.EMPTY_ARRAY; - } else { - return tiers.split(","); - } - } - static boolean tierNodesPresent(String singleTier, DiscoveryNodes nodes) { assert singleTier.equals(DiscoveryNodeRole.DATA_ROLE.roleName()) || DataTier.validTierName(singleTier) : "tier " + singleTier + " is an invalid tier name"; diff --git a/x-pack/plugin/core/src/main/java/org/elasticsearch/xpack/cluster/routing/allocation/mapper/DataTierFieldMapper.java b/x-pack/plugin/core/src/main/java/org/elasticsearch/xpack/cluster/routing/allocation/mapper/DataTierFieldMapper.java index 7b51e3360cdd6..fb77ae0f30463 100644 --- a/x-pack/plugin/core/src/main/java/org/elasticsearch/xpack/cluster/routing/allocation/mapper/DataTierFieldMapper.java +++ b/x-pack/plugin/core/src/main/java/org/elasticsearch/xpack/cluster/routing/allocation/mapper/DataTierFieldMapper.java @@ -10,6 +10,7 @@ import org.apache.lucene.search.MatchAllDocsQuery; import org.apache.lucene.search.MatchNoDocsQuery; import org.apache.lucene.search.Query; +import org.elasticsearch.cluster.routing.allocation.DataTier; import org.elasticsearch.common.Strings; import org.elasticsearch.common.regex.Regex; import org.elasticsearch.common.settings.Settings; @@ -18,7 +19,6 @@ import org.elasticsearch.index.mapper.MetadataFieldMapper; import org.elasticsearch.index.mapper.ValueFetcher; import org.elasticsearch.index.query.SearchExecutionContext; -import org.elasticsearch.xpack.cluster.routing.allocation.DataTierAllocationDecider; import java.util.Collections; import java.util.List; @@ -89,7 +89,7 @@ public ValueFetcher valueFetcher(SearchExecutionContext context, String format) */ private String getTierPreference(SearchExecutionContext context) { Settings settings = context.getIndexSettings().getSettings(); - String value = DataTierAllocationDecider.TIER_PREFERENCE_SETTING.get(settings); + String value = DataTier.TIER_PREFERENCE_SETTING.get(settings); if (Strings.hasText(value) == false) { return null; diff --git a/x-pack/plugin/core/src/main/java/org/elasticsearch/xpack/core/DataTiersUsageTransportAction.java b/x-pack/plugin/core/src/main/java/org/elasticsearch/xpack/core/DataTiersUsageTransportAction.java index faf49641524aa..5f0a228051004 100644 --- a/x-pack/plugin/core/src/main/java/org/elasticsearch/xpack/core/DataTiersUsageTransportAction.java +++ b/x-pack/plugin/core/src/main/java/org/elasticsearch/xpack/core/DataTiersUsageTransportAction.java @@ -21,6 +21,7 @@ import org.elasticsearch.cluster.routing.RoutingNodes; import org.elasticsearch.cluster.routing.ShardRouting; import org.elasticsearch.cluster.routing.ShardRoutingState; +import org.elasticsearch.cluster.routing.allocation.DataTier; import org.elasticsearch.cluster.service.ClusterService; import org.elasticsearch.common.Strings; import org.elasticsearch.common.collect.ImmutableOpenMap; @@ -32,7 +33,6 @@ import org.elasticsearch.tasks.Task; import org.elasticsearch.threadpool.ThreadPool; import org.elasticsearch.transport.TransportService; -import org.elasticsearch.xpack.cluster.routing.allocation.DataTierAllocationDecider; import org.elasticsearch.xpack.core.action.XPackUsageFeatureAction; import org.elasticsearch.xpack.core.action.XPackUsageFeatureResponse; import org.elasticsearch.xpack.core.action.XPackUsageFeatureTransportAction; @@ -83,7 +83,7 @@ protected void masterOperation(Task task, XPackUsageRequest request, ClusterStat static Map tierIndices(ImmutableOpenMap indices) { Map indexByTier = new HashMap<>(); indices.forEach(entry -> { - String tierPref = entry.value.getSettings().get(DataTierAllocationDecider.TIER_PREFERENCE); + String tierPref = entry.value.getSettings().get(DataTier.TIER_PREFERENCE); if (Strings.hasText(tierPref)) { String[] tiers = tierPref.split(","); if (tiers.length > 0) { diff --git a/x-pack/plugin/core/src/main/java/org/elasticsearch/xpack/core/XPackPlugin.java b/x-pack/plugin/core/src/main/java/org/elasticsearch/xpack/core/XPackPlugin.java index f276ead608e33..30cf991994688 100644 --- a/x-pack/plugin/core/src/main/java/org/elasticsearch/xpack/core/XPackPlugin.java +++ b/x-pack/plugin/core/src/main/java/org/elasticsearch/xpack/core/XPackPlugin.java @@ -19,6 +19,7 @@ import org.elasticsearch.cluster.metadata.Metadata; import org.elasticsearch.cluster.node.DiscoveryNode; import org.elasticsearch.cluster.node.DiscoveryNodes; +import org.elasticsearch.cluster.routing.allocation.DataTier; import org.elasticsearch.cluster.routing.allocation.decider.AllocationDecider; import org.elasticsearch.cluster.service.ClusterService; import org.elasticsearch.common.inject.Binder; @@ -384,7 +385,7 @@ public Optional getEngineFactory(IndexSettings indexSettings) { public List> getSettings() { List> settings = super.getSettings(); settings.add(SourceOnlySnapshotRepository.SOURCE_ONLY); - settings.add(DataTierAllocationDecider.TIER_PREFERENCE_SETTING); + settings.add(DataTier.TIER_PREFERENCE_SETTING); return settings; } diff --git a/x-pack/plugin/core/src/main/java/org/elasticsearch/xpack/core/ilm/DataTierMigrationRoutedStep.java b/x-pack/plugin/core/src/main/java/org/elasticsearch/xpack/core/ilm/DataTierMigrationRoutedStep.java index b3f8ae321efc9..91f04f114986d 100644 --- a/x-pack/plugin/core/src/main/java/org/elasticsearch/xpack/core/ilm/DataTierMigrationRoutedStep.java +++ b/x-pack/plugin/core/src/main/java/org/elasticsearch/xpack/core/ilm/DataTierMigrationRoutedStep.java @@ -12,7 +12,6 @@ import org.elasticsearch.cluster.ClusterState; import org.elasticsearch.cluster.metadata.IndexMetadata; import org.elasticsearch.cluster.routing.allocation.decider.AllocationDeciders; -import org.elasticsearch.common.Strings; import org.elasticsearch.index.Index; import org.elasticsearch.xpack.cluster.routing.allocation.DataTierAllocationDecider; import org.elasticsearch.xpack.core.ilm.step.info.AllocationInfo; @@ -21,7 +20,6 @@ import java.util.Locale; import java.util.Optional; -import static org.elasticsearch.xpack.cluster.routing.allocation.DataTierAllocationDecider.TIER_PREFERENCE_SETTING; import static org.elasticsearch.xpack.core.ilm.AllocationRoutedStep.getPendingAllocations; import static org.elasticsearch.xpack.core.ilm.step.info.AllocationInfo.waitingForActiveShardsAllocationInfo; @@ -56,28 +54,28 @@ public Result isConditionMet(Index index, ClusterState clusterState) { logger.debug("[{}] lifecycle action for index [{}] executed but index no longer exists", getKey().getAction(), index.getName()); return new Result(false, null); } - String preferredTierConfiguration = TIER_PREFERENCE_SETTING.get(idxMeta.getSettings()); + List preferredTierConfiguration = idxMeta.getTierPreference(); Optional availableDestinationTier = DataTierAllocationDecider.preferredAvailableTier(preferredTierConfiguration, clusterState.getNodes()); if (ActiveShardCount.ALL.enoughShardsActive(clusterState, index.getName()) == false) { - if (Strings.isEmpty(preferredTierConfiguration)) { + if (preferredTierConfiguration.isEmpty()) { logger.debug("[{}] lifecycle action for index [{}] cannot make progress because not all shards are active", getKey().getAction(), index.getName()); } else { if (availableDestinationTier.isPresent()) { - logger.debug("[{}] migration of index [{}] to the [{}] tier preference cannot progress, as not all shards are active", + logger.debug("[{}] migration of index [{}] to the {} tier preference cannot progress, as not all shards are active", getKey().getAction(), index.getName(), preferredTierConfiguration); } else { logger.debug("[{}] migration of index [{}] to the next tier cannot progress as there is no available tier for the " + - "configured preferred tiers [{}] and not all shards are active", getKey().getAction(), index.getName(), + "configured preferred tiers {} and not all shards are active", getKey().getAction(), index.getName(), preferredTierConfiguration); } } return new Result(false, waitingForActiveShardsAllocationInfo(idxMeta.getNumberOfReplicas())); } - if (Strings.isEmpty(preferredTierConfiguration)) { + if (preferredTierConfiguration.isEmpty()) { logger.debug("index [{}] has no data tier routing preference setting configured and all its shards are active. considering " + "the [{}] step condition met and continuing to the next step", index.getName(), getKey().getName()); // the user removed the tier routing setting and all the shards are active so we'll cary on @@ -89,10 +87,10 @@ public Result isConditionMet(Index index, ClusterState clusterState) { if (allocationPendingAllShards > 0) { String statusMessage = availableDestinationTier.map( s -> String.format(Locale.ROOT, "[%s] lifecycle action [%s] waiting for [%s] shards to be moved to the [%s] tier (tier " + - "migration preference configuration is [%s])", index.getName(), getKey().getAction(), allocationPendingAllShards, s, + "migration preference configuration is %s)", index.getName(), getKey().getAction(), allocationPendingAllShards, s, preferredTierConfiguration) ).orElseGet( - () -> String.format(Locale.ROOT, "index [%s] has a preference for tiers [%s], but no nodes for any of those tiers are " + + () -> String.format(Locale.ROOT, "index [%s] has a preference for tiers %s, but no nodes for any of those tiers are " + "available in the cluster", index.getName(), preferredTierConfiguration)); logger.debug(statusMessage); return new Result(false, new AllocationInfo(idxMeta.getNumberOfReplicas(), allocationPendingAllShards, true, statusMessage)); diff --git a/x-pack/plugin/core/src/main/java/org/elasticsearch/xpack/core/ilm/MigrateAction.java b/x-pack/plugin/core/src/main/java/org/elasticsearch/xpack/core/ilm/MigrateAction.java index 17a0f9ce720d4..8d629ee08ae90 100644 --- a/x-pack/plugin/core/src/main/java/org/elasticsearch/xpack/core/ilm/MigrateAction.java +++ b/x-pack/plugin/core/src/main/java/org/elasticsearch/xpack/core/ilm/MigrateAction.java @@ -18,19 +18,18 @@ import org.elasticsearch.common.xcontent.XContentBuilder; import org.elasticsearch.common.xcontent.XContentParser; import org.elasticsearch.snapshots.SearchableSnapshotsSettings; -import org.elasticsearch.xpack.cluster.routing.allocation.DataTierAllocationDecider; -import org.elasticsearch.xpack.core.DataTier; +import org.elasticsearch.cluster.routing.allocation.DataTier; import org.elasticsearch.xpack.core.ilm.Step.StepKey; import java.io.IOException; import java.util.List; import java.util.Objects; -import static org.elasticsearch.xpack.core.DataTier.getPreferredTiersConfiguration; +import static org.elasticsearch.cluster.routing.allocation.DataTier.getPreferredTiersConfiguration; /** * A {@link LifecycleAction} which enables or disables the automatic migration of data between - * {@link org.elasticsearch.xpack.core.DataTier}s. + * {@link DataTier}s. */ public class MigrateAction implements LifecycleAction { public static final String NAME = "migrate"; @@ -117,7 +116,7 @@ public List toSteps(Client client, String phase, StepKey nextStepKey) { }); UpdateSettingsStep updateMigrationSettingStep = new UpdateSettingsStep(migrationKey, migrationRoutedKey, client, Settings.builder() - .put(DataTierAllocationDecider.TIER_PREFERENCE, getPreferredTiersConfiguration(targetTier)) + .put(DataTier.TIER_PREFERENCE, getPreferredTiersConfiguration(targetTier)) .build()); DataTierMigrationRoutedStep migrationRoutedStep = new DataTierMigrationRoutedStep(migrationRoutedKey, nextStepKey); return List.of(conditionalSkipActionStep, updateMigrationSettingStep, migrationRoutedStep); diff --git a/x-pack/plugin/core/src/main/java/org/elasticsearch/xpack/core/ilm/MountSnapshotStep.java b/x-pack/plugin/core/src/main/java/org/elasticsearch/xpack/core/ilm/MountSnapshotStep.java index 4dac280f856f1..3649798cd3e9d 100644 --- a/x-pack/plugin/core/src/main/java/org/elasticsearch/xpack/core/ilm/MountSnapshotStep.java +++ b/x-pack/plugin/core/src/main/java/org/elasticsearch/xpack/core/ilm/MountSnapshotStep.java @@ -17,8 +17,7 @@ import org.elasticsearch.common.settings.Settings; import org.elasticsearch.core.TimeValue; import org.elasticsearch.rest.RestStatus; -import org.elasticsearch.xpack.cluster.routing.allocation.DataTierAllocationDecider; -import org.elasticsearch.xpack.core.DataTier; +import org.elasticsearch.cluster.routing.allocation.DataTier; import org.elasticsearch.xpack.core.searchablesnapshots.MountSearchableSnapshotAction; import org.elasticsearch.xpack.core.searchablesnapshots.MountSearchableSnapshotRequest; @@ -107,7 +106,7 @@ void performDuringNoSnapshot(IndexMetadata indexMetadata, ClusterState currentCl final Settings.Builder settingsBuilder = Settings.builder(); overrideTierPreference(this.getKey().getPhase()) - .ifPresent(override -> settingsBuilder.put(DataTierAllocationDecider.TIER_PREFERENCE, override)); + .ifPresent(override -> settingsBuilder.put(DataTier.TIER_PREFERENCE, override)); final MountSearchableSnapshotRequest mountSearchableSnapshotRequest = new MountSearchableSnapshotRequest(mountedIndexName, snapshotRepository, snapshotName, indexName, settingsBuilder.build(), diff --git a/x-pack/plugin/core/src/main/java/org/elasticsearch/xpack/core/ilm/WaitForDataTierStep.java b/x-pack/plugin/core/src/main/java/org/elasticsearch/xpack/core/ilm/WaitForDataTierStep.java index f84a8f277cfb0..5679b50191b41 100644 --- a/x-pack/plugin/core/src/main/java/org/elasticsearch/xpack/core/ilm/WaitForDataTierStep.java +++ b/x-pack/plugin/core/src/main/java/org/elasticsearch/xpack/core/ilm/WaitForDataTierStep.java @@ -8,6 +8,7 @@ package org.elasticsearch.xpack.core.ilm; import org.elasticsearch.cluster.ClusterState; +import org.elasticsearch.cluster.routing.allocation.DataTier; import org.elasticsearch.index.Index; import org.elasticsearch.xpack.cluster.routing.allocation.DataTierAllocationDecider; import org.elasticsearch.xpack.core.ilm.step.info.SingleMessageFieldInfo; @@ -32,7 +33,8 @@ public WaitForDataTierStep(StepKey key, StepKey nextStepKey, String tierPreferen @Override public Result isConditionMet(Index index, ClusterState clusterState) { - boolean present = DataTierAllocationDecider.preferredAvailableTier(tierPreference, clusterState.nodes()).isPresent(); + boolean present = DataTierAllocationDecider.preferredAvailableTier( + DataTier.parseTierList(tierPreference), clusterState.nodes()).isPresent(); SingleMessageFieldInfo info = present ? null : new SingleMessageFieldInfo("no nodes for tiers [" + tierPreference + "] available"); return new Result(present, info); } diff --git a/x-pack/plugin/core/src/main/java/org/elasticsearch/xpack/core/searchablesnapshots/MountSearchableSnapshotRequest.java b/x-pack/plugin/core/src/main/java/org/elasticsearch/xpack/core/searchablesnapshots/MountSearchableSnapshotRequest.java index 9ed50dd2a1d8f..991945e861a63 100644 --- a/x-pack/plugin/core/src/main/java/org/elasticsearch/xpack/core/searchablesnapshots/MountSearchableSnapshotRequest.java +++ b/x-pack/plugin/core/src/main/java/org/elasticsearch/xpack/core/searchablesnapshots/MountSearchableSnapshotRequest.java @@ -21,7 +21,7 @@ import org.elasticsearch.common.xcontent.ObjectParser; import org.elasticsearch.common.xcontent.XContentParser; import org.elasticsearch.rest.RestRequest; -import org.elasticsearch.xpack.core.DataTier; +import org.elasticsearch.cluster.routing.allocation.DataTier; import java.io.IOException; import java.util.Arrays; diff --git a/x-pack/plugin/core/src/main/java/org/elasticsearch/xpack/core/searchablesnapshots/SearchableSnapshotsConstants.java b/x-pack/plugin/core/src/main/java/org/elasticsearch/xpack/core/searchablesnapshots/SearchableSnapshotsConstants.java index 4b166216060da..8951060576a67 100644 --- a/x-pack/plugin/core/src/main/java/org/elasticsearch/xpack/core/searchablesnapshots/SearchableSnapshotsConstants.java +++ b/x-pack/plugin/core/src/main/java/org/elasticsearch/xpack/core/searchablesnapshots/SearchableSnapshotsConstants.java @@ -6,15 +6,8 @@ */ package org.elasticsearch.xpack.core.searchablesnapshots; -import org.elasticsearch.common.settings.Setting; import org.elasticsearch.license.License; import org.elasticsearch.license.LicensedFeature; -import org.elasticsearch.snapshots.SearchableSnapshotsSettings; - -import java.util.Map; - -import static org.elasticsearch.index.IndexModule.INDEX_STORE_TYPE_SETTING; -import static org.elasticsearch.snapshots.SearchableSnapshotsSettings.SEARCHABLE_SNAPSHOT_PARTIAL_SETTING_KEY; public class SearchableSnapshotsConstants { @@ -24,23 +17,4 @@ public class SearchableSnapshotsConstants { public static final LicensedFeature.Momentary SEARCHABLE_SNAPSHOT_FEATURE = LicensedFeature.momentary(null, "searchable-snapshots", License.OperationMode.ENTERPRISE); - public static final Setting SNAPSHOT_PARTIAL_SETTING = Setting.boolSetting( - SEARCHABLE_SNAPSHOT_PARTIAL_SETTING_KEY, - false, - Setting.Property.IndexScope, - Setting.Property.PrivateIndex, - Setting.Property.NotCopyableOnResize - ); - - /** - * Based on a map from setting to value, do the settings represent a partial searchable snapshot index? - * - * Both index.store.type and index.store.snapshot.partial must be supplied. - */ - public static boolean isPartialSearchableSnapshotIndex(Map, Object> indexSettings) { - assert indexSettings.containsKey(INDEX_STORE_TYPE_SETTING) : "must include store type in map"; - assert indexSettings.get(SNAPSHOT_PARTIAL_SETTING) != null : "partial setting must be non-null in map (has default value)"; - return SearchableSnapshotsSettings.SEARCHABLE_SNAPSHOT_STORE_TYPE.equals(indexSettings.get(INDEX_STORE_TYPE_SETTING)) - && (boolean) indexSettings.get(SNAPSHOT_PARTIAL_SETTING); - } } diff --git a/x-pack/plugin/core/src/test/java/org/elasticsearch/cluster/metadata/IndexMetadataConversionTests.java b/x-pack/plugin/core/src/test/java/org/elasticsearch/cluster/metadata/IndexMetadataConversionTests.java index 9c71356a05e44..fa320d59c5910 100644 --- a/x-pack/plugin/core/src/test/java/org/elasticsearch/cluster/metadata/IndexMetadataConversionTests.java +++ b/x-pack/plugin/core/src/test/java/org/elasticsearch/cluster/metadata/IndexMetadataConversionTests.java @@ -13,8 +13,8 @@ import org.elasticsearch.index.IndexModule; import org.elasticsearch.index.mapper.MapperRegistry; import org.elasticsearch.plugins.MapperPlugin; +import org.elasticsearch.snapshots.SearchableSnapshotsSettings; import org.elasticsearch.test.ESTestCase; -import org.elasticsearch.xpack.core.searchablesnapshots.SearchableSnapshotsConstants; import java.util.Collections; @@ -33,7 +33,7 @@ public void testConvertSearchableSnapshotSettings() { // A full_copy searchable snapshot (settings should be untouched) src = newIndexMeta("foo", Settings.builder() .put(IndexModule.INDEX_STORE_TYPE_SETTING.getKey(), SEARCHABLE_SNAPSHOT_STORE_TYPE) - .put(SearchableSnapshotsConstants.SNAPSHOT_PARTIAL_SETTING.getKey(), false) + .put(SearchableSnapshotsSettings.SNAPSHOT_PARTIAL_SETTING.getKey(), false) .put("index.routing.allocation.include._tier", "data_hot") .put("index.routing.allocation.exclude._tier", "data_warm") .put("index.routing.allocation.require._tier", "data_hot") @@ -45,7 +45,7 @@ public void testConvertSearchableSnapshotSettings() { // A shared_cache searchable snapshot with valid settings (metadata should be untouched) src = newIndexMeta("foo", Settings.builder() .put(IndexModule.INDEX_STORE_TYPE_SETTING.getKey(), SEARCHABLE_SNAPSHOT_STORE_TYPE) - .put(SearchableSnapshotsConstants.SNAPSHOT_PARTIAL_SETTING.getKey(), false) + .put(SearchableSnapshotsSettings.SNAPSHOT_PARTIAL_SETTING.getKey(), false) .put("index.routing.allocation.include._tier_preference", "data_frozen") .build()); indexMetadata = service.convertSharedCacheTierPreference(src); @@ -54,7 +54,7 @@ public void testConvertSearchableSnapshotSettings() { // A shared_cache searchable snapshot (should have its settings converted) src = newIndexMeta("foo", Settings.builder() .put(IndexModule.INDEX_STORE_TYPE_SETTING.getKey(), SEARCHABLE_SNAPSHOT_STORE_TYPE) - .put(SearchableSnapshotsConstants.SNAPSHOT_PARTIAL_SETTING.getKey(), true) + .put(SearchableSnapshotsSettings.SNAPSHOT_PARTIAL_SETTING.getKey(), true) .put("index.routing.allocation.include._tier", "data_hot") .put("index.routing.allocation.exclude._tier", "data_warm") .put("index.routing.allocation.require._tier", "data_hot") diff --git a/x-pack/plugin/core/src/test/java/org/elasticsearch/xpack/cluster/routing/allocation/DataTierAllocationDeciderTests.java b/x-pack/plugin/core/src/test/java/org/elasticsearch/xpack/cluster/routing/allocation/DataTierAllocationDeciderTests.java index a5800a2dbb5af..3f1c12c55a39f 100644 --- a/x-pack/plugin/core/src/test/java/org/elasticsearch/xpack/cluster/routing/allocation/DataTierAllocationDeciderTests.java +++ b/x-pack/plugin/core/src/test/java/org/elasticsearch/xpack/cluster/routing/allocation/DataTierAllocationDeciderTests.java @@ -38,8 +38,7 @@ import org.elasticsearch.snapshots.EmptySnapshotsInfoService; import org.elasticsearch.snapshots.SearchableSnapshotsSettings; import org.elasticsearch.test.gateway.TestGatewayAllocator; -import org.elasticsearch.xpack.core.DataTier; -import org.elasticsearch.xpack.core.searchablesnapshots.SearchableSnapshotsConstants; +import org.elasticsearch.cluster.routing.allocation.DataTier; import java.util.ArrayList; import java.util.Arrays; @@ -47,8 +46,8 @@ import java.util.List; import java.util.Optional; -import static org.elasticsearch.xpack.core.DataTier.DATA_COLD; -import static org.elasticsearch.xpack.core.DataTier.DATA_FROZEN; +import static org.elasticsearch.cluster.routing.allocation.DataTier.DATA_COLD; +import static org.elasticsearch.cluster.routing.allocation.DataTier.DATA_FROZEN; import static org.hamcrest.Matchers.containsString; import static org.hamcrest.Matchers.equalTo; @@ -86,7 +85,7 @@ public void testIndexPrefer() { .put(IndexMetadata.SETTING_INDEX_UUID, "myindex") .put(IndexMetadata.SETTING_NUMBER_OF_SHARDS, 1) .put(IndexMetadata.SETTING_NUMBER_OF_REPLICAS, 0) - .put(DataTierAllocationDecider.TIER_PREFERENCE, "data_warm,data_cold") + .put(DataTier.TIER_PREFERENCE, "data_warm,data_cold") .build())) .build()) .build(); @@ -121,7 +120,7 @@ public void testIndexPrefer() { .put(IndexMetadata.SETTING_INDEX_UUID, "myindex") .put(IndexMetadata.SETTING_NUMBER_OF_SHARDS, 1) .put(IndexMetadata.SETTING_NUMBER_OF_REPLICAS, 0) - .put(DataTierAllocationDecider.TIER_PREFERENCE, "data_warm,data_cold") + .put(DataTier.TIER_PREFERENCE, "data_warm,data_cold") .build())) .build()) .build(); @@ -189,24 +188,32 @@ public void testTierNodesPresent() { public void testPreferredTierAvailable() { DiscoveryNodes nodes = DiscoveryNodes.builder().build(); - assertThat(DataTierAllocationDecider.preferredAvailableTier("data", nodes), equalTo(Optional.empty())); - assertThat(DataTierAllocationDecider.preferredAvailableTier("data_hot,data_warm", nodes), equalTo(Optional.empty())); - assertThat(DataTierAllocationDecider.preferredAvailableTier("data_warm,data_content", nodes), equalTo(Optional.empty())); - assertThat(DataTierAllocationDecider.preferredAvailableTier("data_cold", nodes), equalTo(Optional.empty())); + assertThat(DataTierAllocationDecider.preferredAvailableTier( + DataTier.parseTierList("data"), nodes), equalTo(Optional.empty())); + assertThat(DataTierAllocationDecider.preferredAvailableTier( + DataTier.parseTierList("data_hot,data_warm"), nodes), equalTo(Optional.empty())); + assertThat(DataTierAllocationDecider.preferredAvailableTier( + DataTier.parseTierList("data_warm,data_content"), nodes), equalTo(Optional.empty())); + assertThat(DataTierAllocationDecider.preferredAvailableTier( + DataTier.parseTierList("data_cold"), nodes), equalTo(Optional.empty())); nodes = DiscoveryNodes.builder() .add(WARM_NODE) .add(CONTENT_NODE) .build(); - assertThat(DataTierAllocationDecider.preferredAvailableTier("data", nodes), equalTo(Optional.empty())); - assertThat(DataTierAllocationDecider.preferredAvailableTier("data_hot,data_warm", nodes), equalTo(Optional.of("data_warm"))); - assertThat(DataTierAllocationDecider.preferredAvailableTier("data_warm,data_content", nodes), equalTo(Optional.of("data_warm"))); - assertThat(DataTierAllocationDecider.preferredAvailableTier("data_content,data_warm", nodes), equalTo(Optional.of("data_content"))); - assertThat(DataTierAllocationDecider.preferredAvailableTier("data_hot,data_content,data_warm", nodes), - equalTo(Optional.of("data_content"))); - assertThat(DataTierAllocationDecider.preferredAvailableTier("data_hot,data_cold,data_warm", nodes), - equalTo(Optional.of("data_warm"))); + assertThat(DataTierAllocationDecider.preferredAvailableTier( + DataTier.parseTierList("data"), nodes), equalTo(Optional.empty())); + assertThat(DataTierAllocationDecider.preferredAvailableTier( + DataTier.parseTierList("data_hot,data_warm"), nodes), equalTo(Optional.of("data_warm"))); + assertThat(DataTierAllocationDecider.preferredAvailableTier( + DataTier.parseTierList("data_warm,data_content"), nodes), equalTo(Optional.of("data_warm"))); + assertThat(DataTierAllocationDecider.preferredAvailableTier( + DataTier.parseTierList("data_content,data_warm"), nodes), equalTo(Optional.of("data_content"))); + assertThat(DataTierAllocationDecider.preferredAvailableTier( + DataTier.parseTierList("data_hot,data_content,data_warm"), nodes), equalTo(Optional.of("data_content"))); + assertThat(DataTierAllocationDecider.preferredAvailableTier( + DataTier.parseTierList("data_hot,data_cold,data_warm"), nodes), equalTo(Optional.of("data_warm"))); } public void testFrozenIllegalForRegularIndices() { @@ -217,7 +224,7 @@ public void testFrozenIllegalForRegularIndices() { Randomness.shuffle(tierList); String value = Strings.join(tierList, ","); - Setting setting = DataTierAllocationDecider.TIER_PREFERENCE_SETTING; + Setting setting = DataTier.TIER_PREFERENCE_SETTING; Settings.Builder builder = Settings.builder().put(setting.getKey(), value); if (randomBoolean()) { builder.put(IndexModule.INDEX_STORE_TYPE_SETTING.getKey(), SearchableSnapshotsSettings.SEARCHABLE_SNAPSHOT_STORE_TYPE); @@ -229,10 +236,10 @@ public void testFrozenIllegalForRegularIndices() { } public void testFrozenLegalForPartialSnapshot() { - Setting setting = DataTierAllocationDecider.TIER_PREFERENCE_SETTING; + Setting setting = DataTier.TIER_PREFERENCE_SETTING; Settings.Builder builder = Settings.builder().put(setting.getKey(), DATA_FROZEN); builder.put(IndexModule.INDEX_STORE_TYPE_SETTING.getKey(), SearchableSnapshotsSettings.SEARCHABLE_SNAPSHOT_STORE_TYPE); - builder.put(SearchableSnapshotsConstants.SNAPSHOT_PARTIAL_SETTING.getKey(), true); + builder.put(SearchableSnapshotsSettings.SNAPSHOT_PARTIAL_SETTING.getKey(), true); Settings settings = builder.build(); @@ -250,53 +257,53 @@ public void testNonFrozenIllegalForPartialSnapshot() { { String value = Strings.join(tierList, ","); - Settings.Builder builder = Settings.builder().put(DataTierAllocationDecider.TIER_PREFERENCE, value); + Settings.Builder builder = Settings.builder().put(DataTier.TIER_PREFERENCE, value); builder.put(IndexModule.INDEX_STORE_TYPE_SETTING.getKey(), SearchableSnapshotsSettings.SEARCHABLE_SNAPSHOT_STORE_TYPE); - builder.put(SearchableSnapshotsConstants.SNAPSHOT_PARTIAL_SETTING.getKey(), true); + builder.put(SearchableSnapshotsSettings.SNAPSHOT_PARTIAL_SETTING.getKey(), true); Settings settings = builder.build(); IllegalArgumentException e = expectThrows(IllegalArgumentException.class, - () -> DataTierAllocationDecider.TIER_PREFERENCE_SETTING.get(settings)); + () -> DataTier.TIER_PREFERENCE_SETTING.get(settings)); assertThat(e.getMessage(), containsString("only the [data_frozen] tier preference may be used for partial searchable snapshots")); } { - Settings.Builder builder = Settings.builder().put(DataTierAllocationDecider.TIER_PREFERENCE, ""); + Settings.Builder builder = Settings.builder().put(DataTier.TIER_PREFERENCE, ""); builder.put(IndexModule.INDEX_STORE_TYPE_SETTING.getKey(), SearchableSnapshotsSettings.SEARCHABLE_SNAPSHOT_STORE_TYPE); - builder.put(SearchableSnapshotsConstants.SNAPSHOT_PARTIAL_SETTING.getKey(), true); + builder.put(SearchableSnapshotsSettings.SNAPSHOT_PARTIAL_SETTING.getKey(), true); Settings settings = builder.build(); IllegalArgumentException e = expectThrows(IllegalArgumentException.class, - () -> DataTierAllocationDecider.TIER_PREFERENCE_SETTING.get(settings)); + () -> DataTier.TIER_PREFERENCE_SETTING.get(settings)); assertThat(e.getMessage(), containsString("only the [data_frozen] tier preference may be used for partial searchable snapshots")); } { - Settings.Builder builder = Settings.builder().put(DataTierAllocationDecider.TIER_PREFERENCE, " "); + Settings.Builder builder = Settings.builder().put(DataTier.TIER_PREFERENCE, " "); builder.put(IndexModule.INDEX_STORE_TYPE_SETTING.getKey(), SearchableSnapshotsSettings.SEARCHABLE_SNAPSHOT_STORE_TYPE); - builder.put(SearchableSnapshotsConstants.SNAPSHOT_PARTIAL_SETTING.getKey(), true); + builder.put(SearchableSnapshotsSettings.SNAPSHOT_PARTIAL_SETTING.getKey(), true); Settings settings = builder.build(); IllegalArgumentException e = expectThrows(IllegalArgumentException.class, - () -> DataTierAllocationDecider.TIER_PREFERENCE_SETTING.get(settings)); + () -> DataTier.TIER_PREFERENCE_SETTING.get(settings)); assertThat(e.getMessage(), containsString("only the [data_frozen] tier preference may be used for partial searchable snapshots")); } } public void testDefaultValueForPreference() { - assertThat(DataTierAllocationDecider.TIER_PREFERENCE_SETTING.get(Settings.EMPTY), equalTo("")); + assertThat(DataTier.TIER_PREFERENCE_SETTING.get(Settings.EMPTY), equalTo("")); Settings.Builder builder = Settings.builder(); builder.put(IndexModule.INDEX_STORE_TYPE_SETTING.getKey(), SearchableSnapshotsSettings.SEARCHABLE_SNAPSHOT_STORE_TYPE); - builder.put(SearchableSnapshotsConstants.SNAPSHOT_PARTIAL_SETTING.getKey(), true); + builder.put(SearchableSnapshotsSettings.SNAPSHOT_PARTIAL_SETTING.getKey(), true); Settings settings = builder.build(); - assertThat(DataTierAllocationDecider.TIER_PREFERENCE_SETTING.get(settings), equalTo(DATA_FROZEN)); + assertThat(DataTier.TIER_PREFERENCE_SETTING.get(settings), equalTo(DATA_FROZEN)); } } diff --git a/x-pack/plugin/core/src/test/java/org/elasticsearch/xpack/cluster/routing/allocation/mapper/DataTierFieldTypeTests.java b/x-pack/plugin/core/src/test/java/org/elasticsearch/xpack/cluster/routing/allocation/mapper/DataTierFieldTypeTests.java index 1150c9e30e5bb..e336d3b3e36ed 100644 --- a/x-pack/plugin/core/src/test/java/org/elasticsearch/xpack/cluster/routing/allocation/mapper/DataTierFieldTypeTests.java +++ b/x-pack/plugin/core/src/test/java/org/elasticsearch/xpack/cluster/routing/allocation/mapper/DataTierFieldTypeTests.java @@ -11,6 +11,7 @@ import org.apache.lucene.search.MatchNoDocsQuery; import org.elasticsearch.Version; import org.elasticsearch.cluster.metadata.IndexMetadata; +import org.elasticsearch.cluster.routing.allocation.DataTier; import org.elasticsearch.common.regex.Regex; import org.elasticsearch.common.settings.Settings; import org.elasticsearch.index.IndexSettings; @@ -20,7 +21,6 @@ import org.elasticsearch.index.query.QueryShardException; import org.elasticsearch.index.query.SearchExecutionContext; import org.elasticsearch.search.lookup.SourceLookup; -import org.elasticsearch.xpack.cluster.routing.allocation.DataTierAllocationDecider; import java.io.IOException; import java.util.Arrays; @@ -101,7 +101,7 @@ private SearchExecutionContext createContext() { Settings.builder() .put(IndexMetadata.SETTING_VERSION_CREATED, Version.CURRENT) // Tier can be an ordered list of preferences - starting with primary and followed by fallbacks. - .put(DataTierAllocationDecider.TIER_PREFERENCE, "data_warm,data_hot") + .put(DataTier.TIER_PREFERENCE, "data_warm,data_hot") ) .numberOfShards(1) .numberOfReplicas(0) diff --git a/x-pack/plugin/core/src/test/java/org/elasticsearch/xpack/core/DataTierTests.java b/x-pack/plugin/core/src/test/java/org/elasticsearch/xpack/core/DataTierTests.java index bfe7dda5175c1..976fc11829783 100644 --- a/x-pack/plugin/core/src/test/java/org/elasticsearch/xpack/core/DataTierTests.java +++ b/x-pack/plugin/core/src/test/java/org/elasticsearch/xpack/core/DataTierTests.java @@ -11,6 +11,7 @@ import org.elasticsearch.cluster.node.DiscoveryNode; import org.elasticsearch.cluster.node.DiscoveryNodeRole; import org.elasticsearch.cluster.node.DiscoveryNodes; +import org.elasticsearch.cluster.routing.allocation.DataTier; import org.elasticsearch.common.Strings; import org.elasticsearch.common.settings.Settings; import org.elasticsearch.node.NodeRoleSettings; @@ -25,10 +26,10 @@ import java.util.Set; import java.util.concurrent.atomic.AtomicInteger; -import static org.elasticsearch.xpack.core.DataTier.DATA_COLD; -import static org.elasticsearch.xpack.core.DataTier.DATA_HOT; -import static org.elasticsearch.xpack.core.DataTier.DATA_WARM; -import static org.elasticsearch.xpack.core.DataTier.getPreferredTiersConfiguration; +import static org.elasticsearch.cluster.routing.allocation.DataTier.DATA_COLD; +import static org.elasticsearch.cluster.routing.allocation.DataTier.DATA_HOT; +import static org.elasticsearch.cluster.routing.allocation.DataTier.DATA_WARM; +import static org.elasticsearch.cluster.routing.allocation.DataTier.getPreferredTiersConfiguration; import static org.hamcrest.CoreMatchers.is; import static org.hamcrest.Matchers.arrayContainingInAnyOrder; import static org.hamcrest.Matchers.hasItem; diff --git a/x-pack/plugin/core/src/test/java/org/elasticsearch/xpack/core/DataTiersFeatureSetUsageTests.java b/x-pack/plugin/core/src/test/java/org/elasticsearch/xpack/core/DataTiersFeatureSetUsageTests.java index fee3e097f6a5e..1a2cc7c3f4980 100644 --- a/x-pack/plugin/core/src/test/java/org/elasticsearch/xpack/core/DataTiersFeatureSetUsageTests.java +++ b/x-pack/plugin/core/src/test/java/org/elasticsearch/xpack/core/DataTiersFeatureSetUsageTests.java @@ -7,6 +7,7 @@ package org.elasticsearch.xpack.core; +import org.elasticsearch.cluster.routing.allocation.DataTier; import org.elasticsearch.common.io.stream.Writeable; import org.elasticsearch.test.AbstractWireSerializingTestCase; diff --git a/x-pack/plugin/core/src/test/java/org/elasticsearch/xpack/core/DataTiersUsageTransportActionTests.java b/x-pack/plugin/core/src/test/java/org/elasticsearch/xpack/core/DataTiersUsageTransportActionTests.java index 6e47c0a77d4e1..a40bbb2d9ca0f 100644 --- a/x-pack/plugin/core/src/test/java/org/elasticsearch/xpack/core/DataTiersUsageTransportActionTests.java +++ b/x-pack/plugin/core/src/test/java/org/elasticsearch/xpack/core/DataTiersUsageTransportActionTests.java @@ -28,6 +28,7 @@ import org.elasticsearch.cluster.routing.ShardRouting; import org.elasticsearch.cluster.routing.ShardRoutingState; import org.elasticsearch.cluster.routing.TestShardRouting; +import org.elasticsearch.cluster.routing.allocation.DataTier; import org.elasticsearch.common.collect.ImmutableOpenMap; import org.elasticsearch.common.settings.Settings; import org.elasticsearch.core.PathUtils; @@ -40,7 +41,6 @@ import org.elasticsearch.indices.NodeIndicesStats; import org.elasticsearch.search.aggregations.metrics.TDigestState; import org.elasticsearch.test.ESTestCase; -import org.elasticsearch.xpack.cluster.routing.allocation.DataTierAllocationDecider; import java.nio.file.Path; import java.util.ArrayList; @@ -697,9 +697,9 @@ private static IndexMetadata indexMetadata(String indexName, int numberOfShards, for (int idx = 1; idx < dataTierPrefs.length; idx++) { tierBuilder.append(',').append(dataTierPrefs[idx]); } - settingsBuilder.put(DataTierAllocationDecider.TIER_PREFERENCE, tierBuilder.toString()); + settingsBuilder.put(DataTier.TIER_PREFERENCE, tierBuilder.toString()); } else if (dataTierPrefs.length == 1) { - settingsBuilder.put(DataTierAllocationDecider.TIER_PREFERENCE, dataTierPrefs[0]); + settingsBuilder.put(DataTier.TIER_PREFERENCE, dataTierPrefs[0]); } return IndexMetadata.builder(indexName) diff --git a/x-pack/plugin/core/src/test/java/org/elasticsearch/xpack/core/ilm/DataTierMigrationRoutedStepTests.java b/x-pack/plugin/core/src/test/java/org/elasticsearch/xpack/core/ilm/DataTierMigrationRoutedStepTests.java index 3e8acf8ad14c6..6add879cba1b0 100644 --- a/x-pack/plugin/core/src/test/java/org/elasticsearch/xpack/core/ilm/DataTierMigrationRoutedStepTests.java +++ b/x-pack/plugin/core/src/test/java/org/elasticsearch/xpack/core/ilm/DataTierMigrationRoutedStepTests.java @@ -19,7 +19,7 @@ import org.elasticsearch.cluster.routing.TestShardRouting; import org.elasticsearch.index.Index; import org.elasticsearch.index.shard.ShardId; -import org.elasticsearch.xpack.core.DataTier; +import org.elasticsearch.cluster.routing.allocation.DataTier; import org.elasticsearch.xpack.core.ilm.ClusterStateWaitStep.Result; import org.elasticsearch.xpack.core.ilm.Step.StepKey; import org.elasticsearch.xpack.core.ilm.step.info.AllocationInfo; @@ -28,7 +28,7 @@ import java.util.Set; import static java.util.Collections.emptyMap; -import static org.elasticsearch.xpack.cluster.routing.allocation.DataTierAllocationDecider.TIER_PREFERENCE; +import static org.elasticsearch.cluster.routing.allocation.DataTier.TIER_PREFERENCE; import static org.elasticsearch.xpack.core.ilm.CheckShrinkReadyStepTests.randomUnassignedInfo; import static org.elasticsearch.xpack.core.ilm.step.info.AllocationInfo.waitingForActiveShardsAllocationInfo; import static org.hamcrest.Matchers.is; diff --git a/x-pack/plugin/core/src/test/java/org/elasticsearch/xpack/core/ilm/MigrateActionTests.java b/x-pack/plugin/core/src/test/java/org/elasticsearch/xpack/core/ilm/MigrateActionTests.java index 839689c879cf4..eb0aa03c13420 100644 --- a/x-pack/plugin/core/src/test/java/org/elasticsearch/xpack/core/ilm/MigrateActionTests.java +++ b/x-pack/plugin/core/src/test/java/org/elasticsearch/xpack/core/ilm/MigrateActionTests.java @@ -11,9 +11,9 @@ import org.elasticsearch.cluster.ClusterState; import org.elasticsearch.cluster.metadata.IndexMetadata; import org.elasticsearch.cluster.metadata.Metadata; +import org.elasticsearch.cluster.routing.allocation.DataTier; import org.elasticsearch.common.io.stream.Writeable.Reader; import org.elasticsearch.common.xcontent.XContentParser; -import org.elasticsearch.xpack.cluster.routing.allocation.DataTierAllocationDecider; import org.elasticsearch.xpack.core.ilm.Step.StepKey; import java.io.IOException; @@ -21,14 +21,14 @@ import static org.elasticsearch.index.IndexModule.INDEX_STORE_TYPE_SETTING; import static org.elasticsearch.snapshots.SearchableSnapshotsSettings.SEARCHABLE_SNAPSHOT_STORE_TYPE; -import static org.elasticsearch.xpack.core.DataTier.DATA_COLD; -import static org.elasticsearch.xpack.core.DataTier.DATA_HOT; -import static org.elasticsearch.xpack.core.DataTier.DATA_WARM; +import static org.elasticsearch.cluster.routing.allocation.DataTier.DATA_COLD; +import static org.elasticsearch.cluster.routing.allocation.DataTier.DATA_HOT; +import static org.elasticsearch.cluster.routing.allocation.DataTier.DATA_WARM; import static org.elasticsearch.xpack.core.ilm.TimeseriesLifecycleType.COLD_PHASE; import static org.elasticsearch.xpack.core.ilm.TimeseriesLifecycleType.DELETE_PHASE; import static org.elasticsearch.xpack.core.ilm.TimeseriesLifecycleType.HOT_PHASE; import static org.elasticsearch.xpack.core.ilm.TimeseriesLifecycleType.WARM_PHASE; -import static org.elasticsearch.xpack.core.searchablesnapshots.SearchableSnapshotsConstants.SNAPSHOT_PARTIAL_SETTING; +import static org.elasticsearch.snapshots.SearchableSnapshotsSettings.SNAPSHOT_PARTIAL_SETTING; import static org.hamcrest.CoreMatchers.is; public class MigrateActionTests extends AbstractActionTestCase { @@ -89,19 +89,19 @@ public void testMigrateActionsConfiguresTierPreference() { { List steps = action.toSteps(null, HOT_PHASE, nextStepKey); UpdateSettingsStep firstStep = (UpdateSettingsStep) steps.get(1); - assertThat(DataTierAllocationDecider.TIER_PREFERENCE_SETTING.get(firstStep.getSettings()), + assertThat(DataTier.TIER_PREFERENCE_SETTING.get(firstStep.getSettings()), is(DATA_HOT)); } { List steps = action.toSteps(null, WARM_PHASE, nextStepKey); UpdateSettingsStep firstStep = (UpdateSettingsStep) steps.get(1); - assertThat(DataTierAllocationDecider.TIER_PREFERENCE_SETTING.get(firstStep.getSettings()), + assertThat(DataTier.TIER_PREFERENCE_SETTING.get(firstStep.getSettings()), is(DATA_WARM + "," + DATA_HOT)); } { List steps = action.toSteps(null, COLD_PHASE, nextStepKey); UpdateSettingsStep firstStep = (UpdateSettingsStep) steps.get(1); - assertThat(DataTierAllocationDecider.TIER_PREFERENCE_SETTING.get(firstStep.getSettings()), + assertThat(DataTier.TIER_PREFERENCE_SETTING.get(firstStep.getSettings()), is(DATA_COLD + "," + DATA_WARM + "," + DATA_HOT)); } } diff --git a/x-pack/plugin/core/src/test/java/org/elasticsearch/xpack/core/ilm/SearchableSnapshotActionTests.java b/x-pack/plugin/core/src/test/java/org/elasticsearch/xpack/core/ilm/SearchableSnapshotActionTests.java index 763073b7f3be6..c63254d3e0838 100644 --- a/x-pack/plugin/core/src/test/java/org/elasticsearch/xpack/core/ilm/SearchableSnapshotActionTests.java +++ b/x-pack/plugin/core/src/test/java/org/elasticsearch/xpack/core/ilm/SearchableSnapshotActionTests.java @@ -8,7 +8,7 @@ import org.elasticsearch.common.io.stream.Writeable; import org.elasticsearch.common.xcontent.XContentParser; -import org.elasticsearch.xpack.core.DataTier; +import org.elasticsearch.cluster.routing.allocation.DataTier; import org.elasticsearch.xpack.core.ilm.Step.StepKey; import org.elasticsearch.xpack.core.searchablesnapshots.MountSearchableSnapshotRequest; diff --git a/x-pack/plugin/core/src/test/java/org/elasticsearch/xpack/core/ilm/WaitForDataTierStepTests.java b/x-pack/plugin/core/src/test/java/org/elasticsearch/xpack/core/ilm/WaitForDataTierStepTests.java index 20f1cfbddb625..502c925f2bd77 100644 --- a/x-pack/plugin/core/src/test/java/org/elasticsearch/xpack/core/ilm/WaitForDataTierStepTests.java +++ b/x-pack/plugin/core/src/test/java/org/elasticsearch/xpack/core/ilm/WaitForDataTierStepTests.java @@ -15,7 +15,7 @@ import org.elasticsearch.cluster.node.DiscoveryNodes; import org.elasticsearch.common.Strings; import org.elasticsearch.common.UUIDs; -import org.elasticsearch.xpack.core.DataTier; +import org.elasticsearch.cluster.routing.allocation.DataTier; import java.util.Collection; import java.util.List; diff --git a/x-pack/plugin/core/src/test/java/org/elasticsearch/xpack/core/searchablesnapshots/SearchableSnapshotsConstantsTests.java b/x-pack/plugin/core/src/test/java/org/elasticsearch/xpack/core/searchablesnapshots/SearchableSnapshotsConstantsTests.java index a5d6966ca9abb..26e3516d23571 100644 --- a/x-pack/plugin/core/src/test/java/org/elasticsearch/xpack/core/searchablesnapshots/SearchableSnapshotsConstantsTests.java +++ b/x-pack/plugin/core/src/test/java/org/elasticsearch/xpack/core/searchablesnapshots/SearchableSnapshotsConstantsTests.java @@ -18,19 +18,19 @@ public class SearchableSnapshotsConstantsTests extends ESTestCase { public void testIsPartialSearchableSnapshotIndex() { - assertThat(SearchableSnapshotsConstants.isPartialSearchableSnapshotIndex( + assertThat(SearchableSnapshotsSettings.isPartialSearchableSnapshotIndex( Map.of(IndexModule.INDEX_STORE_TYPE_SETTING, SearchableSnapshotsSettings.SEARCHABLE_SNAPSHOT_STORE_TYPE, - SearchableSnapshotsConstants.SNAPSHOT_PARTIAL_SETTING, false)), + SearchableSnapshotsSettings.SNAPSHOT_PARTIAL_SETTING, false)), is(false)); - assertThat(SearchableSnapshotsConstants.isPartialSearchableSnapshotIndex( + assertThat(SearchableSnapshotsSettings.isPartialSearchableSnapshotIndex( Map.of(IndexModule.INDEX_STORE_TYPE_SETTING, "abc", - SearchableSnapshotsConstants.SNAPSHOT_PARTIAL_SETTING, randomBoolean())), + SearchableSnapshotsSettings.SNAPSHOT_PARTIAL_SETTING, randomBoolean())), is(false)); - assertThat(SearchableSnapshotsConstants.isPartialSearchableSnapshotIndex( + assertThat(SearchableSnapshotsSettings.isPartialSearchableSnapshotIndex( Map.of(IndexModule.INDEX_STORE_TYPE_SETTING, SearchableSnapshotsSettings.SEARCHABLE_SNAPSHOT_STORE_TYPE, - SearchableSnapshotsConstants.SNAPSHOT_PARTIAL_SETTING, true)), + SearchableSnapshotsSettings.SNAPSHOT_PARTIAL_SETTING, true)), is(true)); } } diff --git a/x-pack/plugin/data-streams/src/internalClusterTest/java/org/elasticsearch/xpack/datastreams/DataTierDataStreamIT.java b/x-pack/plugin/data-streams/src/internalClusterTest/java/org/elasticsearch/xpack/datastreams/DataTierDataStreamIT.java index ed163fbc6a7d1..619fcf7c3be01 100644 --- a/x-pack/plugin/data-streams/src/internalClusterTest/java/org/elasticsearch/xpack/datastreams/DataTierDataStreamIT.java +++ b/x-pack/plugin/data-streams/src/internalClusterTest/java/org/elasticsearch/xpack/datastreams/DataTierDataStreamIT.java @@ -10,11 +10,10 @@ import org.elasticsearch.action.admin.indices.template.put.PutComposableIndexTemplateAction; import org.elasticsearch.cluster.metadata.ComposableIndexTemplate; import org.elasticsearch.cluster.metadata.DataStream; +import org.elasticsearch.cluster.routing.allocation.DataTier; import org.elasticsearch.common.settings.Settings; import org.elasticsearch.plugins.Plugin; import org.elasticsearch.test.ESIntegTestCase; -import org.elasticsearch.xpack.cluster.routing.allocation.DataTierAllocationDecider; -import org.elasticsearch.xpack.core.DataTier; import org.elasticsearch.xpack.core.LocalStateCompositeXPackPlugin; import org.elasticsearch.xpack.core.action.DeleteDataStreamAction; @@ -60,7 +59,7 @@ public void testDefaultDataStreamAllocateToHot() { .get() .getSettings() .get(DataStream.getDefaultBackingIndexName(index, 1)); - assertThat(DataTierAllocationDecider.TIER_PREFERENCE_SETTING.get(idxSettings), equalTo(DataTier.DATA_HOT)); + assertThat(DataTier.TIER_PREFERENCE_SETTING.get(idxSettings), equalTo(DataTier.DATA_HOT)); logger.info("--> waiting for {} to be yellow", index); ensureYellow(index); @@ -74,7 +73,7 @@ public void testDefaultDataStreamAllocateToHot() { .get() .getSettings() .get(DataStream.getDefaultBackingIndexName(index, 2)); - assertThat(DataTierAllocationDecider.TIER_PREFERENCE_SETTING.get(idxSettings), equalTo(DataTier.DATA_HOT)); + assertThat(DataTier.TIER_PREFERENCE_SETTING.get(idxSettings), equalTo(DataTier.DATA_HOT)); client().execute(DeleteDataStreamAction.INSTANCE, new DeleteDataStreamAction.Request(new String[] { index })); } diff --git a/x-pack/plugin/ilm/qa/multi-node/src/javaRestTest/java/org/elasticsearch/xpack/MigrateToDataTiersIT.java b/x-pack/plugin/ilm/qa/multi-node/src/javaRestTest/java/org/elasticsearch/xpack/MigrateToDataTiersIT.java index 59ca2a95c7adb..ae3f3cb0d3d62 100644 --- a/x-pack/plugin/ilm/qa/multi-node/src/javaRestTest/java/org/elasticsearch/xpack/MigrateToDataTiersIT.java +++ b/x-pack/plugin/ilm/qa/multi-node/src/javaRestTest/java/org/elasticsearch/xpack/MigrateToDataTiersIT.java @@ -14,13 +14,13 @@ import org.elasticsearch.client.Response; import org.elasticsearch.client.ResponseException; import org.elasticsearch.cluster.metadata.IndexMetadata; +import org.elasticsearch.cluster.routing.allocation.DataTier; import org.elasticsearch.common.settings.Settings; import org.elasticsearch.core.TimeValue; import org.elasticsearch.rest.RestStatus; import org.elasticsearch.rest.action.admin.indices.RestPutIndexTemplateAction; import org.elasticsearch.test.rest.ESRestTestCase; import org.elasticsearch.xpack.cluster.action.MigrateToDataTiersResponse; -import org.elasticsearch.xpack.cluster.routing.allocation.DataTierAllocationDecider; import org.elasticsearch.xpack.core.ilm.AllocateAction; import org.elasticsearch.xpack.core.ilm.AllocationRoutedStep; import org.elasticsearch.xpack.core.ilm.DeleteAction; @@ -114,7 +114,7 @@ public void testMigrateToDataTiersAction() throws Exception { .put(IndexMetadata.SETTING_NUMBER_OF_SHARDS, 1) .put(IndexMetadata.SETTING_NUMBER_OF_REPLICAS, 0) .put(LifecycleSettings.LIFECYCLE_NAME, policy) - .putNull(DataTierAllocationDecider.TIER_PREFERENCE) + .putNull(DataTier.TIER_PREFERENCE) .put(RolloverAction.LIFECYCLE_ROLLOVER_ALIAS, alias) ); @@ -136,7 +136,7 @@ public void testMigrateToDataTiersAction() throws Exception { createIndexWithSettings(client(), rolloverIndexPrefix + "-00000" + i, alias + i, Settings.builder() .put(IndexMetadata.SETTING_NUMBER_OF_SHARDS, 1) .put(IndexMetadata.SETTING_NUMBER_OF_REPLICAS, 0) - .putNull(DataTierAllocationDecider.TIER_PREFERENCE) + .putNull(DataTier.TIER_PREFERENCE) .put(RolloverAction.LIFECYCLE_ROLLOVER_ALIAS, alias + i) ); } @@ -177,7 +177,7 @@ public void testMigrateToDataTiersAction() throws Exception { // let's assert the require.data:warm configuration the "indexWithDataWarmRouting" had was migrated to // _tier_preference:data_warm,data_hot Map indexSettings = getOnlyIndexSettings(client(), indexWithDataWarmRouting); - assertThat(indexSettings.get(DataTierAllocationDecider.TIER_PREFERENCE), is("data_warm,data_hot")); + assertThat(indexSettings.get(DataTier.TIER_PREFERENCE), is("data_warm,data_hot")); // let's retrieve the migrated policy and check it was migrated correctly - namely the warm phase should not contain any allocate // action anymore and the cold phase should contain an allocate action that only configures the number of replicas @@ -235,7 +235,7 @@ public void testMigrationDryRun() throws Exception { .put(IndexMetadata.SETTING_NUMBER_OF_SHARDS, 1) .put(IndexMetadata.SETTING_NUMBER_OF_REPLICAS, 0) .put(LifecycleSettings.LIFECYCLE_NAME, policy) - .putNull(DataTierAllocationDecider.TIER_PREFERENCE) + .putNull(DataTier.TIER_PREFERENCE) .put(RolloverAction.LIFECYCLE_ROLLOVER_ALIAS, alias) ); @@ -284,7 +284,7 @@ public void testMigrationDryRun() throws Exception { // the index settings should not contain the _tier_preference Map indexSettings = getOnlyIndexSettings(client(), indexWithDataWarmRouting); - assertThat(indexSettings.get(DataTierAllocationDecider.TIER_PREFERENCE), nullValue()); + assertThat(indexSettings.get(DataTier.TIER_PREFERENCE), nullValue()); // let's check the ILM policy was not migrated - ie. the warm phase still contains the allocate action Request getPolicy = new Request("GET", "/_ilm/policy/" + policy); diff --git a/x-pack/plugin/ilm/qa/multi-node/src/javaRestTest/java/org/elasticsearch/xpack/ilm/actions/SearchableSnapshotActionIT.java b/x-pack/plugin/ilm/qa/multi-node/src/javaRestTest/java/org/elasticsearch/xpack/ilm/actions/SearchableSnapshotActionIT.java index e8897fcd03738..e95633d046821 100644 --- a/x-pack/plugin/ilm/qa/multi-node/src/javaRestTest/java/org/elasticsearch/xpack/ilm/actions/SearchableSnapshotActionIT.java +++ b/x-pack/plugin/ilm/qa/multi-node/src/javaRestTest/java/org/elasticsearch/xpack/ilm/actions/SearchableSnapshotActionIT.java @@ -15,6 +15,7 @@ import org.elasticsearch.cluster.metadata.DataStream; import org.elasticsearch.cluster.metadata.IndexMetadata; import org.elasticsearch.cluster.metadata.Template; +import org.elasticsearch.cluster.routing.allocation.DataTier; import org.elasticsearch.common.Strings; import org.elasticsearch.common.settings.Settings; import org.elasticsearch.core.TimeValue; @@ -22,7 +23,6 @@ import org.elasticsearch.common.xcontent.XContentHelper; import org.elasticsearch.common.xcontent.XContentType; import org.elasticsearch.test.rest.ESRestTestCase; -import org.elasticsearch.xpack.cluster.routing.allocation.DataTierAllocationDecider; import org.elasticsearch.xpack.core.ilm.DeleteAction; import org.elasticsearch.xpack.core.ilm.ForceMergeAction; import org.elasticsearch.xpack.core.ilm.FreezeAction; @@ -497,7 +497,7 @@ snapshotRepo, randomBoolean())) indexDocument(client(), dataStream, true); String firstGenIndex = DataStream.getDefaultBackingIndexName(dataStream, 1L); Map indexSettings = getIndexSettingsAsMap(firstGenIndex); - assertThat(indexSettings.get(DataTierAllocationDecider.TIER_PREFERENCE), is("data_hot")); + assertThat(indexSettings.get(DataTier.TIER_PREFERENCE), is("data_hot")); // rollover the data stream so searchable_snapshot can complete rolloverMaxOneDocCondition(client(), dataStream); @@ -512,7 +512,7 @@ snapshotRepo, randomBoolean())) Map hotIndexSettings = getIndexSettingsAsMap(restoredIndex); // searchable snapshots mounted in the hot phase should be pinned to hot nodes - assertThat(hotIndexSettings.get(DataTierAllocationDecider.TIER_PREFERENCE), + assertThat(hotIndexSettings.get(DataTier.TIER_PREFERENCE), is("data_hot")); } } diff --git a/x-pack/plugin/ilm/qa/multi-node/src/javaRestTest/java/org/elasticsearch/xpack/ilm/actions/ShrinkActionIT.java b/x-pack/plugin/ilm/qa/multi-node/src/javaRestTest/java/org/elasticsearch/xpack/ilm/actions/ShrinkActionIT.java index a187a74af0a84..22933a3b8c7ce 100644 --- a/x-pack/plugin/ilm/qa/multi-node/src/javaRestTest/java/org/elasticsearch/xpack/ilm/actions/ShrinkActionIT.java +++ b/x-pack/plugin/ilm/qa/multi-node/src/javaRestTest/java/org/elasticsearch/xpack/ilm/actions/ShrinkActionIT.java @@ -12,6 +12,7 @@ import org.elasticsearch.client.Request; import org.elasticsearch.client.ResponseException; import org.elasticsearch.cluster.metadata.IndexMetadata; +import org.elasticsearch.cluster.routing.allocation.DataTier; import org.elasticsearch.cluster.routing.allocation.decider.ShardsLimitAllocationDecider; import org.elasticsearch.common.Strings; import org.elasticsearch.common.settings.Settings; @@ -20,7 +21,6 @@ import org.elasticsearch.common.xcontent.json.JsonXContent; import org.elasticsearch.rest.action.admin.indices.RestPutIndexTemplateAction; import org.elasticsearch.test.rest.ESRestTestCase; -import org.elasticsearch.xpack.cluster.routing.allocation.DataTierAllocationDecider; import org.elasticsearch.xpack.core.ilm.CheckTargetShardsCountStep; import org.elasticsearch.xpack.core.ilm.LifecycleAction; import org.elasticsearch.xpack.core.ilm.LifecyclePolicy; @@ -208,7 +208,7 @@ public void testSetSingleNodeAllocationRetriesUntilItSucceeds() throws Exception createIndexWithSettings(client(), index, alias, Settings.builder() .put(SETTING_NUMBER_OF_SHARDS, numShards) .put(IndexMetadata.SETTING_NUMBER_OF_REPLICAS, 0) - .putNull(DataTierAllocationDecider.TIER_PREFERENCE)); + .putNull(DataTier.TIER_PREFERENCE)); ensureGreen(index); diff --git a/x-pack/plugin/ilm/src/internalClusterTest/java/org/elasticsearch/xpack/ilm/DataTiersMigrationsTests.java b/x-pack/plugin/ilm/src/internalClusterTest/java/org/elasticsearch/xpack/ilm/DataTiersMigrationsTests.java index aae84b3a625db..6b39068111972 100644 --- a/x-pack/plugin/ilm/src/internalClusterTest/java/org/elasticsearch/xpack/ilm/DataTiersMigrationsTests.java +++ b/x-pack/plugin/ilm/src/internalClusterTest/java/org/elasticsearch/xpack/ilm/DataTiersMigrationsTests.java @@ -13,11 +13,11 @@ import org.elasticsearch.action.admin.indices.settings.put.UpdateSettingsRequest; import org.elasticsearch.cluster.node.DiscoveryNodeRole; import org.elasticsearch.cluster.routing.ShardRoutingState; +import org.elasticsearch.cluster.routing.allocation.DataTier; import org.elasticsearch.common.settings.Settings; import org.elasticsearch.core.TimeValue; import org.elasticsearch.plugins.Plugin; import org.elasticsearch.test.ESIntegTestCase; -import org.elasticsearch.xpack.cluster.routing.allocation.DataTierAllocationDecider; import org.elasticsearch.xpack.core.LocalStateCompositeXPackPlugin; import org.elasticsearch.xpack.core.XPackSettings; import org.elasticsearch.xpack.core.ilm.DataTierMigrationRoutedStep; @@ -190,7 +190,7 @@ public void testUserOptsOutOfTierMigration() throws Exception { assertReplicaIsUnassigned(); }, 30, TimeUnit.SECONDS); - Settings removeTierRoutingSetting = Settings.builder().putNull(DataTierAllocationDecider.TIER_PREFERENCE).build(); + Settings removeTierRoutingSetting = Settings.builder().putNull(DataTier.TIER_PREFERENCE).build(); UpdateSettingsRequest updateSettingsRequest = new UpdateSettingsRequest(managedIndex).settings(removeTierRoutingSetting); assertAcked(client().admin().indices().updateSettings(updateSettingsRequest).actionGet()); diff --git a/x-pack/plugin/ilm/src/main/java/org/elasticsearch/xpack/cluster/metadata/MetadataMigrateToDataTiersRoutingService.java b/x-pack/plugin/ilm/src/main/java/org/elasticsearch/xpack/cluster/metadata/MetadataMigrateToDataTiersRoutingService.java index 8989598e36fec..67d1ed5a4f1f7 100644 --- a/x-pack/plugin/ilm/src/main/java/org/elasticsearch/xpack/cluster/metadata/MetadataMigrateToDataTiersRoutingService.java +++ b/x-pack/plugin/ilm/src/main/java/org/elasticsearch/xpack/cluster/metadata/MetadataMigrateToDataTiersRoutingService.java @@ -21,7 +21,7 @@ import org.elasticsearch.core.Nullable; import org.elasticsearch.core.Tuple; import org.elasticsearch.license.XPackLicenseState; -import org.elasticsearch.xpack.core.DataTier; +import org.elasticsearch.cluster.routing.allocation.DataTier; import org.elasticsearch.xpack.core.ilm.AllocateAction; import org.elasticsearch.xpack.core.ilm.IndexLifecycleMetadata; import org.elasticsearch.xpack.core.ilm.LifecycleAction; @@ -48,7 +48,7 @@ import static org.elasticsearch.cluster.metadata.IndexMetadata.INDEX_ROUTING_EXCLUDE_GROUP_SETTING; import static org.elasticsearch.cluster.metadata.IndexMetadata.INDEX_ROUTING_INCLUDE_GROUP_SETTING; import static org.elasticsearch.cluster.metadata.IndexMetadata.INDEX_ROUTING_REQUIRE_GROUP_SETTING; -import static org.elasticsearch.xpack.cluster.routing.allocation.DataTierAllocationDecider.TIER_PREFERENCE; +import static org.elasticsearch.cluster.routing.allocation.DataTier.TIER_PREFERENCE; import static org.elasticsearch.xpack.core.ilm.LifecycleExecutionState.ILM_CUSTOM_METADATA_KEY; import static org.elasticsearch.xpack.core.ilm.OperationMode.STOPPED; import static org.elasticsearch.xpack.core.ilm.PhaseCacheManagement.updateIndicesForPolicy; diff --git a/x-pack/plugin/ilm/src/test/java/org/elasticsearch/xpack/cluster/metadata/MetadataMigrateToDataTiersRoutingServiceTests.java b/x-pack/plugin/ilm/src/test/java/org/elasticsearch/xpack/cluster/metadata/MetadataMigrateToDataTiersRoutingServiceTests.java index dc52f3280fc50..b5ffe6af619cc 100644 --- a/x-pack/plugin/ilm/src/test/java/org/elasticsearch/xpack/cluster/metadata/MetadataMigrateToDataTiersRoutingServiceTests.java +++ b/x-pack/plugin/ilm/src/test/java/org/elasticsearch/xpack/cluster/metadata/MetadataMigrateToDataTiersRoutingServiceTests.java @@ -48,12 +48,12 @@ import static org.elasticsearch.cluster.metadata.IndexMetadata.INDEX_ROUTING_EXCLUDE_GROUP_SETTING; import static org.elasticsearch.cluster.metadata.IndexMetadata.INDEX_ROUTING_INCLUDE_GROUP_SETTING; import static org.elasticsearch.cluster.metadata.IndexMetadata.INDEX_ROUTING_REQUIRE_GROUP_SETTING; +import static org.elasticsearch.cluster.routing.allocation.DataTier.TIER_PREFERENCE; import static org.elasticsearch.xpack.cluster.metadata.MetadataMigrateToDataTiersRoutingService.allocateActionDefinesRoutingRules; import static org.elasticsearch.xpack.cluster.metadata.MetadataMigrateToDataTiersRoutingService.convertAttributeValueToTierPreference; import static org.elasticsearch.xpack.cluster.metadata.MetadataMigrateToDataTiersRoutingService.migrateIlmPolicies; import static org.elasticsearch.xpack.cluster.metadata.MetadataMigrateToDataTiersRoutingService.migrateIndices; import static org.elasticsearch.xpack.cluster.metadata.MetadataMigrateToDataTiersRoutingService.migrateToDataTiersRouting; -import static org.elasticsearch.xpack.cluster.routing.allocation.DataTierAllocationDecider.TIER_PREFERENCE; import static org.elasticsearch.xpack.core.ilm.LifecycleExecutionState.ILM_CUSTOM_METADATA_KEY; import static org.hamcrest.Matchers.empty; import static org.hamcrest.Matchers.is; diff --git a/x-pack/plugin/searchable-snapshots/src/internalClusterTest/java/org/elasticsearch/xpack/searchablesnapshots/FrozenSearchableSnapshotsIntegTests.java b/x-pack/plugin/searchable-snapshots/src/internalClusterTest/java/org/elasticsearch/xpack/searchablesnapshots/FrozenSearchableSnapshotsIntegTests.java index 1d3dc9224483b..3adbd1d3f69f3 100644 --- a/x-pack/plugin/searchable-snapshots/src/internalClusterTest/java/org/elasticsearch/xpack/searchablesnapshots/FrozenSearchableSnapshotsIntegTests.java +++ b/x-pack/plugin/searchable-snapshots/src/internalClusterTest/java/org/elasticsearch/xpack/searchablesnapshots/FrozenSearchableSnapshotsIntegTests.java @@ -25,6 +25,7 @@ import org.elasticsearch.cluster.metadata.IndexMetadata; import org.elasticsearch.cluster.node.DiscoveryNode; import org.elasticsearch.cluster.routing.ShardRouting; +import org.elasticsearch.cluster.routing.allocation.DataTier; import org.elasticsearch.cluster.routing.allocation.decider.Decision; import org.elasticsearch.cluster.routing.allocation.decider.DiskThresholdDecider; import org.elasticsearch.common.Priority; @@ -46,12 +47,10 @@ import org.elasticsearch.indices.IndicesService; import org.elasticsearch.search.aggregations.bucket.histogram.DateHistogramInterval; import org.elasticsearch.search.aggregations.bucket.histogram.Histogram; +import org.elasticsearch.snapshots.SearchableSnapshotsSettings; import org.elasticsearch.snapshots.SnapshotInfo; -import org.elasticsearch.xpack.cluster.routing.allocation.DataTierAllocationDecider; -import org.elasticsearch.xpack.core.DataTier; import org.elasticsearch.xpack.core.searchablesnapshots.MountSearchableSnapshotAction; import org.elasticsearch.xpack.core.searchablesnapshots.MountSearchableSnapshotRequest; -import org.elasticsearch.xpack.core.searchablesnapshots.SearchableSnapshotsConstants; import org.elasticsearch.xpack.searchablesnapshots.action.SearchableSnapshotsStatsAction; import org.elasticsearch.xpack.searchablesnapshots.action.SearchableSnapshotsStatsRequest; @@ -299,8 +298,8 @@ public void testCreateAndRestorePartialSearchableSnapshot() throws Exception { assertTrue(SearchableSnapshots.SNAPSHOT_INDEX_ID_SETTING.exists(settings)); assertThat(IndexMetadata.INDEX_AUTO_EXPAND_REPLICAS_SETTING.get(settings).toString(), equalTo("false")); assertThat(IndexMetadata.INDEX_NUMBER_OF_REPLICAS_SETTING.get(settings), equalTo(expectedReplicas)); - assertThat(DataTierAllocationDecider.TIER_PREFERENCE_SETTING.get(settings), equalTo(expectedDataTiersPreference)); - assertTrue(SearchableSnapshotsConstants.SNAPSHOT_PARTIAL_SETTING.get(settings)); + assertThat(DataTier.TIER_PREFERENCE_SETTING.get(settings), equalTo(expectedDataTiersPreference)); + assertTrue(SearchableSnapshotsSettings.SNAPSHOT_PARTIAL_SETTING.get(settings)); assertTrue(DiskThresholdDecider.SETTING_IGNORE_DISK_WATERMARKS.get(settings)); assertThat(IndexSettings.INDEX_CHECK_ON_STARTUP.get(settings), equalTo(indexCheckOnStartup)); @@ -432,7 +431,7 @@ public void testCreateAndRestorePartialSearchableSnapshot() throws Exception { Settings.builder() .putNull(IndexModule.INDEX_STORE_TYPE_SETTING.getKey()) .putNull(IndexModule.INDEX_RECOVERY_TYPE_SETTING.getKey()) - .put(DataTierAllocationDecider.TIER_PREFERENCE, DataTier.DATA_HOT) + .put(DataTier.TIER_PREFERENCE, DataTier.DATA_HOT) .build() ) ); diff --git a/x-pack/plugin/searchable-snapshots/src/internalClusterTest/java/org/elasticsearch/xpack/searchablesnapshots/SearchableSnapshotsIntegTests.java b/x-pack/plugin/searchable-snapshots/src/internalClusterTest/java/org/elasticsearch/xpack/searchablesnapshots/SearchableSnapshotsIntegTests.java index 4db3e0c5e74ec..b4dda826f97f6 100644 --- a/x-pack/plugin/searchable-snapshots/src/internalClusterTest/java/org/elasticsearch/xpack/searchablesnapshots/SearchableSnapshotsIntegTests.java +++ b/x-pack/plugin/searchable-snapshots/src/internalClusterTest/java/org/elasticsearch/xpack/searchablesnapshots/SearchableSnapshotsIntegTests.java @@ -29,6 +29,7 @@ import org.elasticsearch.cluster.routing.ShardRouting; import org.elasticsearch.cluster.routing.allocation.AllocateUnassignedDecision; import org.elasticsearch.cluster.routing.allocation.AllocationDecision; +import org.elasticsearch.cluster.routing.allocation.DataTier; import org.elasticsearch.cluster.routing.allocation.NodeAllocationResult; import org.elasticsearch.cluster.routing.allocation.decider.Decision; import org.elasticsearch.common.Priority; @@ -49,8 +50,6 @@ import org.elasticsearch.snapshots.SnapshotId; import org.elasticsearch.snapshots.SnapshotInfo; import org.elasticsearch.snapshots.SnapshotsService; -import org.elasticsearch.xpack.cluster.routing.allocation.DataTierAllocationDecider; -import org.elasticsearch.xpack.core.DataTier; import org.elasticsearch.xpack.core.searchablesnapshots.MountSearchableSnapshotAction; import org.elasticsearch.xpack.core.searchablesnapshots.MountSearchableSnapshotRequest; import org.elasticsearch.xpack.core.searchablesnapshots.SearchableSnapshotShardStats; @@ -202,7 +201,7 @@ public void testCreateAndRestoreSearchableSnapshot() throws Exception { DataTier.ALL_DATA_TIERS.stream().filter(tier -> tier.equals(DataTier.DATA_FROZEN) == false).collect(Collectors.toSet()) ) ); - indexSettingsBuilder.put(DataTierAllocationDecider.TIER_PREFERENCE, expectedDataTiersPreference); + indexSettingsBuilder.put(DataTier.TIER_PREFERENCE, expectedDataTiersPreference); } else { expectedDataTiersPreference = MountSearchableSnapshotRequest.Storage.FULL_COPY.defaultDataTiersPreference(); } @@ -246,7 +245,7 @@ public void testCreateAndRestoreSearchableSnapshot() throws Exception { assertTrue(SearchableSnapshots.SNAPSHOT_INDEX_ID_SETTING.exists(settings)); assertThat(IndexMetadata.INDEX_AUTO_EXPAND_REPLICAS_SETTING.get(settings).toString(), equalTo("false")); assertThat(IndexMetadata.INDEX_NUMBER_OF_REPLICAS_SETTING.get(settings), equalTo(expectedReplicas)); - assertThat(DataTierAllocationDecider.TIER_PREFERENCE_SETTING.get(settings), equalTo(expectedDataTiersPreference)); + assertThat(DataTier.TIER_PREFERENCE_SETTING.get(settings), equalTo(expectedDataTiersPreference)); assertThat(IndexSettings.INDEX_CHECK_ON_STARTUP.get(settings), equalTo("false")); checkSoftDeletesNotEagerlyLoaded(restoredIndexName); diff --git a/x-pack/plugin/searchable-snapshots/src/internalClusterTest/java/org/elasticsearch/xpack/searchablesnapshots/SearchableSnapshotsResizeIntegTests.java b/x-pack/plugin/searchable-snapshots/src/internalClusterTest/java/org/elasticsearch/xpack/searchablesnapshots/SearchableSnapshotsResizeIntegTests.java index 334e49f03198f..193900329311e 100644 --- a/x-pack/plugin/searchable-snapshots/src/internalClusterTest/java/org/elasticsearch/xpack/searchablesnapshots/SearchableSnapshotsResizeIntegTests.java +++ b/x-pack/plugin/searchable-snapshots/src/internalClusterTest/java/org/elasticsearch/xpack/searchablesnapshots/SearchableSnapshotsResizeIntegTests.java @@ -8,12 +8,11 @@ package org.elasticsearch.xpack.searchablesnapshots; import org.elasticsearch.action.admin.indices.shrink.ResizeType; +import org.elasticsearch.cluster.routing.allocation.DataTier; import org.elasticsearch.common.settings.Settings; import org.elasticsearch.index.IndexModule; import org.elasticsearch.repositories.fs.FsRepository; import org.elasticsearch.test.ESIntegTestCase; -import org.elasticsearch.xpack.cluster.routing.allocation.DataTierAllocationDecider; -import org.elasticsearch.xpack.core.DataTier; import org.junit.After; import org.junit.Before; @@ -120,7 +119,7 @@ public void testCloneSearchableSnapshotIndex() { Settings.builder() .putNull(IndexModule.INDEX_STORE_TYPE_SETTING.getKey()) .putNull(IndexModule.INDEX_RECOVERY_TYPE_SETTING.getKey()) - .put(DataTierAllocationDecider.TIER_PREFERENCE, DataTier.DATA_HOT) + .put(DataTier.TIER_PREFERENCE, DataTier.DATA_HOT) .put(INDEX_NUMBER_OF_REPLICAS_SETTING.getKey(), 0) .build() ) diff --git a/x-pack/plugin/searchable-snapshots/src/internalClusterTest/java/org/elasticsearch/xpack/searchablesnapshots/allocation/SearchableSnapshotDataTierIntegTests.java b/x-pack/plugin/searchable-snapshots/src/internalClusterTest/java/org/elasticsearch/xpack/searchablesnapshots/allocation/SearchableSnapshotDataTierIntegTests.java index 89e902ac94fb3..e02018797322c 100644 --- a/x-pack/plugin/searchable-snapshots/src/internalClusterTest/java/org/elasticsearch/xpack/searchablesnapshots/allocation/SearchableSnapshotDataTierIntegTests.java +++ b/x-pack/plugin/searchable-snapshots/src/internalClusterTest/java/org/elasticsearch/xpack/searchablesnapshots/allocation/SearchableSnapshotDataTierIntegTests.java @@ -8,10 +8,9 @@ package org.elasticsearch.xpack.searchablesnapshots.allocation; import org.elasticsearch.action.admin.indices.settings.put.UpdateSettingsRequest; +import org.elasticsearch.cluster.routing.allocation.DataTier; import org.elasticsearch.common.settings.Settings; import org.elasticsearch.test.ESIntegTestCase; -import org.elasticsearch.xpack.cluster.routing.allocation.DataTierAllocationDecider; -import org.elasticsearch.xpack.core.DataTier; import org.elasticsearch.xpack.core.searchablesnapshots.MountSearchableSnapshotRequest; import org.elasticsearch.xpack.searchablesnapshots.BaseFrozenSearchableSnapshotsIntegTestCase; @@ -24,9 +23,7 @@ public class SearchableSnapshotDataTierIntegTests extends BaseFrozenSearchableSn private static final String indexName = "test-index"; private static final String snapshotName = "test-snapshot"; private static final String mountedIndexName = "test-index-mounted"; - private static final Settings frozenSettings = Settings.builder() - .put(DataTierAllocationDecider.TIER_PREFERENCE, DataTier.DATA_FROZEN) - .build(); + private static final Settings frozenSettings = Settings.builder().put(DataTier.TIER_PREFERENCE, DataTier.DATA_FROZEN).build(); public void testPartialLegalOnFrozen() throws Exception { createRepository(repoName, "fs"); @@ -63,10 +60,7 @@ public void testFullIllegalOnFrozen() throws Exception { Settings mountSettings = randomFrom( Settings.EMPTY, Settings.builder() - .put( - DataTierAllocationDecider.TIER_PREFERENCE, - randomValueOtherThan(DataTier.DATA_FROZEN, () -> randomFrom(DataTier.ALL_DATA_TIERS)) - ) + .put(DataTier.TIER_PREFERENCE, randomValueOtherThan(DataTier.DATA_FROZEN, () -> randomFrom(DataTier.ALL_DATA_TIERS))) .build() ); mountSnapshot(repoName, snapshotName, indexName, mountedIndexName, mountSettings, MountSearchableSnapshotRequest.Storage.FULL_COPY); @@ -77,7 +71,7 @@ public void testFullIllegalOnFrozen() throws Exception { private void updatePreference(String tier) { client().admin() .indices() - .updateSettings(new UpdateSettingsRequest(mountedIndexName).settings(Map.of(DataTierAllocationDecider.TIER_PREFERENCE, tier))) + .updateSettings(new UpdateSettingsRequest(mountedIndexName).settings(Map.of(DataTier.TIER_PREFERENCE, tier))) .actionGet(); } } diff --git a/x-pack/plugin/searchable-snapshots/src/internalClusterTest/java/org/elasticsearch/xpack/searchablesnapshots/cache/blob/SearchableSnapshotsBlobStoreCacheIntegTests.java b/x-pack/plugin/searchable-snapshots/src/internalClusterTest/java/org/elasticsearch/xpack/searchablesnapshots/cache/blob/SearchableSnapshotsBlobStoreCacheIntegTests.java index 3b07fb37eb8c4..ec639c641a9a8 100644 --- a/x-pack/plugin/searchable-snapshots/src/internalClusterTest/java/org/elasticsearch/xpack/searchablesnapshots/cache/blob/SearchableSnapshotsBlobStoreCacheIntegTests.java +++ b/x-pack/plugin/searchable-snapshots/src/internalClusterTest/java/org/elasticsearch/xpack/searchablesnapshots/cache/blob/SearchableSnapshotsBlobStoreCacheIntegTests.java @@ -18,6 +18,7 @@ import org.elasticsearch.cluster.routing.IndexRoutingTable; import org.elasticsearch.cluster.routing.RoutingNode; import org.elasticsearch.cluster.routing.ShardRouting; +import org.elasticsearch.cluster.routing.allocation.DataTier; import org.elasticsearch.cluster.routing.allocation.RoutingAllocation; import org.elasticsearch.cluster.routing.allocation.decider.AllocationDecider; import org.elasticsearch.cluster.routing.allocation.decider.Decision; @@ -40,7 +41,6 @@ import org.elasticsearch.snapshots.SearchableSnapshotsSettings; import org.elasticsearch.snapshots.SnapshotId; import org.elasticsearch.test.InternalTestCluster; -import org.elasticsearch.xpack.cluster.routing.allocation.DataTierAllocationDecider; import org.elasticsearch.xpack.core.ClientHelper; import org.elasticsearch.xpack.core.searchablesnapshots.MountSearchableSnapshotRequest.Storage; import org.elasticsearch.xpack.core.searchablesnapshots.SearchableSnapshotShardStats; @@ -206,7 +206,7 @@ public void testBlobStoreCache() throws Exception { .indices() .prepareGetSettings(SNAPSHOT_BLOB_CACHE_INDEX) .get() - .getSetting(SNAPSHOT_BLOB_CACHE_INDEX, DataTierAllocationDecider.TIER_PREFERENCE), + .getSetting(SNAPSHOT_BLOB_CACHE_INDEX, DataTier.TIER_PREFERENCE), equalTo("data_content,data_hot") ); diff --git a/x-pack/plugin/searchable-snapshots/src/internalClusterTest/java/org/elasticsearch/xpack/searchablesnapshots/cache/shared/PartiallyCachedShardAllocationIntegTests.java b/x-pack/plugin/searchable-snapshots/src/internalClusterTest/java/org/elasticsearch/xpack/searchablesnapshots/cache/shared/PartiallyCachedShardAllocationIntegTests.java index b6d7047414bab..d95f840d916e7 100644 --- a/x-pack/plugin/searchable-snapshots/src/internalClusterTest/java/org/elasticsearch/xpack/searchablesnapshots/cache/shared/PartiallyCachedShardAllocationIntegTests.java +++ b/x-pack/plugin/searchable-snapshots/src/internalClusterTest/java/org/elasticsearch/xpack/searchablesnapshots/cache/shared/PartiallyCachedShardAllocationIntegTests.java @@ -49,11 +49,11 @@ import java.util.function.Function; import java.util.stream.Collectors; +import static org.elasticsearch.cluster.routing.allocation.DataTier.TIER_PREFERENCE; import static org.elasticsearch.cluster.routing.allocation.decider.EnableAllocationDecider.CLUSTER_ROUTING_REBALANCE_ENABLE_SETTING; import static org.elasticsearch.index.IndexSettings.INDEX_SOFT_DELETES_SETTING; import static org.elasticsearch.test.NodeRoles.onlyRole; import static org.elasticsearch.test.hamcrest.ElasticsearchAssertions.assertAcked; -import static org.elasticsearch.xpack.cluster.routing.allocation.DataTierAllocationDecider.TIER_PREFERENCE; import static org.elasticsearch.xpack.searchablesnapshots.cache.shared.FrozenCacheService.SNAPSHOT_CACHE_SIZE_SETTING; import static org.hamcrest.Matchers.equalTo; import static org.hamcrest.Matchers.hasItem; diff --git a/x-pack/plugin/searchable-snapshots/src/main/java/org/elasticsearch/xpack/searchablesnapshots/SearchableSnapshots.java b/x-pack/plugin/searchable-snapshots/src/main/java/org/elasticsearch/xpack/searchablesnapshots/SearchableSnapshots.java index a16f95d4d9766..f63df97f72cf2 100644 --- a/x-pack/plugin/searchable-snapshots/src/main/java/org/elasticsearch/xpack/searchablesnapshots/SearchableSnapshots.java +++ b/x-pack/plugin/searchable-snapshots/src/main/java/org/elasticsearch/xpack/searchablesnapshots/SearchableSnapshots.java @@ -22,6 +22,7 @@ import org.elasticsearch.cluster.node.DiscoveryNode; import org.elasticsearch.cluster.node.DiscoveryNodes; import org.elasticsearch.cluster.routing.RerouteService; +import org.elasticsearch.cluster.routing.allocation.DataTier; import org.elasticsearch.cluster.routing.allocation.ExistingShardsAllocator; import org.elasticsearch.cluster.routing.allocation.decider.AllocationDecider; import org.elasticsearch.cluster.service.ClusterService; @@ -73,13 +74,10 @@ import org.elasticsearch.threadpool.ScalingExecutorBuilder; import org.elasticsearch.threadpool.ThreadPool; import org.elasticsearch.watcher.ResourceWatcherService; -import org.elasticsearch.xpack.cluster.routing.allocation.DataTierAllocationDecider; -import org.elasticsearch.xpack.core.DataTier; import org.elasticsearch.xpack.core.XPackPlugin; import org.elasticsearch.xpack.core.action.XPackInfoFeatureAction; import org.elasticsearch.xpack.core.action.XPackUsageFeatureAction; import org.elasticsearch.xpack.core.searchablesnapshots.MountSearchableSnapshotAction; -import org.elasticsearch.xpack.core.searchablesnapshots.SearchableSnapshotsConstants; import org.elasticsearch.xpack.searchablesnapshots.action.ClearSearchableSnapshotsCacheAction; import org.elasticsearch.xpack.searchablesnapshots.action.SearchableSnapshotsInfoTransportAction; import org.elasticsearch.xpack.searchablesnapshots.action.SearchableSnapshotsStatsAction; @@ -299,7 +297,7 @@ public List> getSettings() { SNAPSHOT_CACHE_EXCLUDED_FILE_TYPES_SETTING, SNAPSHOT_UNCACHED_CHUNK_SIZE_SETTING, DELETE_SEARCHABLE_SNAPSHOT_ON_INDEX_DELETION, - SearchableSnapshotsConstants.SNAPSHOT_PARTIAL_SETTING, + SearchableSnapshotsSettings.SNAPSHOT_PARTIAL_SETTING, SNAPSHOT_BLOB_CACHE_METADATA_FILES_MAX_LENGTH_SETTING, CacheService.SNAPSHOT_CACHE_RANGE_SIZE_SETTING, CacheService.SNAPSHOT_CACHE_RECOVERY_RANGE_SIZE_SETTING, @@ -598,7 +596,7 @@ private Settings getIndexSettings() { .put(IndexMetadata.SETTING_AUTO_EXPAND_REPLICAS, "0-1") .put(IndexMetadata.SETTING_PRIORITY, "900") .put(IndexSettings.INDEX_TRANSLOG_DURABILITY_SETTING.getKey(), Translog.Durability.ASYNC) - .put(DataTierAllocationDecider.TIER_PREFERENCE, DATA_TIERS_CACHE_INDEX_PREFERENCE) + .put(DataTier.TIER_PREFERENCE, DATA_TIERS_CACHE_INDEX_PREFERENCE) .build(); } diff --git a/x-pack/plugin/searchable-snapshots/src/main/java/org/elasticsearch/xpack/searchablesnapshots/action/SearchableSnapshotsUsageTransportAction.java b/x-pack/plugin/searchable-snapshots/src/main/java/org/elasticsearch/xpack/searchablesnapshots/action/SearchableSnapshotsUsageTransportAction.java index c1212fb5f8a05..abc602bc91da6 100644 --- a/x-pack/plugin/searchable-snapshots/src/main/java/org/elasticsearch/xpack/searchablesnapshots/action/SearchableSnapshotsUsageTransportAction.java +++ b/x-pack/plugin/searchable-snapshots/src/main/java/org/elasticsearch/xpack/searchablesnapshots/action/SearchableSnapshotsUsageTransportAction.java @@ -24,7 +24,6 @@ import org.elasticsearch.xpack.core.action.XPackUsageFeatureResponse; import org.elasticsearch.xpack.core.action.XPackUsageFeatureTransportAction; import org.elasticsearch.xpack.core.searchablesnapshots.SearchableSnapshotFeatureSetUsage; -import org.elasticsearch.xpack.core.searchablesnapshots.SearchableSnapshotsConstants; import static org.elasticsearch.xpack.core.searchablesnapshots.SearchableSnapshotsConstants.SEARCHABLE_SNAPSHOT_FEATURE; @@ -63,7 +62,7 @@ protected void masterOperation( int numSharedCacheSnapIndices = 0; for (IndexMetadata indexMetadata : state.metadata()) { if (SearchableSnapshotsSettings.isSearchableSnapshotStore(indexMetadata.getSettings())) { - if (SearchableSnapshotsConstants.SNAPSHOT_PARTIAL_SETTING.get(indexMetadata.getSettings())) { + if (SearchableSnapshotsSettings.SNAPSHOT_PARTIAL_SETTING.get(indexMetadata.getSettings())) { numSharedCacheSnapIndices++; } else { numFullCopySnapIndices++; diff --git a/x-pack/plugin/searchable-snapshots/src/main/java/org/elasticsearch/xpack/searchablesnapshots/action/TransportMountSearchableSnapshotAction.java b/x-pack/plugin/searchable-snapshots/src/main/java/org/elasticsearch/xpack/searchablesnapshots/action/TransportMountSearchableSnapshotAction.java index ce8f9edd7f76a..f95d580124e03 100644 --- a/x-pack/plugin/searchable-snapshots/src/main/java/org/elasticsearch/xpack/searchablesnapshots/action/TransportMountSearchableSnapshotAction.java +++ b/x-pack/plugin/searchable-snapshots/src/main/java/org/elasticsearch/xpack/searchablesnapshots/action/TransportMountSearchableSnapshotAction.java @@ -19,6 +19,7 @@ import org.elasticsearch.cluster.block.ClusterBlockException; import org.elasticsearch.cluster.metadata.IndexMetadata; import org.elasticsearch.cluster.metadata.IndexNameExpressionResolver; +import org.elasticsearch.cluster.routing.allocation.DataTier; import org.elasticsearch.cluster.routing.allocation.ExistingShardsAllocator; import org.elasticsearch.cluster.routing.allocation.decider.DiskThresholdDecider; import org.elasticsearch.cluster.service.ClusterService; @@ -35,14 +36,13 @@ import org.elasticsearch.repositories.RepositoriesService; import org.elasticsearch.repositories.Repository; import org.elasticsearch.repositories.RepositoryData; +import org.elasticsearch.snapshots.SearchableSnapshotsSettings; import org.elasticsearch.snapshots.SnapshotId; import org.elasticsearch.tasks.Task; import org.elasticsearch.threadpool.ThreadPool; import org.elasticsearch.transport.TransportService; -import org.elasticsearch.xpack.cluster.routing.allocation.DataTierAllocationDecider; import org.elasticsearch.xpack.core.searchablesnapshots.MountSearchableSnapshotAction; import org.elasticsearch.xpack.core.searchablesnapshots.MountSearchableSnapshotRequest; -import org.elasticsearch.xpack.core.searchablesnapshots.SearchableSnapshotsConstants; import org.elasticsearch.xpack.searchablesnapshots.SearchableSnapshots; import org.elasticsearch.xpack.searchablesnapshots.allocation.SearchableSnapshotAllocator; @@ -71,9 +71,7 @@ public class TransportMountSearchableSnapshotAction extends TransportMasterNodeA MountSearchableSnapshotRequest, RestoreSnapshotResponse> { - private static final Collection> DATA_TIER_ALLOCATION_SETTINGS = List.of( - DataTierAllocationDecider.TIER_PREFERENCE_SETTING - ); + private static final Collection> DATA_TIER_ALLOCATION_SETTINGS = List.of(DataTier.TIER_PREFERENCE_SETTING); private final Client client; private final RepositoriesService repositoriesService; @@ -147,7 +145,7 @@ private static Settings buildIndexSettings( if (minNodeVersion.before(Version.V_7_12_0)) { throw new IllegalArgumentException("shared cache searchable snapshots require minimum node version " + Version.V_7_12_0); } - settings.put(SearchableSnapshotsConstants.SNAPSHOT_PARTIAL_SETTING.getKey(), true) + settings.put(SearchableSnapshotsSettings.SNAPSHOT_PARTIAL_SETTING.getKey(), true) .put(DiskThresholdDecider.SETTING_IGNORE_DISK_WATERMARKS.getKey(), true); // we cannot apply this setting during rolling upgrade. @@ -233,7 +231,7 @@ protected void masterOperation( .put(IndexMetadata.SETTING_NUMBER_OF_REPLICAS, 0) // can be overridden .put(IndexMetadata.SETTING_AUTO_EXPAND_REPLICAS, false) // can be overridden .put(IndexSettings.INDEX_CHECK_ON_STARTUP.getKey(), false) // can be overridden - .put(DataTierAllocationDecider.TIER_PREFERENCE, request.storage().defaultDataTiersPreference()) + .put(DataTier.TIER_PREFERENCE, request.storage().defaultDataTiersPreference()) .put(request.indexSettings()) .put( buildIndexSettings( diff --git a/x-pack/plugin/searchable-snapshots/src/main/java/org/elasticsearch/xpack/searchablesnapshots/action/cache/TransportSearchableSnapshotCacheStoresAction.java b/x-pack/plugin/searchable-snapshots/src/main/java/org/elasticsearch/xpack/searchablesnapshots/action/cache/TransportSearchableSnapshotCacheStoresAction.java index f5f7e02ae8a1e..5ac0decd23761 100644 --- a/x-pack/plugin/searchable-snapshots/src/main/java/org/elasticsearch/xpack/searchablesnapshots/action/cache/TransportSearchableSnapshotCacheStoresAction.java +++ b/x-pack/plugin/searchable-snapshots/src/main/java/org/elasticsearch/xpack/searchablesnapshots/action/cache/TransportSearchableSnapshotCacheStoresAction.java @@ -32,7 +32,7 @@ import java.util.List; import java.util.Optional; -import static org.elasticsearch.xpack.core.searchablesnapshots.SearchableSnapshotsConstants.SNAPSHOT_PARTIAL_SETTING; +import static org.elasticsearch.snapshots.SearchableSnapshotsSettings.SNAPSHOT_PARTIAL_SETTING; public class TransportSearchableSnapshotCacheStoresAction extends TransportNodesAction< TransportSearchableSnapshotCacheStoresAction.Request, diff --git a/x-pack/plugin/searchable-snapshots/src/main/java/org/elasticsearch/xpack/searchablesnapshots/allocation/SearchableSnapshotAllocator.java b/x-pack/plugin/searchable-snapshots/src/main/java/org/elasticsearch/xpack/searchablesnapshots/allocation/SearchableSnapshotAllocator.java index 3490cd6315529..9e46d7abe8361 100644 --- a/x-pack/plugin/searchable-snapshots/src/main/java/org/elasticsearch/xpack/searchablesnapshots/allocation/SearchableSnapshotAllocator.java +++ b/x-pack/plugin/searchable-snapshots/src/main/java/org/elasticsearch/xpack/searchablesnapshots/allocation/SearchableSnapshotAllocator.java @@ -64,7 +64,7 @@ import java.util.stream.StreamSupport; import static java.util.Collections.emptyList; -import static org.elasticsearch.xpack.core.searchablesnapshots.SearchableSnapshotsConstants.SNAPSHOT_PARTIAL_SETTING; +import static org.elasticsearch.snapshots.SearchableSnapshotsSettings.SNAPSHOT_PARTIAL_SETTING; import static org.elasticsearch.xpack.searchablesnapshots.SearchableSnapshots.SNAPSHOT_INDEX_ID_SETTING; import static org.elasticsearch.xpack.searchablesnapshots.SearchableSnapshots.SNAPSHOT_INDEX_NAME_SETTING; import static org.elasticsearch.xpack.searchablesnapshots.SearchableSnapshots.SNAPSHOT_REPOSITORY_NAME_SETTING; diff --git a/x-pack/plugin/searchable-snapshots/src/main/java/org/elasticsearch/xpack/searchablesnapshots/cache/shared/FrozenCacheService.java b/x-pack/plugin/searchable-snapshots/src/main/java/org/elasticsearch/xpack/searchablesnapshots/cache/shared/FrozenCacheService.java index 3ca5a09e0b0c6..95c3a72e736d7 100644 --- a/x-pack/plugin/searchable-snapshots/src/main/java/org/elasticsearch/xpack/searchablesnapshots/cache/shared/FrozenCacheService.java +++ b/x-pack/plugin/searchable-snapshots/src/main/java/org/elasticsearch/xpack/searchablesnapshots/cache/shared/FrozenCacheService.java @@ -15,6 +15,7 @@ import org.elasticsearch.action.StepListener; import org.elasticsearch.cluster.node.DiscoveryNode; import org.elasticsearch.cluster.node.DiscoveryNodeRole; +import org.elasticsearch.cluster.routing.allocation.DataTier; import org.elasticsearch.common.settings.Setting; import org.elasticsearch.common.settings.Settings; import org.elasticsearch.common.settings.SettingsException; @@ -34,7 +35,6 @@ import org.elasticsearch.monitor.fs.FsProbe; import org.elasticsearch.node.NodeRoleSettings; import org.elasticsearch.threadpool.ThreadPool; -import org.elasticsearch.xpack.core.DataTier; import org.elasticsearch.xpack.searchablesnapshots.cache.common.ByteRange; import org.elasticsearch.xpack.searchablesnapshots.cache.common.CacheKey; import org.elasticsearch.xpack.searchablesnapshots.cache.common.SparseFileTracker; diff --git a/x-pack/plugin/searchable-snapshots/src/main/java/org/elasticsearch/xpack/searchablesnapshots/store/SearchableSnapshotDirectory.java b/x-pack/plugin/searchable-snapshots/src/main/java/org/elasticsearch/xpack/searchablesnapshots/store/SearchableSnapshotDirectory.java index d1c3c65e78b83..23ae7722cfb09 100644 --- a/x-pack/plugin/searchable-snapshots/src/main/java/org/elasticsearch/xpack/searchablesnapshots/store/SearchableSnapshotDirectory.java +++ b/x-pack/plugin/searchable-snapshots/src/main/java/org/elasticsearch/xpack/searchablesnapshots/store/SearchableSnapshotDirectory.java @@ -88,7 +88,7 @@ import static org.elasticsearch.index.IndexModule.INDEX_STORE_TYPE_SETTING; import static org.elasticsearch.snapshots.SearchableSnapshotsSettings.SEARCHABLE_SNAPSHOT_STORE_TYPE; -import static org.elasticsearch.xpack.core.searchablesnapshots.SearchableSnapshotsConstants.SNAPSHOT_PARTIAL_SETTING; +import static org.elasticsearch.snapshots.SearchableSnapshotsSettings.SNAPSHOT_PARTIAL_SETTING; import static org.elasticsearch.xpack.searchablesnapshots.SearchableSnapshots.SNAPSHOT_BLOB_CACHE_METADATA_FILES_MAX_LENGTH_SETTING; import static org.elasticsearch.xpack.searchablesnapshots.SearchableSnapshots.SNAPSHOT_CACHE_ENABLED_SETTING; import static org.elasticsearch.xpack.searchablesnapshots.SearchableSnapshots.SNAPSHOT_CACHE_EXCLUDED_FILE_TYPES_SETTING; diff --git a/x-pack/plugin/searchable-snapshots/src/test/java/org/elasticsearch/xpack/searchablesnapshots/allocation/SearchableSnapshotAllocatorTests.java b/x-pack/plugin/searchable-snapshots/src/test/java/org/elasticsearch/xpack/searchablesnapshots/allocation/SearchableSnapshotAllocatorTests.java index bfc60dff44848..c9e220fe289ae 100644 --- a/x-pack/plugin/searchable-snapshots/src/test/java/org/elasticsearch/xpack/searchablesnapshots/allocation/SearchableSnapshotAllocatorTests.java +++ b/x-pack/plugin/searchable-snapshots/src/test/java/org/elasticsearch/xpack/searchablesnapshots/allocation/SearchableSnapshotAllocatorTests.java @@ -53,7 +53,7 @@ import java.util.function.UnaryOperator; import java.util.stream.Collectors; -import static org.elasticsearch.xpack.core.searchablesnapshots.SearchableSnapshotsConstants.SNAPSHOT_PARTIAL_SETTING; +import static org.elasticsearch.snapshots.SearchableSnapshotsSettings.SNAPSHOT_PARTIAL_SETTING; import static org.hamcrest.Matchers.empty; public class SearchableSnapshotAllocatorTests extends ESAllocationTestCase { diff --git a/x-pack/plugin/searchable-snapshots/src/test/java/org/elasticsearch/xpack/searchablesnapshots/store/SearchableSnapshotDirectoryStatsTests.java b/x-pack/plugin/searchable-snapshots/src/test/java/org/elasticsearch/xpack/searchablesnapshots/store/SearchableSnapshotDirectoryStatsTests.java index 8c2673955d151..515322b315b51 100644 --- a/x-pack/plugin/searchable-snapshots/src/test/java/org/elasticsearch/xpack/searchablesnapshots/store/SearchableSnapshotDirectoryStatsTests.java +++ b/x-pack/plugin/searchable-snapshots/src/test/java/org/elasticsearch/xpack/searchablesnapshots/store/SearchableSnapshotDirectoryStatsTests.java @@ -31,9 +31,9 @@ import org.elasticsearch.index.store.StoreFileMetadata; import org.elasticsearch.indices.recovery.RecoveryState; import org.elasticsearch.repositories.IndexId; +import org.elasticsearch.snapshots.SearchableSnapshotsSettings; import org.elasticsearch.snapshots.Snapshot; import org.elasticsearch.snapshots.SnapshotId; -import org.elasticsearch.xpack.core.searchablesnapshots.SearchableSnapshotsConstants; import org.elasticsearch.xpack.searchablesnapshots.AbstractSearchableSnapshotsTestCase; import org.elasticsearch.xpack.searchablesnapshots.cache.common.TestUtils; import org.elasticsearch.xpack.searchablesnapshots.cache.common.TestUtils.NoopBlobStoreCacheService; @@ -197,7 +197,7 @@ public void testDirectBytesReadsWithCache() throws Exception { createFrozenCacheService(ByteSizeValue.ZERO, randomFrozenCacheRangeSize()), Settings.builder() .put(SNAPSHOT_CACHE_ENABLED_SETTING.getKey(), true) - .put(SearchableSnapshotsConstants.SNAPSHOT_PARTIAL_SETTING.getKey(), true) + .put(SearchableSnapshotsSettings.SNAPSHOT_PARTIAL_SETTING.getKey(), true) .build(), (fileName, fileContent, directory) -> { assertThat(directory.getStats(fileName), nullValue()); @@ -553,7 +553,7 @@ private void executeTestCase(final TriConsumer