From bd5693f6ca2f3945838810ac05999c91e1c2796f Mon Sep 17 00:00:00 2001 From: Boaz Leskes Date: Sat, 16 Apr 2016 17:26:54 +0200 Subject: [PATCH 01/34] persistency works (node join problems TBD) --- .../cluster/node/DiscoveryNodeService.java | 21 +--- .../common/settings/ClusterSettings.java | 3 +- .../elasticsearch/env/NodeEnvironment.java | 79 ++++++++++-- .../org/elasticsearch/env/NodeMetaData.java | 115 ++++++++++++++++++ .../gateway/MetaDataStateFormat.java | 15 ++- .../gateway/MetaStateService.java | 5 +- .../elasticsearch/index/shard/IndexShard.java | 3 +- .../java/org/elasticsearch/node/Node.java | 5 +- ...rdFailedClusterStateTaskExecutorTests.java | 6 +- .../node/DiscoveryNodeServiceTests.java | 7 +- .../common/util/IndexFolderUpgraderTests.java | 6 +- .../PublishClusterStateActionTests.java | 10 +- .../gateway/GatewayIndexStateIT.java | 6 +- .../gateway/MetaDataStateFormatTests.java | 10 +- .../index/shard/IndexShardTests.java | 2 +- .../index/shard/ShardPathTests.java | 8 +- .../elasticsearch/tribe/TribeUnitTests.java | 6 +- .../test/InternalTestCluster.java | 8 +- 18 files changed, 238 insertions(+), 77 deletions(-) create mode 100644 core/src/main/java/org/elasticsearch/env/NodeMetaData.java diff --git a/core/src/main/java/org/elasticsearch/cluster/node/DiscoveryNodeService.java b/core/src/main/java/org/elasticsearch/cluster/node/DiscoveryNodeService.java index 177c67f298613..759139f11c567 100644 --- a/core/src/main/java/org/elasticsearch/cluster/node/DiscoveryNodeService.java +++ b/core/src/main/java/org/elasticsearch/cluster/node/DiscoveryNodeService.java @@ -20,12 +20,8 @@ package org.elasticsearch.cluster.node; import org.elasticsearch.Version; -import org.elasticsearch.common.Randomness; -import org.elasticsearch.common.UUIDs; import org.elasticsearch.common.component.AbstractComponent; import org.elasticsearch.common.inject.Inject; -import org.elasticsearch.common.settings.Setting; -import org.elasticsearch.common.settings.Setting.Property; import org.elasticsearch.common.settings.Settings; import org.elasticsearch.common.transport.TransportAddress; import org.elasticsearch.node.Node; @@ -34,17 +30,14 @@ import java.util.HashSet; import java.util.List; import java.util.Map; -import java.util.Random; import java.util.Set; import java.util.concurrent.CopyOnWriteArrayList; +import java.util.function.Supplier; /** */ public class DiscoveryNodeService extends AbstractComponent { - public static final Setting NODE_ID_SEED_SETTING = - // don't use node.id.seed so it won't be seen as an attribute - Setting.longSetting("node_id.seed", 0L, Long.MIN_VALUE, Property.NodeScope); private final List customAttributesProviders = new CopyOnWriteArrayList<>(); private final Version version; @@ -54,18 +47,12 @@ public DiscoveryNodeService(Settings settings, Version version) { this.version = version; } - public static String generateNodeId(Settings settings) { - Random random = Randomness.get(settings, NODE_ID_SEED_SETTING); - return UUIDs.randomBase64UUID(random); - } - public DiscoveryNodeService addCustomAttributeProvider(CustomAttributesProvider customAttributesProvider) { customAttributesProviders.add(customAttributesProvider); return this; } - public DiscoveryNode buildLocalNode(TransportAddress publishAddress) { - final String nodeId = generateNodeId(settings); + public DiscoveryNode buildLocalNode(TransportAddress publishAddress, Supplier nodeIdSupplier) { Map attributes = new HashMap<>(Node.NODE_ATTRIBUTES.get(this.settings).getAsMap()); Set roles = new HashSet<>(); if (Node.NODE_INGEST_SETTING.get(settings)) { @@ -92,8 +79,8 @@ public DiscoveryNode buildLocalNode(TransportAddress publishAddress) { logger.warn("failed to build custom attributes from provider [{}]", e, provider); } } - return new DiscoveryNode(Node.NODE_NAME_SETTING.get(settings), nodeId, publishAddress, attributes, - roles, version); + return new DiscoveryNode(Node.NODE_NAME_SETTING.get(settings), nodeIdSupplier.get(), publishAddress, + attributes, roles, version); } public interface CustomAttributesProvider { diff --git a/core/src/main/java/org/elasticsearch/common/settings/ClusterSettings.java b/core/src/main/java/org/elasticsearch/common/settings/ClusterSettings.java index dc6e7c4dda191..ab3adad2f0caf 100644 --- a/core/src/main/java/org/elasticsearch/common/settings/ClusterSettings.java +++ b/core/src/main/java/org/elasticsearch/common/settings/ClusterSettings.java @@ -33,7 +33,6 @@ import org.elasticsearch.cluster.NodeConnectionsService; import org.elasticsearch.cluster.action.index.MappingUpdatedAction; import org.elasticsearch.cluster.metadata.MetaData; -import org.elasticsearch.cluster.node.DiscoveryNodeService; import org.elasticsearch.cluster.routing.allocation.allocator.BalancedShardsAllocator; import org.elasticsearch.cluster.routing.allocation.decider.AwarenessAllocationDecider; import org.elasticsearch.cluster.routing.allocation.decider.ClusterRebalanceAllocationDecider; @@ -335,7 +334,7 @@ public void apply(Settings value, Settings current, Settings previous) { Environment.PATH_SCRIPTS_SETTING, Environment.PATH_SHARED_DATA_SETTING, Environment.PIDFILE_SETTING, - DiscoveryNodeService.NODE_ID_SEED_SETTING, + NodeEnvironment.NODE_ID_SEED_SETTING, DiscoverySettings.INITIAL_STATE_TIMEOUT_SETTING, DiscoveryModule.DISCOVERY_TYPE_SETTING, DiscoveryModule.ZEN_MASTER_SERVICE_TYPE_SETTING, diff --git a/core/src/main/java/org/elasticsearch/env/NodeEnvironment.java b/core/src/main/java/org/elasticsearch/env/NodeEnvironment.java index c6eec09b1c891..a46903bb3015f 100644 --- a/core/src/main/java/org/elasticsearch/env/NodeEnvironment.java +++ b/core/src/main/java/org/elasticsearch/env/NodeEnvironment.java @@ -31,10 +31,13 @@ import org.elasticsearch.ElasticsearchException; import org.elasticsearch.cluster.metadata.IndexMetaData; import org.elasticsearch.cluster.node.DiscoveryNode; +import org.elasticsearch.common.Randomness; +import org.elasticsearch.common.Strings; import org.elasticsearch.common.SuppressForbidden; import org.elasticsearch.common.component.AbstractComponent; import org.elasticsearch.common.inject.Inject; import org.elasticsearch.common.io.FileSystemUtils; +import org.elasticsearch.common.logging.ESLogger; import org.elasticsearch.common.settings.Setting; import org.elasticsearch.common.settings.Setting.Property; import org.elasticsearch.common.settings.Settings; @@ -64,12 +67,13 @@ import java.util.HashMap; import java.util.HashSet; import java.util.List; -import java.util.Locale; import java.util.Map; +import java.util.Random; import java.util.Set; import java.util.concurrent.Semaphore; import java.util.concurrent.TimeUnit; import java.util.concurrent.atomic.AtomicBoolean; +import java.util.stream.Collectors; import static java.util.Collections.unmodifiableSet; @@ -77,6 +81,7 @@ * A component that holds all data paths for a single node. */ public final class NodeEnvironment extends AbstractComponent implements Closeable { + public static class NodePath { /* ${data.paths}/nodes/{node.id} */ public final Path path; @@ -130,10 +135,12 @@ public String toString() { private final boolean addNodeId; - private final int localNodeId; + private final int nodeLockId; private final AtomicBoolean closed = new AtomicBoolean(false); private final Map shardLocks = new HashMap<>(); + private final NodeMetaData nodeMetaData; + /** * Maximum number of data nodes that should run in an environment. */ @@ -143,8 +150,19 @@ public String toString() { /** * If true automatically append node id to custom data paths. */ + @Deprecated public static final Setting ADD_NODE_ID_TO_CUSTOM_PATH = - Setting.boolSetting("node.add_id_to_custom_path", true, Property.NodeScope); + Setting.boolSetting("node.add_id_to_custom_path", true, Property.Deprecated, Property.NodeScope); + + public static final Setting ADD_NODE_LOCK_ID_TO_CUSTOM_PATH = + Setting.boolSetting("node.add_lock_id_to_custom_path", ADD_NODE_ID_TO_CUSTOM_PATH, + Property.NodeScope); + + + public static final Setting NODE_ID_SEED_SETTING = + // don't use node.id.seed so it won't be seen as an attribute + Setting.longSetting("node_id.seed", 0L, Long.MIN_VALUE, Property.NodeScope); + /** * If true the [verbose] SegmentInfos.infoStream logging is sent to System.out. @@ -166,7 +184,9 @@ public NodeEnvironment(Settings settings, Environment environment) throws IOExce nodePaths = null; sharedDataPath = null; locks = null; - localNodeId = -1; + nodeLockId = -1; + // nocommit - this a big shame for coordinating nodes.. + nodeMetaData = new NodeMetaData(generateNodeId(settings)); return; } final NodePath[] nodePaths = new NodePath[environment.dataWithClusterFiles().length]; @@ -175,7 +195,7 @@ public NodeEnvironment(Settings settings, Environment environment) throws IOExce try { sharedDataPath = environment.sharedDataFile(); - int localNodeId = -1; + int nodeLockId = -1; IOException lastException = null; int maxLocalStorageNodes = MAX_LOCAL_STORAGE_NODES_SETTING.get(settings); for (int possibleLockId = 0; possibleLockId < maxLocalStorageNodes; possibleLockId++) { @@ -188,7 +208,7 @@ public NodeEnvironment(Settings settings, Environment environment) throws IOExce try { locks[dirIndex] = luceneDir.obtainLock(NODE_LOCK_FILENAME); nodePaths[dirIndex] = new NodePath(dir); - localNodeId = possibleLockId; + nodeLockId = possibleLockId; } catch (LockObtainFailedException ex) { logger.trace("failed to obtain node lock on {}", dir.toAbsolutePath()); // release all the ones that were obtained up until now @@ -215,17 +235,19 @@ public NodeEnvironment(Settings settings, Environment environment) throws IOExce + Arrays.toString(environment.dataWithClusterFiles()), lastException); } - this.localNodeId = localNodeId; + this.nodeLockId = nodeLockId; this.locks = locks; this.nodePaths = nodePaths; if (logger.isDebugEnabled()) { - logger.debug("using node location [{}], local_node_id [{}]", nodePaths, localNodeId); + logger.debug("using node location [{}], local_lock_id [{}]", nodePaths, nodeLockId); } maybeLogPathDetails(); maybeLogHeapDetails(); + this.nodeMetaData = loadOrCreateNodeMetaData(settings, logger, nodePaths); + applySegmentInfosTrace(settings); assertCanWrite(); success = true; @@ -236,6 +258,30 @@ public NodeEnvironment(Settings settings, Environment environment) throws IOExce } } + /** + * scans the node paths and loads existing metaData file. If not found a new meta data will be generated + * and persisted into the nodePaths + * + */ + // package private for testing + static NodeMetaData loadOrCreateNodeMetaData(Settings settings, ESLogger logger, + NodePath... nodePaths) throws IOException { + List pathList = Arrays.stream(nodePaths).map(np -> np.path).collect(Collectors.toList()); + final Path[] paths = pathList.toArray(new Path[pathList.size()]); + NodeMetaData metaData = NodeMetaData.FORMAT.loadLatestState(logger, paths); + if (metaData == null) { + metaData = new NodeMetaData(generateNodeId(settings)); + } + // we write again to make sure all paths have the latest state file + NodeMetaData.FORMAT.write(metaData, paths); + return metaData; + } + + public static String generateNodeId(Settings settings) { + Random random = Randomness.get(settings, NODE_ID_SEED_SETTING); + return Strings.randomBase64UUID(random); + } + private static void releaseAndNullLocks(Lock[] locks) { for (int i = 0; i < locks.length; i++) { if (locks[i] != null) { @@ -616,10 +662,6 @@ void acquire(long timeoutInMillis) throws LockObtainFailedException{ } } - public int localNodeId() { - return this.localNodeId; - } - public boolean hasNodeFile() { return nodePaths != null && locks != null; } @@ -640,6 +682,17 @@ public Path[] nodeDataPaths() { return paths; } + /** + * returns the unique uuid describing this node. The uuid is persistent in the data folder of this node + * and remains across restarts. + **/ + public String nodeID() { + // we currently only return the ID and hide the underlying nodeMetaData implementation in order to avoid + // confusion with other "metadata" like node settings found in elasticsearch.yml. In future + // we can encapsulate both (and more) in one NodeMetaData (or NodeSettings) object ala IndexSettings + return nodeMetaData.nodeID(); + } + /** * Returns an array of all of the {@link NodePath}s. */ @@ -845,7 +898,7 @@ public Path resolveBaseCustomLocation(IndexSettings indexSettings) { // This assert is because this should be caught by MetaDataCreateIndexService assert sharedDataPath != null; if (addNodeId) { - return sharedDataPath.resolve(customDataDir).resolve(Integer.toString(this.localNodeId)); + return sharedDataPath.resolve(customDataDir).resolve(Integer.toString(this.nodeLockId)); } else { return sharedDataPath.resolve(customDataDir); } diff --git a/core/src/main/java/org/elasticsearch/env/NodeMetaData.java b/core/src/main/java/org/elasticsearch/env/NodeMetaData.java new file mode 100644 index 0000000000000..045c6b6b85b9f --- /dev/null +++ b/core/src/main/java/org/elasticsearch/env/NodeMetaData.java @@ -0,0 +1,115 @@ +/* + * Licensed to Elasticsearch under one or more contributor + * license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright + * ownership. Elasticsearch licenses this file to you under + * the Apache License, Version 2.0 (the "License"); you may + * not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package org.elasticsearch.env; + +import org.elasticsearch.common.ParseField; +import org.elasticsearch.common.ParseFieldMatcher; +import org.elasticsearch.common.ParseFieldMatcherSupplier; +import org.elasticsearch.common.xcontent.ObjectParser; +import org.elasticsearch.common.xcontent.XContentBuilder; +import org.elasticsearch.common.xcontent.XContentParser; +import org.elasticsearch.common.xcontent.XContentType; +import org.elasticsearch.gateway.MetaDataStateFormat; + +import java.io.IOException; +import java.io.OutputStream; +import java.util.Objects; + +/** + */ +public final class NodeMetaData { + + private static final String NODE_ID_KEY = "allocation_id"; + + private final String nodeID; + + public NodeMetaData(final String nodeID) { + Objects.requireNonNull(nodeID); + this.nodeID = nodeID; + } + + @Override + public boolean equals(Object o) { + if (this == o) { + return true; + } + if (o == null || getClass() != o.getClass()) { + return false; + } + + NodeMetaData that = (NodeMetaData) o; + + return Objects.equals(this.nodeID, that.nodeID); + } + + @Override + public int hashCode() { + return this.nodeID.hashCode(); + } + + @Override + public String toString() { + return "node_id [" + nodeID + "]"; + } + + private static ObjectParser PARSER = new ObjectParser<>("node_meta_data", + Builder::new); + + static { + PARSER.declareString(Builder::setNodeID, new ParseField(NODE_ID_KEY)); + } + + public String nodeID() { + return nodeID; + } + + private static class Builder { + String nodeID; + + public void setNodeID(String nodeID) { + this.nodeID = nodeID; + } + + public NodeMetaData build() { + return new NodeMetaData(nodeID); + } + } + + + public static final MetaDataStateFormat FORMAT = new MetaDataStateFormat(XContentType.SMILE, "node-") { + + @Override + protected XContentBuilder newXContentBuilder(XContentType type, OutputStream stream) throws IOException { + XContentBuilder xContentBuilder = super.newXContentBuilder(type, stream); + xContentBuilder.prettyPrint(); + return xContentBuilder; + } + + @Override + public void toXContent(XContentBuilder builder, NodeMetaData nodeMetaData) throws IOException { + builder.field(NODE_ID_KEY, nodeMetaData.nodeID); + } + + @Override + public NodeMetaData fromXContent(XContentParser parser) throws IOException { + return PARSER.apply(parser, () -> ParseFieldMatcher.STRICT).build(); + } + }; +} diff --git a/core/src/main/java/org/elasticsearch/gateway/MetaDataStateFormat.java b/core/src/main/java/org/elasticsearch/gateway/MetaDataStateFormat.java index 9a78925368eda..9ecbe79ee7826 100644 --- a/core/src/main/java/org/elasticsearch/gateway/MetaDataStateFormat.java +++ b/core/src/main/java/org/elasticsearch/gateway/MetaDataStateFormat.java @@ -64,7 +64,8 @@ public abstract class MetaDataStateFormat { public static final String STATE_DIR_NAME = "_state"; public static final String STATE_FILE_EXTENSION = ".st"; private static final String STATE_FILE_CODEC = "state"; - private static final int STATE_FILE_VERSION = 0; + private static final int MIN_COMPATIBLE_STATE_FILE_VERSION = 0; + private static final int STATE_FILE_VERSION = 1; private static final int BUFFER_SIZE = 4096; private final XContentType format; private final String prefix; @@ -96,11 +97,10 @@ public XContentType format() { * it's target filename of the pattern {prefix}{version}.st. * * @param state the state object to write - * @param version the version of the state * @param locations the locations where the state should be written to. * @throws IOException if an IOException occurs */ - public final void write(final T state, final long version, final Path... locations) throws IOException { + public final void write(final T state, final Path... locations) throws IOException { if (locations == null) { throw new IllegalArgumentException("Locations must not be null"); } @@ -119,7 +119,6 @@ public final void write(final T state, final long version, final Path... locatio try (OutputStreamIndexOutput out = new OutputStreamIndexOutput(resourceDesc, fileName, Files.newOutputStream(tmpStatePath), BUFFER_SIZE)) { CodecUtil.writeHeader(out, STATE_FILE_CODEC, STATE_FILE_VERSION); out.writeInt(format.index()); - out.writeLong(version); try (XContentBuilder builder = newXContentBuilder(format, new IndexOutputOutputStream(out) { @Override public void close() throws IOException { @@ -182,9 +181,13 @@ public final T read(Path file) throws IOException { try (final IndexInput indexInput = dir.openInput(file.getFileName().toString(), IOContext.DEFAULT)) { // We checksum the entire file before we even go and parse it. If it's corrupted we barf right here. CodecUtil.checksumEntireFile(indexInput); - CodecUtil.checkHeader(indexInput, STATE_FILE_CODEC, STATE_FILE_VERSION, STATE_FILE_VERSION); + final int fileVersion = CodecUtil.checkHeader(indexInput, STATE_FILE_CODEC, STATE_FILE_VERSION, STATE_FILE_VERSION); final XContentType xContentType = XContentType.values()[indexInput.readInt()]; - indexInput.readLong(); // version currently unused + if (fileVersion == 0) { + // format version 0, write a version that always came from the content state file + // and was never used. + indexInput.readLong(); + } long filePointer = indexInput.getFilePointer(); long contentSize = indexInput.length() - CodecUtil.footerLength() - filePointer; try (IndexInput slice = indexInput.slice("state_xcontent", filePointer, contentSize)) { diff --git a/core/src/main/java/org/elasticsearch/gateway/MetaStateService.java b/core/src/main/java/org/elasticsearch/gateway/MetaStateService.java index 89192c47d0962..a9835097d2b97 100644 --- a/core/src/main/java/org/elasticsearch/gateway/MetaStateService.java +++ b/core/src/main/java/org/elasticsearch/gateway/MetaStateService.java @@ -29,7 +29,6 @@ import org.elasticsearch.index.Index; import java.io.IOException; -import java.nio.file.Path; import java.util.ArrayList; import java.util.List; import java.util.function.Predicate; @@ -124,7 +123,7 @@ void writeIndex(String reason, IndexMetaData indexMetaData) throws IOException { final Index index = indexMetaData.getIndex(); logger.trace("[{}] writing state, reason [{}]", index, reason); try { - IndexMetaData.FORMAT.write(indexMetaData, indexMetaData.getVersion(), + IndexMetaData.FORMAT.write(indexMetaData, nodeEnv.indexPaths(indexMetaData.getIndex())); } catch (Throwable ex) { logger.warn("[{}]: failed to write index state", ex, index); @@ -138,7 +137,7 @@ void writeIndex(String reason, IndexMetaData indexMetaData) throws IOException { void writeGlobalState(String reason, MetaData metaData) throws Exception { logger.trace("[_global] writing state, reason [{}]", reason); try { - MetaData.FORMAT.write(metaData, metaData.version(), nodeEnv.nodeDataPaths()); + MetaData.FORMAT.write(metaData, nodeEnv.nodeDataPaths()); } catch (Throwable ex) { logger.warn("[_global]: failed to write global state", ex); throw new IOException("failed to write global state", ex); diff --git a/core/src/main/java/org/elasticsearch/index/shard/IndexShard.java b/core/src/main/java/org/elasticsearch/index/shard/IndexShard.java index 979dfd18152f9..0e39c62ac98bf 100644 --- a/core/src/main/java/org/elasticsearch/index/shard/IndexShard.java +++ b/core/src/main/java/org/elasticsearch/index/shard/IndexShard.java @@ -84,7 +84,6 @@ import org.elasticsearch.index.mapper.SourceToParse; import org.elasticsearch.index.mapper.Uid; import org.elasticsearch.index.merge.MergeStats; -import org.elasticsearch.index.percolator.PercolatorFieldMapper; import org.elasticsearch.index.recovery.RecoveryStats; import org.elasticsearch.index.refresh.RefreshStats; import org.elasticsearch.index.search.stats.SearchStats; @@ -1386,7 +1385,7 @@ void persistMetadata(ShardRouting newRouting, @Nullable ShardRouting currentRout } logger.trace("{} writing shard state, reason [{}]", shardId, writeReason); final ShardStateMetaData newShardStateMetadata = new ShardStateMetaData(newRouting.primary(), getIndexUUID(), newRouting.allocationId()); - ShardStateMetaData.FORMAT.write(newShardStateMetadata, newShardStateMetadata.legacyVersion, shardPath().getShardStatePath()); + ShardStateMetaData.FORMAT.write(newShardStateMetadata, shardPath().getShardStatePath()); } else { logger.trace("{} skip writing shard state, has been written before", shardId); } diff --git a/core/src/main/java/org/elasticsearch/node/Node.java b/core/src/main/java/org/elasticsearch/node/Node.java index d543d116c8f4a..a24745d52b263 100644 --- a/core/src/main/java/org/elasticsearch/node/Node.java +++ b/core/src/main/java/org/elasticsearch/node/Node.java @@ -322,8 +322,9 @@ public Node start() { validateNodeBeforeAcceptingRequests(settings, transportService.boundAddress()); - DiscoveryNode localNode = injector.getInstance(DiscoveryNodeService.class) - .buildLocalNode(transportService.boundAddress().publishAddress()); + DiscoveryNode localNode = injector.getInstance(DiscoveryNodeService.class).buildLocalNode( + transportService.boundAddress().publishAddress(), + injector.getInstance(NodeEnvironment.class)::nodeID); // TODO: need to find a cleaner way to start/construct a service with some initial parameters, // playing nice with the life cycle interfaces diff --git a/core/src/test/java/org/elasticsearch/cluster/action/shard/ShardFailedClusterStateTaskExecutorTests.java b/core/src/test/java/org/elasticsearch/cluster/action/shard/ShardFailedClusterStateTaskExecutorTests.java index 317bb6cfc4726..ef85966770c09 100644 --- a/core/src/test/java/org/elasticsearch/cluster/action/shard/ShardFailedClusterStateTaskExecutorTests.java +++ b/core/src/test/java/org/elasticsearch/cluster/action/shard/ShardFailedClusterStateTaskExecutorTests.java @@ -27,7 +27,6 @@ import org.elasticsearch.cluster.ClusterStateTaskExecutor; import org.elasticsearch.cluster.metadata.IndexMetaData; import org.elasticsearch.cluster.metadata.MetaData; -import org.elasticsearch.cluster.node.DiscoveryNodeService; import org.elasticsearch.cluster.node.DiscoveryNodes; import org.elasticsearch.cluster.routing.GroupShardsIterator; import org.elasticsearch.cluster.routing.IndexShardRoutingTable; @@ -41,6 +40,7 @@ import org.elasticsearch.cluster.routing.allocation.FailedRerouteAllocation; import org.elasticsearch.cluster.routing.allocation.RoutingAllocation; import org.elasticsearch.cluster.routing.allocation.decider.ClusterRebalanceAllocationDecider; +import org.elasticsearch.common.Strings; import org.elasticsearch.common.settings.Settings; import org.elasticsearch.index.Index; import org.elasticsearch.test.ESAllocationTestCase; @@ -303,8 +303,8 @@ private static ShardRouting randomInvalidSourceShard(ClusterState currentState, if (!shards.isEmpty()) { return randomSubsetOf(1, shards.toArray(new ShardRouting[0])).get(0); } else { - return - TestShardRouting.newShardRouting(shardRouting.index(), shardRouting.id(), DiscoveryNodeService.generateNodeId(Settings.EMPTY), randomBoolean(), randomFrom(ShardRoutingState.values())); + return TestShardRouting.newShardRouting(shardRouting.index(), shardRouting.id(), + Strings.randomBase64UUID(random()), randomBoolean(), randomFrom(ShardRoutingState.values())); } } diff --git a/core/src/test/java/org/elasticsearch/cluster/node/DiscoveryNodeServiceTests.java b/core/src/test/java/org/elasticsearch/cluster/node/DiscoveryNodeServiceTests.java index c03a5ab06dc43..0e110f5fda930 100644 --- a/core/src/test/java/org/elasticsearch/cluster/node/DiscoveryNodeServiceTests.java +++ b/core/src/test/java/org/elasticsearch/cluster/node/DiscoveryNodeServiceTests.java @@ -22,6 +22,7 @@ import org.elasticsearch.Version; import org.elasticsearch.common.settings.Settings; import org.elasticsearch.common.transport.DummyTransportAddress; +import org.elasticsearch.env.NodeEnvironment; import org.elasticsearch.test.ESTestCase; import java.util.HashMap; @@ -55,7 +56,8 @@ public void testBuildLocalNode() { } } DiscoveryNodeService discoveryNodeService = new DiscoveryNodeService(builder.build(), Version.CURRENT); - DiscoveryNode discoveryNode = discoveryNodeService.buildLocalNode(DummyTransportAddress.INSTANCE); + DiscoveryNode discoveryNode = discoveryNodeService.buildLocalNode( + DummyTransportAddress.INSTANCE, () -> NodeEnvironment.generateNodeId(Settings.EMPTY)); assertThat(discoveryNode.getRoles(), equalTo(selectedRoles)); assertThat(discoveryNode.getAttributes(), equalTo(expectedAttributes)); } @@ -77,7 +79,8 @@ public void testBuildAttributesWithCustomAttributeServiceProvider() { expectedAttributes.putAll(customAttributes); discoveryNodeService.addCustomAttributeProvider(() -> customAttributes); - DiscoveryNode discoveryNode = discoveryNodeService.buildLocalNode(DummyTransportAddress.INSTANCE); + DiscoveryNode discoveryNode = discoveryNodeService.buildLocalNode( + DummyTransportAddress.INSTANCE, () -> NodeEnvironment.generateNodeId(Settings.EMPTY)); assertThat(discoveryNode.getAttributes(), equalTo(expectedAttributes)); } } diff --git a/core/src/test/java/org/elasticsearch/common/util/IndexFolderUpgraderTests.java b/core/src/test/java/org/elasticsearch/common/util/IndexFolderUpgraderTests.java index 26d6af1cd5f1f..0b53e8f6a8362 100644 --- a/core/src/test/java/org/elasticsearch/common/util/IndexFolderUpgraderTests.java +++ b/core/src/test/java/org/elasticsearch/common/util/IndexFolderUpgraderTests.java @@ -256,7 +256,7 @@ public void testNeedsUpgrade() throws IOException { .numberOfReplicas(0) .build(); try (NodeEnvironment nodeEnvironment = newNodeEnvironment()) { - IndexMetaData.FORMAT.write(indexState, 1, nodeEnvironment.indexPaths(index)); + IndexMetaData.FORMAT.write(indexState, nodeEnvironment.indexPaths(index)); assertFalse(IndexFolderUpgrader.needsUpgrade(index, index.getUUID())); } } @@ -305,7 +305,7 @@ private void writeIndex(NodeEnvironment nodeEnv, IndexSettings indexSettings, for (int i = 0; i < nodePaths.length; i++) { oldIndexPaths[i] = nodePaths[i].indicesPath.resolve(indexSettings.getIndex().getName()); } - IndexMetaData.FORMAT.write(indexSettings.getIndexMetaData(), 1, oldIndexPaths); + IndexMetaData.FORMAT.write(indexSettings.getIndexMetaData(), oldIndexPaths); for (int id = 0; id < indexSettings.getNumberOfShards(); id++) { Path oldIndexPath = randomFrom(oldIndexPaths); ShardId shardId = new ShardId(indexSettings.getIndex(), id); @@ -316,7 +316,7 @@ private void writeIndex(NodeEnvironment nodeEnv, IndexSettings indexSettings, writeShard(shardId, oldIndexPath, numIdxFiles, numTranslogFiles); } ShardStateMetaData state = new ShardStateMetaData(true, indexSettings.getUUID(), AllocationId.newInitializing()); - ShardStateMetaData.FORMAT.write(state, 1, oldIndexPath.resolve(String.valueOf(shardId.getId()))); + ShardStateMetaData.FORMAT.write(state, oldIndexPath.resolve(String.valueOf(shardId.getId()))); } } diff --git a/core/src/test/java/org/elasticsearch/discovery/zen/publish/PublishClusterStateActionTests.java b/core/src/test/java/org/elasticsearch/discovery/zen/publish/PublishClusterStateActionTests.java index 335d1b3e8fdc1..ace329ca68a76 100644 --- a/core/src/test/java/org/elasticsearch/discovery/zen/publish/PublishClusterStateActionTests.java +++ b/core/src/test/java/org/elasticsearch/discovery/zen/publish/PublishClusterStateActionTests.java @@ -43,6 +43,7 @@ import org.elasticsearch.discovery.Discovery; import org.elasticsearch.discovery.DiscoverySettings; import org.elasticsearch.discovery.zen.DiscoveryNodesProvider; +import org.elasticsearch.env.NodeEnvironment; import org.elasticsearch.node.Node; import org.elasticsearch.test.ESTestCase; import org.elasticsearch.test.junit.annotations.TestLogging; @@ -152,16 +153,17 @@ public MockNode createMockNode(final String name, Settings settings, Version ver return createMockNode(name, settings, version, null); } - public MockNode createMockNode(String name, Settings settings, Version version, @Nullable ClusterStateListener listener) throws Exception { - settings = Settings.builder() + public MockNode createMockNode(String name, Settings baseSettings, Version version, @Nullable ClusterStateListener listener) throws Exception { + final Settings settings = Settings.builder() .put("name", name) .put(TransportService.TRACE_LOG_INCLUDE_SETTING.getKey(), "", TransportService.TRACE_LOG_EXCLUDE_SETTING.getKey(), "NOTHING") - .put(settings) + .put(baseSettings) .build(); MockTransportService service = buildTransportService(settings, version); DiscoveryNodeService discoveryNodeService = new DiscoveryNodeService(settings, version); - DiscoveryNode discoveryNode = discoveryNodeService.buildLocalNode(service.boundAddress().publishAddress()); + DiscoveryNode discoveryNode = discoveryNodeService.buildLocalNode( + service.boundAddress().publishAddress(), () -> NodeEnvironment.generateNodeId(settings)); MockNode node = new MockNode(discoveryNode, service, listener, logger); node.action = buildPublishClusterStateAction(settings, service, () -> node.clusterState, node); final CountDownLatch latch = new CountDownLatch(nodes.size() * 2 + 1); diff --git a/core/src/test/java/org/elasticsearch/gateway/GatewayIndexStateIT.java b/core/src/test/java/org/elasticsearch/gateway/GatewayIndexStateIT.java index 9572f06b3e229..b204db0d9bd23 100644 --- a/core/src/test/java/org/elasticsearch/gateway/GatewayIndexStateIT.java +++ b/core/src/test/java/org/elasticsearch/gateway/GatewayIndexStateIT.java @@ -413,7 +413,7 @@ public void testRecoverBrokenIndexMetadata() throws Exception { // this one is not validated ahead of time and breaks allocation .put("index.analysis.filter.myCollator.type", "icu_collation") ).build(); - IndexMetaData.FORMAT.write(brokenMeta, brokenMeta.getVersion(), services.indexPaths(brokenMeta.getIndex())); + IndexMetaData.FORMAT.write(brokenMeta, services.indexPaths(brokenMeta.getIndex())); } internalCluster().fullRestart(); // ensureGreen(closedIndex) waits for the index to show up in the metadata @@ -477,7 +477,7 @@ public void testRecoverMissingAnalyzer() throws Exception { for (NodeEnvironment services : internalCluster().getInstances(NodeEnvironment.class)) { IndexMetaData brokenMeta = IndexMetaData.builder(metaData).settings(metaData.getSettings() .filter((s) -> "index.analysis.analyzer.test.tokenizer".equals(s) == false)).build(); - IndexMetaData.FORMAT.write(brokenMeta, brokenMeta.getVersion(), services.indexPaths(brokenMeta.getIndex())); + IndexMetaData.FORMAT.write(brokenMeta, services.indexPaths(brokenMeta.getIndex())); } internalCluster().fullRestart(); // ensureGreen(closedIndex) waits for the index to show up in the metadata @@ -522,7 +522,7 @@ public void testArchiveBrokenClusterSettings() throws Exception { MetaData brokenMeta = MetaData.builder(metaData).persistentSettings(Settings.builder() .put(metaData.persistentSettings()).put("this.is.unknown", true) .put(ElectMasterService.DISCOVERY_ZEN_MINIMUM_MASTER_NODES_SETTING.getKey(), "broken").build()).build(); - MetaData.FORMAT.write(brokenMeta, metaData.version(), nodeEnv.nodeDataPaths()); + MetaData.FORMAT.write(brokenMeta, nodeEnv.nodeDataPaths()); } internalCluster().fullRestart(); ensureYellow("test"); // wait for state recovery diff --git a/core/src/test/java/org/elasticsearch/gateway/MetaDataStateFormatTests.java b/core/src/test/java/org/elasticsearch/gateway/MetaDataStateFormatTests.java index b2c6f121a149f..792ba89768076 100644 --- a/core/src/test/java/org/elasticsearch/gateway/MetaDataStateFormatTests.java +++ b/core/src/test/java/org/elasticsearch/gateway/MetaDataStateFormatTests.java @@ -101,7 +101,7 @@ public void testReadWriteState() throws IOException { Format format = new Format(randomFrom(XContentType.values()), "foo-"); DummyState state = new DummyState(randomRealisticUnicodeOfCodepointLengthBetween(1, 1000), randomInt(), randomLong(), randomDouble(), randomBoolean()); int version = between(0, Integer.MAX_VALUE/2); - format.write(state, version, dirs); + format.write(state, dirs); for (Path file : dirs) { Path[] list = content("*", file); assertEquals(list.length, 1); @@ -116,7 +116,7 @@ public void testReadWriteState() throws IOException { } final int version2 = between(version, Integer.MAX_VALUE); DummyState state2 = new DummyState(randomRealisticUnicodeOfCodepointLengthBetween(1, 1000), randomInt(), randomLong(), randomDouble(), randomBoolean()); - format.write(state2, version2, dirs); + format.write(state2, dirs); for (Path file : dirs) { Path[] list = content("*", file); @@ -143,7 +143,7 @@ public void testVersionMismatch() throws IOException { Format format = new Format(randomFrom(XContentType.values()), "foo-"); DummyState state = new DummyState(randomRealisticUnicodeOfCodepointLengthBetween(1, 1000), randomInt(), randomLong(), randomDouble(), randomBoolean()); int version = between(0, Integer.MAX_VALUE/2); - format.write(state, version, dirs); + format.write(state, dirs); for (Path file : dirs) { Path[] list = content("*", file); assertEquals(list.length, 1); @@ -167,7 +167,7 @@ public void testCorruption() throws IOException { Format format = new Format(randomFrom(XContentType.values()), "foo-"); DummyState state = new DummyState(randomRealisticUnicodeOfCodepointLengthBetween(1, 1000), randomInt(), randomLong(), randomDouble(), randomBoolean()); int version = between(0, Integer.MAX_VALUE/2); - format.write(state, version, dirs); + format.write(state, dirs); for (Path file : dirs) { Path[] list = content("*", file); assertEquals(list.length, 1); @@ -258,7 +258,7 @@ public void testLoadState() throws IOException { } } for (int j = numLegacy; j < numStates; j++) { - format.write(meta.get(j), j, dirs[i]); + format.write(meta.get(j), dirs[i]); if (randomBoolean() && (j < numStates - 1 || dirs.length > 0 && i != 0)) { // corrupt a file that we do not necessarily need here.... Path file = dirs[i].resolve(MetaDataStateFormat.STATE_DIR_NAME).resolve("global-" + j + ".st"); corruptedFiles.add(file); diff --git a/core/src/test/java/org/elasticsearch/index/shard/IndexShardTests.java b/core/src/test/java/org/elasticsearch/index/shard/IndexShardTests.java index f21973d19db14..9bb0302e0b7e3 100644 --- a/core/src/test/java/org/elasticsearch/index/shard/IndexShardTests.java +++ b/core/src/test/java/org/elasticsearch/index/shard/IndexShardTests.java @@ -449,7 +449,7 @@ public static ShardStateMetaData load(ESLogger logger, Path... shardPaths) throw public static void write(ShardStateMetaData shardStateMetaData, Path... shardPaths) throws IOException { - ShardStateMetaData.FORMAT.write(shardStateMetaData, shardStateMetaData.legacyVersion, shardPaths); + ShardStateMetaData.FORMAT.write(shardStateMetaData, shardPaths); } public void testDurableFlagHasEffect() { diff --git a/core/src/test/java/org/elasticsearch/index/shard/ShardPathTests.java b/core/src/test/java/org/elasticsearch/index/shard/ShardPathTests.java index b2bd7e1f9ff9e..fdf8842638453 100644 --- a/core/src/test/java/org/elasticsearch/index/shard/ShardPathTests.java +++ b/core/src/test/java/org/elasticsearch/index/shard/ShardPathTests.java @@ -45,7 +45,7 @@ public void testLoadShardPath() throws IOException { ShardId shardId = new ShardId("foo", "0xDEADBEEF", 0); Path[] paths = env.availableShardPaths(shardId); Path path = randomFrom(paths); - ShardStateMetaData.FORMAT.write(new ShardStateMetaData(2, true, "0xDEADBEEF", AllocationId.newInitializing()), 2, path); + ShardStateMetaData.FORMAT.write(new ShardStateMetaData(2, true, "0xDEADBEEF", AllocationId.newInitializing()), path); ShardPath shardPath = ShardPath.loadShardPath(logger, env, shardId, IndexSettingsModule.newIndexSettings(shardId.getIndex(), settings)); assertEquals(path, shardPath.getDataPath()); assertEquals("0xDEADBEEF", shardPath.getShardId().getIndex().getUUID()); @@ -65,7 +65,7 @@ public void testFailLoadShardPathOnMultiState() throws IOException { Path[] paths = env.availableShardPaths(shardId); assumeTrue("This test tests multi data.path but we only got one", paths.length > 1); int id = randomIntBetween(1, 10); - ShardStateMetaData.FORMAT.write(new ShardStateMetaData(id, true, indexUUID, AllocationId.newInitializing()), id, paths); + ShardStateMetaData.FORMAT.write(new ShardStateMetaData(id, true, indexUUID, AllocationId.newInitializing()), paths); ShardPath.loadShardPath(logger, env, shardId, IndexSettingsModule.newIndexSettings(shardId.getIndex(), settings)); fail("Expected IllegalStateException"); } catch (IllegalStateException e) { @@ -82,7 +82,7 @@ public void testFailLoadShardPathIndexUUIDMissmatch() throws IOException { Path[] paths = env.availableShardPaths(shardId); Path path = randomFrom(paths); int id = randomIntBetween(1, 10); - ShardStateMetaData.FORMAT.write(new ShardStateMetaData(id, true, "0xDEADBEEF", AllocationId.newInitializing()), id, path); + ShardStateMetaData.FORMAT.write(new ShardStateMetaData(id, true, "0xDEADBEEF", AllocationId.newInitializing()), path); ShardPath.loadShardPath(logger, env, shardId, IndexSettingsModule.newIndexSettings(shardId.getIndex(), settings)); fail("Expected IllegalStateException"); } catch (IllegalStateException e) { @@ -139,7 +139,7 @@ public void testGetRootPaths() throws IOException { ShardId shardId = new ShardId("foo", indexUUID, 0); Path[] paths = env.availableShardPaths(shardId); Path path = randomFrom(paths); - ShardStateMetaData.FORMAT.write(new ShardStateMetaData(2, true, indexUUID, AllocationId.newInitializing()), 2, path); + ShardStateMetaData.FORMAT.write(new ShardStateMetaData(2, true, indexUUID, AllocationId.newInitializing()), path); ShardPath shardPath = ShardPath.loadShardPath(logger, env, shardId, IndexSettingsModule.newIndexSettings(shardId.getIndex(), indexSettings)); boolean found = false; for (Path p : env.nodeDataPaths()) { diff --git a/qa/evil-tests/src/test/java/org/elasticsearch/tribe/TribeUnitTests.java b/qa/evil-tests/src/test/java/org/elasticsearch/tribe/TribeUnitTests.java index 63c09890acc00..0a1698efcab6e 100644 --- a/qa/evil-tests/src/test/java/org/elasticsearch/tribe/TribeUnitTests.java +++ b/qa/evil-tests/src/test/java/org/elasticsearch/tribe/TribeUnitTests.java @@ -23,10 +23,10 @@ import org.elasticsearch.client.Client; import org.elasticsearch.cluster.ClusterState; import org.elasticsearch.cluster.node.DiscoveryNode; -import org.elasticsearch.cluster.node.DiscoveryNodeService; import org.elasticsearch.common.SuppressForbidden; import org.elasticsearch.common.settings.Settings; import org.elasticsearch.env.Environment; +import org.elasticsearch.env.NodeEnvironment; import org.elasticsearch.node.Node; import org.elasticsearch.node.internal.InternalSettingsPreparer; import org.elasticsearch.test.ESIntegTestCase; @@ -66,14 +66,14 @@ public static void createTribes() { .put(baseSettings) .put("cluster.name", "tribe1") .put("node.name", "tribe1_node") - .put(DiscoveryNodeService.NODE_ID_SEED_SETTING.getKey(), random().nextLong()) + .put(NodeEnvironment.NODE_ID_SEED_SETTING.getKey(), random().nextLong()) .build()).start(); tribe2 = new TribeClientNode( Settings.builder() .put(baseSettings) .put("cluster.name", "tribe2") .put("node.name", "tribe2_node") - .put(DiscoveryNodeService.NODE_ID_SEED_SETTING.getKey(), random().nextLong()) + .put(NodeEnvironment.NODE_ID_SEED_SETTING.getKey(), random().nextLong()) .build()).start(); } diff --git a/test/framework/src/main/java/org/elasticsearch/test/InternalTestCluster.java b/test/framework/src/main/java/org/elasticsearch/test/InternalTestCluster.java index 6a72d5cc1a969..7691fc2d01bcb 100644 --- a/test/framework/src/main/java/org/elasticsearch/test/InternalTestCluster.java +++ b/test/framework/src/main/java/org/elasticsearch/test/InternalTestCluster.java @@ -39,7 +39,6 @@ import org.elasticsearch.cluster.action.index.MappingUpdatedAction; import org.elasticsearch.cluster.metadata.IndexMetaData; import org.elasticsearch.cluster.node.DiscoveryNode; -import org.elasticsearch.cluster.node.DiscoveryNodeService; import org.elasticsearch.cluster.node.DiscoveryNodes; import org.elasticsearch.cluster.routing.OperationRouting; import org.elasticsearch.cluster.routing.ShardRouting; @@ -578,7 +577,7 @@ private NodeAndClient buildNode(int nodeId, long seed, Settings settings, Versio .put(Environment.PATH_HOME_SETTING.getKey(), baseDir) // allow overriding path.home .put(settings) .put("node.name", name) - .put(DiscoveryNodeService.NODE_ID_SEED_SETTING.getKey(), seed) + .put(NodeEnvironment.NODE_ID_SEED_SETTING.getKey(), seed) .build(); MockNode node = new MockNode(finalSettings, version, plugins); return new NodeAndClient(name, node); @@ -826,8 +825,9 @@ void restart(RestartCallback callback) throws Exception { IOUtils.rm(nodeEnv.nodeDataPaths()); } } - final long newIdSeed = DiscoveryNodeService.NODE_ID_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(DiscoveryNodeService.NODE_ID_SEED_SETTING.getKey(), newIdSeed).build(); + // use a new seed to make sure we have new node id, if the node type is one that generates new ids + final long newIdSeed = NodeEnvironment.NODE_ID_SEED_SETTING.get(node.settings()) + 1; + Settings finalSettings = Settings.builder().put(node.settings()).put(newSettings).put(NodeEnvironment.NODE_ID_SEED_SETTING.getKey(), newIdSeed).build(); Collection> plugins = node.getPlugins(); Version version = node.getVersion(); node = new MockNode(finalSettings, version, plugins); From 31bc94e366ee2483f9a94e8e6cf591eff6dada05 Mon Sep 17 00:00:00 2001 From: Boaz Leskes Date: Sat, 16 Apr 2016 22:13:10 +0200 Subject: [PATCH 02/34] enforce unique network addresses across nodes in cluster --- .../cluster/node/DiscoveryNodes.java | 26 +++++++ .../transport/DummyTransportAddress.java | 74 ------------------- .../transport/LocalTransportAddress.java | 9 +++ .../TransportAddressSerializers.java | 10 +++ .../discovery/zen/NodeJoinController.java | 38 ++++++++-- .../transport/local/LocalTransport.java | 7 +- .../ClusterAllocationExplanationTests.java | 4 +- .../IndicesShardStoreResponseTests.java | 8 +- .../ingest/IngestProxyActionFilterTests.java | 4 +- .../TransportBroadcastByNodeActionTests.java | 4 +- .../TransportMasterNodeActionTests.java | 6 +- .../nodes/TransportNodesActionTests.java | 4 +- .../ClusterStateCreationUtils.java | 4 +- .../cluster/ClusterChangedEventTests.java | 4 +- .../cluster/ClusterStateTests.java | 6 +- .../elasticsearch/cluster/DiskUsageTests.java | 16 ++-- .../cluster/NodeConnectionsServiceTests.java | 4 +- .../node/DiscoveryNodeFiltersTests.java | 26 +++---- .../node/DiscoveryNodeServiceTests.java | 6 +- .../cluster/node/DiscoveryNodesTests.java | 4 +- .../NodeVersionAllocationDeciderTests.java | 14 ++-- .../allocation/SameShardRoutingTests.java | 8 +- .../DiskThresholdDeciderUnitTests.java | 9 +-- .../ClusterStateToStringTests.java | 4 +- .../cluster/service/ClusterServiceTests.java | 4 +- .../cluster/service/ClusterServiceUtils.java | 4 +- ...usterStatePublishResponseHandlerTests.java | 4 +- .../zen/ElectMasterServiceTests.java | 4 +- .../zen/NodeJoinControllerTests.java | 13 +++- .../discovery/zen/ZenDiscoveryUnitTests.java | 11 ++- .../discovery/zen/ZenPingTests.java | 4 +- .../PendingClusterStatesQueueTests.java | 4 +- .../gateway/AsyncShardFetchTests.java | 6 +- .../index/shard/IndexShardTests.java | 18 ++--- .../IndexingMemoryControllerTests.java | 10 +-- ...dicesLifecycleListenerSingleNodeTests.java | 4 +- .../recovery/RecoverySourceHandlerTests.java | 14 ++-- .../indices/recovery/RecoveryTargetTests.java | 4 +- .../indices/state/RareClusterStateIT.java | 4 +- .../nodesinfo/NodeInfoStreamingTests.java | 6 +- .../recovery/RecoveriesCollectionTests.java | 4 +- .../MockInternalClusterInfoService.java | 4 +- .../test/ESAllocationTestCase.java | 10 +-- 43 files changed, 211 insertions(+), 220 deletions(-) delete mode 100644 core/src/main/java/org/elasticsearch/common/transport/DummyTransportAddress.java diff --git a/core/src/main/java/org/elasticsearch/cluster/node/DiscoveryNodes.java b/core/src/main/java/org/elasticsearch/cluster/node/DiscoveryNodes.java index ed4cfbd913447..04be959cd843a 100644 --- a/core/src/main/java/org/elasticsearch/cluster/node/DiscoveryNodes.java +++ b/core/src/main/java/org/elasticsearch/cluster/node/DiscoveryNodes.java @@ -22,6 +22,7 @@ import com.carrotsearch.hppc.ObjectHashSet; import com.carrotsearch.hppc.cursors.ObjectCursor; import com.carrotsearch.hppc.cursors.ObjectObjectCursor; +import org.elasticsearch.ElasticsearchException; import org.elasticsearch.Version; import org.elasticsearch.cluster.AbstractDiffable; import org.elasticsearch.common.Booleans; @@ -565,6 +566,10 @@ public Builder(DiscoveryNodes nodes) { } public Builder put(DiscoveryNode node) { + final String preflight = preflightPut(node); + if (preflight != null) { + throw new ElasticsearchException(preflight); + } nodes.put(node.getId(), node); return this; } @@ -584,6 +589,25 @@ public Builder localNodeId(String localNodeId) { return this; } + /** + * Checks that a node can be safely added to this node collection. + * + * @return null if all is OK or an error message explaining why a node can not be added. + * + * Note: if this method returns a non-null value, calling {@link #put(DiscoveryNode)} will fail with an + * exception + */ + public String preflightPut(DiscoveryNode node) { + for (ObjectCursor cursor : nodes.values()) { + final DiscoveryNode existingNode = cursor.value; + if (node.getAddress().equals(existingNode.getAddress()) && + node.equals(existingNode) == false) { + return "can't add node " + node +", found existing node " + existingNode + " with same address"; + } + } + return null; + } + public DiscoveryNodes build() { ImmutableOpenMap.Builder dataNodesBuilder = ImmutableOpenMap.builder(); ImmutableOpenMap.Builder masterNodesBuilder = ImmutableOpenMap.builder(); @@ -614,5 +638,7 @@ public DiscoveryNodes build() { public static DiscoveryNodes readFrom(StreamInput in, @Nullable DiscoveryNode localNode) throws IOException { return PROTO.readFrom(in, localNode); } + + } } diff --git a/core/src/main/java/org/elasticsearch/common/transport/DummyTransportAddress.java b/core/src/main/java/org/elasticsearch/common/transport/DummyTransportAddress.java deleted file mode 100644 index dae4a37765d21..0000000000000 --- a/core/src/main/java/org/elasticsearch/common/transport/DummyTransportAddress.java +++ /dev/null @@ -1,74 +0,0 @@ -/* - * Licensed to Elasticsearch under one or more contributor - * license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright - * ownership. Elasticsearch licenses this file to you under - * the Apache License, Version 2.0 (the "License"); you may - * not use this file except in compliance with the License. - * You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, - * software distributed under the License is distributed on an - * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY - * KIND, either express or implied. See the License for the - * specific language governing permissions and limitations - * under the License. - */ - -package org.elasticsearch.common.transport; - -import org.elasticsearch.common.io.stream.StreamOutput; - -import java.io.IOException; - -/** - * - */ -public class DummyTransportAddress implements TransportAddress { - - public static final DummyTransportAddress INSTANCE = new DummyTransportAddress(); - - private DummyTransportAddress() { - } - - @Override - public short uniqueAddressTypeId() { - return 0; - } - - @Override - public boolean sameHost(TransportAddress other) { - return other == INSTANCE; - } - - @Override - public boolean isLoopbackOrLinkLocalAddress() { - return false; - } - - @Override - public String getHost() { - return "dummy"; - } - - @Override - public String getAddress() { - return "0.0.0.0"; // see https://en.wikipedia.org/wiki/0.0.0.0 - } - - @Override - public int getPort() { - return 42; - } - - @Override - public void writeTo(StreamOutput out) throws IOException { - } - - @Override - public String toString() { - return "_dummy_addr_"; - } -} diff --git a/core/src/main/java/org/elasticsearch/common/transport/LocalTransportAddress.java b/core/src/main/java/org/elasticsearch/common/transport/LocalTransportAddress.java index 9ded8dd23e170..97c9541a7ee1f 100644 --- a/core/src/main/java/org/elasticsearch/common/transport/LocalTransportAddress.java +++ b/core/src/main/java/org/elasticsearch/common/transport/LocalTransportAddress.java @@ -23,6 +23,7 @@ import org.elasticsearch.common.io.stream.StreamOutput; import java.io.IOException; +import java.util.concurrent.atomic.AtomicLong; /** * @@ -30,6 +31,14 @@ public final class LocalTransportAddress implements TransportAddress { public static final short TYPE_ID = 2; + private static final AtomicLong transportAddressIdGenerator = new AtomicLong(); + + + /** generates a new unique address */ + static public LocalTransportAddress buildUnique() { + return new LocalTransportAddress(Long.toString(transportAddressIdGenerator.incrementAndGet())); + } + private String id; public LocalTransportAddress(String id) { diff --git a/core/src/main/java/org/elasticsearch/common/transport/TransportAddressSerializers.java b/core/src/main/java/org/elasticsearch/common/transport/TransportAddressSerializers.java index d5db255816340..fa55b32278654 100644 --- a/core/src/main/java/org/elasticsearch/common/transport/TransportAddressSerializers.java +++ b/core/src/main/java/org/elasticsearch/common/transport/TransportAddressSerializers.java @@ -36,10 +36,20 @@ public abstract class TransportAddressSerializers { private static final Map> ADDRESS_REGISTRY; static { +<<<<<<< bd5693f6ca2f3945838810ac05999c91e1c2796f Map> registry = new HashMap<>(); addAddressType(registry, DummyTransportAddress.INSTANCE.uniqueAddressTypeId(), (in) -> DummyTransportAddress.INSTANCE); addAddressType(registry, InetSocketTransportAddress.TYPE_ID, InetSocketTransportAddress::new); addAddressType(registry, LocalTransportAddress.TYPE_ID, LocalTransportAddress::new); +======= + Map registry = new HashMap<>(); + try { + addAddressType(registry, InetSocketTransportAddress.PROTO); + addAddressType(registry, LocalTransportAddress.PROTO); + } catch (Exception e) { + logger.warn("Failed to setup TransportAddresses", e); + } +>>>>>>> enforce unique network addresses across nodes in cluster ADDRESS_REGISTRY = unmodifiableMap(registry); } diff --git a/core/src/main/java/org/elasticsearch/discovery/zen/NodeJoinController.java b/core/src/main/java/org/elasticsearch/discovery/zen/NodeJoinController.java index 39a3f51ac3279..cc1c1e409215b 100644 --- a/core/src/main/java/org/elasticsearch/discovery/zen/NodeJoinController.java +++ b/core/src/main/java/org/elasticsearch/discovery/zen/NodeJoinController.java @@ -29,6 +29,7 @@ import org.elasticsearch.cluster.routing.allocation.RoutingAllocation; import org.elasticsearch.cluster.service.ClusterService; import org.elasticsearch.common.Priority; +import org.elasticsearch.common.collect.Tuple; import org.elasticsearch.common.component.AbstractComponent; import org.elasticsearch.common.settings.Settings; import org.elasticsearch.common.unit.TimeValue; @@ -357,6 +358,7 @@ private void assertClusterStateThread() { class ProcessJoinsTask extends ClusterStateUpdateTask { private final List joinCallbacksToRespondTo = new ArrayList<>(); + private final List> joinCallbacksToFail = new ArrayList<>(); private boolean nodeAdded = false; public ProcessJoinsTask(Priority priority) { @@ -376,20 +378,26 @@ public ClusterState execute(ClusterState currentState) { while (iterator.hasNext()) { Map.Entry> entry = iterator.next(); final DiscoveryNode node = entry.getKey(); - joinCallbacksToRespondTo.addAll(entry.getValue()); iterator.remove(); - if (currentState.nodes().nodeExists(node.getId())) { + final String preflight = nodesBuilder.preflightPut(node); + if (preflight != null) { + Throwable failure = new IllegalStateException(preflight); + for (MembershipAction.JoinCallback callback: entry.getValue()) { + joinCallbacksToFail.add(new Tuple<>(callback, failure)); + } + } else if (currentState.nodes().nodeExists(node.getId())) { logger.debug("received a join request for an existing node [{}]", node); + joinCallbacksToRespondTo.addAll(entry.getValue()); } else { nodeAdded = true; nodesBuilder.put(node); - for (DiscoveryNode existingNode : currentState.nodes()) { - if (node.getAddress().equals(existingNode.getAddress())) { - nodesBuilder.remove(existingNode.getId()); - logger.warn("received join request from node [{}], but found existing node {} with same address, removing existing node", node, existingNode); - } - } + joinCallbacksToRespondTo.addAll(entry.getValue()); } + + assert entry.getValue().stream().allMatch(cb -> + joinCallbacksToRespondTo.contains(cb) ^ + joinCallbacksToFail.stream().filter(tuple -> tuple.v1().equals(cb)).count() > 0 + ) : "failed to add " + entry.getValue() + "to joinCallbacksToRespondTo or joinCallbacksToFail"; } } @@ -426,6 +434,13 @@ void innerOnFailure(Throwable t) { logger.error("error during task failure", e); } } + for (Tuple tuple : joinCallbacksToFail) { + try { + tuple.v1().onFailure(tuple.v2()); + } catch (Exception e) { + logger.error("error during task failure", e); + } + } } @Override @@ -449,6 +464,13 @@ public void clusterStateProcessed(String source, ClusterState oldState, ClusterS logger.error("unexpected error during [{}]", e, source); } } + for (Tuple tuple : joinCallbacksToFail) { + try { + tuple.v1().onFailure(tuple.v2()); + } catch (Exception e) { + logger.error("error during callback failure", e); + } + } } } } diff --git a/core/src/main/java/org/elasticsearch/transport/local/LocalTransport.java b/core/src/main/java/org/elasticsearch/transport/local/LocalTransport.java index 1c224e1aeef76..60aaaf9e0b2dd 100644 --- a/core/src/main/java/org/elasticsearch/transport/local/LocalTransport.java +++ b/core/src/main/java/org/elasticsearch/transport/local/LocalTransport.java @@ -65,7 +65,6 @@ import java.util.concurrent.ThreadFactory; import java.util.concurrent.ThreadPoolExecutor; import java.util.concurrent.TimeUnit; -import java.util.concurrent.atomic.AtomicLong; import static org.elasticsearch.common.util.concurrent.ConcurrentCollections.newConcurrentMap; @@ -82,7 +81,6 @@ public class LocalTransport extends AbstractLifecycleComponent implem private volatile BoundTransportAddress boundAddress; private volatile LocalTransportAddress localAddress; private final static ConcurrentMap transports = newConcurrentMap(); - private static final AtomicLong transportAddressIdGenerator = new AtomicLong(); private final ConcurrentMap connectedNodes = newConcurrentMap(); protected final NamedWriteableRegistry namedWriteableRegistry; private final CircuitBreakerService circuitBreakerService; @@ -121,9 +119,10 @@ public boolean addressSupported(Class address) { protected void doStart() { String address = settings.get(TRANSPORT_LOCAL_ADDRESS); if (address == null) { - address = Long.toString(transportAddressIdGenerator.incrementAndGet()); + localAddress = LocalTransportAddress.buildUnique(); + } else { + localAddress = new LocalTransportAddress(address); } - localAddress = new LocalTransportAddress(address); LocalTransport previous = transports.put(localAddress, this); if (previous != null) { throw new ElasticsearchException("local address [" + address + "] is already bound"); diff --git a/core/src/test/java/org/elasticsearch/action/admin/cluster/allocation/ClusterAllocationExplanationTests.java b/core/src/test/java/org/elasticsearch/action/admin/cluster/allocation/ClusterAllocationExplanationTests.java index 3a873f4252435..162ada87af941 100644 --- a/core/src/test/java/org/elasticsearch/action/admin/cluster/allocation/ClusterAllocationExplanationTests.java +++ b/core/src/test/java/org/elasticsearch/action/admin/cluster/allocation/ClusterAllocationExplanationTests.java @@ -24,7 +24,7 @@ import org.elasticsearch.cluster.routing.allocation.decider.Decision; import org.elasticsearch.common.io.stream.BytesStreamOutput; import org.elasticsearch.common.io.stream.StreamInput; -import org.elasticsearch.common.transport.DummyTransportAddress; +import org.elasticsearch.common.transport.LocalTransportAddress; import org.elasticsearch.index.shard.ShardId; import org.elasticsearch.test.ESTestCase; @@ -56,7 +56,7 @@ public void testExplanationSerialization() throws Exception { Map nodeToDecisions = new HashMap<>(); Map nodeToWeight = new HashMap<>(); for (int i = randomIntBetween(2, 5); i > 0; i--) { - DiscoveryNode dn = new DiscoveryNode("node-" + i, DummyTransportAddress.INSTANCE, emptyMap(), emptySet(), Version.CURRENT); + DiscoveryNode dn = new DiscoveryNode("node-" + i, LocalTransportAddress.buildUnique(), emptyMap(), emptySet(), Version.CURRENT); Decision.Multi d = new Decision.Multi(); d.add(Decision.single(Decision.Type.NO, "no label", "because I said no")); d.add(Decision.single(Decision.Type.YES, "yes label", "yes please")); diff --git a/core/src/test/java/org/elasticsearch/action/admin/indices/shards/IndicesShardStoreResponseTests.java b/core/src/test/java/org/elasticsearch/action/admin/indices/shards/IndicesShardStoreResponseTests.java index 3c12d7d9b100c..9705009a044bb 100644 --- a/core/src/test/java/org/elasticsearch/action/admin/indices/shards/IndicesShardStoreResponseTests.java +++ b/core/src/test/java/org/elasticsearch/action/admin/indices/shards/IndicesShardStoreResponseTests.java @@ -26,7 +26,7 @@ import org.elasticsearch.common.bytes.BytesReference; import org.elasticsearch.common.collect.ImmutableOpenIntMap; import org.elasticsearch.common.collect.ImmutableOpenMap; -import org.elasticsearch.common.transport.DummyTransportAddress; +import org.elasticsearch.common.transport.LocalTransportAddress; import org.elasticsearch.common.xcontent.ToXContent; import org.elasticsearch.common.xcontent.XContentBuilder; import org.elasticsearch.common.xcontent.XContentFactory; @@ -53,8 +53,8 @@ public void testBasicSerialization() throws Exception { List failures = new ArrayList<>(); ImmutableOpenIntMap.Builder> storeStatuses = ImmutableOpenIntMap.builder(); - DiscoveryNode node1 = new DiscoveryNode("node1", DummyTransportAddress.INSTANCE, emptyMap(), emptySet(), Version.CURRENT); - DiscoveryNode node2 = new DiscoveryNode("node2", DummyTransportAddress.INSTANCE, emptyMap(), emptySet(), Version.CURRENT); + DiscoveryNode node1 = new DiscoveryNode("node1", LocalTransportAddress.buildUnique(), emptyMap(), emptySet(), Version.CURRENT); + DiscoveryNode node2 = new DiscoveryNode("node2", LocalTransportAddress.buildUnique(), emptyMap(), emptySet(), Version.CURRENT); List storeStatusList = new ArrayList<>(); storeStatusList.add(new IndicesShardStoresResponse.StoreStatus(node1, 3, null, IndicesShardStoresResponse.StoreStatus.AllocationStatus.PRIMARY, null)); storeStatusList.add(new IndicesShardStoresResponse.StoreStatus(node2, ShardStateMetaData.NO_VERSION, UUIDs.randomBase64UUID(), IndicesShardStoresResponse.StoreStatus.AllocationStatus.REPLICA, null)); @@ -122,7 +122,7 @@ public void testBasicSerialization() throws Exception { } public void testStoreStatusOrdering() throws Exception { - DiscoveryNode node1 = new DiscoveryNode("node1", DummyTransportAddress.INSTANCE, emptyMap(), emptySet(), Version.CURRENT); + DiscoveryNode node1 = new DiscoveryNode("node1", LocalTransportAddress.buildUnique(), emptyMap(), emptySet(), Version.CURRENT); List orderedStoreStatuses = new ArrayList<>(); orderedStoreStatuses.add(new IndicesShardStoresResponse.StoreStatus(node1, ShardStateMetaData.NO_VERSION, UUIDs.randomBase64UUID(), IndicesShardStoresResponse.StoreStatus.AllocationStatus.PRIMARY, null)); orderedStoreStatuses.add(new IndicesShardStoresResponse.StoreStatus(node1, ShardStateMetaData.NO_VERSION, UUIDs.randomBase64UUID(), IndicesShardStoresResponse.StoreStatus.AllocationStatus.REPLICA, null)); diff --git a/core/src/test/java/org/elasticsearch/action/ingest/IngestProxyActionFilterTests.java b/core/src/test/java/org/elasticsearch/action/ingest/IngestProxyActionFilterTests.java index a62946bf0f675..3d1a1a1c69d26 100644 --- a/core/src/test/java/org/elasticsearch/action/ingest/IngestProxyActionFilterTests.java +++ b/core/src/test/java/org/elasticsearch/action/ingest/IngestProxyActionFilterTests.java @@ -33,7 +33,7 @@ import org.elasticsearch.cluster.node.DiscoveryNode; import org.elasticsearch.cluster.node.DiscoveryNodes; import org.elasticsearch.cluster.service.ClusterService; -import org.elasticsearch.common.transport.DummyTransportAddress; +import org.elasticsearch.common.transport.LocalTransportAddress; import org.elasticsearch.tasks.Task; import org.elasticsearch.test.ESTestCase; import org.elasticsearch.test.VersionUtils; @@ -78,7 +78,7 @@ private IngestProxyActionFilter buildFilter(int ingestNodes, int totalNodes) { if (i < ingestNodes) { roles.add(DiscoveryNode.Role.INGEST); } - DiscoveryNode node = new DiscoveryNode(nodeId, nodeId, DummyTransportAddress.INSTANCE, attributes, roles, VersionUtils.randomVersion(random())); + DiscoveryNode node = new DiscoveryNode(nodeId, nodeId, LocalTransportAddress.buildUnique(), attributes, roles, VersionUtils.randomVersion(random())); builder.put(node); if (i == totalNodes - 1) { localNode = node; diff --git a/core/src/test/java/org/elasticsearch/action/support/broadcast/node/TransportBroadcastByNodeActionTests.java b/core/src/test/java/org/elasticsearch/action/support/broadcast/node/TransportBroadcastByNodeActionTests.java index f76fdf4fd2072..b9e794f7d434a 100644 --- a/core/src/test/java/org/elasticsearch/action/support/broadcast/node/TransportBroadcastByNodeActionTests.java +++ b/core/src/test/java/org/elasticsearch/action/support/broadcast/node/TransportBroadcastByNodeActionTests.java @@ -49,7 +49,7 @@ import org.elasticsearch.cluster.service.ClusterService; import org.elasticsearch.common.io.stream.StreamInput; import org.elasticsearch.common.settings.Settings; -import org.elasticsearch.common.transport.DummyTransportAddress; +import org.elasticsearch.common.transport.LocalTransportAddress; import org.elasticsearch.index.Index; import org.elasticsearch.index.shard.ShardId; import org.elasticsearch.rest.RestStatus; @@ -246,7 +246,7 @@ void setClusterState(ClusterService clusterService, String index) { } static DiscoveryNode newNode(int nodeId) { - return new DiscoveryNode("node_" + nodeId, DummyTransportAddress.INSTANCE, emptyMap(), emptySet(), Version.CURRENT); + return new DiscoveryNode("node_" + nodeId, LocalTransportAddress.buildUnique(), emptyMap(), emptySet(), Version.CURRENT); } @AfterClass diff --git a/core/src/test/java/org/elasticsearch/action/support/master/TransportMasterNodeActionTests.java b/core/src/test/java/org/elasticsearch/action/support/master/TransportMasterNodeActionTests.java index a35accc5fc52e..ce6b7e3cf4695 100644 --- a/core/src/test/java/org/elasticsearch/action/support/master/TransportMasterNodeActionTests.java +++ b/core/src/test/java/org/elasticsearch/action/support/master/TransportMasterNodeActionTests.java @@ -37,7 +37,7 @@ import org.elasticsearch.cluster.node.DiscoveryNode; import org.elasticsearch.cluster.service.ClusterService; import org.elasticsearch.common.settings.Settings; -import org.elasticsearch.common.transport.DummyTransportAddress; +import org.elasticsearch.common.transport.LocalTransportAddress; import org.elasticsearch.common.unit.TimeValue; import org.elasticsearch.discovery.Discovery; import org.elasticsearch.discovery.MasterNotDiscoveredException; @@ -88,9 +88,9 @@ public void setUp() throws Exception { transportService = new TransportService(transport, threadPool); transportService.start(); transportService.acceptIncomingRequests(); - localNode = new DiscoveryNode("local_node", DummyTransportAddress.INSTANCE, Collections.emptyMap(), + localNode = new DiscoveryNode("local_node", LocalTransportAddress.buildUnique(), Collections.emptyMap(), Collections.singleton(DiscoveryNode.Role.MASTER), Version.CURRENT); - remoteNode = new DiscoveryNode("remote_node", DummyTransportAddress.INSTANCE, Collections.emptyMap(), + remoteNode = new DiscoveryNode("remote_node", LocalTransportAddress.buildUnique(), Collections.emptyMap(), Collections.singleton(DiscoveryNode.Role.MASTER), Version.CURRENT); allNodes = new DiscoveryNode[]{localNode, remoteNode}; } diff --git a/core/src/test/java/org/elasticsearch/action/support/nodes/TransportNodesActionTests.java b/core/src/test/java/org/elasticsearch/action/support/nodes/TransportNodesActionTests.java index ad2326c314835..37d36b3c12168 100644 --- a/core/src/test/java/org/elasticsearch/action/support/nodes/TransportNodesActionTests.java +++ b/core/src/test/java/org/elasticsearch/action/support/nodes/TransportNodesActionTests.java @@ -29,7 +29,7 @@ import org.elasticsearch.cluster.node.DiscoveryNodes; import org.elasticsearch.cluster.service.ClusterService; import org.elasticsearch.common.settings.Settings; -import org.elasticsearch.common.transport.DummyTransportAddress; +import org.elasticsearch.common.transport.LocalTransportAddress; import org.elasticsearch.test.ESTestCase; import org.elasticsearch.test.transport.CapturingTransport; import org.elasticsearch.threadpool.ThreadPool; @@ -162,7 +162,7 @@ public void tearDown() throws Exception { private static DiscoveryNode newNode(int nodeId, Map attributes, Set roles) { String node = "node_" + nodeId; - return new DiscoveryNode(node, node, DummyTransportAddress.INSTANCE, attributes, roles, Version.CURRENT); + return new DiscoveryNode(node, node, LocalTransportAddress.buildUnique(), attributes, roles, Version.CURRENT); } private static class TestTransportNodesAction extends TransportNodesAction(Arrays.asList(DiscoveryNode.Role.values())), Version.CURRENT); } diff --git a/core/src/test/java/org/elasticsearch/cluster/ClusterChangedEventTests.java b/core/src/test/java/org/elasticsearch/cluster/ClusterChangedEventTests.java index e488c1b1bcc76..a8fd033265b2f 100644 --- a/core/src/test/java/org/elasticsearch/cluster/ClusterChangedEventTests.java +++ b/core/src/test/java/org/elasticsearch/cluster/ClusterChangedEventTests.java @@ -28,7 +28,7 @@ import org.elasticsearch.cluster.routing.RoutingTable; import org.elasticsearch.common.UUIDs; import org.elasticsearch.common.settings.Settings; -import org.elasticsearch.common.transport.DummyTransportAddress; +import org.elasticsearch.common.transport.LocalTransportAddress; import org.elasticsearch.test.ESTestCase; import java.util.ArrayList; @@ -316,7 +316,7 @@ private static DiscoveryNodes createDiscoveryNodes(final int numNodes, final boo // Create a new DiscoveryNode private static DiscoveryNode newNode(final String nodeId, Set roles) { - return new DiscoveryNode(nodeId, nodeId, DummyTransportAddress.INSTANCE, Collections.emptyMap(), roles, Version.CURRENT); + return new DiscoveryNode(nodeId, nodeId, LocalTransportAddress.buildUnique(), Collections.emptyMap(), roles, Version.CURRENT); } // Create the metadata for a cluster state. diff --git a/core/src/test/java/org/elasticsearch/cluster/ClusterStateTests.java b/core/src/test/java/org/elasticsearch/cluster/ClusterStateTests.java index 92be2331a8c92..751f441ece068 100644 --- a/core/src/test/java/org/elasticsearch/cluster/ClusterStateTests.java +++ b/core/src/test/java/org/elasticsearch/cluster/ClusterStateTests.java @@ -21,7 +21,7 @@ import org.elasticsearch.Version; import org.elasticsearch.cluster.node.DiscoveryNode; import org.elasticsearch.cluster.node.DiscoveryNodes; -import org.elasticsearch.common.transport.DummyTransportAddress; +import org.elasticsearch.common.transport.LocalTransportAddress; import org.elasticsearch.test.ESTestCase; import static java.util.Collections.emptyMap; @@ -31,8 +31,8 @@ public class ClusterStateTests extends ESTestCase { public void testSupersedes() { - final DiscoveryNode node1 = new DiscoveryNode("node1", DummyTransportAddress.INSTANCE, emptyMap(), emptySet(), Version.CURRENT); - final DiscoveryNode node2 = new DiscoveryNode("node2", DummyTransportAddress.INSTANCE, emptyMap(), emptySet(), Version.CURRENT); + final DiscoveryNode node1 = new DiscoveryNode("node1", LocalTransportAddress.buildUnique(), emptyMap(), emptySet(), Version.CURRENT); + final DiscoveryNode node2 = new DiscoveryNode("node2", LocalTransportAddress.buildUnique(), emptyMap(), emptySet(), Version.CURRENT); final DiscoveryNodes nodes = DiscoveryNodes.builder().put(node1).put(node2).build(); ClusterState noMaster1 = ClusterState.builder(ClusterName.DEFAULT).version(randomInt(5)).nodes(nodes).build(); ClusterState noMaster2 = ClusterState.builder(ClusterName.DEFAULT).version(randomInt(5)).nodes(nodes).build(); diff --git a/core/src/test/java/org/elasticsearch/cluster/DiskUsageTests.java b/core/src/test/java/org/elasticsearch/cluster/DiskUsageTests.java index 1826ced24ea30..ec9dd7e363c32 100644 --- a/core/src/test/java/org/elasticsearch/cluster/DiskUsageTests.java +++ b/core/src/test/java/org/elasticsearch/cluster/DiskUsageTests.java @@ -23,8 +23,6 @@ import org.elasticsearch.action.admin.cluster.node.stats.NodeStats; import org.elasticsearch.action.admin.indices.stats.CommonStats; import org.elasticsearch.action.admin.indices.stats.ShardStats; -import org.elasticsearch.cluster.ClusterName; -import org.elasticsearch.cluster.ClusterState; import org.elasticsearch.cluster.metadata.IndexMetaData; import org.elasticsearch.cluster.metadata.MetaData; import org.elasticsearch.cluster.node.DiscoveryNode; @@ -33,7 +31,7 @@ import org.elasticsearch.cluster.routing.UnassignedInfo; import org.elasticsearch.common.collect.ImmutableOpenMap; import org.elasticsearch.common.settings.Settings; -import org.elasticsearch.common.transport.DummyTransportAddress; +import org.elasticsearch.common.transport.LocalTransportAddress; import org.elasticsearch.index.Index; import org.elasticsearch.index.shard.ShardPath; import org.elasticsearch.index.store.StoreStats; @@ -197,11 +195,11 @@ public void testFillDiskUsage() { new FsInfo.Path("/most", "/dev/sda", 100, 90, 80), }; NodeStats[] nodeStats = new NodeStats[] { - new NodeStats(new DiscoveryNode("node_1", DummyTransportAddress.INSTANCE, emptyMap(), emptySet(), Version.CURRENT), 0, + new NodeStats(new DiscoveryNode("node_1", LocalTransportAddress.buildUnique(), emptyMap(), emptySet(), Version.CURRENT), 0, null,null,null,null,null,new FsInfo(0, node1FSInfo), null,null,null,null,null, null), - new NodeStats(new DiscoveryNode("node_2", DummyTransportAddress.INSTANCE, emptyMap(), emptySet(), Version.CURRENT), 0, + new NodeStats(new DiscoveryNode("node_2", LocalTransportAddress.buildUnique(), emptyMap(), emptySet(), Version.CURRENT), 0, null,null,null,null,null, new FsInfo(0, node2FSInfo), null,null,null,null,null, null), - new NodeStats(new DiscoveryNode("node_3", DummyTransportAddress.INSTANCE, emptyMap(), emptySet(), Version.CURRENT), 0, + new NodeStats(new DiscoveryNode("node_3", LocalTransportAddress.buildUnique(), emptyMap(), emptySet(), Version.CURRENT), 0, null,null,null,null,null, new FsInfo(0, node3FSInfo), null,null,null,null,null, null) }; InternalClusterInfoService.fillDiskUsagePerNode(logger, nodeStats, newLeastAvaiableUsages, newMostAvaiableUsages); @@ -238,11 +236,11 @@ public void testFillDiskUsageSomeInvalidValues() { new FsInfo.Path("/least", "/dev/sda", 10, -8, 0), }; NodeStats[] nodeStats = new NodeStats[] { - new NodeStats(new DiscoveryNode("node_1", DummyTransportAddress.INSTANCE, emptyMap(), emptySet(), Version.CURRENT), 0, + new NodeStats(new DiscoveryNode("node_1", LocalTransportAddress.buildUnique(), emptyMap(), emptySet(), Version.CURRENT), 0, null,null,null,null,null,new FsInfo(0, node1FSInfo), null,null,null,null,null, null), - new NodeStats(new DiscoveryNode("node_2", DummyTransportAddress.INSTANCE, emptyMap(), emptySet(), Version.CURRENT), 0, + new NodeStats(new DiscoveryNode("node_2", LocalTransportAddress.buildUnique(), emptyMap(), emptySet(), Version.CURRENT), 0, null,null,null,null,null, new FsInfo(0, node2FSInfo), null,null,null,null,null, null), - new NodeStats(new DiscoveryNode("node_3", DummyTransportAddress.INSTANCE, emptyMap(), emptySet(), Version.CURRENT), 0, + new NodeStats(new DiscoveryNode("node_3", LocalTransportAddress.buildUnique(), emptyMap(), emptySet(), Version.CURRENT), 0, null,null,null,null,null, new FsInfo(0, node3FSInfo), null,null,null,null,null, null) }; InternalClusterInfoService.fillDiskUsagePerNode(logger, nodeStats, newLeastAvailableUsages, newMostAvailableUsages); diff --git a/core/src/test/java/org/elasticsearch/cluster/NodeConnectionsServiceTests.java b/core/src/test/java/org/elasticsearch/cluster/NodeConnectionsServiceTests.java index c263bcbcf378c..754a2f4990bb3 100644 --- a/core/src/test/java/org/elasticsearch/cluster/NodeConnectionsServiceTests.java +++ b/core/src/test/java/org/elasticsearch/cluster/NodeConnectionsServiceTests.java @@ -26,7 +26,7 @@ import org.elasticsearch.common.component.LifecycleListener; import org.elasticsearch.common.settings.Settings; import org.elasticsearch.common.transport.BoundTransportAddress; -import org.elasticsearch.common.transport.DummyTransportAddress; +import org.elasticsearch.common.transport.LocalTransportAddress; import org.elasticsearch.common.transport.TransportAddress; import org.elasticsearch.common.util.concurrent.ConcurrentCollections; import org.elasticsearch.test.ESTestCase; @@ -64,7 +64,7 @@ private List generateNodes() { List nodes = new ArrayList<>(); for (int i = randomIntBetween(20, 50); i > 0; i--) { Set roles = new HashSet<>(randomSubsetOf(Arrays.asList(DiscoveryNode.Role.values()))); - nodes.add(new DiscoveryNode("node_" + i, "" + i, DummyTransportAddress.INSTANCE, Collections.emptyMap(), + nodes.add(new DiscoveryNode("node_" + i, "" + i, LocalTransportAddress.buildUnique(), Collections.emptyMap(), roles, Version.CURRENT)); } return nodes; diff --git a/core/src/test/java/org/elasticsearch/cluster/node/DiscoveryNodeFiltersTests.java b/core/src/test/java/org/elasticsearch/cluster/node/DiscoveryNodeFiltersTests.java index 38aa73a99359f..6b1ae72e359e7 100644 --- a/core/src/test/java/org/elasticsearch/cluster/node/DiscoveryNodeFiltersTests.java +++ b/core/src/test/java/org/elasticsearch/cluster/node/DiscoveryNodeFiltersTests.java @@ -21,8 +21,8 @@ import org.elasticsearch.Version; import org.elasticsearch.common.settings.Settings; -import org.elasticsearch.common.transport.DummyTransportAddress; import org.elasticsearch.common.transport.InetSocketTransportAddress; +import org.elasticsearch.common.transport.LocalTransportAddress; import org.elasticsearch.test.ESTestCase; import org.junit.AfterClass; import org.junit.BeforeClass; @@ -64,10 +64,10 @@ public void testNameMatch() { .build(); DiscoveryNodeFilters filters = DiscoveryNodeFilters.buildFromSettings(OR, "xxx.", settings); - DiscoveryNode node = new DiscoveryNode("name1", "id1", DummyTransportAddress.INSTANCE, emptyMap(), emptySet(), Version.CURRENT); + DiscoveryNode node = new DiscoveryNode("name1", "id1", LocalTransportAddress.buildUnique(), emptyMap(), emptySet(), Version.CURRENT); assertThat(filters.match(node), equalTo(true)); - node = new DiscoveryNode("name2", "id2", DummyTransportAddress.INSTANCE, emptyMap(), emptySet(), Version.CURRENT); + node = new DiscoveryNode("name2", "id2", LocalTransportAddress.buildUnique(), emptyMap(), emptySet(), Version.CURRENT); assertThat(filters.match(node), equalTo(false)); } @@ -77,10 +77,10 @@ public void testIdMatch() { .build(); DiscoveryNodeFilters filters = DiscoveryNodeFilters.buildFromSettings(OR, "xxx.", settings); - DiscoveryNode node = new DiscoveryNode("name1", "id1", DummyTransportAddress.INSTANCE, emptyMap(), emptySet(), Version.CURRENT); + DiscoveryNode node = new DiscoveryNode("name1", "id1", LocalTransportAddress.buildUnique(), emptyMap(), emptySet(), Version.CURRENT); assertThat(filters.match(node), equalTo(true)); - node = new DiscoveryNode("name2", "id2", DummyTransportAddress.INSTANCE, emptyMap(), emptySet(), Version.CURRENT); + node = new DiscoveryNode("name2", "id2", LocalTransportAddress.buildUnique(), emptyMap(), emptySet(), Version.CURRENT); assertThat(filters.match(node), equalTo(false)); } @@ -91,13 +91,13 @@ public void testIdOrNameMatch() { .build()); DiscoveryNodeFilters filters = DiscoveryNodeFilters.buildFromSettings(OR, "xxx.", settings); - DiscoveryNode node = new DiscoveryNode("name1", "id1", DummyTransportAddress.INSTANCE, emptyMap(), emptySet(), Version.CURRENT); + DiscoveryNode node = new DiscoveryNode("name1", "id1", LocalTransportAddress.buildUnique(), emptyMap(), emptySet(), Version.CURRENT); assertThat(filters.match(node), equalTo(true)); - node = new DiscoveryNode("name2", "id2", DummyTransportAddress.INSTANCE, emptyMap(), emptySet(), Version.CURRENT); + node = new DiscoveryNode("name2", "id2", LocalTransportAddress.buildUnique(), emptyMap(), emptySet(), Version.CURRENT); assertThat(filters.match(node), equalTo(true)); - node = new DiscoveryNode("name3", "id3", DummyTransportAddress.INSTANCE, emptyMap(), emptySet(), Version.CURRENT); + node = new DiscoveryNode("name3", "id3", LocalTransportAddress.buildUnique(), emptyMap(), emptySet(), Version.CURRENT); assertThat(filters.match(node), equalTo(false)); } @@ -111,7 +111,7 @@ public void testTagAndGroupMatch() { Map attributes = new HashMap<>(); attributes.put("tag", "A"); attributes.put("group", "B"); - DiscoveryNode node = new DiscoveryNode("name1", "id1", DummyTransportAddress.INSTANCE, + DiscoveryNode node = new DiscoveryNode("name1", "id1", LocalTransportAddress.buildUnique(), attributes, emptySet(), Version.CURRENT); assertThat(filters.match(node), equalTo(true)); @@ -119,7 +119,7 @@ public void testTagAndGroupMatch() { attributes.put("tag", "A"); attributes.put("group", "B"); attributes.put("name", "X"); - node = new DiscoveryNode("name2", "id2", DummyTransportAddress.INSTANCE, + node = new DiscoveryNode("name2", "id2", LocalTransportAddress.buildUnique(), attributes, emptySet(), Version.CURRENT); assertThat(filters.match(node), equalTo(true)); @@ -127,11 +127,11 @@ public void testTagAndGroupMatch() { attributes.put("tag", "A"); attributes.put("group", "F"); attributes.put("name", "X"); - node = new DiscoveryNode("name3", "id3", DummyTransportAddress.INSTANCE, + node = new DiscoveryNode("name3", "id3", LocalTransportAddress.buildUnique(), attributes, emptySet(), Version.CURRENT); assertThat(filters.match(node), equalTo(false)); - node = new DiscoveryNode("name4", "id4", DummyTransportAddress.INSTANCE, emptyMap(), emptySet(), Version.CURRENT); + node = new DiscoveryNode("name4", "id4", LocalTransportAddress.buildUnique(), emptyMap(), emptySet(), Version.CURRENT); assertThat(filters.match(node), equalTo(false)); } @@ -141,7 +141,7 @@ public void testStarMatch() { .build(); DiscoveryNodeFilters filters = DiscoveryNodeFilters.buildFromSettings(OR, "xxx.", settings); - DiscoveryNode node = new DiscoveryNode("name1", "id1", DummyTransportAddress.INSTANCE, emptyMap(), emptySet(), Version.CURRENT); + DiscoveryNode node = new DiscoveryNode("name1", "id1", LocalTransportAddress.buildUnique(), emptyMap(), emptySet(), Version.CURRENT); assertThat(filters.match(node), equalTo(true)); } diff --git a/core/src/test/java/org/elasticsearch/cluster/node/DiscoveryNodeServiceTests.java b/core/src/test/java/org/elasticsearch/cluster/node/DiscoveryNodeServiceTests.java index 0e110f5fda930..a2c48b86295b0 100644 --- a/core/src/test/java/org/elasticsearch/cluster/node/DiscoveryNodeServiceTests.java +++ b/core/src/test/java/org/elasticsearch/cluster/node/DiscoveryNodeServiceTests.java @@ -21,7 +21,7 @@ import org.elasticsearch.Version; import org.elasticsearch.common.settings.Settings; -import org.elasticsearch.common.transport.DummyTransportAddress; +import org.elasticsearch.common.transport.LocalTransportAddress; import org.elasticsearch.env.NodeEnvironment; import org.elasticsearch.test.ESTestCase; @@ -57,7 +57,7 @@ public void testBuildLocalNode() { } DiscoveryNodeService discoveryNodeService = new DiscoveryNodeService(builder.build(), Version.CURRENT); DiscoveryNode discoveryNode = discoveryNodeService.buildLocalNode( - DummyTransportAddress.INSTANCE, () -> NodeEnvironment.generateNodeId(Settings.EMPTY)); + LocalTransportAddress.buildUnique(), () -> NodeEnvironment.generateNodeId(Settings.EMPTY)); assertThat(discoveryNode.getRoles(), equalTo(selectedRoles)); assertThat(discoveryNode.getAttributes(), equalTo(expectedAttributes)); } @@ -80,7 +80,7 @@ public void testBuildAttributesWithCustomAttributeServiceProvider() { discoveryNodeService.addCustomAttributeProvider(() -> customAttributes); DiscoveryNode discoveryNode = discoveryNodeService.buildLocalNode( - DummyTransportAddress.INSTANCE, () -> NodeEnvironment.generateNodeId(Settings.EMPTY)); + LocalTransportAddress.buildUnique(), () -> NodeEnvironment.generateNodeId(Settings.EMPTY)); assertThat(discoveryNode.getAttributes(), equalTo(expectedAttributes)); } } diff --git a/core/src/test/java/org/elasticsearch/cluster/node/DiscoveryNodesTests.java b/core/src/test/java/org/elasticsearch/cluster/node/DiscoveryNodesTests.java index b0942ab401c87..7c1c35b9ef50e 100644 --- a/core/src/test/java/org/elasticsearch/cluster/node/DiscoveryNodesTests.java +++ b/core/src/test/java/org/elasticsearch/cluster/node/DiscoveryNodesTests.java @@ -20,7 +20,7 @@ package org.elasticsearch.cluster.node; import org.elasticsearch.Version; -import org.elasticsearch.common.transport.DummyTransportAddress; +import org.elasticsearch.common.transport.LocalTransportAddress; import org.elasticsearch.test.ESTestCase; import java.util.ArrayList; @@ -117,7 +117,7 @@ private static DiscoveryNodes buildDiscoveryNodes() { } private static DiscoveryNode newNode(int nodeId, Map attributes, Set roles) { - return new DiscoveryNode("name_" + nodeId, "node_" + nodeId, DummyTransportAddress.INSTANCE, attributes, roles, Version.CURRENT); + return new DiscoveryNode("name_" + nodeId, "node_" + nodeId, LocalTransportAddress.buildUnique(), attributes, roles, Version.CURRENT); } private enum NodeSelector { diff --git a/core/src/test/java/org/elasticsearch/cluster/routing/allocation/NodeVersionAllocationDeciderTests.java b/core/src/test/java/org/elasticsearch/cluster/routing/allocation/NodeVersionAllocationDeciderTests.java index 97a3003ab2fd6..351139be34931 100644 --- a/core/src/test/java/org/elasticsearch/cluster/routing/allocation/NodeVersionAllocationDeciderTests.java +++ b/core/src/test/java/org/elasticsearch/cluster/routing/allocation/NodeVersionAllocationDeciderTests.java @@ -46,7 +46,7 @@ import org.elasticsearch.common.logging.ESLogger; import org.elasticsearch.common.logging.Loggers; import org.elasticsearch.common.settings.Settings; -import org.elasticsearch.common.transport.DummyTransportAddress; +import org.elasticsearch.common.transport.LocalTransportAddress; import org.elasticsearch.index.shard.ShardId; import org.elasticsearch.test.ESAllocationTestCase; import org.elasticsearch.test.VersionUtils; @@ -305,11 +305,11 @@ public void testRollingRestart() { public void testRebalanceDoesNotAllocatePrimaryAndReplicasOnDifferentVersionNodes() { ShardId shard1 = new ShardId("test1", "_na_", 0); ShardId shard2 = new ShardId("test2", "_na_", 0); - final DiscoveryNode newNode = new DiscoveryNode("newNode", DummyTransportAddress.INSTANCE, emptyMap(), + final DiscoveryNode newNode = new DiscoveryNode("newNode", LocalTransportAddress.buildUnique(), emptyMap(), MASTER_DATA_ROLES, Version.CURRENT); - final DiscoveryNode oldNode1 = new DiscoveryNode("oldNode1", DummyTransportAddress.INSTANCE, emptyMap(), + final DiscoveryNode oldNode1 = new DiscoveryNode("oldNode1", LocalTransportAddress.buildUnique(), emptyMap(), MASTER_DATA_ROLES, VersionUtils.getPreviousVersion()); - final DiscoveryNode oldNode2 = new DiscoveryNode("oldNode2", DummyTransportAddress.INSTANCE, emptyMap(), + final DiscoveryNode oldNode2 = new DiscoveryNode("oldNode2", LocalTransportAddress.buildUnique(), emptyMap(), MASTER_DATA_ROLES, VersionUtils.getPreviousVersion()); MetaData metaData = MetaData.builder() .put(IndexMetaData.builder(shard1.getIndexName()).settings(settings(Version.CURRENT).put(Settings.EMPTY)).numberOfShards(1).numberOfReplicas(1)) @@ -345,11 +345,11 @@ public void testRebalanceDoesNotAllocatePrimaryAndReplicasOnDifferentVersionNode } public void testRestoreDoesNotAllocateSnapshotOnOlderNodes() { - final DiscoveryNode newNode = new DiscoveryNode("newNode", DummyTransportAddress.INSTANCE, emptyMap(), + final DiscoveryNode newNode = new DiscoveryNode("newNode", LocalTransportAddress.buildUnique(), emptyMap(), MASTER_DATA_ROLES, Version.CURRENT); - final DiscoveryNode oldNode1 = new DiscoveryNode("oldNode1", DummyTransportAddress.INSTANCE, emptyMap(), + final DiscoveryNode oldNode1 = new DiscoveryNode("oldNode1", LocalTransportAddress.buildUnique(), emptyMap(), MASTER_DATA_ROLES, VersionUtils.getPreviousVersion()); - final DiscoveryNode oldNode2 = new DiscoveryNode("oldNode2", DummyTransportAddress.INSTANCE, emptyMap(), + final DiscoveryNode oldNode2 = new DiscoveryNode("oldNode2", LocalTransportAddress.buildUnique(), emptyMap(), MASTER_DATA_ROLES, VersionUtils.getPreviousVersion()); int numberOfShards = randomIntBetween(1, 3); diff --git a/core/src/test/java/org/elasticsearch/cluster/routing/allocation/SameShardRoutingTests.java b/core/src/test/java/org/elasticsearch/cluster/routing/allocation/SameShardRoutingTests.java index 9984890e8a4be..5eaf915736bf2 100644 --- a/core/src/test/java/org/elasticsearch/cluster/routing/allocation/SameShardRoutingTests.java +++ b/core/src/test/java/org/elasticsearch/cluster/routing/allocation/SameShardRoutingTests.java @@ -32,7 +32,7 @@ import org.elasticsearch.common.logging.ESLogger; import org.elasticsearch.common.logging.Loggers; import org.elasticsearch.common.settings.Settings; -import org.elasticsearch.common.transport.DummyTransportAddress; +import org.elasticsearch.common.transport.LocalTransportAddress; import org.elasticsearch.test.ESAllocationTestCase; import static java.util.Collections.emptyMap; @@ -62,9 +62,9 @@ public void testSameHost() { logger.info("--> adding two nodes with the same host"); clusterState = ClusterState.builder(clusterState).nodes( DiscoveryNodes.builder() - .put(new DiscoveryNode("node1", "node1", "test1", "test1", DummyTransportAddress.INSTANCE, emptyMap(), + .put(new DiscoveryNode("node1", "node1", "test1", "test1", LocalTransportAddress.buildUnique(), emptyMap(), MASTER_DATA_ROLES, Version.CURRENT)) - .put(new DiscoveryNode("node2", "node2", "test1", "test1", DummyTransportAddress.INSTANCE, emptyMap(), + .put(new DiscoveryNode("node2", "node2", "test1", "test1", LocalTransportAddress.buildUnique(), emptyMap(), MASTER_DATA_ROLES, Version.CURRENT))).build(); routingTable = strategy.reroute(clusterState, "reroute").routingTable(); clusterState = ClusterState.builder(clusterState).routingTable(routingTable).build(); @@ -81,7 +81,7 @@ public void testSameHost() { logger.info("--> add another node, with a different host, replicas will be allocating"); clusterState = ClusterState.builder(clusterState).nodes(DiscoveryNodes.builder(clusterState.nodes()) - .put(new DiscoveryNode("node3", "node3", "test2", "test2", DummyTransportAddress.INSTANCE, emptyMap(), + .put(new DiscoveryNode("node3", "node3", "test2", "test2", LocalTransportAddress.buildUnique(), emptyMap(), MASTER_DATA_ROLES, Version.CURRENT))).build(); routingTable = strategy.reroute(clusterState, "reroute").routingTable(); clusterState = ClusterState.builder(clusterState).routingTable(routingTable).build(); diff --git a/core/src/test/java/org/elasticsearch/cluster/routing/allocation/decider/DiskThresholdDeciderUnitTests.java b/core/src/test/java/org/elasticsearch/cluster/routing/allocation/decider/DiskThresholdDeciderUnitTests.java index 15b1463e4d0c4..0d1e74e71307c 100644 --- a/core/src/test/java/org/elasticsearch/cluster/routing/allocation/decider/DiskThresholdDeciderUnitTests.java +++ b/core/src/test/java/org/elasticsearch/cluster/routing/allocation/decider/DiskThresholdDeciderUnitTests.java @@ -39,7 +39,6 @@ import org.elasticsearch.common.collect.ImmutableOpenMap; import org.elasticsearch.common.settings.ClusterSettings; import org.elasticsearch.common.settings.Settings; -import org.elasticsearch.common.transport.DummyTransportAddress; import org.elasticsearch.common.transport.LocalTransportAddress; import org.elasticsearch.common.unit.ByteSizeValue; import org.elasticsearch.index.Index; @@ -107,9 +106,9 @@ public void testCanAllocateUsesMaxAvailableSpace() { final Index index = metaData.index("test").getIndex(); ShardRouting test_0 = ShardRouting.newUnassigned(index, 0, null, true, new UnassignedInfo(UnassignedInfo.Reason.INDEX_CREATED, "foo")); - DiscoveryNode node_0 = new DiscoveryNode("node_0", DummyTransportAddress.INSTANCE, Collections.emptyMap(), + DiscoveryNode node_0 = new DiscoveryNode("node_0", LocalTransportAddress.buildUnique(), Collections.emptyMap(), new HashSet<>(Arrays.asList(DiscoveryNode.Role.values())), Version.CURRENT); - DiscoveryNode node_1 = new DiscoveryNode("node_1", DummyTransportAddress.INSTANCE, Collections.emptyMap(), + DiscoveryNode node_1 = new DiscoveryNode("node_1", LocalTransportAddress.buildUnique(), Collections.emptyMap(), new HashSet<>(Arrays.asList(DiscoveryNode.Role.values())), Version.CURRENT); RoutingTable routingTable = RoutingTable.builder() @@ -146,9 +145,9 @@ public void testCanRemainUsesLeastAvailableSpace() { DiskThresholdDecider decider = new DiskThresholdDecider(Settings.EMPTY, nss, cis, null); ImmutableOpenMap.Builder shardRoutingMap = ImmutableOpenMap.builder(); - DiscoveryNode node_0 = new DiscoveryNode("node_0", DummyTransportAddress.INSTANCE, Collections.emptyMap(), + DiscoveryNode node_0 = new DiscoveryNode("node_0", LocalTransportAddress.buildUnique(), Collections.emptyMap(), new HashSet<>(Arrays.asList(DiscoveryNode.Role.values())), Version.CURRENT); - DiscoveryNode node_1 = new DiscoveryNode("node_1", DummyTransportAddress.INSTANCE, Collections.emptyMap(), + DiscoveryNode node_1 = new DiscoveryNode("node_1", LocalTransportAddress.buildUnique(), Collections.emptyMap(), new HashSet<>(Arrays.asList(DiscoveryNode.Role.values())), Version.CURRENT); MetaData metaData = MetaData.builder() diff --git a/core/src/test/java/org/elasticsearch/cluster/serialization/ClusterStateToStringTests.java b/core/src/test/java/org/elasticsearch/cluster/serialization/ClusterStateToStringTests.java index 3a30bd5c6aa49..ae09d2311bbf3 100644 --- a/core/src/test/java/org/elasticsearch/cluster/serialization/ClusterStateToStringTests.java +++ b/core/src/test/java/org/elasticsearch/cluster/serialization/ClusterStateToStringTests.java @@ -27,7 +27,7 @@ import org.elasticsearch.cluster.node.DiscoveryNodes; import org.elasticsearch.cluster.routing.RoutingTable; import org.elasticsearch.cluster.routing.allocation.AllocationService; -import org.elasticsearch.common.transport.DummyTransportAddress; +import org.elasticsearch.common.transport.LocalTransportAddress; import org.elasticsearch.test.ESAllocationTestCase; import static java.util.Collections.emptyMap; @@ -48,7 +48,7 @@ public void testClusterStateSerialization() throws Exception { .addAsNew(metaData.index("test_idx")) .build(); - DiscoveryNodes nodes = DiscoveryNodes.builder().put(new DiscoveryNode("node_foo", DummyTransportAddress.INSTANCE, + DiscoveryNodes nodes = DiscoveryNodes.builder().put(new DiscoveryNode("node_foo", LocalTransportAddress.buildUnique(), emptyMap(), emptySet(), Version.CURRENT)).localNodeId("node_foo").masterNodeId("node_foo").build(); ClusterState clusterState = ClusterState.builder(org.elasticsearch.cluster.ClusterName.DEFAULT).nodes(nodes).metaData(metaData) diff --git a/core/src/test/java/org/elasticsearch/cluster/service/ClusterServiceTests.java b/core/src/test/java/org/elasticsearch/cluster/service/ClusterServiceTests.java index 7303d5a67df70..c373d659612ce 100644 --- a/core/src/test/java/org/elasticsearch/cluster/service/ClusterServiceTests.java +++ b/core/src/test/java/org/elasticsearch/cluster/service/ClusterServiceTests.java @@ -37,7 +37,7 @@ import org.elasticsearch.common.collect.Tuple; import org.elasticsearch.common.settings.ClusterSettings; import org.elasticsearch.common.settings.Settings; -import org.elasticsearch.common.transport.DummyTransportAddress; +import org.elasticsearch.common.transport.LocalTransportAddress; import org.elasticsearch.common.unit.TimeValue; import org.elasticsearch.test.ESTestCase; import org.elasticsearch.test.MockLogAppender; @@ -103,7 +103,7 @@ TimedClusterService createTimedClusterService(boolean makeMaster) throws Interru TimedClusterService timedClusterService = new TimedClusterService(Settings.EMPTY, null, new ClusterSettings(Settings.EMPTY, ClusterSettings.BUILT_IN_CLUSTER_SETTINGS), threadPool, new ClusterName("ClusterServiceTests")); - timedClusterService.setLocalNode(new DiscoveryNode("node1", DummyTransportAddress.INSTANCE, emptyMap(), + timedClusterService.setLocalNode(new DiscoveryNode("node1", LocalTransportAddress.buildUnique(), emptyMap(), emptySet(), Version.CURRENT)); timedClusterService.setNodeConnectionsService(new NodeConnectionsService(Settings.EMPTY, null, null) { @Override diff --git a/core/src/test/java/org/elasticsearch/cluster/service/ClusterServiceUtils.java b/core/src/test/java/org/elasticsearch/cluster/service/ClusterServiceUtils.java index e3f132b18fbcf..4d328f4065743 100644 --- a/core/src/test/java/org/elasticsearch/cluster/service/ClusterServiceUtils.java +++ b/core/src/test/java/org/elasticsearch/cluster/service/ClusterServiceUtils.java @@ -29,7 +29,7 @@ import org.elasticsearch.cluster.node.DiscoveryNodes; import org.elasticsearch.common.settings.ClusterSettings; import org.elasticsearch.common.settings.Settings; -import org.elasticsearch.common.transport.DummyTransportAddress; +import org.elasticsearch.common.transport.LocalTransportAddress; import org.elasticsearch.threadpool.ThreadPool; import java.util.Arrays; @@ -45,7 +45,7 @@ public static ClusterService createClusterService(ThreadPool threadPool) { ClusterService clusterService = new ClusterService(Settings.EMPTY, null, new ClusterSettings(Settings.EMPTY, ClusterSettings.BUILT_IN_CLUSTER_SETTINGS), threadPool, new ClusterName("ClusterServiceTests")); - clusterService.setLocalNode(new DiscoveryNode("node", DummyTransportAddress.INSTANCE, Collections.emptyMap(), + clusterService.setLocalNode(new DiscoveryNode("node", LocalTransportAddress.buildUnique(), Collections.emptyMap(), new HashSet<>(Arrays.asList(DiscoveryNode.Role.values())),Version.CURRENT)); clusterService.setNodeConnectionsService(new NodeConnectionsService(Settings.EMPTY, null, null) { @Override diff --git a/core/src/test/java/org/elasticsearch/discovery/BlockingClusterStatePublishResponseHandlerTests.java b/core/src/test/java/org/elasticsearch/discovery/BlockingClusterStatePublishResponseHandlerTests.java index f6aac190c4bf1..89b4576b03298 100644 --- a/core/src/test/java/org/elasticsearch/discovery/BlockingClusterStatePublishResponseHandlerTests.java +++ b/core/src/test/java/org/elasticsearch/discovery/BlockingClusterStatePublishResponseHandlerTests.java @@ -21,7 +21,7 @@ import org.elasticsearch.Version; import org.elasticsearch.cluster.node.DiscoveryNode; import org.elasticsearch.common.logging.ESLogger; -import org.elasticsearch.common.transport.DummyTransportAddress; +import org.elasticsearch.common.transport.LocalTransportAddress; import org.elasticsearch.common.unit.TimeValue; import org.elasticsearch.common.util.concurrent.AbstractRunnable; import org.elasticsearch.test.ESTestCase; @@ -77,7 +77,7 @@ public void testConcurrentAccess() throws InterruptedException { int nodeCount = scaledRandomIntBetween(10, 20); DiscoveryNode[] allNodes = new DiscoveryNode[nodeCount]; for (int i = 0; i < nodeCount; i++) { - DiscoveryNode node = new DiscoveryNode("node_" + i, DummyTransportAddress.INSTANCE, emptyMap(), emptySet(), Version.CURRENT); + DiscoveryNode node = new DiscoveryNode("node_" + i, LocalTransportAddress.buildUnique(), emptyMap(), emptySet(), Version.CURRENT); allNodes[i] = node; } diff --git a/core/src/test/java/org/elasticsearch/discovery/zen/ElectMasterServiceTests.java b/core/src/test/java/org/elasticsearch/discovery/zen/ElectMasterServiceTests.java index 498aff28ee6be..b7b7744d086b0 100644 --- a/core/src/test/java/org/elasticsearch/discovery/zen/ElectMasterServiceTests.java +++ b/core/src/test/java/org/elasticsearch/discovery/zen/ElectMasterServiceTests.java @@ -22,7 +22,7 @@ import org.elasticsearch.Version; import org.elasticsearch.cluster.node.DiscoveryNode; import org.elasticsearch.common.settings.Settings; -import org.elasticsearch.common.transport.DummyTransportAddress; +import org.elasticsearch.common.transport.LocalTransportAddress; import org.elasticsearch.discovery.zen.elect.ElectMasterService; import org.elasticsearch.test.ESTestCase; @@ -49,7 +49,7 @@ List generateRandomNodes() { if (randomBoolean()) { roles.add(DiscoveryNode.Role.MASTER); } - DiscoveryNode node = new DiscoveryNode("n_" + i, "n_" + i, DummyTransportAddress.INSTANCE, Collections.emptyMap(), + DiscoveryNode node = new DiscoveryNode("n_" + i, "n_" + i, LocalTransportAddress.buildUnique(), Collections.emptyMap(), roles, Version.CURRENT); nodes.add(node); } diff --git a/core/src/test/java/org/elasticsearch/discovery/zen/NodeJoinControllerTests.java b/core/src/test/java/org/elasticsearch/discovery/zen/NodeJoinControllerTests.java index 3a81b905eb577..6959003edebc8 100644 --- a/core/src/test/java/org/elasticsearch/discovery/zen/NodeJoinControllerTests.java +++ b/core/src/test/java/org/elasticsearch/discovery/zen/NodeJoinControllerTests.java @@ -33,7 +33,6 @@ import org.elasticsearch.cluster.service.ClusterService; import org.elasticsearch.common.settings.ClusterSettings; import org.elasticsearch.common.settings.Settings; -import org.elasticsearch.common.transport.DummyTransportAddress; import org.elasticsearch.common.transport.LocalTransportAddress; import org.elasticsearch.common.unit.TimeValue; import org.elasticsearch.common.util.concurrent.AbstractRunnable; @@ -67,6 +66,7 @@ import static java.util.Collections.shuffle; import static org.elasticsearch.cluster.service.ClusterServiceUtils.createClusterService; import static org.elasticsearch.cluster.service.ClusterServiceUtils.setState; +import static org.hamcrest.Matchers.containsString; import static org.hamcrest.Matchers.equalTo; import static org.hamcrest.Matchers.instanceOf; @@ -403,7 +403,7 @@ public void onFailure(Throwable t) { public void testNewClusterStateOnExistingNodeJoin() throws InterruptedException, ExecutionException { ClusterState state = clusterService.state(); final DiscoveryNodes.Builder nodesBuilder = DiscoveryNodes.builder(state.nodes()); - final DiscoveryNode other_node = new DiscoveryNode("other_node", DummyTransportAddress.INSTANCE, + final DiscoveryNode other_node = new DiscoveryNode("other_node", LocalTransportAddress.buildUnique(), emptyMap(), emptySet(), Version.CURRENT); nodesBuilder.put(other_node); setState(clusterService, ClusterState.builder(state).nodes(nodesBuilder)); @@ -413,6 +413,15 @@ public void testNewClusterStateOnExistingNodeJoin() throws InterruptedException, assertTrue("failed to publish a new state upon existing join", clusterService.state() != state); } + public void testRejectingJoinWithSameAddress() throws InterruptedException, ExecutionException { + ClusterState state = clusterService.state(); + final DiscoveryNode other_node = new DiscoveryNode("other_node", state.nodes().getLocalNode().getAddress(), + emptyMap(), emptySet(), Version.CURRENT); + + ExecutionException e = expectThrows(ExecutionException.class, () -> joinNode(other_node)); + assertThat(e.getMessage(), containsString("found existing node")); + } + public void testNormalConcurrentJoins() throws InterruptedException { Thread[] threads = new Thread[3 + randomInt(5)]; ArrayList nodes = new ArrayList<>(); diff --git a/core/src/test/java/org/elasticsearch/discovery/zen/ZenDiscoveryUnitTests.java b/core/src/test/java/org/elasticsearch/discovery/zen/ZenDiscoveryUnitTests.java index a6638eb19cf02..9db83f48f0ea7 100644 --- a/core/src/test/java/org/elasticsearch/discovery/zen/ZenDiscoveryUnitTests.java +++ b/core/src/test/java/org/elasticsearch/discovery/zen/ZenDiscoveryUnitTests.java @@ -24,8 +24,7 @@ import org.elasticsearch.cluster.ClusterState; import org.elasticsearch.cluster.node.DiscoveryNode; import org.elasticsearch.cluster.node.DiscoveryNodes; -import org.elasticsearch.common.Strings; -import org.elasticsearch.common.transport.DummyTransportAddress; +import org.elasticsearch.common.transport.LocalTransportAddress; import org.elasticsearch.discovery.zen.ping.ZenPing; import org.elasticsearch.test.ESTestCase; @@ -51,9 +50,9 @@ public void testShouldIgnoreNewClusterState() { ClusterName clusterName = new ClusterName("abc"); DiscoveryNodes.Builder currentNodes = DiscoveryNodes.builder(); - currentNodes.masterNodeId("a").put(new DiscoveryNode("a", DummyTransportAddress.INSTANCE, emptyMap(), emptySet(), Version.CURRENT)); + currentNodes.masterNodeId("a").put(new DiscoveryNode("a", LocalTransportAddress.buildUnique(), emptyMap(), emptySet(), Version.CURRENT)); DiscoveryNodes.Builder newNodes = DiscoveryNodes.builder(); - newNodes.masterNodeId("a").put(new DiscoveryNode("a", DummyTransportAddress.INSTANCE, emptyMap(), emptySet(), Version.CURRENT)); + newNodes.masterNodeId("a").put(new DiscoveryNode("a", LocalTransportAddress.buildUnique(), emptyMap(), emptySet(), Version.CURRENT)); ClusterState.Builder currentState = ClusterState.builder(clusterName); currentState.nodes(currentNodes); @@ -71,7 +70,7 @@ public void testShouldIgnoreNewClusterState() { assertFalse("should not ignore, because new state's version is higher to current state's version", shouldIgnoreOrRejectNewClusterState(logger, currentState.build(), newState.build())); currentNodes = DiscoveryNodes.builder(); - currentNodes.masterNodeId("b").put(new DiscoveryNode("b", DummyTransportAddress.INSTANCE, emptyMap(), emptySet(), Version.CURRENT)); + currentNodes.masterNodeId("b").put(new DiscoveryNode("b", LocalTransportAddress.buildUnique(), emptyMap(), emptySet(), Version.CURRENT)); ; // version isn't taken into account, so randomize it to ensure this. if (randomBoolean()) { @@ -109,7 +108,7 @@ public void testFilterNonMasterPingResponse() { ArrayList allNodes = new ArrayList<>(); for (int i = randomIntBetween(10, 20); i >= 0; i--) { Set roles = new HashSet<>(randomSubsetOf(Arrays.asList(DiscoveryNode.Role.values()))); - DiscoveryNode node = new DiscoveryNode("node_" + i, "id_" + i, DummyTransportAddress.INSTANCE, Collections.emptyMap(), + DiscoveryNode node = new DiscoveryNode("node_" + i, "id_" + i, LocalTransportAddress.buildUnique(), Collections.emptyMap(), roles, Version.CURRENT); responses.add(new ZenPing.PingResponse(node, randomBoolean() ? null : node, new ClusterName("test"), randomBoolean())); allNodes.add(node); diff --git a/core/src/test/java/org/elasticsearch/discovery/zen/ZenPingTests.java b/core/src/test/java/org/elasticsearch/discovery/zen/ZenPingTests.java index 90b31ff334278..f4266eebcf439 100644 --- a/core/src/test/java/org/elasticsearch/discovery/zen/ZenPingTests.java +++ b/core/src/test/java/org/elasticsearch/discovery/zen/ZenPingTests.java @@ -22,7 +22,7 @@ import org.elasticsearch.Version; import org.elasticsearch.cluster.ClusterName; import org.elasticsearch.cluster.node.DiscoveryNode; -import org.elasticsearch.common.transport.DummyTransportAddress; +import org.elasticsearch.common.transport.LocalTransportAddress; import org.elasticsearch.discovery.zen.ping.ZenPing; import org.elasticsearch.test.ESTestCase; @@ -41,7 +41,7 @@ public void testPingCollection() { boolean hasJoinedOncePerNode[] = new boolean[nodes.length]; ArrayList pings = new ArrayList<>(); for (int i = 0; i < nodes.length; i++) { - nodes[i] = new DiscoveryNode("" + i, DummyTransportAddress.INSTANCE, emptyMap(), emptySet(), Version.CURRENT); + nodes[i] = new DiscoveryNode("" + i, LocalTransportAddress.buildUnique(), emptyMap(), emptySet(), Version.CURRENT); } for (int pingCount = scaledRandomIntBetween(10, nodes.length * 10); pingCount > 0; pingCount--) { diff --git a/core/src/test/java/org/elasticsearch/discovery/zen/publish/PendingClusterStatesQueueTests.java b/core/src/test/java/org/elasticsearch/discovery/zen/publish/PendingClusterStatesQueueTests.java index 9c90e8de90074..4d46582bba449 100644 --- a/core/src/test/java/org/elasticsearch/discovery/zen/publish/PendingClusterStatesQueueTests.java +++ b/core/src/test/java/org/elasticsearch/discovery/zen/publish/PendingClusterStatesQueueTests.java @@ -24,7 +24,7 @@ import org.elasticsearch.cluster.ClusterState; import org.elasticsearch.cluster.node.DiscoveryNode; import org.elasticsearch.cluster.node.DiscoveryNodes; -import org.elasticsearch.common.transport.DummyTransportAddress; +import org.elasticsearch.common.transport.LocalTransportAddress; import org.elasticsearch.discovery.zen.publish.PendingClusterStatesQueue.ClusterStateContext; import org.elasticsearch.test.ESTestCase; @@ -236,7 +236,7 @@ List randomStates(int count, String... masters) { ClusterState state = lastClusterStatePerMaster[masterIndex]; if (state == null) { state = ClusterState.builder(ClusterName.DEFAULT).nodes(DiscoveryNodes.builder() - .put(new DiscoveryNode(masters[masterIndex], DummyTransportAddress.INSTANCE, + .put(new DiscoveryNode(masters[masterIndex], LocalTransportAddress.buildUnique(), emptyMap(), emptySet(),Version.CURRENT)).masterNodeId(masters[masterIndex]).build() ).build(); } else { diff --git a/core/src/test/java/org/elasticsearch/gateway/AsyncShardFetchTests.java b/core/src/test/java/org/elasticsearch/gateway/AsyncShardFetchTests.java index 3ab15baf2e03a..2b8cdf46f7d34 100644 --- a/core/src/test/java/org/elasticsearch/gateway/AsyncShardFetchTests.java +++ b/core/src/test/java/org/elasticsearch/gateway/AsyncShardFetchTests.java @@ -24,7 +24,7 @@ import org.elasticsearch.cluster.node.DiscoveryNode; import org.elasticsearch.cluster.node.DiscoveryNodes; import org.elasticsearch.common.logging.Loggers; -import org.elasticsearch.common.transport.DummyTransportAddress; +import org.elasticsearch.common.transport.LocalTransportAddress; import org.elasticsearch.index.shard.ShardId; import org.elasticsearch.test.ESTestCase; import org.elasticsearch.threadpool.ThreadPool; @@ -44,11 +44,11 @@ /** */ public class AsyncShardFetchTests extends ESTestCase { - private final DiscoveryNode node1 = new DiscoveryNode("node1", DummyTransportAddress.INSTANCE, Collections.emptyMap(), + private final DiscoveryNode node1 = new DiscoveryNode("node1", LocalTransportAddress.buildUnique(), Collections.emptyMap(), Collections.singleton(DiscoveryNode.Role.DATA), Version.CURRENT); private final Response response1 = new Response(node1); private final Throwable failure1 = new Throwable("simulated failure 1"); - private final DiscoveryNode node2 = new DiscoveryNode("node2", DummyTransportAddress.INSTANCE, Collections.emptyMap(), + private final DiscoveryNode node2 = new DiscoveryNode("node2", LocalTransportAddress.buildUnique(), Collections.emptyMap(), Collections.singleton(DiscoveryNode.Role.DATA), Version.CURRENT); private final Response response2 = new Response(node2); private final Throwable failure2 = new Throwable("simulate failure 2"); diff --git a/core/src/test/java/org/elasticsearch/index/shard/IndexShardTests.java b/core/src/test/java/org/elasticsearch/index/shard/IndexShardTests.java index 9bb0302e0b7e3..ae219433228d3 100644 --- a/core/src/test/java/org/elasticsearch/index/shard/IndexShardTests.java +++ b/core/src/test/java/org/elasticsearch/index/shard/IndexShardTests.java @@ -64,7 +64,7 @@ import org.elasticsearch.common.lease.Releasables; import org.elasticsearch.common.logging.ESLogger; import org.elasticsearch.common.settings.Settings; -import org.elasticsearch.common.transport.DummyTransportAddress; +import org.elasticsearch.common.transport.LocalTransportAddress; import org.elasticsearch.common.unit.ByteSizeUnit; import org.elasticsearch.common.unit.ByteSizeValue; import org.elasticsearch.common.xcontent.XContentBuilder; @@ -973,7 +973,7 @@ public void testRecoverFromStore() throws IOException { ShardRoutingHelper.reinit(routing); IndexShard newShard = test.createShard(routing); newShard.updateRoutingEntry(routing, false); - DiscoveryNode localNode = new DiscoveryNode("foo", DummyTransportAddress.INSTANCE, emptyMap(), emptySet(), Version.CURRENT); + DiscoveryNode localNode = new DiscoveryNode("foo", LocalTransportAddress.buildUnique(), emptyMap(), emptySet(), Version.CURRENT); newShard.markAsRecovering("store", new RecoveryState(newShard.shardId(), routing.primary(), RecoveryState.Type.STORE, localNode, localNode)); assertTrue(newShard.recoverFromStore(localNode)); assertEquals(translogOps, newShard.recoveryState().getTranslog().recoveredOperations()); @@ -1002,7 +1002,7 @@ public void testRecoverFromCleanStore() throws IOException { ShardRoutingHelper.reinit(routing, UnassignedInfo.Reason.INDEX_CREATED); IndexShard newShard = test.createShard(routing); newShard.updateRoutingEntry(routing, false); - DiscoveryNode localNode = new DiscoveryNode("foo", DummyTransportAddress.INSTANCE, emptyMap(), emptySet(), Version.CURRENT); + DiscoveryNode localNode = new DiscoveryNode("foo", LocalTransportAddress.buildUnique(), emptyMap(), emptySet(), Version.CURRENT); newShard.markAsRecovering("store", new RecoveryState(newShard.shardId(), routing.primary(), RecoveryState.Type.STORE, localNode, localNode)); assertTrue(newShard.recoverFromStore(localNode)); @@ -1021,7 +1021,7 @@ public void testFailIfIndexNotPresentInRecoverFromStore() throws Exception { createIndex("test"); ensureGreen(); IndicesService indicesService = getInstanceFromNode(IndicesService.class); - DiscoveryNode localNode = new DiscoveryNode("foo", DummyTransportAddress.INSTANCE, emptyMap(), emptySet(), Version.CURRENT); + DiscoveryNode localNode = new DiscoveryNode("foo", LocalTransportAddress.buildUnique(), emptyMap(), emptySet(), Version.CURRENT); IndexService test = indicesService.indexService(resolveIndex("test")); final IndexShard shard = test.getShardOrNull(0); @@ -1121,7 +1121,7 @@ public void testRestoreShard() throws IOException { Store targetStore = test_target_shard.store(); test_target_shard.updateRoutingEntry(routing, false); - DiscoveryNode localNode = new DiscoveryNode("foo", DummyTransportAddress.INSTANCE, emptyMap(), emptySet(), Version.CURRENT); + DiscoveryNode localNode = new DiscoveryNode("foo", LocalTransportAddress.buildUnique(), emptyMap(), emptySet(), Version.CURRENT); test_target_shard.markAsRecovering("store", new RecoveryState(routing.shardId(), routing.primary(), RecoveryState.Type.SNAPSHOT, routing.restoreSource(), localNode)); assertTrue(test_target_shard.restoreFromRepository(new IndexShardRepository() { @Override @@ -1410,7 +1410,7 @@ public static final IndexShard recoverShard(IndexShard newShard, ShardRouting o ShardRouting routing = new ShardRouting(oldRouting); ShardRoutingHelper.reinit(routing); newShard.updateRoutingEntry(routing, false); - DiscoveryNode localNode = new DiscoveryNode("foo", DummyTransportAddress.INSTANCE, emptyMap(), emptySet(), Version.CURRENT); + DiscoveryNode localNode = new DiscoveryNode("foo", LocalTransportAddress.buildUnique(), emptyMap(), emptySet(), Version.CURRENT); newShard.markAsRecovering("store", new RecoveryState(newShard.shardId(), routing.primary(), RecoveryState.Type.STORE, localNode, localNode)); assertTrue(newShard.recoverFromStore(localNode)); routing = new ShardRouting(routing); @@ -1444,7 +1444,7 @@ public void testTranslogRecoverySyncsTranslog() throws IOException { ShardRouting routing = new ShardRouting(shard.routingEntry()); test.removeShard(0, "b/c britta says so"); IndexShard newShard = test.createShard(routing); - DiscoveryNode localNode = new DiscoveryNode("foo", DummyTransportAddress.INSTANCE, emptyMap(), emptySet(), Version.CURRENT); + DiscoveryNode localNode = new DiscoveryNode("foo", LocalTransportAddress.buildUnique(), emptyMap(), emptySet(), Version.CURRENT); newShard.markAsRecovering("for testing", new RecoveryState(newShard.shardId(), routing.primary(), RecoveryState.Type.REPLICA, localNode, localNode)); List operations = new ArrayList<>(); operations.add(new Translog.Index("testtype", "1", jsonBuilder().startObject().field("foo", "bar").endObject().bytes().toBytes())); @@ -1472,7 +1472,7 @@ public void testIndexingBufferDuringInternalRecovery() throws IOException { test.removeShard(0, "b/c britta says so"); IndexShard newShard = test.createShard(routing); newShard.shardRouting = routing; - DiscoveryNode localNode = new DiscoveryNode("foo", DummyTransportAddress.INSTANCE, emptyMap(), emptySet(), Version.CURRENT); + DiscoveryNode localNode = new DiscoveryNode("foo", LocalTransportAddress.buildUnique(), emptyMap(), emptySet(), Version.CURRENT); newShard.markAsRecovering("for testing", new RecoveryState(newShard.shardId(), routing.primary(), RecoveryState.Type.REPLICA, localNode, localNode)); // Shard is still inactive since we haven't started recovering yet assertFalse(newShard.isActive()); @@ -1501,7 +1501,7 @@ public void testIndexingBufferDuringPeerRecovery() throws IOException { test.removeShard(0, "b/c britta says so"); IndexShard newShard = test.createShard(routing); newShard.shardRouting = routing; - DiscoveryNode localNode = new DiscoveryNode("foo", DummyTransportAddress.INSTANCE, emptyMap(), emptySet(), Version.CURRENT); + DiscoveryNode localNode = new DiscoveryNode("foo", LocalTransportAddress.buildUnique(), emptyMap(), emptySet(), Version.CURRENT); newShard.markAsRecovering("for testing", new RecoveryState(newShard.shardId(), routing.primary(), RecoveryState.Type.REPLICA, localNode, localNode)); // Shard is still inactive since we haven't started recovering yet assertFalse(newShard.isActive()); diff --git a/core/src/test/java/org/elasticsearch/indices/IndexingMemoryControllerTests.java b/core/src/test/java/org/elasticsearch/indices/IndexingMemoryControllerTests.java index 759288f3dc40f..a01eb6cae5fb8 100644 --- a/core/src/test/java/org/elasticsearch/indices/IndexingMemoryControllerTests.java +++ b/core/src/test/java/org/elasticsearch/indices/IndexingMemoryControllerTests.java @@ -18,25 +18,20 @@ */ package org.elasticsearch.indices; -import org.apache.lucene.index.DirectoryReader; -import org.elasticsearch.ExceptionsHelper; import org.elasticsearch.Version; import org.elasticsearch.action.admin.indices.forcemerge.ForceMergeResponse; -import org.elasticsearch.action.index.IndexRequestBuilder; import org.elasticsearch.cluster.node.DiscoveryNode; import org.elasticsearch.cluster.routing.ShardRouting; import org.elasticsearch.cluster.routing.ShardRoutingHelper; import org.elasticsearch.common.settings.Settings; -import org.elasticsearch.common.transport.DummyTransportAddress; +import org.elasticsearch.common.transport.LocalTransportAddress; import org.elasticsearch.common.unit.ByteSizeUnit; import org.elasticsearch.common.unit.ByteSizeValue; -import org.elasticsearch.common.unit.TimeValue; import org.elasticsearch.index.IndexService; import org.elasticsearch.index.engine.Engine; import org.elasticsearch.index.shard.IndexSearcherWrapper; import org.elasticsearch.index.shard.IndexShard; import org.elasticsearch.index.shard.IndexShardTests; -import org.elasticsearch.index.shard.IndexingOperationListener; import org.elasticsearch.indices.recovery.RecoveryState; import org.elasticsearch.test.ESSingleNodeTestCase; import org.elasticsearch.threadpool.ThreadPool; @@ -46,7 +41,6 @@ import java.util.Collections; import java.util.HashMap; import java.util.HashSet; -import java.util.Iterator; import java.util.List; import java.util.Map; import java.util.Set; @@ -457,7 +451,7 @@ protected void writeIndexingBufferAsync(IndexShard shard) { ShardRouting routing = new ShardRouting(shard.routingEntry()); ShardRoutingHelper.reinit(routing); newShard.updateRoutingEntry(routing, false); - DiscoveryNode localNode = new DiscoveryNode("foo", DummyTransportAddress.INSTANCE, emptyMap(), emptySet(), Version.CURRENT); + DiscoveryNode localNode = new DiscoveryNode("foo", LocalTransportAddress.buildUnique(), emptyMap(), emptySet(), Version.CURRENT); newShard.markAsRecovering("store", new RecoveryState(newShard.shardId(), routing.primary(), RecoveryState.Type.STORE, localNode, localNode)); assertEquals(1, imc.availableShards().size()); diff --git a/core/src/test/java/org/elasticsearch/indices/IndicesLifecycleListenerSingleNodeTests.java b/core/src/test/java/org/elasticsearch/indices/IndicesLifecycleListenerSingleNodeTests.java index 8763db6b28b05..dce7a684ea34b 100644 --- a/core/src/test/java/org/elasticsearch/indices/IndicesLifecycleListenerSingleNodeTests.java +++ b/core/src/test/java/org/elasticsearch/indices/IndicesLifecycleListenerSingleNodeTests.java @@ -25,7 +25,7 @@ import org.elasticsearch.cluster.routing.ShardRoutingHelper; import org.elasticsearch.cluster.routing.UnassignedInfo; import org.elasticsearch.common.settings.Settings; -import org.elasticsearch.common.transport.DummyTransportAddress; +import org.elasticsearch.common.transport.LocalTransportAddress; import org.elasticsearch.index.Index; import org.elasticsearch.index.IndexService; import org.elasticsearch.index.NodeServicesProvider; @@ -103,7 +103,7 @@ public void afterIndexShardDeleted(ShardId shardId, Settings indexSettings) { ShardRoutingHelper.initialize(newRouting, nodeId); IndexShard shard = index.createShard(newRouting); shard.updateRoutingEntry(newRouting, true); - final DiscoveryNode localNode = new DiscoveryNode("foo", DummyTransportAddress.INSTANCE, + final DiscoveryNode localNode = new DiscoveryNode("foo", LocalTransportAddress.buildUnique(), emptyMap(), emptySet(), Version.CURRENT); shard.markAsRecovering("store", new RecoveryState(shard.shardId(), newRouting.primary(), RecoveryState.Type.SNAPSHOT, newRouting.restoreSource(), localNode)); shard.recoverFromStore(localNode); diff --git a/core/src/test/java/org/elasticsearch/indices/recovery/RecoverySourceHandlerTests.java b/core/src/test/java/org/elasticsearch/indices/recovery/RecoverySourceHandlerTests.java index 0d6d512200617..2c52cd3301567 100644 --- a/core/src/test/java/org/elasticsearch/indices/recovery/RecoverySourceHandlerTests.java +++ b/core/src/test/java/org/elasticsearch/indices/recovery/RecoverySourceHandlerTests.java @@ -38,7 +38,7 @@ import org.elasticsearch.common.lucene.store.IndexOutputOutputStream; import org.elasticsearch.common.settings.ClusterSettings; import org.elasticsearch.common.settings.Settings; -import org.elasticsearch.common.transport.DummyTransportAddress; +import org.elasticsearch.common.transport.LocalTransportAddress; import org.elasticsearch.index.IndexSettings; import org.elasticsearch.index.shard.ShardId; import org.elasticsearch.index.store.DirectoryService; @@ -69,8 +69,8 @@ public void testSendFiles() throws Throwable { put("indices.recovery.concurrent_small_file_streams", 1).build(); final RecoverySettings recoverySettings = new RecoverySettings(settings, service); StartRecoveryRequest request = new StartRecoveryRequest(shardId, - new DiscoveryNode("b", DummyTransportAddress.INSTANCE, emptyMap(), emptySet(), Version.CURRENT), - new DiscoveryNode("b", DummyTransportAddress.INSTANCE, emptyMap(), emptySet(), Version.CURRENT), + new DiscoveryNode("b", LocalTransportAddress.buildUnique(), emptyMap(), emptySet(), Version.CURRENT), + new DiscoveryNode("b", LocalTransportAddress.buildUnique(), emptyMap(), emptySet(), Version.CURRENT), null, RecoveryState.Type.STORE, randomLong()); Store store = newStore(createTempDir()); RecoverySourceHandler handler = new RecoverySourceHandler(null, null, request, recoverySettings.getChunkSize().bytesAsInt(), @@ -119,8 +119,8 @@ public void testHandleCorruptedIndexOnSendSendFiles() throws Throwable { put("indices.recovery.concurrent_small_file_streams", 1).build(); final RecoverySettings recoverySettings = new RecoverySettings(settings, service); StartRecoveryRequest request = new StartRecoveryRequest(shardId, - new DiscoveryNode("b", DummyTransportAddress.INSTANCE, emptyMap(), emptySet(), Version.CURRENT), - new DiscoveryNode("b", DummyTransportAddress.INSTANCE, emptyMap(), emptySet(), Version.CURRENT), + new DiscoveryNode("b", LocalTransportAddress.buildUnique(), emptyMap(), emptySet(), Version.CURRENT), + new DiscoveryNode("b", LocalTransportAddress.buildUnique(), emptyMap(), emptySet(), Version.CURRENT), null, RecoveryState.Type.STORE, randomLong()); Path tempDir = createTempDir(); Store store = newStore(tempDir, false); @@ -182,8 +182,8 @@ public void testHandleExceptinoOnSendSendFiles() throws Throwable { put("indices.recovery.concurrent_small_file_streams", 1).build(); final RecoverySettings recoverySettings = new RecoverySettings(settings, service); StartRecoveryRequest request = new StartRecoveryRequest(shardId, - new DiscoveryNode("b", DummyTransportAddress.INSTANCE, emptyMap(), emptySet(), Version.CURRENT), - new DiscoveryNode("b", DummyTransportAddress.INSTANCE, emptyMap(), emptySet(), Version.CURRENT), + new DiscoveryNode("b", LocalTransportAddress.buildUnique(), emptyMap(), emptySet(), Version.CURRENT), + new DiscoveryNode("b", LocalTransportAddress.buildUnique(), emptyMap(), emptySet(), Version.CURRENT), null, RecoveryState.Type.STORE, randomLong()); Path tempDir = createTempDir(); Store store = newStore(tempDir, false); diff --git a/core/src/test/java/org/elasticsearch/indices/recovery/RecoveryTargetTests.java b/core/src/test/java/org/elasticsearch/indices/recovery/RecoveryTargetTests.java index 8552db2d37612..3bcedeb379e70 100644 --- a/core/src/test/java/org/elasticsearch/indices/recovery/RecoveryTargetTests.java +++ b/core/src/test/java/org/elasticsearch/indices/recovery/RecoveryTargetTests.java @@ -24,7 +24,7 @@ import org.elasticsearch.common.io.stream.BytesStreamOutput; import org.elasticsearch.common.io.stream.StreamInput; import org.elasticsearch.common.io.stream.Streamable; -import org.elasticsearch.common.transport.DummyTransportAddress; +import org.elasticsearch.common.transport.LocalTransportAddress; import org.elasticsearch.index.shard.ShardId; import org.elasticsearch.indices.recovery.RecoveryState.File; import org.elasticsearch.indices.recovery.RecoveryState.Index; @@ -339,7 +339,7 @@ Index createObj() { } public void testStageSequenceEnforcement() { - final DiscoveryNode discoveryNode = new DiscoveryNode("1", DummyTransportAddress.INSTANCE, emptyMap(), emptySet(), Version.CURRENT); + final DiscoveryNode discoveryNode = new DiscoveryNode("1", LocalTransportAddress.buildUnique(), emptyMap(), emptySet(), Version.CURRENT); Stage[] stages = Stage.values(); int i = randomIntBetween(0, stages.length - 1); int j; diff --git a/core/src/test/java/org/elasticsearch/indices/state/RareClusterStateIT.java b/core/src/test/java/org/elasticsearch/indices/state/RareClusterStateIT.java index 0b1723bd5aa86..bb2d4369c49bd 100644 --- a/core/src/test/java/org/elasticsearch/indices/state/RareClusterStateIT.java +++ b/core/src/test/java/org/elasticsearch/indices/state/RareClusterStateIT.java @@ -42,7 +42,7 @@ import org.elasticsearch.cluster.service.ClusterService; import org.elasticsearch.common.collect.ImmutableOpenMap; import org.elasticsearch.common.settings.Settings; -import org.elasticsearch.common.transport.DummyTransportAddress; +import org.elasticsearch.common.transport.LocalTransportAddress; import org.elasticsearch.common.unit.TimeValue; import org.elasticsearch.discovery.DiscoveryModule; import org.elasticsearch.discovery.DiscoverySettings; @@ -126,7 +126,7 @@ public ClusterState execute(ClusterState currentState) throws Exception { // inject a node ClusterState.Builder builder = ClusterState.builder(currentState); builder.nodes(DiscoveryNodes.builder(currentState.nodes()).put(new DiscoveryNode("_non_existent", - DummyTransportAddress.INSTANCE, emptyMap(), emptySet(), Version.CURRENT))); + LocalTransportAddress.buildUnique(), emptyMap(), emptySet(), Version.CURRENT))); // open index final IndexMetaData indexMetaData = IndexMetaData.builder(currentState.metaData().index(index)).state(IndexMetaData.State.OPEN).build(); diff --git a/core/src/test/java/org/elasticsearch/nodesinfo/NodeInfoStreamingTests.java b/core/src/test/java/org/elasticsearch/nodesinfo/NodeInfoStreamingTests.java index 92dac4bd78490..d87529dd716fb 100644 --- a/core/src/test/java/org/elasticsearch/nodesinfo/NodeInfoStreamingTests.java +++ b/core/src/test/java/org/elasticsearch/nodesinfo/NodeInfoStreamingTests.java @@ -28,7 +28,7 @@ import org.elasticsearch.common.io.stream.StreamInput; import org.elasticsearch.common.settings.Settings; import org.elasticsearch.common.transport.BoundTransportAddress; -import org.elasticsearch.common.transport.DummyTransportAddress; +import org.elasticsearch.common.transport.LocalTransportAddress; import org.elasticsearch.common.transport.TransportAddress; import org.elasticsearch.common.xcontent.ToXContent; import org.elasticsearch.common.xcontent.XContentBuilder; @@ -121,7 +121,7 @@ private void compareJsonOutput(ToXContent param1, ToXContent param2) throws IOEx private NodeInfo createNodeInfo() { Build build = Build.CURRENT; - DiscoveryNode node = new DiscoveryNode("test_node", DummyTransportAddress.INSTANCE, + DiscoveryNode node = new DiscoveryNode("test_node", LocalTransportAddress.buildUnique(), emptyMap(), emptySet(), VersionUtils.randomVersion(random())); Map serviceAttributes = new HashMap<>(); serviceAttributes.put("test", "attribute"); @@ -133,7 +133,7 @@ private NodeInfo createNodeInfo() { threadPoolInfos.add(new ThreadPool.Info("test_threadpool", ThreadPool.ThreadPoolType.FIXED, 5)); ThreadPoolInfo threadPoolInfo = new ThreadPoolInfo(threadPoolInfos); Map profileAddresses = new HashMap<>(); - BoundTransportAddress dummyBoundTransportAddress = new BoundTransportAddress(new TransportAddress[]{DummyTransportAddress.INSTANCE}, DummyTransportAddress.INSTANCE); + BoundTransportAddress dummyBoundTransportAddress = new BoundTransportAddress(new TransportAddress[]{LocalTransportAddress.buildUnique()}, LocalTransportAddress.buildUnique()); profileAddresses.put("test_address", dummyBoundTransportAddress); TransportInfo transport = new TransportInfo(dummyBoundTransportAddress, profileAddresses); HttpInfo htttpInfo = new HttpInfo(dummyBoundTransportAddress, randomLong()); diff --git a/core/src/test/java/org/elasticsearch/recovery/RecoveriesCollectionTests.java b/core/src/test/java/org/elasticsearch/recovery/RecoveriesCollectionTests.java index 65bec885ea8e6..58fa2d9f652fe 100644 --- a/core/src/test/java/org/elasticsearch/recovery/RecoveriesCollectionTests.java +++ b/core/src/test/java/org/elasticsearch/recovery/RecoveriesCollectionTests.java @@ -22,7 +22,7 @@ import org.elasticsearch.cluster.metadata.IndexMetaData; import org.elasticsearch.cluster.node.DiscoveryNode; import org.elasticsearch.common.settings.Settings; -import org.elasticsearch.common.transport.DummyTransportAddress; +import org.elasticsearch.common.transport.LocalTransportAddress; import org.elasticsearch.common.unit.TimeValue; import org.elasticsearch.index.shard.IndexShard; import org.elasticsearch.index.shard.ShardId; @@ -168,7 +168,7 @@ long startRecovery(RecoveriesCollection collection) { long startRecovery(RecoveriesCollection collection, RecoveryTargetService.RecoveryListener listener, TimeValue timeValue) { IndicesService indexServices = getInstanceFromNode(IndicesService.class); IndexShard indexShard = indexServices.indexServiceSafe(resolveIndex("test")).getShardOrNull(0); - final DiscoveryNode sourceNode = new DiscoveryNode("id", DummyTransportAddress.INSTANCE, emptyMap(), emptySet(), Version.CURRENT); + final DiscoveryNode sourceNode = new DiscoveryNode("id", LocalTransportAddress.buildUnique(), emptyMap(), emptySet(), Version.CURRENT); return collection.startRecovery(indexShard, sourceNode, listener, timeValue); } } diff --git a/test/framework/src/main/java/org/elasticsearch/cluster/MockInternalClusterInfoService.java b/test/framework/src/main/java/org/elasticsearch/cluster/MockInternalClusterInfoService.java index d66b1f9fc9ee7..c897b7f491e23 100644 --- a/test/framework/src/main/java/org/elasticsearch/cluster/MockInternalClusterInfoService.java +++ b/test/framework/src/main/java/org/elasticsearch/cluster/MockInternalClusterInfoService.java @@ -32,7 +32,7 @@ import org.elasticsearch.common.inject.Inject; import org.elasticsearch.common.settings.ClusterSettings; import org.elasticsearch.common.settings.Settings; -import org.elasticsearch.common.transport.DummyTransportAddress; +import org.elasticsearch.common.transport.LocalTransportAddress; import org.elasticsearch.common.unit.TimeValue; import org.elasticsearch.monitor.fs.FsInfo; import org.elasticsearch.plugins.Plugin; @@ -73,7 +73,7 @@ public static NodeStats makeStats(String nodeName, DiskUsage usage) { usage.getTotalBytes(), usage.getFreeBytes(), usage.getFreeBytes()); paths[0] = path; FsInfo fsInfo = new FsInfo(System.currentTimeMillis(), paths); - return new NodeStats(new DiscoveryNode(nodeName, DummyTransportAddress.INSTANCE, emptyMap(), emptySet(), Version.CURRENT), + return new NodeStats(new DiscoveryNode(nodeName, LocalTransportAddress.buildUnique(), emptyMap(), emptySet(), Version.CURRENT), System.currentTimeMillis(), null, null, null, null, null, fsInfo, diff --git a/test/framework/src/main/java/org/elasticsearch/test/ESAllocationTestCase.java b/test/framework/src/main/java/org/elasticsearch/test/ESAllocationTestCase.java index d7d5df542402f..ea7e2964388aa 100644 --- a/test/framework/src/main/java/org/elasticsearch/test/ESAllocationTestCase.java +++ b/test/framework/src/main/java/org/elasticsearch/test/ESAllocationTestCase.java @@ -41,7 +41,7 @@ import org.elasticsearch.common.Randomness; import org.elasticsearch.common.settings.ClusterSettings; import org.elasticsearch.common.settings.Settings; -import org.elasticsearch.common.transport.DummyTransportAddress; +import org.elasticsearch.common.transport.LocalTransportAddress; import org.elasticsearch.gateway.AsyncShardFetch; import org.elasticsearch.gateway.GatewayAllocator; import org.elasticsearch.gateway.ReplicaShardAllocator; @@ -132,19 +132,19 @@ protected static DiscoveryNode newNode(String nodeId) { } protected static DiscoveryNode newNode(String nodeName, String nodeId, Map attributes) { - return new DiscoveryNode(nodeName, nodeId, DummyTransportAddress.INSTANCE, attributes, MASTER_DATA_ROLES, Version.CURRENT); + return new DiscoveryNode(nodeName, nodeId, LocalTransportAddress.buildUnique(), attributes, MASTER_DATA_ROLES, Version.CURRENT); } protected static DiscoveryNode newNode(String nodeId, Map attributes) { - return new DiscoveryNode(nodeId, DummyTransportAddress.INSTANCE, attributes, MASTER_DATA_ROLES, Version.CURRENT); + return new DiscoveryNode(nodeId, LocalTransportAddress.buildUnique(), attributes, MASTER_DATA_ROLES, Version.CURRENT); } protected static DiscoveryNode newNode(String nodeId, Set roles) { - return new DiscoveryNode(nodeId, DummyTransportAddress.INSTANCE, emptyMap(), roles, Version.CURRENT); + return new DiscoveryNode(nodeId, LocalTransportAddress.buildUnique(), emptyMap(), roles, Version.CURRENT); } protected static DiscoveryNode newNode(String nodeId, Version version) { - return new DiscoveryNode(nodeId, DummyTransportAddress.INSTANCE, emptyMap(), MASTER_DATA_ROLES, version); + return new DiscoveryNode(nodeId, LocalTransportAddress.buildUnique(), emptyMap(), MASTER_DATA_ROLES, version); } protected static ClusterState startRandomInitializingShard(ClusterState clusterState, AllocationService strategy) { From 512c4f4f6a9863c9759606833451ab92d7c27b8c Mon Sep 17 00:00:00 2001 From: Boaz Leskes Date: Sat, 16 Apr 2016 22:28:00 +0200 Subject: [PATCH 03/34] add node environment test --- .../org/elasticsearch/env/NodeEnvironment.java | 2 +- .../elasticsearch/env/NodeEnvironmentTests.java | 14 ++++++++++++++ 2 files changed, 15 insertions(+), 1 deletion(-) diff --git a/core/src/main/java/org/elasticsearch/env/NodeEnvironment.java b/core/src/main/java/org/elasticsearch/env/NodeEnvironment.java index a46903bb3015f..a16d11ca18546 100644 --- a/core/src/main/java/org/elasticsearch/env/NodeEnvironment.java +++ b/core/src/main/java/org/elasticsearch/env/NodeEnvironment.java @@ -178,7 +178,7 @@ public String toString() { public NodeEnvironment(Settings settings, Environment environment) throws IOException { super(settings); - this.addNodeId = ADD_NODE_ID_TO_CUSTOM_PATH.get(settings); + this.addNodeId = ADD_NODE_LOCK_ID_TO_CUSTOM_PATH.get(settings); if (!DiscoveryNode.nodeRequiresLocalStorage(settings)) { nodePaths = null; diff --git a/core/src/test/java/org/elasticsearch/env/NodeEnvironmentTests.java b/core/src/test/java/org/elasticsearch/env/NodeEnvironmentTests.java index 3c13351a125b9..aa96626a68e86 100644 --- a/core/src/test/java/org/elasticsearch/env/NodeEnvironmentTests.java +++ b/core/src/test/java/org/elasticsearch/env/NodeEnvironmentTests.java @@ -51,6 +51,7 @@ import static org.hamcrest.Matchers.arrayWithSize; import static org.hamcrest.Matchers.containsString; import static org.hamcrest.Matchers.empty; +import static org.hamcrest.Matchers.not; @LuceneTestCase.SuppressFileSystems("ExtrasFS") // TODO: fix test to allow extras public class NodeEnvironmentTests extends ESTestCase { @@ -404,6 +405,19 @@ public void testCustomDataPaths() throws Exception { env2.close(); } + public void testPersistentNodeId() throws IOException { + String[] paths = tmpPaths(); + NodeEnvironment env = newNodeEnvironment(paths, Settings.EMPTY); + String nodeID = env.nodeID(); + env.close(); + env = newNodeEnvironment(paths, Settings.EMPTY); + assertThat(env.nodeID(), equalTo(nodeID)); + env.close(); + env = newNodeEnvironment(Settings.EMPTY); + assertThat(env.nodeID(), not(equalTo(nodeID))); + env.close(); + } + /** Converts an array of Strings to an array of Paths, adding an additional child if specified */ private Path[] stringsToPaths(String[] strings, String additional) { Path[] locations = new Path[strings.length]; From 199e7e5cd2111368b3c3d7f62bdd0f4dc57d734e Mon Sep 17 00:00:00 2001 From: Boaz Leskes Date: Sun, 17 Apr 2016 12:10:23 +0300 Subject: [PATCH 04/34] introduce explicit node storage setting --- .../org/elasticsearch/cluster/node/DiscoveryNode.java | 10 +++++++++- .../common/settings/ClusterSettings.java | 1 + .../java/org/elasticsearch/env/NodeEnvironment.java | 1 - core/src/main/java/org/elasticsearch/node/Node.java | 11 +++++++++++ 4 files changed, 21 insertions(+), 2 deletions(-) diff --git a/core/src/main/java/org/elasticsearch/cluster/node/DiscoveryNode.java b/core/src/main/java/org/elasticsearch/cluster/node/DiscoveryNode.java index ff37b6e6d4cb1..7007c139c9916 100644 --- a/core/src/main/java/org/elasticsearch/cluster/node/DiscoveryNode.java +++ b/core/src/main/java/org/elasticsearch/cluster/node/DiscoveryNode.java @@ -64,7 +64,15 @@ public static boolean isLocalNode(Settings settings) { } public static boolean nodeRequiresLocalStorage(Settings settings) { - return Node.NODE_DATA_SETTING.get(settings) || Node.NODE_MASTER_SETTING.get(settings); + boolean localStorageEnable = Node.NODE_LOCAL_STORAGE_ENABLED_SETTING.get(settings); + if (localStorageEnable == false && + (Node.NODE_DATA_SETTING.get(settings) || + Node.NODE_MASTER_SETTING.get(settings)) + ) { + // TODO: make this a proper setting validation logic, requiring multi-settings validation + throw new IllegalArgumentException("storage can not be disabled for master and data nodes"); + } + return localStorageEnable; } public static boolean isMasterNode(Settings settings) { diff --git a/core/src/main/java/org/elasticsearch/common/settings/ClusterSettings.java b/core/src/main/java/org/elasticsearch/common/settings/ClusterSettings.java index ab3adad2f0caf..609b1931b1395 100644 --- a/core/src/main/java/org/elasticsearch/common/settings/ClusterSettings.java +++ b/core/src/main/java/org/elasticsearch/common/settings/ClusterSettings.java @@ -363,6 +363,7 @@ public void apply(Settings value, Settings current, Settings previous) { Node.NODE_MODE_SETTING, Node.NODE_INGEST_SETTING, Node.NODE_ATTRIBUTES, + Node.NODE_LOCAL_STORAGE_ENABLED_SETTING, URLRepository.ALLOWED_URLS_SETTING, URLRepository.REPOSITORIES_LIST_DIRECTORIES_SETTING, URLRepository.REPOSITORIES_URL_SETTING, diff --git a/core/src/main/java/org/elasticsearch/env/NodeEnvironment.java b/core/src/main/java/org/elasticsearch/env/NodeEnvironment.java index a16d11ca18546..8783fcd650137 100644 --- a/core/src/main/java/org/elasticsearch/env/NodeEnvironment.java +++ b/core/src/main/java/org/elasticsearch/env/NodeEnvironment.java @@ -185,7 +185,6 @@ public NodeEnvironment(Settings settings, Environment environment) throws IOExce sharedDataPath = null; locks = null; nodeLockId = -1; - // nocommit - this a big shame for coordinating nodes.. nodeMetaData = new NodeMetaData(generateNodeId(settings)); return; } diff --git a/core/src/main/java/org/elasticsearch/node/Node.java b/core/src/main/java/org/elasticsearch/node/Node.java index a24745d52b263..da83194c194e7 100644 --- a/core/src/main/java/org/elasticsearch/node/Node.java +++ b/core/src/main/java/org/elasticsearch/node/Node.java @@ -140,6 +140,17 @@ public class Node implements Closeable { new Setting<>("node.mode", "network", Function.identity(), Property.NodeScope); public static final Setting NODE_INGEST_SETTING = Setting.boolSetting("node.ingest", true, Property.NodeScope); + + /** + * controls whether the node is allowed to persist things like metadata to disk + * Note that this does not control whether the node store actual indices (see + * {@link #NODE_DATA_SETTING}). However, if this is false, {@link #NODE_DATA_SETTING} + * and {@link #NODE_MASTER_SETTING} must also be false. + * + */ + public static final Setting NODE_LOCAL_STORAGE_ENABLED_SETTING = + Setting.boolSetting("node.local_storage.enabled", true, Property.NodeScope); + public static final Setting NODE_NAME_SETTING = Setting.simpleString("node.name", Property.NodeScope); public static final Setting NODE_ATTRIBUTES = Setting.groupSetting("node.attr.", Property.NodeScope); From af1ecee4391c2c0b1663b7ab0c86c2071d3abaf4 Mon Sep 17 00:00:00 2001 From: Boaz Leskes Date: Sun, 17 Apr 2016 12:11:05 +0300 Subject: [PATCH 05/34] minor naming tweak --- .../main/java/org/elasticsearch/env/NodeEnvironment.java | 6 +++--- 1 file changed, 3 insertions(+), 3 deletions(-) diff --git a/core/src/main/java/org/elasticsearch/env/NodeEnvironment.java b/core/src/main/java/org/elasticsearch/env/NodeEnvironment.java index 8783fcd650137..12a795929dc83 100644 --- a/core/src/main/java/org/elasticsearch/env/NodeEnvironment.java +++ b/core/src/main/java/org/elasticsearch/env/NodeEnvironment.java @@ -133,7 +133,7 @@ public String toString() { private final Path sharedDataPath; private final Lock[] locks; - private final boolean addNodeId; + private final boolean addLockIdToCustomPath; private final int nodeLockId; private final AtomicBoolean closed = new AtomicBoolean(false); @@ -178,7 +178,7 @@ public String toString() { public NodeEnvironment(Settings settings, Environment environment) throws IOException { super(settings); - this.addNodeId = ADD_NODE_LOCK_ID_TO_CUSTOM_PATH.get(settings); + this.addLockIdToCustomPath = ADD_NODE_LOCK_ID_TO_CUSTOM_PATH.get(settings); if (!DiscoveryNode.nodeRequiresLocalStorage(settings)) { nodePaths = null; @@ -896,7 +896,7 @@ public Path resolveBaseCustomLocation(IndexSettings indexSettings) { if (customDataDir != null) { // This assert is because this should be caught by MetaDataCreateIndexService assert sharedDataPath != null; - if (addNodeId) { + if (addLockIdToCustomPath) { return sharedDataPath.resolve(customDataDir).resolve(Integer.toString(this.nodeLockId)); } else { return sharedDataPath.resolve(customDataDir); From 536a37735d3c44e4f54e1195cdc2715981e1e5c4 Mon Sep 17 00:00:00 2001 From: Boaz Leskes Date: Sun, 17 Apr 2016 12:20:46 +0300 Subject: [PATCH 06/34] line lengths --- .../node/DiscoveryNodeFiltersTests.java | 45 ++++++++++++------- .../cluster/node/DiscoveryNodesTests.java | 3 +- .../indices/recovery/RecoveryTargetTests.java | 3 +- .../recovery/RecoveriesCollectionTests.java | 3 +- 4 files changed, 36 insertions(+), 18 deletions(-) diff --git a/core/src/test/java/org/elasticsearch/cluster/node/DiscoveryNodeFiltersTests.java b/core/src/test/java/org/elasticsearch/cluster/node/DiscoveryNodeFiltersTests.java index 6b1ae72e359e7..97a94d24d1c73 100644 --- a/core/src/test/java/org/elasticsearch/cluster/node/DiscoveryNodeFiltersTests.java +++ b/core/src/test/java/org/elasticsearch/cluster/node/DiscoveryNodeFiltersTests.java @@ -64,10 +64,12 @@ public void testNameMatch() { .build(); DiscoveryNodeFilters filters = DiscoveryNodeFilters.buildFromSettings(OR, "xxx.", settings); - DiscoveryNode node = new DiscoveryNode("name1", "id1", LocalTransportAddress.buildUnique(), emptyMap(), emptySet(), Version.CURRENT); + DiscoveryNode node = new DiscoveryNode("name1", "id1", LocalTransportAddress.buildUnique(), emptyMap(), + emptySet(), Version.CURRENT); assertThat(filters.match(node), equalTo(true)); - node = new DiscoveryNode("name2", "id2", LocalTransportAddress.buildUnique(), emptyMap(), emptySet(), Version.CURRENT); + node = new DiscoveryNode("name2", "id2", LocalTransportAddress.buildUnique(), emptyMap(), emptySet(), + Version.CURRENT); assertThat(filters.match(node), equalTo(false)); } @@ -77,10 +79,12 @@ public void testIdMatch() { .build(); DiscoveryNodeFilters filters = DiscoveryNodeFilters.buildFromSettings(OR, "xxx.", settings); - DiscoveryNode node = new DiscoveryNode("name1", "id1", LocalTransportAddress.buildUnique(), emptyMap(), emptySet(), Version.CURRENT); + DiscoveryNode node = new DiscoveryNode("name1", "id1", LocalTransportAddress.buildUnique(), emptyMap(), + emptySet(), Version.CURRENT); assertThat(filters.match(node), equalTo(true)); - node = new DiscoveryNode("name2", "id2", LocalTransportAddress.buildUnique(), emptyMap(), emptySet(), Version.CURRENT); + node = new DiscoveryNode("name2", "id2", LocalTransportAddress.buildUnique(), emptyMap(), emptySet(), + Version.CURRENT); assertThat(filters.match(node), equalTo(false)); } @@ -91,13 +95,16 @@ public void testIdOrNameMatch() { .build()); DiscoveryNodeFilters filters = DiscoveryNodeFilters.buildFromSettings(OR, "xxx.", settings); - DiscoveryNode node = new DiscoveryNode("name1", "id1", LocalTransportAddress.buildUnique(), emptyMap(), emptySet(), Version.CURRENT); + DiscoveryNode node = new DiscoveryNode("name1", "id1", LocalTransportAddress.buildUnique(), emptyMap(), + emptySet(), Version.CURRENT); assertThat(filters.match(node), equalTo(true)); - node = new DiscoveryNode("name2", "id2", LocalTransportAddress.buildUnique(), emptyMap(), emptySet(), Version.CURRENT); + node = new DiscoveryNode("name2", "id2", LocalTransportAddress.buildUnique(), emptyMap(), emptySet(), + Version.CURRENT); assertThat(filters.match(node), equalTo(true)); - node = new DiscoveryNode("name3", "id3", LocalTransportAddress.buildUnique(), emptyMap(), emptySet(), Version.CURRENT); + node = new DiscoveryNode("name3", "id3", LocalTransportAddress.buildUnique(), emptyMap(), emptySet(), + Version.CURRENT); assertThat(filters.match(node), equalTo(false)); } @@ -141,7 +148,8 @@ public void testStarMatch() { .build(); DiscoveryNodeFilters filters = DiscoveryNodeFilters.buildFromSettings(OR, "xxx.", settings); - DiscoveryNode node = new DiscoveryNode("name1", "id1", LocalTransportAddress.buildUnique(), emptyMap(), emptySet(), Version.CURRENT); + DiscoveryNode node = new DiscoveryNode("name1", "id1", LocalTransportAddress.buildUnique(), emptyMap(), + emptySet(), Version.CURRENT); assertThat(filters.match(node), equalTo(true)); } @@ -152,7 +160,8 @@ public void testIpBindFilteringMatchingAnd() { .build()); DiscoveryNodeFilters filters = DiscoveryNodeFilters.buildFromSettings(AND, "xxx.", settings); - DiscoveryNode node = new DiscoveryNode("", "", "", "192.1.1.54", localAddress, singletonMap("tag", "A"), emptySet(), null); + DiscoveryNode node = new DiscoveryNode("", "", "", "192.1.1.54", localAddress, singletonMap("tag", "A"), + emptySet(), null); assertThat(filters.match(node), equalTo(true)); } @@ -163,7 +172,8 @@ public void testIpBindFilteringNotMatching() { .build()); DiscoveryNodeFilters filters = DiscoveryNodeFilters.buildFromSettings(AND, "xxx.", settings); - DiscoveryNode node = new DiscoveryNode("", "", "", "192.1.1.54", localAddress, singletonMap("tag", "A"), emptySet(), null); + DiscoveryNode node = new DiscoveryNode("", "", "", "192.1.1.54", localAddress, singletonMap("tag", "A"), + emptySet(), null); assertThat(filters.match(node), equalTo(false)); } @@ -174,7 +184,8 @@ public void testIpBindFilteringNotMatchingAnd() { .build()); DiscoveryNodeFilters filters = DiscoveryNodeFilters.buildFromSettings(AND, "xxx.", settings); - DiscoveryNode node = new DiscoveryNode("", "", "", "192.1.1.54", localAddress, singletonMap("tag", "A"), emptySet(), null); + DiscoveryNode node = new DiscoveryNode("", "", "", "192.1.1.54", localAddress, singletonMap("tag", "A"), + emptySet(), null); assertThat(filters.match(node), equalTo(false)); } @@ -207,7 +218,8 @@ public void testIpPublishFilteringMatchingAnd() { .build()); DiscoveryNodeFilters filters = DiscoveryNodeFilters.buildFromSettings(AND, "xxx.", settings); - DiscoveryNode node = new DiscoveryNode("", "", "", "192.1.1.54", localAddress, singletonMap("tag", "A"), emptySet(), null); + DiscoveryNode node = new DiscoveryNode("", "", "", "192.1.1.54", localAddress, singletonMap("tag", "A"), + emptySet(), null); assertThat(filters.match(node), equalTo(true)); } @@ -218,7 +230,8 @@ public void testIpPublishFilteringNotMatchingAnd() { .build()); DiscoveryNodeFilters filters = DiscoveryNodeFilters.buildFromSettings(AND, "xxx.", settings); - DiscoveryNode node = new DiscoveryNode("", "", "", "192.1.1.54", localAddress, singletonMap("tag", "A"), emptySet(), null); + DiscoveryNode node = new DiscoveryNode("", "", "", "192.1.1.54", localAddress, singletonMap("tag", "A"), + emptySet(), null); assertThat(filters.match(node), equalTo(false)); } @@ -229,7 +242,8 @@ public void testIpPublishFilteringMatchingOr() { .build()); DiscoveryNodeFilters filters = DiscoveryNodeFilters.buildFromSettings(OR, "xxx.", settings); - DiscoveryNode node = new DiscoveryNode("", "", "", "192.1.1.54", localAddress, singletonMap("tag", "A"), emptySet(), null); + DiscoveryNode node = new DiscoveryNode("", "", "", "192.1.1.54", localAddress, singletonMap("tag", "A"), + emptySet(), null); assertThat(filters.match(node), equalTo(true)); } @@ -240,7 +254,8 @@ public void testIpPublishFilteringNotMatchingOr() { .build()); DiscoveryNodeFilters filters = DiscoveryNodeFilters.buildFromSettings(OR, "xxx.", settings); - DiscoveryNode node = new DiscoveryNode("", "", "", "192.1.1.54", localAddress, singletonMap("tag", "A"), emptySet(), null); + DiscoveryNode node = new DiscoveryNode("", "", "", "192.1.1.54", localAddress, singletonMap("tag", "A"), + emptySet(), null); assertThat(filters.match(node), equalTo(true)); } diff --git a/core/src/test/java/org/elasticsearch/cluster/node/DiscoveryNodesTests.java b/core/src/test/java/org/elasticsearch/cluster/node/DiscoveryNodesTests.java index 7c1c35b9ef50e..1d1ee7c717720 100644 --- a/core/src/test/java/org/elasticsearch/cluster/node/DiscoveryNodesTests.java +++ b/core/src/test/java/org/elasticsearch/cluster/node/DiscoveryNodesTests.java @@ -117,7 +117,8 @@ private static DiscoveryNodes buildDiscoveryNodes() { } private static DiscoveryNode newNode(int nodeId, Map attributes, Set roles) { - return new DiscoveryNode("name_" + nodeId, "node_" + nodeId, LocalTransportAddress.buildUnique(), attributes, roles, Version.CURRENT); + return new DiscoveryNode("name_" + nodeId, "node_" + nodeId, LocalTransportAddress.buildUnique(), attributes, + roles, Version.CURRENT); } private enum NodeSelector { diff --git a/core/src/test/java/org/elasticsearch/indices/recovery/RecoveryTargetTests.java b/core/src/test/java/org/elasticsearch/indices/recovery/RecoveryTargetTests.java index 3bcedeb379e70..94fb798da74ca 100644 --- a/core/src/test/java/org/elasticsearch/indices/recovery/RecoveryTargetTests.java +++ b/core/src/test/java/org/elasticsearch/indices/recovery/RecoveryTargetTests.java @@ -339,7 +339,8 @@ Index createObj() { } public void testStageSequenceEnforcement() { - final DiscoveryNode discoveryNode = new DiscoveryNode("1", LocalTransportAddress.buildUnique(), emptyMap(), emptySet(), Version.CURRENT); + final DiscoveryNode discoveryNode = new DiscoveryNode("1", LocalTransportAddress.buildUnique(), emptyMap(), + emptySet(), Version.CURRENT); Stage[] stages = Stage.values(); int i = randomIntBetween(0, stages.length - 1); int j; diff --git a/core/src/test/java/org/elasticsearch/recovery/RecoveriesCollectionTests.java b/core/src/test/java/org/elasticsearch/recovery/RecoveriesCollectionTests.java index 58fa2d9f652fe..6f4a64810c154 100644 --- a/core/src/test/java/org/elasticsearch/recovery/RecoveriesCollectionTests.java +++ b/core/src/test/java/org/elasticsearch/recovery/RecoveriesCollectionTests.java @@ -168,7 +168,8 @@ long startRecovery(RecoveriesCollection collection) { long startRecovery(RecoveriesCollection collection, RecoveryTargetService.RecoveryListener listener, TimeValue timeValue) { IndicesService indexServices = getInstanceFromNode(IndicesService.class); IndexShard indexShard = indexServices.indexServiceSafe(resolveIndex("test")).getShardOrNull(0); - final DiscoveryNode sourceNode = new DiscoveryNode("id", LocalTransportAddress.buildUnique(), emptyMap(), emptySet(), Version.CURRENT); + final DiscoveryNode sourceNode = new DiscoveryNode("id", LocalTransportAddress.buildUnique(), emptyMap(), + emptySet(), Version.CURRENT); return collection.startRecovery(indexShard, sourceNode, listener, timeValue); } } From ea8931e7b32aa049b7154eaf315a962e024663c5 Mon Sep 17 00:00:00 2001 From: Boaz Leskes Date: Sun, 17 Apr 2016 12:40:29 +0300 Subject: [PATCH 07/34] fix min state version check --- .../java/org/elasticsearch/gateway/MetaDataStateFormat.java | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/core/src/main/java/org/elasticsearch/gateway/MetaDataStateFormat.java b/core/src/main/java/org/elasticsearch/gateway/MetaDataStateFormat.java index 9ecbe79ee7826..939b65129e3cd 100644 --- a/core/src/main/java/org/elasticsearch/gateway/MetaDataStateFormat.java +++ b/core/src/main/java/org/elasticsearch/gateway/MetaDataStateFormat.java @@ -181,7 +181,8 @@ public final T read(Path file) throws IOException { try (final IndexInput indexInput = dir.openInput(file.getFileName().toString(), IOContext.DEFAULT)) { // We checksum the entire file before we even go and parse it. If it's corrupted we barf right here. CodecUtil.checksumEntireFile(indexInput); - final int fileVersion = CodecUtil.checkHeader(indexInput, STATE_FILE_CODEC, STATE_FILE_VERSION, STATE_FILE_VERSION); + final int fileVersion = CodecUtil.checkHeader(indexInput, STATE_FILE_CODEC, MIN_COMPATIBLE_STATE_FILE_VERSION, + STATE_FILE_VERSION); final XContentType xContentType = XContentType.values()[indexInput.readInt()]; if (fileVersion == 0) { // format version 0, write a version that always came from the content state file From 4ae867dc01aa70daccb40c448bf2b4ad26611a96 Mon Sep 17 00:00:00 2001 From: Boaz Leskes Date: Sun, 17 Apr 2016 12:50:34 +0300 Subject: [PATCH 08/34] Disable persistence for tribe node clients and strengthen test --- .../java/org/elasticsearch/tribe/TribeService.java | 1 + .../org/elasticsearch/env/NodeEnvironmentTests.java | 10 +++++++++- 2 files changed, 10 insertions(+), 1 deletion(-) diff --git a/core/src/main/java/org/elasticsearch/tribe/TribeService.java b/core/src/main/java/org/elasticsearch/tribe/TribeService.java index b66f428367c0e..d7d4b8fa5b867 100644 --- a/core/src/main/java/org/elasticsearch/tribe/TribeService.java +++ b/core/src/main/java/org/elasticsearch/tribe/TribeService.java @@ -241,6 +241,7 @@ static Settings buildClientSettings(String tribeName, Settings globalSettings, S sb.put(Node.NODE_DATA_SETTING.getKey(), false); sb.put(Node.NODE_MASTER_SETTING.getKey(), false); sb.put(Node.NODE_INGEST_SETTING.getKey(), false); + sb.put(Node.NODE_LOCAL_STORAGE_ENABLED_SETTING.getKey(), false); return sb.build(); } diff --git a/core/src/test/java/org/elasticsearch/env/NodeEnvironmentTests.java b/core/src/test/java/org/elasticsearch/env/NodeEnvironmentTests.java index aa96626a68e86..a7f266808f3b8 100644 --- a/core/src/test/java/org/elasticsearch/env/NodeEnvironmentTests.java +++ b/core/src/test/java/org/elasticsearch/env/NodeEnvironmentTests.java @@ -407,10 +407,18 @@ public void testCustomDataPaths() throws Exception { public void testPersistentNodeId() throws IOException { String[] paths = tmpPaths(); - NodeEnvironment env = newNodeEnvironment(paths, Settings.EMPTY); + NodeEnvironment env = newNodeEnvironment(paths, Settings.builder() + .put("node.local_storage.enabled", false) + .put("node.master", false) + .put("node.data", false) + .build()); String nodeID = env.nodeID(); env.close(); env = newNodeEnvironment(paths, Settings.EMPTY); + assertThat("previous node didn't have local storage enabled, id should change", env.nodeID(), not(equalTo(nodeID))); + nodeID = env.nodeID(); + env.close(); + env = newNodeEnvironment(paths, Settings.EMPTY); assertThat(env.nodeID(), equalTo(nodeID)); env.close(); env = newNodeEnvironment(Settings.EMPTY); From 570ff47aca6b587f0a0d06f29a603b119e65b366 Mon Sep 17 00:00:00 2001 From: Boaz Leskes Date: Sun, 17 Apr 2016 13:05:51 +0300 Subject: [PATCH 09/34] fix tribe test and rename local_storage method --- .../java/org/elasticsearch/cluster/node/DiscoveryNode.java | 2 +- .../org/elasticsearch/common/settings/ClusterSettings.java | 2 +- core/src/main/java/org/elasticsearch/node/Node.java | 4 ++-- core/src/main/java/org/elasticsearch/tribe/TribeService.java | 2 +- .../test/java/org/elasticsearch/tribe/TribeServiceTests.java | 3 ++- 5 files changed, 7 insertions(+), 6 deletions(-) diff --git a/core/src/main/java/org/elasticsearch/cluster/node/DiscoveryNode.java b/core/src/main/java/org/elasticsearch/cluster/node/DiscoveryNode.java index 7007c139c9916..5a44a867c111b 100644 --- a/core/src/main/java/org/elasticsearch/cluster/node/DiscoveryNode.java +++ b/core/src/main/java/org/elasticsearch/cluster/node/DiscoveryNode.java @@ -64,7 +64,7 @@ public static boolean isLocalNode(Settings settings) { } public static boolean nodeRequiresLocalStorage(Settings settings) { - boolean localStorageEnable = Node.NODE_LOCAL_STORAGE_ENABLED_SETTING.get(settings); + boolean localStorageEnable = Node.NODE_LOCAL_STORAGE_SETTING.get(settings); if (localStorageEnable == false && (Node.NODE_DATA_SETTING.get(settings) || Node.NODE_MASTER_SETTING.get(settings)) diff --git a/core/src/main/java/org/elasticsearch/common/settings/ClusterSettings.java b/core/src/main/java/org/elasticsearch/common/settings/ClusterSettings.java index 609b1931b1395..6fbdd156c7140 100644 --- a/core/src/main/java/org/elasticsearch/common/settings/ClusterSettings.java +++ b/core/src/main/java/org/elasticsearch/common/settings/ClusterSettings.java @@ -363,7 +363,7 @@ public void apply(Settings value, Settings current, Settings previous) { Node.NODE_MODE_SETTING, Node.NODE_INGEST_SETTING, Node.NODE_ATTRIBUTES, - Node.NODE_LOCAL_STORAGE_ENABLED_SETTING, + Node.NODE_LOCAL_STORAGE_SETTING, URLRepository.ALLOWED_URLS_SETTING, URLRepository.REPOSITORIES_LIST_DIRECTORIES_SETTING, URLRepository.REPOSITORIES_URL_SETTING, diff --git a/core/src/main/java/org/elasticsearch/node/Node.java b/core/src/main/java/org/elasticsearch/node/Node.java index da83194c194e7..892545e99fb20 100644 --- a/core/src/main/java/org/elasticsearch/node/Node.java +++ b/core/src/main/java/org/elasticsearch/node/Node.java @@ -148,8 +148,8 @@ public class Node implements Closeable { * and {@link #NODE_MASTER_SETTING} must also be false. * */ - public static final Setting NODE_LOCAL_STORAGE_ENABLED_SETTING = - Setting.boolSetting("node.local_storage.enabled", true, Property.NodeScope); + public static final Setting NODE_LOCAL_STORAGE_SETTING = + Setting.boolSetting("node.local_storage", true, Property.NodeScope); public static final Setting NODE_NAME_SETTING = Setting.simpleString("node.name", Property.NodeScope); public static final Setting NODE_ATTRIBUTES = Setting.groupSetting("node.attr.", Property.NodeScope); diff --git a/core/src/main/java/org/elasticsearch/tribe/TribeService.java b/core/src/main/java/org/elasticsearch/tribe/TribeService.java index d7d4b8fa5b867..b214a13ebdb7c 100644 --- a/core/src/main/java/org/elasticsearch/tribe/TribeService.java +++ b/core/src/main/java/org/elasticsearch/tribe/TribeService.java @@ -241,7 +241,7 @@ static Settings buildClientSettings(String tribeName, Settings globalSettings, S sb.put(Node.NODE_DATA_SETTING.getKey(), false); sb.put(Node.NODE_MASTER_SETTING.getKey(), false); sb.put(Node.NODE_INGEST_SETTING.getKey(), false); - sb.put(Node.NODE_LOCAL_STORAGE_ENABLED_SETTING.getKey(), false); + sb.put(Node.NODE_LOCAL_STORAGE_SETTING.getKey(), false); return sb.build(); } diff --git a/core/src/test/java/org/elasticsearch/tribe/TribeServiceTests.java b/core/src/test/java/org/elasticsearch/tribe/TribeServiceTests.java index daeb80d194c62..97ffb7a2c8182 100644 --- a/core/src/test/java/org/elasticsearch/tribe/TribeServiceTests.java +++ b/core/src/test/java/org/elasticsearch/tribe/TribeServiceTests.java @@ -35,7 +35,8 @@ public void testMinimalSettings() { assertEquals("false", clientSettings.get("node.master")); assertEquals("false", clientSettings.get("node.data")); assertEquals("false", clientSettings.get("node.ingest")); - assertEquals(7, clientSettings.getAsMap().size()); + assertEquals("false", clientSettings.get("node.local_storage")); + assertEquals(8, clientSettings.getAsMap().size()); } public void testEnvironmentSettings() { From 5cc42364bbd7933f969adeef580b98cbe6005902 Mon Sep 17 00:00:00 2001 From: Boaz Leskes Date: Sun, 17 Apr 2016 13:16:47 +0300 Subject: [PATCH 10/34] add migration docs --- .../main/java/org/elasticsearch/env/NodeEnvironment.java | 9 ++------- docs/reference/migration/migrate_5_0/fs.asciidoc | 7 +++++++ docs/reference/migration/migrate_5_0/settings.asciidoc | 4 ++++ 3 files changed, 13 insertions(+), 7 deletions(-) diff --git a/core/src/main/java/org/elasticsearch/env/NodeEnvironment.java b/core/src/main/java/org/elasticsearch/env/NodeEnvironment.java index 12a795929dc83..059990c2450f2 100644 --- a/core/src/main/java/org/elasticsearch/env/NodeEnvironment.java +++ b/core/src/main/java/org/elasticsearch/env/NodeEnvironment.java @@ -148,15 +148,10 @@ public String toString() { Property.NodeScope); /** - * If true automatically append node id to custom data paths. + * If true automatically append node lock id to custom data paths. */ - @Deprecated - public static final Setting ADD_NODE_ID_TO_CUSTOM_PATH = - Setting.boolSetting("node.add_id_to_custom_path", true, Property.Deprecated, Property.NodeScope); - public static final Setting ADD_NODE_LOCK_ID_TO_CUSTOM_PATH = - Setting.boolSetting("node.add_lock_id_to_custom_path", ADD_NODE_ID_TO_CUSTOM_PATH, - Property.NodeScope); + Setting.boolSetting("node.add_lock_id_to_custom_path", true, Property.NodeScope); public static final Setting NODE_ID_SEED_SETTING = diff --git a/docs/reference/migration/migrate_5_0/fs.asciidoc b/docs/reference/migration/migrate_5_0/fs.asciidoc index 2d582702690b7..6ef61f001684c 100644 --- a/docs/reference/migration/migrate_5_0/fs.asciidoc +++ b/docs/reference/migration/migrate_5_0/fs.asciidoc @@ -8,3 +8,10 @@ there is nothing to worry about since this is only address space consumption and the actual memory usage of Elasticsearch will stay similar to what it was in 2.x. See http://blog.thetaphi.de/2012/07/use-lucenes-mmapdirectory-on-64bit.html for more information. + + +==== Local files + +Prior to 5.0, nodes that were marked with both `node.data: false` and `node.master: false` (or the now removed `node.client: true`) +didn't write any files or folder to disk. 5.x added persistent node ids, requiring nodes to store that information. As such, all +node types will write a small state file to their data folders. diff --git a/docs/reference/migration/migrate_5_0/settings.asciidoc b/docs/reference/migration/migrate_5_0/settings.asciidoc index 342e8d922b6c3..d9b9304be61c9 100644 --- a/docs/reference/migration/migrate_5_0/settings.asciidoc +++ b/docs/reference/migration/migrate_5_0/settings.asciidoc @@ -252,3 +252,7 @@ scripts the following settings have been replaced to: * `script.engine.*.stored.search` has been replaced by `script.engine.*.stored.search` (where `*` represents the script language, like `groovy`, `mustache`, `plainless` etc.) * `script.engine.*.stored.update` has been replaced by `script.engine.*.stored.update` (where `*` represents the script language, like `groovy`, `mustache`, `plainless` etc.) * `script.engine.*.stored.plugin` has been replaced by `script.engine.*.stored.plugin` (where `*` represents the script language, like `groovy`, `mustache`, `plainless` etc.) + +==== Data Path Settings + +The `node.add_id_to_custom_path` has been renamed to `node.add_lock_id_to_custom_path`. From 07e55588377ad2a171f681f767840925280854ac Mon Sep 17 00:00:00 2001 From: Boaz Leskes Date: Sun, 17 Apr 2016 13:31:04 +0300 Subject: [PATCH 11/34] stupid fixes --- .../elasticsearch/common/settings/ClusterSettings.java | 2 +- .../common/util/IndexFolderUpgraderTests.java | 8 ++++---- .../java/org/elasticsearch/env/NodeEnvironmentTests.java | 4 ++-- .../elasticsearch/index/IndexWithShadowReplicasIT.java | 2 +- .../org/elasticsearch/index/shard/ShardPathTests.java | 2 +- 5 files changed, 9 insertions(+), 9 deletions(-) diff --git a/core/src/main/java/org/elasticsearch/common/settings/ClusterSettings.java b/core/src/main/java/org/elasticsearch/common/settings/ClusterSettings.java index 6fbdd156c7140..2896ba4c409ce 100644 --- a/core/src/main/java/org/elasticsearch/common/settings/ClusterSettings.java +++ b/core/src/main/java/org/elasticsearch/common/settings/ClusterSettings.java @@ -388,7 +388,7 @@ public void apply(Settings value, Settings current, Settings previous) { TribeService.TRIBE_NAME_SETTING, NodeEnvironment.MAX_LOCAL_STORAGE_NODES_SETTING, NodeEnvironment.ENABLE_LUCENE_SEGMENT_INFOS_TRACE_SETTING, - NodeEnvironment.ADD_NODE_ID_TO_CUSTOM_PATH, + NodeEnvironment.ADD_NODE_LOCK_ID_TO_CUSTOM_PATH, OsService.REFRESH_INTERVAL_SETTING, ProcessService.REFRESH_INTERVAL_SETTING, JvmService.REFRESH_INTERVAL_SETTING, diff --git a/core/src/test/java/org/elasticsearch/common/util/IndexFolderUpgraderTests.java b/core/src/test/java/org/elasticsearch/common/util/IndexFolderUpgraderTests.java index 0b53e8f6a8362..5302ba8d55c1b 100644 --- a/core/src/test/java/org/elasticsearch/common/util/IndexFolderUpgraderTests.java +++ b/core/src/test/java/org/elasticsearch/common/util/IndexFolderUpgraderTests.java @@ -67,7 +67,7 @@ public class IndexFolderUpgraderTests extends ESTestCase { public void testUpgradeCustomDataPath() throws IOException { Path customPath = createTempDir(); final Settings nodeSettings = Settings.builder() - .put(NodeEnvironment.ADD_NODE_ID_TO_CUSTOM_PATH.getKey(), randomBoolean()) + .put(NodeEnvironment.ADD_NODE_LOCK_ID_TO_CUSTOM_PATH.getKey(), randomBoolean()) .put(Environment.PATH_SHARED_DATA_SETTING.getKey(), customPath.toAbsolutePath().toString()).build(); try (NodeEnvironment nodeEnv = newNodeEnvironment(nodeSettings)) { final Index index = new Index(randomAsciiOfLength(10), UUIDs.randomBase64UUID()); @@ -96,7 +96,7 @@ public void testUpgradeCustomDataPath() throws IOException { public void testPartialUpgradeCustomDataPath() throws IOException { Path customPath = createTempDir(); final Settings nodeSettings = Settings.builder() - .put(NodeEnvironment.ADD_NODE_ID_TO_CUSTOM_PATH.getKey(), randomBoolean()) + .put(NodeEnvironment.ADD_NODE_LOCK_ID_TO_CUSTOM_PATH.getKey(), randomBoolean()) .put(Environment.PATH_SHARED_DATA_SETTING.getKey(), customPath.toAbsolutePath().toString()).build(); try (NodeEnvironment nodeEnv = newNodeEnvironment(nodeSettings)) { final Index index = new Index(randomAsciiOfLength(10), UUIDs.randomBase64UUID()); @@ -136,7 +136,7 @@ void upgrade(Index index, Path source, Path target) throws IOException { public void testUpgrade() throws IOException { final Settings nodeSettings = Settings.builder() - .put(NodeEnvironment.ADD_NODE_ID_TO_CUSTOM_PATH.getKey(), randomBoolean()).build(); + .put(NodeEnvironment.ADD_NODE_LOCK_ID_TO_CUSTOM_PATH.getKey(), randomBoolean()).build(); try (NodeEnvironment nodeEnv = newNodeEnvironment(nodeSettings)) { final Index index = new Index(randomAsciiOfLength(10), UUIDs.randomBase64UUID()); Settings settings = Settings.builder() @@ -159,7 +159,7 @@ public void testUpgrade() throws IOException { public void testUpgradeIndices() throws IOException { final Settings nodeSettings = Settings.builder() - .put(NodeEnvironment.ADD_NODE_ID_TO_CUSTOM_PATH.getKey(), randomBoolean()).build(); + .put(NodeEnvironment.ADD_NODE_LOCK_ID_TO_CUSTOM_PATH.getKey(), randomBoolean()).build(); try (NodeEnvironment nodeEnv = newNodeEnvironment(nodeSettings)) { Map> indexSettingsMap = new HashMap<>(); for (int i = 0; i < randomIntBetween(2, 5); i++) { diff --git a/core/src/test/java/org/elasticsearch/env/NodeEnvironmentTests.java b/core/src/test/java/org/elasticsearch/env/NodeEnvironmentTests.java index a7f266808f3b8..315a45b98fe13 100644 --- a/core/src/test/java/org/elasticsearch/env/NodeEnvironmentTests.java +++ b/core/src/test/java/org/elasticsearch/env/NodeEnvironmentTests.java @@ -390,7 +390,7 @@ public void testCustomDataPaths() throws Exception { env.close(); NodeEnvironment env2 = newNodeEnvironment(dataPaths, "/tmp", - Settings.builder().put(NodeEnvironment.ADD_NODE_ID_TO_CUSTOM_PATH.getKey(), false).build()); + Settings.builder().put(NodeEnvironment.ADD_NODE_LOCK_ID_TO_CUSTOM_PATH.getKey(), false).build()); assertThat(env2.availableShardPaths(sid), equalTo(env2.availableShardPaths(sid))); assertThat(env2.resolveCustomLocation(s2, sid), equalTo(PathUtils.get("/tmp/foo/" + index.getUUID() + "/0"))); @@ -408,7 +408,7 @@ public void testCustomDataPaths() throws Exception { public void testPersistentNodeId() throws IOException { String[] paths = tmpPaths(); NodeEnvironment env = newNodeEnvironment(paths, Settings.builder() - .put("node.local_storage.enabled", false) + .put("node.local_storage", false) .put("node.master", false) .put("node.data", false) .build()); diff --git a/core/src/test/java/org/elasticsearch/index/IndexWithShadowReplicasIT.java b/core/src/test/java/org/elasticsearch/index/IndexWithShadowReplicasIT.java index 4eb674397be62..648c46b64b415 100644 --- a/core/src/test/java/org/elasticsearch/index/IndexWithShadowReplicasIT.java +++ b/core/src/test/java/org/elasticsearch/index/IndexWithShadowReplicasIT.java @@ -94,7 +94,7 @@ private Settings nodeSettings(Path dataPath) { private Settings nodeSettings(String dataPath) { return Settings.builder() - .put(NodeEnvironment.ADD_NODE_ID_TO_CUSTOM_PATH.getKey(), false) + .put(NodeEnvironment.ADD_NODE_LOCK_ID_TO_CUSTOM_PATH.getKey(), false) .put(Environment.PATH_SHARED_DATA_SETTING.getKey(), dataPath) .put(FsDirectoryService.INDEX_LOCK_FACTOR_SETTING.getKey(), randomFrom("native", "simple")) .build(); diff --git a/core/src/test/java/org/elasticsearch/index/shard/ShardPathTests.java b/core/src/test/java/org/elasticsearch/index/shard/ShardPathTests.java index fdf8842638453..749b1621e4d90 100644 --- a/core/src/test/java/org/elasticsearch/index/shard/ShardPathTests.java +++ b/core/src/test/java/org/elasticsearch/index/shard/ShardPathTests.java @@ -124,7 +124,7 @@ public void testGetRootPaths() throws IOException { final boolean includeNodeId = randomBoolean(); indexSettings = indexSettingsBuilder.put(IndexMetaData.SETTING_DATA_PATH, "custom").build(); nodeSettings = Settings.builder().put(Environment.PATH_SHARED_DATA_SETTING.getKey(), path.toAbsolutePath().toAbsolutePath()) - .put(NodeEnvironment.ADD_NODE_ID_TO_CUSTOM_PATH.getKey(), includeNodeId).build(); + .put(NodeEnvironment.ADD_NODE_LOCK_ID_TO_CUSTOM_PATH.getKey(), includeNodeId).build(); if (includeNodeId) { customPath = path.resolve("custom").resolve("0"); } else { From 2ac579d5f8b867e178d978f95b83574753dc4d0d Mon Sep 17 00:00:00 2001 From: Boaz Leskes Date: Sun, 17 Apr 2016 13:35:13 +0300 Subject: [PATCH 12/34] unneeded lines --- .../java/org/elasticsearch/cluster/node/DiscoveryNodes.java | 2 -- 1 file changed, 2 deletions(-) diff --git a/core/src/main/java/org/elasticsearch/cluster/node/DiscoveryNodes.java b/core/src/main/java/org/elasticsearch/cluster/node/DiscoveryNodes.java index 04be959cd843a..ba913c89c3f0b 100644 --- a/core/src/main/java/org/elasticsearch/cluster/node/DiscoveryNodes.java +++ b/core/src/main/java/org/elasticsearch/cluster/node/DiscoveryNodes.java @@ -638,7 +638,5 @@ public DiscoveryNodes build() { public static DiscoveryNodes readFrom(StreamInput in, @Nullable DiscoveryNode localNode) throws IOException { return PROTO.readFrom(in, localNode); } - - } } From 56d68d5e0d0c15f43dc2284ef2c8d4275c1e3e99 Mon Sep 17 00:00:00 2001 From: Boaz Leskes Date: Sun, 17 Apr 2016 13:36:51 +0300 Subject: [PATCH 13/34] private things private --- .../main/java/org/elasticsearch/env/NodeEnvironment.java | 6 ++---- 1 file changed, 2 insertions(+), 4 deletions(-) diff --git a/core/src/main/java/org/elasticsearch/env/NodeEnvironment.java b/core/src/main/java/org/elasticsearch/env/NodeEnvironment.java index 059990c2450f2..d4e956376f9ed 100644 --- a/core/src/main/java/org/elasticsearch/env/NodeEnvironment.java +++ b/core/src/main/java/org/elasticsearch/env/NodeEnvironment.java @@ -255,11 +255,9 @@ public NodeEnvironment(Settings settings, Environment environment) throws IOExce /** * scans the node paths and loads existing metaData file. If not found a new meta data will be generated * and persisted into the nodePaths - * */ - // package private for testing - static NodeMetaData loadOrCreateNodeMetaData(Settings settings, ESLogger logger, - NodePath... nodePaths) throws IOException { + private static NodeMetaData loadOrCreateNodeMetaData(Settings settings, ESLogger logger, + NodePath... nodePaths) throws IOException { List pathList = Arrays.stream(nodePaths).map(np -> np.path).collect(Collectors.toList()); final Path[] paths = pathList.toArray(new Path[pathList.size()]); NodeMetaData metaData = NodeMetaData.FORMAT.loadLatestState(logger, paths); From 881b24f81ade33ca21f65e9b87b105222507c018 Mon Sep 17 00:00:00 2001 From: Boaz Leskes Date: Sun, 17 Apr 2016 13:38:06 +0300 Subject: [PATCH 14/34] woops --- core/src/main/java/org/elasticsearch/env/NodeMetaData.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/core/src/main/java/org/elasticsearch/env/NodeMetaData.java b/core/src/main/java/org/elasticsearch/env/NodeMetaData.java index 045c6b6b85b9f..143a924224334 100644 --- a/core/src/main/java/org/elasticsearch/env/NodeMetaData.java +++ b/core/src/main/java/org/elasticsearch/env/NodeMetaData.java @@ -36,7 +36,7 @@ */ public final class NodeMetaData { - private static final String NODE_ID_KEY = "allocation_id"; + private static final String NODE_ID_KEY = "node_id"; private final String nodeID; From 7e96a08f235de3b271e2921de5197a1433bf818f Mon Sep 17 00:00:00 2001 From: Boaz Leskes Date: Sun, 17 Apr 2016 13:39:34 +0300 Subject: [PATCH 15/34] tweaks --- .../main/java/org/elasticsearch/gateway/MetaStateService.java | 3 +-- core/src/main/java/org/elasticsearch/node/Node.java | 3 +-- 2 files changed, 2 insertions(+), 4 deletions(-) diff --git a/core/src/main/java/org/elasticsearch/gateway/MetaStateService.java b/core/src/main/java/org/elasticsearch/gateway/MetaStateService.java index a9835097d2b97..f4bbf69570cf5 100644 --- a/core/src/main/java/org/elasticsearch/gateway/MetaStateService.java +++ b/core/src/main/java/org/elasticsearch/gateway/MetaStateService.java @@ -123,8 +123,7 @@ void writeIndex(String reason, IndexMetaData indexMetaData) throws IOException { final Index index = indexMetaData.getIndex(); logger.trace("[{}] writing state, reason [{}]", index, reason); try { - IndexMetaData.FORMAT.write(indexMetaData, - nodeEnv.indexPaths(indexMetaData.getIndex())); + IndexMetaData.FORMAT.write(indexMetaData, nodeEnv.indexPaths(indexMetaData.getIndex())); } catch (Throwable ex) { logger.warn("[{}]: failed to write index state", ex, index); throw new IOException("failed to write state for [" + index + "]", ex); diff --git a/core/src/main/java/org/elasticsearch/node/Node.java b/core/src/main/java/org/elasticsearch/node/Node.java index 892545e99fb20..ad01c6d3f1c78 100644 --- a/core/src/main/java/org/elasticsearch/node/Node.java +++ b/core/src/main/java/org/elasticsearch/node/Node.java @@ -148,8 +148,7 @@ public class Node implements Closeable { * and {@link #NODE_MASTER_SETTING} must also be false. * */ - public static final Setting NODE_LOCAL_STORAGE_SETTING = - Setting.boolSetting("node.local_storage", true, Property.NodeScope); + public static final Setting NODE_LOCAL_STORAGE_SETTING = Setting.boolSetting("node.local_storage", true, Property.NodeScope); public static final Setting NODE_NAME_SETTING = Setting.simpleString("node.name", Property.NodeScope); public static final Setting NODE_ATTRIBUTES = Setting.groupSetting("node.attr.", Property.NodeScope); From 79687369ca96ec08319974a1a801c90057e6c85e Mon Sep 17 00:00:00 2001 From: Boaz Leskes Date: Sun, 17 Apr 2016 13:42:03 +0300 Subject: [PATCH 16/34] more renaming of `add_lock_id_to_custom_path` --- .../org/elasticsearch/index/IndexWithShadowReplicasIT.java | 2 +- docs/reference/indices/shadow-replicas.asciidoc | 6 +++--- 2 files changed, 4 insertions(+), 4 deletions(-) diff --git a/core/src/test/java/org/elasticsearch/index/IndexWithShadowReplicasIT.java b/core/src/test/java/org/elasticsearch/index/IndexWithShadowReplicasIT.java index 648c46b64b415..389db1fc7280c 100644 --- a/core/src/test/java/org/elasticsearch/index/IndexWithShadowReplicasIT.java +++ b/core/src/test/java/org/elasticsearch/index/IndexWithShadowReplicasIT.java @@ -454,7 +454,7 @@ public void run() { public void testPrimaryRelocationWhereRecoveryFails() throws Exception { Path dataPath = createTempDir(); Settings nodeSettings = Settings.builder() - .put("node.add_id_to_custom_path", false) + .put("node.add_lock_id_to_custom_path", false) .put(Environment.PATH_SHARED_DATA_SETTING.getKey(), dataPath) .build(); diff --git a/docs/reference/indices/shadow-replicas.asciidoc b/docs/reference/indices/shadow-replicas.asciidoc index 60360c147b575..3a0b23852b0cc 100644 --- a/docs/reference/indices/shadow-replicas.asciidoc +++ b/docs/reference/indices/shadow-replicas.asciidoc @@ -10,12 +10,12 @@ index. In order to fully utilize the `index.data_path` and `index.shadow_replicas` settings, you need to allow Elasticsearch to use the same data directory for -multiple instances by setting `node.add_id_to_custom_path` to false in +multiple instances by setting `node.add_lock_id_to_custom_path` to false in elasticsearch.yml: [source,yaml] -------------------------------------------------- -node.add_id_to_custom_path: false +node.add_lock_id_to_custom_path: false -------------------------------------------------- You will also need to indicate to the security manager where the custom indices @@ -114,7 +114,7 @@ settings API: These are non-dynamic settings that need to be configured in `elasticsearch.yml` -`node.add_id_to_custom_path`:: +`node.add_lock_id_to_custom_path`:: Boolean setting indicating whether Elasticsearch should append the node's ordinal to the custom data path. For example, if this is enabled and a path of "/tmp/foo" is used, the first locally-running node will use "/tmp/foo/0", From bb6f0622a44e5603cfb0da6ecad3e512fc56d05a Mon Sep 17 00:00:00 2001 From: Boaz Leskes Date: Tue, 19 Apr 2016 19:57:35 +0300 Subject: [PATCH 17/34] add protect against same node id in local discovery --- .../org/elasticsearch/discovery/local/LocalDiscovery.java | 8 ++++++++ 1 file changed, 8 insertions(+) diff --git a/core/src/main/java/org/elasticsearch/discovery/local/LocalDiscovery.java b/core/src/main/java/org/elasticsearch/discovery/local/LocalDiscovery.java index 03f27d093e893..a5712e705cfb0 100644 --- a/core/src/main/java/org/elasticsearch/discovery/local/LocalDiscovery.java +++ b/core/src/main/java/org/elasticsearch/discovery/local/LocalDiscovery.java @@ -46,6 +46,7 @@ import org.elasticsearch.discovery.DiscoveryStats; import java.util.HashSet; +import java.util.Optional; import java.util.Queue; import java.util.Set; import java.util.concurrent.ConcurrentMap; @@ -102,6 +103,13 @@ public void startInitialJoin() { } logger.debug("Connected to cluster [{}]", clusterName); + Optional current = clusterGroup.members().stream().filter(other -> other.localNode().equals(this.localNode())) + .findFirst(); + if (current.isPresent()) { + throw new IllegalStateException("current cluster group already contains a node with the same id. current " + + current.get().localNode() + ", this node " + localNode()); + } + clusterGroup.members().add(this); LocalDiscovery firstMaster = null; From 76f1a1d7b17ee515641201ae0cd7bfbc61bc69ff Mon Sep 17 00:00:00 2001 From: Boaz Leskes Date: Tue, 19 Apr 2016 20:23:36 +0300 Subject: [PATCH 18/34] apply feedback from @ywelsch --- .../resources/checkstyle_suppressions.xml | 3 -- .../cluster/node/DiscoveryNodes.java | 16 +++++--- .../discovery/zen/NodeJoinController.java | 21 ++++++---- .../elasticsearch/env/NodeEnvironment.java | 7 ++-- .../org/elasticsearch/env/NodeMetaData.java | 29 +++++++------- .../java/org/elasticsearch/node/Node.java | 4 +- .../zen/NodeJoinControllerTests.java | 38 ++++++++++++++----- .../env/NodeEnvironmentTests.java | 10 ++--- 8 files changed, 77 insertions(+), 51 deletions(-) diff --git a/buildSrc/src/main/resources/checkstyle_suppressions.xml b/buildSrc/src/main/resources/checkstyle_suppressions.xml index 7d3322cc603e6..f06bdadd78b42 100644 --- a/buildSrc/src/main/resources/checkstyle_suppressions.xml +++ b/buildSrc/src/main/resources/checkstyle_suppressions.xml @@ -298,7 +298,6 @@ - @@ -380,7 +379,6 @@ - @@ -979,7 +977,6 @@ - diff --git a/core/src/main/java/org/elasticsearch/cluster/node/DiscoveryNodes.java b/core/src/main/java/org/elasticsearch/cluster/node/DiscoveryNodes.java index ba913c89c3f0b..6ee1cc809baeb 100644 --- a/core/src/main/java/org/elasticsearch/cluster/node/DiscoveryNodes.java +++ b/core/src/main/java/org/elasticsearch/cluster/node/DiscoveryNodes.java @@ -22,7 +22,6 @@ import com.carrotsearch.hppc.ObjectHashSet; import com.carrotsearch.hppc.cursors.ObjectCursor; import com.carrotsearch.hppc.cursors.ObjectObjectCursor; -import org.elasticsearch.ElasticsearchException; import org.elasticsearch.Version; import org.elasticsearch.cluster.AbstractDiffable; import org.elasticsearch.common.Booleans; @@ -248,7 +247,8 @@ public Version getSmallestNonClientNodeVersion() { public DiscoveryNode resolveNode(String node) { String[] resolvedNodeIds = resolveNodesIds(node); if (resolvedNodeIds.length > 1) { - throw new IllegalArgumentException("resolved [" + node + "] into [" + resolvedNodeIds.length + "] nodes, where expected to be resolved to a single node"); + throw new IllegalArgumentException("resolved [" + node + "] into [" + resolvedNodeIds.length + + "] nodes, where expected to be resolved to a single node"); } if (resolvedNodeIds.length == 0) { throw new IllegalArgumentException("failed to resolve [" + node + "], no matching nodes"); @@ -371,7 +371,8 @@ public Delta delta(DiscoveryNodes other) { newMasterNode = getMasterNode(); } } - return new Delta(previousMasterNode, newMasterNode, localNodeId, Collections.unmodifiableList(removed), Collections.unmodifiableList(added)); + return new Delta(previousMasterNode, newMasterNode, localNodeId, Collections.unmodifiableList(removed), + Collections.unmodifiableList(added)); } @Override @@ -413,7 +414,8 @@ public Delta(String localNodeId, List removed, List removed, List added) { + public Delta(@Nullable DiscoveryNode previousMasterNode, @Nullable DiscoveryNode newMasterNode, String localNodeId, + List removed, List added) { this.previousMasterNode = previousMasterNode; this.newMasterNode = newMasterNode; this.localNodeId = localNodeId; @@ -568,7 +570,7 @@ public Builder(DiscoveryNodes nodes) { public Builder put(DiscoveryNode node) { final String preflight = preflightPut(node); if (preflight != null) { - throw new ElasticsearchException(preflight); + throw new IllegalArgumentException(preflight); } nodes.put(node.getId(), node); return this; @@ -604,6 +606,10 @@ public String preflightPut(DiscoveryNode node) { node.equals(existingNode) == false) { return "can't add node " + node +", found existing node " + existingNode + " with same address"; } + if (node.getId().equals(existingNode.getId()) && + node.getAddress().equals(existingNode.getAddress()) == false) { + return "can't add node " + node +", found existing node " + existingNode + " with the same id, but a different address"; + } } return null; } diff --git a/core/src/main/java/org/elasticsearch/discovery/zen/NodeJoinController.java b/core/src/main/java/org/elasticsearch/discovery/zen/NodeJoinController.java index cc1c1e409215b..952eb086c3748 100644 --- a/core/src/main/java/org/elasticsearch/discovery/zen/NodeJoinController.java +++ b/core/src/main/java/org/elasticsearch/discovery/zen/NodeJoinController.java @@ -63,7 +63,8 @@ public class NodeJoinController extends AbstractComponent { protected final Map> pendingJoinRequests = new HashMap<>(); - public NodeJoinController(ClusterService clusterService, RoutingService routingService, DiscoverySettings discoverySettings, Settings settings) { + public NodeJoinController(ClusterService clusterService, RoutingService routingService, DiscoverySettings discoverySettings, + Settings settings) { super(settings); this.clusterService = clusterService; this.routingService = routingService; @@ -152,7 +153,8 @@ public ClusterState execute(ClusterState currentState) throws Exception { @Override public void onFailure(String source, Throwable updateFailure) { - logger.warn("unexpected error while trying to fail election context due to [{}]. original exception [{}]", updateFailure, reason, throwable); + logger.warn("unexpected error while trying to fail election context due to [{}]. original exception [{}]", updateFailure, + reason, throwable); context.onFailure(updateFailure); } }); @@ -243,13 +245,16 @@ public ClusterState execute(ClusterState currentState) { if (currentState.nodes().getMasterNode() != null) { // TODO can we tie break here? we don't have a remote master cluster state version to decide on - logger.trace("join thread elected local node as master, but there is already a master in place: {}", currentState.nodes().getMasterNode()); + logger.trace("join thread elected local node as master, but there is already a master in place: {}", + currentState.nodes().getMasterNode()); throw new NotMasterException("Node [" + clusterService.localNode() + "] not master for join request"); } - DiscoveryNodes.Builder builder = new DiscoveryNodes.Builder(currentState.nodes()).masterNodeId(currentState.nodes().getLocalNode().getId()); + DiscoveryNodes.Builder builder = new DiscoveryNodes.Builder(currentState.nodes()) + .masterNodeId(currentState.nodes().getLocalNode().getId()); // update the fact that we are the master... - ClusterBlocks clusterBlocks = ClusterBlocks.builder().blocks(currentState.blocks()).removeGlobalBlock(discoverySettings.getNoMasterBlock()).build(); + ClusterBlocks clusterBlocks = ClusterBlocks.builder().blocks(currentState.blocks()) + .removeGlobalBlock(discoverySettings.getNoMasterBlock()).build(); currentState = ClusterState.builder(currentState).nodes(builder).blocks(clusterBlocks).build(); // reroute now to remove any dead nodes (master may have stepped down when they left and didn't update the routing table) @@ -374,7 +379,8 @@ public ClusterState execute(ClusterState currentState) { } nodesBuilder = DiscoveryNodes.builder(currentState.nodes()); - Iterator>> iterator = pendingJoinRequests.entrySet().iterator(); + Iterator>> iterator = + pendingJoinRequests.entrySet().iterator(); while (iterator.hasNext()) { Map.Entry> entry = iterator.next(); final DiscoveryNode node = entry.getKey(); @@ -415,7 +421,8 @@ public ClusterState execute(ClusterState currentState) { public void onNoLongerMaster(String source) { // we are rejected, so drain all pending task (execute never run) synchronized (pendingJoinRequests) { - Iterator>> iterator = pendingJoinRequests.entrySet().iterator(); + Iterator>> iterator = + pendingJoinRequests.entrySet().iterator(); while (iterator.hasNext()) { Map.Entry> entry = iterator.next(); joinCallbacksToRespondTo.addAll(entry.getValue()); diff --git a/core/src/main/java/org/elasticsearch/env/NodeEnvironment.java b/core/src/main/java/org/elasticsearch/env/NodeEnvironment.java index d4e956376f9ed..45cd642b9a741 100644 --- a/core/src/main/java/org/elasticsearch/env/NodeEnvironment.java +++ b/core/src/main/java/org/elasticsearch/env/NodeEnvironment.java @@ -155,8 +155,7 @@ public String toString() { public static final Setting NODE_ID_SEED_SETTING = - // don't use node.id.seed so it won't be seen as an attribute - Setting.longSetting("node_id.seed", 0L, Long.MIN_VALUE, Property.NodeScope); + Setting.longSetting("node.id.seed", 0L, Long.MIN_VALUE, Property.NodeScope); /** @@ -678,11 +677,11 @@ public Path[] nodeDataPaths() { * returns the unique uuid describing this node. The uuid is persistent in the data folder of this node * and remains across restarts. **/ - public String nodeID() { + public String nodeId() { // we currently only return the ID and hide the underlying nodeMetaData implementation in order to avoid // confusion with other "metadata" like node settings found in elasticsearch.yml. In future // we can encapsulate both (and more) in one NodeMetaData (or NodeSettings) object ala IndexSettings - return nodeMetaData.nodeID(); + return nodeMetaData.nodeId(); } /** diff --git a/core/src/main/java/org/elasticsearch/env/NodeMetaData.java b/core/src/main/java/org/elasticsearch/env/NodeMetaData.java index 143a924224334..c4d7ea43a831c 100644 --- a/core/src/main/java/org/elasticsearch/env/NodeMetaData.java +++ b/core/src/main/java/org/elasticsearch/env/NodeMetaData.java @@ -38,11 +38,10 @@ public final class NodeMetaData { private static final String NODE_ID_KEY = "node_id"; - private final String nodeID; + private final String nodeId; - public NodeMetaData(final String nodeID) { - Objects.requireNonNull(nodeID); - this.nodeID = nodeID; + public NodeMetaData(final String nodeId) { + this.nodeId = Objects.requireNonNull(nodeId); } @Override @@ -56,39 +55,39 @@ public boolean equals(Object o) { NodeMetaData that = (NodeMetaData) o; - return Objects.equals(this.nodeID, that.nodeID); + return Objects.equals(this.nodeId, that.nodeId); } @Override public int hashCode() { - return this.nodeID.hashCode(); + return this.nodeId.hashCode(); } @Override public String toString() { - return "node_id [" + nodeID + "]"; + return "node_id [" + nodeId + "]"; } private static ObjectParser PARSER = new ObjectParser<>("node_meta_data", Builder::new); static { - PARSER.declareString(Builder::setNodeID, new ParseField(NODE_ID_KEY)); + PARSER.declareString(Builder::setnodeId, new ParseField(NODE_ID_KEY)); } - public String nodeID() { - return nodeID; + public String nodeId() { + return nodeId; } private static class Builder { - String nodeID; + String nodeId; - public void setNodeID(String nodeID) { - this.nodeID = nodeID; + public void setnodeId(String nodeId) { + this.nodeId = nodeId; } public NodeMetaData build() { - return new NodeMetaData(nodeID); + return new NodeMetaData(nodeId); } } @@ -104,7 +103,7 @@ protected XContentBuilder newXContentBuilder(XContentType type, OutputStream str @Override public void toXContent(XContentBuilder builder, NodeMetaData nodeMetaData) throws IOException { - builder.field(NODE_ID_KEY, nodeMetaData.nodeID); + builder.field(NODE_ID_KEY, nodeMetaData.nodeId); } @Override diff --git a/core/src/main/java/org/elasticsearch/node/Node.java b/core/src/main/java/org/elasticsearch/node/Node.java index ad01c6d3f1c78..72403a67e5d33 100644 --- a/core/src/main/java/org/elasticsearch/node/Node.java +++ b/core/src/main/java/org/elasticsearch/node/Node.java @@ -143,7 +143,7 @@ public class Node implements Closeable { /** * controls whether the node is allowed to persist things like metadata to disk - * Note that this does not control whether the node store actual indices (see + * Note that this does not control whether the node stores actual indices (see * {@link #NODE_DATA_SETTING}). However, if this is false, {@link #NODE_DATA_SETTING} * and {@link #NODE_MASTER_SETTING} must also be false. * @@ -334,7 +334,7 @@ public Node start() { DiscoveryNode localNode = injector.getInstance(DiscoveryNodeService.class).buildLocalNode( transportService.boundAddress().publishAddress(), - injector.getInstance(NodeEnvironment.class)::nodeID); + injector.getInstance(NodeEnvironment.class)::nodeId); // TODO: need to find a cleaner way to start/construct a service with some initial parameters, // playing nice with the life cycle interfaces diff --git a/core/src/test/java/org/elasticsearch/discovery/zen/NodeJoinControllerTests.java b/core/src/test/java/org/elasticsearch/discovery/zen/NodeJoinControllerTests.java index 6959003edebc8..4ebcfbca5d1a7 100644 --- a/core/src/test/java/org/elasticsearch/discovery/zen/NodeJoinControllerTests.java +++ b/core/src/test/java/org/elasticsearch/discovery/zen/NodeJoinControllerTests.java @@ -205,10 +205,12 @@ public void onFailure(Throwable t) { @Override protected void doRun() throws Exception { - nodeJoinController.waitToBeElectedAsMaster(requiredJoins, TimeValue.timeValueHours(30), new NodeJoinController.ElectionCallback() { + nodeJoinController.waitToBeElectedAsMaster(requiredJoins, TimeValue.timeValueHours(30), + new NodeJoinController.ElectionCallback() { @Override public void onElectedAsMaster(ClusterState state) { - assertThat("callback called with elected as master, but state disagrees", state.nodes().isLocalNodeElectedMaster(), equalTo(true)); + assertThat("callback called with elected as master, but state disagrees", state.nodes().isLocalNodeElectedMaster(), + equalTo(true)); electionFuture.markAsDone(); } @@ -253,10 +255,12 @@ public void onFailure(Throwable t) { @Override protected void doRun() throws Exception { - nodeJoinController.waitToBeElectedAsMaster(requiredJoins, TimeValue.timeValueHours(30), new NodeJoinController.ElectionCallback() { + nodeJoinController.waitToBeElectedAsMaster(requiredJoins, TimeValue.timeValueHours(30), + new NodeJoinController.ElectionCallback() { @Override public void onElectedAsMaster(ClusterState state) { - assertThat("callback called with elected as master, but state disagrees", state.nodes().isLocalNodeElectedMaster(), equalTo(true)); + assertThat("callback called with elected as master, but state disagrees", state.nodes().isLocalNodeElectedMaster(), + equalTo(true)); electionFuture.markAsDone(); } @@ -298,7 +302,8 @@ public void onFailure(Throwable t) { } logger.debug("--> asserting master election didn't finish yet"); - assertThat("election finished after [" + initialJoins + "] master nodes but required joins is [" + requiredJoins + "]", electionFuture.isDone(), equalTo(false)); + assertThat("election finished after [" + initialJoins + "] master nodes but required joins is [" + requiredJoins + "]", + electionFuture.isDone(), equalTo(false)); final int finalJoins = requiredJoins - initialJoins + randomInt(5); nodesToJoin.clear(); @@ -374,7 +379,8 @@ public void testMasterElectionTimeout() throws InterruptedException { nodeJoinController.waitToBeElectedAsMaster(requiredJoins, TimeValue.timeValueMillis(1), new NodeJoinController.ElectionCallback() { @Override public void onElectedAsMaster(ClusterState state) { - assertThat("callback called with elected as master, but state disagrees", state.nodes().isLocalNodeElectedMaster(), equalTo(true)); + assertThat("callback called with elected as master, but state disagrees", state.nodes().isLocalNodeElectedMaster(), + equalTo(true)); latch.countDown(); } @@ -413,7 +419,7 @@ public void testNewClusterStateOnExistingNodeJoin() throws InterruptedException, assertTrue("failed to publish a new state upon existing join", clusterService.state() != state); } - public void testRejectingJoinWithSameAddress() throws InterruptedException, ExecutionException { + public void testRejectingJoinWithSameAddressButDifferentId() throws InterruptedException, ExecutionException { ClusterState state = clusterService.state(); final DiscoveryNode other_node = new DiscoveryNode("other_node", state.nodes().getLocalNode().getAddress(), emptyMap(), emptySet(), Version.CURRENT); @@ -422,6 +428,15 @@ public void testRejectingJoinWithSameAddress() throws InterruptedException, Exec assertThat(e.getMessage(), containsString("found existing node")); } + public void testRejectingJoinWithSameIdButDifferentAddress() throws InterruptedException, ExecutionException { + ClusterState state = clusterService.state(); + final DiscoveryNode other_node = new DiscoveryNode(state.nodes().getLocalNode().getId(), LocalTransportAddress.buildUnique(), + emptyMap(), emptySet(), Version.CURRENT); + + ExecutionException e = expectThrows(ExecutionException.class, () -> joinNode(other_node)); + assertThat(e.getMessage(), containsString("found existing node")); + } + public void testNormalConcurrentJoins() throws InterruptedException { Thread[] threads = new Thread[3 + randomInt(5)]; ArrayList nodes = new ArrayList<>(); @@ -501,7 +516,8 @@ protected void doRun() throws Exception { nodeJoinController.waitToBeElectedAsMaster(requiredJoins, TimeValue.timeValueHours(30), new NodeJoinController.ElectionCallback() { @Override public void onElectedAsMaster(ClusterState state) { - assertThat("callback called with elected as master, but state disagrees", state.nodes().isLocalNodeElectedMaster(), equalTo(true)); + assertThat("callback called with elected as master, but state disagrees", state.nodes().isLocalNodeElectedMaster(), + equalTo(true)); latch.countDown(); } @@ -544,12 +560,14 @@ public NoopAllocationService(Settings settings) { } @Override - public RoutingAllocation.Result applyStartedShards(ClusterState clusterState, List startedShards, boolean withReroute) { + public RoutingAllocation.Result applyStartedShards(ClusterState clusterState, List startedShards, + boolean withReroute) { return new RoutingAllocation.Result(false, clusterState.routingTable(), clusterState.metaData()); } @Override - public RoutingAllocation.Result applyFailedShards(ClusterState clusterState, List failedShards) { + public RoutingAllocation.Result applyFailedShards(ClusterState clusterState, + List failedShards) { return new RoutingAllocation.Result(false, clusterState.routingTable(), clusterState.metaData()); } diff --git a/core/src/test/java/org/elasticsearch/env/NodeEnvironmentTests.java b/core/src/test/java/org/elasticsearch/env/NodeEnvironmentTests.java index 315a45b98fe13..fec194a28064e 100644 --- a/core/src/test/java/org/elasticsearch/env/NodeEnvironmentTests.java +++ b/core/src/test/java/org/elasticsearch/env/NodeEnvironmentTests.java @@ -412,17 +412,17 @@ public void testPersistentNodeId() throws IOException { .put("node.master", false) .put("node.data", false) .build()); - String nodeID = env.nodeID(); + String nodeID = env.nodeId(); env.close(); env = newNodeEnvironment(paths, Settings.EMPTY); - assertThat("previous node didn't have local storage enabled, id should change", env.nodeID(), not(equalTo(nodeID))); - nodeID = env.nodeID(); + assertThat("previous node didn't have local storage enabled, id should change", env.nodeId(), not(equalTo(nodeID))); + nodeID = env.nodeId(); env.close(); env = newNodeEnvironment(paths, Settings.EMPTY); - assertThat(env.nodeID(), equalTo(nodeID)); + assertThat(env.nodeId(), equalTo(nodeID)); env.close(); env = newNodeEnvironment(Settings.EMPTY); - assertThat(env.nodeID(), not(equalTo(nodeID))); + assertThat(env.nodeId(), not(equalTo(nodeID))); env.close(); } From fce829d95595e275aaca6e05bf1b460ef10013fb Mon Sep 17 00:00:00 2001 From: Boaz Leskes Date: Tue, 19 Apr 2016 20:31:12 +0300 Subject: [PATCH 19/34] fix ClusterStateDiffIT --- .../test/java/org/elasticsearch/cluster/ClusterStateDiffIT.java | 1 + 1 file changed, 1 insertion(+) diff --git a/core/src/test/java/org/elasticsearch/cluster/ClusterStateDiffIT.java b/core/src/test/java/org/elasticsearch/cluster/ClusterStateDiffIT.java index 8a634474ae1e1..eeb32a454f710 100644 --- a/core/src/test/java/org/elasticsearch/cluster/ClusterStateDiffIT.java +++ b/core/src/test/java/org/elasticsearch/cluster/ClusterStateDiffIT.java @@ -190,6 +190,7 @@ private ClusterState.Builder randomNodes(ClusterState clusterState) { if (randomBoolean()) { nodes.remove(nodeId); } else { + nodes.remove(nodeId); // simulate a node restart, where it got a new address; nodes.put(new DiscoveryNode(nodeId, new LocalTransportAddress(randomAsciiOfLength(10)), emptyMap(), emptySet(), randomVersion(random()))); } From afb14c3987be88d7f51b40df38843a427f1456b9 Mon Sep 17 00:00:00 2001 From: Boaz Leskes Date: Tue, 19 Apr 2016 22:19:20 +0300 Subject: [PATCH 20/34] when resetting shared test cluster, shut down new nodes first as they might have reused an node id of the initial shared nodes. --- .../test/InternalTestCluster.java | 41 +++++++------------ 1 file changed, 14 insertions(+), 27 deletions(-) diff --git a/test/framework/src/main/java/org/elasticsearch/test/InternalTestCluster.java b/test/framework/src/main/java/org/elasticsearch/test/InternalTestCluster.java index 7691fc2d01bcb..328ceeb2ead9b 100644 --- a/test/framework/src/main/java/org/elasticsearch/test/InternalTestCluster.java +++ b/test/framework/src/main/java/org/elasticsearch/test/InternalTestCluster.java @@ -921,55 +921,42 @@ private synchronized void reset(boolean wipeData) throws IOException { } logger.debug("Cluster is NOT consistent - restarting shared nodes - nodes: [{}] nextNodeId: [{}] numSharedNodes: [{}]", nodes.keySet(), nextNodeId.get(), sharedNodesSeeds.length); - - Set sharedNodes = new HashSet<>(); assert sharedNodesSeeds.length == numSharedAllRolesNodes + numShareCoordOnlyNodes; - boolean changed = false; + + // close down new nodes, so the shared nodes can reclaim their data folders + for (int i = sharedNodesSeeds.length; i < nextNodeId.get(); i++) { + String buildNodeName = buildNodeName(i); + NodeAndClient nodeAndClient = nodes.get(buildNodeName); + if (nodeAndClient != null) { + logger.debug("Close Node [{}] not shared", nodeAndClient.name); + nodeAndClient.close(); + nodes.remove(buildNodeName); + } + } + for (int i = 0; i < numSharedAllRolesNodes; i++) { String buildNodeName = buildNodeName(i); NodeAndClient nodeAndClient = nodes.get(buildNodeName); if (nodeAndClient == null) { - changed = true; nodeAndClient = buildNode(i, sharedNodesSeeds[i], null, Version.CURRENT); nodeAndClient.node.start(); logger.info("Start Shared Node [{}] not shared", nodeAndClient.name); + publishNode(nodeAndClient); } - sharedNodes.add(nodeAndClient); } for (int i = numSharedAllRolesNodes; i < numSharedAllRolesNodes + numShareCoordOnlyNodes; i++) { String buildNodeName = buildNodeName(i); NodeAndClient nodeAndClient = nodes.get(buildNodeName); if (nodeAndClient == null) { - changed = true; Builder clientSettingsBuilder = Settings.builder().put(Node.NODE_MASTER_SETTING.getKey(), false) .put(Node.NODE_DATA_SETTING.getKey(), false).put(Node.NODE_INGEST_SETTING.getKey(), false); nodeAndClient = buildNode(i, sharedNodesSeeds[i], clientSettingsBuilder.build(), Version.CURRENT); nodeAndClient.node.start(); logger.info("Start Shared Node [{}] not shared", nodeAndClient.name); + publishNode(nodeAndClient); } - sharedNodes.add(nodeAndClient); - } - if (!changed && sharedNodes.size() == nodes.size()) { - logger.debug("Cluster is consistent - moving out - nodes: [{}] nextNodeId: [{}] numSharedNodes: [{}]", nodes.keySet(), nextNodeId.get(), sharedNodesSeeds.length); - if (size() > 0) { - client().admin().cluster().prepareHealth().setWaitForNodes(Integer.toString(sharedNodesSeeds.length)).get(); - } - return; // we are consistent - return - } - for (NodeAndClient nodeAndClient : sharedNodes) { - nodes.remove(nodeAndClient.name); } - // trash the remaining nodes - final Collection toShutDown = nodes.values(); - for (NodeAndClient nodeAndClient : toShutDown) { - logger.debug("Close Node [{}] not shared", nodeAndClient.name); - nodeAndClient.close(); - } - nodes.clear(); - for (NodeAndClient nodeAndClient : sharedNodes) { - publishNode(nodeAndClient); - } nextNodeId.set(sharedNodesSeeds.length); assert size() == sharedNodesSeeds.length; if (size() > 0) { From 697419c9a9e651998f0dcc70378990009ef119fc Mon Sep 17 00:00:00 2001 From: Boaz Leskes Date: Tue, 19 Apr 2016 22:57:45 +0300 Subject: [PATCH 21/34] fix some references to node_id.seed --- .../test/java/org/elasticsearch/tribe/TribeUnitTests.java | 8 ++++---- .../resources/org/elasticsearch/tribe/elasticsearch.yml | 4 ++-- 2 files changed, 6 insertions(+), 6 deletions(-) diff --git a/qa/evil-tests/src/test/java/org/elasticsearch/tribe/TribeUnitTests.java b/qa/evil-tests/src/test/java/org/elasticsearch/tribe/TribeUnitTests.java index 0a1698efcab6e..20a8ec67773e6 100644 --- a/qa/evil-tests/src/test/java/org/elasticsearch/tribe/TribeUnitTests.java +++ b/qa/evil-tests/src/test/java/org/elasticsearch/tribe/TribeUnitTests.java @@ -88,8 +88,8 @@ public void testThatTribeClientsIgnoreGlobalSysProps() throws Exception { System.setProperty("es.cluster.name", "tribe_node_cluster"); System.setProperty("es.tribe.t1.cluster.name", "tribe1"); System.setProperty("es.tribe.t2.cluster.name", "tribe2"); - System.setProperty("es.tribe.t1.node_id.seed", Long.toString(random().nextLong())); - System.setProperty("es.tribe.t2.node_id.seed", Long.toString(random().nextLong())); + System.setProperty("es.tribe.t1.node.id.seed", Long.toString(random().nextLong())); + System.setProperty("es.tribe.t2.node.id.seed", Long.toString(random().nextLong())); try { assertTribeNodeSuccessfullyCreated(Settings.EMPTY); @@ -97,8 +97,8 @@ public void testThatTribeClientsIgnoreGlobalSysProps() throws Exception { System.clearProperty("es.cluster.name"); System.clearProperty("es.tribe.t1.cluster.name"); System.clearProperty("es.tribe.t2.cluster.name"); - System.clearProperty("es.tribe.t1.node_id.seed"); - System.clearProperty("es.tribe.t2.node_id.seed"); + System.clearProperty("es.tribe.t1.node.id.seed"); + System.clearProperty("es.tribe.t2.node.id.seed"); } } diff --git a/qa/evil-tests/src/test/resources/org/elasticsearch/tribe/elasticsearch.yml b/qa/evil-tests/src/test/resources/org/elasticsearch/tribe/elasticsearch.yml index d4fa8d8d13031..19b2a7b5dd92f 100644 --- a/qa/evil-tests/src/test/resources/org/elasticsearch/tribe/elasticsearch.yml +++ b/qa/evil-tests/src/test/resources/org/elasticsearch/tribe/elasticsearch.yml @@ -1,5 +1,5 @@ cluster.name: tribe_node_cluster tribe.t1.cluster.name: tribe1 tribe.t2.cluster.name: tribe2 -tribe.t1.node_id.seed: 1 -tribe.t2.node_id.seed: 2 +tribe.t1.node.id.seed: 1 +tribe.t2.node.id.seed: 2 From 409029002517d88cfb90d6b14e54992d3b720c1b Mon Sep 17 00:00:00 2001 From: Boaz Leskes Date: Wed, 20 Apr 2016 11:40:07 +0300 Subject: [PATCH 22/34] do replace existing node to accomodate for a quick restart and change of meta data --- .../cluster/node/DiscoveryNode.java | 36 ++++++++ .../discovery/zen/NodeJoinController.java | 13 ++- .../cluster/node/DiscoveryNodeTests.java | 87 +++++++++++++++++++ .../zen/NodeJoinControllerTests.java | 18 ++++ 4 files changed, 153 insertions(+), 1 deletion(-) create mode 100644 core/src/test/java/org/elasticsearch/cluster/node/DiscoveryNodeTests.java diff --git a/core/src/main/java/org/elasticsearch/cluster/node/DiscoveryNode.java b/core/src/main/java/org/elasticsearch/cluster/node/DiscoveryNode.java index 5a44a867c111b..c445324421d94 100644 --- a/core/src/main/java/org/elasticsearch/cluster/node/DiscoveryNode.java +++ b/core/src/main/java/org/elasticsearch/cluster/node/DiscoveryNode.java @@ -299,7 +299,43 @@ public String getHostName() { public String getHostAddress() { return this.hostAddress; } + + /** + * Checks whether this node is has the same id as another node *and* has + * the same attributes, network address and so fourth. This is in contrast to {@link #equals(Object)} + * which only uses the ids. + */ + public boolean equalsIncludingMetaData(DiscoveryNode other) { + if (this.equals(other) == false) { + return false; + } + if (!nodeName.equals(other.nodeName)) { + return false; + } + if (!hostName.equals(other.hostName)) { + return false; + } + if (!hostAddress.equals(other.hostAddress)) { + return false; + } + if (!address.equals(other.address)) { + return false; + } + if (!attributes.equals(other.attributes)) { + return false; + } + if (!version.equals(other.version)) { + return false; + } + return roles.equals(other.roles); + } + + + /** + * Checks for equality based on the value {@link #getId()} **alone**. This is done so that this class can be used + * efficiently as a key in a map + */ @Override public boolean equals(Object obj) { if (!(obj instanceof DiscoveryNode)) { diff --git a/core/src/main/java/org/elasticsearch/discovery/zen/NodeJoinController.java b/core/src/main/java/org/elasticsearch/discovery/zen/NodeJoinController.java index 952eb086c3748..91820d3aeb110 100644 --- a/core/src/main/java/org/elasticsearch/discovery/zen/NodeJoinController.java +++ b/core/src/main/java/org/elasticsearch/discovery/zen/NodeJoinController.java @@ -392,7 +392,18 @@ public ClusterState execute(ClusterState currentState) { joinCallbacksToFail.add(new Tuple<>(callback, failure)); } } else if (currentState.nodes().nodeExists(node.getId())) { - logger.debug("received a join request for an existing node [{}]", node); + // it may be that a quick node restart can cause a join to arrive + // before the node leave has been processed. In that we need to check that things + // like attributes didn't change. + final DiscoveryNode existing = currentState.nodes().get(node.getId()); + if (existing.equalsIncludingMetaData(node)) { + logger.debug("received a join request for an existing node [{}]", node); + } else { + logger.debug("received a join request for an existing node, but with different meta data, replacing existing" + + " {} with new {}", existing, node); + nodeAdded = true; + nodesBuilder.put(node); + } joinCallbacksToRespondTo.addAll(entry.getValue()); } else { nodeAdded = true; diff --git a/core/src/test/java/org/elasticsearch/cluster/node/DiscoveryNodeTests.java b/core/src/test/java/org/elasticsearch/cluster/node/DiscoveryNodeTests.java new file mode 100644 index 0000000000000..c0f998e536e64 --- /dev/null +++ b/core/src/test/java/org/elasticsearch/cluster/node/DiscoveryNodeTests.java @@ -0,0 +1,87 @@ +/* + * Licensed to Elasticsearch under one or more contributor + * license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright + * ownership. Elasticsearch licenses this file to you under + * the Apache License, Version 2.0 (the "License"); you may + * not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ +package org.elasticsearch.cluster.node; + +import org.elasticsearch.common.transport.LocalTransportAddress; +import org.elasticsearch.test.ESTestCase; + +import java.util.Arrays; +import java.util.HashMap; +import java.util.HashSet; +import java.util.Map; +import java.util.concurrent.atomic.AtomicInteger; + +import static org.elasticsearch.test.VersionUtils.randomVersion; + +public class DiscoveryNodeTests extends ESTestCase { + + static AtomicInteger idGenerator = new AtomicInteger(); + + /** + * guarantees to generate a node that is unique for all aspects but for it's roles and version (which are chosen randomly) + */ + DiscoveryNode randomNode() { + Map attributes = new HashMap<>(); + for (int attr = (idGenerator.get() > 0 ? 1 : 0) + randomInt(3); + attr > 0; + attr--) { + attributes.put("attr_" + attr, "v_" + idGenerator.incrementAndGet()); + } + final DiscoveryNode node = new DiscoveryNode( + "name_" + idGenerator.incrementAndGet(), + "id_" + idGenerator.incrementAndGet(), + LocalTransportAddress.buildUnique(), + attributes, new HashSet<>(randomSubsetOf(Arrays.asList(DiscoveryNode.Role.values()))), + randomVersion(random())); + return node; + } + + public void testEquality() { + DiscoveryNode node1 = randomNode(); + DiscoveryNode node2 = randomNode(); + logger.info("node1: {}", node1); + logger.info("node2: {}", node2); + + assertFalse(node1.equals(node2)); + assertFalse(node1.equalsIncludingMetaData(node2)); + + DiscoveryNode sameIdDifferentMeta; + + sameIdDifferentMeta = new DiscoveryNode(node2.getName(), node1.getId(), node2.getAddress(), node2.getAttributes(), + node2.getRoles(), node2.getVersion()); + assertTrue(node1.equals(sameIdDifferentMeta)); + assertFalse(node1.equalsIncludingMetaData(sameIdDifferentMeta)); + + sameIdDifferentMeta = new DiscoveryNode(node2.getName(), node1.getId(), node1.getAddress(), node1.getAttributes(), + (randomBoolean() ? node1 : node2).getRoles(), (randomBoolean() ? node1 : node2).getVersion()); + assertTrue(node1.equals(sameIdDifferentMeta)); + assertFalse(node1.equalsIncludingMetaData(sameIdDifferentMeta)); + + sameIdDifferentMeta = new DiscoveryNode(node1.getName(), node1.getId(), node2.getAddress(), node1.getAttributes(), + (randomBoolean() ? node1 : node2).getRoles(), (randomBoolean() ? node1 : node2).getVersion()); + assertTrue(node1.equals(sameIdDifferentMeta)); + assertFalse(node1.equalsIncludingMetaData(sameIdDifferentMeta)); + + sameIdDifferentMeta = new DiscoveryNode(node1.getName(), node1.getId(), node1.getAddress(), node2.getAttributes(), + (randomBoolean() ? node1 : node2).getRoles(), (randomBoolean() ? node1 : node2).getVersion()); + assertTrue(node1.equals(sameIdDifferentMeta)); + assertFalse(node1.equalsIncludingMetaData(sameIdDifferentMeta)); + + } +} diff --git a/core/src/test/java/org/elasticsearch/discovery/zen/NodeJoinControllerTests.java b/core/src/test/java/org/elasticsearch/discovery/zen/NodeJoinControllerTests.java index 4ebcfbca5d1a7..9ea3883b40fb7 100644 --- a/core/src/test/java/org/elasticsearch/discovery/zen/NodeJoinControllerTests.java +++ b/core/src/test/java/org/elasticsearch/discovery/zen/NodeJoinControllerTests.java @@ -40,6 +40,7 @@ import org.elasticsearch.discovery.DiscoverySettings; import org.elasticsearch.discovery.zen.membership.MembershipAction; import org.elasticsearch.test.ESTestCase; +import org.elasticsearch.test.VersionUtils; import org.elasticsearch.test.junit.annotations.TestLogging; import org.elasticsearch.threadpool.ThreadPool; import org.junit.After; @@ -48,6 +49,8 @@ import org.junit.BeforeClass; import java.util.ArrayList; +import java.util.Arrays; +import java.util.Collections; import java.util.HashSet; import java.util.List; import java.util.Set; @@ -437,6 +440,21 @@ public void testRejectingJoinWithSameIdButDifferentAddress() throws InterruptedE assertThat(e.getMessage(), containsString("found existing node")); } + public void testJoinWithSameIdSameAddressButDifferentMeta() throws InterruptedException, ExecutionException { + ClusterState state = clusterService.state(); + final DiscoveryNode localNode = state.nodes().getLocalNode(); + final DiscoveryNode other_node = new DiscoveryNode( + randomBoolean() ? localNode.getName() : "other_name", + localNode.getId(), localNode.getAddress(), + randomBoolean() ? localNode.getAttributes() : Collections.singletonMap("attr", "other"), + randomBoolean() ? localNode.getRoles() : new HashSet<>(randomSubsetOf(Arrays.asList(DiscoveryNode.Role.values()))), + randomBoolean() ? localNode.getVersion() : VersionUtils.randomVersion(random())); + + joinNode(other_node); + + assertTrue(clusterService.localNode().equalsIncludingMetaData(other_node)); + } + public void testNormalConcurrentJoins() throws InterruptedException { Thread[] threads = new Thread[3 + randomInt(5)]; ArrayList nodes = new ArrayList<>(); From 2a7daf1900742b44bcf87f30c31d451a3c18d5e4 Mon Sep 17 00:00:00 2001 From: Boaz Leskes Date: Wed, 20 Apr 2016 11:58:39 +0300 Subject: [PATCH 23/34] typo --- core/src/main/java/org/elasticsearch/env/NodeMetaData.java | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/core/src/main/java/org/elasticsearch/env/NodeMetaData.java b/core/src/main/java/org/elasticsearch/env/NodeMetaData.java index c4d7ea43a831c..d07644b387945 100644 --- a/core/src/main/java/org/elasticsearch/env/NodeMetaData.java +++ b/core/src/main/java/org/elasticsearch/env/NodeMetaData.java @@ -72,7 +72,7 @@ public String toString() { Builder::new); static { - PARSER.declareString(Builder::setnodeId, new ParseField(NODE_ID_KEY)); + PARSER.declareString(Builder::setNodeId, new ParseField(NODE_ID_KEY)); } public String nodeId() { @@ -82,7 +82,7 @@ public String nodeId() { private static class Builder { String nodeId; - public void setnodeId(String nodeId) { + public void setNodeId(String nodeId) { this.nodeId = nodeId; } From 39fc924acc41c71f6ee5cb8e743dd14f9d0d3f60 Mon Sep 17 00:00:00 2001 From: Yannick Welsch Date: Mon, 25 Apr 2016 10:52:39 +0200 Subject: [PATCH 24/34] added process id --- .../TransportClientNodesService.java | 6 +- .../cluster/node/DiscoveryNode.java | 123 +++++++++++------- .../cluster/node/DiscoveryNodeService.java | 22 +++- .../cluster/node/DiscoveryNodes.java | 118 +++++++++-------- .../common/settings/ClusterSettings.java | 2 + .../TransportAddressSerializers.java | 11 -- .../discovery/local/LocalDiscovery.java | 11 -- .../discovery/zen/NodeJoinController.java | 61 +++++---- .../zen/ping/unicast/UnicastZenPing.java | 2 +- .../elasticsearch/env/NodeEnvironment.java | 10 +- .../java/org/elasticsearch/node/Node.java | 4 +- .../org/elasticsearch/tribe/TribeService.java | 5 +- .../TransportClientNodesServiceTests.java | 2 +- ...rdFailedClusterStateTaskExecutorTests.java | 4 +- .../node/DiscoveryNodeFiltersTests.java | 45 +++---- .../node/DiscoveryNodeServiceTests.java | 4 +- .../cluster/node/DiscoveryNodeTests.java | 29 +++-- .../cluster/node/DiscoveryNodesTests.java | 6 +- .../allocation/SameShardRoutingTests.java | 6 +- .../decider/DiskThresholdDeciderTests.java | 6 +- .../zen/NodeJoinControllerTests.java | 33 ++--- .../discovery/zen/ZenDiscoveryUnitTests.java | 6 +- .../PublishClusterStateActionTests.java | 5 +- .../gateway/AsyncShardFetchTests.java | 4 +- .../ingest/PipelineStoreTests.java | 2 +- .../elasticsearch/tribe/TribeUnitTests.java | 4 +- .../MockInternalClusterInfoService.java | 3 +- .../test/ESAllocationTestCase.java | 8 +- .../test/InternalTestCluster.java | 11 +- 29 files changed, 295 insertions(+), 258 deletions(-) diff --git a/core/src/main/java/org/elasticsearch/client/transport/TransportClientNodesService.java b/core/src/main/java/org/elasticsearch/client/transport/TransportClientNodesService.java index e407a2e7adad1..e844dbff1bf54 100644 --- a/core/src/main/java/org/elasticsearch/client/transport/TransportClientNodesService.java +++ b/core/src/main/java/org/elasticsearch/client/transport/TransportClientNodesService.java @@ -383,9 +383,9 @@ public LivenessResponse newInstance() { // use discovered information but do keep the original transport address, // so people can control which address is exactly used. DiscoveryNode nodeWithInfo = livenessResponse.getDiscoveryNode(); - newNodes.add(new DiscoveryNode(nodeWithInfo.getName(), nodeWithInfo.getId(), nodeWithInfo.getHostName(), - nodeWithInfo.getHostAddress(), listedNode.getAddress(), nodeWithInfo.getAttributes(), - nodeWithInfo.getRoles(), nodeWithInfo.getVersion())); + newNodes.add(new DiscoveryNode(nodeWithInfo.getName(), nodeWithInfo.getId(), nodeWithInfo.getPersistentNodeId(), + nodeWithInfo.getHostName(), nodeWithInfo.getHostAddress(), listedNode.getAddress(), + nodeWithInfo.getAttributes(), nodeWithInfo.getRoles(), nodeWithInfo.getVersion())); } else { // although we asked for one node, our target may not have completed // initialization yet and doesn't have cluster nodes diff --git a/core/src/main/java/org/elasticsearch/cluster/node/DiscoveryNode.java b/core/src/main/java/org/elasticsearch/cluster/node/DiscoveryNode.java index c445324421d94..e343904ec2d26 100644 --- a/core/src/main/java/org/elasticsearch/cluster/node/DiscoveryNode.java +++ b/core/src/main/java/org/elasticsearch/cluster/node/DiscoveryNode.java @@ -88,7 +88,8 @@ public static boolean isIngestNode(Settings settings) { } private final String nodeName; - private final String nodeId; + private final String processId; + private final String persistentNodeId; private final String hostName; private final String hostAddress; private final TransportAddress address; @@ -105,14 +106,15 @@ public static boolean isIngestNode(Settings settings) { * and updated. *

* - * @param nodeId the nodes unique id. - * @param address the nodes transport address - * @param attributes node attributes - * @param roles node roles - * @param version the version of the node. + * @param processAndNodeId the nodes unique process and node id + * @param address the nodes transport address + * @param attributes node attributes + * @param roles node roles + * @param version the version of the node */ - public DiscoveryNode(String nodeId, TransportAddress address, Map attributes, Set roles, Version version) { - this("", nodeId, address.getHost(), address.getAddress(), address, attributes, roles, version); + public DiscoveryNode(String processAndNodeId, TransportAddress address, Map attributes, Set roles, + Version version) { + this(processAndNodeId, processAndNodeId, address, attributes, roles, version); } /** @@ -124,20 +126,20 @@ public DiscoveryNode(String nodeId, TransportAddress address, Map * - * @param nodeName the nodes name - * @param nodeId the nodes unique id. - * @param address the nodes transport address - * @param attributes node attributes - * @param roles node roles - * @param version the version of the node. + * @param processId the nodes unique process id + * @param persistentNodeId the nodes unique persistent id + * @param address the nodes transport address + * @param attributes node attributes + * @param roles node roles + * @param version the version of the node */ - public DiscoveryNode(String nodeName, String nodeId, TransportAddress address, Map attributes, + public DiscoveryNode(String processId, String persistentNodeId, TransportAddress address, Map attributes, Set roles, Version version) { - this(nodeName, nodeId, address.getHost(), address.getAddress(), address, attributes, roles, version); + this("", processId, persistentNodeId, address.getHost(), address.getAddress(), address, attributes, roles, version); } /** - * Creates a new {@link DiscoveryNode}. + * Creates a new {@link DiscoveryNode} *

* Note: if the version of the node is unknown {@link Version#minimumCompatibilityVersion()} should be used for the current * version. it corresponds to the minimum version this elasticsearch version can communicate with. If a higher version is used @@ -145,23 +147,46 @@ public DiscoveryNode(String nodeName, String nodeId, TransportAddress address, M * and updated. *

* - * @param nodeName the nodes name - * @param nodeId the nodes unique id. - * @param hostName the nodes hostname - * @param hostAddress the nodes host address - * @param address the nodes transport address - * @param attributes node attributes - * @param roles node roles - * @param version the version of the node. + * @param nodeName the nodes name + * @param processId the nodes unique process id + * @param persistentNodeId the nodes unique persistent id + * @param address the nodes transport address + * @param attributes node attributes + * @param roles node roles + * @param version the version of the node */ - public DiscoveryNode(String nodeName, String nodeId, String hostName, String hostAddress, TransportAddress address, + public DiscoveryNode(String nodeName, String processId, String persistentNodeId, TransportAddress address, Map attributes, Set roles, Version version) { + this(nodeName, processId, persistentNodeId, address.getHost(), address.getAddress(), address, attributes, roles, version); + } + + /** + * Creates a new {@link DiscoveryNode}. + *

+ * Note: if the version of the node is unknown {@link Version#minimumCompatibilityVersion()} should be used for the current + * version. it corresponds to the minimum version this elasticsearch version can communicate with. If a higher version is used + * the node might not be able to communicate with the remove node. After initial handshakes node versions will be discovered + * and updated. + *

+ * + * @param nodeName the nodes name + * @param processId the nodes unique process id + * @param persistentNodeId the nodes unique persistent id + * @param hostAddress the nodes host address + * @param address the nodes transport address + * @param attributes node attributes + * @param roles node roles + * @param version the version of the node + */ + public DiscoveryNode(String nodeName, String processId, String persistentNodeId, String hostName, String hostAddress, + TransportAddress address, Map attributes, Set roles, Version version) { if (nodeName != null) { this.nodeName = nodeName.intern(); } else { this.nodeName = ""; } - this.nodeId = nodeId.intern(); + this.processId = processId.intern(); + this.persistentNodeId = persistentNodeId.intern(); this.hostName = hostName.intern(); this.hostAddress = hostAddress.intern(); this.address = address; @@ -191,7 +216,8 @@ public DiscoveryNode(String nodeName, String nodeId, String hostName, String hos */ public DiscoveryNode(StreamInput in) throws IOException { this.nodeName = in.readString().intern(); - this.nodeId = in.readString().intern(); + this.processId = in.readString().intern(); + this.persistentNodeId = in.readString().intern(); this.hostName = in.readString().intern(); this.hostAddress = in.readString().intern(); this.address = TransportAddressSerializers.addressFromStream(in); @@ -215,7 +241,8 @@ public DiscoveryNode(StreamInput in) throws IOException { @Override public void writeTo(StreamOutput out) throws IOException { out.writeString(nodeName); - out.writeString(nodeId); + out.writeString(processId); + out.writeString(persistentNodeId); out.writeString(hostName); out.writeString(hostAddress); addressToStream(out, address); @@ -239,10 +266,17 @@ public TransportAddress getAddress() { } /** - * The unique id of the node. + * The unique process id of the node. */ public String getId() { - return nodeId; + return processId; + } + + /** + * The unique process id of the node. + */ + public String getPersistentNodeId() { + return persistentNodeId; } /** @@ -299,17 +333,20 @@ public String getHostName() { public String getHostAddress() { return this.hostAddress; } - + /** - * Checks whether this node is has the same id as another node *and* has + * Checks whether this node has the same id as another node *and* has * the same attributes, network address and so fourth. This is in contrast to {@link #equals(Object)} - * which only uses the ids. + * which only compares the process ids. */ public boolean equalsIncludingMetaData(DiscoveryNode other) { if (this.equals(other) == false) { return false; } + if (!persistentNodeId.equals(other.persistentNodeId)) { + return false; + } if (!nodeName.equals(other.nodeName)) { return false; } @@ -343,12 +380,12 @@ public boolean equals(Object obj) { } DiscoveryNode other = (DiscoveryNode) obj; - return this.nodeId.equals(other.nodeId); + return this.processId.equals(other.processId); } @Override public int hashCode() { - return nodeId.hashCode(); + return processId.hashCode(); } @Override @@ -357,15 +394,10 @@ public String toString() { if (nodeName.length() > 0) { sb.append('{').append(nodeName).append('}'); } - if (nodeId != null) { - sb.append('{').append(nodeId).append('}'); - } - if (Strings.hasLength(hostName)) { - sb.append('{').append(hostName).append('}'); - } - if (address != null) { - sb.append('{').append(address).append('}'); - } + sb.append('{').append(processId).append('}'); + sb.append('{').append(persistentNodeId).append('}'); + sb.append('{').append(hostName).append('}'); + sb.append('{').append(address).append('}'); if (!attributes.isEmpty()) { sb.append(attributes); } @@ -376,6 +408,7 @@ public String toString() { public XContentBuilder toXContent(XContentBuilder builder, Params params) throws IOException { builder.startObject(getId()); builder.field("name", getName()); + builder.field("persistent_id", getPersistentNodeId()); builder.field("transport_address", getAddress().toString()); builder.startObject("attributes"); diff --git a/core/src/main/java/org/elasticsearch/cluster/node/DiscoveryNodeService.java b/core/src/main/java/org/elasticsearch/cluster/node/DiscoveryNodeService.java index 759139f11c567..80ef993e9a99b 100644 --- a/core/src/main/java/org/elasticsearch/cluster/node/DiscoveryNodeService.java +++ b/core/src/main/java/org/elasticsearch/cluster/node/DiscoveryNodeService.java @@ -20,8 +20,11 @@ package org.elasticsearch.cluster.node; import org.elasticsearch.Version; +import org.elasticsearch.common.Randomness; +import org.elasticsearch.common.UUIDs; import org.elasticsearch.common.component.AbstractComponent; import org.elasticsearch.common.inject.Inject; +import org.elasticsearch.common.settings.Setting; import org.elasticsearch.common.settings.Settings; import org.elasticsearch.common.transport.TransportAddress; import org.elasticsearch.node.Node; @@ -30,6 +33,7 @@ import java.util.HashSet; import java.util.List; import java.util.Map; +import java.util.Random; import java.util.Set; import java.util.concurrent.CopyOnWriteArrayList; import java.util.function.Supplier; @@ -38,13 +42,22 @@ */ public class DiscoveryNodeService extends AbstractComponent { + public static final Setting PROCESS_ID_SEED_SETTING = + Setting.longSetting("node.process_id.seed", 0L, Long.MIN_VALUE, Setting.Property.NodeScope); + private final List customAttributesProviders = new CopyOnWriteArrayList<>(); private final Version version; + private final String processId; @Inject public DiscoveryNodeService(Settings settings, Version version) { super(settings); this.version = version; + this.processId = generateProcessId(settings); + } + + public String getProcessId() { + return processId; } public DiscoveryNodeService addCustomAttributeProvider(CustomAttributesProvider customAttributesProvider) { @@ -52,7 +65,12 @@ public DiscoveryNodeService addCustomAttributeProvider(CustomAttributesProvider return this; } - public DiscoveryNode buildLocalNode(TransportAddress publishAddress, Supplier nodeIdSupplier) { + public static String generateProcessId(Settings settings) { + Random random = Randomness.get(settings, PROCESS_ID_SEED_SETTING); + return UUIDs.randomBase64UUID(random); + } + + public DiscoveryNode buildLocalNode(TransportAddress publishAddress, String nodeId) { Map attributes = new HashMap<>(Node.NODE_ATTRIBUTES.get(this.settings).getAsMap()); Set roles = new HashSet<>(); if (Node.NODE_INGEST_SETTING.get(settings)) { @@ -79,7 +97,7 @@ public DiscoveryNode buildLocalNode(TransportAddress publishAddress, Supplier entry : node.getAttributes().entrySet()) { - String attrName = entry.getKey(); - String attrValue = entry.getValue(); - if (Regex.simpleMatch(matchAttrName, attrName) && Regex.simpleMatch(matchAttrValue, attrValue)) { - resolvedNodesIds.add(node.getId()); + for (DiscoveryNode node : this) { + for (Map.Entry entry : node.getAttributes().entrySet()) { + String attrName = entry.getKey(); + String attrValue = entry.getValue(); + if (Regex.simpleMatch(matchAttrName, attrName) && Regex.simpleMatch(matchAttrValue, attrValue)) { + resolvedNodesIds.add(node.getId()); + } } } } } } + } } return resolvedNodesIds.toArray(String.class); @@ -568,10 +584,9 @@ public Builder(DiscoveryNodes nodes) { } public Builder put(DiscoveryNode node) { - final String preflight = preflightPut(node); - if (preflight != null) { - throw new IllegalArgumentException(preflight); - } + assert nodes.get(node.getId()) == null : "node with process id " + node.getId() + " already exists"; + assert nodeWithSameAddress(node) == null : "node with address " + node.getAddress() + " already exists"; + assert nodeWithSamePersistentId(node) == null : "node with persistent id " + node.getPersistentNodeId() + " already exists"; nodes.put(node.getId(), node); return this; } @@ -591,24 +606,21 @@ public Builder localNodeId(String localNodeId) { return this; } - /** - * Checks that a node can be safely added to this node collection. - * - * @return null if all is OK or an error message explaining why a node can not be added. - * - * Note: if this method returns a non-null value, calling {@link #put(DiscoveryNode)} will fail with an - * exception - */ - public String preflightPut(DiscoveryNode node) { + public DiscoveryNode nodeWithSameAddress(DiscoveryNode node) { for (ObjectCursor cursor : nodes.values()) { final DiscoveryNode existingNode = cursor.value; - if (node.getAddress().equals(existingNode.getAddress()) && - node.equals(existingNode) == false) { - return "can't add node " + node +", found existing node " + existingNode + " with same address"; + if (node.getAddress().equals(existingNode.getAddress())) { + return existingNode; } - if (node.getId().equals(existingNode.getId()) && - node.getAddress().equals(existingNode.getAddress()) == false) { - return "can't add node " + node +", found existing node " + existingNode + " with the same id, but a different address"; + } + return null; + } + + public DiscoveryNode nodeWithSamePersistentId(DiscoveryNode node) { + for (ObjectCursor cursor : nodes.values()) { + final DiscoveryNode existingNode = cursor.value; + if (node.getPersistentNodeId().equals(existingNode.getPersistentNodeId())) { + return existingNode; } } return null; diff --git a/core/src/main/java/org/elasticsearch/common/settings/ClusterSettings.java b/core/src/main/java/org/elasticsearch/common/settings/ClusterSettings.java index 2896ba4c409ce..57fdbdb312653 100644 --- a/core/src/main/java/org/elasticsearch/common/settings/ClusterSettings.java +++ b/core/src/main/java/org/elasticsearch/common/settings/ClusterSettings.java @@ -33,6 +33,7 @@ import org.elasticsearch.cluster.NodeConnectionsService; import org.elasticsearch.cluster.action.index.MappingUpdatedAction; import org.elasticsearch.cluster.metadata.MetaData; +import org.elasticsearch.cluster.node.DiscoveryNodeService; import org.elasticsearch.cluster.routing.allocation.allocator.BalancedShardsAllocator; import org.elasticsearch.cluster.routing.allocation.decider.AwarenessAllocationDecider; import org.elasticsearch.cluster.routing.allocation.decider.ClusterRebalanceAllocationDecider; @@ -335,6 +336,7 @@ public void apply(Settings value, Settings current, Settings previous) { Environment.PATH_SHARED_DATA_SETTING, Environment.PIDFILE_SETTING, NodeEnvironment.NODE_ID_SEED_SETTING, + DiscoveryNodeService.PROCESS_ID_SEED_SETTING, DiscoverySettings.INITIAL_STATE_TIMEOUT_SETTING, DiscoveryModule.DISCOVERY_TYPE_SETTING, DiscoveryModule.ZEN_MASTER_SERVICE_TYPE_SETTING, diff --git a/core/src/main/java/org/elasticsearch/common/transport/TransportAddressSerializers.java b/core/src/main/java/org/elasticsearch/common/transport/TransportAddressSerializers.java index fa55b32278654..784bee52d63e3 100644 --- a/core/src/main/java/org/elasticsearch/common/transport/TransportAddressSerializers.java +++ b/core/src/main/java/org/elasticsearch/common/transport/TransportAddressSerializers.java @@ -36,20 +36,9 @@ public abstract class TransportAddressSerializers { private static final Map> ADDRESS_REGISTRY; static { -<<<<<<< bd5693f6ca2f3945838810ac05999c91e1c2796f Map> registry = new HashMap<>(); - addAddressType(registry, DummyTransportAddress.INSTANCE.uniqueAddressTypeId(), (in) -> DummyTransportAddress.INSTANCE); addAddressType(registry, InetSocketTransportAddress.TYPE_ID, InetSocketTransportAddress::new); addAddressType(registry, LocalTransportAddress.TYPE_ID, LocalTransportAddress::new); -======= - Map registry = new HashMap<>(); - try { - addAddressType(registry, InetSocketTransportAddress.PROTO); - addAddressType(registry, LocalTransportAddress.PROTO); - } catch (Exception e) { - logger.warn("Failed to setup TransportAddresses", e); - } ->>>>>>> enforce unique network addresses across nodes in cluster ADDRESS_REGISTRY = unmodifiableMap(registry); } diff --git a/core/src/main/java/org/elasticsearch/discovery/local/LocalDiscovery.java b/core/src/main/java/org/elasticsearch/discovery/local/LocalDiscovery.java index a5712e705cfb0..053e4cae12e40 100644 --- a/core/src/main/java/org/elasticsearch/discovery/local/LocalDiscovery.java +++ b/core/src/main/java/org/elasticsearch/discovery/local/LocalDiscovery.java @@ -46,11 +46,9 @@ import org.elasticsearch.discovery.DiscoveryStats; import java.util.HashSet; -import java.util.Optional; import java.util.Queue; import java.util.Set; import java.util.concurrent.ConcurrentMap; -import java.util.concurrent.atomic.AtomicBoolean; import static org.elasticsearch.cluster.ClusterState.Builder; @@ -69,8 +67,6 @@ public class LocalDiscovery extends AbstractLifecycleComponent implem private volatile boolean master = false; - private final AtomicBoolean initialStateSent = new AtomicBoolean(); - private static final ConcurrentMap clusterGroups = ConcurrentCollections.newConcurrentMap(); private volatile ClusterState lastProcessedClusterState; @@ -103,13 +99,6 @@ public void startInitialJoin() { } logger.debug("Connected to cluster [{}]", clusterName); - Optional current = clusterGroup.members().stream().filter(other -> other.localNode().equals(this.localNode())) - .findFirst(); - if (current.isPresent()) { - throw new IllegalStateException("current cluster group already contains a node with the same id. current " - + current.get().localNode() + ", this node " + localNode()); - } - clusterGroup.members().add(this); LocalDiscovery firstMaster = null; diff --git a/core/src/main/java/org/elasticsearch/discovery/zen/NodeJoinController.java b/core/src/main/java/org/elasticsearch/discovery/zen/NodeJoinController.java index 91820d3aeb110..c022becbbb352 100644 --- a/core/src/main/java/org/elasticsearch/discovery/zen/NodeJoinController.java +++ b/core/src/main/java/org/elasticsearch/discovery/zen/NodeJoinController.java @@ -385,36 +385,43 @@ public ClusterState execute(ClusterState currentState) { Map.Entry> entry = iterator.next(); final DiscoveryNode node = entry.getKey(); iterator.remove(); - final String preflight = nodesBuilder.preflightPut(node); - if (preflight != null) { - Throwable failure = new IllegalStateException(preflight); - for (MembershipAction.JoinCallback callback: entry.getValue()) { - joinCallbacksToFail.add(new Tuple<>(callback, failure)); - } - } else if (currentState.nodes().nodeExists(node.getId())) { - // it may be that a quick node restart can cause a join to arrive - // before the node leave has been processed. In that we need to check that things - // like attributes didn't change. - final DiscoveryNode existing = currentState.nodes().get(node.getId()); - if (existing.equalsIncludingMetaData(node)) { - logger.debug("received a join request for an existing node [{}]", node); - } else { - logger.debug("received a join request for an existing node, but with different meta data, replacing existing" + - " {} with new {}", existing, node); - nodeAdded = true; - nodesBuilder.put(node); - } + DiscoveryNode existingNodeWithSameProcessId = currentState.nodes().get(node.getId()); + if (existingNodeWithSameProcessId != null) { + logger.debug("received a join request for an existing node [{}]", node); + assert existingNodeWithSameProcessId.equalsIncludingMetaData(node); joinCallbacksToRespondTo.addAll(entry.getValue()); } else { - nodeAdded = true; - nodesBuilder.put(node); - joinCallbacksToRespondTo.addAll(entry.getValue()); - } + DiscoveryNode nodeWithSameAddress = nodesBuilder.nodeWithSameAddress(node); + if (nodeWithSameAddress != null) { + logger.warn("received join request from node [{}], but found existing node {} with same address, removing " + + "existing node", node, nodeWithSameAddress); + nodesBuilder.remove(nodeWithSameAddress.getId()); + nodesBuilder.put(node); + nodeAdded = true; + joinCallbacksToRespondTo.addAll(entry.getValue()); + } else { + DiscoveryNode nodeWithSamePersistentId = nodesBuilder.nodeWithSamePersistentId(node); + if (nodeWithSamePersistentId != null) { + logger.warn("received join request from node [{}], but found existing node {} with same persistent node " + + "id, failing join request ", node); + Throwable failure = new IllegalStateException("can't add node " + node + ", found existing node " + + nodeWithSamePersistentId + " with same persistent node id"); + for (MembershipAction.JoinCallback callback: entry.getValue()) { + joinCallbacksToFail.add(new Tuple<>(callback, failure)); + } + } else { + logger.trace("received join request for a new node [{}]", node); + nodesBuilder.put(node); + nodeAdded = true; + joinCallbacksToRespondTo.addAll(entry.getValue()); + } + } - assert entry.getValue().stream().allMatch(cb -> - joinCallbacksToRespondTo.contains(cb) ^ - joinCallbacksToFail.stream().filter(tuple -> tuple.v1().equals(cb)).count() > 0 - ) : "failed to add " + entry.getValue() + "to joinCallbacksToRespondTo or joinCallbacksToFail"; + assert entry.getValue().stream().allMatch(cb -> + joinCallbacksToRespondTo.contains(cb) ^ + joinCallbacksToFail.stream().filter(tuple -> tuple.v1().equals(cb)).count() > 0 + ) : "failed to add " + entry.getValue() + " to either joinCallbacksToRespondTo or joinCallbacksToFail"; + } } } diff --git a/core/src/main/java/org/elasticsearch/discovery/zen/ping/unicast/UnicastZenPing.java b/core/src/main/java/org/elasticsearch/discovery/zen/ping/unicast/UnicastZenPing.java index 640582af22687..b723bc2b77984 100644 --- a/core/src/main/java/org/elasticsearch/discovery/zen/ping/unicast/UnicastZenPing.java +++ b/core/src/main/java/org/elasticsearch/discovery/zen/ping/unicast/UnicastZenPing.java @@ -379,7 +379,7 @@ void sendPings(final TimeValue timeout, @Nullable TimeValue waitTime, final Send // but will be added again during the pinging. We therefore create a new temporary node if (!nodeFoundByAddress) { if (!nodeToSend.getId().startsWith(UNICAST_NODE_PREFIX)) { - DiscoveryNode tempNode = new DiscoveryNode("", + DiscoveryNode tempNode = new DiscoveryNode( UNICAST_NODE_PREFIX + unicastNodeIdGenerator.incrementAndGet() + "_" + nodeToSend.getId() + "#", nodeToSend.getHostName(), nodeToSend.getHostAddress(), nodeToSend.getAddress(), nodeToSend.getAttributes(), nodeToSend.getRoles(), nodeToSend.getVersion()); diff --git a/core/src/main/java/org/elasticsearch/env/NodeEnvironment.java b/core/src/main/java/org/elasticsearch/env/NodeEnvironment.java index 45cd642b9a741..ed75249625802 100644 --- a/core/src/main/java/org/elasticsearch/env/NodeEnvironment.java +++ b/core/src/main/java/org/elasticsearch/env/NodeEnvironment.java @@ -32,8 +32,8 @@ import org.elasticsearch.cluster.metadata.IndexMetaData; import org.elasticsearch.cluster.node.DiscoveryNode; import org.elasticsearch.common.Randomness; -import org.elasticsearch.common.Strings; import org.elasticsearch.common.SuppressForbidden; +import org.elasticsearch.common.UUIDs; import org.elasticsearch.common.component.AbstractComponent; import org.elasticsearch.common.inject.Inject; import org.elasticsearch.common.io.FileSystemUtils; @@ -270,7 +270,7 @@ private static NodeMetaData loadOrCreateNodeMetaData(Settings settings, ESLogger public static String generateNodeId(Settings settings) { Random random = Randomness.get(settings, NODE_ID_SEED_SETTING); - return Strings.randomBase64UUID(random); + return UUIDs.randomBase64UUID(random); } private static void releaseAndNullLocks(Lock[] locks) { @@ -674,12 +674,12 @@ public Path[] nodeDataPaths() { } /** - * returns the unique uuid describing this node. The uuid is persistent in the data folder of this node - * and remains across restarts. + * returns the unique uuid describing this node. The uuid is persisted in the data folder of this node + * and is reused across restarts. **/ public String nodeId() { // we currently only return the ID and hide the underlying nodeMetaData implementation in order to avoid - // confusion with other "metadata" like node settings found in elasticsearch.yml. In future + // confusion with other "metadata" like node settings found in elasticsearch.yml. In the future // we can encapsulate both (and more) in one NodeMetaData (or NodeSettings) object ala IndexSettings return nodeMetaData.nodeId(); } diff --git a/core/src/main/java/org/elasticsearch/node/Node.java b/core/src/main/java/org/elasticsearch/node/Node.java index 72403a67e5d33..77ea02f811806 100644 --- a/core/src/main/java/org/elasticsearch/node/Node.java +++ b/core/src/main/java/org/elasticsearch/node/Node.java @@ -332,9 +332,9 @@ public Node start() { validateNodeBeforeAcceptingRequests(settings, transportService.boundAddress()); + String nodeId = injector.getInstance(NodeEnvironment.class).nodeId(); DiscoveryNode localNode = injector.getInstance(DiscoveryNodeService.class).buildLocalNode( - transportService.boundAddress().publishAddress(), - injector.getInstance(NodeEnvironment.class)::nodeId); + transportService.boundAddress().publishAddress(), nodeId); // TODO: need to find a cleaner way to start/construct a service with some initial parameters, // playing nice with the life cycle interfaces diff --git a/core/src/main/java/org/elasticsearch/tribe/TribeService.java b/core/src/main/java/org/elasticsearch/tribe/TribeService.java index b214a13ebdb7c..87c04544944ed 100644 --- a/core/src/main/java/org/elasticsearch/tribe/TribeService.java +++ b/core/src/main/java/org/elasticsearch/tribe/TribeService.java @@ -368,8 +368,9 @@ private ClusterState applyUpdate(ClusterState currentState, ClusterChangedEvent // a new node, add it, but also add the tribe name to the attributes Map tribeAttr = new HashMap<>(tribe.getAttributes()); tribeAttr.put(TRIBE_NAME_SETTING.getKey(), tribeName); - DiscoveryNode discoNode = new DiscoveryNode(tribe.getName(), tribe.getId(), tribe.getHostName(), tribe.getHostAddress(), - tribe.getAddress(), unmodifiableMap(tribeAttr), tribe.getRoles(), tribe.getVersion()); + DiscoveryNode discoNode = new DiscoveryNode(tribe.getName(), tribe.getId(), tribe.getPersistentNodeId(), + tribe.getHostName(), tribe.getHostAddress(), tribe.getAddress(), unmodifiableMap(tribeAttr), tribe.getRoles(), + tribe.getVersion()); clusterStateChanged = true; logger.info("[{}] adding node [{}]", tribeName, discoNode); nodes.put(discoNode); diff --git a/core/src/test/java/org/elasticsearch/client/transport/TransportClientNodesServiceTests.java b/core/src/test/java/org/elasticsearch/client/transport/TransportClientNodesServiceTests.java index 0246ec227dde0..05a29d041294a 100644 --- a/core/src/test/java/org/elasticsearch/client/transport/TransportClientNodesServiceTests.java +++ b/core/src/test/java/org/elasticsearch/client/transport/TransportClientNodesServiceTests.java @@ -119,7 +119,7 @@ public T newInstance() { @SuppressWarnings("unchecked") public void handleResponse(T response) { LivenessResponse livenessResponse = new LivenessResponse(ClusterName.DEFAULT, - new DiscoveryNode(node.getName(), node.getId(), "liveness-hostname" + node.getId(), + new DiscoveryNode(node.getName(), node.getId(), node.getPersistentNodeId(), "liveness-hostname" + node.getId(), "liveness-hostaddress" + node.getId(), new LocalTransportAddress("liveness-address-" + node.getId()), node.getAttributes(), node.getRoles(), node.getVersion())); diff --git a/core/src/test/java/org/elasticsearch/cluster/action/shard/ShardFailedClusterStateTaskExecutorTests.java b/core/src/test/java/org/elasticsearch/cluster/action/shard/ShardFailedClusterStateTaskExecutorTests.java index ef85966770c09..67db195eaa5d3 100644 --- a/core/src/test/java/org/elasticsearch/cluster/action/shard/ShardFailedClusterStateTaskExecutorTests.java +++ b/core/src/test/java/org/elasticsearch/cluster/action/shard/ShardFailedClusterStateTaskExecutorTests.java @@ -40,7 +40,7 @@ import org.elasticsearch.cluster.routing.allocation.FailedRerouteAllocation; import org.elasticsearch.cluster.routing.allocation.RoutingAllocation; import org.elasticsearch.cluster.routing.allocation.decider.ClusterRebalanceAllocationDecider; -import org.elasticsearch.common.Strings; +import org.elasticsearch.common.UUIDs; import org.elasticsearch.common.settings.Settings; import org.elasticsearch.index.Index; import org.elasticsearch.test.ESAllocationTestCase; @@ -304,7 +304,7 @@ private static ShardRouting randomInvalidSourceShard(ClusterState currentState, return randomSubsetOf(1, shards.toArray(new ShardRouting[0])).get(0); } else { return TestShardRouting.newShardRouting(shardRouting.index(), shardRouting.id(), - Strings.randomBase64UUID(random()), randomBoolean(), randomFrom(ShardRoutingState.values())); + UUIDs.randomBase64UUID(random()), randomBoolean(), randomFrom(ShardRoutingState.values())); } } diff --git a/core/src/test/java/org/elasticsearch/cluster/node/DiscoveryNodeFiltersTests.java b/core/src/test/java/org/elasticsearch/cluster/node/DiscoveryNodeFiltersTests.java index 97a94d24d1c73..f6a06a5c51750 100644 --- a/core/src/test/java/org/elasticsearch/cluster/node/DiscoveryNodeFiltersTests.java +++ b/core/src/test/java/org/elasticsearch/cluster/node/DiscoveryNodeFiltersTests.java @@ -64,46 +64,46 @@ public void testNameMatch() { .build(); DiscoveryNodeFilters filters = DiscoveryNodeFilters.buildFromSettings(OR, "xxx.", settings); - DiscoveryNode node = new DiscoveryNode("name1", "id1", LocalTransportAddress.buildUnique(), emptyMap(), + DiscoveryNode node = new DiscoveryNode("name1", "pid1", "nid1", LocalTransportAddress.buildUnique(), emptyMap(), emptySet(), Version.CURRENT); assertThat(filters.match(node), equalTo(true)); - node = new DiscoveryNode("name2", "id2", LocalTransportAddress.buildUnique(), emptyMap(), emptySet(), + node = new DiscoveryNode("name2", "pid2", "nid2", LocalTransportAddress.buildUnique(), emptyMap(), emptySet(), Version.CURRENT); assertThat(filters.match(node), equalTo(false)); } public void testIdMatch() { Settings settings = Settings.builder() - .put("xxx._id", "id1") + .put("xxx._id", "pid1") .build(); DiscoveryNodeFilters filters = DiscoveryNodeFilters.buildFromSettings(OR, "xxx.", settings); - DiscoveryNode node = new DiscoveryNode("name1", "id1", LocalTransportAddress.buildUnique(), emptyMap(), + DiscoveryNode node = new DiscoveryNode("name1", "pid1", "nid1", LocalTransportAddress.buildUnique(), emptyMap(), emptySet(), Version.CURRENT); assertThat(filters.match(node), equalTo(true)); - node = new DiscoveryNode("name2", "id2", LocalTransportAddress.buildUnique(), emptyMap(), emptySet(), + node = new DiscoveryNode("name2", "pid2", "nid2", LocalTransportAddress.buildUnique(), emptyMap(), emptySet(), Version.CURRENT); assertThat(filters.match(node), equalTo(false)); } public void testIdOrNameMatch() { Settings settings = shuffleSettings(Settings.builder() - .put("xxx._id", "id1,blah") + .put("xxx._id", "pid1,blah") .put("xxx.name", "blah,name2") .build()); DiscoveryNodeFilters filters = DiscoveryNodeFilters.buildFromSettings(OR, "xxx.", settings); - DiscoveryNode node = new DiscoveryNode("name1", "id1", LocalTransportAddress.buildUnique(), emptyMap(), + DiscoveryNode node = new DiscoveryNode("name1", "pid1", "nid1", LocalTransportAddress.buildUnique(), emptyMap(), emptySet(), Version.CURRENT); assertThat(filters.match(node), equalTo(true)); - node = new DiscoveryNode("name2", "id2", LocalTransportAddress.buildUnique(), emptyMap(), emptySet(), + node = new DiscoveryNode("name2", "pid2", "nid2", LocalTransportAddress.buildUnique(), emptyMap(), emptySet(), Version.CURRENT); assertThat(filters.match(node), equalTo(true)); - node = new DiscoveryNode("name3", "id3", LocalTransportAddress.buildUnique(), emptyMap(), emptySet(), + node = new DiscoveryNode("name3", "pid3", "nid3", LocalTransportAddress.buildUnique(), emptyMap(), emptySet(), Version.CURRENT); assertThat(filters.match(node), equalTo(false)); } @@ -148,7 +148,7 @@ public void testStarMatch() { .build(); DiscoveryNodeFilters filters = DiscoveryNodeFilters.buildFromSettings(OR, "xxx.", settings); - DiscoveryNode node = new DiscoveryNode("name1", "id1", LocalTransportAddress.buildUnique(), emptyMap(), + DiscoveryNode node = new DiscoveryNode("name1", "pid1", "nid1", LocalTransportAddress.buildUnique(), emptyMap(), emptySet(), Version.CURRENT); assertThat(filters.match(node), equalTo(true)); } @@ -160,8 +160,7 @@ public void testIpBindFilteringMatchingAnd() { .build()); DiscoveryNodeFilters filters = DiscoveryNodeFilters.buildFromSettings(AND, "xxx.", settings); - DiscoveryNode node = new DiscoveryNode("", "", "", "192.1.1.54", localAddress, singletonMap("tag", "A"), - emptySet(), null); + DiscoveryNode node = new DiscoveryNode("", "", "", "", "192.1.1.54", localAddress, singletonMap("tag", "A"), emptySet(), null); assertThat(filters.match(node), equalTo(true)); } @@ -172,8 +171,7 @@ public void testIpBindFilteringNotMatching() { .build()); DiscoveryNodeFilters filters = DiscoveryNodeFilters.buildFromSettings(AND, "xxx.", settings); - DiscoveryNode node = new DiscoveryNode("", "", "", "192.1.1.54", localAddress, singletonMap("tag", "A"), - emptySet(), null); + DiscoveryNode node = new DiscoveryNode("", "", "", "", "192.1.1.54", localAddress, singletonMap("tag", "A"), emptySet(), null); assertThat(filters.match(node), equalTo(false)); } @@ -184,8 +182,7 @@ public void testIpBindFilteringNotMatchingAnd() { .build()); DiscoveryNodeFilters filters = DiscoveryNodeFilters.buildFromSettings(AND, "xxx.", settings); - DiscoveryNode node = new DiscoveryNode("", "", "", "192.1.1.54", localAddress, singletonMap("tag", "A"), - emptySet(), null); + DiscoveryNode node = new DiscoveryNode("", "", "", "", "192.1.1.54", localAddress, singletonMap("tag", "A"), emptySet(), null); assertThat(filters.match(node), equalTo(false)); } @@ -196,7 +193,7 @@ public void testIpBindFilteringMatchingOr() { .build()); DiscoveryNodeFilters filters = DiscoveryNodeFilters.buildFromSettings(OR, "xxx.", settings); - DiscoveryNode node = new DiscoveryNode("", "", "", "192.1.1.54", localAddress, singletonMap("tag", "A"), emptySet(), null); + DiscoveryNode node = new DiscoveryNode("", "", "", "", "192.1.1.54", localAddress, singletonMap("tag", "A"), emptySet(), null); assertThat(filters.match(node), equalTo(true)); } @@ -207,7 +204,7 @@ public void testIpBindFilteringNotMatchingOr() { .build()); DiscoveryNodeFilters filters = DiscoveryNodeFilters.buildFromSettings(OR, "xxx.", settings); - DiscoveryNode node = new DiscoveryNode("", "", "", "192.1.1.54", localAddress, singletonMap("tag", "A"), emptySet(), null); + DiscoveryNode node = new DiscoveryNode("", "", "", "", "192.1.1.54", localAddress, singletonMap("tag", "A"), emptySet(), null); assertThat(filters.match(node), equalTo(true)); } @@ -218,8 +215,7 @@ public void testIpPublishFilteringMatchingAnd() { .build()); DiscoveryNodeFilters filters = DiscoveryNodeFilters.buildFromSettings(AND, "xxx.", settings); - DiscoveryNode node = new DiscoveryNode("", "", "", "192.1.1.54", localAddress, singletonMap("tag", "A"), - emptySet(), null); + DiscoveryNode node = new DiscoveryNode("", "", "", "", "192.1.1.54", localAddress, singletonMap("tag", "A"), emptySet(), null); assertThat(filters.match(node), equalTo(true)); } @@ -230,8 +226,7 @@ public void testIpPublishFilteringNotMatchingAnd() { .build()); DiscoveryNodeFilters filters = DiscoveryNodeFilters.buildFromSettings(AND, "xxx.", settings); - DiscoveryNode node = new DiscoveryNode("", "", "", "192.1.1.54", localAddress, singletonMap("tag", "A"), - emptySet(), null); + DiscoveryNode node = new DiscoveryNode("", "", "", "", "192.1.1.54", localAddress, singletonMap("tag", "A"), emptySet(), null); assertThat(filters.match(node), equalTo(false)); } @@ -242,8 +237,7 @@ public void testIpPublishFilteringMatchingOr() { .build()); DiscoveryNodeFilters filters = DiscoveryNodeFilters.buildFromSettings(OR, "xxx.", settings); - DiscoveryNode node = new DiscoveryNode("", "", "", "192.1.1.54", localAddress, singletonMap("tag", "A"), - emptySet(), null); + DiscoveryNode node = new DiscoveryNode("", "", "", "", "192.1.1.54", localAddress, singletonMap("tag", "A"), emptySet(), null); assertThat(filters.match(node), equalTo(true)); } @@ -254,8 +248,7 @@ public void testIpPublishFilteringNotMatchingOr() { .build()); DiscoveryNodeFilters filters = DiscoveryNodeFilters.buildFromSettings(OR, "xxx.", settings); - DiscoveryNode node = new DiscoveryNode("", "", "", "192.1.1.54", localAddress, singletonMap("tag", "A"), - emptySet(), null); + DiscoveryNode node = new DiscoveryNode("", "", "", "", "192.1.1.54", localAddress, singletonMap("tag", "A"), emptySet(), null); assertThat(filters.match(node), equalTo(true)); } diff --git a/core/src/test/java/org/elasticsearch/cluster/node/DiscoveryNodeServiceTests.java b/core/src/test/java/org/elasticsearch/cluster/node/DiscoveryNodeServiceTests.java index a2c48b86295b0..4b7084515fc04 100644 --- a/core/src/test/java/org/elasticsearch/cluster/node/DiscoveryNodeServiceTests.java +++ b/core/src/test/java/org/elasticsearch/cluster/node/DiscoveryNodeServiceTests.java @@ -57,7 +57,7 @@ public void testBuildLocalNode() { } DiscoveryNodeService discoveryNodeService = new DiscoveryNodeService(builder.build(), Version.CURRENT); DiscoveryNode discoveryNode = discoveryNodeService.buildLocalNode( - LocalTransportAddress.buildUnique(), () -> NodeEnvironment.generateNodeId(Settings.EMPTY)); + LocalTransportAddress.buildUnique(), NodeEnvironment.generateNodeId(Settings.EMPTY)); assertThat(discoveryNode.getRoles(), equalTo(selectedRoles)); assertThat(discoveryNode.getAttributes(), equalTo(expectedAttributes)); } @@ -80,7 +80,7 @@ public void testBuildAttributesWithCustomAttributeServiceProvider() { discoveryNodeService.addCustomAttributeProvider(() -> customAttributes); DiscoveryNode discoveryNode = discoveryNodeService.buildLocalNode( - LocalTransportAddress.buildUnique(), () -> NodeEnvironment.generateNodeId(Settings.EMPTY)); + LocalTransportAddress.buildUnique(), NodeEnvironment.generateNodeId(Settings.EMPTY)); assertThat(discoveryNode.getAttributes(), equalTo(expectedAttributes)); } } diff --git a/core/src/test/java/org/elasticsearch/cluster/node/DiscoveryNodeTests.java b/core/src/test/java/org/elasticsearch/cluster/node/DiscoveryNodeTests.java index c0f998e536e64..779e814727def 100644 --- a/core/src/test/java/org/elasticsearch/cluster/node/DiscoveryNodeTests.java +++ b/core/src/test/java/org/elasticsearch/cluster/node/DiscoveryNodeTests.java @@ -31,21 +31,20 @@ public class DiscoveryNodeTests extends ESTestCase { - static AtomicInteger idGenerator = new AtomicInteger(); + private static final AtomicInteger idGenerator = new AtomicInteger(); /** * guarantees to generate a node that is unique for all aspects but for it's roles and version (which are chosen randomly) */ DiscoveryNode randomNode() { Map attributes = new HashMap<>(); - for (int attr = (idGenerator.get() > 0 ? 1 : 0) + randomInt(3); - attr > 0; - attr--) { + for (int attr = (idGenerator.get() > 0 ? 1 : 0) + randomInt(3); attr > 0; attr--) { attributes.put("attr_" + attr, "v_" + idGenerator.incrementAndGet()); } final DiscoveryNode node = new DiscoveryNode( "name_" + idGenerator.incrementAndGet(), - "id_" + idGenerator.incrementAndGet(), + "processId_" + idGenerator.incrementAndGet(), + "persistentNodeId_" + idGenerator.incrementAndGet(), LocalTransportAddress.buildUnique(), attributes, new HashSet<>(randomSubsetOf(Arrays.asList(DiscoveryNode.Role.values()))), randomVersion(random())); @@ -63,25 +62,29 @@ public void testEquality() { DiscoveryNode sameIdDifferentMeta; - sameIdDifferentMeta = new DiscoveryNode(node2.getName(), node1.getId(), node2.getAddress(), node2.getAttributes(), - node2.getRoles(), node2.getVersion()); + sameIdDifferentMeta = new DiscoveryNode(node2.getName(), node1.getId(), node1.getPersistentNodeId(), node2.getAddress(), + node2.getAttributes(), node2.getRoles(), node2.getVersion()); assertTrue(node1.equals(sameIdDifferentMeta)); assertFalse(node1.equalsIncludingMetaData(sameIdDifferentMeta)); - sameIdDifferentMeta = new DiscoveryNode(node2.getName(), node1.getId(), node1.getAddress(), node1.getAttributes(), - (randomBoolean() ? node1 : node2).getRoles(), (randomBoolean() ? node1 : node2).getVersion()); + sameIdDifferentMeta = new DiscoveryNode(node1.getName(), node1.getId(), node2.getPersistentNodeId(), node1.getAddress(), + node1.getAttributes(), node1.getRoles(), node1.getVersion()); assertTrue(node1.equals(sameIdDifferentMeta)); assertFalse(node1.equalsIncludingMetaData(sameIdDifferentMeta)); - sameIdDifferentMeta = new DiscoveryNode(node1.getName(), node1.getId(), node2.getAddress(), node1.getAttributes(), - (randomBoolean() ? node1 : node2).getRoles(), (randomBoolean() ? node1 : node2).getVersion()); + sameIdDifferentMeta = new DiscoveryNode(node2.getName(), node1.getId(), node1.getPersistentNodeId(), node1.getAddress(), + node1.getAttributes(), (randomBoolean() ? node1 : node2).getRoles(), (randomBoolean() ? node1 : node2).getVersion()); assertTrue(node1.equals(sameIdDifferentMeta)); assertFalse(node1.equalsIncludingMetaData(sameIdDifferentMeta)); - sameIdDifferentMeta = new DiscoveryNode(node1.getName(), node1.getId(), node1.getAddress(), node2.getAttributes(), - (randomBoolean() ? node1 : node2).getRoles(), (randomBoolean() ? node1 : node2).getVersion()); + sameIdDifferentMeta = new DiscoveryNode(node1.getName(), node1.getId(), node1.getPersistentNodeId(), node2.getAddress(), + node1.getAttributes(), (randomBoolean() ? node1 : node2).getRoles(), (randomBoolean() ? node1 : node2).getVersion()); assertTrue(node1.equals(sameIdDifferentMeta)); assertFalse(node1.equalsIncludingMetaData(sameIdDifferentMeta)); + sameIdDifferentMeta = new DiscoveryNode(node1.getName(), node1.getId(), node1.getPersistentNodeId(), node1.getAddress(), + node2.getAttributes(), (randomBoolean() ? node1 : node2).getRoles(), (randomBoolean() ? node1 : node2).getVersion()); + assertTrue(node1.equals(sameIdDifferentMeta)); + assertFalse(node1.equalsIncludingMetaData(sameIdDifferentMeta)); } } diff --git a/core/src/test/java/org/elasticsearch/cluster/node/DiscoveryNodesTests.java b/core/src/test/java/org/elasticsearch/cluster/node/DiscoveryNodesTests.java index 1d1ee7c717720..7f25c1846e897 100644 --- a/core/src/test/java/org/elasticsearch/cluster/node/DiscoveryNodesTests.java +++ b/core/src/test/java/org/elasticsearch/cluster/node/DiscoveryNodesTests.java @@ -41,7 +41,7 @@ public void testResolveNodeByIdOrName() { DiscoveryNodes discoveryNodes = buildDiscoveryNodes(); DiscoveryNode[] nodes = discoveryNodes.getNodes().values().toArray(DiscoveryNode.class); DiscoveryNode node = randomFrom(nodes); - DiscoveryNode resolvedNode = discoveryNodes.resolveNode(randomBoolean() ? node.getId() : node.getName()); + DiscoveryNode resolvedNode = discoveryNodes.resolveNode(randomFrom(node.getId(), node.getPersistentNodeId(), node.getName())); assertThat(resolvedNode.getId(), equalTo(node.getId())); } @@ -117,8 +117,8 @@ private static DiscoveryNodes buildDiscoveryNodes() { } private static DiscoveryNode newNode(int nodeId, Map attributes, Set roles) { - return new DiscoveryNode("name_" + nodeId, "node_" + nodeId, LocalTransportAddress.buildUnique(), attributes, - roles, Version.CURRENT); + return new DiscoveryNode("name_" + nodeId, "process_" + nodeId, "persistent_node_" + nodeId, LocalTransportAddress.buildUnique(), + attributes, roles, Version.CURRENT); } private enum NodeSelector { diff --git a/core/src/test/java/org/elasticsearch/cluster/routing/allocation/SameShardRoutingTests.java b/core/src/test/java/org/elasticsearch/cluster/routing/allocation/SameShardRoutingTests.java index 5eaf915736bf2..1bb92ff3f01b4 100644 --- a/core/src/test/java/org/elasticsearch/cluster/routing/allocation/SameShardRoutingTests.java +++ b/core/src/test/java/org/elasticsearch/cluster/routing/allocation/SameShardRoutingTests.java @@ -62,9 +62,9 @@ public void testSameHost() { logger.info("--> adding two nodes with the same host"); clusterState = ClusterState.builder(clusterState).nodes( DiscoveryNodes.builder() - .put(new DiscoveryNode("node1", "node1", "test1", "test1", LocalTransportAddress.buildUnique(), emptyMap(), + .put(new DiscoveryNode("node1", "node1", "node1", "test1", "test1", LocalTransportAddress.buildUnique(), emptyMap(), MASTER_DATA_ROLES, Version.CURRENT)) - .put(new DiscoveryNode("node2", "node2", "test1", "test1", LocalTransportAddress.buildUnique(), emptyMap(), + .put(new DiscoveryNode("node2", "node2", "node2", "test1", "test1", LocalTransportAddress.buildUnique(), emptyMap(), MASTER_DATA_ROLES, Version.CURRENT))).build(); routingTable = strategy.reroute(clusterState, "reroute").routingTable(); clusterState = ClusterState.builder(clusterState).routingTable(routingTable).build(); @@ -81,7 +81,7 @@ public void testSameHost() { logger.info("--> add another node, with a different host, replicas will be allocating"); clusterState = ClusterState.builder(clusterState).nodes(DiscoveryNodes.builder(clusterState.nodes()) - .put(new DiscoveryNode("node3", "node3", "test2", "test2", LocalTransportAddress.buildUnique(), emptyMap(), + .put(new DiscoveryNode("node3", "node3", "node3", "test2", "test2", LocalTransportAddress.buildUnique(), emptyMap(), MASTER_DATA_ROLES, Version.CURRENT))).build(); routingTable = strategy.reroute(clusterState, "reroute").routingTable(); clusterState = ClusterState.builder(clusterState).routingTable(routingTable).build(); diff --git a/core/src/test/java/org/elasticsearch/cluster/routing/allocation/decider/DiskThresholdDeciderTests.java b/core/src/test/java/org/elasticsearch/cluster/routing/allocation/decider/DiskThresholdDeciderTests.java index 1d1a88e1db9f7..a38fd902c0678 100644 --- a/core/src/test/java/org/elasticsearch/cluster/routing/allocation/decider/DiskThresholdDeciderTests.java +++ b/core/src/test/java/org/elasticsearch/cluster/routing/allocation/decider/DiskThresholdDeciderTests.java @@ -962,9 +962,9 @@ public void testForSingleDataNode() { .build(); logger.info("--> adding one master node, one data node"); - DiscoveryNode discoveryNode1 = new DiscoveryNode("", "node1", new LocalTransportAddress("1"), emptyMap(), + DiscoveryNode discoveryNode1 = new DiscoveryNode("", "node1", "node1", new LocalTransportAddress("1"), emptyMap(), singleton(DiscoveryNode.Role.MASTER), Version.CURRENT); - DiscoveryNode discoveryNode2 = new DiscoveryNode("", "node2", new LocalTransportAddress("2"), emptyMap(), + DiscoveryNode discoveryNode2 = new DiscoveryNode("", "node2", "node2", new LocalTransportAddress("2"), emptyMap(), singleton(DiscoveryNode.Role.DATA), Version.CURRENT); DiscoveryNodes discoveryNodes = DiscoveryNodes.builder().put(discoveryNode1).put(discoveryNode2).build(); @@ -1030,7 +1030,7 @@ public void addListener(Listener listener) { // Add another datanode, it should relocate. logger.info("--> adding node3"); - DiscoveryNode discoveryNode3 = new DiscoveryNode("", "node3", new LocalTransportAddress("3"), emptyMap(), + DiscoveryNode discoveryNode3 = new DiscoveryNode("", "node3", "node3", new LocalTransportAddress("3"), emptyMap(), singleton(DiscoveryNode.Role.DATA), Version.CURRENT); ClusterState updateClusterState = ClusterState.builder(clusterState).nodes(DiscoveryNodes.builder(clusterState.nodes()) .put(discoveryNode3)).build(); diff --git a/core/src/test/java/org/elasticsearch/discovery/zen/NodeJoinControllerTests.java b/core/src/test/java/org/elasticsearch/discovery/zen/NodeJoinControllerTests.java index 9ea3883b40fb7..b06b7b45e2add 100644 --- a/core/src/test/java/org/elasticsearch/discovery/zen/NodeJoinControllerTests.java +++ b/core/src/test/java/org/elasticsearch/discovery/zen/NodeJoinControllerTests.java @@ -31,6 +31,7 @@ import org.elasticsearch.cluster.routing.allocation.FailedRerouteAllocation; import org.elasticsearch.cluster.routing.allocation.RoutingAllocation; import org.elasticsearch.cluster.service.ClusterService; +import org.elasticsearch.common.UUIDs; import org.elasticsearch.common.settings.ClusterSettings; import org.elasticsearch.common.settings.Settings; import org.elasticsearch.common.transport.LocalTransportAddress; @@ -40,7 +41,6 @@ import org.elasticsearch.discovery.DiscoverySettings; import org.elasticsearch.discovery.zen.membership.MembershipAction; import org.elasticsearch.test.ESTestCase; -import org.elasticsearch.test.VersionUtils; import org.elasticsearch.test.junit.annotations.TestLogging; import org.elasticsearch.threadpool.ThreadPool; import org.junit.After; @@ -49,8 +49,6 @@ import org.junit.BeforeClass; import java.util.ArrayList; -import java.util.Arrays; -import java.util.Collections; import java.util.HashSet; import java.util.List; import java.util.Set; @@ -422,39 +420,26 @@ public void testNewClusterStateOnExistingNodeJoin() throws InterruptedException, assertTrue("failed to publish a new state upon existing join", clusterService.state() != state); } - public void testRejectingJoinWithSameAddressButDifferentId() throws InterruptedException, ExecutionException { + public void testRemoveExistingNodeWhenJoinWithSameAddressButDifferentProcessId() throws InterruptedException, ExecutionException { ClusterState state = clusterService.state(); final DiscoveryNode other_node = new DiscoveryNode("other_node", state.nodes().getLocalNode().getAddress(), emptyMap(), emptySet(), Version.CURRENT); - ExecutionException e = expectThrows(ExecutionException.class, () -> joinNode(other_node)); - assertThat(e.getMessage(), containsString("found existing node")); + joinNode(other_node); + state = clusterService.state(); + assertThat(state.getNodes().getSize(), equalTo(1)); + assertTrue(state.getNodes().get(other_node.getId()).equalsIncludingMetaData(other_node)); } - public void testRejectingJoinWithSameIdButDifferentAddress() throws InterruptedException, ExecutionException { + public void testRejectingJoinWithSamePersistentIdButDifferentAddress() throws InterruptedException, ExecutionException { ClusterState state = clusterService.state(); - final DiscoveryNode other_node = new DiscoveryNode(state.nodes().getLocalNode().getId(), LocalTransportAddress.buildUnique(), - emptyMap(), emptySet(), Version.CURRENT); + final DiscoveryNode other_node = new DiscoveryNode(UUIDs.randomBase64UUID(), state.nodes().getLocalNode().getId(), + LocalTransportAddress.buildUnique(), emptyMap(), emptySet(), Version.CURRENT); ExecutionException e = expectThrows(ExecutionException.class, () -> joinNode(other_node)); assertThat(e.getMessage(), containsString("found existing node")); } - public void testJoinWithSameIdSameAddressButDifferentMeta() throws InterruptedException, ExecutionException { - ClusterState state = clusterService.state(); - final DiscoveryNode localNode = state.nodes().getLocalNode(); - final DiscoveryNode other_node = new DiscoveryNode( - randomBoolean() ? localNode.getName() : "other_name", - localNode.getId(), localNode.getAddress(), - randomBoolean() ? localNode.getAttributes() : Collections.singletonMap("attr", "other"), - randomBoolean() ? localNode.getRoles() : new HashSet<>(randomSubsetOf(Arrays.asList(DiscoveryNode.Role.values()))), - randomBoolean() ? localNode.getVersion() : VersionUtils.randomVersion(random())); - - joinNode(other_node); - - assertTrue(clusterService.localNode().equalsIncludingMetaData(other_node)); - } - public void testNormalConcurrentJoins() throws InterruptedException { Thread[] threads = new Thread[3 + randomInt(5)]; ArrayList nodes = new ArrayList<>(); diff --git a/core/src/test/java/org/elasticsearch/discovery/zen/ZenDiscoveryUnitTests.java b/core/src/test/java/org/elasticsearch/discovery/zen/ZenDiscoveryUnitTests.java index 9db83f48f0ea7..4cdb523ec3f92 100644 --- a/core/src/test/java/org/elasticsearch/discovery/zen/ZenDiscoveryUnitTests.java +++ b/core/src/test/java/org/elasticsearch/discovery/zen/ZenDiscoveryUnitTests.java @@ -50,9 +50,9 @@ public void testShouldIgnoreNewClusterState() { ClusterName clusterName = new ClusterName("abc"); DiscoveryNodes.Builder currentNodes = DiscoveryNodes.builder(); - currentNodes.masterNodeId("a").put(new DiscoveryNode("a", LocalTransportAddress.buildUnique(), emptyMap(), emptySet(), Version.CURRENT)); + currentNodes.masterNodeId("a").put(new DiscoveryNode("a", "a", LocalTransportAddress.buildUnique(), emptyMap(), emptySet(), Version.CURRENT)); DiscoveryNodes.Builder newNodes = DiscoveryNodes.builder(); - newNodes.masterNodeId("a").put(new DiscoveryNode("a", LocalTransportAddress.buildUnique(), emptyMap(), emptySet(), Version.CURRENT)); + newNodes.masterNodeId("a").put(new DiscoveryNode("a", "a", LocalTransportAddress.buildUnique(), emptyMap(), emptySet(), Version.CURRENT)); ClusterState.Builder currentState = ClusterState.builder(clusterName); currentState.nodes(currentNodes); @@ -70,7 +70,7 @@ public void testShouldIgnoreNewClusterState() { assertFalse("should not ignore, because new state's version is higher to current state's version", shouldIgnoreOrRejectNewClusterState(logger, currentState.build(), newState.build())); currentNodes = DiscoveryNodes.builder(); - currentNodes.masterNodeId("b").put(new DiscoveryNode("b", LocalTransportAddress.buildUnique(), emptyMap(), emptySet(), Version.CURRENT)); + currentNodes.masterNodeId("b").put(new DiscoveryNode("b", "b", LocalTransportAddress.buildUnique(), emptyMap(), emptySet(), Version.CURRENT)); ; // version isn't taken into account, so randomize it to ensure this. if (randomBoolean()) { diff --git a/core/src/test/java/org/elasticsearch/discovery/zen/publish/PublishClusterStateActionTests.java b/core/src/test/java/org/elasticsearch/discovery/zen/publish/PublishClusterStateActionTests.java index ace329ca68a76..b684db8e7eafe 100644 --- a/core/src/test/java/org/elasticsearch/discovery/zen/publish/PublishClusterStateActionTests.java +++ b/core/src/test/java/org/elasticsearch/discovery/zen/publish/PublishClusterStateActionTests.java @@ -163,7 +163,7 @@ public MockNode createMockNode(String name, Settings baseSettings, Version versi MockTransportService service = buildTransportService(settings, version); DiscoveryNodeService discoveryNodeService = new DiscoveryNodeService(settings, version); DiscoveryNode discoveryNode = discoveryNodeService.buildLocalNode( - service.boundAddress().publishAddress(), () -> NodeEnvironment.generateNodeId(settings)); + service.boundAddress().publishAddress(), NodeEnvironment.generateNodeId(Settings.EMPTY)); MockNode node = new MockNode(discoveryNode, service, listener, logger); node.action = buildPublishClusterStateAction(settings, service, () -> node.clusterState, node); final CountDownLatch latch = new CountDownLatch(nodes.size() * 2 + 1); @@ -320,9 +320,6 @@ public void testSimpleClusterStatePublishing() throws Exception { // node B becomes the master and sends a version of the cluster state that goes back discoveryNodes = DiscoveryNodes.builder(discoveryNodes) - .put(nodeA.discoveryNode) - .put(nodeB.discoveryNode) - .put(nodeC.discoveryNode) .masterNodeId(nodeB.discoveryNode.getId()) .localNodeId(nodeB.discoveryNode.getId()) .build(); diff --git a/core/src/test/java/org/elasticsearch/gateway/AsyncShardFetchTests.java b/core/src/test/java/org/elasticsearch/gateway/AsyncShardFetchTests.java index 2b8cdf46f7d34..8fa027711df8b 100644 --- a/core/src/test/java/org/elasticsearch/gateway/AsyncShardFetchTests.java +++ b/core/src/test/java/org/elasticsearch/gateway/AsyncShardFetchTests.java @@ -44,11 +44,11 @@ /** */ public class AsyncShardFetchTests extends ESTestCase { - private final DiscoveryNode node1 = new DiscoveryNode("node1", LocalTransportAddress.buildUnique(), Collections.emptyMap(), + private final DiscoveryNode node1 = new DiscoveryNode("process1", "node1", LocalTransportAddress.buildUnique(), Collections.emptyMap(), Collections.singleton(DiscoveryNode.Role.DATA), Version.CURRENT); private final Response response1 = new Response(node1); private final Throwable failure1 = new Throwable("simulated failure 1"); - private final DiscoveryNode node2 = new DiscoveryNode("node2", LocalTransportAddress.buildUnique(), Collections.emptyMap(), + private final DiscoveryNode node2 = new DiscoveryNode("process2", "node2", LocalTransportAddress.buildUnique(), Collections.emptyMap(), Collections.singleton(DiscoveryNode.Role.DATA), Version.CURRENT); private final Response response2 = new Response(node2); private final Throwable failure2 = new Throwable("simulate failure 2"); diff --git a/core/src/test/java/org/elasticsearch/ingest/PipelineStoreTests.java b/core/src/test/java/org/elasticsearch/ingest/PipelineStoreTests.java index 10321e30301b8..2a2a34d4daef1 100644 --- a/core/src/test/java/org/elasticsearch/ingest/PipelineStoreTests.java +++ b/core/src/test/java/org/elasticsearch/ingest/PipelineStoreTests.java @@ -222,7 +222,7 @@ public void testValidate() throws Exception { store.validatePipeline(ingestInfos, putRequest); fail("exception expected"); } catch (IllegalArgumentException e) { - assertThat(e.getMessage(), equalTo("Processor type [remove] is not installed on node [{_node_id2}{local}{local[_id]}]")); + assertThat(e.getMessage(), equalTo("Processor type [remove] is not installed on node [" + node2 + "]")); } ingestInfos.put(node2, new IngestInfo(Arrays.asList(new ProcessorInfo("set"), new ProcessorInfo("remove")))); diff --git a/qa/evil-tests/src/test/java/org/elasticsearch/tribe/TribeUnitTests.java b/qa/evil-tests/src/test/java/org/elasticsearch/tribe/TribeUnitTests.java index 20a8ec67773e6..49c2d56d6760a 100644 --- a/qa/evil-tests/src/test/java/org/elasticsearch/tribe/TribeUnitTests.java +++ b/qa/evil-tests/src/test/java/org/elasticsearch/tribe/TribeUnitTests.java @@ -66,14 +66,14 @@ public static void createTribes() { .put(baseSettings) .put("cluster.name", "tribe1") .put("node.name", "tribe1_node") - .put(NodeEnvironment.NODE_ID_SEED_SETTING.getKey(), random().nextLong()) + .put(NodeEnvironment.NODE_ID_SEED_SETTING.getKey(), random().nextLong()) .build()).start(); tribe2 = new TribeClientNode( Settings.builder() .put(baseSettings) .put("cluster.name", "tribe2") .put("node.name", "tribe2_node") - .put(NodeEnvironment.NODE_ID_SEED_SETTING.getKey(), random().nextLong()) + .put(NodeEnvironment.NODE_ID_SEED_SETTING.getKey(), random().nextLong()) .build()).start(); } diff --git a/test/framework/src/main/java/org/elasticsearch/cluster/MockInternalClusterInfoService.java b/test/framework/src/main/java/org/elasticsearch/cluster/MockInternalClusterInfoService.java index c897b7f491e23..3371821f1ca0c 100644 --- a/test/framework/src/main/java/org/elasticsearch/cluster/MockInternalClusterInfoService.java +++ b/test/framework/src/main/java/org/elasticsearch/cluster/MockInternalClusterInfoService.java @@ -73,7 +73,8 @@ public static NodeStats makeStats(String nodeName, DiskUsage usage) { usage.getTotalBytes(), usage.getFreeBytes(), usage.getFreeBytes()); paths[0] = path; FsInfo fsInfo = new FsInfo(System.currentTimeMillis(), paths); - return new NodeStats(new DiscoveryNode(nodeName, LocalTransportAddress.buildUnique(), emptyMap(), emptySet(), Version.CURRENT), + return new NodeStats(new DiscoveryNode(nodeName, nodeName, nodeName, LocalTransportAddress.buildUnique(), emptyMap(), emptySet(), + Version.CURRENT), System.currentTimeMillis(), null, null, null, null, null, fsInfo, diff --git a/test/framework/src/main/java/org/elasticsearch/test/ESAllocationTestCase.java b/test/framework/src/main/java/org/elasticsearch/test/ESAllocationTestCase.java index ea7e2964388aa..9b1202e8c2418 100644 --- a/test/framework/src/main/java/org/elasticsearch/test/ESAllocationTestCase.java +++ b/test/framework/src/main/java/org/elasticsearch/test/ESAllocationTestCase.java @@ -132,19 +132,19 @@ protected static DiscoveryNode newNode(String nodeId) { } protected static DiscoveryNode newNode(String nodeName, String nodeId, Map attributes) { - return new DiscoveryNode(nodeName, nodeId, LocalTransportAddress.buildUnique(), attributes, MASTER_DATA_ROLES, Version.CURRENT); + return new DiscoveryNode(nodeName, nodeId, nodeId, LocalTransportAddress.buildUnique(), attributes, MASTER_DATA_ROLES, Version.CURRENT); } protected static DiscoveryNode newNode(String nodeId, Map attributes) { - return new DiscoveryNode(nodeId, LocalTransportAddress.buildUnique(), attributes, MASTER_DATA_ROLES, Version.CURRENT); + return new DiscoveryNode(nodeId, nodeId, LocalTransportAddress.buildUnique(), attributes, MASTER_DATA_ROLES, Version.CURRENT); } protected static DiscoveryNode newNode(String nodeId, Set roles) { - return new DiscoveryNode(nodeId, LocalTransportAddress.buildUnique(), emptyMap(), roles, Version.CURRENT); + return new DiscoveryNode(nodeId, nodeId, LocalTransportAddress.buildUnique(), emptyMap(), roles, Version.CURRENT); } protected static DiscoveryNode newNode(String nodeId, Version version) { - return new DiscoveryNode(nodeId, LocalTransportAddress.buildUnique(), emptyMap(), MASTER_DATA_ROLES, version); + return new DiscoveryNode(nodeId, nodeId, LocalTransportAddress.buildUnique(), emptyMap(), MASTER_DATA_ROLES, version); } protected static ClusterState startRandomInitializingShard(ClusterState clusterState, AllocationService strategy) { diff --git a/test/framework/src/main/java/org/elasticsearch/test/InternalTestCluster.java b/test/framework/src/main/java/org/elasticsearch/test/InternalTestCluster.java index 328ceeb2ead9b..69e1fd3a2568e 100644 --- a/test/framework/src/main/java/org/elasticsearch/test/InternalTestCluster.java +++ b/test/framework/src/main/java/org/elasticsearch/test/InternalTestCluster.java @@ -39,6 +39,7 @@ import org.elasticsearch.cluster.action.index.MappingUpdatedAction; import org.elasticsearch.cluster.metadata.IndexMetaData; import org.elasticsearch.cluster.node.DiscoveryNode; +import org.elasticsearch.cluster.node.DiscoveryNodeService; import org.elasticsearch.cluster.node.DiscoveryNodes; import org.elasticsearch.cluster.routing.OperationRouting; import org.elasticsearch.cluster.routing.ShardRouting; @@ -577,6 +578,7 @@ private NodeAndClient buildNode(int nodeId, long seed, Settings settings, Versio .put(Environment.PATH_HOME_SETTING.getKey(), baseDir) // allow overriding path.home .put(settings) .put("node.name", name) + .put(DiscoveryNodeService.PROCESS_ID_SEED_SETTING.getKey(), seed) .put(NodeEnvironment.NODE_ID_SEED_SETTING.getKey(), seed) .build(); MockNode node = new MockNode(finalSettings, version, plugins); @@ -825,9 +827,14 @@ void restart(RestartCallback callback) throws Exception { IOUtils.rm(nodeEnv.nodeDataPaths()); } } + // use a new seed to make sure we have new process id + final long newProcessIdSeed = DiscoveryNodeService.PROCESS_ID_SEED_SETTING.get(node.settings()) + 1; // use a new seed to make sure we have new node id, if the node type is one that generates new ids - final long newIdSeed = NodeEnvironment.NODE_ID_SEED_SETTING.get(node.settings()) + 1; - Settings finalSettings = Settings.builder().put(node.settings()).put(newSettings).put(NodeEnvironment.NODE_ID_SEED_SETTING.getKey(), newIdSeed).build(); + final long newNodeIdSeed = NodeEnvironment.NODE_ID_SEED_SETTING.get(node.settings()) + 1; + Settings finalSettings = Settings.builder().put(node.settings()).put(newSettings) + .put(DiscoveryNodeService.PROCESS_ID_SEED_SETTING.getKey(), newProcessIdSeed) + .put(NodeEnvironment.NODE_ID_SEED_SETTING.getKey(), newNodeIdSeed) + .build(); Collection> plugins = node.getPlugins(); Version version = node.getVersion(); node = new MockNode(finalSettings, version, plugins); From e57232f9e1f1437c67c5607f2a93700d42529da3 Mon Sep 17 00:00:00 2001 From: Yannick Welsch Date: Thu, 28 Apr 2016 17:47:40 +0200 Subject: [PATCH 25/34] Generate node id of a tribe client node based on node id of parent tribe node --- .../org/elasticsearch/tribe/TribeService.java | 16 ++++++-- .../tribe/TribeServiceTests.java | 38 ++++++++++++++++--- 2 files changed, 45 insertions(+), 9 deletions(-) diff --git a/core/src/main/java/org/elasticsearch/tribe/TribeService.java b/core/src/main/java/org/elasticsearch/tribe/TribeService.java index 87c04544944ed..b0114b29ad927 100644 --- a/core/src/main/java/org/elasticsearch/tribe/TribeService.java +++ b/core/src/main/java/org/elasticsearch/tribe/TribeService.java @@ -19,6 +19,8 @@ package org.elasticsearch.tribe; +import org.apache.lucene.util.BytesRef; +import org.apache.lucene.util.StringHelper; import org.elasticsearch.ElasticsearchException; import org.elasticsearch.action.support.master.TransportMasterNodeReadAction; import org.elasticsearch.cluster.ClusterChangedEvent; @@ -52,6 +54,7 @@ import org.elasticsearch.discovery.DiscoveryModule; import org.elasticsearch.discovery.DiscoverySettings; import org.elasticsearch.env.Environment; +import org.elasticsearch.env.NodeEnvironment; import org.elasticsearch.gateway.GatewayService; import org.elasticsearch.node.Node; import org.elasticsearch.rest.RestStatus; @@ -128,6 +131,7 @@ public static Settings processSettings(Settings settings) { // internal settings only public static final Setting TRIBE_NAME_SETTING = Setting.simpleString("tribe.name", Property.NodeScope); private final ClusterService clusterService; + private final NodeEnvironment nodeEnvironment; private final String[] blockIndicesWrite; private final String[] blockIndicesRead; private final String[] blockIndicesMetadata; @@ -176,14 +180,15 @@ public static Settings processSettings(Settings settings) { private final List nodes = new CopyOnWriteArrayList<>(); @Inject - public TribeService(Settings settings, ClusterService clusterService) { + public TribeService(Settings settings, ClusterService clusterService, NodeEnvironment nodeEnvironment) { super(settings); this.clusterService = clusterService; + this.nodeEnvironment = nodeEnvironment; Map nodesSettings = new HashMap<>(settings.getGroups("tribe", true)); nodesSettings.remove("blocks"); // remove prefix settings that don't indicate a client nodesSettings.remove("on_conflict"); // remove prefix settings that don't indicate a client for (Map.Entry entry : nodesSettings.entrySet()) { - Settings clientSettings = buildClientSettings(entry.getKey(), settings, entry.getValue()); + Settings clientSettings = buildClientSettings(entry.getKey(), nodeEnvironment.nodeId(), settings, entry.getValue()); nodes.add(new TribeClientNode(clientSettings)); } @@ -208,7 +213,7 @@ public TribeService(Settings settings, ClusterService clusterService) { * Builds node settings for a tribe client node from the tribe node's global settings, * combined with tribe specific settings. */ - static Settings buildClientSettings(String tribeName, Settings globalSettings, Settings tribeSettings) { + static Settings buildClientSettings(String tribeName, String parentNodeId, Settings globalSettings, Settings tribeSettings) { for (String tribeKey : tribeSettings.getAsMap().keySet()) { if (tribeKey.startsWith("path.")) { throw new IllegalArgumentException("Setting [" + tribeKey + "] not allowed in tribe client [" + tribeName + "]"); @@ -241,7 +246,10 @@ static Settings buildClientSettings(String tribeName, Settings globalSettings, S sb.put(Node.NODE_DATA_SETTING.getKey(), false); sb.put(Node.NODE_MASTER_SETTING.getKey(), false); sb.put(Node.NODE_INGEST_SETTING.getKey(), false); - sb.put(Node.NODE_LOCAL_STORAGE_SETTING.getKey(), false); + // node id of a tribe client node is determined by node id of parent node and tribe name + int nodeIdSeed = StringHelper.murmurhash3_x86_32(new BytesRef(parentNodeId + "/" + tribeName), 0); + sb.put(NodeEnvironment.NODE_ID_SEED_SETTING.getKey(), nodeIdSeed); + sb.put(Node.NODE_LOCAL_STORAGE_SETTING.getKey(), false); // ensures node id is not persisted for tribe client node return sb.build(); } diff --git a/core/src/test/java/org/elasticsearch/tribe/TribeServiceTests.java b/core/src/test/java/org/elasticsearch/tribe/TribeServiceTests.java index 97ffb7a2c8182..9bab67d26c6a5 100644 --- a/core/src/test/java/org/elasticsearch/tribe/TribeServiceTests.java +++ b/core/src/test/java/org/elasticsearch/tribe/TribeServiceTests.java @@ -19,15 +19,18 @@ package org.elasticsearch.tribe; +import org.elasticsearch.cluster.node.DiscoveryNode; import org.elasticsearch.common.settings.Settings; import org.elasticsearch.test.ESTestCase; +import static org.elasticsearch.env.NodeEnvironment.NODE_ID_SEED_SETTING; + public class TribeServiceTests extends ESTestCase { public void testMinimalSettings() { Settings globalSettings = Settings.builder() .put("node.name", "nodename") .put("path.home", "some/path").build(); - Settings clientSettings = TribeService.buildClientSettings("tribe1", globalSettings, Settings.EMPTY); + Settings clientSettings = TribeService.buildClientSettings("tribe1", "dummyNodeId", globalSettings, Settings.EMPTY); assertEquals("some/path", clientSettings.get("path.home")); assertEquals("nodename/tribe1", clientSettings.get("node.name")); assertEquals("tribe1", clientSettings.get("tribe.name")); @@ -47,7 +50,7 @@ public void testEnvironmentSettings() { .put("path.plugins", "plugins/path") .put("path.scripts", "scripts/path") .put("path.logs", "logs/path").build(); - Settings clientSettings = TribeService.buildClientSettings("tribe1", globalSettings, Settings.EMPTY); + Settings clientSettings = TribeService.buildClientSettings("tribe1", "dummyNodeId", globalSettings, Settings.EMPTY); assertEquals("some/path", clientSettings.get("path.home")); assertEquals("conf/path", clientSettings.get("path.conf")); assertEquals("plugins/path", clientSettings.get("path.plugins")); @@ -57,7 +60,7 @@ public void testEnvironmentSettings() { Settings tribeSettings = Settings.builder() .put("path.home", "alternate/path").build(); IllegalArgumentException e = expectThrows(IllegalArgumentException.class, () -> { - TribeService.buildClientSettings("tribe1", globalSettings, tribeSettings); + TribeService.buildClientSettings("tribe1", "dummyNodeId", globalSettings, tribeSettings); }); assertTrue(e.getMessage(), e.getMessage().contains("Setting [path.home] not allowed in tribe client")); } @@ -72,7 +75,7 @@ public void testPassthroughSettings() { .put("transport.host", "3.3.3.3") .put("transport.bind_host", "4.4.4.4") .put("transport.publish_host", "5.5.5.5").build(); - Settings clientSettings = TribeService.buildClientSettings("tribe1", globalSettings, Settings.EMPTY); + Settings clientSettings = TribeService.buildClientSettings("tribe1", "dummyNodeId", globalSettings, Settings.EMPTY); assertEquals("0.0.0.0", clientSettings.get("network.host")); assertEquals("1.1.1.1", clientSettings.get("network.bind_host")); assertEquals("2.2.2.2", clientSettings.get("network.publish_host")); @@ -88,7 +91,7 @@ public void testPassthroughSettings() { .put("transport.host", "6.6.6.6") .put("transport.bind_host", "7.7.7.7") .put("transport.publish_host", "8.8.8.8").build(); - clientSettings = TribeService.buildClientSettings("tribe1", globalSettings, tribeSettings); + clientSettings = TribeService.buildClientSettings("tribe1", "dummyNodeId", globalSettings, tribeSettings); assertEquals("3.3.3.3", clientSettings.get("network.host")); assertEquals("4.4.4.4", clientSettings.get("network.bind_host")); assertEquals("5.5.5.5", clientSettings.get("network.publish_host")); @@ -96,4 +99,29 @@ public void testPassthroughSettings() { assertEquals("7.7.7.7", clientSettings.get("transport.bind_host")); assertEquals("8.8.8.8", clientSettings.get("transport.publish_host")); } + + public void testNodeIdSettings() { + Settings globalSettings = Settings.builder() + .put("node.name", "nodename") + .put("path.home", "some/path").build(); + String parentNodeId = randomAsciiOfLengthBetween(5, 10); + String tribe1Name = randomAsciiOfLengthBetween(5, 10); + Settings client1Settings = TribeService.buildClientSettings(tribe1Name, parentNodeId, globalSettings, Settings.EMPTY); + assertTrue(NODE_ID_SEED_SETTING.exists(client1Settings)); + assertFalse(DiscoveryNode.nodeRequiresLocalStorage(client1Settings)); + + // node id of tribe client node is deterministic + assertEquals(NODE_ID_SEED_SETTING.get(client1Settings), + NODE_ID_SEED_SETTING.get(TribeService.buildClientSettings(tribe1Name, parentNodeId, globalSettings, Settings.EMPTY))); + + // tribe client node with different id gets a different node id + String tribe2Name = randomAsciiOfLengthBetween(5, 10); + Settings client2Settings = TribeService.buildClientSettings(tribe2Name, parentNodeId, globalSettings, Settings.EMPTY); + assertNotEquals(NODE_ID_SEED_SETTING.get(client1Settings), NODE_ID_SEED_SETTING.get(client2Settings)); + + // tribe client node for a different parent gets a different node id + assertNotEquals(NODE_ID_SEED_SETTING.get(client1Settings), + NODE_ID_SEED_SETTING.get(TribeService.buildClientSettings(tribe1Name, randomAsciiOfLengthBetween(5, 10), globalSettings, + Settings.EMPTY))); + } } From 49b871afda705fe2abd80b965580e9a99d0eaf1d Mon Sep 17 00:00:00 2001 From: Yannick Welsch Date: Thu, 28 Apr 2016 17:54:26 +0200 Subject: [PATCH 26/34] fix failing test --- .../test/java/org/elasticsearch/tribe/TribeServiceTests.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/core/src/test/java/org/elasticsearch/tribe/TribeServiceTests.java b/core/src/test/java/org/elasticsearch/tribe/TribeServiceTests.java index 9bab67d26c6a5..3f79236db77c1 100644 --- a/core/src/test/java/org/elasticsearch/tribe/TribeServiceTests.java +++ b/core/src/test/java/org/elasticsearch/tribe/TribeServiceTests.java @@ -39,7 +39,7 @@ public void testMinimalSettings() { assertEquals("false", clientSettings.get("node.data")); assertEquals("false", clientSettings.get("node.ingest")); assertEquals("false", clientSettings.get("node.local_storage")); - assertEquals(8, clientSettings.getAsMap().size()); + assertEquals(9, clientSettings.getAsMap().size()); } public void testEnvironmentSettings() { From 7f95af0904d6f1a23a1b452c018e551d696b4afb Mon Sep 17 00:00:00 2001 From: Yannick Welsch Date: Thu, 28 Apr 2016 18:12:19 +0200 Subject: [PATCH 27/34] naming --- .../TransportClientNodesService.java | 2 +- .../cluster/node/DiscoveryNode.java | 35 +++++++++---------- .../cluster/node/DiscoveryNodes.java | 6 ++-- .../org/elasticsearch/tribe/TribeService.java | 2 +- .../TransportClientNodesServiceTests.java | 2 +- .../cluster/node/DiscoveryNodeTests.java | 14 ++++---- .../cluster/node/DiscoveryNodesTests.java | 2 +- 7 files changed, 31 insertions(+), 32 deletions(-) diff --git a/core/src/main/java/org/elasticsearch/client/transport/TransportClientNodesService.java b/core/src/main/java/org/elasticsearch/client/transport/TransportClientNodesService.java index e844dbff1bf54..ed57b4d86a3a0 100644 --- a/core/src/main/java/org/elasticsearch/client/transport/TransportClientNodesService.java +++ b/core/src/main/java/org/elasticsearch/client/transport/TransportClientNodesService.java @@ -383,7 +383,7 @@ public LivenessResponse newInstance() { // use discovered information but do keep the original transport address, // so people can control which address is exactly used. DiscoveryNode nodeWithInfo = livenessResponse.getDiscoveryNode(); - newNodes.add(new DiscoveryNode(nodeWithInfo.getName(), nodeWithInfo.getId(), nodeWithInfo.getPersistentNodeId(), + newNodes.add(new DiscoveryNode(nodeWithInfo.getName(), nodeWithInfo.getId(), nodeWithInfo.getNodeId(), nodeWithInfo.getHostName(), nodeWithInfo.getHostAddress(), listedNode.getAddress(), nodeWithInfo.getAttributes(), nodeWithInfo.getRoles(), nodeWithInfo.getVersion())); } else { diff --git a/core/src/main/java/org/elasticsearch/cluster/node/DiscoveryNode.java b/core/src/main/java/org/elasticsearch/cluster/node/DiscoveryNode.java index e343904ec2d26..967aa11a788ea 100644 --- a/core/src/main/java/org/elasticsearch/cluster/node/DiscoveryNode.java +++ b/core/src/main/java/org/elasticsearch/cluster/node/DiscoveryNode.java @@ -20,7 +20,6 @@ package org.elasticsearch.cluster.node; import org.elasticsearch.Version; -import org.elasticsearch.common.Strings; import org.elasticsearch.common.io.stream.StreamInput; import org.elasticsearch.common.io.stream.StreamOutput; import org.elasticsearch.common.io.stream.Writeable; @@ -89,7 +88,7 @@ public static boolean isIngestNode(Settings settings) { private final String nodeName; private final String processId; - private final String persistentNodeId; + private final String nodeId; private final String hostName; private final String hostAddress; private final TransportAddress address; @@ -127,15 +126,15 @@ public DiscoveryNode(String processAndNodeId, TransportAddress address, Map * * @param processId the nodes unique process id - * @param persistentNodeId the nodes unique persistent id + * @param nodeId the nodes unique persistent id * @param address the nodes transport address * @param attributes node attributes * @param roles node roles * @param version the version of the node */ - public DiscoveryNode(String processId, String persistentNodeId, TransportAddress address, Map attributes, + public DiscoveryNode(String processId, String nodeId, TransportAddress address, Map attributes, Set roles, Version version) { - this("", processId, persistentNodeId, address.getHost(), address.getAddress(), address, attributes, roles, version); + this("", processId, nodeId, address.getHost(), address.getAddress(), address, attributes, roles, version); } /** @@ -149,15 +148,15 @@ public DiscoveryNode(String processId, String persistentNodeId, TransportAddress * * @param nodeName the nodes name * @param processId the nodes unique process id - * @param persistentNodeId the nodes unique persistent id + * @param nodeId the nodes unique persistent id * @param address the nodes transport address * @param attributes node attributes * @param roles node roles * @param version the version of the node */ - public DiscoveryNode(String nodeName, String processId, String persistentNodeId, TransportAddress address, + public DiscoveryNode(String nodeName, String processId, String nodeId, TransportAddress address, Map attributes, Set roles, Version version) { - this(nodeName, processId, persistentNodeId, address.getHost(), address.getAddress(), address, attributes, roles, version); + this(nodeName, processId, nodeId, address.getHost(), address.getAddress(), address, attributes, roles, version); } /** @@ -171,14 +170,14 @@ public DiscoveryNode(String nodeName, String processId, String persistentNodeId, * * @param nodeName the nodes name * @param processId the nodes unique process id - * @param persistentNodeId the nodes unique persistent id + * @param nodeId the nodes unique persistent id * @param hostAddress the nodes host address * @param address the nodes transport address * @param attributes node attributes * @param roles node roles * @param version the version of the node */ - public DiscoveryNode(String nodeName, String processId, String persistentNodeId, String hostName, String hostAddress, + public DiscoveryNode(String nodeName, String processId, String nodeId, String hostName, String hostAddress, TransportAddress address, Map attributes, Set roles, Version version) { if (nodeName != null) { this.nodeName = nodeName.intern(); @@ -186,7 +185,7 @@ public DiscoveryNode(String nodeName, String processId, String persistentNodeId, this.nodeName = ""; } this.processId = processId.intern(); - this.persistentNodeId = persistentNodeId.intern(); + this.nodeId = nodeId.intern(); this.hostName = hostName.intern(); this.hostAddress = hostAddress.intern(); this.address = address; @@ -217,7 +216,7 @@ public DiscoveryNode(String nodeName, String processId, String persistentNodeId, public DiscoveryNode(StreamInput in) throws IOException { this.nodeName = in.readString().intern(); this.processId = in.readString().intern(); - this.persistentNodeId = in.readString().intern(); + this.nodeId = in.readString().intern(); this.hostName = in.readString().intern(); this.hostAddress = in.readString().intern(); this.address = TransportAddressSerializers.addressFromStream(in); @@ -242,7 +241,7 @@ public DiscoveryNode(StreamInput in) throws IOException { public void writeTo(StreamOutput out) throws IOException { out.writeString(nodeName); out.writeString(processId); - out.writeString(persistentNodeId); + out.writeString(nodeId); out.writeString(hostName); out.writeString(hostAddress); addressToStream(out, address); @@ -275,8 +274,8 @@ public String getId() { /** * The unique process id of the node. */ - public String getPersistentNodeId() { - return persistentNodeId; + public String getNodeId() { + return nodeId; } /** @@ -344,7 +343,7 @@ public boolean equalsIncludingMetaData(DiscoveryNode other) { return false; } - if (!persistentNodeId.equals(other.persistentNodeId)) { + if (!nodeId.equals(other.nodeId)) { return false; } if (!nodeName.equals(other.nodeName)) { @@ -395,7 +394,7 @@ public String toString() { sb.append('{').append(nodeName).append('}'); } sb.append('{').append(processId).append('}'); - sb.append('{').append(persistentNodeId).append('}'); + sb.append('{').append(nodeId).append('}'); sb.append('{').append(hostName).append('}'); sb.append('{').append(address).append('}'); if (!attributes.isEmpty()) { @@ -408,7 +407,7 @@ public String toString() { public XContentBuilder toXContent(XContentBuilder builder, Params params) throws IOException { builder.startObject(getId()); builder.field("name", getName()); - builder.field("persistent_id", getPersistentNodeId()); + builder.field("node_id", getNodeId()); builder.field("transport_address", getAddress().toString()); builder.startObject("attributes"); diff --git a/core/src/main/java/org/elasticsearch/cluster/node/DiscoveryNodes.java b/core/src/main/java/org/elasticsearch/cluster/node/DiscoveryNodes.java index 3f1ff163764f7..a66fd97eaee8d 100644 --- a/core/src/main/java/org/elasticsearch/cluster/node/DiscoveryNodes.java +++ b/core/src/main/java/org/elasticsearch/cluster/node/DiscoveryNodes.java @@ -282,7 +282,7 @@ public String[] resolveNodesIds(String... nodesIds) { } else { boolean matchedPersistentNodeId = false; for (DiscoveryNode node : this) { - if (node.getPersistentNodeId().equals(nodeId)) { + if (node.getNodeId().equals(nodeId)) { matchedPersistentNodeId = true; resolvedNodesIds.add(node.getId()); break; // persistent node id is unique @@ -586,7 +586,7 @@ public Builder(DiscoveryNodes nodes) { public Builder put(DiscoveryNode node) { assert nodes.get(node.getId()) == null : "node with process id " + node.getId() + " already exists"; assert nodeWithSameAddress(node) == null : "node with address " + node.getAddress() + " already exists"; - assert nodeWithSamePersistentId(node) == null : "node with persistent id " + node.getPersistentNodeId() + " already exists"; + assert nodeWithSamePersistentId(node) == null : "node with persistent id " + node.getNodeId() + " already exists"; nodes.put(node.getId(), node); return this; } @@ -619,7 +619,7 @@ public DiscoveryNode nodeWithSameAddress(DiscoveryNode node) { public DiscoveryNode nodeWithSamePersistentId(DiscoveryNode node) { for (ObjectCursor cursor : nodes.values()) { final DiscoveryNode existingNode = cursor.value; - if (node.getPersistentNodeId().equals(existingNode.getPersistentNodeId())) { + if (node.getNodeId().equals(existingNode.getNodeId())) { return existingNode; } } diff --git a/core/src/main/java/org/elasticsearch/tribe/TribeService.java b/core/src/main/java/org/elasticsearch/tribe/TribeService.java index b0114b29ad927..30a3c02fdee18 100644 --- a/core/src/main/java/org/elasticsearch/tribe/TribeService.java +++ b/core/src/main/java/org/elasticsearch/tribe/TribeService.java @@ -376,7 +376,7 @@ private ClusterState applyUpdate(ClusterState currentState, ClusterChangedEvent // a new node, add it, but also add the tribe name to the attributes Map tribeAttr = new HashMap<>(tribe.getAttributes()); tribeAttr.put(TRIBE_NAME_SETTING.getKey(), tribeName); - DiscoveryNode discoNode = new DiscoveryNode(tribe.getName(), tribe.getId(), tribe.getPersistentNodeId(), + DiscoveryNode discoNode = new DiscoveryNode(tribe.getName(), tribe.getId(), tribe.getNodeId(), tribe.getHostName(), tribe.getHostAddress(), tribe.getAddress(), unmodifiableMap(tribeAttr), tribe.getRoles(), tribe.getVersion()); clusterStateChanged = true; diff --git a/core/src/test/java/org/elasticsearch/client/transport/TransportClientNodesServiceTests.java b/core/src/test/java/org/elasticsearch/client/transport/TransportClientNodesServiceTests.java index 05a29d041294a..01b968abcd558 100644 --- a/core/src/test/java/org/elasticsearch/client/transport/TransportClientNodesServiceTests.java +++ b/core/src/test/java/org/elasticsearch/client/transport/TransportClientNodesServiceTests.java @@ -119,7 +119,7 @@ public T newInstance() { @SuppressWarnings("unchecked") public void handleResponse(T response) { LivenessResponse livenessResponse = new LivenessResponse(ClusterName.DEFAULT, - new DiscoveryNode(node.getName(), node.getId(), node.getPersistentNodeId(), "liveness-hostname" + node.getId(), + new DiscoveryNode(node.getName(), node.getId(), node.getNodeId(), "liveness-hostname" + node.getId(), "liveness-hostaddress" + node.getId(), new LocalTransportAddress("liveness-address-" + node.getId()), node.getAttributes(), node.getRoles(), node.getVersion())); diff --git a/core/src/test/java/org/elasticsearch/cluster/node/DiscoveryNodeTests.java b/core/src/test/java/org/elasticsearch/cluster/node/DiscoveryNodeTests.java index 779e814727def..41602630fabde 100644 --- a/core/src/test/java/org/elasticsearch/cluster/node/DiscoveryNodeTests.java +++ b/core/src/test/java/org/elasticsearch/cluster/node/DiscoveryNodeTests.java @@ -34,7 +34,7 @@ public class DiscoveryNodeTests extends ESTestCase { private static final AtomicInteger idGenerator = new AtomicInteger(); /** - * guarantees to generate a node that is unique for all aspects but for it's roles and version (which are chosen randomly) + * guarantees to generate a node that is unique for all aspects but for its roles and version (which are chosen randomly) */ DiscoveryNode randomNode() { Map attributes = new HashMap<>(); @@ -44,7 +44,7 @@ DiscoveryNode randomNode() { final DiscoveryNode node = new DiscoveryNode( "name_" + idGenerator.incrementAndGet(), "processId_" + idGenerator.incrementAndGet(), - "persistentNodeId_" + idGenerator.incrementAndGet(), + "nodeId_" + idGenerator.incrementAndGet(), LocalTransportAddress.buildUnique(), attributes, new HashSet<>(randomSubsetOf(Arrays.asList(DiscoveryNode.Role.values()))), randomVersion(random())); @@ -62,27 +62,27 @@ public void testEquality() { DiscoveryNode sameIdDifferentMeta; - sameIdDifferentMeta = new DiscoveryNode(node2.getName(), node1.getId(), node1.getPersistentNodeId(), node2.getAddress(), + sameIdDifferentMeta = new DiscoveryNode(node2.getName(), node1.getId(), node1.getNodeId(), node2.getAddress(), node2.getAttributes(), node2.getRoles(), node2.getVersion()); assertTrue(node1.equals(sameIdDifferentMeta)); assertFalse(node1.equalsIncludingMetaData(sameIdDifferentMeta)); - sameIdDifferentMeta = new DiscoveryNode(node1.getName(), node1.getId(), node2.getPersistentNodeId(), node1.getAddress(), + sameIdDifferentMeta = new DiscoveryNode(node1.getName(), node1.getId(), node2.getNodeId(), node1.getAddress(), node1.getAttributes(), node1.getRoles(), node1.getVersion()); assertTrue(node1.equals(sameIdDifferentMeta)); assertFalse(node1.equalsIncludingMetaData(sameIdDifferentMeta)); - sameIdDifferentMeta = new DiscoveryNode(node2.getName(), node1.getId(), node1.getPersistentNodeId(), node1.getAddress(), + sameIdDifferentMeta = new DiscoveryNode(node2.getName(), node1.getId(), node1.getNodeId(), node1.getAddress(), node1.getAttributes(), (randomBoolean() ? node1 : node2).getRoles(), (randomBoolean() ? node1 : node2).getVersion()); assertTrue(node1.equals(sameIdDifferentMeta)); assertFalse(node1.equalsIncludingMetaData(sameIdDifferentMeta)); - sameIdDifferentMeta = new DiscoveryNode(node1.getName(), node1.getId(), node1.getPersistentNodeId(), node2.getAddress(), + sameIdDifferentMeta = new DiscoveryNode(node1.getName(), node1.getId(), node1.getNodeId(), node2.getAddress(), node1.getAttributes(), (randomBoolean() ? node1 : node2).getRoles(), (randomBoolean() ? node1 : node2).getVersion()); assertTrue(node1.equals(sameIdDifferentMeta)); assertFalse(node1.equalsIncludingMetaData(sameIdDifferentMeta)); - sameIdDifferentMeta = new DiscoveryNode(node1.getName(), node1.getId(), node1.getPersistentNodeId(), node1.getAddress(), + sameIdDifferentMeta = new DiscoveryNode(node1.getName(), node1.getId(), node1.getNodeId(), node1.getAddress(), node2.getAttributes(), (randomBoolean() ? node1 : node2).getRoles(), (randomBoolean() ? node1 : node2).getVersion()); assertTrue(node1.equals(sameIdDifferentMeta)); assertFalse(node1.equalsIncludingMetaData(sameIdDifferentMeta)); diff --git a/core/src/test/java/org/elasticsearch/cluster/node/DiscoveryNodesTests.java b/core/src/test/java/org/elasticsearch/cluster/node/DiscoveryNodesTests.java index 7f25c1846e897..96974a155abe7 100644 --- a/core/src/test/java/org/elasticsearch/cluster/node/DiscoveryNodesTests.java +++ b/core/src/test/java/org/elasticsearch/cluster/node/DiscoveryNodesTests.java @@ -41,7 +41,7 @@ public void testResolveNodeByIdOrName() { DiscoveryNodes discoveryNodes = buildDiscoveryNodes(); DiscoveryNode[] nodes = discoveryNodes.getNodes().values().toArray(DiscoveryNode.class); DiscoveryNode node = randomFrom(nodes); - DiscoveryNode resolvedNode = discoveryNodes.resolveNode(randomFrom(node.getId(), node.getPersistentNodeId(), node.getName())); + DiscoveryNode resolvedNode = discoveryNodes.resolveNode(randomFrom(node.getId(), node.getNodeId(), node.getName())); assertThat(resolvedNode.getId(), equalTo(node.getId())); } From 7dbe1bf50230b9f61bc815350f91cd0c5341257c Mon Sep 17 00:00:00 2001 From: Yannick Welsch Date: Tue, 3 May 2016 10:49:48 +0200 Subject: [PATCH 28/34] rename nodeId() to getNodeId() and documentation --- .../java/org/elasticsearch/env/NodeEnvironment.java | 8 ++++++-- .../main/java/org/elasticsearch/env/NodeMetaData.java | 6 ++++-- .../org/elasticsearch/gateway/MetaDataStateFormat.java | 9 +++++---- core/src/main/java/org/elasticsearch/node/Node.java | 2 +- .../java/org/elasticsearch/tribe/TribeService.java | 2 +- .../routing/allocation/BalanceConfigurationTests.java | 6 +++--- .../org/elasticsearch/env/NodeEnvironmentTests.java | 10 +++++----- 7 files changed, 25 insertions(+), 18 deletions(-) diff --git a/core/src/main/java/org/elasticsearch/env/NodeEnvironment.java b/core/src/main/java/org/elasticsearch/env/NodeEnvironment.java index ed75249625802..fa982be2a8619 100644 --- a/core/src/main/java/org/elasticsearch/env/NodeEnvironment.java +++ b/core/src/main/java/org/elasticsearch/env/NodeEnvironment.java @@ -154,6 +154,10 @@ public String toString() { Setting.boolSetting("node.add_lock_id_to_custom_path", true, Property.NodeScope); + /** + * Seed for determining the unique uuid of this node. If the node has already a persisted uuid on disk, + * this seed will be ignored and the uuid from disk will be reused. + */ public static final Setting NODE_ID_SEED_SETTING = Setting.longSetting("node.id.seed", 0L, Long.MIN_VALUE, Property.NodeScope); @@ -677,11 +681,11 @@ public Path[] nodeDataPaths() { * returns the unique uuid describing this node. The uuid is persisted in the data folder of this node * and is reused across restarts. **/ - public String nodeId() { + public String getNodeId() { // we currently only return the ID and hide the underlying nodeMetaData implementation in order to avoid // confusion with other "metadata" like node settings found in elasticsearch.yml. In the future // we can encapsulate both (and more) in one NodeMetaData (or NodeSettings) object ala IndexSettings - return nodeMetaData.nodeId(); + return nodeMetaData.getNodeId(); } /** diff --git a/core/src/main/java/org/elasticsearch/env/NodeMetaData.java b/core/src/main/java/org/elasticsearch/env/NodeMetaData.java index d07644b387945..386f818500db6 100644 --- a/core/src/main/java/org/elasticsearch/env/NodeMetaData.java +++ b/core/src/main/java/org/elasticsearch/env/NodeMetaData.java @@ -33,6 +33,8 @@ import java.util.Objects; /** + * Metadata associated with this node. Currently only contains the unique uuid describing this node. + * The metadata is persisted in the data folder of this node and is reused across restarts. */ public final class NodeMetaData { @@ -40,7 +42,7 @@ public final class NodeMetaData { private final String nodeId; - public NodeMetaData(final String nodeId) { + NodeMetaData(final String nodeId) { this.nodeId = Objects.requireNonNull(nodeId); } @@ -75,7 +77,7 @@ public String toString() { PARSER.declareString(Builder::setNodeId, new ParseField(NODE_ID_KEY)); } - public String nodeId() { + public String getNodeId() { return nodeId; } diff --git a/core/src/main/java/org/elasticsearch/gateway/MetaDataStateFormat.java b/core/src/main/java/org/elasticsearch/gateway/MetaDataStateFormat.java index 939b65129e3cd..553334837c70a 100644 --- a/core/src/main/java/org/elasticsearch/gateway/MetaDataStateFormat.java +++ b/core/src/main/java/org/elasticsearch/gateway/MetaDataStateFormat.java @@ -64,8 +64,9 @@ public abstract class MetaDataStateFormat { public static final String STATE_DIR_NAME = "_state"; public static final String STATE_FILE_EXTENSION = ".st"; private static final String STATE_FILE_CODEC = "state"; - private static final int MIN_COMPATIBLE_STATE_FILE_VERSION = 0; + private static final int STATE_FILE_VERSION_ES_2X_AND_BELOW = 0; private static final int STATE_FILE_VERSION = 1; + private static final int MIN_COMPATIBLE_STATE_FILE_VERSION = STATE_FILE_VERSION_ES_2X_AND_BELOW; private static final int BUFFER_SIZE = 4096; private final XContentType format; private final String prefix; @@ -184,9 +185,9 @@ public final T read(Path file) throws IOException { final int fileVersion = CodecUtil.checkHeader(indexInput, STATE_FILE_CODEC, MIN_COMPATIBLE_STATE_FILE_VERSION, STATE_FILE_VERSION); final XContentType xContentType = XContentType.values()[indexInput.readInt()]; - if (fileVersion == 0) { - // format version 0, write a version that always came from the content state file - // and was never used. + if (fileVersion == STATE_FILE_VERSION_ES_2X_AND_BELOW) { + // ES 1.x/2.x wrote a version that always came from the content state file + // remove this once we stop supporting reading 2.x files indexInput.readLong(); } long filePointer = indexInput.getFilePointer(); diff --git a/core/src/main/java/org/elasticsearch/node/Node.java b/core/src/main/java/org/elasticsearch/node/Node.java index 77ea02f811806..57397b78038b7 100644 --- a/core/src/main/java/org/elasticsearch/node/Node.java +++ b/core/src/main/java/org/elasticsearch/node/Node.java @@ -332,7 +332,7 @@ public Node start() { validateNodeBeforeAcceptingRequests(settings, transportService.boundAddress()); - String nodeId = injector.getInstance(NodeEnvironment.class).nodeId(); + String nodeId = injector.getInstance(NodeEnvironment.class).getNodeId(); DiscoveryNode localNode = injector.getInstance(DiscoveryNodeService.class).buildLocalNode( transportService.boundAddress().publishAddress(), nodeId); diff --git a/core/src/main/java/org/elasticsearch/tribe/TribeService.java b/core/src/main/java/org/elasticsearch/tribe/TribeService.java index 30a3c02fdee18..568fb7d002e72 100644 --- a/core/src/main/java/org/elasticsearch/tribe/TribeService.java +++ b/core/src/main/java/org/elasticsearch/tribe/TribeService.java @@ -188,7 +188,7 @@ public TribeService(Settings settings, ClusterService clusterService, NodeEnviro nodesSettings.remove("blocks"); // remove prefix settings that don't indicate a client nodesSettings.remove("on_conflict"); // remove prefix settings that don't indicate a client for (Map.Entry entry : nodesSettings.entrySet()) { - Settings clientSettings = buildClientSettings(entry.getKey(), nodeEnvironment.nodeId(), settings, entry.getValue()); + Settings clientSettings = buildClientSettings(entry.getKey(), nodeEnvironment.getNodeId(), settings, entry.getValue()); nodes.add(new TribeClientNode(clientSettings)); } diff --git a/core/src/test/java/org/elasticsearch/cluster/routing/allocation/BalanceConfigurationTests.java b/core/src/test/java/org/elasticsearch/cluster/routing/allocation/BalanceConfigurationTests.java index 37c5c4d4f4588..7e1f909bca65f 100644 --- a/core/src/test/java/org/elasticsearch/cluster/routing/allocation/BalanceConfigurationTests.java +++ b/core/src/test/java/org/elasticsearch/cluster/routing/allocation/BalanceConfigurationTests.java @@ -236,7 +236,7 @@ private void assertReplicaBalance(ESLogger logger, RoutingNodes nodes, int numbe final int maxAvgNumberOfShards = Math.round(Math.round(Math.ceil(avgNumShards + treshold))); for (RoutingNode node : nodes) { -// logger.info(node.nodeId() + ": " + node.shardsWithState(INITIALIZING, STARTED).size() + " shards ("+minAvgNumberOfShards+" to "+maxAvgNumberOfShards+")"); +// logger.info(node.getNodeId() + ": " + node.shardsWithState(INITIALIZING, STARTED).size() + " shards ("+minAvgNumberOfShards+" to "+maxAvgNumberOfShards+")"); assertThat(node.shardsWithState(STARTED).size(), Matchers.greaterThanOrEqualTo(minAvgNumberOfShards)); assertThat(node.shardsWithState(STARTED).size(), Matchers.lessThanOrEqualTo(maxAvgNumberOfShards)); } @@ -251,7 +251,7 @@ private void assertIndexBalance(ESLogger logger, RoutingNodes nodes, int numberO for (ObjectCursor index : nodes.getRoutingTable().indicesRouting().keys()) { for (RoutingNode node : nodes) { -// logger.info(node.nodeId() +":"+index+ ": " + node.shardsWithState(index, INITIALIZING, STARTED).size() + " shards ("+minAvgNumberOfShards+" to "+maxAvgNumberOfShards+")"); +// logger.info(node.getNodeId() +":"+index+ ": " + node.shardsWithState(index, INITIALIZING, STARTED).size() + " shards ("+minAvgNumberOfShards+" to "+maxAvgNumberOfShards+")"); assertThat(node.shardsWithState(index.value, STARTED).size(), Matchers.greaterThanOrEqualTo(minAvgNumberOfShards)); assertThat(node.shardsWithState(index.value, STARTED).size(), Matchers.lessThanOrEqualTo(maxAvgNumberOfShards)); } @@ -271,7 +271,7 @@ private void assertPrimaryBalance(ESLogger logger, RoutingNodes nodes, int numbe for (ShardRouting shard : node.shardsWithState(index.value, STARTED)) { primaries += shard.primary() ? 1 : 0; } -// logger.info(node.nodeId() + ": " + primaries + " primaries ("+minAvgNumberOfShards+" to "+maxAvgNumberOfShards+")"); +// logger.info(node.getNodeId() + ": " + primaries + " primaries ("+minAvgNumberOfShards+" to "+maxAvgNumberOfShards+")"); assertThat(primaries, Matchers.greaterThanOrEqualTo(minAvgNumberOfShards)); assertThat(primaries, Matchers.lessThanOrEqualTo(maxAvgNumberOfShards)); } diff --git a/core/src/test/java/org/elasticsearch/env/NodeEnvironmentTests.java b/core/src/test/java/org/elasticsearch/env/NodeEnvironmentTests.java index fec194a28064e..5be6d813be5ed 100644 --- a/core/src/test/java/org/elasticsearch/env/NodeEnvironmentTests.java +++ b/core/src/test/java/org/elasticsearch/env/NodeEnvironmentTests.java @@ -412,17 +412,17 @@ public void testPersistentNodeId() throws IOException { .put("node.master", false) .put("node.data", false) .build()); - String nodeID = env.nodeId(); + String nodeID = env.getNodeId(); env.close(); env = newNodeEnvironment(paths, Settings.EMPTY); - assertThat("previous node didn't have local storage enabled, id should change", env.nodeId(), not(equalTo(nodeID))); - nodeID = env.nodeId(); + assertThat("previous node didn't have local storage enabled, id should change", env.getNodeId(), not(equalTo(nodeID))); + nodeID = env.getNodeId(); env.close(); env = newNodeEnvironment(paths, Settings.EMPTY); - assertThat(env.nodeId(), equalTo(nodeID)); + assertThat(env.getNodeId(), equalTo(nodeID)); env.close(); env = newNodeEnvironment(Settings.EMPTY); - assertThat(env.nodeId(), not(equalTo(nodeID))); + assertThat(env.getNodeId(), not(equalTo(nodeID))); env.close(); } From 4f382b9ec1b70c47c5951524b8b9599db3accb37 Mon Sep 17 00:00:00 2001 From: Yannick Welsch Date: Tue, 3 May 2016 17:04:55 +0200 Subject: [PATCH 29/34] Make node.local_storage filtered setting --- core/src/main/java/org/elasticsearch/node/Node.java | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/core/src/main/java/org/elasticsearch/node/Node.java b/core/src/main/java/org/elasticsearch/node/Node.java index 57397b78038b7..03d77b5b840a5 100644 --- a/core/src/main/java/org/elasticsearch/node/Node.java +++ b/core/src/main/java/org/elasticsearch/node/Node.java @@ -148,7 +148,8 @@ public class Node implements Closeable { * and {@link #NODE_MASTER_SETTING} must also be false. * */ - public static final Setting NODE_LOCAL_STORAGE_SETTING = Setting.boolSetting("node.local_storage", true, Property.NodeScope); + public static final Setting NODE_LOCAL_STORAGE_SETTING = Setting.boolSetting( + "node.local_storage", true, Property.NodeScope, Property.Filtered); public static final Setting NODE_NAME_SETTING = Setting.simpleString("node.name", Property.NodeScope); public static final Setting NODE_ATTRIBUTES = Setting.groupSetting("node.attr.", Property.NodeScope); From 13c993c9e69f38c47eb86f3f5017b1f0e2369ef5 Mon Sep 17 00:00:00 2001 From: Yannick Welsch Date: Tue, 10 May 2016 13:04:59 +0200 Subject: [PATCH 30/34] Use ephemeral id for discovery and persistent node id for all else --- .../TransportClientNodesService.java | 2 +- .../cluster/node/DiscoveryNode.java | 99 ++++++++--------- .../cluster/node/DiscoveryNodeService.java | 19 ++-- .../cluster/node/DiscoveryNodes.java | 50 ++++++--- .../cluster/service/ClusterService.java | 5 +- .../common/settings/ClusterSettings.java | 2 +- .../discovery/local/LocalDiscovery.java | 2 +- .../discovery/zen/NodeJoinController.java | 40 +++---- .../discovery/zen/ZenDiscovery.java | 21 +++- .../zen/fd/MasterFaultDetection.java | 30 ++++-- .../discovery/zen/fd/NodesFaultDetection.java | 19 ++-- .../zen/ping/unicast/UnicastZenPing.java | 4 +- .../publish/PublishClusterStateAction.java | 4 +- .../transport/netty/NettyTransport.java | 6 +- .../org/elasticsearch/tribe/TribeService.java | 6 +- .../TransportClientNodesServiceTests.java | 2 +- .../cluster/ClusterChangedEventTests.java | 8 +- .../node/DiscoveryNodeFiltersTests.java | 16 +-- .../cluster/node/DiscoveryNodeTests.java | 29 +++-- .../cluster/node/DiscoveryNodesTests.java | 100 ++++++++++++++++-- .../allocation/BalanceConfigurationTests.java | 6 +- .../zen/NodeJoinControllerTests.java | 20 ++-- .../gateway/AsyncShardFetchTests.java | 4 +- .../test/InternalTestCluster.java | 6 +- .../test/transport/CapturingTransport.java | 4 +- 25 files changed, 322 insertions(+), 182 deletions(-) diff --git a/core/src/main/java/org/elasticsearch/client/transport/TransportClientNodesService.java b/core/src/main/java/org/elasticsearch/client/transport/TransportClientNodesService.java index ed57b4d86a3a0..4874b4bbb99a0 100644 --- a/core/src/main/java/org/elasticsearch/client/transport/TransportClientNodesService.java +++ b/core/src/main/java/org/elasticsearch/client/transport/TransportClientNodesService.java @@ -383,7 +383,7 @@ public LivenessResponse newInstance() { // use discovered information but do keep the original transport address, // so people can control which address is exactly used. DiscoveryNode nodeWithInfo = livenessResponse.getDiscoveryNode(); - newNodes.add(new DiscoveryNode(nodeWithInfo.getName(), nodeWithInfo.getId(), nodeWithInfo.getNodeId(), + newNodes.add(new DiscoveryNode(nodeWithInfo.getName(), nodeWithInfo.getId(), nodeWithInfo.getEphemeralId(), nodeWithInfo.getHostName(), nodeWithInfo.getHostAddress(), listedNode.getAddress(), nodeWithInfo.getAttributes(), nodeWithInfo.getRoles(), nodeWithInfo.getVersion())); } else { diff --git a/core/src/main/java/org/elasticsearch/cluster/node/DiscoveryNode.java b/core/src/main/java/org/elasticsearch/cluster/node/DiscoveryNode.java index 967aa11a788ea..d4bc1fcb1cfca 100644 --- a/core/src/main/java/org/elasticsearch/cluster/node/DiscoveryNode.java +++ b/core/src/main/java/org/elasticsearch/cluster/node/DiscoveryNode.java @@ -87,8 +87,8 @@ public static boolean isIngestNode(Settings settings) { } private final String nodeName; - private final String processId; private final String nodeId; + private final String ephemeralId; private final String hostName; private final String hostAddress; private final TransportAddress address; @@ -105,15 +105,15 @@ public static boolean isIngestNode(Settings settings) { * and updated. *

* - * @param processAndNodeId the nodes unique process and node id + * @param id the nodes unique (ephemeral and persistent) node id * @param address the nodes transport address * @param attributes node attributes * @param roles node roles * @param version the version of the node */ - public DiscoveryNode(String processAndNodeId, TransportAddress address, Map attributes, Set roles, + public DiscoveryNode(String id, TransportAddress address, Map attributes, Set roles, Version version) { - this(processAndNodeId, processAndNodeId, address, attributes, roles, version); + this(id, id, address, attributes, roles, version); } /** @@ -125,16 +125,16 @@ public DiscoveryNode(String processAndNodeId, TransportAddress address, Map * - * @param processId the nodes unique process id - * @param nodeId the nodes unique persistent id + * @param nodeId the nodes unique persistent id + * @param ephemeralId the nodes unique ephemeral id * @param address the nodes transport address * @param attributes node attributes * @param roles node roles * @param version the version of the node */ - public DiscoveryNode(String processId, String nodeId, TransportAddress address, Map attributes, + public DiscoveryNode(String nodeId, String ephemeralId, TransportAddress address, Map attributes, Set roles, Version version) { - this("", processId, nodeId, address.getHost(), address.getAddress(), address, attributes, roles, version); + this("", nodeId, ephemeralId, address.getHost(), address.getAddress(), address, attributes, roles, version); } /** @@ -147,16 +147,16 @@ public DiscoveryNode(String processId, String nodeId, TransportAddress address, *

* * @param nodeName the nodes name - * @param processId the nodes unique process id - * @param nodeId the nodes unique persistent id + * @param nodeId the nodes unique persistent id + * @param ephemeralId the nodes unique ephemeral id * @param address the nodes transport address * @param attributes node attributes * @param roles node roles * @param version the version of the node */ - public DiscoveryNode(String nodeName, String processId, String nodeId, TransportAddress address, + public DiscoveryNode(String nodeName, String nodeId, String ephemeralId, TransportAddress address, Map attributes, Set roles, Version version) { - this(nodeName, processId, nodeId, address.getHost(), address.getAddress(), address, attributes, roles, version); + this(nodeName, nodeId, ephemeralId, address.getHost(), address.getAddress(), address, attributes, roles, version); } /** @@ -169,23 +169,23 @@ public DiscoveryNode(String nodeName, String processId, String nodeId, Transport *

* * @param nodeName the nodes name - * @param processId the nodes unique process id - * @param nodeId the nodes unique persistent id + * @param nodeId the nodes unique persistent id + * @param ephemeralId the nodes unique ephemeral id * @param hostAddress the nodes host address * @param address the nodes transport address * @param attributes node attributes * @param roles node roles * @param version the version of the node */ - public DiscoveryNode(String nodeName, String processId, String nodeId, String hostName, String hostAddress, + public DiscoveryNode(String nodeName, String nodeId, String ephemeralId, String hostName, String hostAddress, TransportAddress address, Map attributes, Set roles, Version version) { if (nodeName != null) { this.nodeName = nodeName.intern(); } else { this.nodeName = ""; } - this.processId = processId.intern(); this.nodeId = nodeId.intern(); + this.ephemeralId = ephemeralId.intern(); this.hostName = hostName.intern(); this.hostAddress = hostAddress.intern(); this.address = address; @@ -215,8 +215,8 @@ public DiscoveryNode(String nodeName, String processId, String nodeId, String ho */ public DiscoveryNode(StreamInput in) throws IOException { this.nodeName = in.readString().intern(); - this.processId = in.readString().intern(); this.nodeId = in.readString().intern(); + this.ephemeralId = in.readString().intern(); this.hostName = in.readString().intern(); this.hostAddress = in.readString().intern(); this.address = TransportAddressSerializers.addressFromStream(in); @@ -240,8 +240,8 @@ public DiscoveryNode(StreamInput in) throws IOException { @Override public void writeTo(StreamOutput out) throws IOException { out.writeString(nodeName); - out.writeString(processId); out.writeString(nodeId); + out.writeString(ephemeralId); out.writeString(hostName); out.writeString(hostAddress); addressToStream(out, address); @@ -265,17 +265,17 @@ public TransportAddress getAddress() { } /** - * The unique process id of the node. + * The unique id of the node. */ public String getId() { - return processId; + return nodeId; } /** - * The unique process id of the node. + * The unique ephemeral id of the node. */ - public String getNodeId() { - return nodeId; + public String getEphemeralId() { + return ephemeralId; } /** @@ -333,58 +333,51 @@ public String getHostAddress() { return this.hostAddress; } - /** - * Checks whether this node has the same id as another node *and* has - * the same attributes, network address and so fourth. This is in contrast to {@link #equals(Object)} - * which only compares the process ids. - */ - public boolean equalsIncludingMetaData(DiscoveryNode other) { - if (this.equals(other) == false) { + @Override + public boolean equals(Object o) { + if (this == o) { + return true; + } + if (o == null || getClass() != o.getClass()) { return false; } - if (!nodeId.equals(other.nodeId)) { + DiscoveryNode that = (DiscoveryNode) o; + + if (!nodeId.equals(that.nodeId)) { return false; } - if (!nodeName.equals(other.nodeName)) { + if (!ephemeralId.equals(that.ephemeralId)) { return false; } - if (!hostName.equals(other.hostName)) { + if (!nodeName.equals(that.nodeName)) { return false; } - if (!hostAddress.equals(other.hostAddress)) { + if (!hostName.equals(that.hostName)) { return false; } - if (!address.equals(other.address)) { + if (!hostAddress.equals(that.hostAddress)) { return false; } - if (!attributes.equals(other.attributes)) { + if (!address.equals(that.address)) { return false; } - if (!version.equals(other.version)) { + if (!attributes.equals(that.attributes)) { return false; } - return roles.equals(other.roles); - } - - - /** - * Checks for equality based on the value {@link #getId()} **alone**. This is done so that this class can be used - * efficiently as a key in a map - */ - @Override - public boolean equals(Object obj) { - if (!(obj instanceof DiscoveryNode)) { + if (!version.equals(that.version)) { return false; } + return roles.equals(that.roles); - DiscoveryNode other = (DiscoveryNode) obj; - return this.processId.equals(other.processId); } @Override public int hashCode() { - return processId.hashCode(); + // we only need to hash the id because it's highly unlikely that two nodes + // in our system will have the same id but be different + // This is done so that this class can be used efficiently as a key in a map + return nodeId.hashCode(); } @Override @@ -393,7 +386,7 @@ public String toString() { if (nodeName.length() > 0) { sb.append('{').append(nodeName).append('}'); } - sb.append('{').append(processId).append('}'); + sb.append('{').append(ephemeralId).append('}'); sb.append('{').append(nodeId).append('}'); sb.append('{').append(hostName).append('}'); sb.append('{').append(address).append('}'); @@ -407,7 +400,7 @@ public String toString() { public XContentBuilder toXContent(XContentBuilder builder, Params params) throws IOException { builder.startObject(getId()); builder.field("name", getName()); - builder.field("node_id", getNodeId()); + builder.field("ephemeral_id", getEphemeralId()); builder.field("transport_address", getAddress().toString()); builder.startObject("attributes"); diff --git a/core/src/main/java/org/elasticsearch/cluster/node/DiscoveryNodeService.java b/core/src/main/java/org/elasticsearch/cluster/node/DiscoveryNodeService.java index 80ef993e9a99b..6937a96d7f2d9 100644 --- a/core/src/main/java/org/elasticsearch/cluster/node/DiscoveryNodeService.java +++ b/core/src/main/java/org/elasticsearch/cluster/node/DiscoveryNodeService.java @@ -36,28 +36,27 @@ import java.util.Random; import java.util.Set; import java.util.concurrent.CopyOnWriteArrayList; -import java.util.function.Supplier; /** */ public class DiscoveryNodeService extends AbstractComponent { - public static final Setting PROCESS_ID_SEED_SETTING = - Setting.longSetting("node.process_id.seed", 0L, Long.MIN_VALUE, Setting.Property.NodeScope); + public static final Setting EPHEMERAL_ID_SEED_SETTING = + Setting.longSetting("node.ephemeral_id.seed", 0L, Long.MIN_VALUE, Setting.Property.NodeScope); private final List customAttributesProviders = new CopyOnWriteArrayList<>(); private final Version version; - private final String processId; + private final String ephemeralId; @Inject public DiscoveryNodeService(Settings settings, Version version) { super(settings); this.version = version; - this.processId = generateProcessId(settings); + this.ephemeralId = generateEphemeralId(settings); } - public String getProcessId() { - return processId; + public String getEphemeralId() { + return ephemeralId; } public DiscoveryNodeService addCustomAttributeProvider(CustomAttributesProvider customAttributesProvider) { @@ -65,8 +64,8 @@ public DiscoveryNodeService addCustomAttributeProvider(CustomAttributesProvider return this; } - public static String generateProcessId(Settings settings) { - Random random = Randomness.get(settings, PROCESS_ID_SEED_SETTING); + public static String generateEphemeralId(Settings settings) { + Random random = Randomness.get(settings, EPHEMERAL_ID_SEED_SETTING); return UUIDs.randomBase64UUID(random); } @@ -97,7 +96,7 @@ public DiscoveryNode buildLocalNode(TransportAddress publishAddress, String node logger.warn("failed to build custom attributes from provider [{}]", e, provider); } } - return new DiscoveryNode(Node.NODE_NAME_SETTING.get(settings), processId, nodeId, publishAddress, + return new DiscoveryNode(Node.NODE_NAME_SETTING.get(settings), ephemeralId, nodeId, publishAddress, attributes, roles, version); } diff --git a/core/src/main/java/org/elasticsearch/cluster/node/DiscoveryNodes.java b/core/src/main/java/org/elasticsearch/cluster/node/DiscoveryNodes.java index a66fd97eaee8d..a2615969c725f 100644 --- a/core/src/main/java/org/elasticsearch/cluster/node/DiscoveryNodes.java +++ b/core/src/main/java/org/elasticsearch/cluster/node/DiscoveryNodes.java @@ -162,6 +162,31 @@ public boolean nodeExists(String nodeId) { return nodes.containsKey(nodeId); } + /** + * Determine if a given node exists with the given ephemeral id + * + * @param nodeId id of the node which existence should be verified + * @return true if the node exists. Otherwise false + */ + public boolean nodeExists(String nodeId, String ephemeralId) { + DiscoveryNode node = nodes.get(nodeId); + if (node != null) { + return node.getEphemeralId().equals(ephemeralId); + } + return false; + } + + /** + * Determine if a given node exists that exactly matches the provided DiscoveryNode + * + * @param node node which existence should be verified + * @return true if the node exists. Otherwise false + */ + public boolean nodeExists(DiscoveryNode node) { + DiscoveryNode existing = nodes.get(node.getId()); + return existing != null && existing.equals(node); + } + /** * Get the id of the master node * @@ -280,16 +305,16 @@ public String[] resolveNodesIds(String... nodesIds) { } else if (nodeExists(nodeId)) { resolvedNodesIds.add(nodeId); } else { - boolean matchedPersistentNodeId = false; + boolean matchedEphemeralId = false; for (DiscoveryNode node : this) { - if (node.getNodeId().equals(nodeId)) { - matchedPersistentNodeId = true; + if (node.getEphemeralId().equals(nodeId)) { + matchedEphemeralId = true; resolvedNodesIds.add(node.getId()); break; // persistent node id is unique } } - if (matchedPersistentNodeId == false) { - // neither process nor persistent node id, try and search by name + if (matchedEphemeralId == false) { + // neither ephemeral nor persistent node id, try and search by name for (DiscoveryNode node : this) { if (Regex.simpleMatch(nodeId, node.getName())) { resolvedNodesIds.add(node.getId()); @@ -370,19 +395,20 @@ public Delta delta(DiscoveryNodes other) { List removed = new ArrayList<>(); List added = new ArrayList<>(); for (DiscoveryNode node : other) { - if (!this.nodeExists(node.getId())) { + if (!this.nodeExists(node)) { removed.add(node); } } for (DiscoveryNode node : this) { - if (!other.nodeExists(node.getId())) { + if (!other.nodeExists(node)) { added.add(node); } } DiscoveryNode previousMasterNode = null; DiscoveryNode newMasterNode = null; if (masterNodeId != null) { - if (other.masterNodeId == null || !other.masterNodeId.equals(masterNodeId)) { + DiscoveryNode otherMasterNode = other.getMasterNode(); + if (otherMasterNode == null || otherMasterNode.equals(this.getMasterNode()) == false) { previousMasterNode = other.getMasterNode(); newMasterNode = getMasterNode(); } @@ -584,9 +610,9 @@ public Builder(DiscoveryNodes nodes) { } public Builder put(DiscoveryNode node) { - assert nodes.get(node.getId()) == null : "node with process id " + node.getId() + " already exists"; + assert nodes.get(node.getId()) == null : "node with id " + node.getId() + " already exists"; assert nodeWithSameAddress(node) == null : "node with address " + node.getAddress() + " already exists"; - assert nodeWithSamePersistentId(node) == null : "node with persistent id " + node.getNodeId() + " already exists"; + assert nodeWithSameEphemeralId(node) == null : "node with ephemeral id " + node.getEphemeralId() + " already exists"; nodes.put(node.getId(), node); return this; } @@ -616,10 +642,10 @@ public DiscoveryNode nodeWithSameAddress(DiscoveryNode node) { return null; } - public DiscoveryNode nodeWithSamePersistentId(DiscoveryNode node) { + public DiscoveryNode nodeWithSameEphemeralId(DiscoveryNode node) { for (ObjectCursor cursor : nodes.values()) { final DiscoveryNode existingNode = cursor.value; - if (node.getNodeId().equals(existingNode.getNodeId())) { + if (node.getEphemeralId().equals(existingNode.getEphemeralId())) { return existingNode; } } diff --git a/core/src/main/java/org/elasticsearch/cluster/service/ClusterService.java b/core/src/main/java/org/elasticsearch/cluster/service/ClusterService.java index 15c2b5c393908..0011969b459bd 100644 --- a/core/src/main/java/org/elasticsearch/cluster/service/ClusterService.java +++ b/core/src/main/java/org/elasticsearch/cluster/service/ClusterService.java @@ -980,7 +980,10 @@ public void run() { public void onNodeAck(DiscoveryNode node, @Nullable Throwable t) { if (!ackedTaskListener.mustAck(node)) { //we always wait for the master ack anyway - if (!node.equals(nodes.getMasterNode())) { + DiscoveryNode masterNode = nodes.getMasterNode(); + if (masterNode == null || + node.getId().equals(masterNode.getId()) == false || + node.getEphemeralId().equals(masterNode.getEphemeralId()) == false) { return; } } diff --git a/core/src/main/java/org/elasticsearch/common/settings/ClusterSettings.java b/core/src/main/java/org/elasticsearch/common/settings/ClusterSettings.java index 57fdbdb312653..03095c15944bb 100644 --- a/core/src/main/java/org/elasticsearch/common/settings/ClusterSettings.java +++ b/core/src/main/java/org/elasticsearch/common/settings/ClusterSettings.java @@ -336,7 +336,7 @@ public void apply(Settings value, Settings current, Settings previous) { Environment.PATH_SHARED_DATA_SETTING, Environment.PIDFILE_SETTING, NodeEnvironment.NODE_ID_SEED_SETTING, - DiscoveryNodeService.PROCESS_ID_SEED_SETTING, + DiscoveryNodeService.EPHEMERAL_ID_SEED_SETTING, DiscoverySettings.INITIAL_STATE_TIMEOUT_SETTING, DiscoveryModule.DISCOVERY_TYPE_SETTING, DiscoveryModule.ZEN_MASTER_SERVICE_TYPE_SETTING, diff --git a/core/src/main/java/org/elasticsearch/discovery/local/LocalDiscovery.java b/core/src/main/java/org/elasticsearch/discovery/local/LocalDiscovery.java index 053e4cae12e40..d170c18783c50 100644 --- a/core/src/main/java/org/elasticsearch/discovery/local/LocalDiscovery.java +++ b/core/src/main/java/org/elasticsearch/discovery/local/LocalDiscovery.java @@ -309,7 +309,7 @@ private void publish(LocalDiscovery[] members, ClusterChangedEvent clusterChange synchronized (this) { // we do the marshaling intentionally, to check it works well... // check if we published cluster state at least once and node was in the cluster when we published cluster state the last time - if (discovery.lastProcessedClusterState != null && clusterChangedEvent.previousState().nodes().nodeExists(discovery.localNode().getId())) { + if (discovery.lastProcessedClusterState != null && clusterChangedEvent.previousState().nodes().nodeExists(discovery.localNode().getId(), discovery.localNode().getEphemeralId())) { // both conditions are true - which means we can try sending cluster state as diffs if (clusterStateDiffBytes == null) { Diff diff = clusterState.diff(clusterChangedEvent.previousState()); diff --git a/core/src/main/java/org/elasticsearch/discovery/zen/NodeJoinController.java b/core/src/main/java/org/elasticsearch/discovery/zen/NodeJoinController.java index c022becbbb352..a9190574d874e 100644 --- a/core/src/main/java/org/elasticsearch/discovery/zen/NodeJoinController.java +++ b/core/src/main/java/org/elasticsearch/discovery/zen/NodeJoinController.java @@ -385,36 +385,38 @@ public ClusterState execute(ClusterState currentState) { Map.Entry> entry = iterator.next(); final DiscoveryNode node = entry.getKey(); iterator.remove(); - DiscoveryNode existingNodeWithSameProcessId = currentState.nodes().get(node.getId()); - if (existingNodeWithSameProcessId != null) { + DiscoveryNode existingNodeWithSameId = currentState.nodes().get(node.getId()); + if (existingNodeWithSameId != null && existingNodeWithSameId.getEphemeralId().equals(node.getEphemeralId())) { + assert existingNodeWithSameId.equals(node) : "node [" + existingNodeWithSameId + "] with " + + "same ephemeral id but different metadata joined (existing node [" + existingNodeWithSameId + "])"; logger.debug("received a join request for an existing node [{}]", node); - assert existingNodeWithSameProcessId.equalsIncludingMetaData(node); joinCallbacksToRespondTo.addAll(entry.getValue()); } else { DiscoveryNode nodeWithSameAddress = nodesBuilder.nodeWithSameAddress(node); if (nodeWithSameAddress != null) { - logger.warn("received join request from node [{}], but found existing node {} with same address, removing " + + logger.warn("received join request from node [{}], but found existing node [{}] with same address, removing " + "existing node", node, nodeWithSameAddress); nodesBuilder.remove(nodeWithSameAddress.getId()); nodesBuilder.put(node); nodeAdded = true; joinCallbacksToRespondTo.addAll(entry.getValue()); - } else { - DiscoveryNode nodeWithSamePersistentId = nodesBuilder.nodeWithSamePersistentId(node); - if (nodeWithSamePersistentId != null) { - logger.warn("received join request from node [{}], but found existing node {} with same persistent node " + - "id, failing join request ", node); - Throwable failure = new IllegalStateException("can't add node " + node + ", found existing node " + - nodeWithSamePersistentId + " with same persistent node id"); - for (MembershipAction.JoinCallback callback: entry.getValue()) { - joinCallbacksToFail.add(new Tuple<>(callback, failure)); - } - } else { - logger.trace("received join request for a new node [{}]", node); - nodesBuilder.put(node); - nodeAdded = true; - joinCallbacksToRespondTo.addAll(entry.getValue()); + } else if (existingNodeWithSameId != null) { + logger.warn("received join request from node [{}], but found existing node [{}] with same node id but " + + "different ephemeral id, failing join request ", node, existingNodeWithSameId); + // probably means that the data folder containing the persisted node id was copied to another node + Throwable failure = new IllegalStateException("can't add node " + node + ", found existing node " + + existingNodeWithSameId + " with same node id but different ephemeral id"); + for (MembershipAction.JoinCallback callback : entry.getValue()) { + joinCallbacksToFail.add(new Tuple<>(callback, failure)); } + } else { + assert existingNodeWithSameId == null; + assert nodesBuilder.nodeWithSameEphemeralId(node) == null : "received join request from node [" + node + "] " + + "but found existing node with same ephemeral id"; + logger.trace("received join request for a new node [{}]", node); + nodesBuilder.put(node); + nodeAdded = true; + joinCallbacksToRespondTo.addAll(entry.getValue()); } assert entry.getValue().stream().allMatch(cb -> diff --git a/core/src/main/java/org/elasticsearch/discovery/zen/ZenDiscovery.java b/core/src/main/java/org/elasticsearch/discovery/zen/ZenDiscovery.java index 607f8d34db182..10ae6d196a10c 100644 --- a/core/src/main/java/org/elasticsearch/discovery/zen/ZenDiscovery.java +++ b/core/src/main/java/org/elasticsearch/discovery/zen/ZenDiscovery.java @@ -508,7 +508,10 @@ private void handleLeaveRequest(final DiscoveryNode node) { clusterService.submitStateUpdateTask("zen-disco-node_left(" + node + ")", new ClusterStateUpdateTask(Priority.IMMEDIATE) { @Override public ClusterState execute(ClusterState currentState) { - DiscoveryNodes.Builder builder = DiscoveryNodes.builder(currentState.nodes()).remove(node.getId()); + DiscoveryNodes.Builder builder = DiscoveryNodes.builder(currentState.nodes()); + if (currentState.nodes().nodeExists(node.getId(), node.getEphemeralId())) { + builder.remove(node.getId()); + } currentState = ClusterState.builder(currentState).nodes(builder).build(); // check if we have enough master nodes, if not, we need to move into joining the cluster again if (!electMaster.hasEnoughMasterNodes(currentState.nodes())) { @@ -531,8 +534,13 @@ public void onFailure(String source, Throwable t) { logger.error("unexpected failure during [{}]", t, source); } }); - } else if (node.equals(nodes().getMasterNode())) { - handleMasterGone(node, null, "shut_down"); + } else { + DiscoveryNode masterNode = nodes().getMasterNode(); + if (masterNode == null || + masterNode.getId().equals(node.getId()) == false || + masterNode.getEphemeralId().equals(node.getEphemeralId()) == false) { + handleMasterGone(node, null, "shut_down"); + } } } @@ -548,7 +556,7 @@ private void handleNodeFailure(final DiscoveryNode node, String reason) { clusterService.submitStateUpdateTask("zen-disco-node_failed(" + node + "), reason " + reason, new ClusterStateUpdateTask(Priority.IMMEDIATE) { @Override public ClusterState execute(ClusterState currentState) { - if (currentState.nodes().get(node.getId()) == null) { + if (currentState.nodes().nodeExists(node.getId(), node.getEphemeralId()) == false) { logger.debug("node [{}] already removed from cluster state. ignoring.", node); return currentState; } @@ -641,7 +649,10 @@ public boolean runOnlyOnMaster() { @Override public ClusterState execute(ClusterState currentState) { - if (!masterNode.getId().equals(currentState.nodes().getMasterNodeId())) { + DiscoveryNode currentMasterNode = currentState.nodes().getMasterNode(); + if (currentMasterNode == null || + currentMasterNode.getId().equals(masterNode.getId()) == false || + currentMasterNode.getEphemeralId().equals(masterNode.getId()) == false) { // master got switched on us, no need to send anything return currentState; } diff --git a/core/src/main/java/org/elasticsearch/discovery/zen/fd/MasterFaultDetection.java b/core/src/main/java/org/elasticsearch/discovery/zen/fd/MasterFaultDetection.java index 4e7ea5be3af99..bd1528777a481 100644 --- a/core/src/main/java/org/elasticsearch/discovery/zen/fd/MasterFaultDetection.java +++ b/core/src/main/java/org/elasticsearch/discovery/zen/fd/MasterFaultDetection.java @@ -169,7 +169,9 @@ public void close() { @Override protected void handleTransportDisconnect(DiscoveryNode node) { synchronized (masterNodeMutex) { - if (!node.equals(this.masterNode)) { + if (this.masterNode == null || + node.getId().equals(this.masterNode.getId()) == false || + node.getEphemeralId().equals(this.masterNode.getEphemeralId())) { return; } if (connectOnNetworkDisconnect) { @@ -227,7 +229,9 @@ public void run() { threadPool.schedule(pingInterval, ThreadPool.Names.SAME, MasterPinger.this); return; } - final MasterPingRequest request = new MasterPingRequest(clusterService.localNode().getId(), masterToPing.getId(), clusterName); + final DiscoveryNode localNode = clusterService.localNode(); + final MasterPingRequest request = new MasterPingRequest(localNode.getId(), localNode.getEphemeralId(), + masterToPing.getId(), masterToPing.getEphemeralId(), clusterName); final TransportRequestOptions options = TransportRequestOptions.builder().withType(TransportRequestOptions.Type.PING).withTimeout(pingRetryTimeout).build(); transportService.sendRequest(masterToPing, MASTER_PING_ACTION_NAME, request, options, new BaseTransportResponseHandler() { @@ -328,12 +332,13 @@ public void messageReceived(final MasterPingRequest request, final TransportChan final DiscoveryNodes nodes = clusterService.state().nodes(); // check if we are really the same master as the one we seemed to be think we are // this can happen if the master got "kill -9" and then another node started using the same port - if (!request.masterNodeId.equals(nodes.getLocalNodeId())) { + DiscoveryNode localNode = nodes.getLocalNode(); + if (request.masterNodeId.equals(localNode.getId()) == false || + request.masterEphemeralId.equals(localNode.getEphemeralId()) == false) { throw new ThisIsNotTheMasterYouAreLookingForException(); } - // ping from nodes of version < 1.4.0 will have the clustername set to null - if (request.clusterName != null && !request.clusterName.equals(clusterName)) { + if (clusterName.equals(request.clusterName) == false) { logger.trace("master fault detection ping request is targeted for a different [{}] cluster then us [{}]", request.clusterName, clusterName); throw new ThisIsNotTheMasterYouAreLookingForException("master fault detection ping request is targeted for a different [" + request.clusterName + "] cluster then us [" + clusterName + "]"); } @@ -346,7 +351,7 @@ public void messageReceived(final MasterPingRequest request, final TransportChan // all processing is finished. // - if (!nodes.isLocalNodeElectedMaster() || !nodes.nodeExists(request.nodeId)) { + if (!nodes.isLocalNodeElectedMaster() || !nodes.nodeExists(request.nodeId, request.ephemeralId)) { logger.trace("checking ping from [{}] under a cluster state thread", request.nodeId); clusterService.submitStateUpdateTask("master ping (from: [" + request.nodeId + "])", new ClusterStateUpdateTask() { @@ -354,7 +359,7 @@ public void messageReceived(final MasterPingRequest request, final TransportChan public ClusterState execute(ClusterState currentState) throws Exception { // if we are no longer master, fail... DiscoveryNodes nodes = currentState.nodes(); - if (!nodes.nodeExists(request.nodeId)) { + if (!nodes.nodeExists(request.nodeId, request.ephemeralId)) { throw new NodeDoesNotExistOnMasterException(); } return currentState; @@ -397,16 +402,19 @@ public void clusterStateProcessed(String source, ClusterState oldState, ClusterS public static class MasterPingRequest extends TransportRequest { private String nodeId; - + private String ephemeralId; private String masterNodeId; + private String masterEphemeralId; private ClusterName clusterName; public MasterPingRequest() { } - private MasterPingRequest(String nodeId, String masterNodeId, ClusterName clusterName) { + private MasterPingRequest(String nodeId, String ephemeralId, String masterNodeId, String masterEphemeralId, ClusterName clusterName) { this.nodeId = nodeId; + this.ephemeralId = ephemeralId; this.masterNodeId = masterNodeId; + this.masterEphemeralId = masterEphemeralId; this.clusterName = clusterName; } @@ -414,7 +422,9 @@ private MasterPingRequest(String nodeId, String masterNodeId, ClusterName cluste public void readFrom(StreamInput in) throws IOException { super.readFrom(in); nodeId = in.readString(); + ephemeralId = in.readString(); masterNodeId = in.readString(); + masterEphemeralId = in.readString(); clusterName = ClusterName.readClusterName(in); } @@ -422,7 +432,9 @@ public void readFrom(StreamInput in) throws IOException { public void writeTo(StreamOutput out) throws IOException { super.writeTo(out); out.writeString(nodeId); + out.writeString(ephemeralId); out.writeString(masterNodeId); + out.writeString(masterEphemeralId); clusterName.writeTo(out); } } diff --git a/core/src/main/java/org/elasticsearch/discovery/zen/fd/NodesFaultDetection.java b/core/src/main/java/org/elasticsearch/discovery/zen/fd/NodesFaultDetection.java index 7dbe50577cea7..0abe1eb438337 100644 --- a/core/src/main/java/org/elasticsearch/discovery/zen/fd/NodesFaultDetection.java +++ b/core/src/main/java/org/elasticsearch/discovery/zen/fd/NodesFaultDetection.java @@ -200,7 +200,7 @@ public void run() { if (!running()) { return; } - final PingRequest pingRequest = new PingRequest(node.getId(), clusterName, localNode, clusterStateVersion); + final PingRequest pingRequest = new PingRequest(node.getId(), node.getEphemeralId(), clusterName, localNode, clusterStateVersion); final TransportRequestOptions options = TransportRequestOptions.builder().withType(TransportRequestOptions.Type.PING).withTimeout(pingRetryTimeout).build(); transportService.sendRequest(node, PING_ACTION_NAME, pingRequest, options, new BaseTransportResponseHandler() { @Override @@ -255,12 +255,11 @@ class PingRequestHandler implements TransportRequestHandler { public void messageReceived(PingRequest request, TransportChannel channel) throws Exception { // if we are not the node we are supposed to be pinged, send an exception // this can happen when a kill -9 is sent, and another node is started using the same port - if (!localNode.getId().equals(request.nodeId)) { - throw new IllegalStateException("Got pinged as node [" + request.nodeId + "], but I am node [" + localNode.getId() + "]"); + if (localNode.getId().equals(request.nodeId()) == false || localNode.getEphemeralId().equals(request.ephemeralId()) == false) { + throw new IllegalStateException("Got pinged as node [" + request.nodeId() + "/" + request.ephemeralId() + "], but I am node [" + localNode + "]"); } - // PingRequest will have clusterName set to null if it came from a node of version <1.4.0 - if (request.clusterName != null && !request.clusterName.equals(clusterName)) { + if (clusterName.equals(request.clusterName) == false) { // Don't introduce new exception for bwc reasons throw new IllegalStateException("Got pinged with cluster name [" + request.clusterName + "], but I'm part of cluster [" + clusterName + "]"); } @@ -276,6 +275,7 @@ public static class PingRequest extends TransportRequest { // the (assumed) node id we are pinging private String nodeId; + private String ephemeralId; private ClusterName clusterName; @@ -286,8 +286,9 @@ public static class PingRequest extends TransportRequest { public PingRequest() { } - PingRequest(String nodeId, ClusterName clusterName, DiscoveryNode masterNode, long clusterStateVersion) { + PingRequest(String nodeId, String ephemeralId, ClusterName clusterName, DiscoveryNode masterNode, long clusterStateVersion) { this.nodeId = nodeId; + this.ephemeralId = ephemeralId; this.clusterName = clusterName; this.masterNode = masterNode; this.clusterStateVersion = clusterStateVersion; @@ -297,6 +298,10 @@ public String nodeId() { return nodeId; } + public String ephemeralId() { + return ephemeralId; + } + public ClusterName clusterName() { return clusterName; } @@ -313,6 +318,7 @@ public long clusterStateVersion() { public void readFrom(StreamInput in) throws IOException { super.readFrom(in); nodeId = in.readString(); + ephemeralId = in.readString(); clusterName = ClusterName.readClusterName(in); masterNode = new DiscoveryNode(in); clusterStateVersion = in.readLong(); @@ -322,6 +328,7 @@ public void readFrom(StreamInput in) throws IOException { public void writeTo(StreamOutput out) throws IOException { super.writeTo(out); out.writeString(nodeId); + out.writeString(ephemeralId); clusterName.writeTo(out); masterNode.writeTo(out); out.writeLong(clusterStateVersion); diff --git a/core/src/main/java/org/elasticsearch/discovery/zen/ping/unicast/UnicastZenPing.java b/core/src/main/java/org/elasticsearch/discovery/zen/ping/unicast/UnicastZenPing.java index b723bc2b77984..f8a481447a500 100644 --- a/core/src/main/java/org/elasticsearch/discovery/zen/ping/unicast/UnicastZenPing.java +++ b/core/src/main/java/org/elasticsearch/discovery/zen/ping/unicast/UnicastZenPing.java @@ -379,8 +379,8 @@ void sendPings(final TimeValue timeout, @Nullable TimeValue waitTime, final Send // but will be added again during the pinging. We therefore create a new temporary node if (!nodeFoundByAddress) { if (!nodeToSend.getId().startsWith(UNICAST_NODE_PREFIX)) { - DiscoveryNode tempNode = new DiscoveryNode( - UNICAST_NODE_PREFIX + unicastNodeIdGenerator.incrementAndGet() + "_" + nodeToSend.getId() + "#", + String id = UNICAST_NODE_PREFIX + unicastNodeIdGenerator.incrementAndGet() + "_" + nodeToSend.getId() + "#"; + DiscoveryNode tempNode = new DiscoveryNode(id, id, id, nodeToSend.getHostName(), nodeToSend.getHostAddress(), nodeToSend.getAddress(), nodeToSend.getAttributes(), nodeToSend.getRoles(), nodeToSend.getVersion()); diff --git a/core/src/main/java/org/elasticsearch/discovery/zen/publish/PublishClusterStateAction.java b/core/src/main/java/org/elasticsearch/discovery/zen/publish/PublishClusterStateAction.java index c2333ff177d37..1c5709be51b60 100644 --- a/core/src/main/java/org/elasticsearch/discovery/zen/publish/PublishClusterStateAction.java +++ b/core/src/main/java/org/elasticsearch/discovery/zen/publish/PublishClusterStateAction.java @@ -185,7 +185,7 @@ private void innerPublish(final ClusterChangedEvent clusterChangedEvent, final S // try and serialize the cluster state once (or per version), so we don't serialize it // per node when we send it over the wire, compress it while we are at it... // we don't send full version if node didn't exist in the previous version of cluster state - if (sendFullVersion || !previousState.nodes().nodeExists(node.getId())) { + if (sendFullVersion || !previousState.nodes().nodeExists(node.getId(), node.getEphemeralId())) { sendFullClusterState(clusterState, serializedStates, node, publishTimeout, sendingController); } else { sendClusterStateDiff(clusterState, serializedDiffs, serializedStates, node, publishTimeout, sendingController); @@ -216,7 +216,7 @@ private void buildDiffAndSerializeStates(ClusterState clusterState, ClusterState Diff diff = null; for (final DiscoveryNode node : nodesToPublishTo) { try { - if (sendFullVersion || !previousState.nodes().nodeExists(node.getId())) { + if (sendFullVersion || !previousState.nodes().nodeExists(node.getId(), node.getEphemeralId())) { // will send a full reference if (serializedStates.containsKey(node.getVersion()) == false) { serializedStates.put(node.getVersion(), serializeFullClusterState(clusterState, node.getVersion())); diff --git a/core/src/main/java/org/elasticsearch/transport/netty/NettyTransport.java b/core/src/main/java/org/elasticsearch/transport/netty/NettyTransport.java index 59649bff78365..f496d0729d10c 100644 --- a/core/src/main/java/org/elasticsearch/transport/netty/NettyTransport.java +++ b/core/src/main/java/org/elasticsearch/transport/netty/NettyTransport.java @@ -958,7 +958,7 @@ public void connectToNode(DiscoveryNode node, boolean light) { globalLock.readLock().lock(); try { - try (Releasable ignored = connectionLock.acquire(node.getId())) { + try (Releasable ignored = connectionLock.acquire(node.getEphemeralId())) { if (!lifecycle.started()) { throw new IllegalStateException("can't add nodes to a stopped transport"); } @@ -1116,7 +1116,7 @@ protected void connectToChannels(NodeChannels nodeChannels, DiscoveryNode node) @Override public void disconnectFromNode(DiscoveryNode node) { - try (Releasable ignored = connectionLock.acquire(node.getId())) { + try (Releasable ignored = connectionLock.acquire(node.getEphemeralId())) { NodeChannels nodeChannels = connectedNodes.remove(node); if (nodeChannels != null) { try { @@ -1138,7 +1138,7 @@ protected boolean disconnectFromNode(DiscoveryNode node, Channel channel, String // check outside of the lock NodeChannels nodeChannels = connectedNodes.get(node); if (nodeChannels != null && nodeChannels.hasChannel(channel)) { - try (Releasable ignored = connectionLock.acquire(node.getId())) { + try (Releasable ignored = connectionLock.acquire(node.getEphemeralId())) { nodeChannels = connectedNodes.get(node); // check again within the connection lock, if its still applicable to remove it if (nodeChannels != null && nodeChannels.hasChannel(channel)) { diff --git a/core/src/main/java/org/elasticsearch/tribe/TribeService.java b/core/src/main/java/org/elasticsearch/tribe/TribeService.java index 568fb7d002e72..a061a997eb09b 100644 --- a/core/src/main/java/org/elasticsearch/tribe/TribeService.java +++ b/core/src/main/java/org/elasticsearch/tribe/TribeService.java @@ -363,7 +363,7 @@ private ClusterState applyUpdate(ClusterState currentState, ClusterChangedEvent for (DiscoveryNode discoNode : currentState.nodes()) { String markedTribeName = discoNode.getAttributes().get(TRIBE_NAME_SETTING.getKey()); if (markedTribeName != null && markedTribeName.equals(tribeName)) { - if (tribeState.nodes().get(discoNode.getId()) == null) { + if (tribeState.nodes().nodeExists(discoNode.getId(), discoNode.getEphemeralId()) == false) { clusterStateChanged = true; logger.info("[{}] removing node [{}]", tribeName, discoNode); nodes.remove(discoNode.getId()); @@ -372,11 +372,11 @@ private ClusterState applyUpdate(ClusterState currentState, ClusterChangedEvent } // go over tribe nodes, and see if they need to be added for (DiscoveryNode tribe : tribeState.nodes()) { - if (currentState.nodes().get(tribe.getId()) == null) { + if (currentState.nodes().nodeExists(tribe.getId(), tribe.getEphemeralId()) == false) { // a new node, add it, but also add the tribe name to the attributes Map tribeAttr = new HashMap<>(tribe.getAttributes()); tribeAttr.put(TRIBE_NAME_SETTING.getKey(), tribeName); - DiscoveryNode discoNode = new DiscoveryNode(tribe.getName(), tribe.getId(), tribe.getNodeId(), + DiscoveryNode discoNode = new DiscoveryNode(tribe.getName(), tribe.getId(), tribe.getEphemeralId(), tribe.getHostName(), tribe.getHostAddress(), tribe.getAddress(), unmodifiableMap(tribeAttr), tribe.getRoles(), tribe.getVersion()); clusterStateChanged = true; diff --git a/core/src/test/java/org/elasticsearch/client/transport/TransportClientNodesServiceTests.java b/core/src/test/java/org/elasticsearch/client/transport/TransportClientNodesServiceTests.java index 01b968abcd558..461c928d42587 100644 --- a/core/src/test/java/org/elasticsearch/client/transport/TransportClientNodesServiceTests.java +++ b/core/src/test/java/org/elasticsearch/client/transport/TransportClientNodesServiceTests.java @@ -119,7 +119,7 @@ public T newInstance() { @SuppressWarnings("unchecked") public void handleResponse(T response) { LivenessResponse livenessResponse = new LivenessResponse(ClusterName.DEFAULT, - new DiscoveryNode(node.getName(), node.getId(), node.getNodeId(), "liveness-hostname" + node.getId(), + new DiscoveryNode(node.getName(), node.getId(), node.getEphemeralId(), "liveness-hostname" + node.getId(), "liveness-hostaddress" + node.getId(), new LocalTransportAddress("liveness-address-" + node.getId()), node.getAttributes(), node.getRoles(), node.getVersion())); diff --git a/core/src/test/java/org/elasticsearch/cluster/ClusterChangedEventTests.java b/core/src/test/java/org/elasticsearch/cluster/ClusterChangedEventTests.java index a8fd033265b2f..01d8b9acdc982 100644 --- a/core/src/test/java/org/elasticsearch/cluster/ClusterChangedEventTests.java +++ b/core/src/test/java/org/elasticsearch/cluster/ClusterChangedEventTests.java @@ -298,10 +298,10 @@ private static DiscoveryNodes createDiscoveryNodes(final int numNodes, final boo roles.add(DiscoveryNode.Role.DATA); } else { // remaining nodes can be anything (except for master) - if (randomBoolean()) { + if (i % 3 == 0) { roles.add(DiscoveryNode.Role.MASTER); } - if (randomBoolean()) { + if (i % 5 == 0) { roles.add(DiscoveryNode.Role.DATA); } } @@ -315,8 +315,8 @@ private static DiscoveryNodes createDiscoveryNodes(final int numNodes, final boo } // Create a new DiscoveryNode - private static DiscoveryNode newNode(final String nodeId, Set roles) { - return new DiscoveryNode(nodeId, nodeId, LocalTransportAddress.buildUnique(), Collections.emptyMap(), roles, Version.CURRENT); + private static DiscoveryNode newNode(final String id, Set roles) { + return new DiscoveryNode(id, id, new LocalTransportAddress("addr_" + id), Collections.emptyMap(), roles, Version.CURRENT); } // Create the metadata for a cluster state. diff --git a/core/src/test/java/org/elasticsearch/cluster/node/DiscoveryNodeFiltersTests.java b/core/src/test/java/org/elasticsearch/cluster/node/DiscoveryNodeFiltersTests.java index f6a06a5c51750..78082f58eed91 100644 --- a/core/src/test/java/org/elasticsearch/cluster/node/DiscoveryNodeFiltersTests.java +++ b/core/src/test/java/org/elasticsearch/cluster/node/DiscoveryNodeFiltersTests.java @@ -64,11 +64,11 @@ public void testNameMatch() { .build(); DiscoveryNodeFilters filters = DiscoveryNodeFilters.buildFromSettings(OR, "xxx.", settings); - DiscoveryNode node = new DiscoveryNode("name1", "pid1", "nid1", LocalTransportAddress.buildUnique(), emptyMap(), + DiscoveryNode node = new DiscoveryNode("name1", "nid1", "pid1", LocalTransportAddress.buildUnique(), emptyMap(), emptySet(), Version.CURRENT); assertThat(filters.match(node), equalTo(true)); - node = new DiscoveryNode("name2", "pid2", "nid2", LocalTransportAddress.buildUnique(), emptyMap(), emptySet(), + node = new DiscoveryNode("name2", "nid2", "pid2", LocalTransportAddress.buildUnique(), emptyMap(), emptySet(), Version.CURRENT); assertThat(filters.match(node), equalTo(false)); } @@ -79,11 +79,11 @@ public void testIdMatch() { .build(); DiscoveryNodeFilters filters = DiscoveryNodeFilters.buildFromSettings(OR, "xxx.", settings); - DiscoveryNode node = new DiscoveryNode("name1", "pid1", "nid1", LocalTransportAddress.buildUnique(), emptyMap(), + DiscoveryNode node = new DiscoveryNode("name1", "nid1", "pid1", LocalTransportAddress.buildUnique(), emptyMap(), emptySet(), Version.CURRENT); assertThat(filters.match(node), equalTo(true)); - node = new DiscoveryNode("name2", "pid2", "nid2", LocalTransportAddress.buildUnique(), emptyMap(), emptySet(), + node = new DiscoveryNode("name2", "nid2", "pid2", LocalTransportAddress.buildUnique(), emptyMap(), emptySet(), Version.CURRENT); assertThat(filters.match(node), equalTo(false)); } @@ -95,15 +95,15 @@ public void testIdOrNameMatch() { .build()); DiscoveryNodeFilters filters = DiscoveryNodeFilters.buildFromSettings(OR, "xxx.", settings); - DiscoveryNode node = new DiscoveryNode("name1", "pid1", "nid1", LocalTransportAddress.buildUnique(), emptyMap(), + DiscoveryNode node = new DiscoveryNode("name1", "nid1", "pid1", LocalTransportAddress.buildUnique(), emptyMap(), emptySet(), Version.CURRENT); assertThat(filters.match(node), equalTo(true)); - node = new DiscoveryNode("name2", "pid2", "nid2", LocalTransportAddress.buildUnique(), emptyMap(), emptySet(), + node = new DiscoveryNode("name2", "nid2", "pid2", LocalTransportAddress.buildUnique(), emptyMap(), emptySet(), Version.CURRENT); assertThat(filters.match(node), equalTo(true)); - node = new DiscoveryNode("name3", "pid3", "nid3", LocalTransportAddress.buildUnique(), emptyMap(), emptySet(), + node = new DiscoveryNode("name3", "nid3", "pid3", LocalTransportAddress.buildUnique(), emptyMap(), emptySet(), Version.CURRENT); assertThat(filters.match(node), equalTo(false)); } @@ -148,7 +148,7 @@ public void testStarMatch() { .build(); DiscoveryNodeFilters filters = DiscoveryNodeFilters.buildFromSettings(OR, "xxx.", settings); - DiscoveryNode node = new DiscoveryNode("name1", "pid1", "nid1", LocalTransportAddress.buildUnique(), emptyMap(), + DiscoveryNode node = new DiscoveryNode("name1", "nid1", "pid1", LocalTransportAddress.buildUnique(), emptyMap(), emptySet(), Version.CURRENT); assertThat(filters.match(node), equalTo(true)); } diff --git a/core/src/test/java/org/elasticsearch/cluster/node/DiscoveryNodeTests.java b/core/src/test/java/org/elasticsearch/cluster/node/DiscoveryNodeTests.java index 41602630fabde..e7b9bc86c1c91 100644 --- a/core/src/test/java/org/elasticsearch/cluster/node/DiscoveryNodeTests.java +++ b/core/src/test/java/org/elasticsearch/cluster/node/DiscoveryNodeTests.java @@ -58,33 +58,30 @@ public void testEquality() { logger.info("node2: {}", node2); assertFalse(node1.equals(node2)); - assertFalse(node1.equalsIncludingMetaData(node2)); DiscoveryNode sameIdDifferentMeta; - sameIdDifferentMeta = new DiscoveryNode(node2.getName(), node1.getId(), node1.getNodeId(), node2.getAddress(), + sameIdDifferentMeta = new DiscoveryNode(node2.getName(), node1.getId(), node1.getEphemeralId(), node2.getAddress(), node2.getAttributes(), node2.getRoles(), node2.getVersion()); - assertTrue(node1.equals(sameIdDifferentMeta)); - assertFalse(node1.equalsIncludingMetaData(sameIdDifferentMeta)); + assertFalse(node1.equals(sameIdDifferentMeta)); - sameIdDifferentMeta = new DiscoveryNode(node1.getName(), node1.getId(), node2.getNodeId(), node1.getAddress(), + sameIdDifferentMeta = new DiscoveryNode(node1.getName(), node1.getId(), node2.getEphemeralId(), node1.getAddress(), node1.getAttributes(), node1.getRoles(), node1.getVersion()); - assertTrue(node1.equals(sameIdDifferentMeta)); - assertFalse(node1.equalsIncludingMetaData(sameIdDifferentMeta)); + assertFalse(node1.equals(sameIdDifferentMeta)); - sameIdDifferentMeta = new DiscoveryNode(node2.getName(), node1.getId(), node1.getNodeId(), node1.getAddress(), + sameIdDifferentMeta = new DiscoveryNode(node2.getName(), node1.getId(), node1.getEphemeralId(), node1.getAddress(), node1.getAttributes(), (randomBoolean() ? node1 : node2).getRoles(), (randomBoolean() ? node1 : node2).getVersion()); - assertTrue(node1.equals(sameIdDifferentMeta)); - assertFalse(node1.equalsIncludingMetaData(sameIdDifferentMeta)); + assertFalse(node1.equals(sameIdDifferentMeta)); - sameIdDifferentMeta = new DiscoveryNode(node1.getName(), node1.getId(), node1.getNodeId(), node2.getAddress(), + sameIdDifferentMeta = new DiscoveryNode(node1.getName(), node1.getId(), node1.getEphemeralId(), node2.getAddress(), node1.getAttributes(), (randomBoolean() ? node1 : node2).getRoles(), (randomBoolean() ? node1 : node2).getVersion()); - assertTrue(node1.equals(sameIdDifferentMeta)); - assertFalse(node1.equalsIncludingMetaData(sameIdDifferentMeta)); + assertFalse(node1.equals(sameIdDifferentMeta)); - sameIdDifferentMeta = new DiscoveryNode(node1.getName(), node1.getId(), node1.getNodeId(), node1.getAddress(), + sameIdDifferentMeta = new DiscoveryNode(node1.getName(), node1.getId(), node1.getEphemeralId(), node1.getAddress(), node2.getAttributes(), (randomBoolean() ? node1 : node2).getRoles(), (randomBoolean() ? node1 : node2).getVersion()); - assertTrue(node1.equals(sameIdDifferentMeta)); - assertFalse(node1.equalsIncludingMetaData(sameIdDifferentMeta)); + assertFalse(node1.equals(sameIdDifferentMeta)); + + assertTrue(node1.equals(new DiscoveryNode(node1.getName(), node1.getId(), node1.getEphemeralId(), node1.getAddress(), + node1.getAttributes(), node1.getRoles(), node1.getVersion()))); } } diff --git a/core/src/test/java/org/elasticsearch/cluster/node/DiscoveryNodesTests.java b/core/src/test/java/org/elasticsearch/cluster/node/DiscoveryNodesTests.java index 96974a155abe7..74309d1cab261 100644 --- a/core/src/test/java/org/elasticsearch/cluster/node/DiscoveryNodesTests.java +++ b/core/src/test/java/org/elasticsearch/cluster/node/DiscoveryNodesTests.java @@ -19,6 +19,7 @@ package org.elasticsearch.cluster.node; +import com.carrotsearch.randomizedtesting.generators.RandomPicks; import org.elasticsearch.Version; import org.elasticsearch.common.transport.LocalTransportAddress; import org.elasticsearch.test.ESTestCase; @@ -30,10 +31,15 @@ import java.util.HashSet; import java.util.List; import java.util.Map; +import java.util.Objects; import java.util.Set; +import java.util.concurrent.atomic.AtomicInteger; +import java.util.stream.Collectors; import static org.hamcrest.CoreMatchers.containsString; import static org.hamcrest.CoreMatchers.equalTo; +import static org.hamcrest.Matchers.containsInAnyOrder; +import static org.hamcrest.Matchers.nullValue; public class DiscoveryNodesTests extends ESTestCase { @@ -41,7 +47,7 @@ public void testResolveNodeByIdOrName() { DiscoveryNodes discoveryNodes = buildDiscoveryNodes(); DiscoveryNode[] nodes = discoveryNodes.getNodes().values().toArray(DiscoveryNode.class); DiscoveryNode node = randomFrom(nodes); - DiscoveryNode resolvedNode = discoveryNodes.resolveNode(randomFrom(node.getId(), node.getNodeId(), node.getName())); + DiscoveryNode resolvedNode = discoveryNodes.resolveNode(randomFrom(node.getId(), node.getEphemeralId(), node.getName())); assertThat(resolvedNode.getId(), equalTo(node.getId())); } @@ -53,7 +59,7 @@ public void testResolveNodeByAttribute() { DiscoveryNode resolvedNode = discoveryNodes.resolveNode(nodeSelector.selector); assertThat(matchingNodeIds.size(), equalTo(1)); assertThat(resolvedNode.getId(), equalTo(matchingNodeIds.iterator().next())); - } catch(IllegalArgumentException e) { + } catch (IllegalArgumentException e) { if (matchingNodeIds.size() == 0) { assertThat(e.getMessage(), equalTo("failed to resolve [" + nodeSelector.selector + "], no matching nodes")); } else if (matchingNodeIds.size() > 1) { @@ -90,6 +96,13 @@ public void testResolveNodesIds() { nodeSelectors.add(discoveryNode.getName()); expectedNodeIdsSet.add(discoveryNode.getId()); } + int numEphemeralIds = randomIntBetween(0, 3); + DiscoveryNode[] ephemeralNodes = discoveryNodes.getNodes().values().toArray(DiscoveryNode.class); + for (int i = 0; i < numEphemeralIds; i++) { + DiscoveryNode discoveryNode = randomFrom(ephemeralNodes); + nodeSelectors.add(discoveryNode.getEphemeralId()); + expectedNodeIdsSet.add(discoveryNode.getId()); + } String[] resolvedNodesIds = discoveryNodes.resolveNodesIds(nodeSelectors.toArray(new String[nodeSelectors.size()])); Arrays.sort(resolvedNodesIds); @@ -98,26 +111,97 @@ public void testResolveNodesIds() { assertThat(resolvedNodesIds, equalTo(expectedNodesIds)); } - private static DiscoveryNodes buildDiscoveryNodes() { - int numNodes = randomIntBetween(1, 10); - DiscoveryNodes.Builder discoBuilder = DiscoveryNodes.builder(); + public void testDeltas() { + Set nodesA = new HashSet<>(); + nodesA.addAll(randomNodes(1 + randomInt(10))); + Set nodesB = new HashSet<>(); + nodesB.addAll(randomNodes(1 + randomInt(5))); + for (DiscoveryNode node : randomSubsetOf(nodesA)) { + if (randomBoolean()) { + // change an attribute + Map attrs = new HashMap<>(node.getAttributes()); + attrs.put("new", "new"); + node = new DiscoveryNode(node.getName(), node.getId(), node.getAddress(), attrs, node.getRoles(), node.getVersion()); + } + nodesB.add(node); + } + + DiscoveryNode masterA = randomBoolean() ? null : RandomPicks.randomFrom(random(), nodesA); + DiscoveryNode masterB = randomBoolean() ? null : RandomPicks.randomFrom(random(), nodesB); + + DiscoveryNodes.Builder builderA = DiscoveryNodes.builder(); + nodesA.stream().forEach(builderA::put); + final String masterAId = masterA == null ? null : masterA.getId(); + builderA.masterNodeId(masterAId); + builderA.localNodeId(RandomPicks.randomFrom(random(), nodesA).getId()); + + DiscoveryNodes.Builder builderB = DiscoveryNodes.builder(); + nodesB.stream().forEach(builderB::put); + final String masterBId = masterB == null ? null : masterB.getId(); + builderB.masterNodeId(masterBId); + builderB.localNodeId(RandomPicks.randomFrom(random(), nodesB).getId()); + + final DiscoveryNodes discoNodesA = builderA.build(); + final DiscoveryNodes discoNodesB = builderB.build(); + logger.info("nodes A: {}", discoNodesA.prettyPrint()); + logger.info("nodes B: {}", discoNodesB.prettyPrint()); + + DiscoveryNodes.Delta delta = discoNodesB.delta(discoNodesA); + + if (masterB == null || Objects.equals(masterAId, masterBId)) { + assertFalse(delta.masterNodeChanged()); + assertThat(delta.previousMasterNode(), nullValue()); + assertThat(delta.newMasterNode(), nullValue()); + } else { + assertTrue(delta.masterNodeChanged()); + assertThat(delta.newMasterNode().getId(), equalTo(masterBId)); + assertThat(delta.previousMasterNode() != null ? delta.previousMasterNode().getId() : null, + equalTo(masterAId)); + } + + Set newNodes = new HashSet<>(nodesB); + newNodes.removeAll(nodesA); + assertThat(delta.added(), equalTo(newNodes.isEmpty() == false)); + assertThat(delta.addedNodes(), containsInAnyOrder(newNodes.stream().collect(Collectors.toList()).toArray())); + assertThat(delta.addedNodes().size(), equalTo(newNodes.size())); + + Set removedNodes = new HashSet<>(nodesA); + removedNodes.removeAll(nodesB); + assertThat(delta.removed(), equalTo(removedNodes.isEmpty() == false)); + assertThat(delta.removedNodes(), containsInAnyOrder(removedNodes.stream().collect(Collectors.toList()).toArray())); + assertThat(delta.removedNodes().size(), equalTo(removedNodes.size())); + } + + private static AtomicInteger idGenerator = new AtomicInteger(); + + private static List randomNodes(final int numNodes) { List nodesList = new ArrayList<>(); for (int i = 0; i < numNodes; i++) { Map attributes = new HashMap<>(); if (frequently()) { attributes.put("custom", randomBoolean() ? "match" : randomAsciiOfLengthBetween(3, 5)); } - final DiscoveryNode node = newNode(i, attributes, new HashSet<>(randomSubsetOf(Arrays.asList(DiscoveryNode.Role.values())))); - discoBuilder = discoBuilder.put(node); + final DiscoveryNode node = newNode(idGenerator.getAndIncrement(), attributes, + new HashSet<>(randomSubsetOf(Arrays.asList(DiscoveryNode.Role.values())))); nodesList.add(node); } + return nodesList; + } + + private static DiscoveryNodes buildDiscoveryNodes() { + int numNodes = randomIntBetween(1, 10); + DiscoveryNodes.Builder discoBuilder = DiscoveryNodes.builder(); + List nodesList = randomNodes(numNodes); + for (DiscoveryNode node : nodesList) { + discoBuilder = discoBuilder.put(node); + } discoBuilder.localNodeId(randomFrom(nodesList).getId()); discoBuilder.masterNodeId(randomFrom(nodesList).getId()); return discoBuilder.build(); } private static DiscoveryNode newNode(int nodeId, Map attributes, Set roles) { - return new DiscoveryNode("name_" + nodeId, "process_" + nodeId, "persistent_node_" + nodeId, LocalTransportAddress.buildUnique(), + return new DiscoveryNode("name_" + nodeId, "node_" + nodeId, "pid_" + nodeId, LocalTransportAddress.buildUnique(), attributes, roles, Version.CURRENT); } diff --git a/core/src/test/java/org/elasticsearch/cluster/routing/allocation/BalanceConfigurationTests.java b/core/src/test/java/org/elasticsearch/cluster/routing/allocation/BalanceConfigurationTests.java index 7e1f909bca65f..67d8c5059ef52 100644 --- a/core/src/test/java/org/elasticsearch/cluster/routing/allocation/BalanceConfigurationTests.java +++ b/core/src/test/java/org/elasticsearch/cluster/routing/allocation/BalanceConfigurationTests.java @@ -236,7 +236,7 @@ private void assertReplicaBalance(ESLogger logger, RoutingNodes nodes, int numbe final int maxAvgNumberOfShards = Math.round(Math.round(Math.ceil(avgNumShards + treshold))); for (RoutingNode node : nodes) { -// logger.info(node.getNodeId() + ": " + node.shardsWithState(INITIALIZING, STARTED).size() + " shards ("+minAvgNumberOfShards+" to "+maxAvgNumberOfShards+")"); +// logger.info(node.getEphemeralId() + ": " + node.shardsWithState(INITIALIZING, STARTED).size() + " shards ("+minAvgNumberOfShards+" to "+maxAvgNumberOfShards+")"); assertThat(node.shardsWithState(STARTED).size(), Matchers.greaterThanOrEqualTo(minAvgNumberOfShards)); assertThat(node.shardsWithState(STARTED).size(), Matchers.lessThanOrEqualTo(maxAvgNumberOfShards)); } @@ -251,7 +251,7 @@ private void assertIndexBalance(ESLogger logger, RoutingNodes nodes, int numberO for (ObjectCursor index : nodes.getRoutingTable().indicesRouting().keys()) { for (RoutingNode node : nodes) { -// logger.info(node.getNodeId() +":"+index+ ": " + node.shardsWithState(index, INITIALIZING, STARTED).size() + " shards ("+minAvgNumberOfShards+" to "+maxAvgNumberOfShards+")"); +// logger.info(node.getEphemeralId() +":"+index+ ": " + node.shardsWithState(index, INITIALIZING, STARTED).size() + " shards ("+minAvgNumberOfShards+" to "+maxAvgNumberOfShards+")"); assertThat(node.shardsWithState(index.value, STARTED).size(), Matchers.greaterThanOrEqualTo(minAvgNumberOfShards)); assertThat(node.shardsWithState(index.value, STARTED).size(), Matchers.lessThanOrEqualTo(maxAvgNumberOfShards)); } @@ -271,7 +271,7 @@ private void assertPrimaryBalance(ESLogger logger, RoutingNodes nodes, int numbe for (ShardRouting shard : node.shardsWithState(index.value, STARTED)) { primaries += shard.primary() ? 1 : 0; } -// logger.info(node.getNodeId() + ": " + primaries + " primaries ("+minAvgNumberOfShards+" to "+maxAvgNumberOfShards+")"); +// logger.info(node.getEphemeralId() + ": " + primaries + " primaries ("+minAvgNumberOfShards+" to "+maxAvgNumberOfShards+")"); assertThat(primaries, Matchers.greaterThanOrEqualTo(minAvgNumberOfShards)); assertThat(primaries, Matchers.lessThanOrEqualTo(maxAvgNumberOfShards)); } diff --git a/core/src/test/java/org/elasticsearch/discovery/zen/NodeJoinControllerTests.java b/core/src/test/java/org/elasticsearch/discovery/zen/NodeJoinControllerTests.java index b06b7b45e2add..e33b74d6bab23 100644 --- a/core/src/test/java/org/elasticsearch/discovery/zen/NodeJoinControllerTests.java +++ b/core/src/test/java/org/elasticsearch/discovery/zen/NodeJoinControllerTests.java @@ -67,6 +67,7 @@ import static java.util.Collections.shuffle; import static org.elasticsearch.cluster.service.ClusterServiceUtils.createClusterService; import static org.elasticsearch.cluster.service.ClusterServiceUtils.setState; +import static org.hamcrest.Matchers.both; import static org.hamcrest.Matchers.containsString; import static org.hamcrest.Matchers.equalTo; import static org.hamcrest.Matchers.instanceOf; @@ -420,24 +421,29 @@ public void testNewClusterStateOnExistingNodeJoin() throws InterruptedException, assertTrue("failed to publish a new state upon existing join", clusterService.state() != state); } - public void testRemoveExistingNodeWhenJoinWithSameAddressButDifferentProcessId() throws InterruptedException, ExecutionException { + public void testRemoveExistingNodeWhenJoinWithSameAddress() throws InterruptedException, ExecutionException { ClusterState state = clusterService.state(); - final DiscoveryNode other_node = new DiscoveryNode("other_node", state.nodes().getLocalNode().getAddress(), - emptyMap(), emptySet(), Version.CURRENT); + DiscoveryNode localNode = state.nodes().getLocalNode(); + String nodeId = randomFrom("other_node", localNode.getId()); + // consider case where node id and ephemeral id are not both identical + String ephemeralId = nodeId.equals("other_node") ? randomFrom(localNode.getEphemeralId(), "other_id") : localNode.getEphemeralId(); + final DiscoveryNode other_node = new DiscoveryNode(nodeId, ephemeralId, localNode.getAddress(), + localNode.getAttributes(), localNode.getRoles(), localNode.getVersion()); joinNode(other_node); state = clusterService.state(); assertThat(state.getNodes().getSize(), equalTo(1)); - assertTrue(state.getNodes().get(other_node.getId()).equalsIncludingMetaData(other_node)); + assertTrue(state.getNodes().get(other_node.getId()).equals(other_node)); } - public void testRejectingJoinWithSamePersistentIdButDifferentAddress() throws InterruptedException, ExecutionException { + public void testRejectingJoinWithSameNodeIdButDifferentEphemeralIdAndAddress() throws InterruptedException, ExecutionException { ClusterState state = clusterService.state(); - final DiscoveryNode other_node = new DiscoveryNode(UUIDs.randomBase64UUID(), state.nodes().getLocalNode().getId(), + final DiscoveryNode other_node = new DiscoveryNode(state.nodes().getLocalNode().getId(), UUIDs.randomBase64UUID(), LocalTransportAddress.buildUnique(), emptyMap(), emptySet(), Version.CURRENT); ExecutionException e = expectThrows(ExecutionException.class, () -> joinNode(other_node)); - assertThat(e.getMessage(), containsString("found existing node")); + assertThat(e.getMessage(), both(containsString("found existing node")).and( + containsString("with same node id but different ephemeral id"))); } public void testNormalConcurrentJoins() throws InterruptedException { diff --git a/core/src/test/java/org/elasticsearch/gateway/AsyncShardFetchTests.java b/core/src/test/java/org/elasticsearch/gateway/AsyncShardFetchTests.java index 8fa027711df8b..00ca8d4c80472 100644 --- a/core/src/test/java/org/elasticsearch/gateway/AsyncShardFetchTests.java +++ b/core/src/test/java/org/elasticsearch/gateway/AsyncShardFetchTests.java @@ -44,11 +44,11 @@ /** */ public class AsyncShardFetchTests extends ESTestCase { - private final DiscoveryNode node1 = new DiscoveryNode("process1", "node1", LocalTransportAddress.buildUnique(), Collections.emptyMap(), + private final DiscoveryNode node1 = new DiscoveryNode("node1", "process1", LocalTransportAddress.buildUnique(), Collections.emptyMap(), Collections.singleton(DiscoveryNode.Role.DATA), Version.CURRENT); private final Response response1 = new Response(node1); private final Throwable failure1 = new Throwable("simulated failure 1"); - private final DiscoveryNode node2 = new DiscoveryNode("process2", "node2", LocalTransportAddress.buildUnique(), Collections.emptyMap(), + private final DiscoveryNode node2 = new DiscoveryNode("node2", "process2", LocalTransportAddress.buildUnique(), Collections.emptyMap(), Collections.singleton(DiscoveryNode.Role.DATA), Version.CURRENT); private final Response response2 = new Response(node2); private final Throwable failure2 = new Throwable("simulate failure 2"); diff --git a/test/framework/src/main/java/org/elasticsearch/test/InternalTestCluster.java b/test/framework/src/main/java/org/elasticsearch/test/InternalTestCluster.java index 69e1fd3a2568e..e812dba23fae9 100644 --- a/test/framework/src/main/java/org/elasticsearch/test/InternalTestCluster.java +++ b/test/framework/src/main/java/org/elasticsearch/test/InternalTestCluster.java @@ -578,7 +578,7 @@ private NodeAndClient buildNode(int nodeId, long seed, Settings settings, Versio .put(Environment.PATH_HOME_SETTING.getKey(), baseDir) // allow overriding path.home .put(settings) .put("node.name", name) - .put(DiscoveryNodeService.PROCESS_ID_SEED_SETTING.getKey(), seed) + .put(DiscoveryNodeService.EPHEMERAL_ID_SEED_SETTING.getKey(), seed) .put(NodeEnvironment.NODE_ID_SEED_SETTING.getKey(), seed) .build(); MockNode node = new MockNode(finalSettings, version, plugins); @@ -828,11 +828,11 @@ void restart(RestartCallback callback) throws Exception { } } // use a new seed to make sure we have new process id - final long newProcessIdSeed = DiscoveryNodeService.PROCESS_ID_SEED_SETTING.get(node.settings()) + 1; + final long newProcessIdSeed = DiscoveryNodeService.EPHEMERAL_ID_SEED_SETTING.get(node.settings()) + 1; // use a new seed to make sure we have new node id, if the node type is one that generates new ids final long newNodeIdSeed = NodeEnvironment.NODE_ID_SEED_SETTING.get(node.settings()) + 1; Settings finalSettings = Settings.builder().put(node.settings()).put(newSettings) - .put(DiscoveryNodeService.PROCESS_ID_SEED_SETTING.getKey(), newProcessIdSeed) + .put(DiscoveryNodeService.EPHEMERAL_ID_SEED_SETTING.getKey(), newProcessIdSeed) .put(NodeEnvironment.NODE_ID_SEED_SETTING.getKey(), newNodeIdSeed) .build(); Collection> plugins = node.getPlugins(); diff --git a/test/framework/src/main/java/org/elasticsearch/test/transport/CapturingTransport.java b/test/framework/src/main/java/org/elasticsearch/test/transport/CapturingTransport.java index 654a1c971cae0..116bf0def7d95 100644 --- a/test/framework/src/main/java/org/elasticsearch/test/transport/CapturingTransport.java +++ b/test/framework/src/main/java/org/elasticsearch/test/transport/CapturingTransport.java @@ -93,10 +93,10 @@ public CapturedRequest[] getCapturedRequestsAndClear() { public Map> capturedRequestsByTargetNode() { Map> map = new HashMap<>(); for (CapturedRequest request : capturedRequests) { - List nodeList = map.get(request.node.getId()); + List nodeList = map.get(request.node.getEphemeralId()); if (nodeList == null) { nodeList = new ArrayList<>(); - map.put(request.node.getId(), nodeList); + map.put(request.node.getEphemeralId(), nodeList); } nodeList.add(request); } From 258fbad5ae36c5c329a4ce51f06ce7f2df773ed5 Mon Sep 17 00:00:00 2001 From: Yannick Welsch Date: Tue, 10 May 2016 13:41:23 +0200 Subject: [PATCH 31/34] fix tests --- .../elasticsearch/discovery/zen/fd/MasterFaultDetection.java | 2 +- .../elasticsearch/cluster/node/DiscoveryNodeFiltersTests.java | 4 ++-- 2 files changed, 3 insertions(+), 3 deletions(-) diff --git a/core/src/main/java/org/elasticsearch/discovery/zen/fd/MasterFaultDetection.java b/core/src/main/java/org/elasticsearch/discovery/zen/fd/MasterFaultDetection.java index bd1528777a481..724f34469246c 100644 --- a/core/src/main/java/org/elasticsearch/discovery/zen/fd/MasterFaultDetection.java +++ b/core/src/main/java/org/elasticsearch/discovery/zen/fd/MasterFaultDetection.java @@ -171,7 +171,7 @@ protected void handleTransportDisconnect(DiscoveryNode node) { synchronized (masterNodeMutex) { if (this.masterNode == null || node.getId().equals(this.masterNode.getId()) == false || - node.getEphemeralId().equals(this.masterNode.getEphemeralId())) { + node.getEphemeralId().equals(this.masterNode.getEphemeralId()) == false) { return; } if (connectOnNetworkDisconnect) { diff --git a/core/src/test/java/org/elasticsearch/cluster/node/DiscoveryNodeFiltersTests.java b/core/src/test/java/org/elasticsearch/cluster/node/DiscoveryNodeFiltersTests.java index 78082f58eed91..e69f56f9ab445 100644 --- a/core/src/test/java/org/elasticsearch/cluster/node/DiscoveryNodeFiltersTests.java +++ b/core/src/test/java/org/elasticsearch/cluster/node/DiscoveryNodeFiltersTests.java @@ -75,7 +75,7 @@ public void testNameMatch() { public void testIdMatch() { Settings settings = Settings.builder() - .put("xxx._id", "pid1") + .put("xxx._id", "nid1") .build(); DiscoveryNodeFilters filters = DiscoveryNodeFilters.buildFromSettings(OR, "xxx.", settings); @@ -90,7 +90,7 @@ public void testIdMatch() { public void testIdOrNameMatch() { Settings settings = shuffleSettings(Settings.builder() - .put("xxx._id", "pid1,blah") + .put("xxx._id", "nid1,blah") .put("xxx.name", "blah,name2") .build()); DiscoveryNodeFilters filters = DiscoveryNodeFilters.buildFromSettings(OR, "xxx.", settings); From 10bbddbc8ad4839bea8af2b761bf355ed0bdf038 Mon Sep 17 00:00:00 2001 From: Yannick Welsch Date: Tue, 10 May 2016 15:55:02 +0200 Subject: [PATCH 32/34] minor fixes --- .../main/java/org/elasticsearch/cluster/node/DiscoveryNode.java | 2 +- .../org/elasticsearch/cluster/node/DiscoveryNodeService.java | 2 +- 2 files changed, 2 insertions(+), 2 deletions(-) diff --git a/core/src/main/java/org/elasticsearch/cluster/node/DiscoveryNode.java b/core/src/main/java/org/elasticsearch/cluster/node/DiscoveryNode.java index d4bc1fcb1cfca..39c292b284642 100644 --- a/core/src/main/java/org/elasticsearch/cluster/node/DiscoveryNode.java +++ b/core/src/main/java/org/elasticsearch/cluster/node/DiscoveryNode.java @@ -386,8 +386,8 @@ public String toString() { if (nodeName.length() > 0) { sb.append('{').append(nodeName).append('}'); } - sb.append('{').append(ephemeralId).append('}'); sb.append('{').append(nodeId).append('}'); + sb.append('{').append(ephemeralId).append('}'); sb.append('{').append(hostName).append('}'); sb.append('{').append(address).append('}'); if (!attributes.isEmpty()) { diff --git a/core/src/main/java/org/elasticsearch/cluster/node/DiscoveryNodeService.java b/core/src/main/java/org/elasticsearch/cluster/node/DiscoveryNodeService.java index 6937a96d7f2d9..e743503048485 100644 --- a/core/src/main/java/org/elasticsearch/cluster/node/DiscoveryNodeService.java +++ b/core/src/main/java/org/elasticsearch/cluster/node/DiscoveryNodeService.java @@ -96,7 +96,7 @@ public DiscoveryNode buildLocalNode(TransportAddress publishAddress, String node logger.warn("failed to build custom attributes from provider [{}]", e, provider); } } - return new DiscoveryNode(Node.NODE_NAME_SETTING.get(settings), ephemeralId, nodeId, publishAddress, + return new DiscoveryNode(Node.NODE_NAME_SETTING.get(settings), nodeId, ephemeralId, publishAddress, attributes, roles, version); } From a1ff30854d6e86d8cc43593b2e7b1778d8695634 Mon Sep 17 00:00:00 2001 From: Yannick Welsch Date: Tue, 10 May 2016 16:53:16 +0200 Subject: [PATCH 33/34] Fix test --- .../discovery/zen/NodeJoinControllerTests.java | 7 ------- 1 file changed, 7 deletions(-) diff --git a/core/src/test/java/org/elasticsearch/discovery/zen/NodeJoinControllerTests.java b/core/src/test/java/org/elasticsearch/discovery/zen/NodeJoinControllerTests.java index e33b74d6bab23..baad82c8b73bd 100644 --- a/core/src/test/java/org/elasticsearch/discovery/zen/NodeJoinControllerTests.java +++ b/core/src/test/java/org/elasticsearch/discovery/zen/NodeJoinControllerTests.java @@ -129,17 +129,10 @@ public void testSimpleJoinAccumulation() throws InterruptedException, ExecutionE pendingJoins.add(joinNodeAsync(node)); } nodeJoinController.stopAccumulatingJoins("test"); - boolean hadSyncJoin = false; for (int i = randomInt(5); i > 0; i--) { DiscoveryNode node = newNode(nodeId++); nodes.add(node); joinNode(node); - hadSyncJoin = true; - } - if (hadSyncJoin) { - for (Future joinFuture : pendingJoins) { - assertThat(joinFuture.isDone(), equalTo(true)); - } } for (Future joinFuture : pendingJoins) { joinFuture.get(); From 0174ea5010e82a98eb5ecc1682b569c17d0e18d7 Mon Sep 17 00:00:00 2001 From: Yannick Welsch Date: Thu, 12 May 2016 10:42:58 +0200 Subject: [PATCH 34/34] Stop overloading madness --- .../cluster/node/DiscoveryNode.java | 23 +------------------ .../ingest/IngestProxyActionFilterTests.java | 2 +- .../nodes/TransportNodesActionTests.java | 3 +-- .../cluster/ClusterChangedEventTests.java | 2 +- .../cluster/NodeConnectionsServiceTests.java | 2 +- .../node/DiscoveryNodeFiltersTests.java | 8 +++---- .../cluster/node/DiscoveryNodesTests.java | 3 ++- .../discovery/ZenFaultDetectionTests.java | 4 ++-- .../zen/ElectMasterServiceTests.java | 2 +- .../zen/NodeJoinControllerTests.java | 8 +++---- .../discovery/zen/ZenDiscoveryUnitTests.java | 10 ++++---- .../gateway/AsyncShardFetchTests.java | 4 ++-- .../AbstractSimpleTransportTestCase.java | 2 +- .../netty/NettyScheduledPingTests.java | 4 ++-- .../test/ESAllocationTestCase.java | 6 ++--- 15 files changed, 31 insertions(+), 52 deletions(-) diff --git a/core/src/main/java/org/elasticsearch/cluster/node/DiscoveryNode.java b/core/src/main/java/org/elasticsearch/cluster/node/DiscoveryNode.java index 39c292b284642..cff00645b0af0 100644 --- a/core/src/main/java/org/elasticsearch/cluster/node/DiscoveryNode.java +++ b/core/src/main/java/org/elasticsearch/cluster/node/DiscoveryNode.java @@ -113,28 +113,7 @@ public static boolean isIngestNode(Settings settings) { */ public DiscoveryNode(String id, TransportAddress address, Map attributes, Set roles, Version version) { - this(id, id, address, attributes, roles, version); - } - - /** - * Creates a new {@link DiscoveryNode} - *

- * Note: if the version of the node is unknown {@link Version#minimumCompatibilityVersion()} should be used for the current - * version. it corresponds to the minimum version this elasticsearch version can communicate with. If a higher version is used - * the node might not be able to communicate with the remove node. After initial handshakes node versions will be discovered - * and updated. - *

- * - * @param nodeId the nodes unique persistent id - * @param ephemeralId the nodes unique ephemeral id - * @param address the nodes transport address - * @param attributes node attributes - * @param roles node roles - * @param version the version of the node - */ - public DiscoveryNode(String nodeId, String ephemeralId, TransportAddress address, Map attributes, - Set roles, Version version) { - this("", nodeId, ephemeralId, address.getHost(), address.getAddress(), address, attributes, roles, version); + this("", id, id, address, attributes, roles, version); } /** diff --git a/core/src/test/java/org/elasticsearch/action/ingest/IngestProxyActionFilterTests.java b/core/src/test/java/org/elasticsearch/action/ingest/IngestProxyActionFilterTests.java index 3d1a1a1c69d26..ebe0f64933a76 100644 --- a/core/src/test/java/org/elasticsearch/action/ingest/IngestProxyActionFilterTests.java +++ b/core/src/test/java/org/elasticsearch/action/ingest/IngestProxyActionFilterTests.java @@ -78,7 +78,7 @@ private IngestProxyActionFilter buildFilter(int ingestNodes, int totalNodes) { if (i < ingestNodes) { roles.add(DiscoveryNode.Role.INGEST); } - DiscoveryNode node = new DiscoveryNode(nodeId, nodeId, LocalTransportAddress.buildUnique(), attributes, roles, VersionUtils.randomVersion(random())); + DiscoveryNode node = new DiscoveryNode(nodeId, LocalTransportAddress.buildUnique(), attributes, roles, VersionUtils.randomVersion(random())); builder.put(node); if (i == totalNodes - 1) { localNode = node; diff --git a/core/src/test/java/org/elasticsearch/action/support/nodes/TransportNodesActionTests.java b/core/src/test/java/org/elasticsearch/action/support/nodes/TransportNodesActionTests.java index 37d36b3c12168..b5e224a71274e 100644 --- a/core/src/test/java/org/elasticsearch/action/support/nodes/TransportNodesActionTests.java +++ b/core/src/test/java/org/elasticsearch/action/support/nodes/TransportNodesActionTests.java @@ -161,8 +161,7 @@ public void tearDown() throws Exception { } private static DiscoveryNode newNode(int nodeId, Map attributes, Set roles) { - String node = "node_" + nodeId; - return new DiscoveryNode(node, node, LocalTransportAddress.buildUnique(), attributes, roles, Version.CURRENT); + return new DiscoveryNode("node_" + nodeId, LocalTransportAddress.buildUnique(), attributes, roles, Version.CURRENT); } private static class TestTransportNodesAction extends TransportNodesAction roles) { - return new DiscoveryNode(id, id, new LocalTransportAddress("addr_" + id), Collections.emptyMap(), roles, Version.CURRENT); + return new DiscoveryNode(id, new LocalTransportAddress("addr_" + id), Collections.emptyMap(), roles, Version.CURRENT); } // Create the metadata for a cluster state. diff --git a/core/src/test/java/org/elasticsearch/cluster/NodeConnectionsServiceTests.java b/core/src/test/java/org/elasticsearch/cluster/NodeConnectionsServiceTests.java index 754a2f4990bb3..63367bd5049a0 100644 --- a/core/src/test/java/org/elasticsearch/cluster/NodeConnectionsServiceTests.java +++ b/core/src/test/java/org/elasticsearch/cluster/NodeConnectionsServiceTests.java @@ -64,7 +64,7 @@ private List generateNodes() { List nodes = new ArrayList<>(); for (int i = randomIntBetween(20, 50); i > 0; i--) { Set roles = new HashSet<>(randomSubsetOf(Arrays.asList(DiscoveryNode.Role.values()))); - nodes.add(new DiscoveryNode("node_" + i, "" + i, LocalTransportAddress.buildUnique(), Collections.emptyMap(), + nodes.add(new DiscoveryNode("node_" + i, LocalTransportAddress.buildUnique(), Collections.emptyMap(), roles, Version.CURRENT)); } return nodes; diff --git a/core/src/test/java/org/elasticsearch/cluster/node/DiscoveryNodeFiltersTests.java b/core/src/test/java/org/elasticsearch/cluster/node/DiscoveryNodeFiltersTests.java index e69f56f9ab445..0c08a0b6d4539 100644 --- a/core/src/test/java/org/elasticsearch/cluster/node/DiscoveryNodeFiltersTests.java +++ b/core/src/test/java/org/elasticsearch/cluster/node/DiscoveryNodeFiltersTests.java @@ -118,7 +118,7 @@ public void testTagAndGroupMatch() { Map attributes = new HashMap<>(); attributes.put("tag", "A"); attributes.put("group", "B"); - DiscoveryNode node = new DiscoveryNode("name1", "id1", LocalTransportAddress.buildUnique(), + DiscoveryNode node = new DiscoveryNode("id1", LocalTransportAddress.buildUnique(), attributes, emptySet(), Version.CURRENT); assertThat(filters.match(node), equalTo(true)); @@ -126,7 +126,7 @@ public void testTagAndGroupMatch() { attributes.put("tag", "A"); attributes.put("group", "B"); attributes.put("name", "X"); - node = new DiscoveryNode("name2", "id2", LocalTransportAddress.buildUnique(), + node = new DiscoveryNode("id2", LocalTransportAddress.buildUnique(), attributes, emptySet(), Version.CURRENT); assertThat(filters.match(node), equalTo(true)); @@ -134,11 +134,11 @@ public void testTagAndGroupMatch() { attributes.put("tag", "A"); attributes.put("group", "F"); attributes.put("name", "X"); - node = new DiscoveryNode("name3", "id3", LocalTransportAddress.buildUnique(), + node = new DiscoveryNode("id3", LocalTransportAddress.buildUnique(), attributes, emptySet(), Version.CURRENT); assertThat(filters.match(node), equalTo(false)); - node = new DiscoveryNode("name4", "id4", LocalTransportAddress.buildUnique(), emptyMap(), emptySet(), Version.CURRENT); + node = new DiscoveryNode("id4", LocalTransportAddress.buildUnique(), emptyMap(), emptySet(), Version.CURRENT); assertThat(filters.match(node), equalTo(false)); } diff --git a/core/src/test/java/org/elasticsearch/cluster/node/DiscoveryNodesTests.java b/core/src/test/java/org/elasticsearch/cluster/node/DiscoveryNodesTests.java index 74309d1cab261..c5db4ecd3a91c 100644 --- a/core/src/test/java/org/elasticsearch/cluster/node/DiscoveryNodesTests.java +++ b/core/src/test/java/org/elasticsearch/cluster/node/DiscoveryNodesTests.java @@ -121,7 +121,8 @@ public void testDeltas() { // change an attribute Map attrs = new HashMap<>(node.getAttributes()); attrs.put("new", "new"); - node = new DiscoveryNode(node.getName(), node.getId(), node.getAddress(), attrs, node.getRoles(), node.getVersion()); + node = new DiscoveryNode(node.getName(), node.getId(), node.getEphemeralId(), node.getAddress(), attrs, node.getRoles(), + node.getVersion()); } nodesB.add(node); } diff --git a/core/src/test/java/org/elasticsearch/discovery/ZenFaultDetectionTests.java b/core/src/test/java/org/elasticsearch/discovery/ZenFaultDetectionTests.java index ab834ca6ae2ec..6a14854aa6e42 100644 --- a/core/src/test/java/org/elasticsearch/discovery/ZenFaultDetectionTests.java +++ b/core/src/test/java/org/elasticsearch/discovery/ZenFaultDetectionTests.java @@ -72,9 +72,9 @@ public void setUp() throws Exception { clusterService = createClusterService(threadPool); circuitBreakerService = new NoneCircuitBreakerService(); serviceA = build(Settings.builder().put("name", "TS_A").build(), version0); - nodeA = new DiscoveryNode("TS_A", "TS_A", serviceA.boundAddress().publishAddress(), emptyMap(), emptySet(), version0); + nodeA = new DiscoveryNode("TS_A", serviceA.boundAddress().publishAddress(), emptyMap(), emptySet(), version0); serviceB = build(Settings.builder().put("name", "TS_B").build(), version1); - nodeB = new DiscoveryNode("TS_B", "TS_B", serviceB.boundAddress().publishAddress(), emptyMap(), emptySet(), version1); + nodeB = new DiscoveryNode("TS_B", serviceB.boundAddress().publishAddress(), emptyMap(), emptySet(), version1); // wait till all nodes are properly connected and the event has been sent, so tests in this class // will not get this callback called on the connections done in this setup diff --git a/core/src/test/java/org/elasticsearch/discovery/zen/ElectMasterServiceTests.java b/core/src/test/java/org/elasticsearch/discovery/zen/ElectMasterServiceTests.java index b7b7744d086b0..07c2d816c56e9 100644 --- a/core/src/test/java/org/elasticsearch/discovery/zen/ElectMasterServiceTests.java +++ b/core/src/test/java/org/elasticsearch/discovery/zen/ElectMasterServiceTests.java @@ -49,7 +49,7 @@ List generateRandomNodes() { if (randomBoolean()) { roles.add(DiscoveryNode.Role.MASTER); } - DiscoveryNode node = new DiscoveryNode("n_" + i, "n_" + i, LocalTransportAddress.buildUnique(), Collections.emptyMap(), + DiscoveryNode node = new DiscoveryNode("n_" + i, LocalTransportAddress.buildUnique(), Collections.emptyMap(), roles, Version.CURRENT); nodes.add(node); } diff --git a/core/src/test/java/org/elasticsearch/discovery/zen/NodeJoinControllerTests.java b/core/src/test/java/org/elasticsearch/discovery/zen/NodeJoinControllerTests.java index baad82c8b73bd..de2cd43e98d87 100644 --- a/core/src/test/java/org/elasticsearch/discovery/zen/NodeJoinControllerTests.java +++ b/core/src/test/java/org/elasticsearch/discovery/zen/NodeJoinControllerTests.java @@ -420,7 +420,7 @@ public void testRemoveExistingNodeWhenJoinWithSameAddress() throws InterruptedEx String nodeId = randomFrom("other_node", localNode.getId()); // consider case where node id and ephemeral id are not both identical String ephemeralId = nodeId.equals("other_node") ? randomFrom(localNode.getEphemeralId(), "other_id") : localNode.getEphemeralId(); - final DiscoveryNode other_node = new DiscoveryNode(nodeId, ephemeralId, localNode.getAddress(), + final DiscoveryNode other_node = new DiscoveryNode(localNode.getName(), nodeId, ephemeralId, localNode.getAddress(), localNode.getAttributes(), localNode.getRoles(), localNode.getVersion()); joinNode(other_node); @@ -431,8 +431,8 @@ public void testRemoveExistingNodeWhenJoinWithSameAddress() throws InterruptedEx public void testRejectingJoinWithSameNodeIdButDifferentEphemeralIdAndAddress() throws InterruptedException, ExecutionException { ClusterState state = clusterService.state(); - final DiscoveryNode other_node = new DiscoveryNode(state.nodes().getLocalNode().getId(), UUIDs.randomBase64UUID(), - LocalTransportAddress.buildUnique(), emptyMap(), emptySet(), Version.CURRENT); + final DiscoveryNode other_node = new DiscoveryNode(state.nodes().getLocalNode().getName(), state.nodes().getLocalNode().getId(), + UUIDs.randomBase64UUID(), LocalTransportAddress.buildUnique(), emptyMap(), emptySet(), Version.CURRENT); ExecutionException e = expectThrows(ExecutionException.class, () -> joinNode(other_node)); assertThat(e.getMessage(), both(containsString("found existing node")).and( @@ -645,6 +645,6 @@ protected DiscoveryNode newNode(int i, boolean master) { roles.add(DiscoveryNode.Role.MASTER); } final String prefix = master ? "master_" : "data_"; - return new DiscoveryNode(prefix + i, i + "", new LocalTransportAddress("test_" + i), emptyMap(), roles, Version.CURRENT); + return new DiscoveryNode(prefix + i, new LocalTransportAddress("test_" + i), emptyMap(), roles, Version.CURRENT); } } diff --git a/core/src/test/java/org/elasticsearch/discovery/zen/ZenDiscoveryUnitTests.java b/core/src/test/java/org/elasticsearch/discovery/zen/ZenDiscoveryUnitTests.java index 4cdb523ec3f92..6c56eb01797dc 100644 --- a/core/src/test/java/org/elasticsearch/discovery/zen/ZenDiscoveryUnitTests.java +++ b/core/src/test/java/org/elasticsearch/discovery/zen/ZenDiscoveryUnitTests.java @@ -50,9 +50,9 @@ public void testShouldIgnoreNewClusterState() { ClusterName clusterName = new ClusterName("abc"); DiscoveryNodes.Builder currentNodes = DiscoveryNodes.builder(); - currentNodes.masterNodeId("a").put(new DiscoveryNode("a", "a", LocalTransportAddress.buildUnique(), emptyMap(), emptySet(), Version.CURRENT)); + currentNodes.masterNodeId("a").put(new DiscoveryNode("a", LocalTransportAddress.buildUnique(), emptyMap(), emptySet(), Version.CURRENT)); DiscoveryNodes.Builder newNodes = DiscoveryNodes.builder(); - newNodes.masterNodeId("a").put(new DiscoveryNode("a", "a", LocalTransportAddress.buildUnique(), emptyMap(), emptySet(), Version.CURRENT)); + newNodes.masterNodeId("a").put(new DiscoveryNode("a", LocalTransportAddress.buildUnique(), emptyMap(), emptySet(), Version.CURRENT)); ClusterState.Builder currentState = ClusterState.builder(clusterName); currentState.nodes(currentNodes); @@ -70,7 +70,7 @@ public void testShouldIgnoreNewClusterState() { assertFalse("should not ignore, because new state's version is higher to current state's version", shouldIgnoreOrRejectNewClusterState(logger, currentState.build(), newState.build())); currentNodes = DiscoveryNodes.builder(); - currentNodes.masterNodeId("b").put(new DiscoveryNode("b", "b", LocalTransportAddress.buildUnique(), emptyMap(), emptySet(), Version.CURRENT)); + currentNodes.masterNodeId("b").put(new DiscoveryNode("b", LocalTransportAddress.buildUnique(), emptyMap(), emptySet(), Version.CURRENT)); ; // version isn't taken into account, so randomize it to ensure this. if (randomBoolean()) { @@ -108,8 +108,8 @@ public void testFilterNonMasterPingResponse() { ArrayList allNodes = new ArrayList<>(); for (int i = randomIntBetween(10, 20); i >= 0; i--) { Set roles = new HashSet<>(randomSubsetOf(Arrays.asList(DiscoveryNode.Role.values()))); - DiscoveryNode node = new DiscoveryNode("node_" + i, "id_" + i, LocalTransportAddress.buildUnique(), Collections.emptyMap(), - roles, Version.CURRENT); + DiscoveryNode node = new DiscoveryNode("name_" + i, "node_" + i, "id_" + i, LocalTransportAddress.buildUnique(), + Collections.emptyMap(), roles, Version.CURRENT); responses.add(new ZenPing.PingResponse(node, randomBoolean() ? null : node, new ClusterName("test"), randomBoolean())); allNodes.add(node); if (node.isMasterNode()) { diff --git a/core/src/test/java/org/elasticsearch/gateway/AsyncShardFetchTests.java b/core/src/test/java/org/elasticsearch/gateway/AsyncShardFetchTests.java index 00ca8d4c80472..2b8cdf46f7d34 100644 --- a/core/src/test/java/org/elasticsearch/gateway/AsyncShardFetchTests.java +++ b/core/src/test/java/org/elasticsearch/gateway/AsyncShardFetchTests.java @@ -44,11 +44,11 @@ /** */ public class AsyncShardFetchTests extends ESTestCase { - private final DiscoveryNode node1 = new DiscoveryNode("node1", "process1", LocalTransportAddress.buildUnique(), Collections.emptyMap(), + private final DiscoveryNode node1 = new DiscoveryNode("node1", LocalTransportAddress.buildUnique(), Collections.emptyMap(), Collections.singleton(DiscoveryNode.Role.DATA), Version.CURRENT); private final Response response1 = new Response(node1); private final Throwable failure1 = new Throwable("simulated failure 1"); - private final DiscoveryNode node2 = new DiscoveryNode("node2", "process2", LocalTransportAddress.buildUnique(), Collections.emptyMap(), + private final DiscoveryNode node2 = new DiscoveryNode("node2", LocalTransportAddress.buildUnique(), Collections.emptyMap(), Collections.singleton(DiscoveryNode.Role.DATA), Version.CURRENT); private final Response response2 = new Response(node2); private final Throwable failure2 = new Throwable("simulate failure 2"); diff --git a/core/src/test/java/org/elasticsearch/transport/AbstractSimpleTransportTestCase.java b/core/src/test/java/org/elasticsearch/transport/AbstractSimpleTransportTestCase.java index edeecd61d8eee..8a8b8e53d6da2 100644 --- a/core/src/test/java/org/elasticsearch/transport/AbstractSimpleTransportTestCase.java +++ b/core/src/test/java/org/elasticsearch/transport/AbstractSimpleTransportTestCase.java @@ -1308,7 +1308,7 @@ version0, new NamedWriteableRegistry() }); DiscoveryNode node = - new DiscoveryNode("TS_TEST", "TS_TEST", service.boundAddress().publishAddress(), emptyMap(), emptySet(), version0); + new DiscoveryNode("TS_TEST", service.boundAddress().publishAddress(), emptyMap(), emptySet(), version0); serviceA.connectToNode(node); CountDownLatch latch = new CountDownLatch(1); diff --git a/core/src/test/java/org/elasticsearch/transport/netty/NettyScheduledPingTests.java b/core/src/test/java/org/elasticsearch/transport/netty/NettyScheduledPingTests.java index 49f86b909a617..98039717e924c 100644 --- a/core/src/test/java/org/elasticsearch/transport/netty/NettyScheduledPingTests.java +++ b/core/src/test/java/org/elasticsearch/transport/netty/NettyScheduledPingTests.java @@ -75,9 +75,9 @@ public void testScheduledPing() throws Exception { serviceB.acceptIncomingRequests(); DiscoveryNode nodeA = - new DiscoveryNode("TS_A", "TS_A", serviceA.boundAddress().publishAddress(), emptyMap(), emptySet(), Version.CURRENT); + new DiscoveryNode("TS_A", serviceA.boundAddress().publishAddress(), emptyMap(), emptySet(), Version.CURRENT); DiscoveryNode nodeB = - new DiscoveryNode("TS_B", "TS_B", serviceB.boundAddress().publishAddress(), emptyMap(), emptySet(), Version.CURRENT); + new DiscoveryNode("TS_B", serviceB.boundAddress().publishAddress(), emptyMap(), emptySet(), Version.CURRENT); serviceA.connectToNode(nodeB); serviceB.connectToNode(nodeA); diff --git a/test/framework/src/main/java/org/elasticsearch/test/ESAllocationTestCase.java b/test/framework/src/main/java/org/elasticsearch/test/ESAllocationTestCase.java index 9b1202e8c2418..f72ebe454f6ae 100644 --- a/test/framework/src/main/java/org/elasticsearch/test/ESAllocationTestCase.java +++ b/test/framework/src/main/java/org/elasticsearch/test/ESAllocationTestCase.java @@ -136,15 +136,15 @@ protected static DiscoveryNode newNode(String nodeName, String nodeId, Map attributes) { - return new DiscoveryNode(nodeId, nodeId, LocalTransportAddress.buildUnique(), attributes, MASTER_DATA_ROLES, Version.CURRENT); + return new DiscoveryNode(nodeId, LocalTransportAddress.buildUnique(), attributes, MASTER_DATA_ROLES, Version.CURRENT); } protected static DiscoveryNode newNode(String nodeId, Set roles) { - return new DiscoveryNode(nodeId, nodeId, LocalTransportAddress.buildUnique(), emptyMap(), roles, Version.CURRENT); + return new DiscoveryNode(nodeId, LocalTransportAddress.buildUnique(), emptyMap(), roles, Version.CURRENT); } protected static DiscoveryNode newNode(String nodeId, Version version) { - return new DiscoveryNode(nodeId, nodeId, LocalTransportAddress.buildUnique(), emptyMap(), MASTER_DATA_ROLES, version); + return new DiscoveryNode(nodeId, LocalTransportAddress.buildUnique(), emptyMap(), MASTER_DATA_ROLES, version); } protected static ClusterState startRandomInitializingShard(ClusterState clusterState, AllocationService strategy) {