Update Settings: Properly ignore settings that are not allowed to be updated dynamically, closes #985.

This commit is contained in:
kimchy 2011-05-31 22:27:43 +03:00
parent 1f17e9d49d
commit c134233241
13 changed files with 149 additions and 11 deletions

View File

@ -35,6 +35,8 @@ import org.elasticsearch.common.xcontent.XContentFactory;
import org.elasticsearch.common.xcontent.XContentParser; import org.elasticsearch.common.xcontent.XContentParser;
import java.io.IOException; import java.io.IOException;
import java.util.Arrays;
import java.util.HashSet;
import java.util.Map; import java.util.Map;
import static org.elasticsearch.common.settings.ImmutableSettings.*; import static org.elasticsearch.common.settings.ImmutableSettings.*;
@ -45,6 +47,21 @@ import static org.elasticsearch.common.settings.ImmutableSettings.*;
@Immutable @Immutable
public class IndexMetaData { public class IndexMetaData {
private static ImmutableSet<String> dynamicSettings = ImmutableSet.<String>builder()
.add(IndexMetaData.SETTING_NUMBER_OF_REPLICAS)
.add(IndexMetaData.SETTING_AUTO_EXPAND_REPLICAS)
.build();
public static ImmutableSet<String> dynamicSettings() {
return dynamicSettings;
}
public static synchronized void addDynamicSettings(String... settings) {
HashSet<String> updatedSettings = new HashSet<String>(dynamicSettings);
updatedSettings.addAll(Arrays.asList(settings));
dynamicSettings = ImmutableSet.copyOf(updatedSettings);
}
public static enum State { public static enum State {
OPEN((byte) 0), OPEN((byte) 0),
CLOSE((byte) 1); CLOSE((byte) 1);

View File

@ -19,7 +19,6 @@
package org.elasticsearch.cluster.metadata; package org.elasticsearch.cluster.metadata;
import org.elasticsearch.ElasticSearchIllegalArgumentException;
import org.elasticsearch.cluster.ClusterChangedEvent; import org.elasticsearch.cluster.ClusterChangedEvent;
import org.elasticsearch.cluster.ClusterService; import org.elasticsearch.cluster.ClusterService;
import org.elasticsearch.cluster.ClusterState; import org.elasticsearch.cluster.ClusterState;
@ -27,12 +26,14 @@ import org.elasticsearch.cluster.ClusterStateListener;
import org.elasticsearch.cluster.ProcessedClusterStateUpdateTask; import org.elasticsearch.cluster.ProcessedClusterStateUpdateTask;
import org.elasticsearch.cluster.routing.RoutingTable; import org.elasticsearch.cluster.routing.RoutingTable;
import org.elasticsearch.common.Booleans; import org.elasticsearch.common.Booleans;
import org.elasticsearch.common.collect.Sets;
import org.elasticsearch.common.component.AbstractComponent; import org.elasticsearch.common.component.AbstractComponent;
import org.elasticsearch.common.inject.Inject; import org.elasticsearch.common.inject.Inject;
import org.elasticsearch.common.settings.ImmutableSettings; import org.elasticsearch.common.settings.ImmutableSettings;
import org.elasticsearch.common.settings.Settings; import org.elasticsearch.common.settings.Settings;
import java.util.Map; import java.util.Map;
import java.util.Set;
import static org.elasticsearch.cluster.routing.RoutingTable.*; import static org.elasticsearch.cluster.routing.RoutingTable.*;
@ -103,17 +104,29 @@ public class MetaDataUpdateSettingsService extends AbstractComponent implements
public void updateSettings(final Settings pSettings, final String[] indices, final Listener listener) { public void updateSettings(final Settings pSettings, final String[] indices, final Listener listener) {
ImmutableSettings.Builder updatedSettingsBuilder = ImmutableSettings.settingsBuilder(); ImmutableSettings.Builder updatedSettingsBuilder = ImmutableSettings.settingsBuilder();
for (Map.Entry<String, String> entry : pSettings.getAsMap().entrySet()) { for (Map.Entry<String, String> entry : pSettings.getAsMap().entrySet()) {
if (entry.getKey().equals("index")) {
continue;
}
if (!entry.getKey().startsWith("index.")) { if (!entry.getKey().startsWith("index.")) {
updatedSettingsBuilder.put("index." + entry.getKey(), entry.getValue()); updatedSettingsBuilder.put("index." + entry.getKey(), entry.getValue());
} else { } else {
updatedSettingsBuilder.put(entry.getKey(), entry.getValue()); updatedSettingsBuilder.put(entry.getKey(), entry.getValue());
} }
} }
final Settings settings = updatedSettingsBuilder.build(); Set<String> removedSettings = Sets.newHashSet();
if (settings.get(IndexMetaData.SETTING_NUMBER_OF_SHARDS) != null) { for (String key : updatedSettingsBuilder.internalMap().keySet()) {
listener.onFailure(new ElasticSearchIllegalArgumentException("can't change the number of shards for an index")); if (!IndexMetaData.dynamicSettings().contains(key)) {
return; removedSettings.add(key);
} }
}
if (!removedSettings.isEmpty()) {
logger.warn("{} ignoring non dynamic index level settings: {}", indices, removedSettings);
for (String removedSetting : removedSettings) {
updatedSettingsBuilder.remove(removedSetting);
}
}
final Settings settings = updatedSettingsBuilder.build();
clusterService.submitStateUpdateTask("update-settings", new ProcessedClusterStateUpdateTask() { clusterService.submitStateUpdateTask("update-settings", new ProcessedClusterStateUpdateTask() {
@Override public ClusterState execute(ClusterState currentState) { @Override public ClusterState execute(ClusterState currentState) {
try { try {

View File

@ -41,7 +41,11 @@ import java.io.IOException;
import java.io.InputStream; import java.io.InputStream;
import java.io.InputStreamReader; import java.io.InputStreamReader;
import java.net.URL; import java.net.URL;
import java.util.*; import java.util.Collections;
import java.util.LinkedHashMap;
import java.util.List;
import java.util.Map;
import java.util.Properties;
import java.util.concurrent.TimeUnit; import java.util.concurrent.TimeUnit;
import static org.elasticsearch.common.Strings.*; import static org.elasticsearch.common.Strings.*;
@ -344,6 +348,10 @@ public class ImmutableSettings implements Settings {
} }
public Map<String, String> internalMap() {
return this.map;
}
/** /**
* Removes the provided setting. * Removes the provided setting.
*/ */

View File

@ -20,6 +20,7 @@
package org.elasticsearch.index.cache.field.data.resident; package org.elasticsearch.index.cache.field.data.resident;
import org.elasticsearch.ElasticSearchException; import org.elasticsearch.ElasticSearchException;
import org.elasticsearch.cluster.metadata.IndexMetaData;
import org.elasticsearch.common.Nullable; import org.elasticsearch.common.Nullable;
import org.elasticsearch.common.base.Objects; import org.elasticsearch.common.base.Objects;
import org.elasticsearch.common.collect.MapEvictionListener; import org.elasticsearch.common.collect.MapEvictionListener;
@ -91,6 +92,13 @@ public class ResidentFieldDataCache extends AbstractConcurrentMapFieldDataCache
evictions.incrementAndGet(); evictions.incrementAndGet();
} }
static {
IndexMetaData.addDynamicSettings(
"index.cache.field.max_size",
"index.cache.field.expire"
);
}
class ApplySettings implements IndexSettingsService.Listener { class ApplySettings implements IndexSettingsService.Listener {
@Override public void onRefreshSettings(Settings settings) { @Override public void onRefreshSettings(Settings settings) {
int maxSize = settings.getAsInt("index.cache.field.max_size", ResidentFieldDataCache.this.maxSize); int maxSize = settings.getAsInt("index.cache.field.max_size", ResidentFieldDataCache.this.maxSize);

View File

@ -20,6 +20,7 @@
package org.elasticsearch.index.cache.filter.resident; package org.elasticsearch.index.cache.filter.resident;
import org.apache.lucene.search.Filter; import org.apache.lucene.search.Filter;
import org.elasticsearch.cluster.metadata.IndexMetaData;
import org.elasticsearch.common.base.Objects; import org.elasticsearch.common.base.Objects;
import org.elasticsearch.common.collect.MapEvictionListener; import org.elasticsearch.common.collect.MapEvictionListener;
import org.elasticsearch.common.collect.MapMaker; import org.elasticsearch.common.collect.MapMaker;
@ -91,6 +92,13 @@ public class ResidentFilterCache extends AbstractConcurrentMapFilterCache implem
evictions.incrementAndGet(); evictions.incrementAndGet();
} }
static {
IndexMetaData.addDynamicSettings(
"index.cache.field.max_size",
"index.cache.field.expire"
);
}
class ApplySettings implements IndexSettingsService.Listener { class ApplySettings implements IndexSettingsService.Listener {
@Override public void onRefreshSettings(Settings settings) { @Override public void onRefreshSettings(Settings settings) {
int maxSize = settings.getAsInt("index.cache.filter.max_size", ResidentFilterCache.this.maxSize); int maxSize = settings.getAsInt("index.cache.filter.max_size", ResidentFilterCache.this.maxSize);

View File

@ -20,6 +20,7 @@
package org.elasticsearch.index.cache.filter.soft; package org.elasticsearch.index.cache.filter.soft;
import org.apache.lucene.search.Filter; import org.apache.lucene.search.Filter;
import org.elasticsearch.cluster.metadata.IndexMetaData;
import org.elasticsearch.common.base.Objects; import org.elasticsearch.common.base.Objects;
import org.elasticsearch.common.collect.MapEvictionListener; import org.elasticsearch.common.collect.MapEvictionListener;
import org.elasticsearch.common.collect.MapMaker; import org.elasticsearch.common.collect.MapMaker;
@ -93,6 +94,13 @@ public class SoftFilterCache extends AbstractConcurrentMapFilterCache implements
evictions.incrementAndGet(); evictions.incrementAndGet();
} }
static {
IndexMetaData.addDynamicSettings(
"index.cache.field.max_size",
"index.cache.field.expire"
);
}
class ApplySettings implements IndexSettingsService.Listener { class ApplySettings implements IndexSettingsService.Listener {
@Override public void onRefreshSettings(Settings settings) { @Override public void onRefreshSettings(Settings settings) {
int maxSize = settings.getAsInt("index.cache.filter.max_size", SoftFilterCache.this.maxSize); int maxSize = settings.getAsInt("index.cache.filter.max_size", SoftFilterCache.this.maxSize);

View File

@ -20,6 +20,7 @@
package org.elasticsearch.index.cache.filter.weak; package org.elasticsearch.index.cache.filter.weak;
import org.apache.lucene.search.Filter; import org.apache.lucene.search.Filter;
import org.elasticsearch.cluster.metadata.IndexMetaData;
import org.elasticsearch.common.base.Objects; import org.elasticsearch.common.base.Objects;
import org.elasticsearch.common.collect.MapEvictionListener; import org.elasticsearch.common.collect.MapEvictionListener;
import org.elasticsearch.common.collect.MapMaker; import org.elasticsearch.common.collect.MapMaker;
@ -91,6 +92,13 @@ public class WeakFilterCache extends AbstractConcurrentMapFilterCache implements
evictions.incrementAndGet(); evictions.incrementAndGet();
} }
static {
IndexMetaData.addDynamicSettings(
"index.cache.field.max_size",
"index.cache.field.expire"
);
}
class ApplySettings implements IndexSettingsService.Listener { class ApplySettings implements IndexSettingsService.Listener {
@Override public void onRefreshSettings(Settings settings) { @Override public void onRefreshSettings(Settings settings) {
int maxSize = settings.getAsInt("index.cache.filter.max_size", WeakFilterCache.this.maxSize); int maxSize = settings.getAsInt("index.cache.filter.max_size", WeakFilterCache.this.maxSize);

View File

@ -19,12 +19,17 @@
package org.elasticsearch.index.engine.robin; package org.elasticsearch.index.engine.robin;
import org.apache.lucene.index.*; import org.apache.lucene.index.ExtendedIndexSearcher;
import org.apache.lucene.index.IndexReader;
import org.apache.lucene.index.IndexWriter;
import org.apache.lucene.index.IndexWriterConfig;
import org.apache.lucene.index.Term;
import org.apache.lucene.search.FilteredQuery; import org.apache.lucene.search.FilteredQuery;
import org.apache.lucene.search.Query; import org.apache.lucene.search.Query;
import org.apache.lucene.store.AlreadyClosedException; import org.apache.lucene.store.AlreadyClosedException;
import org.apache.lucene.util.UnicodeUtil; import org.apache.lucene.util.UnicodeUtil;
import org.elasticsearch.ElasticSearchException; import org.elasticsearch.ElasticSearchException;
import org.elasticsearch.cluster.metadata.IndexMetaData;
import org.elasticsearch.common.Preconditions; import org.elasticsearch.common.Preconditions;
import org.elasticsearch.common.Unicode; import org.elasticsearch.common.Unicode;
import org.elasticsearch.common.bloom.BloomFilter; import org.elasticsearch.common.bloom.BloomFilter;
@ -1078,6 +1083,15 @@ public class RobinEngine extends AbstractIndexShardComponent implements Engine {
return indexWriter; return indexWriter;
} }
static {
IndexMetaData.addDynamicSettings(
"index.term_index_interval",
"index.term_index_divisor",
"index.index_concurrency"
);
}
class ApplySettings implements IndexSettingsService.Listener { class ApplySettings implements IndexSettingsService.Listener {
@Override public void onRefreshSettings(Settings settings) { @Override public void onRefreshSettings(Settings settings) {
int termIndexInterval = settings.getAsInt("index.term_index_interval", RobinEngine.this.termIndexInterval); int termIndexInterval = settings.getAsInt("index.term_index_interval", RobinEngine.this.termIndexInterval);

View File

@ -19,6 +19,7 @@
package org.elasticsearch.index.gateway; package org.elasticsearch.index.gateway;
import org.elasticsearch.cluster.metadata.IndexMetaData;
import org.elasticsearch.common.inject.Inject; import org.elasticsearch.common.inject.Inject;
import org.elasticsearch.common.settings.Settings; import org.elasticsearch.common.settings.Settings;
import org.elasticsearch.common.unit.ByteSizeValue; import org.elasticsearch.common.unit.ByteSizeValue;
@ -30,7 +31,12 @@ import org.elasticsearch.index.engine.EngineException;
import org.elasticsearch.index.engine.SnapshotFailedEngineException; import org.elasticsearch.index.engine.SnapshotFailedEngineException;
import org.elasticsearch.index.settings.IndexSettings; import org.elasticsearch.index.settings.IndexSettings;
import org.elasticsearch.index.settings.IndexSettingsService; import org.elasticsearch.index.settings.IndexSettingsService;
import org.elasticsearch.index.shard.*; import org.elasticsearch.index.shard.AbstractIndexShardComponent;
import org.elasticsearch.index.shard.IllegalIndexShardStateException;
import org.elasticsearch.index.shard.IndexShardClosedException;
import org.elasticsearch.index.shard.IndexShardNotStartedException;
import org.elasticsearch.index.shard.IndexShardState;
import org.elasticsearch.index.shard.ShardId;
import org.elasticsearch.index.shard.service.IndexShard; import org.elasticsearch.index.shard.service.IndexShard;
import org.elasticsearch.index.shard.service.InternalIndexShard; import org.elasticsearch.index.shard.service.InternalIndexShard;
import org.elasticsearch.index.translog.Translog; import org.elasticsearch.index.translog.Translog;
@ -90,6 +96,10 @@ public class IndexShardGatewayService extends AbstractIndexShardComponent implem
indexSettingsService.addListener(applySettings); indexSettingsService.addListener(applySettings);
} }
static {
IndexMetaData.addDynamicSettings("index.gateway.snapshot_interval");
}
class ApplySettings implements IndexSettingsService.Listener { class ApplySettings implements IndexSettingsService.Listener {
@Override public void onRefreshSettings(Settings settings) { @Override public void onRefreshSettings(Settings settings) {
TimeValue snapshotInterval = settings.getAsTime("index.gateway.snapshot_interval", IndexShardGatewayService.this.snapshotInterval); TimeValue snapshotInterval = settings.getAsTime("index.gateway.snapshot_interval", IndexShardGatewayService.this.snapshotInterval);

View File

@ -24,6 +24,7 @@ import org.apache.lucene.index.LogByteSizeMergePolicy;
import org.apache.lucene.index.SegmentInfo; import org.apache.lucene.index.SegmentInfo;
import org.apache.lucene.index.SegmentInfos; import org.apache.lucene.index.SegmentInfos;
import org.elasticsearch.ElasticSearchException; import org.elasticsearch.ElasticSearchException;
import org.elasticsearch.cluster.metadata.IndexMetaData;
import org.elasticsearch.common.Preconditions; import org.elasticsearch.common.Preconditions;
import org.elasticsearch.common.inject.Inject; import org.elasticsearch.common.inject.Inject;
import org.elasticsearch.common.settings.Settings; import org.elasticsearch.common.settings.Settings;
@ -96,6 +97,16 @@ public class LogByteSizeMergePolicyProvider extends AbstractIndexShardComponent
indexSettingsService.removeListener(applySettings); indexSettingsService.removeListener(applySettings);
} }
static {
IndexMetaData.addDynamicSettings(
"index.merge.policy.min_merge_size",
"index.merge.policy.max_merge_size",
"index.merge.policy.max_merge_docs",
"index.merge.policy.merge_factor",
"index.compound_format"
);
}
class ApplySettings implements IndexSettingsService.Listener { class ApplySettings implements IndexSettingsService.Listener {
@Override public void onRefreshSettings(Settings settings) { @Override public void onRefreshSettings(Settings settings) {
ByteSizeValue minMergeSize = settings.getAsBytesSize("index.merge.policy.min_merge_size", LogByteSizeMergePolicyProvider.this.minMergeSize); ByteSizeValue minMergeSize = settings.getAsBytesSize("index.merge.policy.min_merge_size", LogByteSizeMergePolicyProvider.this.minMergeSize);

View File

@ -24,6 +24,7 @@ import org.apache.lucene.index.LogDocMergePolicy;
import org.apache.lucene.index.SegmentInfo; import org.apache.lucene.index.SegmentInfo;
import org.apache.lucene.index.SegmentInfos; import org.apache.lucene.index.SegmentInfos;
import org.elasticsearch.ElasticSearchException; import org.elasticsearch.ElasticSearchException;
import org.elasticsearch.cluster.metadata.IndexMetaData;
import org.elasticsearch.common.Preconditions; import org.elasticsearch.common.Preconditions;
import org.elasticsearch.common.inject.Inject; import org.elasticsearch.common.inject.Inject;
import org.elasticsearch.common.settings.Settings; import org.elasticsearch.common.settings.Settings;
@ -90,6 +91,15 @@ public class LogDocMergePolicyProvider extends AbstractIndexShardComponent imple
return mergePolicy; return mergePolicy;
} }
static {
IndexMetaData.addDynamicSettings(
"index.merge.policy.min_merge_docs",
"index.merge.policy.max_merge_docs",
"index.merge.policy.merge_factor",
"index.compound_format"
);
}
class ApplySettings implements IndexSettingsService.Listener { class ApplySettings implements IndexSettingsService.Listener {
@Override public void onRefreshSettings(Settings settings) { @Override public void onRefreshSettings(Settings settings) {
int minMergeDocs = settings.getAsInt("index.merge.policy.min_merge_docs", LogDocMergePolicyProvider.this.minMergeDocs); int minMergeDocs = settings.getAsInt("index.merge.policy.min_merge_docs", LogDocMergePolicyProvider.this.minMergeDocs);

View File

@ -29,6 +29,7 @@ import org.apache.lucene.util.ThreadInterruptedException;
import org.elasticsearch.ElasticSearchException; import org.elasticsearch.ElasticSearchException;
import org.elasticsearch.ElasticSearchIllegalArgumentException; import org.elasticsearch.ElasticSearchIllegalArgumentException;
import org.elasticsearch.ElasticSearchIllegalStateException; import org.elasticsearch.ElasticSearchIllegalStateException;
import org.elasticsearch.cluster.metadata.IndexMetaData;
import org.elasticsearch.cluster.routing.ShardRouting; import org.elasticsearch.cluster.routing.ShardRouting;
import org.elasticsearch.common.Nullable; import org.elasticsearch.common.Nullable;
import org.elasticsearch.common.Strings; import org.elasticsearch.common.Strings;
@ -40,8 +41,16 @@ import org.elasticsearch.common.unit.TimeValue;
import org.elasticsearch.common.util.concurrent.ThreadSafe; import org.elasticsearch.common.util.concurrent.ThreadSafe;
import org.elasticsearch.index.aliases.IndexAliasesService; import org.elasticsearch.index.aliases.IndexAliasesService;
import org.elasticsearch.index.cache.IndexCache; import org.elasticsearch.index.cache.IndexCache;
import org.elasticsearch.index.engine.*; import org.elasticsearch.index.engine.Engine;
import org.elasticsearch.index.mapper.*; import org.elasticsearch.index.engine.EngineClosedException;
import org.elasticsearch.index.engine.EngineException;
import org.elasticsearch.index.engine.OptimizeFailedEngineException;
import org.elasticsearch.index.engine.RefreshFailedEngineException;
import org.elasticsearch.index.mapper.DocumentMapper;
import org.elasticsearch.index.mapper.MapperService;
import org.elasticsearch.index.mapper.ParsedDocument;
import org.elasticsearch.index.mapper.SourceToParse;
import org.elasticsearch.index.mapper.Uid;
import org.elasticsearch.index.merge.scheduler.MergeSchedulerProvider; import org.elasticsearch.index.merge.scheduler.MergeSchedulerProvider;
import org.elasticsearch.index.query.IndexQueryParser; import org.elasticsearch.index.query.IndexQueryParser;
import org.elasticsearch.index.query.IndexQueryParserMissingException; import org.elasticsearch.index.query.IndexQueryParserMissingException;
@ -595,6 +604,10 @@ public class InternalIndexShard extends AbstractIndexShardComponent implements I
return query; return query;
} }
static {
IndexMetaData.addDynamicSettings("index.refresh_interval");
}
private class ApplyRefreshSettings implements IndexSettingsService.Listener { private class ApplyRefreshSettings implements IndexSettingsService.Listener {
@Override public void onRefreshSettings(Settings settings) { @Override public void onRefreshSettings(Settings settings) {
synchronized (mutex) { synchronized (mutex) {

View File

@ -19,6 +19,7 @@
package org.elasticsearch.index.translog; package org.elasticsearch.index.translog;
import org.elasticsearch.cluster.metadata.IndexMetaData;
import org.elasticsearch.common.inject.Inject; import org.elasticsearch.common.inject.Inject;
import org.elasticsearch.common.settings.Settings; import org.elasticsearch.common.settings.Settings;
import org.elasticsearch.common.unit.ByteSizeUnit; import org.elasticsearch.common.unit.ByteSizeUnit;
@ -92,6 +93,15 @@ public class TranslogService extends AbstractIndexShardComponent {
this.future.cancel(true); this.future.cancel(true);
} }
static {
IndexMetaData.addDynamicSettings(
"index.translog.flush_threshold_ops",
"index.translog.flush_threshold_size",
"index.translog.flush_threshold_period",
"index.translog.disable_flush"
);
}
class ApplySettings implements IndexSettingsService.Listener { class ApplySettings implements IndexSettingsService.Listener {
@Override public void onRefreshSettings(Settings settings) { @Override public void onRefreshSettings(Settings settings) {
int flushThresholdOperations = settings.getAsInt("index.translog.flush_threshold_ops", TranslogService.this.flushThresholdOperations); int flushThresholdOperations = settings.getAsInt("index.translog.flush_threshold_ops", TranslogService.this.flushThresholdOperations);