Browse Source

Add elasticsearch-node detach-cluster command (#37979)

This commit adds the second part of `elasticsearch-node` tool -
`detach-cluster` command in addition to `unsafe-bootstrap` command.
Also, this commit changes the semantics of `unsafe-bootstrap`, now
`unsafe-bootstrap` changes clusterUUID.
So the algorithm of running `elasticsearch-node` tool is the following:
1) Stop all nodes in the cluster.
2) Pick master-eligible node with the highest (term, version) pair and
run the `unsafe-bootstrap` command on it. If there are no survived
master-eligible nodes - skip this step.
3) Run `detach-cluster` command on the remaining survived nodes.

Detach cluster makes the following changes to the node metadata:
1) Sets clusterUUID committed to false.
2) Sets currentTerm and term to 0. 
3) Removes voting tombstones and sets voting configurations to special
constant MUST_JOIN_ELECTED_MASTER, that prevents initial cluster
bootstrap.

`ElasticsearchNodeCommand` base abstract class is introduced, because
`UnsafeBootstrapMasterCommand` and `DetachClusterCommand` have a lot in
common.
Also, this commit adds "ordinal" parameter to both commands, because it's 
impossible to write IT otherwise.
For MUST_JOIN_ELECTED_MASTER case special handling is introduced in
`ClusterFormationFailureHelper`.
Tests for both commands reside in `ElasticsearchNodeCommandIT` (renamed
from `UnsafeBootstrapMasterIT`).
Andrey Ershov 6 years ago
parent
commit
bda591453c

+ 6 - 0
server/src/main/java/org/elasticsearch/cluster/coordination/ClusterFormationFailureHelper.java

@@ -167,6 +167,12 @@ public class ClusterFormationFailureHelper {
 
             assert clusterState.getLastCommittedConfiguration().isEmpty() == false;
 
+            if (clusterState.getLastCommittedConfiguration().equals(VotingConfiguration.MUST_JOIN_ELECTED_MASTER)) {
+                return String.format(Locale.ROOT,
+                        "master not discovered yet and this node was detached from its previous cluster, have discovered %s; %s",
+                        foundPeers, discoveryWillContinueDescription);
+            }
+
             final String quorumDescription;
             if (clusterState.getLastAcceptedConfiguration().equals(clusterState.getLastCommittedConfiguration())) {
                 quorumDescription = describeQuorum(clusterState.getLastAcceptedConfiguration());

+ 2 - 0
server/src/main/java/org/elasticsearch/cluster/coordination/CoordinationMetaData.java

@@ -325,6 +325,8 @@ public class CoordinationMetaData implements Writeable, ToXContentFragment {
     public static class VotingConfiguration implements Writeable, ToXContentFragment {
 
         public static final VotingConfiguration EMPTY_CONFIG = new VotingConfiguration(Collections.emptySet());
+        public static final VotingConfiguration MUST_JOIN_ELECTED_MASTER = new VotingConfiguration(Collections.singleton(
+                "_must_join_elected_master_"));
 
         private final Set<String> nodeIds;
 

+ 85 - 0
server/src/main/java/org/elasticsearch/cluster/coordination/DetachClusterCommand.java

@@ -0,0 +1,85 @@
+/*
+ * 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.coordination;
+
+import joptsimple.OptionSet;
+import org.elasticsearch.cli.Terminal;
+import org.elasticsearch.cluster.metadata.Manifest;
+import org.elasticsearch.cluster.metadata.MetaData;
+import org.elasticsearch.common.collect.Tuple;
+import org.elasticsearch.env.Environment;
+
+import java.io.IOException;
+import java.nio.file.Path;
+
+public class DetachClusterCommand extends ElasticsearchNodeCommand {
+
+    static final String NODE_DETACHED_MSG = "Node was successfully detached from the cluster";
+    static final String CONFIRMATION_MSG =
+                    "-------------------------------------------------------------------------------\n" +
+                    "\n" +
+                    "You should run this tool only if you have permanently lost all\n" +
+                    "your master-eligible nodes, and you cannot restore the cluster\n" +
+                    "from a snapshot, or you have already run `elasticsearch-node unsafe-bootstrap`\n" +
+                    "on a master-eligible node that formed a cluster with this node.\n" +
+                    "This tool can cause arbitrary data loss and its use should be your last resort.\n" +
+                    "Do you want to proceed?\n";
+
+    public DetachClusterCommand() {
+        super("Detaches this node from its cluster, allowing it to unsafely join a new cluster");
+    }
+
+    @Override
+    protected void execute(Terminal terminal, OptionSet options, Environment env) throws Exception {
+        super.execute(terminal, options, env);
+
+        processNodePathsWithLock(terminal, options, env);
+
+        terminal.println(NODE_DETACHED_MSG);
+    }
+
+    @Override
+    protected void processNodePaths(Terminal terminal, Path[] dataPaths) throws IOException {
+        final Tuple<Manifest, MetaData> manifestMetaDataTuple = loadMetaData(terminal, dataPaths);
+        final Manifest manifest = manifestMetaDataTuple.v1();
+        final MetaData metaData = manifestMetaDataTuple.v2();
+
+        confirm(terminal, CONFIRMATION_MSG);
+
+        writeNewMetaData(terminal, manifest, updateCurrentTerm(), metaData, updateMetaData(metaData), dataPaths);
+    }
+
+    // package-private for tests
+    static MetaData updateMetaData(MetaData oldMetaData) {
+        final CoordinationMetaData coordinationMetaData = CoordinationMetaData.builder()
+                .lastAcceptedConfiguration(CoordinationMetaData.VotingConfiguration.MUST_JOIN_ELECTED_MASTER)
+                .lastCommittedConfiguration(CoordinationMetaData.VotingConfiguration.MUST_JOIN_ELECTED_MASTER)
+                .term(0)
+                .build();
+        return MetaData.builder(oldMetaData)
+                .coordinationMetaData(coordinationMetaData)
+                .clusterUUIDCommitted(false)
+                .build();
+    }
+
+    //package-private for tests
+    static long updateCurrentTerm() {
+        return 0;
+    }
+}

+ 151 - 0
server/src/main/java/org/elasticsearch/cluster/coordination/ElasticsearchNodeCommand.java

@@ -0,0 +1,151 @@
+/*
+ * 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.coordination;
+
+import joptsimple.OptionParser;
+import joptsimple.OptionSet;
+import joptsimple.OptionSpec;
+import org.apache.logging.log4j.LogManager;
+import org.apache.logging.log4j.Logger;
+import org.apache.lucene.store.LockObtainFailedException;
+import org.elasticsearch.ElasticsearchException;
+import org.elasticsearch.cli.EnvironmentAwareCommand;
+import org.elasticsearch.cli.Terminal;
+import org.elasticsearch.cluster.ClusterModule;
+import org.elasticsearch.cluster.metadata.Manifest;
+import org.elasticsearch.cluster.metadata.MetaData;
+import org.elasticsearch.common.collect.Tuple;
+import org.elasticsearch.common.xcontent.NamedXContentRegistry;
+import org.elasticsearch.env.Environment;
+import org.elasticsearch.env.NodeEnvironment;
+
+import java.io.IOException;
+import java.nio.file.Files;
+import java.nio.file.Path;
+import java.util.Arrays;
+import java.util.Objects;
+
+public abstract class ElasticsearchNodeCommand extends EnvironmentAwareCommand {
+    private static final Logger logger = LogManager.getLogger(ElasticsearchNodeCommand.class);
+    protected final NamedXContentRegistry namedXContentRegistry;
+    static final String STOP_WARNING_MSG =
+            "--------------------------------------------------------------------------\n" +
+                    "\n" +
+                    "    WARNING: Elasticsearch MUST be stopped before running this tool." +
+                    "\n";
+    static final String FAILED_TO_OBTAIN_NODE_LOCK_MSG = "failed to lock node's directory, is Elasticsearch still running?";
+    static final String NO_NODE_FOLDER_FOUND_MSG = "no node folder is found in data folder(s), node has not been started yet?";
+    static final String NO_MANIFEST_FILE_FOUND_MSG = "no manifest file is found, do you run pre 7.0 Elasticsearch?";
+    static final String GLOBAL_GENERATION_MISSING_MSG = "no metadata is referenced from the manifest file, cluster has never been " +
+            "bootstrapped?";
+    static final String NO_GLOBAL_METADATA_MSG = "failed to find global metadata, metadata corrupted?";
+    static final String WRITE_METADATA_EXCEPTION_MSG = "exception occurred when writing new metadata to disk";
+    static final String ABORTED_BY_USER_MSG = "aborted by user";
+    final OptionSpec<Integer> nodeOrdinalOption;
+
+    public ElasticsearchNodeCommand(String description) {
+        super(description);
+        nodeOrdinalOption = parser.accepts("ordinal", "Optional node ordinal, 0 if not specified")
+                .withRequiredArg().ofType(Integer.class);
+        namedXContentRegistry = new NamedXContentRegistry(ClusterModule.getNamedXWriteables());
+    }
+
+    protected void processNodePathsWithLock(Terminal terminal, OptionSet options, Environment env) throws IOException {
+        terminal.println(Terminal.Verbosity.VERBOSE, "Obtaining lock for node");
+        Integer nodeOrdinal = nodeOrdinalOption.value(options);
+        if (nodeOrdinal == null) {
+            nodeOrdinal = 0;
+        }
+        try (NodeEnvironment.NodeLock lock = new NodeEnvironment.NodeLock(nodeOrdinal, logger, env, Files::exists)) {
+            final Path[] dataPaths =
+                    Arrays.stream(lock.getNodePaths()).filter(Objects::nonNull).map(p -> p.path).toArray(Path[]::new);
+            if (dataPaths.length == 0) {
+                throw new ElasticsearchException(NO_NODE_FOLDER_FOUND_MSG);
+            }
+            processNodePaths(terminal, dataPaths);
+        } catch (LockObtainFailedException ex) {
+            throw new ElasticsearchException(
+                    FAILED_TO_OBTAIN_NODE_LOCK_MSG + " [" + ex.getMessage() + "]");
+        }
+    }
+
+    protected Tuple<Manifest, MetaData> loadMetaData(Terminal terminal, Path[] dataPaths) throws IOException {
+        terminal.println(Terminal.Verbosity.VERBOSE, "Loading manifest file");
+        final Manifest manifest = Manifest.FORMAT.loadLatestState(logger, namedXContentRegistry, dataPaths);
+
+        if (manifest == null) {
+            throw new ElasticsearchException(NO_MANIFEST_FILE_FOUND_MSG);
+        }
+        if (manifest.isGlobalGenerationMissing()) {
+            throw new ElasticsearchException(GLOBAL_GENERATION_MISSING_MSG);
+        }
+        terminal.println(Terminal.Verbosity.VERBOSE, "Loading global metadata file");
+        final MetaData metaData = MetaData.FORMAT.loadGeneration(logger, namedXContentRegistry, manifest.getGlobalGeneration(),
+                dataPaths);
+        if (metaData == null) {
+            throw new ElasticsearchException(NO_GLOBAL_METADATA_MSG + " [generation = " + manifest.getGlobalGeneration() + "]");
+        }
+
+        return Tuple.tuple(manifest, metaData);
+    }
+
+    protected void confirm(Terminal terminal, String msg) {
+        terminal.println(msg);
+        String text = terminal.readText("Confirm [y/N] ");
+        if (text.equalsIgnoreCase("y") == false) {
+            throw new ElasticsearchException(ABORTED_BY_USER_MSG);
+        }
+    }
+
+    @Override
+    protected void execute(Terminal terminal, OptionSet options, Environment env) throws Exception {
+        terminal.println(STOP_WARNING_MSG);
+    }
+
+    protected abstract void processNodePaths(Terminal terminal, Path[] dataPaths) throws IOException;
+
+
+    protected void writeNewMetaData(Terminal terminal, Manifest oldManifest, long newCurrentTerm,
+                                    MetaData oldMetaData, MetaData newMetaData, Path[] dataPaths) {
+        try {
+            terminal.println(Terminal.Verbosity.VERBOSE,
+                    "[clusterUUID = " + oldMetaData.clusterUUID() + ", committed = " + oldMetaData.clusterUUIDCommitted() + "] => " +
+                         "[clusterUUID = " + newMetaData.clusterUUID() + ", committed = " + newMetaData.clusterUUIDCommitted() + "]");
+            terminal.println(Terminal.Verbosity.VERBOSE, "New coordination metadata is " + newMetaData.coordinationMetaData());
+            terminal.println(Terminal.Verbosity.VERBOSE, "Writing new global metadata to disk");
+            long newGeneration = MetaData.FORMAT.write(newMetaData, dataPaths);
+            Manifest newManifest = new Manifest(newCurrentTerm, oldManifest.getClusterStateVersion(), newGeneration,
+                    oldManifest.getIndexGenerations());
+            terminal.println(Terminal.Verbosity.VERBOSE, "New manifest is " + newManifest);
+            terminal.println(Terminal.Verbosity.VERBOSE, "Writing new manifest file to disk");
+            Manifest.FORMAT.writeAndCleanup(newManifest, dataPaths);
+            terminal.println(Terminal.Verbosity.VERBOSE, "Cleaning up old metadata");
+            MetaData.FORMAT.cleanupOldFiles(newGeneration, dataPaths);
+        } catch (Exception e) {
+            terminal.println(Terminal.Verbosity.VERBOSE, "Cleaning up new metadata");
+            MetaData.FORMAT.cleanupOldFiles(oldManifest.getGlobalGeneration(), dataPaths);
+            throw new ElasticsearchException(WRITE_METADATA_EXCEPTION_MSG, e);
+        }
+    }
+
+    //package-private for testing
+    OptionParser getParser() {
+        return parser;
+    }
+}

+ 1 - 0
server/src/main/java/org/elasticsearch/cluster/coordination/NodeToolCli.java

@@ -35,6 +35,7 @@ public class NodeToolCli extends MultiCommand {
         super("A CLI tool to unsafely recover a cluster after the permanent loss of too many master-eligible nodes", ()->{});
         CommandLoggingConfigurator.configureLoggingWithoutConfig();
         subcommands.put("unsafe-bootstrap", new UnsafeBootstrapMasterCommand());
+        subcommands.put("detach-cluster", new DetachClusterCommand());
     }
 
     public static void main(String[] args) throws Exception {

+ 19 - 82
server/src/main/java/org/elasticsearch/cluster/coordination/UnsafeBootstrapMasterCommand.java

@@ -21,40 +21,27 @@ package org.elasticsearch.cluster.coordination;
 import joptsimple.OptionSet;
 import org.apache.logging.log4j.LogManager;
 import org.apache.logging.log4j.Logger;
-import org.apache.lucene.store.LockObtainFailedException;
 import org.elasticsearch.ElasticsearchException;
-import org.elasticsearch.cli.EnvironmentAwareCommand;
 import org.elasticsearch.cli.Terminal;
-import org.elasticsearch.cluster.ClusterModule;
 import org.elasticsearch.cluster.metadata.Manifest;
 import org.elasticsearch.cluster.metadata.MetaData;
 import org.elasticsearch.cluster.service.ClusterService;
+import org.elasticsearch.common.collect.Tuple;
 import org.elasticsearch.common.settings.Setting;
 import org.elasticsearch.common.settings.Settings;
-import org.elasticsearch.common.xcontent.NamedXContentRegistry;
 import org.elasticsearch.env.Environment;
-import org.elasticsearch.env.NodeEnvironment;
 import org.elasticsearch.env.NodeMetaData;
 import org.elasticsearch.node.Node;
 
 import java.io.IOException;
-import java.nio.file.Files;
 import java.nio.file.Path;
-import java.util.Arrays;
 import java.util.Collections;
 import java.util.Locale;
-import java.util.Objects;
 
-public class UnsafeBootstrapMasterCommand extends EnvironmentAwareCommand {
+public class UnsafeBootstrapMasterCommand extends ElasticsearchNodeCommand {
 
     private static final Logger logger = LogManager.getLogger(UnsafeBootstrapMasterCommand.class);
-    private final NamedXContentRegistry namedXContentRegistry;
 
-    static final String STOP_WARNING_MSG =
-            "--------------------------------------------------------------------------\n" +
-            "\n" +
-            "    WARNING: Elasticsearch MUST be stopped before running this tool." +
-            "\n";
     static final String CLUSTER_STATE_TERM_VERSION_MSG_FORMAT =
             "Current node cluster state (term, version) pair is (%s, %s)";
     static final String CONFIRMATION_MSG =
@@ -62,35 +49,29 @@ public class UnsafeBootstrapMasterCommand extends EnvironmentAwareCommand {
             "\n" +
             "You should run this tool only if you have permanently lost half\n" +
             "or more of the master-eligible nodes, and you cannot restore the cluster\n" +
-            "from a snapshot. This tool can result in arbitrary data loss and\n" +
-            "should be the last resort.\n" +
+            "from a snapshot. This tool can cause arbitrary data loss and its use " +
+            "should be your last resort.\n" +
             "If you have multiple survived master eligible nodes, consider running\n" +
             "this tool on the node with the highest cluster state (term, version) pair.\n" +
             "Do you want to proceed?\n";
-    static final String ABORTED_BY_USER_MSG = "aborted by user";
     static final String NOT_MASTER_NODE_MSG = "unsafe-bootstrap tool can only be run on master eligible node";
-    static final String FAILED_TO_OBTAIN_NODE_LOCK_MSG = "failed to lock node's directory, is Elasticsearch still running?";
-    static final String NO_NODE_FOLDER_FOUND_MSG = "no node folder is found in data folder(s), node has not been started yet?";
+
     static final String NO_NODE_METADATA_FOUND_MSG = "no node meta data is found, node has not been started yet?";
-    static final String NO_MANIFEST_FILE_FOUND_MSG = "no manifest file is found, do you run pre 7.0 Elasticsearch?";
-    static final String GLOBAL_GENERATION_MISSING_MSG = "no metadata is referenced from the manifest file, cluster has never been " +
-            "bootstrapped?";
-    static final String NO_GLOBAL_METADATA_MSG = "failed to find global metadata, metadata corrupted?";
+
     static final String EMPTY_LAST_COMMITTED_VOTING_CONFIG_MSG =
             "last committed voting voting configuration is empty, cluster has never been bootstrapped?";
-    static final String WRITE_METADATA_EXCEPTION_MSG = "exception occurred when writing new metadata to disk";
+
     static final String MASTER_NODE_BOOTSTRAPPED_MSG = "Master node was successfully bootstrapped";
     static final Setting<String> UNSAFE_BOOTSTRAP =
             ClusterService.USER_DEFINED_META_DATA.getConcreteSetting("cluster.metadata.unsafe-bootstrap");
 
     UnsafeBootstrapMasterCommand() {
         super("Forces the successful election of the current node after the permanent loss of the half or more master-eligible nodes");
-        namedXContentRegistry = new NamedXContentRegistry(ClusterModule.getNamedXWriteables());
     }
 
     @Override
     protected void execute(Terminal terminal, OptionSet options, Environment env) throws Exception {
-        terminal.println(STOP_WARNING_MSG);
+        super.execute(terminal, options, env);
 
         Settings settings = env.settings();
         terminal.println(Terminal.Verbosity.VERBOSE, "Checking node.master setting");
@@ -98,27 +79,13 @@ public class UnsafeBootstrapMasterCommand extends EnvironmentAwareCommand {
         if (master == false) {
             throw new ElasticsearchException(NOT_MASTER_NODE_MSG);
         }
-        final int nodeOrdinal = 0;
-
-        terminal.println(Terminal.Verbosity.VERBOSE, "Obtaining lock for node");
 
-        try (NodeEnvironment.NodeLock lock = new NodeEnvironment.NodeLock(nodeOrdinal, logger, env, Files::exists)) {
-            processNodePaths(logger, terminal, lock.getNodePaths());
-        } catch (LockObtainFailedException ex) {
-            throw new ElasticsearchException(
-                    FAILED_TO_OBTAIN_NODE_LOCK_MSG + " [" + ex.getMessage() + "]");
-        }
+        processNodePathsWithLock(terminal, options, env);
 
         terminal.println(MASTER_NODE_BOOTSTRAPPED_MSG);
     }
 
-    private void processNodePaths(Logger logger, Terminal terminal, NodeEnvironment.NodePath[] nodePaths) throws IOException {
-        final Path[] dataPaths =
-                Arrays.stream(nodePaths).filter(Objects::nonNull).map(p -> p.path).toArray(Path[]::new);
-        if (dataPaths.length == 0) {
-            throw new ElasticsearchException(NO_NODE_FOLDER_FOUND_MSG);
-        }
-
+    protected void processNodePaths(Terminal terminal, Path[] dataPaths) throws IOException {
         terminal.println(Terminal.Verbosity.VERBOSE, "Loading node metadata");
         final NodeMetaData nodeMetaData = NodeMetaData.FORMAT.loadLatestState(logger, namedXContentRegistry, dataPaths);
         if (nodeMetaData == null) {
@@ -127,21 +94,10 @@ public class UnsafeBootstrapMasterCommand extends EnvironmentAwareCommand {
 
         String nodeId = nodeMetaData.nodeId();
         terminal.println(Terminal.Verbosity.VERBOSE, "Current nodeId is " + nodeId);
-        terminal.println(Terminal.Verbosity.VERBOSE, "Loading manifest file");
-        final Manifest manifest = Manifest.FORMAT.loadLatestState(logger, namedXContentRegistry, dataPaths);
 
-        if (manifest == null) {
-            throw new ElasticsearchException(NO_MANIFEST_FILE_FOUND_MSG);
-        }
-        if (manifest.isGlobalGenerationMissing()) {
-            throw new ElasticsearchException(GLOBAL_GENERATION_MISSING_MSG);
-        }
-        terminal.println(Terminal.Verbosity.VERBOSE, "Loading global metadata file");
-        final MetaData metaData = MetaData.FORMAT.loadGeneration(logger, namedXContentRegistry, manifest.getGlobalGeneration(),
-                dataPaths);
-        if (metaData == null) {
-            throw new ElasticsearchException(NO_GLOBAL_METADATA_MSG + " [generation = " + manifest.getGlobalGeneration() + "]");
-        }
+        final Tuple<Manifest, MetaData> manifestMetaDataTuple = loadMetaData(terminal, dataPaths);
+        final Manifest manifest = manifestMetaDataTuple.v1();
+        final MetaData metaData = manifestMetaDataTuple.v2();
         final CoordinationMetaData coordinationMetaData = metaData.coordinationMetaData();
         if (coordinationMetaData == null ||
                 coordinationMetaData.getLastCommittedConfiguration() == null ||
@@ -151,45 +107,26 @@ public class UnsafeBootstrapMasterCommand extends EnvironmentAwareCommand {
         terminal.println(String.format(Locale.ROOT, CLUSTER_STATE_TERM_VERSION_MSG_FORMAT, coordinationMetaData.term(),
                 metaData.version()));
 
-        terminal.println(CONFIRMATION_MSG);
-        String text = terminal.readText("Confirm [y/N] ");
-        if (text.equalsIgnoreCase("y") == false) {
-            throw new ElasticsearchException(ABORTED_BY_USER_MSG);
-        }
+        confirm(terminal, CONFIRMATION_MSG);
 
         CoordinationMetaData newCoordinationMetaData = CoordinationMetaData.builder(coordinationMetaData)
                 .clearVotingConfigExclusions()
                 .lastAcceptedConfiguration(new CoordinationMetaData.VotingConfiguration(Collections.singleton(nodeId)))
                 .lastCommittedConfiguration(new CoordinationMetaData.VotingConfiguration(Collections.singleton(nodeId)))
                 .build();
-        terminal.println(Terminal.Verbosity.VERBOSE, "New coordination metadata is constructed " + newCoordinationMetaData);
+
         Settings persistentSettings = Settings.builder()
                 .put(metaData.persistentSettings())
                 .put(UNSAFE_BOOTSTRAP.getKey(), true)
                 .build();
         MetaData newMetaData = MetaData.builder(metaData)
+                .clusterUUID(MetaData.UNKNOWN_CLUSTER_UUID)
+                .generateClusterUuidIfNeeded()
+                .clusterUUIDCommitted(true)
                 .persistentSettings(persistentSettings)
                 .coordinationMetaData(newCoordinationMetaData)
                 .build();
-        writeNewMetaData(terminal, manifest, newMetaData, dataPaths);
-    }
 
-    private void writeNewMetaData(Terminal terminal, Manifest manifest, MetaData newMetaData, Path[] dataPaths) {
-        try {
-            terminal.println(Terminal.Verbosity.VERBOSE, "Writing new global metadata to disk");
-            long newGeneration = MetaData.FORMAT.write(newMetaData, dataPaths);
-            long newCurrentTerm = manifest.getCurrentTerm() + 1;
-            terminal.println(Terminal.Verbosity.VERBOSE, "Incrementing currentTerm. New value is " + newCurrentTerm);
-            Manifest newManifest = new Manifest(newCurrentTerm, manifest.getClusterStateVersion(), newGeneration,
-                    manifest.getIndexGenerations());
-            terminal.println(Terminal.Verbosity.VERBOSE, "Writing new manifest file to disk");
-            Manifest.FORMAT.writeAndCleanup(newManifest, dataPaths);
-            terminal.println(Terminal.Verbosity.VERBOSE, "Cleaning up old metadata");
-            MetaData.FORMAT.cleanupOldFiles(newGeneration, dataPaths);
-        } catch (Exception e) {
-            terminal.println(Terminal.Verbosity.VERBOSE, "Cleaning up new metadata");
-            MetaData.FORMAT.cleanupOldFiles(manifest.getGlobalGeneration(), dataPaths);
-            throw new ElasticsearchException(WRITE_METADATA_EXCEPTION_MSG, e);
-        }
+        writeNewMetaData(terminal, manifest, manifest.getCurrentTerm(), metaData, newMetaData, dataPaths);
     }
 }

+ 36 - 0
server/src/test/java/org/elasticsearch/cluster/coordination/ClusterFormationFailureHelperTests.java

@@ -201,6 +201,42 @@ public class ClusterFormationFailureHelperTests extends ESTestCase {
                 .lastCommittedConfiguration(config(committedConfig)).build())).build();
     }
 
+
+    public void testDescriptionAfterDetachCluster() {
+        final DiscoveryNode localNode = new DiscoveryNode("local", buildNewFakeTransportAddress(), Version.CURRENT);
+
+        final ClusterState clusterState = state(localNode,
+                VotingConfiguration.MUST_JOIN_ELECTED_MASTER.getNodeIds().toArray(new String[0]));
+
+        assertThat(new ClusterFormationState(Settings.EMPTY, clusterState, emptyList(), emptyList(), 0L).getDescription(),
+                is("master not discovered yet and this node was detached from its previous cluster, " +
+                        "have discovered []; " +
+                        "discovery will continue using [] from hosts providers and [" + localNode +
+                        "] from last-known cluster state; node term 0, last-accepted version 0 in term 0"));
+
+        final TransportAddress otherAddress = buildNewFakeTransportAddress();
+        assertThat(new ClusterFormationState(Settings.EMPTY, clusterState, singletonList(otherAddress), emptyList(), 0L).getDescription(),
+                is("master not discovered yet and this node was detached from its previous cluster, " +
+                        "have discovered []; " +
+                        "discovery will continue using [" + otherAddress + "] from hosts providers and [" + localNode +
+                        "] from last-known cluster state; node term 0, last-accepted version 0 in term 0"));
+
+        final DiscoveryNode otherNode = new DiscoveryNode("otherNode", buildNewFakeTransportAddress(), Version.CURRENT);
+        assertThat(new ClusterFormationState(Settings.EMPTY, clusterState, emptyList(), singletonList(otherNode), 0L).getDescription(),
+                is("master not discovered yet and this node was detached from its previous cluster, " +
+                        "have discovered [" + otherNode + "]; " +
+                        "discovery will continue using [] from hosts providers and [" + localNode +
+                        "] from last-known cluster state; node term 0, last-accepted version 0 in term 0"));
+
+        final DiscoveryNode yetAnotherNode = new DiscoveryNode("yetAnotherNode", buildNewFakeTransportAddress(), Version.CURRENT);
+        assertThat(new ClusterFormationState(Settings.EMPTY, clusterState, emptyList(), singletonList(yetAnotherNode), 0L).getDescription(),
+                is("master not discovered yet and this node was detached from its previous cluster, " +
+                        "have discovered [" + yetAnotherNode + "]; " +
+                        "discovery will continue using [] from hosts providers and [" + localNode +
+                        "] from last-known cluster state; node term 0, last-accepted version 0 in term 0"));
+
+    }
+
     public void testDescriptionAfterBootstrapping() {
         final DiscoveryNode localNode = new DiscoveryNode("local", buildNewFakeTransportAddress(), Version.CURRENT);
 

+ 2 - 8
server/src/test/java/org/elasticsearch/cluster/coordination/CoordinatorTests.java

@@ -1048,15 +1048,9 @@ public class CoordinatorTests extends ESTestCase {
         }
         assertTrue(newNode.getLastAppliedClusterState().version() == 0);
 
-        // reset clusterUUIDCommitted (and node / cluster state term) to let node join again
-        // TODO: use elasticsearch-node detach-cluster tool once it's implemented
         final ClusterNode detachedNode = newNode.restartedNode(
-            metaData -> MetaData.builder(metaData)
-                .clusterUUIDCommitted(false)
-                .coordinationMetaData(CoordinationMetaData.builder(metaData.coordinationMetaData())
-                    .term(0L).build())
-                .build(),
-            term -> 0L);
+            metaData -> DetachClusterCommand.updateMetaData(metaData),
+            term -> DetachClusterCommand.updateCurrentTerm());
         cluster1.clusterNodes.replaceAll(cn -> cn == newNode ? detachedNode : cn);
         cluster1.stabilise();
     }

+ 418 - 0
server/src/test/java/org/elasticsearch/cluster/coordination/ElasticsearchNodeCommandIT.java

@@ -0,0 +1,418 @@
+/*
+ * 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.coordination;
+
+import joptsimple.OptionSet;
+import org.elasticsearch.ElasticsearchException;
+import org.elasticsearch.action.admin.cluster.settings.ClusterUpdateSettingsRequest;
+import org.elasticsearch.cli.MockTerminal;
+import org.elasticsearch.cluster.ClusterState;
+import org.elasticsearch.cluster.metadata.Manifest;
+import org.elasticsearch.cluster.metadata.MetaData;
+import org.elasticsearch.common.settings.Settings;
+import org.elasticsearch.discovery.DiscoverySettings;
+import org.elasticsearch.env.Environment;
+import org.elasticsearch.env.NodeEnvironment;
+import org.elasticsearch.env.NodeMetaData;
+import org.elasticsearch.env.TestEnvironment;
+import org.elasticsearch.node.Node;
+import org.elasticsearch.test.ESIntegTestCase;
+import org.elasticsearch.test.InternalTestCluster;
+import org.elasticsearch.test.junit.annotations.TestLogging;
+
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.List;
+import java.util.Locale;
+
+import static org.elasticsearch.action.support.WriteRequest.RefreshPolicy.IMMEDIATE;
+import static org.elasticsearch.index.query.QueryBuilders.matchAllQuery;
+import static org.elasticsearch.indices.recovery.RecoverySettings.INDICES_RECOVERY_MAX_BYTES_PER_SEC_SETTING;
+import static org.elasticsearch.test.hamcrest.ElasticsearchAssertions.assertHitCount;
+import static org.hamcrest.Matchers.containsString;
+import static org.hamcrest.Matchers.equalTo;
+
+@ESIntegTestCase.ClusterScope(scope = ESIntegTestCase.Scope.TEST, numDataNodes = 0, autoMinMasterNodes = false)
+@TestLogging("_root:DEBUG,org.elasticsearch.cluster.service:TRACE,org.elasticsearch.discovery.zen:TRACE")
+public class ElasticsearchNodeCommandIT extends ESIntegTestCase {
+
+    private MockTerminal executeCommand(ElasticsearchNodeCommand command, Environment environment, int nodeOrdinal, boolean abort)
+            throws Exception {
+        final MockTerminal terminal = new MockTerminal();
+        final OptionSet options = command.getParser().parse("-ordinal", Integer.toString(nodeOrdinal));
+        final String input;
+
+        if (abort) {
+            input = randomValueOtherThanMany(c -> c.equalsIgnoreCase("y"), () -> randomAlphaOfLength(1));
+        } else {
+            input = randomBoolean() ? "y" : "Y";
+        }
+
+        terminal.addTextInput(input);
+
+        try {
+            command.execute(terminal, options, environment);
+        } finally {
+            assertThat(terminal.getOutput(), containsString(ElasticsearchNodeCommand.STOP_WARNING_MSG));
+        }
+
+        return terminal;
+    }
+
+    private MockTerminal unsafeBootstrap(Environment environment, int nodeOrdinal, boolean abort) throws Exception {
+        final MockTerminal terminal = executeCommand(new UnsafeBootstrapMasterCommand(), environment, nodeOrdinal, abort);
+        assertThat(terminal.getOutput(), containsString(UnsafeBootstrapMasterCommand.CONFIRMATION_MSG));
+        assertThat(terminal.getOutput(), containsString(UnsafeBootstrapMasterCommand.MASTER_NODE_BOOTSTRAPPED_MSG));
+        return terminal;
+    }
+
+    private MockTerminal detachCluster(Environment environment, int nodeOrdinal, boolean abort) throws Exception {
+        final MockTerminal terminal = executeCommand(new DetachClusterCommand(), environment, nodeOrdinal, abort);
+        assertThat(terminal.getOutput(), containsString(DetachClusterCommand.CONFIRMATION_MSG));
+        assertThat(terminal.getOutput(), containsString(DetachClusterCommand.NODE_DETACHED_MSG));
+        return terminal;
+    }
+
+    private MockTerminal unsafeBootstrap(Environment environment) throws Exception {
+        return unsafeBootstrap(environment, 0, false);
+    }
+
+    private MockTerminal detachCluster(Environment environment) throws Exception {
+        return detachCluster(environment, 0, false);
+    }
+
+    private void expectThrows(ThrowingRunnable runnable, String message) {
+        ElasticsearchException ex = expectThrows(ElasticsearchException.class, runnable);
+        assertThat(ex.getMessage(), containsString(message));
+    }
+
+    public void testBootstrapNotMasterEligible() {
+        final Environment environment = TestEnvironment.newEnvironment(Settings.builder()
+                .put(internalCluster().getDefaultSettings())
+                .put(Node.NODE_MASTER_SETTING.getKey(), false)
+                .build());
+        expectThrows(() -> unsafeBootstrap(environment), UnsafeBootstrapMasterCommand.NOT_MASTER_NODE_MSG);
+    }
+
+    public void testBootstrapNoDataFolder() {
+        final Environment environment = TestEnvironment.newEnvironment(internalCluster().getDefaultSettings());
+        expectThrows(() -> unsafeBootstrap(environment), ElasticsearchNodeCommand.NO_NODE_FOLDER_FOUND_MSG);
+    }
+
+    public void testDetachNoDataFolder() {
+        final Environment environment = TestEnvironment.newEnvironment(internalCluster().getDefaultSettings());
+        expectThrows(() -> detachCluster(environment), ElasticsearchNodeCommand.NO_NODE_FOLDER_FOUND_MSG);
+    }
+
+    public void testBootstrapNodeLocked() throws IOException {
+        Settings envSettings = buildEnvSettings(Settings.EMPTY);
+        Environment environment = TestEnvironment.newEnvironment(envSettings);
+        try (NodeEnvironment ignored = new NodeEnvironment(envSettings, environment)) {
+            expectThrows(() -> unsafeBootstrap(environment), ElasticsearchNodeCommand.FAILED_TO_OBTAIN_NODE_LOCK_MSG);
+        }
+    }
+
+    public void testDetachNodeLocked() throws IOException {
+        Settings envSettings = buildEnvSettings(Settings.EMPTY);
+        Environment environment = TestEnvironment.newEnvironment(envSettings);
+        try (NodeEnvironment ignored = new NodeEnvironment(envSettings, environment)) {
+            expectThrows(() -> detachCluster(environment), ElasticsearchNodeCommand.FAILED_TO_OBTAIN_NODE_LOCK_MSG);
+        }
+    }
+
+    public void testBootstrapNoNodeMetaData() throws IOException {
+        Settings envSettings = buildEnvSettings(Settings.EMPTY);
+        Environment environment = TestEnvironment.newEnvironment(envSettings);
+        try (NodeEnvironment nodeEnvironment = new NodeEnvironment(envSettings, environment)) {
+            NodeMetaData.FORMAT.cleanupOldFiles(-1, nodeEnvironment.nodeDataPaths());
+        }
+
+        expectThrows(() -> unsafeBootstrap(environment), UnsafeBootstrapMasterCommand.NO_NODE_METADATA_FOUND_MSG);
+    }
+
+    public void testBootstrapNotBootstrappedCluster() throws Exception {
+        internalCluster().startNode(
+                Settings.builder()
+                        .put(DiscoverySettings.INITIAL_STATE_TIMEOUT_SETTING.getKey(), "0s") // to ensure quick node startup
+                        .build());
+        assertBusy(() -> {
+            ClusterState state = client().admin().cluster().prepareState().setLocal(true)
+                    .execute().actionGet().getState();
+            assertTrue(state.blocks().hasGlobalBlockWithId(DiscoverySettings.NO_MASTER_BLOCK_ID));
+        });
+
+        internalCluster().stopRandomDataNode();
+
+        Environment environment = TestEnvironment.newEnvironment(internalCluster().getDefaultSettings());
+        expectThrows(() -> unsafeBootstrap(environment), ElasticsearchNodeCommand.GLOBAL_GENERATION_MISSING_MSG);
+    }
+
+    public void testDetachNotBootstrappedCluster() throws Exception {
+        internalCluster().startNode(
+                Settings.builder()
+                        .put(DiscoverySettings.INITIAL_STATE_TIMEOUT_SETTING.getKey(), "0s") // to ensure quick node startup
+                        .build());
+        assertBusy(() -> {
+            ClusterState state = client().admin().cluster().prepareState().setLocal(true)
+                    .execute().actionGet().getState();
+            assertTrue(state.blocks().hasGlobalBlockWithId(DiscoverySettings.NO_MASTER_BLOCK_ID));
+        });
+
+        internalCluster().stopRandomDataNode();
+
+        Environment environment = TestEnvironment.newEnvironment(internalCluster().getDefaultSettings());
+        expectThrows(() -> detachCluster(environment), ElasticsearchNodeCommand.GLOBAL_GENERATION_MISSING_MSG);
+    }
+
+    public void testBootstrapNoManifestFile() throws IOException {
+        internalCluster().setBootstrapMasterNodeIndex(0);
+        internalCluster().startNode();
+        ensureStableCluster(1);
+        NodeEnvironment nodeEnvironment = internalCluster().getMasterNodeInstance(NodeEnvironment.class);
+        internalCluster().stopRandomDataNode();
+        Environment environment = TestEnvironment.newEnvironment(internalCluster().getDefaultSettings());
+        Manifest.FORMAT.cleanupOldFiles(-1, nodeEnvironment.nodeDataPaths());
+
+        expectThrows(() -> unsafeBootstrap(environment), ElasticsearchNodeCommand.NO_MANIFEST_FILE_FOUND_MSG);
+    }
+
+    public void testDetachNoManifestFile() throws IOException {
+        internalCluster().setBootstrapMasterNodeIndex(0);
+        internalCluster().startNode();
+        ensureStableCluster(1);
+        NodeEnvironment nodeEnvironment = internalCluster().getMasterNodeInstance(NodeEnvironment.class);
+        internalCluster().stopRandomDataNode();
+        Environment environment = TestEnvironment.newEnvironment(internalCluster().getDefaultSettings());
+        Manifest.FORMAT.cleanupOldFiles(-1, nodeEnvironment.nodeDataPaths());
+
+        expectThrows(() -> detachCluster(environment), ElasticsearchNodeCommand.NO_MANIFEST_FILE_FOUND_MSG);
+    }
+
+    public void testBootstrapNoMetaData() throws IOException {
+        internalCluster().setBootstrapMasterNodeIndex(0);
+        internalCluster().startNode();
+        ensureStableCluster(1);
+        NodeEnvironment nodeEnvironment = internalCluster().getMasterNodeInstance(NodeEnvironment.class);
+        internalCluster().stopRandomDataNode();
+
+        Environment environment = TestEnvironment.newEnvironment(internalCluster().getDefaultSettings());
+        MetaData.FORMAT.cleanupOldFiles(-1, nodeEnvironment.nodeDataPaths());
+
+        expectThrows(() -> unsafeBootstrap(environment), ElasticsearchNodeCommand.NO_GLOBAL_METADATA_MSG);
+    }
+
+    public void testDetachNoMetaData() throws IOException {
+        internalCluster().setBootstrapMasterNodeIndex(0);
+        internalCluster().startNode();
+        ensureStableCluster(1);
+        NodeEnvironment nodeEnvironment = internalCluster().getMasterNodeInstance(NodeEnvironment.class);
+        internalCluster().stopRandomDataNode();
+
+        Environment environment = TestEnvironment.newEnvironment(internalCluster().getDefaultSettings());
+        MetaData.FORMAT.cleanupOldFiles(-1, nodeEnvironment.nodeDataPaths());
+
+        expectThrows(() -> detachCluster(environment), ElasticsearchNodeCommand.NO_GLOBAL_METADATA_MSG);
+    }
+
+    public void testBootstrapAbortedByUser() throws IOException {
+        internalCluster().setBootstrapMasterNodeIndex(0);
+        internalCluster().startNode();
+        ensureStableCluster(1);
+        internalCluster().stopRandomDataNode();
+
+        Environment environment = TestEnvironment.newEnvironment(internalCluster().getDefaultSettings());
+        expectThrows(() -> unsafeBootstrap(environment, 0, true), ElasticsearchNodeCommand.ABORTED_BY_USER_MSG);
+    }
+
+    public void testDetachAbortedByUser() throws IOException {
+        internalCluster().setBootstrapMasterNodeIndex(0);
+        internalCluster().startNode();
+        ensureStableCluster(1);
+        internalCluster().stopRandomDataNode();
+
+        Environment environment = TestEnvironment.newEnvironment(internalCluster().getDefaultSettings());
+        expectThrows(() -> detachCluster(environment, 0, true), ElasticsearchNodeCommand.ABORTED_BY_USER_MSG);
+    }
+
+    public void test3MasterNodes2Failed() throws Exception {
+        internalCluster().setBootstrapMasterNodeIndex(2);
+        List<String> masterNodes = new ArrayList<>();
+
+        logger.info("--> start 1st master-eligible node");
+        masterNodes.add(internalCluster().startMasterOnlyNode(Settings.builder()
+                .put(DiscoverySettings.INITIAL_STATE_TIMEOUT_SETTING.getKey(), "0s")
+                .build())); // node ordinal 0
+
+        logger.info("--> start one data-only node");
+        String dataNode = internalCluster().startDataOnlyNode(Settings.builder()
+                .put(DiscoverySettings.INITIAL_STATE_TIMEOUT_SETTING.getKey(), "0s")
+                .build()); // node ordinal 1
+
+        logger.info("--> start 2nd and 3rd master-eligible nodes and bootstrap");
+        masterNodes.addAll(internalCluster().startMasterOnlyNodes(2)); // node ordinals 2 and 3
+
+        logger.info("--> create index test");
+        createIndex("test");
+
+        logger.info("--> stop 2nd and 3d master eligible node");
+        internalCluster().stopRandomNode(InternalTestCluster.nameFilter(masterNodes.get(1)));
+        internalCluster().stopRandomNode(InternalTestCluster.nameFilter(masterNodes.get(2)));
+
+        logger.info("--> ensure NO_MASTER_BLOCK on data-only node");
+        assertBusy(() -> {
+            ClusterState state = internalCluster().client(dataNode).admin().cluster().prepareState().setLocal(true)
+                    .execute().actionGet().getState();
+            assertTrue(state.blocks().hasGlobalBlockWithId(DiscoverySettings.NO_MASTER_BLOCK_ID));
+        });
+
+        logger.info("--> try to unsafely bootstrap 1st master-eligible node, while node lock is held");
+        final Environment environment = TestEnvironment.newEnvironment(internalCluster().getDefaultSettings());
+        expectThrows(() -> unsafeBootstrap(environment), UnsafeBootstrapMasterCommand.FAILED_TO_OBTAIN_NODE_LOCK_MSG);
+
+        logger.info("--> stop 1st master-eligible node and data-only node");
+        NodeEnvironment nodeEnvironment = internalCluster().getMasterNodeInstance(NodeEnvironment.class);
+        internalCluster().stopRandomNode(InternalTestCluster.nameFilter(masterNodes.get(0)));
+        internalCluster().stopRandomDataNode();
+
+        logger.info("--> unsafely-bootstrap 1st master-eligible node");
+        MockTerminal terminal = unsafeBootstrap(environment);
+        MetaData metaData = MetaData.FORMAT.loadLatestState(logger, xContentRegistry(), nodeEnvironment.nodeDataPaths());
+        assertThat(terminal.getOutput(), containsString(
+                String.format(Locale.ROOT, UnsafeBootstrapMasterCommand.CLUSTER_STATE_TERM_VERSION_MSG_FORMAT,
+                        metaData.coordinationMetaData().term(), metaData.version())));
+
+        logger.info("--> start 1st master-eligible node");
+        internalCluster().startMasterOnlyNode();
+
+        logger.info("--> detach-cluster on data-only node");
+        detachCluster(environment, 1, false);
+
+        logger.info("--> start data-only node");
+        String dataNode2 = internalCluster().startDataOnlyNode();
+
+        logger.info("--> ensure there is no NO_MASTER_BLOCK and unsafe-bootstrap is reflected in cluster state");
+        assertBusy(() -> {
+            ClusterState state = internalCluster().client(dataNode2).admin().cluster().prepareState().setLocal(true)
+                    .execute().actionGet().getState();
+            assertFalse(state.blocks().hasGlobalBlockWithId(DiscoverySettings.NO_MASTER_BLOCK_ID));
+            assertTrue(state.metaData().persistentSettings().getAsBoolean(UnsafeBootstrapMasterCommand.UNSAFE_BOOTSTRAP.getKey(), false));
+        });
+
+        logger.info("--> ensure index test is green");
+        ensureGreen("test");
+
+        logger.info("--> detach-cluster on 2nd and 3rd master-eligible nodes");
+        detachCluster(environment, 2, false);
+        detachCluster(environment, 3, false);
+
+        logger.info("--> start 2nd and 3rd master-eligible nodes and ensure 4 nodes stable cluster");
+        internalCluster().startMasterOnlyNodes(2);
+        ensureStableCluster(4);
+    }
+
+    public void testAllMasterEligibleNodesFailedDanglingIndexImport() throws Exception {
+        internalCluster().setBootstrapMasterNodeIndex(0);
+
+        logger.info("--> start mixed data and master-eligible node and bootstrap cluster");
+        String masterNode = internalCluster().startNode(); // node ordinal 0
+
+        logger.info("--> start data-only node and ensure 2 nodes stable cluster");
+        String dataNode = internalCluster().startDataOnlyNode(); // node ordinal 1
+        ensureStableCluster(2);
+
+        logger.info("--> index 1 doc and ensure index is green");
+        client().prepareIndex("test", "type1", "1").setSource("field1", "value1").setRefreshPolicy(IMMEDIATE).get();
+        ensureGreen("test");
+
+        logger.info("--> verify 1 doc in the index");
+        assertHitCount(client().prepareSearch().setQuery(matchAllQuery()).get(), 1L);
+        assertThat(client().prepareGet("test", "type1", "1").execute().actionGet().isExists(), equalTo(true));
+
+        logger.info("--> stop data-only node and detach it from the old cluster");
+        internalCluster().stopRandomNode(InternalTestCluster.nameFilter(dataNode));
+        final Environment environment = TestEnvironment.newEnvironment(internalCluster().getDefaultSettings());
+        detachCluster(environment, 1, false);
+
+        logger.info("--> stop master-eligible node, clear its data and start it again - new cluster should form");
+        internalCluster().restartNode(masterNode, new InternalTestCluster.RestartCallback(){
+            @Override
+            public boolean clearData(String nodeName) {
+                return true;
+            }
+        });
+
+        logger.info("--> start data-only only node and ensure 2 nodes stable cluster");
+        internalCluster().startDataOnlyNode();
+        ensureStableCluster(2);
+
+        logger.info("--> verify that the dangling index exists and has green status");
+        assertBusy(() -> {
+            assertThat(client().admin().indices().prepareExists("test").execute().actionGet().isExists(), equalTo(true));
+        });
+        ensureGreen("test");
+
+        logger.info("--> verify the doc is there");
+        assertThat(client().prepareGet("test", "type1", "1").execute().actionGet().isExists(), equalTo(true));
+    }
+
+    public void testNoInitialBootstrapAfterDetach() throws Exception {
+        internalCluster().setBootstrapMasterNodeIndex(0);
+        internalCluster().startMasterOnlyNode();
+        internalCluster().stopCurrentMasterNode();
+
+        final Environment environment = TestEnvironment.newEnvironment(internalCluster().getDefaultSettings());
+        detachCluster(environment);
+
+        String node = internalCluster().startMasterOnlyNode(Settings.builder()
+                // give the cluster 2 seconds to elect the master (it should not)
+                .put(DiscoverySettings.INITIAL_STATE_TIMEOUT_SETTING.getKey(), "2s")
+                .build());
+
+        ClusterState state = internalCluster().client().admin().cluster().prepareState().setLocal(true)
+                .execute().actionGet().getState();
+        assertTrue(state.blocks().hasGlobalBlockWithId(DiscoverySettings.NO_MASTER_BLOCK_ID));
+
+        internalCluster().stopRandomNode(InternalTestCluster.nameFilter(node));
+    }
+
+    public void testCanRunUnsafeBootstrapAfterErroneousDetachWithoutLoosingMetaData() throws Exception {
+        internalCluster().setBootstrapMasterNodeIndex(0);
+        internalCluster().startMasterOnlyNode();
+        ClusterUpdateSettingsRequest req = new ClusterUpdateSettingsRequest().persistentSettings(
+                Settings.builder().put(INDICES_RECOVERY_MAX_BYTES_PER_SEC_SETTING.getKey(), "1234kb"));
+        internalCluster().client().admin().cluster().updateSettings(req).get();
+
+        ClusterState state = internalCluster().client().admin().cluster().prepareState().execute().actionGet().getState();
+        assertThat(state.metaData().persistentSettings().get(INDICES_RECOVERY_MAX_BYTES_PER_SEC_SETTING.getKey()),
+                equalTo("1234kb"));
+
+        internalCluster().stopCurrentMasterNode();
+
+        final Environment environment = TestEnvironment.newEnvironment(internalCluster().getDefaultSettings());
+        detachCluster(environment);
+        unsafeBootstrap(environment);
+
+        internalCluster().startMasterOnlyNode();
+        ensureStableCluster(1);
+
+        state = internalCluster().client().admin().cluster().prepareState().execute().actionGet().getState();
+        assertThat(state.metaData().settings().get(INDICES_RECOVERY_MAX_BYTES_PER_SEC_SETTING.getKey()),
+                equalTo("1234kb"));
+    }
+}

+ 0 - 209
server/src/test/java/org/elasticsearch/cluster/coordination/UnsafeBootstrapMasterIT.java

@@ -1,209 +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.cluster.coordination;
-
-import joptsimple.OptionParser;
-import joptsimple.OptionSet;
-import org.elasticsearch.ElasticsearchException;
-import org.elasticsearch.cli.MockTerminal;
-import org.elasticsearch.client.Client;
-import org.elasticsearch.cluster.ClusterState;
-import org.elasticsearch.cluster.metadata.Manifest;
-import org.elasticsearch.cluster.metadata.MetaData;
-import org.elasticsearch.common.settings.Settings;
-import org.elasticsearch.discovery.DiscoverySettings;
-import org.elasticsearch.env.Environment;
-import org.elasticsearch.env.NodeEnvironment;
-import org.elasticsearch.env.NodeMetaData;
-import org.elasticsearch.env.TestEnvironment;
-import org.elasticsearch.node.Node;
-import org.elasticsearch.test.ESIntegTestCase;
-import org.elasticsearch.test.InternalTestCluster;
-import org.elasticsearch.test.junit.annotations.TestLogging;
-
-import java.io.IOException;
-import java.util.List;
-import java.util.Locale;
-
-import static org.hamcrest.Matchers.containsString;
-
-@ESIntegTestCase.ClusterScope(scope = ESIntegTestCase.Scope.TEST, numDataNodes = 0, autoMinMasterNodes = false)
-@TestLogging("_root:DEBUG,org.elasticsearch.cluster.service:TRACE,org.elasticsearch.discovery.zen:TRACE")
-public class UnsafeBootstrapMasterIT extends ESIntegTestCase {
-
-    private MockTerminal executeCommand(Environment environment, boolean abort) throws Exception {
-        final UnsafeBootstrapMasterCommand command = new UnsafeBootstrapMasterCommand();
-        final MockTerminal terminal = new MockTerminal();
-        final OptionParser parser = new OptionParser();
-        final OptionSet options = parser.parse();
-        final String input;
-
-        if (abort) {
-            input = randomValueOtherThanMany(c -> c.equalsIgnoreCase("y"), () -> randomAlphaOfLength(1));
-        } else {
-            input = randomBoolean() ? "y" : "Y";
-        }
-
-        terminal.addTextInput(input);
-
-        try {
-            command.execute(terminal, options, environment);
-            assertThat(terminal.getOutput(), containsString(UnsafeBootstrapMasterCommand.MASTER_NODE_BOOTSTRAPPED_MSG));
-        } finally {
-            assertThat(terminal.getOutput(), containsString(UnsafeBootstrapMasterCommand.STOP_WARNING_MSG));
-        }
-
-        return terminal;
-    }
-
-    private MockTerminal executeCommand(Environment environment) throws Exception {
-        return executeCommand(environment, false);
-    }
-
-    private void expectThrows(ThrowingRunnable runnable, String message) {
-        ElasticsearchException ex = expectThrows(ElasticsearchException.class, runnable);
-        assertThat(ex.getMessage(), containsString(message));
-    }
-
-    public void testNotMasterEligible() {
-        final Environment environment = TestEnvironment.newEnvironment(Settings.builder()
-                .put(internalCluster().getDefaultSettings())
-                .put(Node.NODE_MASTER_SETTING.getKey(), false)
-                .build());
-        expectThrows(() -> executeCommand(environment), UnsafeBootstrapMasterCommand.NOT_MASTER_NODE_MSG);
-    }
-
-    public void testNoDataFolder() {
-        final Environment environment = TestEnvironment.newEnvironment(internalCluster().getDefaultSettings());
-        expectThrows(() -> executeCommand(environment), UnsafeBootstrapMasterCommand.NO_NODE_FOLDER_FOUND_MSG);
-    }
-
-    public void testNodeLocked() throws IOException {
-        Settings envSettings = buildEnvSettings(Settings.EMPTY);
-        Environment environment = TestEnvironment.newEnvironment(envSettings);
-        try (NodeEnvironment ignored = new NodeEnvironment(envSettings, environment)) {
-            expectThrows(() -> executeCommand(environment), UnsafeBootstrapMasterCommand.FAILED_TO_OBTAIN_NODE_LOCK_MSG);
-        }
-    }
-
-    public void testNoNodeMetaData() throws IOException {
-        Settings envSettings = buildEnvSettings(Settings.EMPTY);
-        Environment environment = TestEnvironment.newEnvironment(envSettings);
-        try (NodeEnvironment nodeEnvironment = new NodeEnvironment(envSettings, environment)) {
-            NodeMetaData.FORMAT.cleanupOldFiles(-1, nodeEnvironment.nodeDataPaths());
-        }
-
-        expectThrows(() -> executeCommand(environment), UnsafeBootstrapMasterCommand.NO_NODE_METADATA_FOUND_MSG);
-    }
-
-    public void testNotBootstrappedCluster() throws Exception {
-        internalCluster().startNode(
-                Settings.builder()
-                        .put(DiscoverySettings.INITIAL_STATE_TIMEOUT_SETTING.getKey(), "0s") // to ensure quick node startup
-                        .build());
-        assertBusy(() -> {
-            ClusterState state = client().admin().cluster().prepareState().setLocal(true)
-                    .execute().actionGet().getState();
-            assertTrue(state.blocks().hasGlobalBlockWithId(DiscoverySettings.NO_MASTER_BLOCK_ID));
-        });
-
-        internalCluster().stopRandomDataNode();
-
-        Environment environment = TestEnvironment.newEnvironment(internalCluster().getDefaultSettings());
-        expectThrows(() -> executeCommand(environment), UnsafeBootstrapMasterCommand.GLOBAL_GENERATION_MISSING_MSG);
-    }
-
-    public void testNoManifestFile() throws IOException {
-        internalCluster().setBootstrapMasterNodeIndex(0);
-        internalCluster().startNode();
-        ensureStableCluster(1);
-        NodeEnvironment nodeEnvironment = internalCluster().getMasterNodeInstance(NodeEnvironment.class);
-        internalCluster().stopRandomDataNode();
-        Environment environment = TestEnvironment.newEnvironment(internalCluster().getDefaultSettings());
-        Manifest.FORMAT.cleanupOldFiles(-1, nodeEnvironment.nodeDataPaths());
-
-        expectThrows(() -> executeCommand(environment), UnsafeBootstrapMasterCommand.NO_MANIFEST_FILE_FOUND_MSG);
-    }
-
-    public void testNoMetaData() throws IOException {
-        internalCluster().setBootstrapMasterNodeIndex(0);
-        internalCluster().startNode();
-        ensureStableCluster(1);
-        NodeEnvironment nodeEnvironment = internalCluster().getMasterNodeInstance(NodeEnvironment.class);
-        internalCluster().stopRandomDataNode();
-
-        Environment environment = TestEnvironment.newEnvironment(internalCluster().getDefaultSettings());
-        MetaData.FORMAT.cleanupOldFiles(-1, nodeEnvironment.nodeDataPaths());
-
-        expectThrows(() -> executeCommand(environment), UnsafeBootstrapMasterCommand.NO_GLOBAL_METADATA_MSG);
-    }
-
-    public void testAbortedByUser() throws IOException {
-        internalCluster().setBootstrapMasterNodeIndex(0);
-        internalCluster().startNode();
-        ensureStableCluster(1);
-        internalCluster().stopRandomDataNode();
-
-        Environment environment = TestEnvironment.newEnvironment(internalCluster().getDefaultSettings());
-        expectThrows(() -> executeCommand(environment, true), UnsafeBootstrapMasterCommand.ABORTED_BY_USER_MSG);
-    }
-
-    public void test3MasterNodes2Failed() throws Exception {
-        internalCluster().setBootstrapMasterNodeIndex(2);
-        List<String> masterNodes = internalCluster().startMasterOnlyNodes(3, Settings.EMPTY);
-
-        String dataNode = internalCluster().startDataOnlyNode();
-        createIndex("test");
-
-        Client dataNodeClient = internalCluster().client(dataNode);
-
-        internalCluster().stopRandomNode(InternalTestCluster.nameFilter(masterNodes.get(1)));
-        internalCluster().stopRandomNode(InternalTestCluster.nameFilter(masterNodes.get(2)));
-
-        assertBusy(() -> {
-            ClusterState state = dataNodeClient.admin().cluster().prepareState().setLocal(true)
-                    .execute().actionGet().getState();
-            assertTrue(state.blocks().hasGlobalBlockWithId(DiscoverySettings.NO_MASTER_BLOCK_ID));
-        });
-
-        final Environment environment = TestEnvironment.newEnvironment(internalCluster().getDefaultSettings());
-        expectThrows(() -> executeCommand(environment), UnsafeBootstrapMasterCommand.FAILED_TO_OBTAIN_NODE_LOCK_MSG);
-
-        NodeEnvironment nodeEnvironment = internalCluster().getMasterNodeInstance(NodeEnvironment.class);
-        internalCluster().stopRandomNode(InternalTestCluster.nameFilter(masterNodes.get(0)));
-
-        MockTerminal terminal = executeCommand(environment);
-
-        MetaData metaData = MetaData.FORMAT.loadLatestState(logger, xContentRegistry(), nodeEnvironment.nodeDataPaths());
-        assertThat(terminal.getOutput(), containsString(
-                String.format(Locale.ROOT, UnsafeBootstrapMasterCommand.CLUSTER_STATE_TERM_VERSION_MSG_FORMAT,
-                        metaData.coordinationMetaData().term(), metaData.version())));
-
-        internalCluster().startMasterOnlyNode();
-
-        assertBusy(() -> {
-            ClusterState state = dataNodeClient.admin().cluster().prepareState().setLocal(true)
-                    .execute().actionGet().getState();
-            assertFalse(state.blocks().hasGlobalBlockWithId(DiscoverySettings.NO_MASTER_BLOCK_ID));
-            assertTrue(state.metaData().persistentSettings().getAsBoolean(UnsafeBootstrapMasterCommand.UNSAFE_BOOTSTRAP.getKey(), false));
-        });
-
-        ensureGreen("test");
-    }
-}

+ 0 - 23
server/src/test/java/org/elasticsearch/discovery/ClusterDisruptionIT.java

@@ -39,7 +39,6 @@ import org.elasticsearch.common.unit.TimeValue;
 import org.elasticsearch.common.xcontent.XContentType;
 import org.elasticsearch.test.ESIntegTestCase;
 import org.elasticsearch.test.InternalTestCluster;
-import org.elasticsearch.test.discovery.TestZenDiscovery;
 import org.elasticsearch.test.disruption.NetworkDisruption;
 import org.elasticsearch.test.disruption.NetworkDisruption.Bridge;
 import org.elasticsearch.test.disruption.NetworkDisruption.NetworkDisconnect;
@@ -63,7 +62,6 @@ import java.util.stream.Collectors;
 
 import static org.elasticsearch.action.DocWriteResponse.Result.CREATED;
 import static org.elasticsearch.action.DocWriteResponse.Result.UPDATED;
-import static org.elasticsearch.common.xcontent.XContentFactory.jsonBuilder;
 import static org.elasticsearch.test.hamcrest.ElasticsearchAssertions.assertAcked;
 import static org.hamcrest.Matchers.equalTo;
 import static org.hamcrest.Matchers.is;
@@ -358,27 +356,6 @@ public class ClusterDisruptionIT extends AbstractDisruptionTestCase {
         }
     }
 
-    public void testIndexImportedFromDataOnlyNodesIfMasterLostDataFolder() throws Exception {
-        // test for https://github.com/elastic/elasticsearch/issues/8823
-        Settings zen1Settings = Settings.builder().put(TestZenDiscovery.USE_ZEN2.getKey(), false).build(); // TODO: needs adaptions for Zen2
-        String masterNode = internalCluster().startMasterOnlyNode(zen1Settings);
-        internalCluster().startDataOnlyNode(zen1Settings);
-        ensureStableCluster(2);
-        assertAcked(prepareCreate("index").setSettings(Settings.builder().put("index.number_of_replicas", 0)));
-        index("index", "_doc", "1", jsonBuilder().startObject().field("text", "some text").endObject());
-        ensureGreen();
-
-        internalCluster().restartNode(masterNode, new InternalTestCluster.RestartCallback() {
-            @Override
-            public boolean clearData(String nodeName) {
-                return true;
-            }
-        });
-
-        ensureGreen("index");
-        assertTrue(client().prepareGet("index", "_doc", "1").get().isExists());
-    }
-
     public void testCannotJoinIfMasterLostDataFolder() throws Exception {
         String masterNode = internalCluster().startMasterOnlyNode();
         String dataNode = internalCluster().startDataOnlyNode();

+ 0 - 53
server/src/test/java/org/elasticsearch/gateway/GatewayIndexStateIT.java

@@ -37,7 +37,6 @@ import org.elasticsearch.cluster.metadata.MetaData;
 import org.elasticsearch.cluster.routing.ShardRoutingState;
 import org.elasticsearch.common.Priority;
 import org.elasticsearch.common.settings.Settings;
-import org.elasticsearch.common.unit.TimeValue;
 import org.elasticsearch.common.xcontent.XContentFactory;
 import org.elasticsearch.common.xcontent.XContentType;
 import org.elasticsearch.discovery.zen.ElectMasterService;
@@ -49,7 +48,6 @@ import org.elasticsearch.test.ESIntegTestCase;
 import org.elasticsearch.test.ESIntegTestCase.ClusterScope;
 import org.elasticsearch.test.ESIntegTestCase.Scope;
 import org.elasticsearch.test.InternalTestCluster.RestartCallback;
-import org.elasticsearch.test.discovery.TestZenDiscovery;
 
 import java.io.IOException;
 import java.util.List;
@@ -275,57 +273,6 @@ public class GatewayIndexStateIT extends ESIntegTestCase {
         }
     }
 
-    public void testDanglingIndices() throws Exception {
-        /*TODO This test test does not work with Zen2, because once master node looses its cluster state during restart
-        it will start with term = 1, which is the same as the term data node has. Data node won't accept cluster state from master
-        after the restart, because the term is the same, but version of the cluster state is greater on the data node.
-        Consider adding term to JoinRequest, so that master node can bump its term if its current term is less than JoinRequest#term.
-        */
-        logger.info("--> starting two nodes");
-
-        final String node_1 = internalCluster().startNodes(2,
-                Settings.builder().put(TestZenDiscovery.USE_ZEN2.getKey(), false).build()).get(0);
-
-        logger.info("--> indexing a simple document");
-        client().prepareIndex("test", "type1", "1").setSource("field1", "value1").setRefreshPolicy(IMMEDIATE).get();
-
-        logger.info("--> waiting for green status");
-        ensureGreen();
-
-        logger.info("--> verify 1 doc in the index");
-        for (int i = 0; i < 10; i++) {
-            assertHitCount(client().prepareSearch().setQuery(matchAllQuery()).get(), 1L);
-        }
-        assertThat(client().prepareGet("test", "type1", "1").execute().actionGet().isExists(), equalTo(true));
-
-        logger.info("--> restarting the nodes");
-        internalCluster().fullRestart(new RestartCallback() {
-            @Override
-            public boolean clearData(String nodeName) {
-                return node_1.equals(nodeName);
-            }
-        });
-
-        logger.info("--> waiting for green status");
-        ensureGreen();
-
-        // spin a bit waiting for the index to exists
-        long time = System.currentTimeMillis();
-        while ((System.currentTimeMillis() - time) < TimeValue.timeValueSeconds(10).millis()) {
-            if (client().admin().indices().prepareExists("test").execute().actionGet().isExists()) {
-                break;
-            }
-        }
-
-        logger.info("--> verify that the dangling index exists");
-        assertThat(client().admin().indices().prepareExists("test").execute().actionGet().isExists(), equalTo(true));
-        logger.info("--> waiting for green status");
-        ensureGreen();
-
-        logger.info("--> verify the doc is there");
-        assertThat(client().prepareGet("test", "type1", "1").execute().actionGet().isExists(), equalTo(true));
-    }
-
     /**
      * This test ensures that when an index deletion takes place while a node is offline, when that
      * node rejoins the cluster, it deletes the index locally instead of importing it as a dangling index.