Move discovery.* settings to new setting infrastructure

Closes #16182
This commit is contained in:
Yannick Welsch 2016-01-22 14:02:40 +01:00
parent 4c7f3d40e2
commit 296b48b9d1
37 changed files with 172 additions and 135 deletions

View File

@ -19,6 +19,7 @@
package org.elasticsearch.common;
import org.elasticsearch.common.settings.Setting;
import org.elasticsearch.common.settings.Settings;
import java.lang.reflect.Method;
@ -40,7 +41,7 @@ import java.util.concurrent.ThreadLocalRandom;
* setting a reproducible seed. When running the Elasticsearch server
* process, non-reproducible sources of randomness are provided (unless
* a setting is provided for a module that exposes a seed setting (e.g.,
* DiscoveryService#SETTING_DISCOVERY_SEED)).
* DiscoveryService#DISCOVERY_SEED_SETTING)).
*/
public final class Randomness {
private static final Method currentMethod;
@ -68,13 +69,12 @@ public final class Randomness {
* seed in the settings with the key setting.
*
* @param settings the settings containing the seed
* @param setting the key to access the seed
* @param setting the setting to access the seed
* @return a reproducible source of randomness
*/
public static Random get(Settings settings, String setting) {
Long maybeSeed = settings.getAsLong(setting, null);
if (maybeSeed != null) {
return new Random(maybeSeed);
public static Random get(Settings settings, Setting<Long> setting) {
if (setting.exists(settings)) {
return new Random(setting.get(settings));
} else {
return get();
}

View File

@ -36,11 +36,15 @@ import org.elasticsearch.cluster.routing.allocation.decider.SnapshotInProgressAl
import org.elasticsearch.cluster.routing.allocation.decider.ThrottlingAllocationDecider;
import org.elasticsearch.cluster.service.InternalClusterService;
import org.elasticsearch.common.logging.ESLoggerFactory;
import org.elasticsearch.discovery.DiscoveryModule;
import org.elasticsearch.discovery.DiscoveryService;
import org.elasticsearch.discovery.DiscoverySettings;
import org.elasticsearch.discovery.zen.ZenDiscovery;
import org.elasticsearch.discovery.zen.elect.ElectMasterService;
import org.elasticsearch.env.Environment;
import org.elasticsearch.gateway.GatewayService;
import org.elasticsearch.discovery.zen.fd.FaultDetection;
import org.elasticsearch.discovery.zen.ping.unicast.UnicastZenPing;
import org.elasticsearch.gateway.PrimaryShardAllocator;
import org.elasticsearch.index.IndexSettings;
import org.elasticsearch.index.store.IndexStoreConfig;
@ -194,6 +198,26 @@ public final class ClusterSettings extends AbstractScopedSettings {
Environment.PATH_REPO_SETTING,
Environment.PATH_SCRIPTS_SETTING,
Environment.PATH_SHARED_DATA_SETTING,
Environment.PIDFILE_SETTING
)));
Environment.PIDFILE_SETTING,
DiscoveryService.DISCOVERY_SEED_SETTING,
DiscoveryService.INITIAL_STATE_TIMEOUT_SETTING,
DiscoveryModule.DISCOVERY_TYPE_SETTING,
DiscoveryModule.ZEN_MASTER_SERVICE_TYPE_SETTING,
FaultDetection.PING_RETRIES_SETTING,
FaultDetection.PING_TIMEOUT_SETTING,
FaultDetection.REGISTER_CONNECTION_LISTENER_SETTING,
FaultDetection.PING_INTERVAL_SETTING,
FaultDetection.CONNECT_ON_NETWORK_DISCONNECT_SETTING,
ZenDiscovery.PING_TIMEOUT_SETTING,
ZenDiscovery.JOIN_TIMEOUT_SETTING,
ZenDiscovery.JOIN_RETRY_ATTEMPTS_SETTING,
ZenDiscovery.JOIN_RETRY_DELAY_SETTING,
ZenDiscovery.MAX_PINGS_FROM_ANOTHER_MASTER_SETTING,
ZenDiscovery.SEND_LEAVE_REQUEST_SETTING,
ZenDiscovery.MASTER_ELECTION_FILTER_CLIENT_SETTING,
ZenDiscovery.MASTER_ELECTION_WAIT_FOR_JOINS_TIMEOUT_SETTING,
ZenDiscovery.MASTER_ELECTION_FILTER_DATA_SETTING,
UnicastZenPing.DISCOVERY_ZEN_PING_UNICAST_HOSTS_SETTING,
UnicastZenPing.DISCOVERY_ZEN_PING_UNICAST_CONCURRENT_CONNECTS_SETTING
)));
}

View File

@ -22,6 +22,7 @@ package org.elasticsearch.discovery;
import org.elasticsearch.cluster.node.DiscoveryNode;
import org.elasticsearch.common.inject.AbstractModule;
import org.elasticsearch.common.inject.multibindings.Multibinder;
import org.elasticsearch.common.settings.Setting;
import org.elasticsearch.common.settings.Settings;
import org.elasticsearch.common.util.ExtensionPoint;
import org.elasticsearch.discovery.local.LocalDiscovery;
@ -36,14 +37,17 @@ import java.util.ArrayList;
import java.util.HashMap;
import java.util.List;
import java.util.Map;
import java.util.function.Function;
/**
* A module for loading classes for node discovery.
*/
public class DiscoveryModule extends AbstractModule {
public static final String DISCOVERY_TYPE_KEY = "discovery.type";
public static final String ZEN_MASTER_SERVICE_TYPE_KEY = "discovery.zen.masterservice.type";
public static final Setting<String> DISCOVERY_TYPE_SETTING = new Setting<>("discovery.type",
settings -> DiscoveryNode.localNode(settings) ? "local" : "zen", Function.identity(), false, Setting.Scope.CLUSTER);
public static final Setting<String> ZEN_MASTER_SERVICE_TYPE_SETTING = new Setting<>("discovery.zen.masterservice.type",
"zen", Function.identity(), false, Setting.Scope.CLUSTER);
private final Settings settings;
private final List<Class<? extends UnicastHostsProvider>> unicastHostProviders = new ArrayList<>();
@ -93,15 +97,14 @@ public class DiscoveryModule extends AbstractModule {
@Override
protected void configure() {
String defaultType = DiscoveryNode.localNode(settings) ? "local" : "zen";
String discoveryType = settings.get(DISCOVERY_TYPE_KEY, defaultType);
String discoveryType = DISCOVERY_TYPE_SETTING.get(settings);
Class<? extends Discovery> discoveryClass = discoveryTypes.get(discoveryType);
if (discoveryClass == null) {
throw new IllegalArgumentException("Unknown Discovery type [" + discoveryType + "]");
}
if (discoveryType.equals("local") == false) {
String masterServiceTypeKey = settings.get(ZEN_MASTER_SERVICE_TYPE_KEY, "zen");
String masterServiceTypeKey = ZEN_MASTER_SERVICE_TYPE_SETTING.get(settings);
final Class<? extends ElectMasterService> masterService = masterServiceType.get(masterServiceTypeKey);
if (masterService == null) {
throw new IllegalArgumentException("Unknown master service type [" + masterServiceTypeKey + "]");
@ -121,4 +124,4 @@ public class DiscoveryModule extends AbstractModule {
bind(Discovery.class).to(discoveryClass).asEagerSingleton();
bind(DiscoveryService.class).asEagerSingleton();
}
}
}

View File

@ -27,6 +27,7 @@ import org.elasticsearch.common.Randomness;
import org.elasticsearch.common.Strings;
import org.elasticsearch.common.component.AbstractLifecycleComponent;
import org.elasticsearch.common.inject.Inject;
import org.elasticsearch.common.settings.Setting;
import org.elasticsearch.common.settings.Settings;
import org.elasticsearch.common.unit.TimeValue;
@ -39,8 +40,8 @@ import java.util.concurrent.TimeUnit;
*/
public class DiscoveryService extends AbstractLifecycleComponent<DiscoveryService> {
public static final String SETTING_INITIAL_STATE_TIMEOUT = "discovery.initial_state_timeout";
public static final String SETTING_DISCOVERY_SEED = "discovery.id.seed";
public static final Setting<TimeValue> INITIAL_STATE_TIMEOUT_SETTING = Setting.positiveTimeSetting("discovery.initial_state_timeout", TimeValue.timeValueSeconds(30), false, Setting.Scope.CLUSTER);
public static final Setting<Long> DISCOVERY_SEED_SETTING = Setting.longSetting("discovery.id.seed", 0l, Long.MIN_VALUE, false, Setting.Scope.CLUSTER);
private static class InitialStateListener implements InitialStateDiscoveryListener {
@ -71,7 +72,7 @@ public class DiscoveryService extends AbstractLifecycleComponent<DiscoveryServic
super(settings);
this.discoverySettings = discoverySettings;
this.discovery = discovery;
this.initialStateTimeout = settings.getAsTime(SETTING_INITIAL_STATE_TIMEOUT, TimeValue.timeValueSeconds(30));
this.initialStateTimeout = INITIAL_STATE_TIMEOUT_SETTING.get(settings);
}
public ClusterBlock getNoMasterBlock() {
@ -132,7 +133,7 @@ public class DiscoveryService extends AbstractLifecycleComponent<DiscoveryServic
}
public static String generateNodeId(Settings settings) {
Random random = Randomness.get(settings, DiscoveryService.SETTING_DISCOVERY_SEED);
Random random = Randomness.get(settings, DiscoveryService.DISCOVERY_SEED_SETTING);
return Strings.randomBase64UUID(random);
}
}

View File

@ -90,15 +90,17 @@ import static org.elasticsearch.common.unit.TimeValue.timeValueSeconds;
public class ZenDiscovery extends AbstractLifecycleComponent<Discovery> implements Discovery, PingContextProvider {
public final static Setting<Boolean> REJOIN_ON_MASTER_GONE_SETTING = Setting.boolSetting("discovery.zen.rejoin_on_master_gone", true, true, Setting.Scope.CLUSTER);
public final static String SETTING_PING_TIMEOUT = "discovery.zen.ping_timeout";
public final static String SETTING_JOIN_TIMEOUT = "discovery.zen.join_timeout";
public final static String SETTING_JOIN_RETRY_ATTEMPTS = "discovery.zen.join_retry_attempts";
public final static String SETTING_JOIN_RETRY_DELAY = "discovery.zen.join_retry_delay";
public final static String SETTING_MAX_PINGS_FROM_ANOTHER_MASTER = "discovery.zen.max_pings_from_another_master";
public final static String SETTING_SEND_LEAVE_REQUEST = "discovery.zen.send_leave_request";
public final static String SETTING_MASTER_ELECTION_FILTER_CLIENT = "discovery.zen.master_election.filter_client";
public final static String SETTING_MASTER_ELECTION_WAIT_FOR_JOINS_TIMEOUT = "discovery.zen.master_election.wait_for_joins_timeout";
public final static String SETTING_MASTER_ELECTION_FILTER_DATA = "discovery.zen.master_election.filter_data";
public final static Setting<TimeValue> PING_TIMEOUT_SETTING = Setting.positiveTimeSetting("discovery.zen.ping_timeout", timeValueSeconds(3), false, Setting.Scope.CLUSTER);
public final static Setting<TimeValue> JOIN_TIMEOUT_SETTING = Setting.timeSetting("discovery.zen.join_timeout",
settings -> TimeValue.timeValueMillis(PING_TIMEOUT_SETTING.get(settings).millis() * 20).toString(), TimeValue.timeValueMillis(0), false, Setting.Scope.CLUSTER);
public final static Setting<Integer> JOIN_RETRY_ATTEMPTS_SETTING = Setting.intSetting("discovery.zen.join_retry_attempts", 3, 1, false, Setting.Scope.CLUSTER);
public final static Setting<TimeValue> JOIN_RETRY_DELAY_SETTING = Setting.positiveTimeSetting("discovery.zen.join_retry_delay", TimeValue.timeValueMillis(100), false, Setting.Scope.CLUSTER);
public final static Setting<Integer> MAX_PINGS_FROM_ANOTHER_MASTER_SETTING = Setting.intSetting("discovery.zen.max_pings_from_another_master", 3, 1, false, Setting.Scope.CLUSTER);
public final static Setting<Boolean> SEND_LEAVE_REQUEST_SETTING = Setting.boolSetting("discovery.zen.send_leave_request", true, false, Setting.Scope.CLUSTER);
public final static Setting<Boolean> MASTER_ELECTION_FILTER_CLIENT_SETTING = Setting.boolSetting("discovery.zen.master_election.filter_client", true, false, Setting.Scope.CLUSTER);
public final static Setting<TimeValue> MASTER_ELECTION_WAIT_FOR_JOINS_TIMEOUT_SETTING = Setting.timeSetting("discovery.zen.master_election.wait_for_joins_timeout",
settings -> TimeValue.timeValueMillis(JOIN_TIMEOUT_SETTING.get(settings).millis() / 2).toString(), TimeValue.timeValueMillis(0), false, Setting.Scope.CLUSTER);
public final static Setting<Boolean> MASTER_ELECTION_FILTER_DATA_SETTING = Setting.boolSetting("discovery.zen.master_election.filter_data", false, false, Setting.Scope.CLUSTER);
public static final String DISCOVERY_REJOIN_ACTION_NAME = "internal:discovery/zen/rejoin";
@ -164,26 +166,19 @@ public class ZenDiscovery extends AbstractLifecycleComponent<Discovery> implemen
this.discoverySettings = discoverySettings;
this.pingService = pingService;
this.electMaster = electMasterService;
this.pingTimeout = settings.getAsTime(SETTING_PING_TIMEOUT, timeValueSeconds(3));
this.pingTimeout = PING_TIMEOUT_SETTING.get(settings);
this.joinTimeout = settings.getAsTime(SETTING_JOIN_TIMEOUT, TimeValue.timeValueMillis(this.pingTimeout.millis() * 20));
this.joinRetryAttempts = settings.getAsInt(SETTING_JOIN_RETRY_ATTEMPTS, 3);
this.joinRetryDelay = settings.getAsTime(SETTING_JOIN_RETRY_DELAY, TimeValue.timeValueMillis(100));
this.maxPingsFromAnotherMaster = settings.getAsInt(SETTING_MAX_PINGS_FROM_ANOTHER_MASTER, 3);
this.sendLeaveRequest = settings.getAsBoolean(SETTING_SEND_LEAVE_REQUEST, true);
this.joinTimeout = JOIN_TIMEOUT_SETTING.get(settings);
this.joinRetryAttempts = JOIN_RETRY_ATTEMPTS_SETTING.get(settings);
this.joinRetryDelay = JOIN_RETRY_DELAY_SETTING.get(settings);
this.maxPingsFromAnotherMaster = MAX_PINGS_FROM_ANOTHER_MASTER_SETTING.get(settings);
this.sendLeaveRequest = SEND_LEAVE_REQUEST_SETTING.get(settings);
this.masterElectionFilterClientNodes = settings.getAsBoolean(SETTING_MASTER_ELECTION_FILTER_CLIENT, true);
this.masterElectionFilterDataNodes = settings.getAsBoolean(SETTING_MASTER_ELECTION_FILTER_DATA, false);
this.masterElectionWaitForJoinsTimeout = settings.getAsTime(SETTING_MASTER_ELECTION_WAIT_FOR_JOINS_TIMEOUT, TimeValue.timeValueMillis(joinTimeout.millis() / 2));
this.masterElectionFilterClientNodes = MASTER_ELECTION_FILTER_CLIENT_SETTING.get(settings);
this.masterElectionFilterDataNodes = MASTER_ELECTION_FILTER_DATA_SETTING.get(settings);
this.masterElectionWaitForJoinsTimeout = MASTER_ELECTION_WAIT_FOR_JOINS_TIMEOUT_SETTING.get(settings);
this.rejoinOnMasterGone = REJOIN_ON_MASTER_GONE_SETTING.get(settings);
if (this.joinRetryAttempts < 1) {
throw new IllegalArgumentException("'" + SETTING_JOIN_RETRY_ATTEMPTS + "' must be a positive number. got [" + SETTING_JOIN_RETRY_ATTEMPTS + "]");
}
if (this.maxPingsFromAnotherMaster < 1) {
throw new IllegalArgumentException("'" + SETTING_MAX_PINGS_FROM_ANOTHER_MASTER + "' must be a positive number. got [" + this.maxPingsFromAnotherMaster + "]");
}
logger.debug("using ping_timeout [{}], join.timeout [{}], master_election.filter_client [{}], master_election.filter_data [{}]", this.pingTimeout, joinTimeout, masterElectionFilterClientNodes, masterElectionFilterDataNodes);
clusterSettings.addSettingsUpdateConsumer(ElectMasterService.DISCOVERY_ZEN_MINIMUM_MASTER_NODES_SETTING, this::handleMinimumMasterNodesChanged, (value) -> {

View File

@ -21,6 +21,8 @@ package org.elasticsearch.discovery.zen.fd;
import org.elasticsearch.cluster.ClusterName;
import org.elasticsearch.cluster.node.DiscoveryNode;
import org.elasticsearch.common.component.AbstractComponent;
import org.elasticsearch.common.settings.Setting;
import org.elasticsearch.common.settings.Setting.Scope;
import org.elasticsearch.common.settings.Settings;
import org.elasticsearch.common.unit.TimeValue;
import org.elasticsearch.threadpool.ThreadPool;
@ -35,11 +37,11 @@ import static org.elasticsearch.common.unit.TimeValue.timeValueSeconds;
*/
public abstract class FaultDetection extends AbstractComponent {
public static final String SETTING_CONNECT_ON_NETWORK_DISCONNECT = "discovery.zen.fd.connect_on_network_disconnect";
public static final String SETTING_PING_INTERVAL = "discovery.zen.fd.ping_interval";
public static final String SETTING_PING_TIMEOUT = "discovery.zen.fd.ping_timeout";
public static final String SETTING_PING_RETRIES = "discovery.zen.fd.ping_retries";
public static final String SETTING_REGISTER_CONNECTION_LISTENER = "discovery.zen.fd.register_connection_listener";
public static final Setting<Boolean> CONNECT_ON_NETWORK_DISCONNECT_SETTING = Setting.boolSetting("discovery.zen.fd.connect_on_network_disconnect", false, false, Scope.CLUSTER);
public static final Setting<TimeValue> PING_INTERVAL_SETTING = Setting.positiveTimeSetting("discovery.zen.fd.ping_interval", timeValueSeconds(1), false, Scope.CLUSTER);
public static final Setting<TimeValue> PING_TIMEOUT_SETTING = Setting.timeSetting("discovery.zen.fd.ping_timeout", timeValueSeconds(30), false, Scope.CLUSTER);
public static final Setting<Integer> PING_RETRIES_SETTING = Setting.intSetting("discovery.zen.fd.ping_retries", 3, false, Scope.CLUSTER);
public static final Setting<Boolean> REGISTER_CONNECTION_LISTENER_SETTING = Setting.boolSetting("discovery.zen.fd.register_connection_listener", true, false, Scope.CLUSTER);
protected final ThreadPool threadPool;
protected final ClusterName clusterName;
@ -60,11 +62,11 @@ public abstract class FaultDetection extends AbstractComponent {
this.transportService = transportService;
this.clusterName = clusterName;
this.connectOnNetworkDisconnect = settings.getAsBoolean(SETTING_CONNECT_ON_NETWORK_DISCONNECT, false);
this.pingInterval = settings.getAsTime(SETTING_PING_INTERVAL, timeValueSeconds(1));
this.pingRetryTimeout = settings.getAsTime(SETTING_PING_TIMEOUT, timeValueSeconds(30));
this.pingRetryCount = settings.getAsInt(SETTING_PING_RETRIES, 3);
this.registerConnectionListener = settings.getAsBoolean(SETTING_REGISTER_CONNECTION_LISTENER, true);
this.connectOnNetworkDisconnect = CONNECT_ON_NETWORK_DISCONNECT_SETTING.get(settings);
this.pingInterval = PING_INTERVAL_SETTING.get(settings);
this.pingRetryTimeout = PING_TIMEOUT_SETTING.get(settings);
this.pingRetryCount = PING_RETRIES_SETTING.get(settings);
this.registerConnectionListener = REGISTER_CONNECTION_LISTENER_SETTING.get(settings);
this.connectionListener = new FDConnectionListener();
if (registerConnectionListener) {

View File

@ -31,6 +31,7 @@ import org.elasticsearch.common.component.AbstractLifecycleComponent;
import org.elasticsearch.common.inject.Inject;
import org.elasticsearch.common.io.stream.StreamInput;
import org.elasticsearch.common.io.stream.StreamOutput;
import org.elasticsearch.common.settings.Setting;
import org.elasticsearch.common.settings.Settings;
import org.elasticsearch.common.transport.TransportAddress;
import org.elasticsearch.common.unit.TimeValue;
@ -58,6 +59,7 @@ import java.io.Closeable;
import java.io.IOException;
import java.util.ArrayList;
import java.util.Arrays;
import java.util.Collections;
import java.util.HashSet;
import java.util.List;
import java.util.Map;
@ -72,6 +74,7 @@ import java.util.concurrent.TimeUnit;
import java.util.concurrent.atomic.AtomicBoolean;
import java.util.concurrent.atomic.AtomicInteger;
import java.util.concurrent.atomic.AtomicReference;
import java.util.function.Function;
import static org.elasticsearch.common.unit.TimeValue.readTimeValue;
import static org.elasticsearch.common.util.concurrent.ConcurrentCollections.newConcurrentMap;
@ -83,7 +86,8 @@ import static org.elasticsearch.discovery.zen.ping.ZenPing.PingResponse.readPing
public class UnicastZenPing extends AbstractLifecycleComponent<ZenPing> implements ZenPing {
public static final String ACTION_NAME = "internal:discovery/zen/unicast";
public static final String DISCOVERY_ZEN_PING_UNICAST_HOSTS = "discovery.zen.ping.unicast.hosts";
public static final Setting<List<String>> DISCOVERY_ZEN_PING_UNICAST_HOSTS_SETTING = Setting.listSetting("discovery.zen.ping.unicast.hosts", Collections.emptyList(), Function.identity(), false, Setting.Scope.CLUSTER);
public static final Setting<Integer> DISCOVERY_ZEN_PING_UNICAST_CONCURRENT_CONNECTS_SETTING = Setting.intSetting("discovery.zen.ping.unicast.concurrent_connects", 10, 0, false, Setting.Scope.CLUSTER);
// these limits are per-address
public static final int LIMIT_FOREIGN_PORTS_COUNT = 1;
@ -135,13 +139,8 @@ public class UnicastZenPing extends AbstractLifecycleComponent<ZenPing> implemen
}
}
this.concurrentConnects = this.settings.getAsInt("discovery.zen.ping.unicast.concurrent_connects", 10);
String[] hostArr = this.settings.getAsArray(DISCOVERY_ZEN_PING_UNICAST_HOSTS);
// trim the hosts
for (int i = 0; i < hostArr.length; i++) {
hostArr[i] = hostArr[i].trim();
}
List<String> hosts = CollectionUtils.arrayAsArrayList(hostArr);
this.concurrentConnects = DISCOVERY_ZEN_PING_UNICAST_CONCURRENT_CONNECTS_SETTING.get(settings);
List<String> hosts = DISCOVERY_ZEN_PING_UNICAST_HOSTS_SETTING.get(settings);
final int limitPortCounts;
if (hosts.isEmpty()) {
// if unicast hosts are not specified, fill with simple defaults on the local machine

View File

@ -44,6 +44,7 @@ import org.elasticsearch.common.inject.Inject;
import org.elasticsearch.common.regex.Regex;
import org.elasticsearch.common.settings.Settings;
import org.elasticsearch.common.util.concurrent.ConcurrentCollections;
import org.elasticsearch.discovery.DiscoveryModule;
import org.elasticsearch.discovery.DiscoveryService;
import org.elasticsearch.env.Environment;
import org.elasticsearch.gateway.GatewayService;
@ -101,8 +102,8 @@ public class TribeService extends AbstractLifecycleComponent<TribeService> {
// its a tribe configured node..., force settings
Settings.Builder sb = Settings.builder().put(settings);
sb.put("node.client", true); // this node should just act as a node client
sb.put("discovery.type", "local"); // a tribe node should not use zen discovery
sb.put("discovery.initial_state_timeout", 0); // nothing is going to be discovered, since no master will be elected
sb.put(DiscoveryModule.DISCOVERY_TYPE_SETTING.getKey(), "local"); // a tribe node should not use zen discovery
sb.put(DiscoveryService.INITIAL_STATE_TIMEOUT_SETTING.getKey(), 0); // nothing is going to be discovered, since no master will be elected
if (sb.get("cluster.name") == null) {
sb.put("cluster.name", "tribe_" + Strings.randomBase64UUID()); // make sure it won't join other tribe nodes in the same JVM
}

View File

@ -60,8 +60,8 @@ public class IndexingMasterFailoverIT extends ESIntegTestCase {
logger.info("--> start 4 nodes, 3 master, 1 data");
final Settings sharedSettings = Settings.builder()
.put(FaultDetection.SETTING_PING_TIMEOUT, "1s") // for hitting simulated network failures quickly
.put(FaultDetection.SETTING_PING_RETRIES, "1") // for hitting simulated network failures quickly
.put(FaultDetection.PING_TIMEOUT_SETTING.getKey(), "1s") // for hitting simulated network failures quickly
.put(FaultDetection.PING_RETRIES_SETTING.getKey(), "1") // for hitting simulated network failures quickly
.put("discovery.zen.join_timeout", "10s") // still long to induce failures but to long so test won't time out
.put(DiscoverySettings.PUBLISH_TIMEOUT_SETTING.getKey(), "1s") // <-- for hitting simulated network failures quickly
.put(ElectMasterService.DISCOVERY_ZEN_MINIMUM_MASTER_NODES_SETTING.getKey(), 2)

View File

@ -642,7 +642,7 @@ public class ClusterServiceIT extends ESIntegTestCase {
Settings settings = settingsBuilder()
.put("discovery.type", "zen")
.put("discovery.zen.minimum_master_nodes", 1)
.put(ZenDiscovery.SETTING_PING_TIMEOUT, "400ms")
.put(ZenDiscovery.PING_TIMEOUT_SETTING.getKey(), "400ms")
.put("discovery.initial_state_timeout", "500ms")
.build();

View File

@ -77,7 +77,7 @@ public class MinimumMasterNodesIT extends ESIntegTestCase {
Settings settings = settingsBuilder()
.put("discovery.type", "zen")
.put("discovery.zen.minimum_master_nodes", 2)
.put(ZenDiscovery.SETTING_PING_TIMEOUT, "200ms")
.put(ZenDiscovery.PING_TIMEOUT_SETTING.getKey(), "200ms")
.put("discovery.initial_state_timeout", "500ms")
.build();
@ -189,7 +189,7 @@ public class MinimumMasterNodesIT extends ESIntegTestCase {
Settings settings = settingsBuilder()
.put("discovery.type", "zen")
.put("discovery.zen.minimum_master_nodes", 3)
.put(ZenDiscovery.SETTING_PING_TIMEOUT, "1s")
.put(ZenDiscovery.PING_TIMEOUT_SETTING.getKey(), "1s")
.put("discovery.initial_state_timeout", "500ms")
.build();
@ -264,7 +264,7 @@ public class MinimumMasterNodesIT extends ESIntegTestCase {
public void testDynamicUpdateMinimumMasterNodes() throws Exception {
Settings settings = settingsBuilder()
.put("discovery.type", "zen")
.put(ZenDiscovery.SETTING_PING_TIMEOUT, "400ms")
.put(ZenDiscovery.PING_TIMEOUT_SETTING.getKey(), "400ms")
.put("discovery.initial_state_timeout", "500ms")
.build();
@ -322,7 +322,7 @@ public class MinimumMasterNodesIT extends ESIntegTestCase {
int nodeCount = scaledRandomIntBetween(1, 5);
Settings.Builder settings = settingsBuilder()
.put("discovery.type", "zen")
.put(ZenDiscovery.SETTING_PING_TIMEOUT, "200ms")
.put(ZenDiscovery.PING_TIMEOUT_SETTING.getKey(), "200ms")
.put("discovery.initial_state_timeout", "500ms");
// set an initial value which is at least quorum to avoid split brains during initial startup
@ -361,8 +361,8 @@ public class MinimumMasterNodesIT extends ESIntegTestCase {
public void testCanNotPublishWithoutMinMastNodes() throws Exception {
Settings settings = settingsBuilder()
.put("discovery.type", "zen")
.put(FaultDetection.SETTING_PING_TIMEOUT, "1h") // disable it
.put(ZenDiscovery.SETTING_PING_TIMEOUT, "200ms")
.put(FaultDetection.PING_TIMEOUT_SETTING.getKey(), "1h") // disable it
.put(ZenDiscovery.PING_TIMEOUT_SETTING.getKey(), "200ms")
.put(ElectMasterService.DISCOVERY_ZEN_MINIMUM_MASTER_NODES_SETTING.getKey(), 2)
.put(DiscoverySettings.COMMIT_TIMEOUT_SETTING.getKey(), "100ms") // speed things up
.build();

View File

@ -65,7 +65,7 @@ public class NoMasterNodeIT extends ESIntegTestCase {
.put("discovery.type", "zen")
.put("action.auto_create_index", autoCreateIndex)
.put("discovery.zen.minimum_master_nodes", 2)
.put(ZenDiscovery.SETTING_PING_TIMEOUT, "200ms")
.put(ZenDiscovery.PING_TIMEOUT_SETTING.getKey(), "200ms")
.put("discovery.initial_state_timeout", "500ms")
.put(DiscoverySettings.NO_MASTER_BLOCK_SETTING.getKey(), "all")
.build();
@ -217,7 +217,7 @@ public class NoMasterNodeIT extends ESIntegTestCase {
.put("discovery.type", "zen")
.put("action.auto_create_index", false)
.put("discovery.zen.minimum_master_nodes", 2)
.put(ZenDiscovery.SETTING_PING_TIMEOUT, "200ms")
.put(ZenDiscovery.PING_TIMEOUT_SETTING.getKey(), "200ms")
.put("discovery.initial_state_timeout", "500ms")
.put(DiscoverySettings.NO_MASTER_BLOCK_SETTING.getKey(), "write")
.build();

View File

@ -108,7 +108,7 @@ public class AwarenessAllocationIT extends ESIntegTestCase {
.put(AwarenessAllocationDecider.CLUSTER_ROUTING_ALLOCATION_AWARENESS_FORCE_GROUP_SETTING.getKey() + "zone.values", "a,b")
.put(AwarenessAllocationDecider.CLUSTER_ROUTING_ALLOCATION_AWARENESS_ATTRIBUTE_SETTING.getKey(), "zone")
.put(ElectMasterService.DISCOVERY_ZEN_MINIMUM_MASTER_NODES_SETTING.getKey(), 3)
.put(ZenDiscovery.SETTING_JOIN_TIMEOUT, "10s")
.put(ZenDiscovery.JOIN_TIMEOUT_SETTING.getKey(), "10s")
.build();
logger.info("--> starting 4 nodes on different zones");

View File

@ -46,7 +46,7 @@ public class DiscoveryModuleTests extends ModuleTestCase {
public void testRegisterMasterElectionService() {
Settings settings = Settings.builder().put("node.local", false).
put(DiscoveryModule.ZEN_MASTER_SERVICE_TYPE_KEY, "custom").build();
put(DiscoveryModule.ZEN_MASTER_SERVICE_TYPE_SETTING.getKey(), "custom").build();
DiscoveryModule module = new DiscoveryModule(settings);
module.addElectMasterService("custom", DummyMasterElectionService.class);
assertBinding(module, ElectMasterService.class, DummyMasterElectionService.class);
@ -55,7 +55,7 @@ public class DiscoveryModuleTests extends ModuleTestCase {
public void testLoadUnregisteredMasterElectionService() {
Settings settings = Settings.builder().put("node.local", false).
put(DiscoveryModule.ZEN_MASTER_SERVICE_TYPE_KEY, "foobar").build();
put(DiscoveryModule.ZEN_MASTER_SERVICE_TYPE_SETTING.getKey(), "foobar").build();
DiscoveryModule module = new DiscoveryModule(settings);
module.addElectMasterService("custom", DummyMasterElectionService.class);
assertBindingFailure(module, "Unknown master service type [foobar]");
@ -71,7 +71,7 @@ public class DiscoveryModuleTests extends ModuleTestCase {
public void testRegisterDiscovery() {
boolean local = randomBoolean();
Settings settings = Settings.builder().put("node.local", local).
put(DiscoveryModule.DISCOVERY_TYPE_KEY, "custom").build();
put(DiscoveryModule.DISCOVERY_TYPE_SETTING.getKey(), "custom").build();
DiscoveryModule module = new DiscoveryModule(settings);
module.addDiscoveryType("custom", DummyDisco.class);
assertBinding(module, Discovery.class, DummyDisco.class);

View File

@ -164,8 +164,8 @@ public class DiscoveryWithServiceDisruptionsIT extends ESIntegTestCase {
}
final static Settings DEFAULT_SETTINGS = Settings.builder()
.put(FaultDetection.SETTING_PING_TIMEOUT, "1s") // for hitting simulated network failures quickly
.put(FaultDetection.SETTING_PING_RETRIES, "1") // for hitting simulated network failures quickly
.put(FaultDetection.PING_TIMEOUT_SETTING.getKey(), "1s") // for hitting simulated network failures quickly
.put(FaultDetection.PING_RETRIES_SETTING.getKey(), "1") // for hitting simulated network failures quickly
.put("discovery.zen.join_timeout", "10s") // still long to induce failures but to long so test won't time out
.put(DiscoverySettings.PUBLISH_TIMEOUT_SETTING.getKey(), "1s") // <-- for hitting simulated network failures quickly
.put("http.enabled", false) // just to make test quicker
@ -962,7 +962,7 @@ public class DiscoveryWithServiceDisruptionsIT extends ESIntegTestCase {
// don't wait for initial state, wat want to add the disruption while the cluster is forming..
internalCluster().startNodesAsync(3,
Settings.builder()
.put(DiscoveryService.SETTING_INITIAL_STATE_TIMEOUT, "1ms")
.put(DiscoveryService.INITIAL_STATE_TIMEOUT_SETTING.getKey(), "1ms")
.put(DiscoverySettings.PUBLISH_TIMEOUT_SETTING.getKey(), "3s")
.build()).get();

View File

@ -131,8 +131,8 @@ public class ZenFaultDetectionTests extends ESTestCase {
Settings.Builder settings = Settings.builder();
boolean shouldRetry = randomBoolean();
// make sure we don't ping again after the initial ping
settings.put(FaultDetection.SETTING_CONNECT_ON_NETWORK_DISCONNECT, shouldRetry)
.put(FaultDetection.SETTING_PING_INTERVAL, "5m");
settings.put(FaultDetection.CONNECT_ON_NETWORK_DISCONNECT_SETTING.getKey(), shouldRetry)
.put(FaultDetection.PING_INTERVAL_SETTING.getKey(), "5m");
ClusterState clusterState = ClusterState.builder(new ClusterName("test")).nodes(buildNodesForA(true)).build();
NodesFaultDetection nodesFDA = new NodesFaultDetection(settings.build(), threadPool, serviceA, clusterState.getClusterName());
nodesFDA.setLocalNode(nodeA);
@ -179,8 +179,8 @@ public class ZenFaultDetectionTests extends ESTestCase {
Settings.Builder settings = Settings.builder();
boolean shouldRetry = randomBoolean();
// make sure we don't ping
settings.put(FaultDetection.SETTING_CONNECT_ON_NETWORK_DISCONNECT, shouldRetry)
.put(FaultDetection.SETTING_PING_INTERVAL, "5m");
settings.put(FaultDetection.CONNECT_ON_NETWORK_DISCONNECT_SETTING.getKey(), shouldRetry)
.put(FaultDetection.PING_INTERVAL_SETTING.getKey(), "5m");
ClusterName clusterName = new ClusterName(randomAsciiOfLengthBetween(3, 20));
final ClusterState state = ClusterState.builder(clusterName).nodes(buildNodesForA(false)).build();
MasterFaultDetection masterFD = new MasterFaultDetection(settings.build(), threadPool, serviceA, clusterName,

View File

@ -95,8 +95,8 @@ public class ZenDiscoveryIT extends ESIntegTestCase {
public void testNoShardRelocationsOccurWhenElectedMasterNodeFails() throws Exception {
Settings defaultSettings = Settings.builder()
.put(FaultDetection.SETTING_PING_TIMEOUT, "1s")
.put(FaultDetection.SETTING_PING_RETRIES, "1")
.put(FaultDetection.PING_TIMEOUT_SETTING.getKey(), "1s")
.put(FaultDetection.PING_RETRIES_SETTING.getKey(), "1")
.put("discovery.type", "zen")
.build();
@ -142,8 +142,8 @@ public class ZenDiscoveryIT extends ESIntegTestCase {
@TestLogging(value = "action.admin.cluster.health:TRACE")
public void testNodeFailuresAreProcessedOnce() throws ExecutionException, InterruptedException, IOException {
Settings defaultSettings = Settings.builder()
.put(FaultDetection.SETTING_PING_TIMEOUT, "1s")
.put(FaultDetection.SETTING_PING_RETRIES, "1")
.put(FaultDetection.PING_TIMEOUT_SETTING.getKey(), "1s")
.put(FaultDetection.PING_RETRIES_SETTING.getKey(), "1")
.put("discovery.type", "zen")
.build();

View File

@ -53,8 +53,8 @@ public class TransportIndexFailuresIT extends ESIntegTestCase {
private static final Settings nodeSettings = Settings.settingsBuilder()
.put("discovery.type", "zen") // <-- To override the local setting if set externally
.put(FaultDetection.SETTING_PING_TIMEOUT, "1s") // <-- for hitting simulated network failures quickly
.put(FaultDetection.SETTING_PING_RETRIES, "1") // <-- for hitting simulated network failures quickly
.put(FaultDetection.PING_TIMEOUT_SETTING.getKey(), "1s") // <-- for hitting simulated network failures quickly
.put(FaultDetection.PING_RETRIES_SETTING.getKey(), "1") // <-- for hitting simulated network failures quickly
.put(DiscoverySettings.PUBLISH_TIMEOUT_SETTING.getKey(), "1s") // <-- for hitting simulated network failures quickly
.put("discovery.zen.minimum_master_nodes", 1)
.build();

View File

@ -171,7 +171,7 @@ public class RareClusterStateIT extends ESIntegTestCase {
@AwaitsFix(bugUrl = "https://github.com/elastic/elasticsearch/issues/14932")
public void testDeleteCreateInOneBulk() throws Exception {
internalCluster().startNodesAsync(2, Settings.builder()
.put(DiscoveryModule.DISCOVERY_TYPE_KEY, "zen")
.put(DiscoveryModule.DISCOVERY_TYPE_SETTING.getKey(), "zen")
.build()).get();
assertFalse(client().admin().cluster().prepareHealth().setWaitForNodes("2").get().isTimedOut());
prepareCreate("test").setSettings(IndexMetaData.SETTING_AUTO_EXPAND_REPLICAS, true).addMapping("type").get();

View File

@ -57,7 +57,7 @@ public class FullRollingRestartIT extends ESIntegTestCase {
}
public void testFullRollingRestart() throws Exception {
Settings settings = Settings.builder().put(ZenDiscovery.SETTING_JOIN_TIMEOUT, "30s").build();
Settings settings = Settings.builder().put(ZenDiscovery.JOIN_TIMEOUT_SETTING.getKey(), "30s").build();
internalCluster().startNode(settings);
createIndex("test");

View File

@ -100,7 +100,7 @@ public class DedicatedClusterSnapshotRestoreIT extends AbstractSnapshotIntegTest
logger.info("--> start 2 nodes");
Settings nodeSettings = settingsBuilder()
.put("discovery.type", "zen")
.put(ZenDiscovery.SETTING_PING_TIMEOUT, "200ms")
.put(ZenDiscovery.PING_TIMEOUT_SETTING.getKey(), "200ms")
.put("discovery.initial_state_timeout", "500ms")
.build();
internalCluster().startNode(nodeSettings);

View File

@ -135,8 +135,8 @@ public class TribeIT extends ESIntegTestCase {
tribe2Defaults.put("tribe.t2." + entry.getKey(), entry.getValue());
}
// give each tribe it's unicast hosts to connect to
tribe1Defaults.putArray("tribe.t1." + UnicastZenPing.DISCOVERY_ZEN_PING_UNICAST_HOSTS, getUnicastHosts(internalCluster().client()));
tribe1Defaults.putArray("tribe.t2." + UnicastZenPing.DISCOVERY_ZEN_PING_UNICAST_HOSTS, getUnicastHosts(cluster2.client()));
tribe1Defaults.putArray("tribe.t1." + UnicastZenPing.DISCOVERY_ZEN_PING_UNICAST_HOSTS_SETTING.getKey(), getUnicastHosts(internalCluster().client()));
tribe1Defaults.putArray("tribe.t2." + UnicastZenPing.DISCOVERY_ZEN_PING_UNICAST_HOSTS_SETTING.getKey(), getUnicastHosts(cluster2.client()));
Settings merged = Settings.builder()
.put("tribe.t1.cluster.name", internalCluster().getClusterName())

View File

@ -30,6 +30,7 @@ import org.elasticsearch.common.inject.Inject;
import org.elasticsearch.common.logging.ESLogger;
import org.elasticsearch.common.logging.Loggers;
import org.elasticsearch.common.settings.Settings;
import org.elasticsearch.discovery.DiscoveryModule;
import org.elasticsearch.discovery.azure.AzureDiscovery;
/**
@ -73,7 +74,7 @@ public class AzureDiscoveryModule extends AbstractModule {
*/
public static boolean isDiscoveryReady(Settings settings, ESLogger logger) {
// User set discovery.type: azure
if (!AzureDiscovery.AZURE.equalsIgnoreCase(settings.get("discovery.type"))) {
if (!AzureDiscovery.AZURE.equalsIgnoreCase(DiscoveryModule.DISCOVERY_TYPE_SETTING.get(settings))) {
logger.trace("discovery.type not set to {}", AzureDiscovery.AZURE);
return false;
}

View File

@ -20,6 +20,11 @@
package org.elasticsearch.cloud.azure.management;
import com.microsoft.windowsazure.management.compute.models.HostedServiceGetDetailedResponse;
import org.elasticsearch.common.settings.Setting;
import org.elasticsearch.common.unit.TimeValue;
import org.elasticsearch.discovery.azure.AzureUnicastHostsProvider;
import java.util.Locale;
/**
*
@ -39,9 +44,11 @@ public interface AzureComputeService {
}
static public final class Discovery {
public static final String REFRESH = "discovery.azure.refresh_interval";
public static final Setting<TimeValue> REFRESH_SETTING = Setting.positiveTimeSetting("discovery.azure.refresh_interval", TimeValue.timeValueSeconds(0), false, Setting.Scope.CLUSTER);
public static final Setting<AzureUnicastHostsProvider.HostType> HOST_TYPE_SETTING = new Setting<>("discovery.azure.host.type",
AzureUnicastHostsProvider.HostType.PRIVATE_IP.name(), AzureUnicastHostsProvider.HostType::fromString, false, Setting.Scope.CLUSTER);
public static final String HOST_TYPE = "discovery.azure.host.type";
public static final String ENDPOINT_NAME = "discovery.azure.endpoint.name";
public static final String DEPLOYMENT_NAME = "discovery.azure.deployment.name";
public static final String DEPLOYMENT_SLOT = "discovery.azure.deployment.slot";

View File

@ -45,7 +45,6 @@ import java.net.InetAddress;
import java.net.InetSocketAddress;
import java.util.ArrayList;
import java.util.List;
import java.util.Locale;
/**
*
@ -68,7 +67,7 @@ public class AzureUnicastHostsProvider extends AbstractComponent implements Unic
return hostType;
}
}
return null;
throw new IllegalArgumentException("invalid value for host type [" + type + "]");
}
}
@ -118,16 +117,9 @@ public class AzureUnicastHostsProvider extends AbstractComponent implements Unic
this.networkService = networkService;
this.version = version;
this.refreshInterval = settings.getAsTime(Discovery.REFRESH, TimeValue.timeValueSeconds(0));
this.refreshInterval = Discovery.REFRESH_SETTING.get(settings);
String strHostType = settings.get(Discovery.HOST_TYPE, HostType.PRIVATE_IP.name()).toUpperCase(Locale.ROOT);
HostType tmpHostType = HostType.fromString(strHostType);
if (tmpHostType == null) {
logger.warn("wrong value for [{}]: [{}]. falling back to [{}]...", Discovery.HOST_TYPE,
strHostType, HostType.PRIVATE_IP.name().toLowerCase(Locale.ROOT));
tmpHostType = HostType.PRIVATE_IP;
}
this.hostType = tmpHostType;
this.hostType = Discovery.HOST_TYPE_SETTING.get(settings);
this.publicEndpointName = settings.get(Discovery.ENDPOINT_NAME, "elasticsearch");
// Deployment name could be set with discovery.azure.deployment.name

View File

@ -20,10 +20,12 @@
package org.elasticsearch.plugin.discovery.azure;
import org.elasticsearch.cloud.azure.AzureDiscoveryModule;
import org.elasticsearch.cloud.azure.management.AzureComputeService;
import org.elasticsearch.common.inject.Module;
import org.elasticsearch.common.logging.ESLogger;
import org.elasticsearch.common.logging.Loggers;
import org.elasticsearch.common.settings.Settings;
import org.elasticsearch.common.settings.SettingsModule;
import org.elasticsearch.discovery.DiscoveryModule;
import org.elasticsearch.discovery.azure.AzureDiscovery;
import org.elasticsearch.discovery.azure.AzureUnicastHostsProvider;
@ -66,4 +68,9 @@ public class AzureDiscoveryPlugin extends Plugin {
discoveryModule.addUnicastHostProvider(AzureUnicastHostsProvider.class);
}
}
public void onModule(SettingsModule settingsModule) {
settingsModule.registerSetting(AzureComputeService.Discovery.REFRESH_SETTING);
settingsModule.registerSetting(AzureComputeService.Discovery.HOST_TYPE_SETTING);
}
}

View File

@ -48,7 +48,7 @@ public abstract class AbstractAzureComputeServiceTestCase extends ESIntegTestCas
// We add a fake subscription_id to start mock compute service
builder.put(Management.SUBSCRIPTION_ID, "fake")
.put(Discovery.REFRESH, "5s")
.put(Discovery.REFRESH_SETTING.getKey(), "5s")
.put(Management.KEYSTORE_PATH, "dummy")
.put(Management.KEYSTORE_PASSWORD, "dummy")
.put(Management.SERVICE_NAME, "dummy");

View File

@ -53,8 +53,8 @@ public class AzureMinimumMasterNodesTests extends AbstractAzureComputeServiceTes
.put(super.nodeSettings(nodeOrdinal))
.put("discovery.zen.minimum_master_nodes", 2)
// Make the test run faster
.put(ZenDiscovery.SETTING_JOIN_TIMEOUT, "50ms")
.put(ZenDiscovery.SETTING_PING_TIMEOUT, "10ms")
.put(ZenDiscovery.JOIN_TIMEOUT_SETTING.getKey(), "50ms")
.put(ZenDiscovery.PING_TIMEOUT_SETTING.getKey(), "10ms")
.put("discovery.initial_state_timeout", "100ms");
return builder.build();
}

View File

@ -26,6 +26,7 @@ import org.elasticsearch.cloud.azure.management.AzureComputeService.Management;
import org.elasticsearch.common.settings.Settings;
import org.elasticsearch.test.ESIntegTestCase;
import static org.hamcrest.Matchers.containsString;
import static org.hamcrest.Matchers.notNullValue;
@ESIntegTestCase.ClusterScope(scope = ESIntegTestCase.Scope.TEST,
@ -40,7 +41,7 @@ public class AzureSimpleTests extends AbstractAzureComputeServiceTestCase {
public void testOneNodeDhouldRunUsingPrivateIp() {
Settings.Builder settings = Settings.settingsBuilder()
.put(Management.SERVICE_NAME, "dummy")
.put(Discovery.HOST_TYPE, "private_ip");
.put(Discovery.HOST_TYPE_SETTING.getKey(), "private_ip");
logger.info("--> start one node");
internalCluster().startNode(settings);
@ -53,7 +54,7 @@ public class AzureSimpleTests extends AbstractAzureComputeServiceTestCase {
public void testOneNodeShouldRunUsingPublicIp() {
Settings.Builder settings = Settings.settingsBuilder()
.put(Management.SERVICE_NAME, "dummy")
.put(Discovery.HOST_TYPE, "public_ip");
.put(Discovery.HOST_TYPE_SETTING.getKey(), "public_ip");
logger.info("--> start one node");
internalCluster().startNode(settings);
@ -66,13 +67,14 @@ public class AzureSimpleTests extends AbstractAzureComputeServiceTestCase {
public void testOneNodeShouldRunUsingWrongSettings() {
Settings.Builder settings = Settings.settingsBuilder()
.put(Management.SERVICE_NAME, "dummy")
.put(Discovery.HOST_TYPE, "do_not_exist");
.put(Discovery.HOST_TYPE_SETTING.getKey(), "do_not_exist");
logger.info("--> start one node");
internalCluster().startNode(settings);
assertThat(client().admin().cluster().prepareState().setMasterNodeTimeout("1s").execute().actionGet().getState().nodes().masterNodeId(), notNullValue());
// We expect having 1 node as part of the cluster, let's test that
checkNumberOfNodes(1);
try {
internalCluster().startNode(settings);
fail("Expected IllegalArgumentException on startup");
} catch (IllegalArgumentException e) {
assertThat(e.getMessage(), containsString("invalid value for host type [do_not_exist]"));
}
}
}

View File

@ -42,7 +42,7 @@ public class AzureTwoStartedNodesTests extends AbstractAzureComputeServiceTestCa
public void testTwoNodesShouldRunUsingPrivateIp() {
Settings.Builder settings = Settings.settingsBuilder()
.put(Management.SERVICE_NAME, "dummy")
.put(Discovery.HOST_TYPE, "private_ip");
.put(Discovery.HOST_TYPE_SETTING.getKey(), "private_ip");
logger.info("--> start first node");
internalCluster().startNode(settings);
@ -60,7 +60,7 @@ public class AzureTwoStartedNodesTests extends AbstractAzureComputeServiceTestCa
public void testTwoNodesShouldRunUsingPublicIp() {
Settings.Builder settings = Settings.settingsBuilder()
.put(Management.SERVICE_NAME, "dummy")
.put(Discovery.HOST_TYPE, "public_ip");
.put(Discovery.HOST_TYPE_SETTING.getKey(), "public_ip");
logger.info("--> start first node");
internalCluster().startNode(settings);

View File

@ -22,6 +22,7 @@ package org.elasticsearch.cloud.aws;
import org.elasticsearch.common.inject.AbstractModule;
import org.elasticsearch.common.logging.ESLogger;
import org.elasticsearch.common.settings.Settings;
import org.elasticsearch.discovery.DiscoveryModule;
import org.elasticsearch.discovery.ec2.Ec2Discovery;
public class Ec2Module extends AbstractModule {
@ -37,7 +38,7 @@ public class Ec2Module extends AbstractModule {
*/
public static boolean isEc2DiscoveryActive(Settings settings, ESLogger logger) {
// User set discovery.type: ec2
if (!Ec2Discovery.EC2.equalsIgnoreCase(settings.get("discovery.type"))) {
if (!Ec2Discovery.EC2.equalsIgnoreCase(DiscoveryModule.DISCOVERY_TYPE_SETTING.get(settings))) {
logger.trace("discovery.type not set to {}", Ec2Discovery.EC2);
return false;
}

View File

@ -115,7 +115,7 @@ public class GceDiscoveryPlugin extends Plugin {
*/
public static boolean isDiscoveryAlive(Settings settings, ESLogger logger) {
// User set discovery.type: gce
if (GceDiscovery.GCE.equalsIgnoreCase(settings.get("discovery.type")) == false) {
if (GceDiscovery.GCE.equalsIgnoreCase(DiscoveryModule.DISCOVERY_TYPE_SETTING.get(settings)) == false) {
logger.debug("discovery.type not set to {}", GceDiscovery.GCE);
return false;
}

View File

@ -35,6 +35,7 @@ import org.elasticsearch.cluster.ClusterState;
import org.elasticsearch.common.Strings;
import org.elasticsearch.common.settings.Settings;
import org.elasticsearch.common.unit.ByteSizeUnit;
import org.elasticsearch.discovery.DiscoveryModule;
import org.elasticsearch.repositories.RepositoryMissingException;
import org.elasticsearch.repositories.RepositoryVerificationException;
import org.elasticsearch.repositories.azure.AzureRepository.Repository;
@ -77,7 +78,7 @@ public class AzureSnapshotRestoreTests extends AbstractAzureWithThirdPartyTestCa
protected Settings nodeSettings(int nodeOrdinal) {
return Settings.builder().put(super.nodeSettings(nodeOrdinal))
// In snapshot tests, we explicitly disable cloud discovery
.put("discovery.type", "local")
.put(DiscoveryModule.DISCOVERY_TYPE_SETTING.getKey(), "local")
.build();
}

View File

@ -64,14 +64,14 @@ public class TribeUnitTests extends ESTestCase {
.put(baseSettings)
.put("cluster.name", "tribe1")
.put("name", "tribe1_node")
.put(DiscoveryService.SETTING_DISCOVERY_SEED, random().nextLong())
.put(DiscoveryService.DISCOVERY_SEED_SETTING.getKey(), random().nextLong())
.build()).start();
tribe2 = new TribeClientNode(
Settings.builder()
.put(baseSettings)
.put("cluster.name", "tribe2")
.put("name", "tribe2_node")
.put(DiscoveryService.SETTING_DISCOVERY_SEED, random().nextLong())
.put(DiscoveryService.DISCOVERY_SEED_SETTING.getKey(), random().nextLong())
.build()).start();
}

View File

@ -53,7 +53,7 @@ final class ExternalNode implements Closeable {
public static final Settings REQUIRED_SETTINGS = Settings.builder()
.put(InternalSettingsPreparer.IGNORE_SYSTEM_PROPERTIES_SETTING, true)
.put(DiscoveryModule.DISCOVERY_TYPE_KEY, "zen")
.put(DiscoveryModule.DISCOVERY_TYPE_SETTING.getKey(), "zen")
.put("node.mode", "network").build(); // we need network mode for this
private final Path path;
@ -112,7 +112,7 @@ final class ExternalNode implements Closeable {
case "node.mode":
case "node.local":
case NetworkModule.TRANSPORT_TYPE_KEY:
case DiscoveryModule.DISCOVERY_TYPE_KEY:
case "discovery.type":
case NetworkModule.TRANSPORT_SERVICE_TYPE_KEY:
case InternalSettingsPreparer.IGNORE_SYSTEM_PROPERTIES_SETTING:
continue;

View File

@ -598,7 +598,7 @@ public final class InternalTestCluster extends TestCluster {
.put(Environment.PATH_HOME_SETTING.getKey(), baseDir) // allow overriding path.home
.put(settings)
.put("name", name)
.put(DiscoveryService.SETTING_DISCOVERY_SEED, seed)
.put(DiscoveryService.DISCOVERY_SEED_SETTING.getKey(), seed)
.build();
MockNode node = new MockNode(finalSettings, version, plugins);
return new NodeAndClient(name, node);
@ -679,7 +679,7 @@ public final class InternalTestCluster extends TestCluster {
Builder builder = settingsBuilder().put(settings).put("node.client", true);
if (size() == 0) {
// if we are the first node - don't wait for a state
builder.put("discovery.initial_state_timeout", 0);
builder.put(DiscoveryService.INITIAL_STATE_TIMEOUT_SETTING.getKey(), 0);
}
String name = startNode(builder);
return nodes.get(name).nodeClient();
@ -845,8 +845,8 @@ public final class InternalTestCluster extends TestCluster {
IOUtils.rm(nodeEnv.nodeDataPaths());
}
}
final long newIdSeed = node.settings().getAsLong(DiscoveryService.SETTING_DISCOVERY_SEED, 0l) + 1; // use a new seed to make sure we have new node id
Settings finalSettings = Settings.builder().put(node.settings()).put(newSettings).put(DiscoveryService.SETTING_DISCOVERY_SEED, newIdSeed).build();
final long newIdSeed = DiscoveryService.DISCOVERY_SEED_SETTING.get(node.settings()) + 1; // use a new seed to make sure we have new node id
Settings finalSettings = Settings.builder().put(node.settings()).put(newSettings).put(DiscoveryService.DISCOVERY_SEED_SETTING.getKey(), newIdSeed).build();
Collection<Class<? extends Plugin>> plugins = node.getPlugins();
Version version = node.getVersion();
node = new MockNode(finalSettings, version, plugins);

View File

@ -24,6 +24,7 @@ import org.elasticsearch.common.SuppressForbidden;
import org.elasticsearch.common.network.NetworkUtils;
import org.elasticsearch.common.settings.Settings;
import org.elasticsearch.common.util.CollectionUtils;
import org.elasticsearch.discovery.DiscoveryModule;
import org.elasticsearch.test.InternalTestCluster;
import org.elasticsearch.test.NodeConfigurationSource;
@ -35,7 +36,7 @@ import java.util.Set;
public class ClusterDiscoveryConfiguration extends NodeConfigurationSource {
static Settings DEFAULT_NODE_SETTINGS = Settings.settingsBuilder().put("discovery.type", "zen").build();
static Settings DEFAULT_NODE_SETTINGS = Settings.settingsBuilder().put(DiscoveryModule.DISCOVERY_TYPE_SETTING.getKey(), "zen").build();
private static final String IP_ADDR = "127.0.0.1";
final int numOfNodes;