Introduce a Few Settings Singleton Instances (#78897) (#78908)

This is mostly motivated by the ILM steps changes, that showed up as hot in
profiling since instantiating a `Settings` instance turns out to be somewhat expensive.
Also cleans up a couple of other spots and some duplication as well.
This commit is contained in:
Armin Braun 2021-10-11 12:20:40 +02:00 committed by GitHub
parent 0989704f28
commit 7630301bd3
No known key found for this signature in database
GPG key ID: 4AEE18F83AFDEB23
33 changed files with 101 additions and 74 deletions

View file

@ -36,8 +36,6 @@ import java.util.Map;
import java.util.Objects;
import java.util.Set;
import static org.elasticsearch.common.settings.Settings.Builder.EMPTY_SETTINGS;
/**
* A request to create an index.
*/
@ -47,7 +45,7 @@ public class CreateIndexRequest extends TimedRequest implements Validatable, ToX
static final ParseField ALIASES = new ParseField("aliases");
private final String index;
private Settings settings = EMPTY_SETTINGS;
private Settings settings = Settings.EMPTY;
private BytesReference mappings;
private XContentType mappingsXContentType;

View file

@ -178,7 +178,7 @@ public class IndexTemplateMetadata {
private List<String> indexPatterns;
private Settings settings = Settings.Builder.EMPTY_SETTINGS;
private Settings settings = Settings.EMPTY;
private MappingMetadata mappings;

View file

@ -40,7 +40,6 @@ import java.util.Set;
import java.util.stream.Collectors;
import static org.elasticsearch.action.ValidateActions.addValidationError;
import static org.elasticsearch.common.settings.Settings.Builder.EMPTY_SETTINGS;
/**
* A request to create an index template.
@ -66,7 +65,7 @@ public class PutIndexTemplateRequest extends MasterNodeRequest<PutIndexTemplateR
private boolean create;
private Settings settings = EMPTY_SETTINGS;
private Settings settings = Settings.EMPTY;
private BytesReference mappings = null;

View file

@ -29,7 +29,7 @@ public class SimpleRecoveryIT extends ESIntegTestCase {
}
protected Settings recoverySettings() {
return Settings.Builder.EMPTY_SETTINGS;
return Settings.EMPTY;
}
@Override

View file

@ -21,7 +21,6 @@ import java.io.IOException;
import java.util.Map;
import static org.elasticsearch.action.ValidateActions.addValidationError;
import static org.elasticsearch.common.settings.Settings.Builder.EMPTY_SETTINGS;
import static org.elasticsearch.common.settings.Settings.readSettingsFromStream;
import static org.elasticsearch.common.settings.Settings.writeSettingsToStream;
@ -39,7 +38,7 @@ public class PutRepositoryRequest extends AcknowledgedRequest<PutRepositoryReque
private boolean verify = true;
private Settings settings = EMPTY_SETTINGS;
private Settings settings = Settings.EMPTY;
public PutRepositoryRequest(StreamInput in) throws IOException {
super(in);

View file

@ -26,7 +26,6 @@ import java.util.Map;
import static org.elasticsearch.action.ValidateActions.addValidationError;
import static org.elasticsearch.common.settings.Settings.readSettingsFromStream;
import static org.elasticsearch.common.settings.Settings.writeSettingsToStream;
import static org.elasticsearch.common.settings.Settings.Builder.EMPTY_SETTINGS;
/**
* Request for an update cluster settings action
@ -44,8 +43,8 @@ public class ClusterUpdateSettingsRequest extends AcknowledgedRequest<ClusterUpd
PARSER.declareObject((r, t) -> r.transientSettings = t, (p, c) -> Settings.fromXContent(p), TRANSIENT);
}
private Settings transientSettings = EMPTY_SETTINGS;
private Settings persistentSettings = EMPTY_SETTINGS;
private Settings transientSettings = Settings.EMPTY;
private Settings persistentSettings = Settings.EMPTY;
public ClusterUpdateSettingsRequest(StreamInput in) throws IOException {
super(in);

View file

@ -33,7 +33,6 @@ import java.util.Objects;
import static org.elasticsearch.action.ValidateActions.addValidationError;
import static org.elasticsearch.common.Strings.EMPTY_ARRAY;
import static org.elasticsearch.common.settings.Settings.Builder.EMPTY_SETTINGS;
import static org.elasticsearch.common.settings.Settings.readSettingsFromStream;
import static org.elasticsearch.common.settings.Settings.writeSettingsToStream;
import static org.elasticsearch.common.xcontent.support.XContentMapValues.nodeBooleanValue;
@ -70,7 +69,7 @@ public class CreateSnapshotRequest extends MasterNodeRequest<CreateSnapshotReque
private boolean partial = false;
private Settings settings = EMPTY_SETTINGS;
private Settings settings = Settings.EMPTY;
private boolean includeGlobalState = true;

View file

@ -31,7 +31,6 @@ import java.util.Map;
import java.util.Objects;
import static org.elasticsearch.action.ValidateActions.addValidationError;
import static org.elasticsearch.common.settings.Settings.Builder.EMPTY_SETTINGS;
import static org.elasticsearch.common.settings.Settings.readSettingsFromStream;
import static org.elasticsearch.common.settings.Settings.writeSettingsToStream;
import static org.elasticsearch.common.xcontent.support.XContentMapValues.nodeBooleanValue;
@ -55,7 +54,7 @@ public class RestoreSnapshotRequest extends MasterNodeRequest<RestoreSnapshotReq
private boolean includeGlobalState = false;
private boolean partial = false;
private boolean includeAliases = true;
private Settings indexSettings = EMPTY_SETTINGS;
private Settings indexSettings = Settings.EMPTY;
private String[] ignoreIndexSettings = Strings.EMPTY_ARRAY;
// This field does not get serialised (except toString for debugging purpose) because it is always set locally by authz

View file

@ -38,7 +38,7 @@ public class CreateIndexClusterStateUpdateRequest extends ClusterStateUpdateRequ
private boolean copySettings;
private SystemDataStreamDescriptor systemDataStreamDescriptor;
private Settings settings = Settings.Builder.EMPTY_SETTINGS;
private Settings settings = Settings.EMPTY;
private final Map<String, String> mappings = new HashMap<>();

View file

@ -45,7 +45,6 @@ import java.util.Objects;
import java.util.Set;
import static org.elasticsearch.action.ValidateActions.addValidationError;
import static org.elasticsearch.common.settings.Settings.Builder.EMPTY_SETTINGS;
import static org.elasticsearch.common.settings.Settings.readSettingsFromStream;
import static org.elasticsearch.common.settings.Settings.writeSettingsToStream;
@ -68,7 +67,7 @@ public class CreateIndexRequest extends AcknowledgedRequest<CreateIndexRequest>
private String index;
private Settings settings = EMPTY_SETTINGS;
private Settings settings = Settings.EMPTY;
private final Map<String, String> mappings = new HashMap<>();
@ -127,7 +126,7 @@ public class CreateIndexRequest extends AcknowledgedRequest<CreateIndexRequest>
* @param index the name of the index
*/
public CreateIndexRequest(String index) {
this(index, EMPTY_SETTINGS);
this(index, Settings.EMPTY);
}
/**

View file

@ -55,6 +55,8 @@ public class MetadataRolloverService {
private static final Pattern INDEX_NAME_PATTERN = Pattern.compile("^.*-\\d+$");
private static final List<IndexAbstraction.Type> VALID_ROLLOVER_TARGETS = org.elasticsearch.core.List.of(ALIAS, DATA_STREAM);
public static final Settings HIDDEN_INDEX_SETTINGS = Settings.builder().put(IndexMetadata.SETTING_INDEX_HIDDEN, true).build();
private final ThreadPool threadPool;
private final MetadataCreateIndexService createIndexService;
private final MetadataIndexAliasesService indexAliasesService;
@ -276,7 +278,7 @@ public class MetadataRolloverService {
final String targetIndexName,
CreateIndexRequest createIndexRequest,
final SystemDataStreamDescriptor descriptor) {
Settings settings = descriptor != null ? Settings.EMPTY : Settings.builder().put("index.hidden", true).build();
Settings settings = descriptor != null ? Settings.EMPTY : HIDDEN_INDEX_SETTINGS;
return prepareCreateIndexRequest(targetIndexName, targetIndexName, "rollover_data_stream", createIndexRequest, settings)
.dataStreamName(dataStreamName)
.systemDataStreamDescriptor(descriptor);

View file

@ -31,7 +31,6 @@ import java.util.Objects;
import static org.elasticsearch.action.ValidateActions.addValidationError;
import static org.elasticsearch.common.settings.Settings.readSettingsFromStream;
import static org.elasticsearch.common.settings.Settings.writeSettingsToStream;
import static org.elasticsearch.common.settings.Settings.Builder.EMPTY_SETTINGS;
/**
* Request for an update index settings action
@ -43,7 +42,7 @@ public class UpdateSettingsRequest extends AcknowledgedRequest<UpdateSettingsReq
private String[] indices;
private IndicesOptions indicesOptions = DEFAULT_INDICES_OPTIONS;
private Settings settings = EMPTY_SETTINGS;
private Settings settings = Settings.EMPTY;
private boolean preserveExisting = false;
private String origin = "";

View file

@ -48,7 +48,6 @@ import java.util.Set;
import java.util.stream.Collectors;
import static org.elasticsearch.action.ValidateActions.addValidationError;
import static org.elasticsearch.common.settings.Settings.Builder.EMPTY_SETTINGS;
import static org.elasticsearch.common.settings.Settings.readSettingsFromStream;
import static org.elasticsearch.common.settings.Settings.writeSettingsToStream;
@ -69,7 +68,7 @@ public class PutIndexTemplateRequest extends MasterNodeRequest<PutIndexTemplateR
private boolean create;
private Settings settings = EMPTY_SETTINGS;
private Settings settings = Settings.EMPTY;
private Map<String, String> mappings = new HashMap<>();

View file

@ -1069,7 +1069,7 @@ public class IndexMetadata implements Diffable<IndexMetadata>, ToXContentFragmen
private long settingsVersion = 1;
private long aliasesVersion = 1;
private long[] primaryTerms = null;
private Settings settings = Settings.Builder.EMPTY_SETTINGS;
private Settings settings = Settings.EMPTY;
private final ImmutableOpenMap.Builder<String, MappingMetadata> mappings;
private final ImmutableOpenMap.Builder<String, AliasMetadata> aliases;
private final ImmutableOpenMap.Builder<String, DiffableStringMap> customMetadata;

View file

@ -254,7 +254,7 @@ public class IndexTemplateMetadata extends AbstractDiffable<IndexTemplateMetadat
private List<String> indexPatterns;
private Settings settings = Settings.Builder.EMPTY_SETTINGS;
private Settings settings = Settings.EMPTY;
private final ImmutableOpenMap.Builder<String, CompressedXContent> mappings;

View file

@ -1151,8 +1151,8 @@ public class Metadata implements Iterable<IndexMetadata>, Diffable<Metadata>, To
private long version;
private CoordinationMetadata coordinationMetadata = CoordinationMetadata.EMPTY_METADATA;
private Settings transientSettings = Settings.Builder.EMPTY_SETTINGS;
private Settings persistentSettings = Settings.Builder.EMPTY_SETTINGS;
private Settings transientSettings = Settings.EMPTY;
private Settings persistentSettings = Settings.EMPTY;
private DiffableStringMap hashesOfConsistentSettings = DiffableStringMap.EMPTY;
private final ImmutableOpenMap.Builder<String, IndexMetadata> indices;

View file

@ -14,6 +14,7 @@ import org.elasticsearch.ResourceAlreadyExistsException;
import org.elasticsearch.Version;
import org.elasticsearch.action.ActionListener;
import org.elasticsearch.action.admin.indices.create.CreateIndexClusterStateUpdateRequest;
import org.elasticsearch.action.admin.indices.rollover.MetadataRolloverService;
import org.elasticsearch.action.support.ActiveShardCount;
import org.elasticsearch.action.support.ActiveShardsObserver;
import org.elasticsearch.action.support.master.AcknowledgedResponse;
@ -23,7 +24,6 @@ import org.elasticsearch.cluster.ack.ClusterStateUpdateRequest;
import org.elasticsearch.cluster.service.ClusterService;
import org.elasticsearch.common.Priority;
import org.elasticsearch.common.Strings;
import org.elasticsearch.common.settings.Settings;
import org.elasticsearch.core.TimeValue;
import org.elasticsearch.index.Index;
import org.elasticsearch.index.mapper.DataStreamTimestampFieldMapper;
@ -202,7 +202,7 @@ public class MetadataCreateDataStreamService {
.systemDataStreamDescriptor(systemDataStreamDescriptor);
if (isSystem == false) {
createIndexRequest.settings(Settings.builder().put("index.hidden", true).build());
createIndexRequest.settings(MetadataRolloverService.HIDDEN_INDEX_SETTINGS);
}
try {

View file

@ -1417,7 +1417,7 @@ public class MetadataIndexTemplateService {
int order;
Integer version;
List<String> indexPatterns;
Settings settings = Settings.Builder.EMPTY_SETTINGS;
Settings settings = Settings.EMPTY;
Map<String, String> mappings = new HashMap<>();
List<Alias> aliases = new ArrayList<>();

View file

@ -21,12 +21,12 @@ import org.elasticsearch.index.shard.IndexSettingProvider;
import org.elasticsearch.snapshots.SearchableSnapshotsSettings;
import java.util.Collection;
import java.util.HashMap;
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;
/**
* The {@code DataTier} class encapsulates the formalization of the "content",
@ -49,6 +49,12 @@ public class DataTier {
public static final Setting.Validator<String> DATA_TIER_SETTING_VALIDATOR = new DataTierSettingValidator();
private static final Settings DATA_CONTENT_TIER_PREFERENCE_SETTINGS = Settings.builder().put(TIER_PREFERENCE, DATA_CONTENT).build();
private static final Settings DATA_HOT_TIER_PREFERENCE_SETTINGS = Settings.builder().put(TIER_PREFERENCE, DATA_HOT).build();
private static final Settings NULL_TIER_PREFERENCE_SETTINGS = Settings.builder().putNull(TIER_PREFERENCE).build();
public static final Setting<String> TIER_PREFERENCE_SETTING = new Setting<>(
new Setting.SimpleKey(TIER_PREFERENCE),
DataTierSettingValidator::getDefaultTierPreference,
@ -70,6 +76,24 @@ public class DataTier {
private static final List<String> ORDERED_FROZEN_TO_HOT_TIERS =
org.elasticsearch.core.List.of(DATA_FROZEN, DATA_COLD, DATA_WARM, DATA_HOT);
private static final Map<String, String> PREFERENCE_TIER_CONFIGURATIONS;
private static final Map<String, Settings> PREFERENCE_TIER_CONFIGURATION_SETTINGS;
static {
final Map<String, String> tmp = new HashMap<>();
final Map<String, Settings> tmpSettings = new HashMap<>();
for (int i = 0, ordered_frozen_to_hot_tiersSize = ORDERED_FROZEN_TO_HOT_TIERS.size(); i < ordered_frozen_to_hot_tiersSize; i++) {
String tier = ORDERED_FROZEN_TO_HOT_TIERS.get(i);
final String prefTierString =
String.join(",", ORDERED_FROZEN_TO_HOT_TIERS.subList(i, ORDERED_FROZEN_TO_HOT_TIERS.size())).intern();
tmp.put(tier, prefTierString);
tmpSettings.put(tier, Settings.builder().put(DataTier.TIER_PREFERENCE, prefTierString).build());
}
PREFERENCE_TIER_CONFIGURATIONS = org.elasticsearch.core.Map.copyOf(tmp);
PREFERENCE_TIER_CONFIGURATION_SETTINGS = org.elasticsearch.core.Map.copyOf(tmpSettings);
}
/**
* Returns true if the given tier name is a valid tier
*/
@ -83,11 +107,19 @@ public class DataTier {
* 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);
if (indexOfTargetTier == -1) {
final String res = PREFERENCE_TIER_CONFIGURATIONS.get(targetTier);
if (res == null) {
throw new IllegalArgumentException("invalid data tier [" + targetTier + "]");
}
return ORDERED_FROZEN_TO_HOT_TIERS.stream().skip(indexOfTargetTier).collect(Collectors.joining(","));
return res;
}
public static Settings getPreferredTiersConfigurationSettings(String targetTier) {
final Settings res = PREFERENCE_TIER_CONFIGURATION_SETTINGS.get(targetTier);
if (res == null) {
throw new IllegalArgumentException("invalid data tier [" + targetTier + "]");
}
return res;
}
/**
@ -157,7 +189,7 @@ public class DataTier {
Set<String> settings = indexSettings.keySet();
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(TIER_PREFERENCE).build();
return NULL_TIER_PREFERENCE_SETTINGS;
} 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 + "."))) {
@ -169,9 +201,9 @@ public class DataTier {
// tier if the index is part of a data stream, the "content"
// tier if it is not.
if (isDataStreamIndex) {
return Settings.builder().put(TIER_PREFERENCE, DATA_HOT).build();
return DATA_HOT_TIER_PREFERENCE_SETTINGS;
} else {
return Settings.builder().put(TIER_PREFERENCE, DATA_CONTENT).build();
return DATA_CONTENT_TIER_PREFERENCE_SETTINGS;
}
}
}

View file

@ -59,6 +59,12 @@ public class DiskThresholdMonitor {
private static final Logger logger = LogManager.getLogger(DiskThresholdMonitor.class);
private static final Settings READ_ONLY_ALLOW_DELETE_SETTINGS = Settings.builder()
.put(IndexMetadata.SETTING_READ_ONLY_ALLOW_DELETE, Boolean.TRUE.toString()).build();
private static final Settings NOT_READ_ONLY_ALLOW_DELETE_SETTINGS =
Settings.builder().putNull(IndexMetadata.SETTING_READ_ONLY_ALLOW_DELETE).build();
private final DiskThresholdSettings diskThresholdSettings;
private final Client client;
private final Supplier<ClusterState> clusterStateSupplier;
@ -378,9 +384,7 @@ public class DiskThresholdMonitor {
setLastRunTimeMillis();
listener.onFailure(e);
});
Settings readOnlySettings = readOnly ? Settings.builder()
.put(IndexMetadata.SETTING_READ_ONLY_ALLOW_DELETE, Boolean.TRUE.toString()).build() :
Settings.builder().putNull(IndexMetadata.SETTING_READ_ONLY_ALLOW_DELETE).build();
Settings readOnlySettings = readOnly ? READ_ONLY_ALLOW_DELETE_SETTINGS : NOT_READ_ONLY_ALLOW_DELETE_SETTINGS;
client.admin().indices().prepareUpdateSettings(indicesToUpdate.toArray(Strings.EMPTY_ARRAY))
.setSettings(readOnlySettings)
.execute(wrappedListener.map(r -> null));

View file

@ -762,8 +762,6 @@ public final class Settings implements ToXContentFragment {
*/
public static class Builder {
public static final Settings EMPTY_SETTINGS = new Builder().build();
// we use a sorted map for consistent serialization when using getAsMap()
private final Map<String, Object> map = new TreeMap<>();

View file

@ -538,7 +538,7 @@ public final class AnalysisRegistry implements Closeable {
if (analyzers.containsKey(DEFAULT_ANALYZER_NAME) == false) {
analyzers.put(DEFAULT_ANALYZER_NAME,
produceAnalyzer(DEFAULT_ANALYZER_NAME,
new StandardAnalyzerProvider(indexSettings, null, DEFAULT_ANALYZER_NAME, Settings.Builder.EMPTY_SETTINGS),
new StandardAnalyzerProvider(indexSettings, null, DEFAULT_ANALYZER_NAME, Settings.EMPTY),
tokenFilterFactoryFactories, charFilterFactoryFactories, tokenizerFactoryFactories));
}
NamedAnalyzer defaultAnalyzer = analyzers.get(DEFAULT_ANALYZER_NAME);

View file

@ -125,7 +125,7 @@ public abstract class Plugin implements Closeable {
* overwritten with the additional settings. These settings added if they don't exist.
*/
public Settings additionalSettings() {
return Settings.Builder.EMPTY_SETTINGS;
return Settings.EMPTY;
}
/**

View file

@ -264,7 +264,7 @@ public class BalanceConfigurationTests extends ESAllocationTestCase {
public void testNoRebalanceOnPrimaryOverload() {
Settings.Builder settings = Settings.builder();
AllocationService strategy = new AllocationService(randomAllocationDeciders(settings.build(),
new ClusterSettings(Settings.Builder.EMPTY_SETTINGS, ClusterSettings.BUILT_IN_CLUSTER_SETTINGS), random()),
new ClusterSettings(Settings.EMPTY, ClusterSettings.BUILT_IN_CLUSTER_SETTINGS), random()),
new TestGatewayAllocator(), new ShardsAllocator() {
/*
* // this allocator tries to rebuild this scenario where a rebalance is

View file

@ -28,7 +28,6 @@ import org.elasticsearch.test.ESTestCase;
import java.util.Collections;
import java.util.function.Supplier;
import static org.elasticsearch.common.settings.Settings.Builder.EMPTY_SETTINGS;
import static org.elasticsearch.index.IndexSettingsTests.newIndexMeta;
import static org.hamcrest.Matchers.containsString;
import static org.hamcrest.Matchers.equalTo;
@ -40,7 +39,7 @@ public class IndexSortSettingsTests extends ESTestCase {
}
public void testNoIndexSort() {
IndexSettings indexSettings = indexSettings(EMPTY_SETTINGS);
IndexSettings indexSettings = indexSettings(Settings.EMPTY);
assertFalse(indexSettings.getIndexSortConfig().hasIndexSort());
}

View file

@ -16,7 +16,6 @@ import org.elasticsearch.test.ESTestCase;
import java.io.IOException;
import static org.elasticsearch.common.settings.Settings.Builder.EMPTY_SETTINGS;
import static org.elasticsearch.index.IndexSettingsTests.newIndexMeta;
import static org.hamcrest.Matchers.containsString;
import static org.hamcrest.Matchers.equalTo;
@ -54,7 +53,7 @@ public class MergePolicySettingsTests extends ESTestCase {
}
public void testUpdateSettings() throws IOException {
IndexSettings indexSettings = indexSettings(EMPTY_SETTINGS);
IndexSettings indexSettings = indexSettings(Settings.EMPTY);
assertThat(indexSettings.getMergePolicy().getNoCFSRatio(), equalTo(0.1));
indexSettings = indexSettings(build(0.9));
assertThat((indexSettings.getMergePolicy()).getNoCFSRatio(), equalTo(0.9));
@ -131,7 +130,7 @@ public class MergePolicySettingsTests extends ESTestCase {
Settings.builder().put(MergePolicyConfig.INDEX_MERGE_POLICY_DELETES_PCT_ALLOWED_SETTING.getKey(), 53).build())));
final Throwable cause = exc.getCause();
assertThat(cause.getMessage(), containsString("must be <= 50.0"));
indexSettings.updateIndexMetadata(newIndexMeta("index", EMPTY_SETTINGS)); // see if defaults are restored
indexSettings.updateIndexMetadata(newIndexMeta("index", Settings.EMPTY)); // see if defaults are restored
assertEquals(((EsTieredMergePolicy) indexSettings.getMergePolicy()).getForceMergeDeletesPctAllowed(),
MergePolicyConfig.DEFAULT_EXPUNGE_DELETES_ALLOWED, 0.0d);
assertEquals(((EsTieredMergePolicy) indexSettings.getMergePolicy()).getFloorSegmentMB(),

View file

@ -60,7 +60,7 @@ public abstract class ESAllocationTestCase extends ESTestCase {
};
public static MockAllocationService createAllocationService() {
return createAllocationService(Settings.Builder.EMPTY_SETTINGS);
return createAllocationService(Settings.EMPTY);
}
public static MockAllocationService createAllocationService(Settings settings) {

View file

@ -17,7 +17,7 @@ import org.elasticsearch.index.IndexSettings;
public class MyFilterTokenFilterFactory extends AbstractTokenFilterFactory {
public MyFilterTokenFilterFactory(IndexSettings indexSettings, Environment env, String name, Settings settings) {
super(indexSettings, name, Settings.Builder.EMPTY_SETTINGS);
super(indexSettings, name, Settings.EMPTY);
}
@Override

View file

@ -36,6 +36,11 @@ import java.util.Objects;
public class ForceMergeAction implements LifecycleAction {
private static final Logger logger = LogManager.getLogger(ForceMergeAction.class);
private static final Settings READ_ONLY_SETTINGS = Settings.builder().put(IndexMetadata.SETTING_BLOCKS_WRITE, true).build();
private static final Settings BEST_COMPRESSION_SETTINGS =
Settings.builder().put(EngineConfig.INDEX_CODEC_SETTING.getKey(), CodecService.BEST_COMPRESSION_CODEC).build();
public static final String NAME = "forcemerge";
public static final ParseField MAX_NUM_SEGMENTS_FIELD = new ParseField("max_num_segments");
public static final ParseField CODEC = new ParseField("index_codec");
@ -120,10 +125,6 @@ public class ForceMergeAction implements LifecycleAction {
@Override
public List<Step> toSteps(Client client, String phase, Step.StepKey nextStepKey) {
Settings readOnlySettings = Settings.builder().put(IndexMetadata.SETTING_BLOCKS_WRITE, true).build();
Settings bestCompressionSettings = Settings.builder()
.put(EngineConfig.INDEX_CODEC_SETTING.getKey(), CodecService.BEST_COMPRESSION_CODEC).build();
final boolean codecChange = codec != null && codec.equals(CodecService.BEST_COMPRESSION_CODEC);
StepKey preForceMergeBranchingKey = new StepKey(phase, NAME, CONDITIONAL_SKIP_FORCE_MERGE_STEP);
@ -153,11 +154,11 @@ public class ForceMergeAction implements LifecycleAction {
CheckNotDataStreamWriteIndexStep checkNotWriteIndexStep = new CheckNotDataStreamWriteIndexStep(checkNotWriteIndex,
readOnlyKey);
UpdateSettingsStep readOnlyStep =
new UpdateSettingsStep(readOnlyKey, codecChange ? closeKey : forceMergeKey, client, readOnlySettings);
new UpdateSettingsStep(readOnlyKey, codecChange ? closeKey : forceMergeKey, client, READ_ONLY_SETTINGS);
CloseIndexStep closeIndexStep = new CloseIndexStep(closeKey, updateCompressionKey, client);
UpdateSettingsStep updateBestCompressionSettings = new UpdateSettingsStep(updateCompressionKey,
openKey, client, bestCompressionSettings);
openKey, client, BEST_COMPRESSION_SETTINGS);
OpenIndexStep openIndexStep = new OpenIndexStep(openKey, waitForGreenIndexKey, client);
WaitForIndexColorStep waitForIndexGreenStep = new WaitForIndexColorStep(waitForGreenIndexKey,
forceMergeKey, ClusterHealthStatus.GREEN);

View file

@ -25,7 +25,7 @@ import java.io.IOException;
import java.util.List;
import java.util.Objects;
import static org.elasticsearch.cluster.routing.allocation.DataTier.getPreferredTiersConfiguration;
import static org.elasticsearch.cluster.routing.allocation.DataTier.getPreferredTiersConfigurationSettings;
/**
* A {@link LifecycleAction} which enables or disables the automatic migration of data between
@ -115,9 +115,7 @@ public class MigrateAction implements LifecycleAction {
return false;
});
UpdateSettingsStep updateMigrationSettingStep = new UpdateSettingsStep(migrationKey, migrationRoutedKey, client,
Settings.builder()
.put(DataTier.TIER_PREFERENCE, getPreferredTiersConfiguration(targetTier))
.build());
getPreferredTiersConfigurationSettings(targetTier));
DataTierMigrationRoutedStep migrationRoutedStep = new DataTierMigrationRoutedStep(migrationRoutedKey, nextStepKey);
return org.elasticsearch.core.List.of(conditionalSkipActionStep, updateMigrationSettingStep, migrationRoutedStep);
} else {

View file

@ -42,6 +42,8 @@ public class RolloverAction implements LifecycleAction {
public static final Setting<String> LIFECYCLE_ROLLOVER_ALIAS_SETTING = Setting.simpleString(LIFECYCLE_ROLLOVER_ALIAS,
Setting.Property.Dynamic, Setting.Property.IndexScope);
private static final Settings INDEXING_COMPLETE = Settings.builder().put(LifecycleSettings.LIFECYCLE_INDEXING_COMPLETE, true).build();
private static final ConstructingObjectParser<RolloverAction, Void> PARSER = new ConstructingObjectParser<>(NAME,
a -> new RolloverAction((ByteSizeValue) a[0], (ByteSizeValue) a[1], (TimeValue) a[2], (Long) a[3]));
@ -168,8 +170,6 @@ public class RolloverAction implements LifecycleAction {
@Override
public List<Step> toSteps(Client client, String phase, Step.StepKey nextStepKey) {
Settings indexingComplete = Settings.builder().put(LifecycleSettings.LIFECYCLE_INDEXING_COMPLETE, true).build();
StepKey waitForRolloverReadyStepKey = new StepKey(phase, NAME, WaitForRolloverReadyStep.NAME);
StepKey rolloverStepKey = new StepKey(phase, NAME, RolloverStep.NAME);
StepKey waitForActiveShardsKey = new StepKey(phase, NAME, WaitForActiveShardsStep.NAME);
@ -183,7 +183,7 @@ public class RolloverAction implements LifecycleAction {
UpdateRolloverLifecycleDateStep updateDateStep = new UpdateRolloverLifecycleDateStep(updateDateStepKey, setIndexingCompleteStepKey,
System::currentTimeMillis);
UpdateSettingsStep setIndexingCompleteStep = new UpdateSettingsStep(setIndexingCompleteStepKey, nextStepKey,
client, indexingComplete);
client, INDEXING_COMPLETE);
return Arrays.asList(waitForRolloverReadyStep, rolloverStep, waitForActiveShardsStep, updateDateStep, setIndexingCompleteStep);
}

View file

@ -35,6 +35,9 @@ public class SetPriorityAction implements LifecycleAction {
private static final ConstructingObjectParser<SetPriorityAction, Void> PARSER = new ConstructingObjectParser<>(NAME,
a -> new SetPriorityAction((Integer) a[0]));
private static final Settings NULL_PRIORITY_SETTINGS =
Settings.builder().putNull(IndexMetadata.INDEX_PRIORITY_SETTING.getKey()).build();
//package private for testing
final Integer recoveryPriority;
@ -90,7 +93,7 @@ public class SetPriorityAction implements LifecycleAction {
public List<Step> toSteps(Client client, String phase, StepKey nextStepKey) {
StepKey key = new StepKey(phase, NAME, NAME);
Settings indexPriority = recoveryPriority == null ?
Settings.builder().putNull(IndexMetadata.INDEX_PRIORITY_SETTING.getKey()).build()
NULL_PRIORITY_SETTINGS
: Settings.builder().put(IndexMetadata.INDEX_PRIORITY_SETTING.getKey(), recoveryPriority).build();
return Collections.singletonList(new UpdateSettingsStep(key, nextStepKey, client, indexPriority));
}

View file

@ -13,6 +13,7 @@ import org.elasticsearch.action.ActionListener;
import org.elasticsearch.action.admin.indices.create.CreateIndexClusterStateUpdateRequest;
import org.elasticsearch.action.admin.indices.create.CreateIndexRequest;
import org.elasticsearch.action.admin.indices.delete.DeleteIndexRequest;
import org.elasticsearch.action.admin.indices.rollover.MetadataRolloverService;
import org.elasticsearch.action.admin.indices.settings.put.UpdateSettingsRequest;
import org.elasticsearch.action.admin.indices.shrink.ResizeRequest;
import org.elasticsearch.action.admin.indices.shrink.ResizeType;
@ -71,6 +72,9 @@ public class TransportRollupAction extends AcknowledgedTransportMasterNodeAction
private static final Logger logger = LogManager.getLogger(TransportRollupAction.class);
private static final Settings VISIBLE_INDEX_SETTINGS = Settings.builder().put(IndexMetadata.SETTING_INDEX_HIDDEN, false).build();
private static final Settings WRITE_BLOCKED_SETTINGS = Settings.builder().put(IndexMetadata.SETTING_BLOCKS_WRITE, true).build();
private final Client client;
private final ClusterService clusterService;
private final MetadataCreateIndexService metadataCreateIndexService;
@ -152,16 +156,13 @@ public class TransportRollupAction extends AcknowledgedTransportMasterNodeAction
"rollup",
tmpIndexName,
tmpIndexName
).settings(Settings.builder().put(IndexMetadata.SETTING_INDEX_HIDDEN, true).build()).mappings(mappingAsMap);
).settings(MetadataRolloverService.HIDDEN_INDEX_SETTINGS).mappings(mappingAsMap);
RollupIndexerAction.Request rollupIndexerRequest = new RollupIndexerAction.Request(request);
ResizeRequest resizeRequest = new ResizeRequest(request.getRollupIndex(), tmpIndexName);
resizeRequest.setResizeType(ResizeType.CLONE);
resizeRequest.getTargetIndexRequest().settings(Settings.builder().put(IndexMetadata.SETTING_INDEX_HIDDEN, false).build());
UpdateSettingsRequest updateSettingsReq = new UpdateSettingsRequest(
Settings.builder().put(IndexMetadata.SETTING_BLOCKS_WRITE, true).build(),
tmpIndexName
);
resizeRequest.getTargetIndexRequest().settings(VISIBLE_INDEX_SETTINGS);
UpdateSettingsRequest updateSettingsReq = new UpdateSettingsRequest(WRITE_BLOCKED_SETTINGS, tmpIndexName);
// 1. validate Rollup Config against Field Caps
// 2. create hidden temporary index