expose default settings via rest API
This commit is contained in:
parent
a61723b538
commit
9562fb76bc
|
@ -28,6 +28,7 @@ import org.elasticsearch.cluster.block.ClusterBlockLevel;
|
|||
import org.elasticsearch.cluster.metadata.IndexNameExpressionResolver;
|
||||
import org.elasticsearch.cluster.metadata.MetaDataIndexTemplateService;
|
||||
import org.elasticsearch.common.inject.Inject;
|
||||
import org.elasticsearch.common.settings.IndexScopeSettings;
|
||||
import org.elasticsearch.common.settings.Settings;
|
||||
import org.elasticsearch.threadpool.ThreadPool;
|
||||
import org.elasticsearch.transport.TransportService;
|
||||
|
@ -38,13 +39,15 @@ import org.elasticsearch.transport.TransportService;
|
|||
public class TransportPutIndexTemplateAction extends TransportMasterNodeAction<PutIndexTemplateRequest, PutIndexTemplateResponse> {
|
||||
|
||||
private final MetaDataIndexTemplateService indexTemplateService;
|
||||
private final IndexScopeSettings indexScopeSettings;
|
||||
|
||||
@Inject
|
||||
public TransportPutIndexTemplateAction(Settings settings, TransportService transportService, ClusterService clusterService,
|
||||
ThreadPool threadPool, MetaDataIndexTemplateService indexTemplateService,
|
||||
ActionFilters actionFilters, IndexNameExpressionResolver indexNameExpressionResolver) {
|
||||
ActionFilters actionFilters, IndexNameExpressionResolver indexNameExpressionResolver, IndexScopeSettings indexScopeSettings) {
|
||||
super(settings, PutIndexTemplateAction.NAME, transportService, clusterService, threadPool, actionFilters, indexNameExpressionResolver, PutIndexTemplateRequest::new);
|
||||
this.indexTemplateService = indexTemplateService;
|
||||
this.indexScopeSettings = indexScopeSettings;
|
||||
}
|
||||
|
||||
@Override
|
||||
|
@ -69,7 +72,7 @@ public class TransportPutIndexTemplateAction extends TransportMasterNodeAction<P
|
|||
if (cause.length() == 0) {
|
||||
cause = "api";
|
||||
}
|
||||
|
||||
indexScopeSettings.validate(request.settings());
|
||||
indexTemplateService.putTemplate(new MetaDataIndexTemplateService.PutRequest(cause, request.name())
|
||||
.template(request.template())
|
||||
.order(request.order())
|
||||
|
|
|
@ -177,9 +177,7 @@ public class MetaDataCreateIndexService extends AbstractComponent {
|
|||
public void createIndex(final CreateIndexClusterStateUpdateRequest request, final ActionListener<ClusterStateUpdateResponse> listener) {
|
||||
Settings.Builder updatedSettingsBuilder = Settings.settingsBuilder();
|
||||
updatedSettingsBuilder.put(request.settings()).normalizePrefix(IndexMetaData.INDEX_SETTING_PREFIX);
|
||||
for (Map.Entry<String, String> entry : updatedSettingsBuilder.internalMap().entrySet()) {
|
||||
indexScopeSettings.validate(entry.getKey(), entry.getValue());
|
||||
}
|
||||
indexScopeSettings.validate(updatedSettingsBuilder);
|
||||
request.settings(updatedSettingsBuilder.build());
|
||||
|
||||
clusterService.submitStateUpdateTask("create-index [" + request.index() + "], cause [" + request.cause() + "]",
|
||||
|
|
|
@ -39,6 +39,7 @@ import org.elasticsearch.discovery.DiscoverySettings;
|
|||
import org.elasticsearch.discovery.zen.ZenDiscovery;
|
||||
import org.elasticsearch.discovery.zen.elect.ElectMasterService;
|
||||
import org.elasticsearch.gateway.PrimaryShardAllocator;
|
||||
import org.elasticsearch.index.IndexSettings;
|
||||
import org.elasticsearch.index.store.IndexStoreConfig;
|
||||
import org.elasticsearch.indices.breaker.HierarchyCircuitBreakerService;
|
||||
import org.elasticsearch.indices.recovery.RecoverySettings;
|
||||
|
@ -63,7 +64,6 @@ public final class ClusterSettings extends AbstractScopedSettings {
|
|||
super(settings, settingsSet, Setting.Scope.CLUSTER);
|
||||
}
|
||||
|
||||
|
||||
@Override
|
||||
public synchronized Settings applySettings(Settings newSettings) {
|
||||
Settings settings = super.applySettings(newSettings);
|
||||
|
@ -156,5 +156,7 @@ public final class ClusterSettings extends AbstractScopedSettings {
|
|||
HierarchyCircuitBreakerService.REQUEST_CIRCUIT_BREAKER_TYPE_SETTING,
|
||||
Transport.TRANSPORT_PROFILES_SETTING,
|
||||
Transport.TRANSPORT_TCP_COMPRESS,
|
||||
IndexSettings.QUERY_STRING_ANALYZE_WILDCARD,
|
||||
IndexSettings.QUERY_STRING_ALLOW_LEADING_WILDCARD,
|
||||
PrimaryShardAllocator.NODE_INITIAL_SHARDS_SETTING)));
|
||||
}
|
||||
|
|
|
@ -18,28 +18,10 @@
|
|||
*/
|
||||
package org.elasticsearch.common.settings;
|
||||
|
||||
import org.elasticsearch.action.admin.indices.close.TransportCloseIndexAction;
|
||||
import org.elasticsearch.action.support.DestructiveOperations;
|
||||
import org.elasticsearch.cluster.InternalClusterInfoService;
|
||||
import org.elasticsearch.cluster.action.index.MappingUpdatedAction;
|
||||
import org.elasticsearch.cluster.metadata.IndexMetaData;
|
||||
import org.elasticsearch.cluster.metadata.MetaData;
|
||||
import org.elasticsearch.cluster.routing.UnassignedInfo;
|
||||
import org.elasticsearch.cluster.routing.allocation.allocator.BalancedShardsAllocator;
|
||||
import org.elasticsearch.cluster.routing.allocation.decider.AwarenessAllocationDecider;
|
||||
import org.elasticsearch.cluster.routing.allocation.decider.ClusterRebalanceAllocationDecider;
|
||||
import org.elasticsearch.cluster.routing.allocation.decider.ConcurrentRebalanceAllocationDecider;
|
||||
import org.elasticsearch.cluster.routing.allocation.decider.DiskThresholdDecider;
|
||||
import org.elasticsearch.cluster.routing.allocation.decider.EnableAllocationDecider;
|
||||
import org.elasticsearch.cluster.routing.allocation.decider.FilterAllocationDecider;
|
||||
import org.elasticsearch.cluster.routing.allocation.decider.ShardsLimitAllocationDecider;
|
||||
import org.elasticsearch.cluster.routing.allocation.decider.SnapshotInProgressAllocationDecider;
|
||||
import org.elasticsearch.cluster.routing.allocation.decider.ThrottlingAllocationDecider;
|
||||
import org.elasticsearch.cluster.service.InternalClusterService;
|
||||
import org.elasticsearch.common.logging.ESLoggerFactory;
|
||||
import org.elasticsearch.discovery.DiscoverySettings;
|
||||
import org.elasticsearch.discovery.zen.ZenDiscovery;
|
||||
import org.elasticsearch.discovery.zen.elect.ElectMasterService;
|
||||
import org.elasticsearch.gateway.PrimaryShardAllocator;
|
||||
import org.elasticsearch.index.IndexModule;
|
||||
import org.elasticsearch.index.IndexSettings;
|
||||
|
@ -48,27 +30,20 @@ import org.elasticsearch.index.MergePolicyConfig;
|
|||
import org.elasticsearch.index.MergeSchedulerConfig;
|
||||
import org.elasticsearch.index.SearchSlowLog;
|
||||
import org.elasticsearch.index.cache.bitset.BitsetFilterCache;
|
||||
import org.elasticsearch.index.engine.EngineConfig;
|
||||
import org.elasticsearch.index.fielddata.IndexFieldDataService;
|
||||
import org.elasticsearch.index.mapper.FieldMapper;
|
||||
import org.elasticsearch.index.mapper.MapperService;
|
||||
import org.elasticsearch.index.percolator.PercolatorQueriesRegistry;
|
||||
import org.elasticsearch.index.store.FsDirectoryService;
|
||||
import org.elasticsearch.index.store.IndexStore;
|
||||
import org.elasticsearch.index.store.IndexStoreConfig;
|
||||
import org.elasticsearch.index.store.Store;
|
||||
import org.elasticsearch.indices.breaker.HierarchyCircuitBreakerService;
|
||||
import org.elasticsearch.indices.cache.request.IndicesRequestCache;
|
||||
import org.elasticsearch.indices.recovery.RecoverySettings;
|
||||
import org.elasticsearch.indices.ttl.IndicesTTLService;
|
||||
import org.elasticsearch.search.SearchService;
|
||||
import org.elasticsearch.threadpool.ThreadPool;
|
||||
import org.elasticsearch.transport.Transport;
|
||||
import org.elasticsearch.transport.TransportService;
|
||||
|
||||
import java.util.Arrays;
|
||||
import java.util.Collections;
|
||||
import java.util.HashSet;
|
||||
import java.util.Map;
|
||||
import java.util.Set;
|
||||
import java.util.function.Predicate;
|
||||
|
||||
|
@ -134,6 +109,8 @@ public final class IndexScopeSettings extends AbstractScopedSettings {
|
|||
IndexSettings.INDEX_TRANSLOG_SYNC_INTERVAL_SETTING,
|
||||
IndexSettings.DEFAULT_FIELD_SETTING,
|
||||
IndexSettings.QUERY_STRING_LENIENT_SETTING,
|
||||
IndexSettings.ALLOW_UNMAPPED,
|
||||
IndexSettings.INDEX_CHECK_ON_STARTUP,
|
||||
ShardsLimitAllocationDecider.INDEX_TOTAL_SHARDS_PER_NODE_SETTING,
|
||||
IndexSettings.INDEX_GC_DELETES_SETTING,
|
||||
IndicesRequestCache.INDEX_CACHE_REQUEST_ENABLED_SETTING,
|
||||
|
@ -153,6 +130,8 @@ public final class IndexScopeSettings extends AbstractScopedSettings {
|
|||
IndexModule.INDEX_QUERY_CACHE_EVERYTHING_SETTING,
|
||||
PrimaryShardAllocator.INDEX_RECOVERY_INITIAL_SHARDS_SETTING,
|
||||
FsDirectoryService.INDEX_LOCK_FACTOR_SETTING,
|
||||
EngineConfig.INDEX_CODEC_SETTING,
|
||||
SearchService.INDEX_NORMS_LOADING_SETTING,
|
||||
// this sucks but we can't really validate all the analyzers/similarity in here
|
||||
Setting.groupSetting("index.similarity.", false, Setting.Scope.INDEX), // this allows similarity settings to be passed
|
||||
Setting.groupSetting("index.analysis.", false, Setting.Scope.INDEX) // this allows analysis settings to be passed
|
||||
|
|
|
@ -51,13 +51,24 @@ public final class IndexSettings {
|
|||
|
||||
public static final Setting<String> DEFAULT_FIELD_SETTING = new Setting<>("index.query.default_field", AllFieldMapper.NAME, Function.identity(), false, Setting.Scope.INDEX);
|
||||
public static final Setting<Boolean> QUERY_STRING_LENIENT_SETTING = Setting.boolSetting("index.query_string.lenient", false, false, Setting.Scope.INDEX);
|
||||
public static final String QUERY_STRING_ANALYZE_WILDCARD = "indices.query.query_string.analyze_wildcard";
|
||||
public static final String QUERY_STRING_ALLOW_LEADING_WILDCARD = "indices.query.query_string.allowLeadingWildcard";
|
||||
public static final String ALLOW_UNMAPPED = "index.query.parse.allow_unmapped_fields";
|
||||
public static final Setting<Boolean> QUERY_STRING_ANALYZE_WILDCARD = Setting.boolSetting("indices.query.query_string.analyze_wildcard", false, false, Setting.Scope.CLUSTER);
|
||||
public static final Setting<Boolean> QUERY_STRING_ALLOW_LEADING_WILDCARD = Setting.boolSetting("indices.query.query_string.allowLeadingWildcard", true, false, Setting.Scope.CLUSTER);
|
||||
public static final Setting<Boolean> ALLOW_UNMAPPED = Setting.boolSetting("index.query.parse.allow_unmapped_fields", true, false, Setting.Scope.INDEX);
|
||||
public static final Setting<TimeValue> INDEX_TRANSLOG_SYNC_INTERVAL_SETTING = Setting.timeSetting("index.translog.sync_interval", TimeValue.timeValueSeconds(5), false, Setting.Scope.INDEX);
|
||||
public static final Setting<Translog.Durability> INDEX_TRANSLOG_DURABILITY_SETTING = new Setting<>("index.translog.durability", Translog.Durability.REQUEST.name(), (value) -> Translog.Durability.valueOf(value.toUpperCase(Locale.ROOT)), true, Setting.Scope.INDEX);
|
||||
public static final Setting<Boolean> INDEX_WARMER_ENABLED_SETTING = Setting.boolSetting("index.warmer.enabled", true, true, Setting.Scope.INDEX);
|
||||
public static final Setting<Boolean> INDEX_TTL_DISABLE_PURGE_SETTING = Setting.boolSetting("index.ttl.disable_purge", false, true, Setting.Scope.INDEX);
|
||||
public static final Setting<String> INDEX_CHECK_ON_STARTUP = new Setting<>("index.shard.check_on_startup", "false", (s) -> {
|
||||
switch(s) {
|
||||
case "false":
|
||||
case "true":
|
||||
case "fix":
|
||||
case "checksum":
|
||||
return s;
|
||||
default:
|
||||
throw new IllegalArgumentException("unknown value for [index.shard.check_on_startup] must be one of [true, false, fix, checksum] but was: " + s);
|
||||
}
|
||||
}, false, Setting.Scope.INDEX);
|
||||
|
||||
/**
|
||||
* Index setting describing the maximum value of from + size on a query.
|
||||
|
@ -183,10 +194,10 @@ public final class IndexSettings {
|
|||
|
||||
this.defaultField = DEFAULT_FIELD_SETTING.get(settings);
|
||||
this.queryStringLenient = QUERY_STRING_LENIENT_SETTING.get(settings);
|
||||
this.queryStringAnalyzeWildcard = settings.getAsBoolean(QUERY_STRING_ANALYZE_WILDCARD, false);
|
||||
this.queryStringAllowLeadingWildcard = settings.getAsBoolean(QUERY_STRING_ALLOW_LEADING_WILDCARD, true);
|
||||
this.queryStringAnalyzeWildcard = QUERY_STRING_ANALYZE_WILDCARD.get(nodeSettings);
|
||||
this.queryStringAllowLeadingWildcard = QUERY_STRING_ALLOW_LEADING_WILDCARD.get(nodeSettings);
|
||||
this.parseFieldMatcher = new ParseFieldMatcher(settings);
|
||||
this.defaultAllowUnmappedFields = settings.getAsBoolean(ALLOW_UNMAPPED, true);
|
||||
this.defaultAllowUnmappedFields = scopedSettings.get(ALLOW_UNMAPPED);
|
||||
this.indexNameMatcher = indexNameMatcher;
|
||||
this.durability = scopedSettings.get(INDEX_TRANSLOG_DURABILITY_SETTING);
|
||||
scopedSettings.addSettingsUpdateConsumer(INDEX_TRANSLOG_DURABILITY_SETTING, this::setTranslogDurability);
|
||||
|
|
|
@ -25,6 +25,7 @@ import org.apache.lucene.index.SnapshotDeletionPolicy;
|
|||
import org.apache.lucene.search.QueryCache;
|
||||
import org.apache.lucene.search.QueryCachingPolicy;
|
||||
import org.apache.lucene.search.similarities.Similarity;
|
||||
import org.elasticsearch.common.settings.Setting;
|
||||
import org.elasticsearch.common.settings.Settings;
|
||||
import org.elasticsearch.common.unit.ByteSizeUnit;
|
||||
import org.elasticsearch.common.unit.ByteSizeValue;
|
||||
|
@ -68,7 +69,7 @@ public final class EngineConfig {
|
|||
* Index setting to change the low level lucene codec used for writing new segments.
|
||||
* This setting is <b>not</b> realtime updateable.
|
||||
*/
|
||||
public static final String INDEX_CODEC_SETTING = "index.codec";
|
||||
public static final Setting<String> INDEX_CODEC_SETTING = new Setting<>("index.codec", EngineConfig.DEFAULT_CODEC_NAME, (s) -> { if (s.isEmpty()) {throw new IllegalArgumentException("index.codec must not be empty");} return s;}, false, Setting.Scope.INDEX);
|
||||
|
||||
/** if set to true the engine will start even if the translog id in the commit point can not be found */
|
||||
public static final String INDEX_FORCE_NEW_TRANSLOG = "index.engine.force_new_translog";
|
||||
|
@ -97,7 +98,7 @@ public final class EngineConfig {
|
|||
this.similarity = similarity;
|
||||
this.codecService = codecService;
|
||||
this.eventListener = eventListener;
|
||||
codecName = settings.get(EngineConfig.INDEX_CODEC_SETTING, EngineConfig.DEFAULT_CODEC_NAME);
|
||||
codecName = indexSettings.getValue(INDEX_CODEC_SETTING);
|
||||
// We give IndexWriter a "huge" (256 MB) buffer, so it won't flush on its own unless the ES indexing buffer is also huge and/or
|
||||
// there are not too many shards allocated to this node. Instead, IndexingMemoryController periodically checks
|
||||
// and refreshes the most heap-consuming shards when total indexing heap usage across all shards is too high:
|
||||
|
|
|
@ -237,7 +237,7 @@ public class IndexShard extends AbstractIndexShardComponent {
|
|||
/* create engine config */
|
||||
logger.debug("state: [CREATED]");
|
||||
|
||||
this.checkIndexOnStartup = settings.get("index.shard.check_on_startup", "false");
|
||||
this.checkIndexOnStartup = indexSettings.getValue(IndexSettings.INDEX_CHECK_ON_STARTUP);
|
||||
this.translogConfig = new TranslogConfig(shardId, shardPath().resolveTranslog(), indexSettings,
|
||||
provider.getBigArrays());
|
||||
final QueryCachingPolicy cachingPolicy;
|
||||
|
@ -1208,7 +1208,7 @@ public class IndexShard extends AbstractIndexShardComponent {
|
|||
BytesStreamOutput os = new BytesStreamOutput();
|
||||
PrintStream out = new PrintStream(os, false, StandardCharsets.UTF_8.name());
|
||||
|
||||
if ("checksum".equalsIgnoreCase(checkIndexOnStartup)) {
|
||||
if ("checksum".equals(checkIndexOnStartup)) {
|
||||
// physical verification only: verify all checksums for the latest commit
|
||||
IOException corrupt = null;
|
||||
MetadataSnapshot metadata = store.getMetadata();
|
||||
|
@ -1240,7 +1240,7 @@ public class IndexShard extends AbstractIndexShardComponent {
|
|||
return;
|
||||
}
|
||||
logger.warn("check index [failure]\n{}", new String(os.bytes().toBytes(), StandardCharsets.UTF_8));
|
||||
if ("fix".equalsIgnoreCase(checkIndexOnStartup)) {
|
||||
if ("fix".equals(checkIndexOnStartup)) {
|
||||
if (logger.isDebugEnabled()) {
|
||||
logger.debug("fixing index, writing new segments file ...");
|
||||
}
|
||||
|
|
|
@ -58,7 +58,7 @@ public class RestClusterGetSettingsAction extends BaseRestHandler {
|
|||
ClusterStateRequest clusterStateRequest = Requests.clusterStateRequest()
|
||||
.routingTable(false)
|
||||
.nodes(false);
|
||||
final boolean renderDefaults = request.paramAsBoolean("defaults", false);
|
||||
final boolean renderDefaults = request.paramAsBoolean("include_defaults", false);
|
||||
clusterStateRequest.local(request.paramAsBoolean("local", clusterStateRequest.local()));
|
||||
client.admin().cluster().state(clusterStateRequest, new RestBuilderListener<ClusterStateResponse>(channel) {
|
||||
@Override
|
||||
|
|
|
@ -29,6 +29,7 @@ import org.elasticsearch.cluster.metadata.MappingMetaData;
|
|||
import org.elasticsearch.common.Strings;
|
||||
import org.elasticsearch.common.collect.ImmutableOpenMap;
|
||||
import org.elasticsearch.common.inject.Inject;
|
||||
import org.elasticsearch.common.settings.IndexScopeSettings;
|
||||
import org.elasticsearch.common.settings.Settings;
|
||||
import org.elasticsearch.common.xcontent.ToXContent.Params;
|
||||
import org.elasticsearch.common.xcontent.XContentBuilder;
|
||||
|
@ -52,9 +53,12 @@ import static org.elasticsearch.rest.RestStatus.OK;
|
|||
*/
|
||||
public class RestGetIndicesAction extends BaseRestHandler {
|
||||
|
||||
private final IndexScopeSettings indexScopeSettings;
|
||||
|
||||
@Inject
|
||||
public RestGetIndicesAction(Settings settings, RestController controller, Client client) {
|
||||
public RestGetIndicesAction(Settings settings, RestController controller, Client client, IndexScopeSettings indexScopeSettings) {
|
||||
super(settings, controller, client);
|
||||
this.indexScopeSettings = indexScopeSettings;
|
||||
controller.registerHandler(GET, "/{index}", this);
|
||||
controller.registerHandler(GET, "/{index}/{type}", this);
|
||||
}
|
||||
|
@ -133,9 +137,15 @@ public class RestGetIndicesAction extends BaseRestHandler {
|
|||
}
|
||||
|
||||
private void writeSettings(Settings settings, XContentBuilder builder, Params params) throws IOException {
|
||||
final boolean renderDefaults = request.paramAsBoolean("include_defaults", false);
|
||||
builder.startObject(Fields.SETTINGS);
|
||||
settings.toXContent(builder, params);
|
||||
builder.endObject();
|
||||
if (renderDefaults) {
|
||||
builder.startObject("defaults");
|
||||
indexScopeSettings.diff(settings, settings).toXContent(builder, request);
|
||||
builder.endObject();
|
||||
}
|
||||
}
|
||||
|
||||
});
|
||||
|
@ -145,7 +155,6 @@ public class RestGetIndicesAction extends BaseRestHandler {
|
|||
static final XContentBuilderString ALIASES = new XContentBuilderString("aliases");
|
||||
static final XContentBuilderString MAPPINGS = new XContentBuilderString("mappings");
|
||||
static final XContentBuilderString SETTINGS = new XContentBuilderString("settings");
|
||||
static final XContentBuilderString WARMERS = new XContentBuilderString("warmers");
|
||||
}
|
||||
|
||||
}
|
||||
|
|
|
@ -57,7 +57,7 @@ public class RestGetSettingsAction extends BaseRestHandler {
|
|||
@Override
|
||||
public void handleRequest(final RestRequest request, final RestChannel channel, final Client client) {
|
||||
final String[] names = request.paramAsStringArrayOrEmptyIfAll("name");
|
||||
final boolean renderDefaults = request.paramAsBoolean("defaults", false);
|
||||
final boolean renderDefaults = request.paramAsBoolean("include_defaults", false);
|
||||
GetSettingsRequest getSettingsRequest = new GetSettingsRequest()
|
||||
.indices(Strings.splitStringByCommaToArray(request.param("index")))
|
||||
.indicesOptions(IndicesOptions.fromRequest(request, IndicesOptions.strictExpandOpen()))
|
||||
|
|
|
@ -121,7 +121,7 @@ import static org.elasticsearch.common.unit.TimeValue.timeValueMinutes;
|
|||
*/
|
||||
public class SearchService extends AbstractLifecycleComponent<SearchService> implements IndexEventListener {
|
||||
|
||||
public static final String NORMS_LOADING_KEY = "index.norms.loading";
|
||||
public static final Setting<Loading> INDEX_NORMS_LOADING_SETTING = new Setting<>("index.norms.loading", Loading.LAZY.toString(), (s) -> Loading.parse(s, Loading.LAZY), false, Setting.Scope.INDEX);
|
||||
public static final String DEFAULT_KEEPALIVE_KEY = "search.default_keep_alive";
|
||||
public static final String KEEPALIVE_INTERVAL_KEY = "search.keep_alive_interval";
|
||||
|
||||
|
@ -962,7 +962,7 @@ public class SearchService extends AbstractLifecycleComponent<SearchService> imp
|
|||
}
|
||||
@Override
|
||||
public TerminationHandle warmNewReaders(final IndexShard indexShard, final Engine.Searcher searcher) {
|
||||
final Loading defaultLoading = Loading.parse(indexShard.getIndexSettings().getSettings().get(NORMS_LOADING_KEY), Loading.LAZY);
|
||||
final Loading defaultLoading = indexShard.indexSettings().getValue(INDEX_NORMS_LOADING_SETTING);
|
||||
final MapperService mapperService = indexShard.mapperService();
|
||||
final ObjectSet<String> warmUp = new ObjectHashSet<>();
|
||||
for (DocumentMapper docMapper : mapperService.docMappers(false)) {
|
||||
|
|
|
@ -166,7 +166,7 @@ public class InternalEngineTests extends ESTestCase {
|
|||
}
|
||||
defaultSettings = IndexSettingsModule.newIndexSettings("test", Settings.builder()
|
||||
.put(IndexSettings.INDEX_GC_DELETES_SETTING, "1h") // make sure this doesn't kick in on us
|
||||
.put(EngineConfig.INDEX_CODEC_SETTING, codecName)
|
||||
.put(EngineConfig.INDEX_CODEC_SETTING.getKey(), codecName)
|
||||
.put(IndexMetaData.SETTING_VERSION_CREATED, Version.CURRENT)
|
||||
.build()); // TODO randomize more settings
|
||||
threadPool = new ThreadPool(getClass().getName());
|
||||
|
|
|
@ -118,7 +118,7 @@ public class ShadowEngineTests extends ESTestCase {
|
|||
}
|
||||
defaultSettings = IndexSettingsModule.newIndexSettings("test", Settings.builder()
|
||||
.put(IndexSettings.INDEX_GC_DELETES_SETTING, "1h") // make sure this doesn't kick in on us
|
||||
.put(EngineConfig.INDEX_CODEC_SETTING, codecName)
|
||||
.put(EngineConfig.INDEX_CODEC_SETTING.getKey(), codecName)
|
||||
.put(IndexMetaData.SETTING_VERSION_CREATED, Version.CURRENT)
|
||||
.build()); // TODO randomize more settings
|
||||
|
||||
|
|
|
@ -227,7 +227,7 @@ public class RandomExceptionCircuitBreakerIT extends ESIntegTestCase {
|
|||
private final double lowLevelRatio;
|
||||
|
||||
ThrowingSubReaderWrapper(Settings settings) {
|
||||
final long seed = settings.getAsLong(SETTING_INDEX_SEED, 0l);
|
||||
final long seed = ESIntegTestCase.INDEX_TEST_SEED_SETTING.get(settings);
|
||||
this.topLevelRatio = EXCEPTION_TOP_LEVEL_RATIO_SETTING.get(settings);
|
||||
this.lowLevelRatio = EXCEPTION_LOW_LEVEL_RATIO_SETTING.get(settings);
|
||||
this.random = new Random(seed);
|
||||
|
|
|
@ -180,7 +180,7 @@ public class SearchWithRandomExceptionsIT extends ESIntegTestCase {
|
|||
private final double lowLevelRatio;
|
||||
|
||||
ThrowingSubReaderWrapper(Settings settings) {
|
||||
final long seed = settings.getAsLong(SETTING_INDEX_SEED, 0l);
|
||||
final long seed = ESIntegTestCase.INDEX_TEST_SEED_SETTING.get(settings);
|
||||
this.topLevelRatio = settings.getAsDouble(EXCEPTION_TOP_LEVEL_RATIO_KEY, 0.1d);
|
||||
this.lowLevelRatio = settings.getAsDouble(EXCEPTION_LOW_LEVEL_RATIO_KEY, 0.1d);
|
||||
this.random = new Random(seed);
|
||||
|
|
|
@ -19,7 +19,7 @@
|
|||
"type" : "time",
|
||||
"description" : "Explicit operation timeout"
|
||||
},
|
||||
"defaults": {
|
||||
"include_defaults": {
|
||||
"type": "boolean",
|
||||
"description": "Whether to return all default clusters setting.",
|
||||
"default": false
|
||||
|
|
|
@ -44,6 +44,11 @@
|
|||
"type": "boolean",
|
||||
"description": "Whether to return version and creation date values in human-readable format.",
|
||||
"default": false
|
||||
},
|
||||
"include_defaults": {
|
||||
"type": "boolean",
|
||||
"description": "Whether to return all default setting for each of the indices.",
|
||||
"default": false
|
||||
}
|
||||
}
|
||||
},
|
||||
|
|
|
@ -43,7 +43,7 @@
|
|||
"description": "Whether to return version and creation date values in human-readable format.",
|
||||
"default": false
|
||||
},
|
||||
"defaults": {
|
||||
"include_defaults": {
|
||||
"type": "boolean",
|
||||
"description": "Whether to return all default setting for each of the indices.",
|
||||
"default": false
|
||||
|
|
|
@ -1,39 +1,33 @@
|
|||
---
|
||||
setup:
|
||||
- do:
|
||||
indices.create:
|
||||
index: test-index
|
||||
body:
|
||||
settings:
|
||||
index:
|
||||
refresh_interval: "10s"
|
||||
|
||||
refresh_interval: 10s
|
||||
index: test-index
|
||||
---
|
||||
"Test reset index settings":
|
||||
Test reset index settings:
|
||||
- do:
|
||||
indices.get_settings:
|
||||
index: test-index
|
||||
flat_settings: true
|
||||
|
||||
index: test-index
|
||||
- match:
|
||||
test-index.settings.index.refresh_interval: "10s"
|
||||
|
||||
test-index.settings.index\.refresh_interval: "10s"
|
||||
- do:
|
||||
indices.put_settings:
|
||||
body:
|
||||
refresh_interval: null
|
||||
|
||||
- do:
|
||||
indices.get_settings:
|
||||
flat_settings: false
|
||||
|
||||
- match:
|
||||
- match: {index.settings: {}}
|
||||
|
||||
- is_false:
|
||||
test-index.settings.index\.refresh_interval
|
||||
- do:
|
||||
indices.get_settings:
|
||||
include_defaults: true
|
||||
flat_settings: true
|
||||
defaults: true
|
||||
|
||||
index: test-index
|
||||
- match:
|
||||
test-index.settings.index.refresh_interval: "1s"
|
||||
|
||||
test-index.defaults.index\.refresh_interval: "1s"
|
||||
|
|
|
@ -75,6 +75,7 @@ import org.elasticsearch.common.Strings;
|
|||
import org.elasticsearch.common.collect.ImmutableOpenMap;
|
||||
import org.elasticsearch.common.collect.Tuple;
|
||||
import org.elasticsearch.common.network.NetworkAddress;
|
||||
import org.elasticsearch.common.network.NetworkModule;
|
||||
import org.elasticsearch.common.regex.Regex;
|
||||
import org.elasticsearch.common.settings.Setting;
|
||||
import org.elasticsearch.common.settings.Settings;
|
||||
|
@ -161,6 +162,7 @@ import java.util.concurrent.TimeUnit;
|
|||
import java.util.concurrent.atomic.AtomicInteger;
|
||||
import java.util.concurrent.atomic.AtomicLong;
|
||||
import java.util.function.BooleanSupplier;
|
||||
import java.util.function.Function;
|
||||
|
||||
import static org.elasticsearch.client.Requests.syncedFlushRequest;
|
||||
import static org.elasticsearch.cluster.metadata.IndexMetaData.SETTING_NUMBER_OF_REPLICAS;
|
||||
|
@ -268,7 +270,7 @@ public abstract class ESIntegTestCase extends ESTestCase {
|
|||
* The value of this seed can be used to initialize a random context for a specific index.
|
||||
* It's set once per test via a generic index template.
|
||||
*/
|
||||
public static final String SETTING_INDEX_SEED = "index.tests.seed";
|
||||
public static final Setting<Long> INDEX_TEST_SEED_SETTING = Setting.longSetting("index.tests.seed", 0, Long.MIN_VALUE, false, Setting.Scope.INDEX);
|
||||
|
||||
/**
|
||||
* A boolean value to enable or disable mock modules. This is useful to test the
|
||||
|
@ -367,8 +369,11 @@ public abstract class ESIntegTestCase extends ESTestCase {
|
|||
// TODO move settings for random directory etc here into the index based randomized settings.
|
||||
if (cluster().size() > 0) {
|
||||
Settings.Builder randomSettingsBuilder =
|
||||
setRandomIndexSettings(getRandom(), Settings.builder())
|
||||
.put(SETTING_INDEX_SEED, getRandom().nextLong());
|
||||
setRandomIndexSettings(getRandom(), Settings.builder());
|
||||
if (isInternalCluster()) {
|
||||
// this is only used by mock plugins and if the cluster is not internal we just can't set it
|
||||
randomSettingsBuilder.put(INDEX_TEST_SEED_SETTING.getKey(), getRandom().nextLong());
|
||||
}
|
||||
|
||||
randomSettingsBuilder.put(SETTING_NUMBER_OF_SHARDS, numberOfShards())
|
||||
.put(SETTING_NUMBER_OF_REPLICAS, numberOfReplicas());
|
||||
|
@ -475,7 +480,7 @@ public abstract class ESIntegTestCase extends ESTestCase {
|
|||
}
|
||||
|
||||
if (random.nextBoolean()) {
|
||||
builder.put("index.shard.check_on_startup", randomFrom(random, "false", "checksum", "true"));
|
||||
builder.put(IndexSettings.INDEX_CHECK_ON_STARTUP.getKey(), randomFrom("false", "checksum", "true"));
|
||||
}
|
||||
|
||||
if (randomBoolean()) {
|
||||
|
@ -505,7 +510,7 @@ public abstract class ESIntegTestCase extends ESTestCase {
|
|||
|
||||
private static Settings.Builder setRandomIndexNormsLoading(Random random, Settings.Builder builder) {
|
||||
if (random.nextBoolean()) {
|
||||
builder.put(SearchService.NORMS_LOADING_KEY, RandomPicks.randomFrom(random, Arrays.asList(MappedFieldType.Loading.EAGER, MappedFieldType.Loading.LAZY)));
|
||||
builder.put(SearchService.INDEX_NORMS_LOADING_SETTING.getKey(), RandomPicks.randomFrom(random, Arrays.asList(MappedFieldType.Loading.EAGER, MappedFieldType.Loading.LAZY)));
|
||||
}
|
||||
return builder;
|
||||
}
|
||||
|
@ -1824,9 +1829,25 @@ public abstract class ESIntegTestCase extends ESTestCase {
|
|||
mocks.add(AssertingLocalTransport.TestPlugin.class);
|
||||
}
|
||||
}
|
||||
mocks.add(TestSeedPlugin.class);
|
||||
return Collections.unmodifiableList(mocks);
|
||||
}
|
||||
|
||||
public static final class TestSeedPlugin extends Plugin {
|
||||
@Override
|
||||
public String name() {
|
||||
return "test-seed-plugin";
|
||||
}
|
||||
@Override
|
||||
public String description() {
|
||||
return "a test plugin that registeres index.tests.seed as an index setting";
|
||||
}
|
||||
public void onModule(SettingsModule module) {
|
||||
module.registerSetting(INDEX_TEST_SEED_SETTING);
|
||||
}
|
||||
|
||||
}
|
||||
|
||||
/**
|
||||
* Returns the client ratio configured via
|
||||
*/
|
||||
|
|
|
@ -95,7 +95,7 @@ public final class MockEngineSupport {
|
|||
shardId = config.getShardId();
|
||||
filterCache = config.getQueryCache();
|
||||
filterCachingPolicy = config.getQueryCachingPolicy();
|
||||
final long seed = settings.getAsLong(ESIntegTestCase.SETTING_INDEX_SEED, 0l);
|
||||
final long seed = config.getIndexSettings().getValue(ESIntegTestCase.INDEX_TEST_SEED_SETTING);
|
||||
Random random = new Random(seed);
|
||||
final double ratio = WRAP_READER_RATIO.get(settings);
|
||||
boolean wrapReader = random.nextDouble() < ratio;
|
||||
|
|
|
@ -82,7 +82,7 @@ public class MockFSDirectoryService extends FsDirectoryService {
|
|||
public MockFSDirectoryService(IndexSettings idxSettings, IndexStore indexStore, final ShardPath path) {
|
||||
super(idxSettings, indexStore, path);
|
||||
Settings indexSettings = idxSettings.getSettings();
|
||||
final long seed = indexSettings.getAsLong(ESIntegTestCase.SETTING_INDEX_SEED, 0l);
|
||||
final long seed = idxSettings.getValue(ESIntegTestCase.INDEX_TEST_SEED_SETTING);
|
||||
this.random = new Random(seed);
|
||||
|
||||
randomIOExceptionRate = RANDOM_IO_EXCEPTION_RATE_SETTING.get(indexSettings);
|
||||
|
|
|
@ -69,7 +69,7 @@ public class AssertingLocalTransport extends LocalTransport {
|
|||
@Inject
|
||||
public AssertingLocalTransport(Settings settings, ThreadPool threadPool, Version version, NamedWriteableRegistry namedWriteableRegistry) {
|
||||
super(settings, threadPool, version, namedWriteableRegistry);
|
||||
final long seed = settings.getAsLong(ESIntegTestCase.SETTING_INDEX_SEED, 0l);
|
||||
final long seed = ESIntegTestCase.INDEX_TEST_SEED_SETTING.get(settings);
|
||||
random = new Random(seed);
|
||||
minVersion = settings.getAsVersion(ASSERTING_TRANSPORT_MIN_VERSION_KEY, Version.V_0_18_0);
|
||||
maxVersion = settings.getAsVersion(ASSERTING_TRANSPORT_MAX_VERSION_KEY, Version.CURRENT);
|
||||
|
|
Loading…
Reference in New Issue