You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@ignite.apache.org by il...@apache.org on 2020/07/10 10:29:00 UTC

[ignite] branch ignite-2.9 updated: IGNITE-13185 CLI and JMX APIs and new event for Cluster ID and Tag feature - Fixes #7964.

This is an automated email from the ASF dual-hosted git repository.

ilyak pushed a commit to branch ignite-2.9
in repository https://gitbox.apache.org/repos/asf/ignite.git


The following commit(s) were added to refs/heads/ignite-2.9 by this push:
     new aa043be  IGNITE-13185 CLI and JMX APIs and new event for Cluster ID and Tag feature - Fixes #7964.
aa043be is described below

commit aa043be3e916b6e78f5b110d4e53fe32aa1ee2b1
Author: Sergey Chugunov <sc...@gridgain.com>
AuthorDate: Wed Jul 8 16:04:09 2020 +0300

    IGNITE-13185 CLI and JMX APIs and new event for Cluster ID and Tag feature - Fixes #7964.
    
    Signed-off-by: Ilya Kasnacheev <il...@gmail.com>
---
 .../ignite/events/ClusterTagUpdatedEvent.java      |  82 +++++++++++
 .../java/org/apache/ignite/events/EventType.java   |  18 ++-
 .../internal/client/GridClientClusterState.java    |  18 ++-
 .../client/impl/GridClientClusterStateImpl.java    |  24 ++++
 .../ignite/internal/cluster/IgniteClusterImpl.java |   7 +-
 .../internal/commandline/BaselineCommand.java      |   2 +-
 .../commandline/ClusterChangeTagCommand.java       | 115 ++++++++++++++++
 .../ignite/internal/commandline/CommandList.java   |   3 +
 .../ignite/internal/commandline/StateCommand.java  |   9 ++
 .../processors/cluster/ClusterProcessor.java       | 142 +++++++++++++++----
 .../cluster/IgniteClusterMXBeanImpl.java           |  62 +++++++++
 .../visor/misc/VisorClusterChangeTagTask.java      |  88 ++++++++++++
 .../visor/misc/VisorClusterChangeTagTaskArg.java   |  59 ++++++++
 .../misc/VisorClusterChangeTagTaskResult.java      |  88 ++++++++++++
 .../internal/visor/misc/VisorIdAndTagViewTask.java |  68 +++++++++
 .../visor/misc/VisorIdAndTagViewTaskResult.java    |  74 ++++++++++
 .../apache/ignite/mxbean/IgniteClusterMXBean.java  |  56 ++++++++
 .../main/resources/META-INF/classnames.properties  |   2 +
 .../internal/cluster/IgniteClusterIdTagTest.java   | 153 ++++++++++++++++++++-
 .../commandline/CommandHandlerParsingTest.java     |  16 ++-
 .../apache/ignite/util/GridCommandHandlerTest.java |  61 ++++++++
 ...ridCommandHandlerClusterByClassTest_help.output |   3 +
 ...andHandlerClusterByClassWithSSLTest_help.output |   3 +
 23 files changed, 1111 insertions(+), 42 deletions(-)

diff --git a/modules/core/src/main/java/org/apache/ignite/events/ClusterTagUpdatedEvent.java b/modules/core/src/main/java/org/apache/ignite/events/ClusterTagUpdatedEvent.java
new file mode 100644
index 0000000..00d3b63
--- /dev/null
+++ b/modules/core/src/main/java/org/apache/ignite/events/ClusterTagUpdatedEvent.java
@@ -0,0 +1,82 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF 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.apache.ignite.events;
+
+import java.util.UUID;
+import org.apache.ignite.cluster.ClusterNode;
+
+/**
+ * Event type indicating that cluster tag has been updated.
+ *
+ * @see EventType#EVT_CLUSTER_TAG_UPDATED
+ */
+public class ClusterTagUpdatedEvent extends EventAdapter {
+    /** */
+    private static final long serialVersionUID = 0L;
+
+    /** ID of cluster. */
+    private final UUID clusterId;
+
+    /** Previous value of tag. */
+    private final String prevTag;
+
+    /** New value of tag. */
+    private final String newTag;
+
+    /**
+     * @param node Node on which the event was fired.
+     * @param msg Optional event message.
+     * @param clusterId ID of cluster which tag was updated.
+     * @param prevTag Previous cluster tag replaced during update.
+     * @param newTag New cluster tag.
+     */
+    public ClusterTagUpdatedEvent(ClusterNode node, String msg, UUID clusterId,
+        String prevTag, String newTag) {
+        super(node, msg, EventType.EVT_CLUSTER_TAG_UPDATED);
+        this.clusterId = clusterId;
+        this.prevTag = prevTag;
+        this.newTag = newTag;
+    }
+
+    /**
+     * Cluster ID which tag was updated.
+     *
+     * @return UUID of cluster.
+     */
+    public UUID clusterId() {
+        return clusterId;
+    }
+
+    /**
+     * Value of cluster tag before update request that triggered this event.
+     *
+     * @return Previous value of tag.
+     */
+    public String previousTag() {
+        return prevTag;
+    }
+
+    /**
+     * Value of cluster tag after update request that triggered this event.
+     *
+     * @return New value of tag.
+     */
+    public String newTag() {
+        return newTag;
+    }
+}
diff --git a/modules/core/src/main/java/org/apache/ignite/events/EventType.java b/modules/core/src/main/java/org/apache/ignite/events/EventType.java
index 25b1a5e..f6ba4da 100644
--- a/modules/core/src/main/java/org/apache/ignite/events/EventType.java
+++ b/modules/core/src/main/java/org/apache/ignite/events/EventType.java
@@ -19,6 +19,7 @@ package org.apache.ignite.events;
 
 import java.util.List;
 import org.apache.ignite.IgniteCache;
+import org.apache.ignite.IgniteCluster;
 import org.apache.ignite.IgniteEvents;
 import org.apache.ignite.configuration.IgniteConfiguration;
 import org.apache.ignite.internal.util.typedef.internal.U;
@@ -926,7 +927,7 @@ public interface EventType {
      * Built-in event type: page replacement started in one of the data regions. The name of the data region will
      * be indicated in the event.
      * <p>
-     * Fired whan all existing free pages are exhausted and Ignite replaces one of the loaded pages with a
+     * Fired when all existing free pages are exhausted and Ignite replaces one of the loaded pages with a
      * cold page from disk.
      * <p>
      * When started, page replacement negatively affects performance; it is recommended to monitor page replacement
@@ -941,6 +942,21 @@ public interface EventType {
     public static final int EVT_PAGE_REPLACEMENT_STARTED = 142;
 
     /**
+     * Built-in event type: cluster tag has been changed by user request.
+     * Event includes the following information: ID of the cluster, old tag and new tag.
+     *
+     * <p>
+     * Fired when new tag is successfully set on all nodes.
+     * </p>
+     * NOTE: all types in range <b>from 1 to 1000 are reserved</b> for
+     * internal Ignite events and should not be used by user-defined events.
+     *
+     * @see IgniteCluster#tag(String)
+     * @see IgniteCluster#id()
+     */
+    public static final int EVT_CLUSTER_TAG_UPDATED = 143;
+
+    /**
      * Built-in event type: Cluster state changed.
      * <p>
      * Fired when cluster state changed.
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/client/GridClientClusterState.java b/modules/core/src/main/java/org/apache/ignite/internal/client/GridClientClusterState.java
index 97d72a1..602d48b 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/client/GridClientClusterState.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/client/GridClientClusterState.java
@@ -17,10 +17,12 @@
 
 package org.apache.ignite.internal.client;
 
+import java.util.UUID;
+
 import org.apache.ignite.cluster.ClusterState;
 
 /**
- *  Interface for manage state of grid cluster.
+ *  Interface for managing state of grid cluster and obtaining information about it: ID and tag.
  */
 public interface GridClientClusterState {
     /**
@@ -30,6 +32,20 @@ public interface GridClientClusterState {
     public ClusterState state() throws GridClientException;
 
     /**
+     * Unique identifier of cluster STATE command was sent to.
+     *
+     * @return ID of the cluster.
+     */
+    public UUID id() throws GridClientException;
+
+    /**
+     * User-defined tag of cluster STATE command was sent to.
+     *
+     * @return Tag of the cluster.
+     */
+    public String tag() throws GridClientException;
+
+    /**
      * Changes cluster state to {@code newState}.
      *
      * @param newState New cluster state.
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/client/impl/GridClientClusterStateImpl.java b/modules/core/src/main/java/org/apache/ignite/internal/client/impl/GridClientClusterStateImpl.java
index db454cf..4f63043 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/client/impl/GridClientClusterStateImpl.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/client/impl/GridClientClusterStateImpl.java
@@ -27,6 +27,9 @@ import org.apache.ignite.internal.client.GridClientNode;
 import org.apache.ignite.internal.client.GridClientPredicate;
 import org.apache.ignite.internal.client.balancer.GridClientLoadBalancer;
 import org.apache.ignite.internal.client.impl.connection.GridClientConnection;
+import org.apache.ignite.internal.visor.VisorTaskArgument;
+import org.apache.ignite.internal.visor.misc.VisorIdAndTagViewTask;
+import org.apache.ignite.internal.visor.misc.VisorIdAndTagViewTaskResult;
 
 import static org.apache.ignite.cluster.ClusterState.INACTIVE;
 import static org.apache.ignite.internal.client.util.GridClientUtils.checkFeatureSupportedByCluster;
@@ -37,6 +40,17 @@ import static org.apache.ignite.internal.client.util.GridClientUtils.checkFeatur
 public class GridClientClusterStateImpl extends GridClientAbstractProjection<GridClientClusterStateImpl>
     implements GridClientClusterState {
     /**
+     * Closure to execute Cluster ID and Tag view action on cluster.
+     */
+    private static final ClientProjectionClosure<VisorIdAndTagViewTaskResult> ID_AND_TAG_VIEW_CL = (conn, nodeId) ->
+        conn.execute(
+            VisorIdAndTagViewTask.class.getName(),
+            new VisorTaskArgument<>(nodeId, null, false),
+            nodeId,
+            false
+        );
+
+    /**
      * Creates projection with specified client.
      *
      * @param client Client instance to use.
@@ -78,6 +92,16 @@ public class GridClientClusterStateImpl extends GridClientAbstractProjection<Gri
     }
 
     /** {@inheritDoc} */
+    @Override public UUID id() throws GridClientException {
+        return withReconnectHandling(ID_AND_TAG_VIEW_CL).get().id();
+    }
+
+    /** {@inheritDoc} */
+    @Override public String tag() throws GridClientException {
+        return withReconnectHandling(ID_AND_TAG_VIEW_CL).get().tag();
+    }
+
+    /** {@inheritDoc} */
     @Override public String clusterName() throws GridClientException {
         return withReconnectHandling(GridClientConnection::clusterName).get();
     }
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/cluster/IgniteClusterImpl.java b/modules/core/src/main/java/org/apache/ignite/internal/cluster/IgniteClusterImpl.java
index 22449b3..b57cf86 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/cluster/IgniteClusterImpl.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/cluster/IgniteClusterImpl.java
@@ -672,8 +672,11 @@ public class IgniteClusterImpl extends ClusterGroupAdapter implements IgniteClus
 
     /** {@inheritDoc} */
     @Override public void tag(String tag) throws IgniteCheckedException {
-        if (tag == null || tag.isEmpty())
-            throw new IgniteCheckedException("Please provide not-null and not empty string for cluster tag");
+        if (tag == null)
+            throw new IgniteCheckedException("Tag cannot be null.");
+
+        if (tag.isEmpty())
+            throw new IgniteCheckedException("Tag should not be empty.");
 
         if (tag.length() > MAX_TAG_LENGTH) {
             throw new IgniteCheckedException("Maximum tag length is exceeded, max length is " +
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/commandline/BaselineCommand.java b/modules/core/src/main/java/org/apache/ignite/internal/commandline/BaselineCommand.java
index 6991dcb..b377b71 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/commandline/BaselineCommand.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/commandline/BaselineCommand.java
@@ -48,7 +48,7 @@ import static org.apache.ignite.internal.commandline.TaskExecutor.executeTaskByN
 import static org.apache.ignite.internal.commandline.baseline.BaselineSubcommands.of;
 
 /**
- * Commands assosiated with baseline functionality.
+ * Commands associated with baseline functionality.
  */
 public class BaselineCommand implements Command<BaselineArguments> {
     /** Arguments. */
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/commandline/ClusterChangeTagCommand.java b/modules/core/src/main/java/org/apache/ignite/internal/commandline/ClusterChangeTagCommand.java
new file mode 100644
index 0000000..3f9e944
--- /dev/null
+++ b/modules/core/src/main/java/org/apache/ignite/internal/commandline/ClusterChangeTagCommand.java
@@ -0,0 +1,115 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF 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.apache.ignite.internal.commandline;
+
+import java.util.Comparator;
+import java.util.UUID;
+import java.util.logging.Logger;
+import org.apache.ignite.internal.client.GridClient;
+import org.apache.ignite.internal.client.GridClientConfiguration;
+import org.apache.ignite.internal.client.GridClientNode;
+import org.apache.ignite.internal.visor.misc.VisorClusterChangeTagTask;
+import org.apache.ignite.internal.visor.misc.VisorClusterChangeTagTaskArg;
+import org.apache.ignite.internal.visor.misc.VisorClusterChangeTagTaskResult;
+
+import static org.apache.ignite.internal.commandline.CommandList.CLUSTER_CHANGE_TAG;
+import static org.apache.ignite.internal.commandline.CommandLogger.optional;
+import static org.apache.ignite.internal.commandline.CommonArgParser.CMD_AUTO_CONFIRMATION;
+import static org.apache.ignite.internal.commandline.TaskExecutor.executeTaskByNameOnNode;
+
+/**
+ * Command to access cluster ID and tag functionality.
+ */
+public class ClusterChangeTagCommand implements Command<String> {
+    /** */
+    private static final String ERR_NO_NEW_TAG_PROVIDED = "Please provide new tag.";
+
+    /** */
+    private static final String ERR_EMPTY_TAG_PROVIDED = "Please provide non-empty tag.";
+
+    /** */
+    private String newTagArg;
+
+    /** {@inheritDoc} */
+    @Override public Object execute(GridClientConfiguration clientCfg, Logger logger) throws Exception {
+        try (GridClient client = Command.startClient(clientCfg)) {
+            UUID coordinatorId = client.compute().nodes().stream()
+                .min(Comparator.comparingLong(GridClientNode::order))
+                .map(GridClientNode::nodeId)
+                .orElse(null);
+
+            VisorClusterChangeTagTaskResult res = executeTaskByNameOnNode(
+                client,
+                VisorClusterChangeTagTask.class.getName(),
+                toVisorArguments(),
+                coordinatorId,
+                clientCfg
+            );
+
+            if (res.success())
+                logger.info("Cluster tag updated successfully, old tag was: " + res.tag());
+            else
+                logger.warning("Error has occurred during tag update: " + res.errorMessage());
+        }
+        catch (Throwable e) {
+            logger.severe("Failed to execute Cluster ID and tag command: ");
+            logger.severe(CommandLogger.errorMessage(e));
+
+            throw e;
+        }
+
+        return null;
+    }
+
+    /** {@inheritDoc} */
+    @Override public String arg() {
+        return newTagArg;
+    }
+
+    /** {@inheritDoc} */
+    @Override public void printUsage(Logger logger) {
+        Command.usage(logger, "Change cluster tag to new value:", CLUSTER_CHANGE_TAG,
+            "newTagValue", optional(CMD_AUTO_CONFIRMATION));
+    }
+
+    /** {@inheritDoc} */
+    @Override public void parseArguments(CommandArgIterator argIter) {
+        if (!argIter.hasNextSubArg())
+            throw new IllegalArgumentException(ERR_NO_NEW_TAG_PROVIDED);
+
+        newTagArg = argIter.nextArg(ERR_NO_NEW_TAG_PROVIDED);
+
+        if (newTagArg == null || newTagArg.isEmpty())
+            throw new IllegalArgumentException(ERR_EMPTY_TAG_PROVIDED);
+    }
+
+    /** {@inheritDoc} */
+    @Override public String name() {
+        return CLUSTER_CHANGE_TAG.toCommandName();
+    }
+
+    /** {@inheritDoc} */
+    @Override public String confirmationPrompt() {
+        return "Warning: the command will change cluster tag.";
+    }
+
+    /** */
+    private VisorClusterChangeTagTaskArg toVisorArguments() {
+        return new VisorClusterChangeTagTaskArg(newTagArg);
+    }
+}
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/commandline/CommandList.java b/modules/core/src/main/java/org/apache/ignite/internal/commandline/CommandList.java
index a86c06d..cbd6f23 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/commandline/CommandList.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/commandline/CommandList.java
@@ -64,6 +64,9 @@ public enum CommandList {
     /** Snapshot commands. */
     SNAPSHOT("--snapshot", new SnapshotCommand()),
 
+    /** Change Cluster tag command. */
+    CLUSTER_CHANGE_TAG("--change-tag", new ClusterChangeTagCommand()),
+
     /** Metadata commands. */
     METADATA("--meta", new MetadataCommand());
 
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/commandline/StateCommand.java b/modules/core/src/main/java/org/apache/ignite/internal/commandline/StateCommand.java
index bf5728f..7084f2f 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/commandline/StateCommand.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/commandline/StateCommand.java
@@ -17,6 +17,7 @@
 
 package org.apache.ignite.internal.commandline;
 
+import java.util.UUID;
 import java.util.logging.Logger;
 import org.apache.ignite.cluster.ClusterState;
 import org.apache.ignite.internal.client.GridClient;
@@ -44,6 +45,14 @@ public class StateCommand implements Command<Void> {
         try (GridClient client = Command.startClient(clientCfg)) {
             GridClientClusterState state = client.state();
 
+            UUID id = state.id();
+            String tag = state.tag();
+
+            log.info("Cluster  ID: " + id);
+            log.info("Cluster tag: " + tag);
+
+            log.info(CommandHandler.DELIM);
+
             ClusterState clusterState = state.state();
 
             switch (clusterState) {
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cluster/ClusterProcessor.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cluster/ClusterProcessor.java
index 4ae86a8..8584062 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cluster/ClusterProcessor.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cluster/ClusterProcessor.java
@@ -27,10 +27,13 @@ import java.util.concurrent.ConcurrentHashMap;
 import java.util.concurrent.atomic.AtomicBoolean;
 import java.util.concurrent.atomic.AtomicLong;
 import java.util.concurrent.atomic.AtomicReference;
+import javax.management.JMException;
+import javax.management.ObjectName;
 import org.apache.ignite.IgniteCheckedException;
 import org.apache.ignite.IgniteLogger;
 import org.apache.ignite.IgniteSystemProperties;
 import org.apache.ignite.cluster.ClusterNode;
+import org.apache.ignite.events.ClusterTagUpdatedEvent;
 import org.apache.ignite.events.DiscoveryEvent;
 import org.apache.ignite.events.Event;
 import org.apache.ignite.failure.FailureContext;
@@ -68,6 +71,7 @@ import org.apache.ignite.lang.IgniteClosure;
 import org.apache.ignite.lang.IgniteFuture;
 import org.apache.ignite.lang.IgniteUuid;
 import org.apache.ignite.marshaller.jdk.JdkMarshaller;
+import org.apache.ignite.mxbean.IgniteClusterMXBean;
 import org.apache.ignite.spi.discovery.DiscoveryDataBag;
 import org.apache.ignite.spi.discovery.DiscoveryDataBag.GridDiscoveryData;
 import org.apache.ignite.spi.discovery.DiscoveryMetricsProvider;
@@ -78,6 +82,7 @@ import static org.apache.ignite.IgniteSystemProperties.IGNITE_CLUSTER_NAME;
 import static org.apache.ignite.IgniteSystemProperties.IGNITE_DIAGNOSTIC_ENABLED;
 import static org.apache.ignite.IgniteSystemProperties.IGNITE_UPDATE_NOTIFIER;
 import static org.apache.ignite.IgniteSystemProperties.getBoolean;
+import static org.apache.ignite.events.EventType.EVT_CLUSTER_TAG_UPDATED;
 import static org.apache.ignite.events.EventType.EVT_NODE_FAILED;
 import static org.apache.ignite.events.EventType.EVT_NODE_LEFT;
 import static org.apache.ignite.internal.GridComponent.DiscoveryDataExchangeType.CLUSTER_PROC;
@@ -96,6 +101,9 @@ public class ClusterProcessor extends GridProcessorAdapter implements Distribute
     private static final String CLUSTER_ID_TAG_KEY =
         DistributedMetaStorage.IGNITE_INTERNAL_KEY_PREFIX + "cluster.id.tag";
 
+    /** */
+    private static final String M_BEAN_NAME = "IgniteCluster";
+
     /** Periodic version check delay. */
     private static final long PERIODIC_VER_CHECK_DELAY = 1000 * 60 * 60; // Every hour.
 
@@ -136,14 +144,17 @@ public class ClusterProcessor extends GridProcessorAdapter implements Distribute
     private boolean sndMetrics;
 
     /** Cluster ID is stored in local variable before activation when it goes to distributed metastorage. */
-    private volatile UUID localClusterId;
+    private volatile UUID locClusterId;
 
     /** Cluster tag is stored in local variable before activation when it goes to distributed metastorage. */
-    private volatile String localClusterTag;
+    private volatile String locClusterTag;
 
     /** */
     private volatile DistributedMetaStorage metastorage;
 
+    /** */
+    private ObjectName mBean;
+
     /**
      * @param ctx Kernal context.
      */
@@ -183,9 +194,44 @@ public class ClusterProcessor extends GridProcessorAdapter implements Distribute
             log.info("Cluster ID and tag has been read from metastorage: " + idAndTag);
 
         if (idAndTag != null) {
-            localClusterId = idAndTag.id();
-            localClusterTag = idAndTag.tag();
+            locClusterId = idAndTag.id();
+            locClusterTag = idAndTag.tag();
         }
+
+        metastorage.listen(
+            (k) -> k.equals(CLUSTER_ID_TAG_KEY),
+            (String k, ClusterIdAndTag oldVal, ClusterIdAndTag newVal) -> {
+                if (log.isInfoEnabled())
+                    log.info(
+                        "Cluster tag will be set to new value: " +
+                            newVal != null ? newVal.tag() : "null" +
+                            ", previous value was: " +
+                            oldVal != null ? oldVal.tag() : "null");
+
+                if (oldVal != null && newVal != null) {
+                    if (ctx.event().isRecordable(EVT_CLUSTER_TAG_UPDATED)) {
+                        String msg = "Tag of cluster with id " +
+                            oldVal.id() +
+                            " has been updated to new value: " +
+                            newVal.tag() +
+                            ", previous value was " +
+                            oldVal.tag();
+
+                        ctx.closure().runLocalSafe(() -> ctx.event().record(
+                            new ClusterTagUpdatedEvent(
+                                ctx.discovery().localNode(),
+                                msg,
+                                oldVal.id(),
+                                oldVal.tag(),
+                                newVal.tag()
+                            )
+                        ));
+                    }
+                }
+
+                cluster.setTag(newVal != null ? newVal.tag() : null);
+            }
+        );
     }
 
     /**
@@ -208,21 +254,6 @@ public class ClusterProcessor extends GridProcessorAdapter implements Distribute
     @Override public void onReadyForWrite(DistributedMetaStorage metastorage) {
         this.metastorage = metastorage;
 
-        metastorage.listen(
-            (k) -> k.equals(CLUSTER_ID_TAG_KEY),
-            (String k, ClusterIdAndTag oldVal, ClusterIdAndTag newVal) -> {
-                if (log.isInfoEnabled()) {
-                    log.info(
-                        "Cluster tag will be set to new value: " +
-                            (newVal != null ? newVal.tag() : "null") +
-                            ", previous value was: " +
-                            (oldVal != null ? oldVal.tag() : "null"));
-                }
-
-                cluster.setTag(newVal != null ? newVal.tag() : null);
-            }
-        );
-
         ctx.closure().runLocalSafe(
             () -> {
                 try {
@@ -248,6 +279,10 @@ public class ClusterProcessor extends GridProcessorAdapter implements Distribute
     public void updateTag(String newTag) throws IgniteCheckedException {
         ClusterIdAndTag oldTag = metastorage.read(CLUSTER_ID_TAG_KEY);
 
+        if (oldTag == null)
+            throw new IgniteCheckedException("Cannot change tag as default tag has not been set yet. " +
+                "Please try again later.");
+
         if (!metastorage.compareAndSet(CLUSTER_ID_TAG_KEY, oldTag, new ClusterIdAndTag(oldTag.id(), newTag))) {
             ClusterIdAndTag concurrentValue = metastorage.read(CLUSTER_ID_TAG_KEY);
 
@@ -270,9 +305,9 @@ public class ClusterProcessor extends GridProcessorAdapter implements Distribute
      * </ul>
      */
     public void onLocalJoin() {
-        cluster.setId(localClusterId != null ? localClusterId : UUID.randomUUID());
+        cluster.setId(locClusterId != null ? locClusterId : UUID.randomUUID());
 
-        cluster.setTag(localClusterTag != null ? localClusterTag :
+        cluster.setTag(locClusterTag != null ? locClusterTag :
             ClusterTagGenerator.generateTag());
     }
 
@@ -280,8 +315,8 @@ public class ClusterProcessor extends GridProcessorAdapter implements Distribute
     @Override public void onDisconnected(IgniteFuture<?> reconnectFut) {
         assert ctx.clientNode();
 
-        localClusterId = null;
-        localClusterTag = null;
+        locClusterId = null;
+        locClusterTag = null;
 
         cluster.setId(null);
         cluster.setTag(null);
@@ -291,8 +326,8 @@ public class ClusterProcessor extends GridProcessorAdapter implements Distribute
     @Override public IgniteInternalFuture<?> onReconnected(boolean clusterRestarted) {
         assert ctx.clientNode();
 
-        cluster.setId(localClusterId);
-        cluster.setTag(localClusterTag);
+        cluster.setId(locClusterId);
+        cluster.setTag(locClusterTag);
 
         return null;
     }
@@ -489,20 +524,20 @@ public class ClusterProcessor extends GridProcessorAdapter implements Distribute
             Serializable remoteClusterId = commonData.id();
 
             if (remoteClusterId != null) {
-                if (localClusterId != null && !localClusterId.equals(remoteClusterId)) {
+                if (locClusterId != null && !locClusterId.equals(remoteClusterId)) {
                     log.warning("Received cluster ID differs from locally stored cluster ID " +
                         "and will be rewritten. " +
                         "Received cluster ID: " + remoteClusterId +
-                        ", local cluster ID: " + localClusterId);
+                        ", local cluster ID: " + locClusterId);
                 }
 
-                localClusterId = (UUID)remoteClusterId;
+                locClusterId = (UUID)remoteClusterId;
             }
 
             String remoteClusterTag = commonData.tag();
 
             if (remoteClusterTag != null)
-                localClusterTag = remoteClusterTag;
+                locClusterTag = remoteClusterTag;
         }
     }
 
@@ -550,6 +585,55 @@ public class ClusterProcessor extends GridProcessorAdapter implements Distribute
 
             ctx.timeout().addTimeoutObject(new MetricsUpdateTimeoutObject(updateFreq));
         }
+
+        IgniteClusterMXBeanImpl mxBeanImpl = new IgniteClusterMXBeanImpl(cluster);
+
+        if (!U.IGNITE_MBEANS_DISABLED) {
+            try {
+                mBean = U.registerMBean(
+                    ctx.config().getMBeanServer(),
+                    ctx.igniteInstanceName(),
+                    M_BEAN_NAME,
+                    mxBeanImpl.getClass().getSimpleName(),
+                    mxBeanImpl,
+                    IgniteClusterMXBean.class);
+
+                if (log.isDebugEnabled())
+                    log.debug("Registered " + M_BEAN_NAME + " MBean: " + mBean);
+            }
+            catch (Throwable e) {
+                U.error(log, "Failed to register MBean for cluster: ", e);
+            }
+        }
+    }
+
+    /** {@inheritDoc} */
+    @Override public void onKernalStop(boolean cancel) {
+        unregisterMBean();
+    }
+
+    /**
+     * Unregister IgniteCluster MBean.
+     */
+    private void unregisterMBean() {
+        ObjectName mBeanName = mBean;
+
+        if (mBeanName == null)
+            return;
+
+        assert !U.IGNITE_MBEANS_DISABLED;
+
+        try {
+            ctx.config().getMBeanServer().unregisterMBean(mBeanName);
+
+            mBean = null;
+
+            if (log.isDebugEnabled())
+                log.debug("Unregistered " + M_BEAN_NAME + " MBean: " + mBeanName);
+        }
+        catch (JMException e) {
+            U.error(log, "Failed to unregister " + M_BEAN_NAME + " MBean: " + mBeanName, e);
+        }
     }
 
     /** {@inheritDoc} */
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cluster/IgniteClusterMXBeanImpl.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cluster/IgniteClusterMXBeanImpl.java
new file mode 100644
index 0000000..ba44628
--- /dev/null
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cluster/IgniteClusterMXBeanImpl.java
@@ -0,0 +1,62 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF 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.apache.ignite.internal.processors.cluster;
+
+import java.util.UUID;
+import javax.management.JMException;
+import org.apache.ignite.IgniteCheckedException;
+import org.apache.ignite.internal.cluster.IgniteClusterImpl;
+import org.apache.ignite.internal.util.typedef.internal.U;
+import org.apache.ignite.mxbean.IgniteClusterMXBean;
+
+/**
+ * Implementation of {@link IgniteClusterMXBean} interface.
+ */
+public class IgniteClusterMXBeanImpl implements IgniteClusterMXBean {
+    /** Cluster instance to delegate method calls to. */
+    private final IgniteClusterImpl cluster;
+
+    /**
+     * @param cluster Cluster.
+     */
+    public IgniteClusterMXBeanImpl(IgniteClusterImpl cluster) {
+        assert cluster != null;
+
+        this.cluster = cluster;
+    }
+
+    /** {@inheritDoc} */
+    @Override public UUID getId() {
+        return cluster.id();
+    }
+
+    /** {@inheritDoc} */
+    @Override public String getTag() {
+        return cluster.tag();
+    }
+
+    /** {@inheritDoc} */
+    @Override public void tag(String newTag) throws JMException {
+        try {
+            cluster.tag(newTag);
+        }
+        catch (IgniteCheckedException e) {
+            throw U.jmException(e);
+        }
+    }
+}
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/visor/misc/VisorClusterChangeTagTask.java b/modules/core/src/main/java/org/apache/ignite/internal/visor/misc/VisorClusterChangeTagTask.java
new file mode 100644
index 0000000..f854963
--- /dev/null
+++ b/modules/core/src/main/java/org/apache/ignite/internal/visor/misc/VisorClusterChangeTagTask.java
@@ -0,0 +1,88 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF 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.apache.ignite.internal.visor.misc;
+
+import org.apache.ignite.IgniteCheckedException;
+import org.apache.ignite.IgniteException;
+import org.apache.ignite.internal.cluster.IgniteClusterEx;
+import org.apache.ignite.internal.processors.task.GridInternal;
+import org.apache.ignite.internal.processors.task.GridVisorManagementTask;
+import org.apache.ignite.internal.visor.VisorJob;
+import org.apache.ignite.internal.visor.VisorOneNodeTask;
+import org.jetbrains.annotations.Nullable;
+
+/**
+ *
+ */
+@GridInternal
+@GridVisorManagementTask
+public class VisorClusterChangeTagTask extends VisorOneNodeTask<VisorClusterChangeTagTaskArg, VisorClusterChangeTagTaskResult> {
+    /** */
+    private static final long serialVersionUID = 0L;
+
+    /** {@inheritDoc} */
+    @Override protected VisorJob<VisorClusterChangeTagTaskArg, VisorClusterChangeTagTaskResult> job(
+        VisorClusterChangeTagTaskArg arg) {
+        return new VisorClusterChangeTagJob(arg, debug);
+    }
+
+    /** */
+    private static class VisorClusterChangeTagJob extends VisorJob<VisorClusterChangeTagTaskArg, VisorClusterChangeTagTaskResult> {
+        /** */
+        private static final long serialVersionUID = 0L;
+
+        /**
+         * Create job with specified argument.
+         *
+         * @param arg Job argument.
+         * @param debug Flag indicating whether debug information should be printed into node log.
+         */
+        VisorClusterChangeTagJob(
+            @Nullable VisorClusterChangeTagTaskArg arg, boolean debug) {
+            super(arg, debug);
+        }
+
+        /** {@inheritDoc} */
+        @Override protected VisorClusterChangeTagTaskResult run(@Nullable VisorClusterChangeTagTaskArg arg) throws IgniteException {
+                return update(arg.newTag());
+        }
+
+        /**
+         * @param newTag New tag.
+         */
+        private VisorClusterChangeTagTaskResult update(String newTag) {
+            IgniteClusterEx cl = ignite.cluster();
+
+            boolean success = false;
+            String errMsg = null;
+
+            String oldTag = cl.tag();
+
+            try {
+                cl.tag(newTag);
+
+                success = true;
+            }
+            catch (IgniteCheckedException e) {
+                errMsg = e.getMessage();
+            }
+
+            return new VisorClusterChangeTagTaskResult(oldTag, Boolean.valueOf(success), errMsg);
+        }
+    }
+}
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/visor/misc/VisorClusterChangeTagTaskArg.java b/modules/core/src/main/java/org/apache/ignite/internal/visor/misc/VisorClusterChangeTagTaskArg.java
new file mode 100644
index 0000000..b307802
--- /dev/null
+++ b/modules/core/src/main/java/org/apache/ignite/internal/visor/misc/VisorClusterChangeTagTaskArg.java
@@ -0,0 +1,59 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF 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.apache.ignite.internal.visor.misc;
+
+import java.io.IOException;
+import java.io.ObjectInput;
+import java.io.ObjectOutput;
+import org.apache.ignite.internal.dto.IgniteDataTransferObject;
+
+/**
+ *
+ */
+public class VisorClusterChangeTagTaskArg extends IgniteDataTransferObject {
+    /** */
+    private static final long serialVersionUID = 0L;
+
+    /** */
+    private String newTag;
+
+    /** */
+    public VisorClusterChangeTagTaskArg() {
+        // No-op.
+    }
+
+    /** */
+    public VisorClusterChangeTagTaskArg(String newTag) {
+        this.newTag = newTag;
+    }
+
+    /** {@inheritDoc} */
+    @Override protected void writeExternalData(ObjectOutput out) throws IOException {
+        out.writeObject(newTag);
+    }
+
+    /** {@inheritDoc} */
+    @Override protected void readExternalData(byte protoVer, ObjectInput in) throws IOException, ClassNotFoundException {
+        newTag = (String)in.readObject();
+    }
+
+    /** */
+    public String newTag() {
+        return newTag;
+    }
+}
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/visor/misc/VisorClusterChangeTagTaskResult.java b/modules/core/src/main/java/org/apache/ignite/internal/visor/misc/VisorClusterChangeTagTaskResult.java
new file mode 100644
index 0000000..24aa2a4
--- /dev/null
+++ b/modules/core/src/main/java/org/apache/ignite/internal/visor/misc/VisorClusterChangeTagTaskResult.java
@@ -0,0 +1,88 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF 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.apache.ignite.internal.visor.misc;
+
+import java.io.IOException;
+import java.io.ObjectInput;
+import java.io.ObjectOutput;
+import org.apache.ignite.internal.dto.IgniteDataTransferObject;
+import org.jetbrains.annotations.Nullable;
+
+/**
+ *
+ */
+public class VisorClusterChangeTagTaskResult extends IgniteDataTransferObject {
+    /** */
+    private static final long serialVersionUID = 0L;
+
+    /** */
+    private String tag;
+
+    /** */
+    private Boolean success;
+
+    /** */
+    private String errResp;
+
+    /** Default constructor. */
+    public VisorClusterChangeTagTaskResult() {
+        // No-op.
+    }
+
+    /**
+     * @param tag Cluster tag.
+     * @param success Success of update tag operation.
+     * @param errResp Error response returned if cluster tag update has failed.
+     */
+    public VisorClusterChangeTagTaskResult(String tag, @Nullable Boolean success, @Nullable String errResp) {
+        this.tag = tag;
+        this.success = success;
+        this.errResp = errResp;
+    }
+
+    /** {@inheritDoc} */
+    @Override protected void writeExternalData(ObjectOutput out) throws IOException {
+        out.writeObject(success);
+        out.writeObject(errResp);
+
+        out.writeObject(tag);
+    }
+
+    /** {@inheritDoc} */
+    @Override protected void readExternalData(byte protoVer, ObjectInput in) throws IOException, ClassNotFoundException {
+        success = (Boolean)in.readObject();
+        errResp = (String)in.readObject();
+
+        tag = (String)in.readObject();
+    }
+
+    /** */
+    public String tag() {
+        return tag;
+    }
+
+    /** */
+    public Boolean success() {
+        return success;
+    }
+
+    /** */
+    public String errorMessage() {
+        return errResp;
+    }
+}
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/visor/misc/VisorIdAndTagViewTask.java b/modules/core/src/main/java/org/apache/ignite/internal/visor/misc/VisorIdAndTagViewTask.java
new file mode 100644
index 0000000..b98672f
--- /dev/null
+++ b/modules/core/src/main/java/org/apache/ignite/internal/visor/misc/VisorIdAndTagViewTask.java
@@ -0,0 +1,68 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF 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.apache.ignite.internal.visor.misc;
+
+import org.apache.ignite.IgniteException;
+import org.apache.ignite.internal.cluster.IgniteClusterEx;
+import org.apache.ignite.internal.processors.task.GridInternal;
+import org.apache.ignite.internal.processors.task.GridVisorManagementTask;
+import org.apache.ignite.internal.visor.VisorJob;
+import org.apache.ignite.internal.visor.VisorOneNodeTask;
+import org.jetbrains.annotations.Nullable;
+
+/**
+ *
+ */
+@GridInternal
+@GridVisorManagementTask
+public class VisorIdAndTagViewTask extends VisorOneNodeTask<Void, VisorIdAndTagViewTaskResult> {
+    /** */
+    private static final long serialVersionUID = 0L;
+
+    /** {@inheritDoc} */
+    @Override protected VisorJob<Void, VisorIdAndTagViewTaskResult> job(Void arg) {
+        return new IdAndTagViewJob(arg, debug);
+    }
+
+    private static class IdAndTagViewJob extends VisorJob<Void, VisorIdAndTagViewTaskResult> {
+        /** */
+        private static final long serialVersionUID = 0L;
+
+        /**
+         * Create job with specified argument.
+         *
+         * @param arg Job argument.
+         * @param debug Flag indicating whether debug information should be printed into node log.
+         */
+        IdAndTagViewJob(Void arg, boolean debug) {
+            super(arg, debug);
+        }
+
+        /** {@inheritDoc} */
+        @Override protected VisorIdAndTagViewTaskResult run(@Nullable Void arg) throws IgniteException {
+            return view();
+        }
+
+        /** */
+        private VisorIdAndTagViewTaskResult view() {
+            IgniteClusterEx cl = ignite.cluster();
+
+            return new VisorIdAndTagViewTaskResult(cl.id(), cl.tag());
+        }
+    }
+}
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/visor/misc/VisorIdAndTagViewTaskResult.java b/modules/core/src/main/java/org/apache/ignite/internal/visor/misc/VisorIdAndTagViewTaskResult.java
new file mode 100644
index 0000000..3efe965
--- /dev/null
+++ b/modules/core/src/main/java/org/apache/ignite/internal/visor/misc/VisorIdAndTagViewTaskResult.java
@@ -0,0 +1,74 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF 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.apache.ignite.internal.visor.misc;
+
+import java.io.IOException;
+import java.io.ObjectInput;
+import java.io.ObjectOutput;
+import java.util.UUID;
+import org.apache.ignite.internal.dto.IgniteDataTransferObject;
+
+/**
+ *
+ */
+public class VisorIdAndTagViewTaskResult extends IgniteDataTransferObject {
+    /** */
+    private static final long serialVersionUID = 0L;
+
+    /** */
+    private UUID id;
+
+    /** */
+    private String tag;
+
+    /** Default constructor. */
+    public VisorIdAndTagViewTaskResult() {
+        // No-op.
+    }
+
+    /**
+     * @param id Cluster ID.
+     * @param tag Cluster tag.
+     */
+    public VisorIdAndTagViewTaskResult(UUID id, String tag) {
+        this.id = id;
+        this.tag = tag;
+    }
+
+    /** {@inheritDoc} */
+    @Override protected void writeExternalData(ObjectOutput out) throws IOException {
+        out.writeObject(id);
+        out.writeObject(tag);
+    }
+
+    /** {@inheritDoc} */
+    @Override protected void readExternalData(byte protoVer, ObjectInput in) throws IOException, ClassNotFoundException {
+        id = (UUID)in.readObject();
+        tag = (String)in.readObject();
+    }
+
+    /** */
+    public UUID id() {
+        return id;
+    }
+
+    /** */
+    public String tag() {
+        return tag;
+    }
+}
diff --git a/modules/core/src/main/java/org/apache/ignite/mxbean/IgniteClusterMXBean.java b/modules/core/src/main/java/org/apache/ignite/mxbean/IgniteClusterMXBean.java
new file mode 100644
index 0000000..46856d1
--- /dev/null
+++ b/modules/core/src/main/java/org/apache/ignite/mxbean/IgniteClusterMXBean.java
@@ -0,0 +1,56 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF 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.apache.ignite.mxbean;
+
+import java.util.UUID;
+import javax.management.JMException;
+
+/**
+ * MX Bean allows to access information about cluster ID and tag and change tag.
+ */
+@MXBeanDescription("MBean that provides access to information about cluster ID and tag.")
+public interface IgniteClusterMXBean {
+    /**
+     * Gets cluster ID.
+     *
+     * @return Cluster ID.
+     */
+    @MXBeanDescription("Unique identifier of the cluster.")
+    public UUID getId();
+
+    /**
+     * Gets current cluster tag.
+     *
+     * @return Current cluster tag.
+     */
+    @MXBeanDescription("User-defined cluster tag.")
+    public String getTag();
+
+    /**
+     * Changes cluster tag to provided value.
+     *
+     * @param newTag New value to be set as cluster tag.
+     * @throws JMException If provided value failed validation or concurrent change tag operation succeeded.
+     */
+    @MXBeanDescription("Set new cluster tag value.")
+    @MXBeanParametersNames("newTag")
+    @MXBeanParametersDescriptions(
+        "New tag value to be set."
+    )
+    public void tag(String newTag) throws JMException;
+}
diff --git a/modules/core/src/main/resources/META-INF/classnames.properties b/modules/core/src/main/resources/META-INF/classnames.properties
index 1077ad9..28aa0c4 100644
--- a/modules/core/src/main/resources/META-INF/classnames.properties
+++ b/modules/core/src/main/resources/META-INF/classnames.properties
@@ -2083,6 +2083,8 @@ org.apache.ignite.internal.visor.baseline.VisorBaselineTaskArg
 org.apache.ignite.internal.visor.baseline.VisorBaselineTaskResult
 org.apache.ignite.internal.visor.baseline.VisorBaselineViewTask
 org.apache.ignite.internal.visor.baseline.VisorBaselineViewTask$VisorBaselineViewJob
+org.apache.ignite.internal.visor.misc.VisorIdAndTagViewTaskResult
+org.apache.ignite.internal.visor.misc.VisorClusterChangeTagTaskResult
 org.apache.ignite.internal.visor.binary.VisorBinaryMetadata
 org.apache.ignite.internal.visor.binary.VisorBinaryMetadataCollectorTask
 org.apache.ignite.internal.visor.binary.VisorBinaryMetadataCollectorTask$VisorBinaryCollectMetadataJob
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/cluster/IgniteClusterIdTagTest.java b/modules/core/src/test/java/org/apache/ignite/internal/cluster/IgniteClusterIdTagTest.java
index 458e89f..4fabcf3 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/cluster/IgniteClusterIdTagTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/cluster/IgniteClusterIdTagTest.java
@@ -19,13 +19,19 @@ package org.apache.ignite.internal.cluster;
 
 import java.util.UUID;
 import java.util.concurrent.atomic.AtomicBoolean;
+import java.util.concurrent.atomic.AtomicReference;
 import org.apache.ignite.Ignite;
 import org.apache.ignite.IgniteCheckedException;
+import org.apache.ignite.IgniteCluster;
 import org.apache.ignite.configuration.DataRegionConfiguration;
 import org.apache.ignite.configuration.DataStorageConfiguration;
 import org.apache.ignite.configuration.IgniteConfiguration;
+import org.apache.ignite.events.ClusterTagUpdatedEvent;
+import org.apache.ignite.events.Event;
 import org.apache.ignite.events.EventType;
 import org.apache.ignite.internal.IgniteEx;
+import org.apache.ignite.lang.IgniteBiPredicate;
+import org.apache.ignite.lang.IgnitePredicate;
 import org.apache.ignite.testframework.GridTestUtils;
 import org.apache.ignite.testframework.junits.common.GridCommonAbstractTest;
 import org.junit.Test;
@@ -67,6 +73,8 @@ public class IgniteClusterIdTagTest extends GridCommonAbstractTest {
             cfg.setDataStorageConfiguration(dsCfg);
         }
 
+        cfg.setIncludeEventTypes(EventType.EVTS_ALL);
+
         return cfg;
     }
 
@@ -233,6 +241,50 @@ public class IgniteClusterIdTagTest extends GridCommonAbstractTest {
     }
 
     /**
+     * Verifies restrictions for new tag provided for {@link IgniteCluster#tag(String)} method:
+     * <ol>
+     *     <li>Not null.</li>
+     *     <li>Non-empty.</li>
+     *     <li>Below 280 symbols (max tag length).</li>
+     * </ol>
+     *
+     * @throws Exception If failed.
+     */
+    @Test
+    public void testChangeTagExceptions() throws Exception {
+        IgniteEx ig0 = startGrid(0);
+
+        try {
+            ig0.cluster().tag(null);
+
+            fail("Expected exception has not been thrown.");
+        }
+        catch (IgniteCheckedException e) {
+            assertTrue(e.getMessage().contains("cannot be null"));
+        }
+
+        try {
+            ig0.cluster().tag("");
+
+            fail("Expected exception has not been thrown.");
+        }
+        catch (IgniteCheckedException e) {
+            assertTrue(e.getMessage().contains("should not be empty"));
+        }
+
+        String longString = new String(new char[281]);
+
+        try {
+            ig0.cluster().tag(longString);
+
+            fail("Expected exception has not been thrown.");
+        }
+        catch (IgniteCheckedException e) {
+            assertTrue(e.getMessage().contains("Maximum tag length is exceeded"));
+        }
+    }
+
+    /**
      *  Verifies consistency of tag when set up in inactive and active clusters and on client nodes.
      *
      * @throws Exception If failed.
@@ -243,18 +295,15 @@ public class IgniteClusterIdTagTest extends GridCommonAbstractTest {
 
         IgniteEx ig0 = startGrid(0);
 
-        boolean expectedExceptionThrown = false;
-
         try {
             ig0.cluster().tag(CUSTOM_TAG_0);
+
+            fail("Expected exception has not been thrown.");
         }
         catch (IgniteCheckedException e) {
-            if (e.getMessage().contains("Can not change cluster tag on inactive cluster."))
-                expectedExceptionThrown = true;
+            assertTrue(e.getMessage().contains("Can not change cluster tag on inactive cluster."));
         }
 
-        assertTrue(expectedExceptionThrown);
-
         IgniteEx ig1 = startGrid(1);
 
         assertEquals(ig0.cluster().tag(), ig1.cluster().tag());
@@ -279,10 +328,100 @@ public class IgniteClusterIdTagTest extends GridCommonAbstractTest {
 
         stopAllGrids();
 
-        ig0 = startGrid(0);
+        startGrid(0);
 
         ig1 = startGrid(1);
 
         assertEquals(CUSTOM_TAG_0, ig1.cluster().tag());
     }
+
+    /**
+     * Verifies that event is fired when tag change request sent by user is completed.
+     *
+     * @throws Exception If failed.
+     */
+    @Test
+    public void testTagChangedEvent() throws Exception {
+        IgniteEx ig = startGrid(0);
+
+        UUID clusterId = ig.cluster().id();
+        String generatedTag = ig.cluster().tag();
+
+        AtomicReference<UUID> clusterIdFromEvent = new AtomicReference<>(null);
+        AtomicReference<String> oldTagFromEvent = new AtomicReference<>(null);
+        AtomicReference<String> newTagFromEvent = new AtomicReference<>(null);
+
+        AtomicBoolean evtFired = new AtomicBoolean(false);
+
+        ig.events().localListen((evt) ->
+            {
+                evtFired.set(true);
+
+                ClusterTagUpdatedEvent tagUpdatedEvt = (ClusterTagUpdatedEvent)evt;
+
+                clusterIdFromEvent.set(tagUpdatedEvt.clusterId());
+                oldTagFromEvent.set(tagUpdatedEvt.previousTag());
+                newTagFromEvent.set(tagUpdatedEvt.newTag());
+
+                return true;
+            },
+            EventType.EVT_CLUSTER_TAG_UPDATED);
+
+        ig.cluster().tag(CUSTOM_TAG_0);
+
+        assertTrue(GridTestUtils.waitForCondition(evtFired::get, 10_000));
+
+        assertEquals(clusterId, clusterIdFromEvent.get());
+        assertEquals(generatedTag, oldTagFromEvent.get());
+        assertEquals(CUSTOM_TAG_0, newTagFromEvent.get());
+    }
+
+    /**
+     * Verifies that event about cluster tag update is fired on remote nodes as well.
+     *
+     * @throws Exception If failed.
+     */
+    @Test
+    public void testTagChangedEventMultinodeWithRemoteFilter() throws Exception {
+        IgniteEx ig0 = startGrid(0);
+
+        IgniteEx ig1 = startGrid(1);
+
+        UUID clusterId = ig0.cluster().id();
+        String generatedTag = ig0.cluster().tag();
+
+        AtomicReference<UUID> eventNodeId = new AtomicReference<>(null);
+
+        AtomicReference<UUID> clusterIdFromEvent = new AtomicReference<>(null);
+        AtomicReference<String> oldTagFromEvent = new AtomicReference<>(null);
+        AtomicReference<String> newTagFromEvent = new AtomicReference<>(null);
+
+        AtomicBoolean evtFired = new AtomicBoolean(false);
+
+        ig0.events(ig0.cluster().forRemotes()).remoteListen(
+            (IgniteBiPredicate<UUID, Event>)(uuid, event) -> {
+                eventNodeId.set(uuid);
+
+                evtFired.set(true);
+
+                ClusterTagUpdatedEvent tagUpdatedEvt = (ClusterTagUpdatedEvent)event;
+
+                clusterIdFromEvent.set(tagUpdatedEvt.clusterId());
+                oldTagFromEvent.set(tagUpdatedEvt.previousTag());
+                newTagFromEvent.set(tagUpdatedEvt.newTag());
+
+                return true;
+            },
+            (IgnitePredicate<Event>)event -> event.type() == EventType.EVT_CLUSTER_TAG_UPDATED);
+
+        ig0.cluster().tag(CUSTOM_TAG_0);
+
+        assertTrue(GridTestUtils.waitForCondition(evtFired::get, 10_000));
+
+        assertEquals(ig1.localNode().id(), eventNodeId.get());
+
+        assertEquals(clusterId, clusterIdFromEvent.get());
+        assertEquals(generatedTag, oldTagFromEvent.get());
+        assertEquals(CUSTOM_TAG_0, newTagFromEvent.get());
+    }
 }
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/commandline/CommandHandlerParsingTest.java b/modules/core/src/test/java/org/apache/ignite/internal/commandline/CommandHandlerParsingTest.java
index b8acdd0..bb36551 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/commandline/CommandHandlerParsingTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/commandline/CommandHandlerParsingTest.java
@@ -49,6 +49,7 @@ import static java.util.Collections.singletonList;
 import static org.apache.ignite.IgniteSystemProperties.IGNITE_ENABLE_EXPERIMENTAL_COMMAND;
 import static org.apache.ignite.internal.QueryMXBeanImpl.EXPECTED_GLOBAL_QRY_ID_FORMAT;
 import static org.apache.ignite.internal.commandline.CommandList.CACHE;
+import static org.apache.ignite.internal.commandline.CommandList.CLUSTER_CHANGE_TAG;
 import static org.apache.ignite.internal.commandline.CommandList.SET_STATE;
 import static org.apache.ignite.internal.commandline.CommandList.WAL;
 import static org.apache.ignite.internal.commandline.CommonArgParser.CMD_VERBOSE;
@@ -336,7 +337,7 @@ public class CommandHandlerParsingTest {
     @Test
     public void testParseAutoConfirmationFlag() {
         for (CommandList cmdL : CommandList.values()) {
-            // SET_STATE command have mandatory argument, which used in confirmation message.
+            // SET_STATE command has mandatory argument used in confirmation message.
             Command cmd = cmdL != SET_STATE ? cmdL.command() : parseArgs(asList(cmdL.text(), "ACTIVE")).command();
 
             if (cmd.confirmationPrompt() == null)
@@ -346,6 +347,8 @@ public class CommandHandlerParsingTest {
 
             if (cmdL == SET_STATE)
                 args = parseArgs(asList(cmdL.text(), "ACTIVE"));
+            else if (cmdL == CLUSTER_CHANGE_TAG)
+                args = parseArgs(asList(cmdL.text(), "newTagValue"));
             else
                 args = parseArgs(asList(cmdL.text()));
 
@@ -411,6 +414,16 @@ public class CommandHandlerParsingTest {
                     assertEquals("xid1", txTaskArg.getXid());
                     assertEquals(10_000, txTaskArg.getMinDuration().longValue());
                     assertEquals(VisorTxOperation.KILL, txTaskArg.getOperation());
+
+                    break;
+                }
+
+                case CLUSTER_CHANGE_TAG: {
+                    args = parseArgs(asList(cmdL.text(), "newTagValue", "--yes"));
+
+                    checkCommonParametersCorrectlyParsed(cmdL, args, true);
+
+                    break;
                 }
 
                 default:
@@ -631,6 +644,7 @@ public class CommandHandlerParsingTest {
             cmd == CommandList.ENCRYPTION ||
             cmd == CommandList.KILL ||
             cmd == CommandList.SNAPSHOT ||
+            cmd == CommandList.CLUSTER_CHANGE_TAG ||
             cmd == CommandList.METADATA;
     }
 }
diff --git a/modules/core/src/test/java/org/apache/ignite/util/GridCommandHandlerTest.java b/modules/core/src/test/java/org/apache/ignite/util/GridCommandHandlerTest.java
index 789cc45..79ad548 100644
--- a/modules/core/src/test/java/org/apache/ignite/util/GridCommandHandlerTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/util/GridCommandHandlerTest.java
@@ -97,6 +97,7 @@ import org.apache.ignite.lang.IgnitePredicate;
 import org.apache.ignite.lang.IgniteUuid;
 import org.apache.ignite.plugin.extensions.communication.Message;
 import org.apache.ignite.spi.metric.LongMetric;
+import org.apache.ignite.testframework.GridTestUtils;
 import org.apache.ignite.testframework.junits.WithSystemProperty;
 import org.apache.ignite.transactions.Transaction;
 import org.apache.ignite.transactions.TransactionRollbackException;
@@ -235,6 +236,37 @@ public class GridCommandHandlerTest extends GridCommandHandlerClusterPerMethodAb
     }
 
     /**
+     * Verifies that update-tag action obeys its specification: doesn't allow updating tag on inactive cluster,
+     *
+     * @throws Exception If failed.
+     */
+    @Test
+    public void testClusterChangeTag() throws Exception {
+        final String newTag = "new_tag";
+
+        IgniteEx cl = startGrid(0);
+
+        injectTestSystemOut();
+
+        assertEquals(EXIT_CODE_OK, execute("--change-tag", newTag));
+
+        String out = testOut.toString();
+
+        //because cluster is inactive
+        assertTrue(out.contains("Error has occurred during tag update:"));
+
+        cl.cluster().active(true);
+
+        //because new tag should be non-empty string
+        assertEquals(EXIT_CODE_INVALID_ARGUMENTS, execute("--change-tag", ""));
+
+        assertEquals(EXIT_CODE_OK, execute("--change-tag", newTag));
+
+        boolean tagUpdated = GridTestUtils.waitForCondition(() -> newTag.equals(cl.cluster().tag()), 10_000);
+        assertTrue("Tag has not been updated in 10 seconds", tagUpdated);
+    }
+
+    /**
      * Test deactivation works via control.sh
      *
      * @throws Exception If failed.
@@ -399,6 +431,8 @@ public class GridCommandHandlerTest extends GridCommandHandlerClusterPerMethodAb
      */
     @Test
     public void testState() throws Exception {
+        final String newTag = "new_tag";
+
         Ignite ignite = startGrids(1);
 
         assertFalse(ignite.cluster().active());
@@ -409,6 +443,14 @@ public class GridCommandHandlerTest extends GridCommandHandlerClusterPerMethodAb
 
         assertContains(log, testOut.toString(), "Cluster is inactive");
 
+        String out = testOut.toString();
+
+        UUID clId = ignite.cluster().id();
+        String clTag = ignite.cluster().tag();
+
+        assertTrue(out.contains("Cluster  ID: " + clId));
+        assertTrue(out.contains("Cluster tag: " + clTag));
+
         ignite.cluster().active(true);
 
         assertTrue(ignite.cluster().active());
@@ -426,6 +468,25 @@ public class GridCommandHandlerTest extends GridCommandHandlerClusterPerMethodAb
         assertEquals(EXIT_CODE_OK, execute("--state"));
 
         assertContains(log, testOut.toString(), "Cluster is active (read-only)");
+
+        boolean tagUpdated = GridTestUtils.waitForCondition(() -> {
+            try {
+                ignite.cluster().tag(newTag);
+            }
+            catch (IgniteCheckedException e) {
+                return false;
+            }
+
+            return true;
+        }, 10_000);
+
+        assertTrue("Tag has not been updated in 10 seconds.", tagUpdated);
+
+        assertEquals(EXIT_CODE_OK, execute("--state"));
+
+        out = testOut.toString();
+
+        assertTrue(out.contains("Cluster tag: " + newTag));
     }
 
     /**
diff --git a/modules/core/src/test/resources/org.apache.ignite.util/GridCommandHandlerClusterByClassTest_help.output b/modules/core/src/test/resources/org.apache.ignite.util/GridCommandHandlerClusterByClassTest_help.output
index 9660e9c..afd8d87 100644
--- a/modules/core/src/test/resources/org.apache.ignite.util/GridCommandHandlerClusterByClassTest_help.output
+++ b/modules/core/src/test/resources/org.apache.ignite.util/GridCommandHandlerClusterByClassTest_help.output
@@ -125,6 +125,9 @@ This utility can do the following commands:
     Parameters:
       snapshot_name  - Snapshot name.
 
+  Change cluster tag to new value:
+    control.(sh|bat) --change-tag newTagValue [--yes]
+
   Print metadata command help:
     control.(sh|bat) --meta help
 
diff --git a/modules/core/src/test/resources/org.apache.ignite.util/GridCommandHandlerClusterByClassWithSSLTest_help.output b/modules/core/src/test/resources/org.apache.ignite.util/GridCommandHandlerClusterByClassWithSSLTest_help.output
index 9660e9c..afd8d87 100644
--- a/modules/core/src/test/resources/org.apache.ignite.util/GridCommandHandlerClusterByClassWithSSLTest_help.output
+++ b/modules/core/src/test/resources/org.apache.ignite.util/GridCommandHandlerClusterByClassWithSSLTest_help.output
@@ -125,6 +125,9 @@ This utility can do the following commands:
     Parameters:
       snapshot_name  - Snapshot name.
 
+  Change cluster tag to new value:
+    control.(sh|bat) --change-tag newTagValue [--yes]
+
   Print metadata command help:
     control.(sh|bat) --meta help