You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@ignite.apache.org by ir...@apache.org on 2020/01/09 13:15:14 UTC

[ignite] branch master updated: IGNITE-12225 Add enum for cluster state - Fixes #7194.

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

irakov pushed a commit to branch master
in repository https://gitbox.apache.org/repos/asf/ignite.git


The following commit(s) were added to refs/heads/master by this push:
     new 2a8ab4a  IGNITE-12225 Add enum for cluster state - Fixes #7194.
2a8ab4a is described below

commit 2a8ab4a5058e0ae0abd2317d0a798805ab1abb46
Author: Sergey Antonov <an...@gmail.com>
AuthorDate: Thu Jan 9 16:14:17 2020 +0300

    IGNITE-12225 Add enum for cluster state - Fixes #7194.
    
    Signed-off-by: Ivan Rakov <ir...@apache.org>
---
 .../internal/jdbc2/JdbcStreamingSelfTest.java      |   91 +-
 .../ignite/jdbc/JdbcErrorsAbstractSelfTest.java    |    9 +-
 .../main/java/org/apache/ignite/IgniteCluster.java |   19 +-
 .../org/apache/ignite/cluster/ClusterState.java    |   73 +
 .../ignite/configuration/IgniteConfiguration.java  |   76 +-
 .../ignite/events/ClusterActivationEvent.java      |    5 +-
 ...tionEvent.java => ClusterStateChangeEvent.java} |   66 +-
 .../java/org/apache/ignite/events/EventType.java   |   12 +
 .../org/apache/ignite/internal/IgniteKernal.java   |   32 +-
 .../internal/client/GridClientClusterState.java    |   20 +-
 .../client/impl/GridClientClusterStateImpl.java    |   29 +-
 .../impl/connection/GridClientConnection.java      |   25 +-
 .../connection/GridClientNioTcpConnection.java     |   57 +-
 .../internal/cluster/IgniteClusterAsyncImpl.java   |    9 +-
 .../ignite/internal/cluster/IgniteClusterImpl.java |   23 +-
 .../internal/commandline/ActivateCommand.java      |    7 +-
 .../ClusterReadOnlyModeDisableCommand.java         |   72 -
 .../ClusterReadOnlyModeEnableCommand.java          |   72 -
 .../commandline/ClusterStateChangeCommand.java     |  105 ++
 .../ignite/internal/commandline/Command.java       |   73 +-
 .../internal/commandline/CommandHandler.java       |    1 +
 .../ignite/internal/commandline/CommandList.java   |   11 +-
 .../internal/commandline/DeactivateCommand.java    |    7 +-
 .../ignite/internal/commandline/StateCommand.java  |   26 +-
 .../internal/commandline/cache/CacheCommands.java  |   45 +-
 .../managers/discovery/GridDiscoveryManager.java   |   11 +-
 .../internal/processors/cache/ExchangeActions.java |   13 +-
 .../cache/GridCachePartitionExchangeManager.java   |   77 +-
 .../processors/cache/StateChangeRequest.java       |   34 +-
 .../dht/GridDhtTopologyFutureAdapter.java          |    3 +-
 .../dht/IgniteClusterReadOnlyException.java}       |   33 +-
 .../preloader/GridDhtPartitionsExchangeFuture.java |   16 +-
 .../cluster/ChangeGlobalStateFinishMessage.java    |   26 +-
 .../cluster/ChangeGlobalStateMessage.java          |   29 +-
 .../cluster/DiscoveryDataClusterState.java         |  111 +-
 .../cluster/GridClusterStateProcessor.java         |  538 ++++----
 .../cluster/IGridClusterStateProcessor.java        |   38 +-
 .../processors/datastreamer/DataStreamerImpl.java  |   12 +-
 .../internal/processors/odbc/SqlStateCode.java     |    4 +-
 .../internal/processors/rest/GridRestCommand.java  |   19 +-
 .../processors/rest/GridRestProcessor.java         |    6 +-
 ...est.java => GridClientClusterStateRequest.java} |   40 +-
 .../client/message/GridClientStateRequest.java     |    3 +-
 ...a => GridChangeClusterStateCommandHandler.java} |   32 +-
 .../rest/protocols/tcp/GridTcpRestNioListener.java |   21 +-
 ...quest.java => GridRestClusterStateRequest.java} |   26 +-
 .../org/apache/ignite/mxbean/IgniteMXBean.java     |   39 +-
 .../main/resources/META-INF/classnames.properties  |    2 +-
 .../cluster/GridClusterStateChangeSelfTest.java    |  144 ++
 .../commandline/CommandHandlerParsingTest.java     |   67 +-
 .../internal/encryption/EncryptionMXBeanTest.java  |    5 +-
 .../ignite/internal/metric/JmxExporterSpiTest.java |   10 +-
 .../cache/ActiveOnStartPropertyTest.java           |  149 +++
 .../cache/AutoActivationPropertyTest.java          |  128 ++
 ...ActiveStateChangeWithNodeOutOfBaselineTest.java |  134 ++
 .../cache/ClusterReadOnlyModeAbstractTest.java     |    3 +-
 .../processors/cache/ClusterReadOnlyModeTest.java  |    4 +-
 .../processors/cache/ClusterStateAbstractTest.java |  395 ++----
 .../cache/ClusterStateClientAbstractTest.java      |   58 +
 ... => ClusterStateClientPartitionedSelfTest.java} |   21 +-
 ...a => ClusterStateClientReplicatedSelfTest.java} |   21 +-
 .../cache/ClusterStateNoRebalanceAbstractTest.java |  219 +++
 ...=> ClusterStateNoRebalancePartitionedTest.java} |   21 +-
 ... => ClusterStateNoRebalanceReplicatedTest.java} |   21 +-
 .../cache/ClusterStateOnStartPropertyTest.java     |  396 ++++++
 .../cache/ClusterStatePartitionedSelfTest.java     |   14 +-
 .../cache/ClusterStateReplicatedSelfTest.java      |   15 +-
 .../cache/ClusterStateServerAbstractTest.java      |  206 +++
 .../processors/cache/ClusterStateTestUtils.java    |  124 ++
 .../cache/ClusterStateThinClientAbstractTest.java  |   80 ++
 ...ClusterStateThinClientPartitionedSelfTest.java} |   21 +-
 ... ClusterStateThinClientReplicatedSelfTest.java} |   21 +-
 .../cache/IgniteClusterActivateDeactivateTest.java | 1388 ++++++++++++--------
 ...usterActivateDeactivateTestWithPersistence.java |  259 ++--
 .../baseline/ClusterActivationEventTest.java       |  296 +++--
 .../ClusterActivationEventWithPersistenceTest.java |   50 +
 .../standbycluster/AbstractNodeJoinTemplate.java   |    8 +-
 .../cluster/ClusterReadOnlyModeNodeJoinTest.java   |    8 +-
 .../cluster/ClusterReadOnlyModeSelfTest.java       |   86 +-
 .../cluster/ClusterStateChangeEventTest.java       |  159 +++
 ...ClusterStateChangeEventWithPersistenceTest.java |   36 +
 .../apache/ignite/testframework/GridTestUtils.java |   41 +-
 .../junits/multijvm/IgniteClusterProcessProxy.java |    5 +-
 .../IgniteBasicWithPersistenceTestSuite.java       |    4 +
 .../testsuites/IgniteCacheMvccTestSuite5.java      |   12 +
 .../ignite/testsuites/IgniteCacheTestSuite5.java   |   12 +
 .../testsuites/IgniteKernalSelfTestSuite.java      |    4 +-
 .../ignite/testsuites/IgnitePdsTestSuite.java      |    7 +
 .../ignite/testsuites/IgnitePdsTestSuite2.java     |    6 +
 .../testsuites/IgniteStandByClusterSuite.java      |    4 +-
 .../util/GridCommandHandlerAbstractTest.java       |   18 +-
 .../util/GridCommandHandlerClusterByClassTest.java |   14 +
 ...ridCommandHandlerClusterByClassWithSSLTest.java |   28 +
 .../ignite/util/GridCommandHandlerSslTest.java     |    7 +-
 .../apache/ignite/util/GridCommandHandlerTest.java |  136 +-
 .../ignite/util/GridCommandHandlerTestUtils.java   |   37 +
 .../ignite/util/GridCommandHandlerWithSSLTest.java |   28 +
 .../org.apache.ignite.util/control.sh_help.output  |   19 +-
 .../processors/query/h2/IgniteH2Indexing.java      |    4 +-
 .../processors/query/h2/dml/DmlBatchSender.java    |    4 +-
 ...usterReadOnlyModeDoesNotBreakSqlSelectTest.java |  121 ++
 .../cache/ttl/CacheTtlReadOnlyModeSelfTest.java    |   15 +-
 ...teCacheWithIndexingAndPersistenceTestSuite.java |    4 +
 .../IgniteCacheWithIndexingTestSuite.java          |    3 +
 ...ndHandlerIndexingClusterByClassWithSSLTest.java |   28 +
 .../GridCommandHandlerIndexingWithSSLTest.java     |   28 +
 .../ApiParity/ClusterParityTest.cs                 |    6 +-
 .../protocols/http/jetty/GridJettyRestHandler.java |   58 +-
 .../spring/src/test/config/state/cluster-state.xml |   30 +
 .../cluster/ClusterStateXmlPropertiesTest.java     |   90 ++
 .../ignite/testsuites/IgniteSpringTestSuite.java   |    5 +-
 111 files changed, 5292 insertions(+), 2151 deletions(-)

diff --git a/modules/clients/src/test/java/org/apache/ignite/internal/jdbc2/JdbcStreamingSelfTest.java b/modules/clients/src/test/java/org/apache/ignite/internal/jdbc2/JdbcStreamingSelfTest.java
index 8162433..219c496 100644
--- a/modules/clients/src/test/java/org/apache/ignite/internal/jdbc2/JdbcStreamingSelfTest.java
+++ b/modules/clients/src/test/java/org/apache/ignite/internal/jdbc2/JdbcStreamingSelfTest.java
@@ -32,7 +32,7 @@ import org.apache.ignite.binary.BinaryObjectBuilder;
 import org.apache.ignite.configuration.CacheConfiguration;
 import org.apache.ignite.configuration.ConnectorConfiguration;
 import org.apache.ignite.configuration.IgniteConfiguration;
-import org.apache.ignite.internal.cluster.ClusterReadOnlyModeCheckedException;
+import org.apache.ignite.internal.processors.cache.distributed.dht.IgniteClusterReadOnlyException;
 import org.apache.ignite.internal.processors.query.QueryUtils;
 import org.apache.ignite.internal.util.typedef.X;
 import org.apache.ignite.internal.util.typedef.internal.U;
@@ -46,6 +46,8 @@ import org.junit.Test;
 import static org.apache.ignite.IgniteJdbcDriver.CFG_URL_PREFIX;
 import static org.apache.ignite.cache.CacheMode.PARTITIONED;
 import static org.apache.ignite.cache.CacheWriteSynchronizationMode.FULL_SYNC;
+import static org.apache.ignite.cluster.ClusterState.ACTIVE;
+import static org.apache.ignite.cluster.ClusterState.ACTIVE_READ_ONLY;
 
 /**
  * Data streaming test.
@@ -169,51 +171,34 @@ public class JdbcStreamingSelfTest extends JdbcThinAbstractSelfTest {
      */
     @Test
     public void testStreamedInsertFailsOnReadOnlyMode() throws Exception {
-        grid(0).cluster().readOnly(true);
-
-        try {
-            assertTrue(grid(0).cluster().readOnly());
+        try (Connection conn = createStreamedConnection(true)) {
+            populateData(conn, 0, 1);
 
-            boolean failed = false;
+            grid(0).cluster().state(ACTIVE_READ_ONLY);
 
-            try (Connection ordinalCon = createOrdinaryConnection();
-                 Statement selectStmt = ordinalCon.createStatement()
-            ) {
-                try (ResultSet rs = selectStmt.executeQuery("select count(*) from PUBLIC.Person")) {
-                    assertTrue(rs.next());
+            try {
+                assertEquals(ACTIVE_READ_ONLY, grid(0).cluster().state());
 
-                    assertEquals(0, rs.getLong(1));
-                }
+                try (Connection ordinalCon = createOrdinaryConnection()) {
+                    assertEquals(1, countPersons(ordinalCon));
 
-                try (Connection conn = createStreamedConnection(true)) {
-                    try (PreparedStatement stmt =
-                             conn.prepareStatement("insert into PUBLIC.Person(\"id\", \"name\") values (?, ?)")
-                    ) {
-                        for (int i = 1; i <= 2; i++) {
-                            stmt.setInt(1, i);
-                            stmt.setString(2, nameForId(i));
+                    try {
+                        populateData(conn, 1, 100);
 
-                            stmt.executeUpdate();
-                        }
+                        fail("Insert should be failed!");
                     }
-                }
-                catch (Exception e) {
-                    log.error("Insert failed", e);
-
-                    failed = X.hasCause(e, ClusterReadOnlyModeCheckedException.class);
-                }
+                    catch (Exception e) {
+                        log.error("Insert failed", e);
 
-                try (ResultSet rs = selectStmt.executeQuery("select count(*) from PUBLIC.Person")) {
-                    assertTrue(rs.next());
+                        assertTrue("Wrong exception", X.hasCause(e, IgniteClusterReadOnlyException.class));
+                    }
 
-                    assertEquals("Insert should be failed", 0, rs.getLong(1));
+                    assertEquals("Insert should be failed", 1, countPersons(ordinalCon));
                 }
             }
-
-            assertTrue(failed);
-        }
-        finally {
-            grid(0).cluster().readOnly(false);
+            finally {
+                grid(0).cluster().state(ACTIVE);
+            }
         }
     }
 
@@ -382,4 +367,38 @@ public class JdbcStreamingSelfTest extends JdbcThinAbstractSelfTest {
 
         return ((BinaryObject)o).field("name");
     }
+
+    /**
+     * Populates data to the table.
+     *
+     * @param conn Connection.
+     * @param from First person id.
+     * @param count Number of persons.
+     * @throws SQLException If something goes wrong.
+     */
+    private void populateData(Connection conn, int from, int count) throws SQLException {
+        try (PreparedStatement stmt = conn.prepareStatement("insert into PUBLIC.Person(\"id\", \"name\") values (?, ?)")) {
+            for (int i = from; i < from + count; i++) {
+                stmt.setInt(1, i);
+                stmt.setString(2, nameForId(i));
+
+                stmt.executeUpdate();
+            }
+        }
+    }
+
+    /**
+     * @param conn Connection.
+     * @return Size of PUBLIC.Person table.
+     * @throws SQLException If something goes wrong.
+     */
+    private long countPersons(Connection conn) throws SQLException {
+        try (Statement selectStmt = conn.createStatement()) {
+            try (ResultSet rs = selectStmt.executeQuery("select count(*) from PUBLIC.Person")) {
+                assertTrue("Result set is empty!", rs.next());
+
+                return rs.getLong(1);
+            }
+        }
+    }
 }
diff --git a/modules/clients/src/test/java/org/apache/ignite/jdbc/JdbcErrorsAbstractSelfTest.java b/modules/clients/src/test/java/org/apache/ignite/jdbc/JdbcErrorsAbstractSelfTest.java
index b2d3000..232492b 100644
--- a/modules/clients/src/test/java/org/apache/ignite/jdbc/JdbcErrorsAbstractSelfTest.java
+++ b/modules/clients/src/test/java/org/apache/ignite/jdbc/JdbcErrorsAbstractSelfTest.java
@@ -36,6 +36,7 @@ import javax.cache.integration.CacheWriterException;
 import org.apache.ignite.cache.CacheInterceptorAdapter;
 import org.apache.ignite.cache.QueryEntity;
 import org.apache.ignite.cache.store.CacheStoreAdapter;
+import org.apache.ignite.cluster.ClusterState;
 import org.apache.ignite.configuration.CacheConfiguration;
 import org.apache.ignite.internal.IgniteEx;
 import org.apache.ignite.lang.IgniteCallable;
@@ -756,7 +757,7 @@ public abstract class JdbcErrorsAbstractSelfTest extends GridCommonAbstractTest
             }
         }
 
-        grid(0).cluster().readOnly(true);
+        grid(0).cluster().state(ClusterState.ACTIVE_READ_ONLY);
 
         try {
             checkErrorState((conn) -> {
@@ -766,7 +767,7 @@ public abstract class JdbcErrorsAbstractSelfTest extends GridCommonAbstractTest
             }, "90097", "Failed to execute DML statement. Cluster in read-only mode");
         }
         finally {
-            grid(0).cluster().readOnly(false);
+            grid(0).cluster().state(ClusterState.ACTIVE);
         }
     }
 
@@ -783,7 +784,7 @@ public abstract class JdbcErrorsAbstractSelfTest extends GridCommonAbstractTest
             }
         }
 
-        grid(0).cluster().readOnly(true);
+        grid(0).cluster().state(ClusterState.ACTIVE_READ_ONLY);
 
         try {
             checkErrorState((conn) -> {
@@ -794,7 +795,7 @@ public abstract class JdbcErrorsAbstractSelfTest extends GridCommonAbstractTest
             }, "90097", null);
         }
         finally {
-            grid(0).cluster().readOnly(false);
+            grid(0).cluster().state(ClusterState.ACTIVE);
         }
     }
 
diff --git a/modules/core/src/main/java/org/apache/ignite/IgniteCluster.java b/modules/core/src/main/java/org/apache/ignite/IgniteCluster.java
index 5a5985c..05dcf3e 100644
--- a/modules/core/src/main/java/org/apache/ignite/IgniteCluster.java
+++ b/modules/core/src/main/java/org/apache/ignite/IgniteCluster.java
@@ -26,6 +26,7 @@ import org.apache.ignite.cluster.BaselineNode;
 import org.apache.ignite.cluster.ClusterGroup;
 import org.apache.ignite.cluster.ClusterNode;
 import org.apache.ignite.cluster.ClusterStartNodeResult;
+import org.apache.ignite.cluster.ClusterState;
 import org.apache.ignite.internal.processors.cluster.baseline.autoadjust.BaselineAutoAdjustStatus;
 import org.apache.ignite.lang.IgniteAsyncSupport;
 import org.apache.ignite.lang.IgniteAsyncSupported;
@@ -447,7 +448,9 @@ public interface IgniteCluster extends ClusterGroup, IgniteAsyncSupport {
      * Checks Ignite grid is active or not active.
      *
      * @return {@code True} if grid is active. {@code False} If grid is not active.
+     * @deprecated Use {@link #state()} instead.
      */
+    @Deprecated
     public boolean active();
 
     /**
@@ -455,23 +458,25 @@ public interface IgniteCluster extends ClusterGroup, IgniteAsyncSupport {
      *
      * @param active If {@code True} start activation process. If {@code False} start deactivation process.
      * @throws IgniteException If there is an already started transaction or lock in the same thread.
+     * @deprecated Use {@link #state(ClusterState)} instead.
      */
+    @Deprecated
     public void active(boolean active);
 
     /**
-     * Checks Ignite grid in read-only mode or not.
+     * Gets current cluster state.
      *
-     * @return {@code true} if grid is in read-only mode and {@code false} If grid allows data modification operations.
+     * @return Current cluster state.
      */
-    public boolean readOnly();
+    public ClusterState state();
 
     /**
-     * Enable or disable Ignite grid read-only mode.
+     * Changes current cluster state to given {@code newState} cluster state.
      *
-     * @param readOnly If {@code true} enable read-only mode. If {@code false} disable read-only mode.
-     * @throws IgniteException If Ignite grid isn't active.
+     * @param newState New cluster state.
+     * @throws IgniteException If there is an already started transaction or lock in the same thread.
      */
-    public void readOnly(boolean readOnly) throws IgniteException;
+    public void state(ClusterState newState) throws IgniteException;
 
     /**
      * Gets current baseline topology. If baseline topology was not set, will return {@code null}.
diff --git a/modules/core/src/main/java/org/apache/ignite/cluster/ClusterState.java b/modules/core/src/main/java/org/apache/ignite/cluster/ClusterState.java
new file mode 100644
index 0000000..b87cb98
--- /dev/null
+++ b/modules/core/src/main/java/org/apache/ignite/cluster/ClusterState.java
@@ -0,0 +1,73 @@
+/*
+ * 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.cluster;
+
+import org.jetbrains.annotations.Nullable;
+
+/**
+ * Cluster states.
+ */
+public enum ClusterState {
+    /** Cluster deactivated. Cache operations aren't allowed. */
+    INACTIVE,
+
+    /** Cluster activated. All cache operations are allowed. */
+    ACTIVE,
+
+    /** Cluster activated. Cache read operation allowed, Cache data change operation aren't allowed. */
+    ACTIVE_READ_ONLY;
+
+    /** Enumerated values. */
+    private static final ClusterState[] VALS = values();
+
+    /**
+     * Efficiently gets enumerated value from its ordinal.
+     *
+     * @param ord Ordinal value.
+     * @return Enumerated value or {@code null} if ordinal out of range.
+     */
+    @Nullable public static ClusterState fromOrdinal(int ord) {
+        return ord >= 0 && ord < VALS.length ? VALS[ord] : null;
+    }
+
+    /**
+     * @param state Cluster state
+     * @return {@code True} if cluster in given cluster {@code state} is activated and {@code False} otherwise.
+     */
+    public static boolean active(ClusterState state) {
+        return state != INACTIVE;
+    }
+
+    /**
+     * @param state1 First given state.
+     * @param state2 Second given state.
+     * @return Lesser of given states. The order: {@link #ACTIVE} > {@link #ACTIVE_READ_ONLY} > {@link #INACTIVE}.
+     */
+    public static ClusterState lesserOf(ClusterState state1, ClusterState state2) {
+        if (state1 == state2)
+            return state1;
+
+        if (state1 == INACTIVE || state2 == INACTIVE)
+            return INACTIVE;
+
+        if (state1 == ACTIVE_READ_ONLY || state2 == ACTIVE_READ_ONLY)
+            return ACTIVE_READ_ONLY;
+
+        throw new IllegalArgumentException("Unknown cluster states. state1: " + state1 + ", state2: " + state2);
+    }
+}
diff --git a/modules/core/src/main/java/org/apache/ignite/configuration/IgniteConfiguration.java b/modules/core/src/main/java/org/apache/ignite/configuration/IgniteConfiguration.java
index ce30c3a..22754e9 100644
--- a/modules/core/src/main/java/org/apache/ignite/configuration/IgniteConfiguration.java
+++ b/modules/core/src/main/java/org/apache/ignite/configuration/IgniteConfiguration.java
@@ -39,6 +39,7 @@ import org.apache.ignite.cache.affinity.AffinityFunction;
 import org.apache.ignite.cache.store.CacheStoreSessionListener;
 import org.apache.ignite.cluster.ClusterGroup;
 import org.apache.ignite.cluster.ClusterNode;
+import org.apache.ignite.cluster.ClusterState;
 import org.apache.ignite.compute.ComputeJob;
 import org.apache.ignite.compute.ComputeTask;
 import org.apache.ignite.events.Event;
@@ -48,6 +49,7 @@ import org.apache.ignite.internal.managers.eventstorage.GridEventStorageManager;
 import org.apache.ignite.internal.processors.odbc.ClientListenerProcessor;
 import org.apache.ignite.internal.util.typedef.internal.A;
 import org.apache.ignite.internal.util.typedef.internal.S;
+import org.apache.ignite.internal.util.typedef.internal.U;
 import org.apache.ignite.lang.IgniteAsyncCallback;
 import org.apache.ignite.lang.IgniteInClosure;
 import org.apache.ignite.lang.IgnitePredicate;
@@ -217,10 +219,15 @@ public class IgniteConfiguration {
     @Deprecated
     public static final boolean DFLT_LATE_AFF_ASSIGNMENT = true;
 
+    /** Default value for cluster state on start. */
+    public static final ClusterState DFLT_STATE_ON_START = ClusterState.ACTIVE;
+
     /** Default value for active on start flag. */
+    @Deprecated
     public static final boolean DFLT_ACTIVE_ON_START = true;
 
     /** Default value for auto-activation flag. */
+    @Deprecated
     public static final boolean DFLT_AUTO_ACTIVATION = true;
 
     /** Default failure detection timeout in millis. */
@@ -537,11 +544,22 @@ public class IgniteConfiguration {
     private DataStorageConfiguration dsCfg;
 
     /** Active on start flag. */
+    @Deprecated
     private boolean activeOnStart = DFLT_ACTIVE_ON_START;
 
+    /** Indicates that activeOnStart property was set explicitly. */
+    private boolean activeOnStartPropSetFlag;
+
     /** Auto-activation flag. */
+    @Deprecated
     private boolean autoActivation = DFLT_AUTO_ACTIVATION;
 
+    /** Indicates that autoActivation property was set explicitly. */
+    private boolean autoActivationPropSetFlag;
+
+    /** Cluster state on start. */
+    private ClusterState clusterStateOnStart;
+
     /** */
     private long longQryWarnTimeout = DFLT_LONG_QRY_WARN_TIMEOUT;
 
@@ -609,12 +627,15 @@ public class IgniteConfiguration {
          * Order alphabetically for maintenance purposes.
          */
         activeOnStart = cfg.isActiveOnStart();
+        activeOnStartPropSetFlag = cfg.activeOnStartPropSetFlag;
         addrRslvr = cfg.getAddressResolver();
         allResolversPassReq = cfg.isAllSegmentationResolversPassRequired();
         atomicCfg = cfg.getAtomicConfiguration();
         authEnabled = cfg.isAuthenticationEnabled();
         autoActivation = cfg.isAutoActivationEnabled();
+        autoActivationPropSetFlag = cfg.autoActivationPropSetFlag;
         binaryCfg = cfg.getBinaryConfiguration();
+        clusterStateOnStart = cfg.getClusterStateOnStart();
         dsCfg = cfg.getDataStorageConfiguration();
         memCfg = cfg.getMemoryConfiguration();
         pstCfg = cfg.getPersistentStoreConfiguration();
@@ -2661,11 +2682,13 @@ public class IgniteConfiguration {
      * <p>
      * Default value is {@link #DFLT_ACTIVE_ON_START}.
      * <p>
-     * This flag is ignored when {@link DataStorageConfiguration} is present:
-     * cluster is always inactive on start when Ignite Persistence is enabled.
+     * This flag is ignored when Ignite Persistence is enabled see {@link DataStorageConfiguration}.
+     * Cluster is always inactive on start when Ignite Persistence is enabled.
      *
      * @return Active on start flag value.
+     * @deprecated Use {@link #getClusterStateOnStart()} instead.
      */
+    @Deprecated
     public boolean isActiveOnStart() {
         return activeOnStart;
     }
@@ -2680,10 +2703,16 @@ public class IgniteConfiguration {
      * @param activeOnStart Active on start flag value.
      * @return {@code this} instance.
      * @see #isActiveOnStart()
+     * @deprecated Use {@link #setClusterStateOnStart(ClusterState)} instead.
      */
+    @Deprecated
     public IgniteConfiguration setActiveOnStart(boolean activeOnStart) {
+        U.warn(log, "Property activeOnStart deprecated. Use clusterStateOnStart instead.");
+
         this.activeOnStart = activeOnStart;
 
+        activeOnStartPropSetFlag = true;
+
         return this;
     }
 
@@ -2698,7 +2727,9 @@ public class IgniteConfiguration {
      * <p>
      *
      * @return Auto activation enabled flag value.
+     * @deprecated Use {@link IgniteConfiguration#getClusterStateOnStart()} instead.
      */
+    @Deprecated
     public boolean isAutoActivationEnabled() {
         return autoActivation;
     }
@@ -2710,10 +2741,51 @@ public class IgniteConfiguration {
      * @param autoActivation Auto activation enabled flag value.
      * @return {@code this} instance.
      * @see #isAutoActivationEnabled()
+     * @deprecated Use {@link IgniteConfiguration#setClusterStateOnStart(ClusterState)} instead.
      */
+    @Deprecated
     public IgniteConfiguration setAutoActivationEnabled(boolean autoActivation) {
+        U.warn(log, "Property autoActivation deprecated. Use clusterStateOnStart instead.");
+
         this.autoActivation = autoActivation;
 
+        autoActivationPropSetFlag = true;
+
+        return this;
+    }
+
+    /**
+     * Gets state of cluster on start.
+     * <br/>
+     * For <b>in-memory cluster</b> this state will be applied to the first started node. If
+     * cluster state on start is {@link ClusterState#INACTIVE}, further hode joins will be handled by cluster faster and
+     * manual cluster activation should be performed in order to start working the cluster and caches.
+     * <br/>
+     * For <b>persistent cluster</b> If state is different from {@link ClusterState#INACTIVE} and BaselineTopology is
+     * set (cluster was activated before, for example before cluster restart) as well then cluster moves to given
+     * cluster state when all nodes from the BaselineTopology join the cluster, i.e. manual activation isn't required
+     * in that case.
+     * <p>
+     * Default value is {@link #DFLT_STATE_ON_START}.
+     * <p>
+     *
+     * @return State of cluster on start or {@code null}, if property wasn't set. {@code Null} means that default
+     * value will be used.
+     */
+    public @Nullable ClusterState getClusterStateOnStart() {
+        return clusterStateOnStart;
+    }
+
+    /**
+     * Sets state of cluster on start.
+     *
+     * @param state New cluster state on start.
+     * @return {@code this} for chaining.
+     * @see #getClusterStateOnStart() 
+     */
+    public IgniteConfiguration setClusterStateOnStart(ClusterState state) {
+        this.clusterStateOnStart = state;
+
         return this;
     }
 
diff --git a/modules/core/src/main/java/org/apache/ignite/events/ClusterActivationEvent.java b/modules/core/src/main/java/org/apache/ignite/events/ClusterActivationEvent.java
index 78626ad..e9fef1b 100644
--- a/modules/core/src/main/java/org/apache/ignite/events/ClusterActivationEvent.java
+++ b/modules/core/src/main/java/org/apache/ignite/events/ClusterActivationEvent.java
@@ -23,6 +23,7 @@ import org.apache.ignite.cluster.BaselineNode;
 import org.apache.ignite.cluster.ClusterNode;
 import org.apache.ignite.configuration.IgniteConfiguration;
 import org.apache.ignite.lang.IgnitePredicate;
+import org.jetbrains.annotations.Nullable;
 
 /**
  * Cluster activation event.
@@ -57,7 +58,9 @@ import org.apache.ignite.lang.IgnitePredicate;
  * event storage SPI if they are disabled in Ignite configuration.
  * @see EventType#EVT_CLUSTER_ACTIVATED
  * @see EventType#EVT_CLUSTER_DEACTIVATED
+ * @deprecated Use {@link ClusterStateChangeEvent} instead.
  */
+@Deprecated
 public class ClusterActivationEvent extends EventAdapter {
     /** */
     private static final long serialVersionUID = 0L;
@@ -86,7 +89,7 @@ public class ClusterActivationEvent extends EventAdapter {
      *
      * @return Baseline nodes.
      */
-    public Collection<BaselineNode> baselineNodes() {
+    public @Nullable Collection<BaselineNode> baselineNodes() {
         return baselineNodes;
     }
 }
diff --git a/modules/core/src/main/java/org/apache/ignite/events/ClusterActivationEvent.java b/modules/core/src/main/java/org/apache/ignite/events/ClusterStateChangeEvent.java
similarity index 63%
copy from modules/core/src/main/java/org/apache/ignite/events/ClusterActivationEvent.java
copy to modules/core/src/main/java/org/apache/ignite/events/ClusterStateChangeEvent.java
index 78626ad..7545a40 100644
--- a/modules/core/src/main/java/org/apache/ignite/events/ClusterActivationEvent.java
+++ b/modules/core/src/main/java/org/apache/ignite/events/ClusterStateChangeEvent.java
@@ -1,12 +1,12 @@
 /*
  * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
+ * 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
+ * the License. You may obtain a copy of the License at
  *
- *      http://www.apache.org/licenses/LICENSE-2.0
+ * 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,
@@ -21,11 +21,16 @@ import java.util.Collection;
 import org.apache.ignite.IgniteEvents;
 import org.apache.ignite.cluster.BaselineNode;
 import org.apache.ignite.cluster.ClusterNode;
+import org.apache.ignite.cluster.ClusterState;
 import org.apache.ignite.configuration.IgniteConfiguration;
+import org.apache.ignite.internal.util.typedef.internal.A;
 import org.apache.ignite.lang.IgnitePredicate;
+import org.jetbrains.annotations.Nullable;
+
+import static org.apache.ignite.events.EventType.EVT_CLUSTER_STATE_CHANGED;
 
 /**
- * Cluster activation event.
+ * Cluster state change event.
  * <p>
  * Grid events are used for notification about what happens within the grid. Note that by
  * design Ignite keeps all events generated on the local node locally and it provides
@@ -55,38 +60,65 @@ import org.apache.ignite.lang.IgnitePredicate;
  * by using {@link IgniteConfiguration#getIncludeEventTypes()} method in Ignite configuration. Note that certain
  * events are required for Ignite's internal operations and such events will still be generated but not stored by
  * event storage SPI if they are disabled in Ignite configuration.
- * @see EventType#EVT_CLUSTER_ACTIVATED
- * @see EventType#EVT_CLUSTER_DEACTIVATED
+ * @see EventType#EVT_CLUSTER_STATE_CHANGED
  */
-public class ClusterActivationEvent extends EventAdapter {
+public class ClusterStateChangeEvent extends EventAdapter {
     /** */
     private static final long serialVersionUID = 0L;
 
-    /** Baseline nodes. */
-    private final Collection<BaselineNode> baselineNodes;
+    /** Previous cluster state. */
+    private final ClusterState prevState;
+
+    /** Current cluster state. */
+    private final ClusterState state;
+
+    /** Baseline nodes of current baseline topology. */
+    private final Collection<? extends BaselineNode> baselineNodes;
 
     /**
-     * Creates deployment event with given parameters.
-     *
+     * @param prevState Previous cluster state.
+     * @param state Current cluster state.
+     * @param baselineNodes Collection of baseline nodes. (Optional for in-memory case)
      * @param node Node.
      * @param msg Optional event message.
-     * @param type Event type.
-     * @param baselineNodes Baseline nodes.
      */
-    public ClusterActivationEvent(ClusterNode node, String msg, int type, Collection<BaselineNode> baselineNodes) {
-        super(node, msg, type);
+    public ClusterStateChangeEvent(
+        ClusterState prevState,
+        ClusterState state,
+        @Nullable Collection<? extends BaselineNode> baselineNodes,
+        ClusterNode node,
+        String msg
+    ) {
+        super(node, msg, EVT_CLUSTER_STATE_CHANGED);
 
-        assert baselineNodes != null;
+        A.notNull(prevState, "prevState");
+        A.notNull(state, "state");
 
+        this.state = state;
+        this.prevState = prevState;
         this.baselineNodes = baselineNodes;
     }
 
     /**
+     * @return Previous cluster state.
+     */
+    public ClusterState previousState() {
+        return prevState;
+    }
+
+    /**
+     * @return Current cluster state.
+     */
+    public ClusterState state() {
+        return state;
+    }
+
+    /**
      * Gets baseline nodes.
      *
      * @return Baseline nodes.
      */
-    public Collection<BaselineNode> baselineNodes() {
+    public @Nullable Collection<? extends BaselineNode> baselineNodes() {
         return baselineNodes;
     }
 }
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 284cea5..58dec60 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
@@ -923,6 +923,18 @@ public interface EventType {
     public static final int EVT_CLUSTER_DEACTIVATED = 141;
 
     /**
+     * Built-in event type: Cluster state changed.
+     * <p>
+     * Fired when cluster state changed.
+     * <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 ClusterActivationEvent
+     */
+    public static final int EVT_CLUSTER_STATE_CHANGED = 144;
+
+    /**
      * All checkpoint events. This array can be directly passed into
      * {@link IgniteEvents#localListen(IgnitePredicate, int...)} method to
      * subscribe to all checkpoint events.
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/IgniteKernal.java b/modules/core/src/main/java/org/apache/ignite/internal/IgniteKernal.java
index 8c0a6c7..a4fdd7c 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/IgniteKernal.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/IgniteKernal.java
@@ -93,6 +93,7 @@ import org.apache.ignite.cluster.BaselineNode;
 import org.apache.ignite.cluster.ClusterGroup;
 import org.apache.ignite.cluster.ClusterMetrics;
 import org.apache.ignite.cluster.ClusterNode;
+import org.apache.ignite.cluster.ClusterState;
 import org.apache.ignite.compute.ComputeJob;
 import org.apache.ignite.configuration.AtomicConfiguration;
 import org.apache.ignite.configuration.BinaryConfiguration;
@@ -765,6 +766,16 @@ public class IgniteKernal implements IgniteEx, IgniteMXBean, Externalizable {
         }
     }
 
+    /** {@inheritDoc} */
+    @Override public String clusterState() {
+        return ctx.state().clusterState().state().toString();
+    }
+
+    /** {@inheritDoc} */
+    @Override public long lastClusterStateChangeTime() {
+        return ctx.state().lastStateChangeTime();
+    }
+
     /**
      * @param name New attribute name.
      * @param val New attribute value.
@@ -4605,8 +4616,8 @@ public class IgniteKernal implements IgniteEx, IgniteMXBean, Externalizable {
         reg.register("active", () -> ctx.state().clusterState().active()/*this::active*/, Boolean.class,
             ACTIVE_DESC);
 
-        reg.register("readOnlyMode", this::readOnlyMode, Boolean.class, READ_ONLY_MODE_DESC);
-        reg.register("readOnlyModeDuration", this::getReadOnlyModeDuration, READ_ONLY_MODE_DURATION_DESC);
+        reg.register("clusterState", this::clusterState, String.class, CLUSTER_STATE_DESC);
+        reg.register("lastClusterStateChangeTime", this::lastClusterStateChangeTime, LAST_CLUSTER_STATE_CHANGE_TIME_DESC);
 
         reg.register("userAttributesFormatted", this::getUserAttributesFormatted, List.class,
             USER_ATTRS_FORMATTED_DESC);
@@ -4734,21 +4745,10 @@ public class IgniteKernal implements IgniteEx, IgniteMXBean, Externalizable {
     }
 
     /** {@inheritDoc} */
-    @Override public boolean readOnlyMode() {
-        return ctx.state().publicApiReadOnlyMode();
-    }
-
-    /** {@inheritDoc} */
-    @Override public void readOnlyMode(boolean readOnly) {
-        ctx.state().changeGlobalState(readOnly);
-    }
+    @Override public void clusterState(String state) {
+        ClusterState newState = ClusterState.valueOf(state);
 
-    /** {@inheritDoc} */
-    @Override public long getReadOnlyModeDuration() {
-        if (ctx.state().publicApiReadOnlyMode())
-            return U.currentTimeMillis() - ctx.state().readOnlyModeStateChangeTime();
-        else
-            return 0;
+        cluster().state(newState);
     }
 
     /** {@inheritDoc} */
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 6f924ee..6476a58 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,33 +17,39 @@
 
 package org.apache.ignite.internal.client;
 
+import org.apache.ignite.cluster.ClusterState;
+
 /**
  *  Interface for manage state of grid cluster.
  */
 public interface GridClientClusterState {
     /**
      * @param active {@code True} activate, {@code False} deactivate.
+     * @deprecated Use {@link #state()} instead.
      */
+    @Deprecated
     public void active(boolean active) throws GridClientException;
 
     /**
      * @return {@code Boolean} - Current cluster state. {@code True} active, {@code False} inactive.
+     * @deprecated Use {@link #state(ClusterState)} instead.
      */
+    @Deprecated
     public boolean active() throws GridClientException;
 
     /**
-     * @return {@code True} if the cluster is in read-only mode and {@code false} otherwise.
-     * @throws GridClientException If request current cluster read-only mode failed.
+     * @return Current cluster state.
+     * @throws GridClientException If the request to get the cluster state failed.
      */
-    public boolean readOnly() throws GridClientException;
+    public ClusterState state() throws GridClientException;
 
     /**
-     * Enable or disable Ignite grid read-only mode.
+     * Changes cluster state to {@code newState}.
      *
-     * @param readOnly If {@code True} enable read-only mode. If {@code false} disable read-only mode.
-     * @throws GridClientException If change of read-only mode is failed.
+     * @param newState New cluster state.
+     * @throws GridClientException If the request to change the cluster state failed.
      */
-    public void readOnly(boolean readOnly) throws GridClientException;
+    public void state(ClusterState newState) throws GridClientException;
 
     /**
      * Get the cluster name.
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 4c1331c..2cdcc14 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
@@ -18,16 +18,13 @@
 package org.apache.ignite.internal.client.impl;
 
 import java.util.Collection;
-import java.util.UUID;
-import org.apache.ignite.internal.client.GridClientClosedException;
+import org.apache.ignite.cluster.ClusterState;
 import org.apache.ignite.internal.client.GridClientClusterState;
 import org.apache.ignite.internal.client.GridClientException;
-import org.apache.ignite.internal.client.GridClientFuture;
 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.client.impl.connection.GridClientConnectionResetException;
 
 /**
  *
@@ -53,14 +50,7 @@ public class GridClientClusterStateImpl extends GridClientAbstractProjection<Gri
 
     /** {@inheritDoc} */
     @Override public void active(final boolean active) throws GridClientException {
-        withReconnectHandling(new ClientProjectionClosure<Void>() {
-            @Override public GridClientFuture apply(
-                GridClientConnection conn,
-                UUID nodeId
-            ) throws GridClientConnectionResetException, GridClientClosedException {
-                return conn.changeState(active, nodeId);
-            }
-        }).get();
+        withReconnectHandling((conn, nodeId) -> conn.changeState(active, nodeId)).get();
     }
 
     /** {@inheritDoc} */
@@ -69,20 +59,13 @@ public class GridClientClusterStateImpl extends GridClientAbstractProjection<Gri
     }
 
     /** {@inheritDoc} */
-    @Override public boolean readOnly() throws GridClientException {
-        return withReconnectHandling(GridClientConnection::readOnlyState).get();
+    @Override public ClusterState state() throws GridClientException {
+        return withReconnectHandling(GridClientConnection::state).get();
     }
 
     /** {@inheritDoc} */
-    @Override public void readOnly(boolean readOnly) throws GridClientException {
-        withReconnectHandling(new ClientProjectionClosure<Void>() {
-            @Override public GridClientFuture apply(
-                GridClientConnection conn,
-                UUID nodeId
-            ) throws GridClientConnectionResetException, GridClientClosedException {
-                return conn.changeReadOnlyState(readOnly, nodeId);
-            }
-        }).get();
+    @Override public void state(ClusterState newState) throws GridClientException {
+        withReconnectHandling((con, nodeId) -> con.changeState(newState, nodeId)).get();
     }
 
     /** {@inheritDoc} */
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/client/impl/connection/GridClientConnection.java b/modules/core/src/main/java/org/apache/ignite/internal/client/impl/connection/GridClientConnection.java
index 7214986..329d5ae 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/client/impl/connection/GridClientConnection.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/client/impl/connection/GridClientConnection.java
@@ -26,6 +26,7 @@ import java.util.Set;
 import java.util.UUID;
 import javax.net.ssl.SSLContext;
 
+import org.apache.ignite.cluster.ClusterState;
 import org.apache.ignite.internal.client.GridClientCacheFlag;
 import org.apache.ignite.internal.client.GridClientClosedException;
 import org.apache.ignite.internal.client.GridClientDataMetrics;
@@ -312,38 +313,42 @@ public abstract class GridClientConnection {
      *
      * @param active Active.
      * @param destNodeId Destination node id.
+     * @deprecated Use {@link #changeState(ClusterState, UUID)} instead.
      */
+    @Deprecated
     public abstract GridClientFuture<?> changeState(boolean active, UUID destNodeId)
             throws GridClientClosedException, GridClientConnectionResetException;
 
     /**
-     * Get current grid state.
+     * Changes grid global state.
      *
+     * @param state New cluster state.
      * @param destNodeId Destination node id.
+     * @throws GridClientConnectionResetException In case of error.
+     * @throws GridClientClosedException If client was manually closed before request was sent over network.
      */
-    public abstract GridClientFuture<Boolean> currentState(UUID destNodeId)
+    public abstract GridClientFuture<?> changeState(ClusterState state, UUID destNodeId)
         throws GridClientClosedException, GridClientConnectionResetException;
 
     /**
-     * Get current read-only mode status. If future contains {@code true} - read-only mode enabled, if {@code false} -
-     * read-only mode disabled.
+     * Get current grid state.
      *
      * @param destNodeId Destination node id.
-     * @throws GridClientConnectionResetException In case of error.
-     * @throws GridClientClosedException If client was manually closed before request was sent over network.
+     * @deprecated Use {@link #state(UUID)} instead.
      */
-    public abstract GridClientFuture<Boolean> readOnlyState(UUID destNodeId)
+    @Deprecated
+    public abstract GridClientFuture<Boolean> currentState(UUID destNodeId)
         throws GridClientClosedException, GridClientConnectionResetException;
 
+
     /**
-     * Change read-only mode. Cluster must be activated.
+     * Gets current grid global state.
      *
-     * @param readOnly Read-only mode enabled flag.
      * @param destNodeId Destination node id.
      * @throws GridClientConnectionResetException In case of error.
      * @throws GridClientClosedException If client was manually closed before request was sent over network.
      */
-    public abstract GridClientFuture<?> changeReadOnlyState(boolean readOnly, UUID destNodeId)
+    public abstract GridClientFuture<ClusterState> state(UUID destNodeId)
         throws GridClientClosedException, GridClientConnectionResetException;
 
     /**
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/client/impl/connection/GridClientNioTcpConnection.java b/modules/core/src/main/java/org/apache/ignite/internal/client/impl/connection/GridClientNioTcpConnection.java
index fb10675..762f6b3 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/client/impl/connection/GridClientNioTcpConnection.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/client/impl/connection/GridClientNioTcpConnection.java
@@ -38,8 +38,8 @@ import java.util.concurrent.atomic.AtomicLong;
 import java.util.logging.Level;
 import java.util.logging.Logger;
 import javax.net.ssl.SSLContext;
-
 import org.apache.ignite.IgniteCheckedException;
+import org.apache.ignite.cluster.ClusterState;
 import org.apache.ignite.internal.IgniteInternalFuture;
 import org.apache.ignite.internal.client.GridClientAuthenticationException;
 import org.apache.ignite.internal.client.GridClientCacheFlag;
@@ -58,14 +58,13 @@ import org.apache.ignite.internal.client.marshaller.jdk.GridClientJdkMarshaller;
 import org.apache.ignite.internal.client.marshaller.optimized.GridClientOptimizedMarshaller;
 import org.apache.ignite.internal.client.marshaller.optimized.GridClientZipOptimizedMarshaller;
 import org.apache.ignite.internal.processors.rest.client.message.GridClientAuthenticationRequest;
+import org.apache.ignite.internal.processors.rest.client.message.GridClientCacheBean;
 import org.apache.ignite.internal.processors.rest.client.message.GridClientCacheRequest;
 import org.apache.ignite.internal.processors.rest.client.message.GridClientClusterNameRequest;
-import org.apache.ignite.internal.processors.rest.client.message.GridClientReadOnlyModeRequest;
-import org.apache.ignite.internal.processors.rest.client.message.GridClientStateRequest;
+import org.apache.ignite.internal.processors.rest.client.message.GridClientClusterStateRequest;
 import org.apache.ignite.internal.processors.rest.client.message.GridClientHandshakeRequest;
 import org.apache.ignite.internal.processors.rest.client.message.GridClientMessage;
 import org.apache.ignite.internal.processors.rest.client.message.GridClientNodeBean;
-import org.apache.ignite.internal.processors.rest.client.message.GridClientCacheBean;
 import org.apache.ignite.internal.processors.rest.client.message.GridClientNodeMetricsBean;
 import org.apache.ignite.internal.processors.rest.client.message.GridClientPingPacket;
 import org.apache.ignite.internal.processors.rest.client.message.GridClientResponse;
@@ -84,6 +83,8 @@ import org.apache.ignite.internal.util.typedef.CI1;
 import org.jetbrains.annotations.Nullable;
 
 import static java.util.concurrent.TimeUnit.MILLISECONDS;
+import static org.apache.ignite.cluster.ClusterState.ACTIVE;
+import static org.apache.ignite.cluster.ClusterState.INACTIVE;
 import static org.apache.ignite.internal.client.GridClientCacheFlag.KEEP_BINARIES;
 import static org.apache.ignite.internal.client.GridClientCacheFlag.encodeCacheFlags;
 import static org.apache.ignite.internal.client.impl.connection.GridClientConnectionCloseReason.CONN_IDLE;
@@ -810,40 +811,46 @@ public class GridClientNioTcpConnection extends GridClientConnection {
     }
 
     /** {@inheritDoc} */
-    @Override public GridClientFuture<?> changeState(boolean active, UUID destNodeId)
-        throws GridClientClosedException, GridClientConnectionResetException {
-        GridClientStateRequest msg = new GridClientStateRequest();
-
-        msg.active(active);
-
-        return makeRequest(msg, destNodeId);
+    @Override public GridClientFuture<?> changeState(
+        boolean active,
+        UUID destNodeId
+    ) throws GridClientClosedException, GridClientConnectionResetException {
+        return changeState(active ? ACTIVE : INACTIVE, destNodeId);
     }
 
     /** {@inheritDoc} */
-    @Override public GridClientFuture<?> changeReadOnlyState(
-        boolean readOnly,
+    @Override public GridClientFuture<?> changeState(
+        ClusterState state,
         UUID destNodeId
     ) throws GridClientClosedException, GridClientConnectionResetException {
-        return readOnly ?
-            makeRequest(GridClientReadOnlyModeRequest.enableReadOnly(), destNodeId) :
-            makeRequest(GridClientReadOnlyModeRequest.disableReadOnly(), destNodeId);
+        assert state != null;
+
+        return makeRequest(GridClientClusterStateRequest.state(state), destNodeId);
     }
 
     /** {@inheritDoc} */
-    @Override public GridClientFuture<Boolean> readOnlyState(
+    @Override public GridClientFuture<Boolean> currentState(
         UUID destNodeId
     ) throws GridClientClosedException, GridClientConnectionResetException {
-        return makeRequest(GridClientReadOnlyModeRequest.currentReadOnlyMode(), destNodeId);
-    }
+        GridClientFutureAdapter<Boolean> resFut = new GridClientFutureAdapter<>();
 
-    /** {@inheritDoc} */
-    @Override public GridClientFuture<Boolean> currentState(UUID destNodeId)
-        throws GridClientClosedException, GridClientConnectionResetException {
-        GridClientStateRequest msg = new GridClientStateRequest();
+        state(destNodeId).listen(fut -> {
+            try {
+                resFut.onDone(ClusterState.active(fut.get()));
+            }
+            catch (GridClientException e) {
+                resFut.onDone(e);
+            }
+        });
 
-        msg.requestCurrentState();
+        return resFut;
+    }
 
-        return makeRequest(msg, destNodeId);
+    /** {@inheritDoc} */
+    @Override public GridClientFuture<ClusterState> state(
+        UUID destNodeId
+    ) throws GridClientClosedException, GridClientConnectionResetException {
+        return makeRequest(GridClientClusterStateRequest.currentState(), destNodeId);
     }
 
     /** {@inheritDoc} */
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/cluster/IgniteClusterAsyncImpl.java b/modules/core/src/main/java/org/apache/ignite/internal/cluster/IgniteClusterAsyncImpl.java
index aaf598d..a001b56a 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/cluster/IgniteClusterAsyncImpl.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/cluster/IgniteClusterAsyncImpl.java
@@ -35,6 +35,7 @@ import org.apache.ignite.cluster.ClusterGroup;
 import org.apache.ignite.cluster.ClusterMetrics;
 import org.apache.ignite.cluster.ClusterNode;
 import org.apache.ignite.cluster.ClusterStartNodeResult;
+import org.apache.ignite.cluster.ClusterState;
 import org.apache.ignite.internal.AsyncSupportAdapter;
 import org.apache.ignite.internal.processors.cluster.baseline.autoadjust.BaselineAutoAdjustStatus;
 import org.apache.ignite.internal.util.typedef.internal.U;
@@ -416,12 +417,12 @@ public class IgniteClusterAsyncImpl extends AsyncSupportAdapter<IgniteCluster>
     }
 
     /** {@inheritDoc} */
-    @Override public boolean readOnly() {
-        return cluster.readOnly();
+    @Override public ClusterState state() {
+        return cluster.state();
     }
 
     /** {@inheritDoc} */
-    @Override public void readOnly(boolean readOnly) throws IgniteException {
-        cluster.readOnly(readOnly);
+    @Override public void state(ClusterState newState) throws IgniteException {
+        cluster.state(newState);
     }
 }
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 a935a32..7fc666a 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
@@ -46,6 +46,7 @@ import org.apache.ignite.cluster.ClusterGroup;
 import org.apache.ignite.cluster.ClusterGroupEmptyException;
 import org.apache.ignite.cluster.ClusterNode;
 import org.apache.ignite.cluster.ClusterStartNodeResult;
+import org.apache.ignite.cluster.ClusterState;
 import org.apache.ignite.configuration.IgniteConfiguration;
 import org.apache.ignite.internal.GridKernalContext;
 import org.apache.ignite.internal.IgniteComponentType;
@@ -72,8 +73,6 @@ import org.apache.ignite.lang.IgnitePredicate;
 import org.apache.ignite.lang.IgniteProductVersion;
 import org.jetbrains.annotations.Nullable;
 
-import static org.apache.ignite.internal.IgniteFeatures.CLUSTER_READ_ONLY_MODE;
-import static org.apache.ignite.internal.IgniteFeatures.allNodesSupports;
 import static org.apache.ignite.internal.IgniteNodeAttributes.ATTR_IPS;
 import static org.apache.ignite.internal.IgniteNodeAttributes.ATTR_MACS;
 import static org.apache.ignite.internal.util.nodestart.IgniteNodeStartUtils.parseFile;
@@ -313,7 +312,7 @@ public class IgniteClusterImpl extends ClusterGroupAdapter implements IgniteClus
         guard();
 
         try {
-            ctx.state().changeGlobalState(active, baselineNodes(), false).get();
+            ctx.state().changeGlobalState(active, serverNodes(), false).get();
         }
         catch (IgniteCheckedException e) {
             throw U.convertException(e);
@@ -324,11 +323,11 @@ public class IgniteClusterImpl extends ClusterGroupAdapter implements IgniteClus
     }
 
     /** {@inheritDoc} */
-    @Override public boolean readOnly() {
+    @Override public ClusterState state() {
         guard();
 
         try {
-            return ctx.state().publicApiReadOnlyMode();
+            return ctx.state().publicApiState(true);
         }
         finally {
             unguard();
@@ -336,13 +335,11 @@ public class IgniteClusterImpl extends ClusterGroupAdapter implements IgniteClus
     }
 
     /** {@inheritDoc} */
-    @Override public void readOnly(boolean readOnly) throws IgniteException {
+    @Override public void state(ClusterState newState) throws IgniteException {
         guard();
 
         try {
-            verifyReadOnlyModeSupport();
-
-            ctx.state().changeGlobalState(readOnly).get();
+            ctx.state().changeGlobalState(newState, serverNodes(), false).get();
         }
         catch (IgniteCheckedException e) {
             throw U.convertException(e);
@@ -353,13 +350,7 @@ public class IgniteClusterImpl extends ClusterGroupAdapter implements IgniteClus
     }
 
     /** */
-    private void verifyReadOnlyModeSupport() {
-        if (!allNodesSupports(ctx.discovery().discoCache().serverNodes(), CLUSTER_READ_ONLY_MODE))
-            throw new IgniteException("Not all nodes in cluster supports cluster read-only mode");
-    }
-
-    /** */
-    private Collection<BaselineNode> baselineNodes() {
+    private Collection<BaselineNode> serverNodes() {
         return new ArrayList<>(ctx.cluster().get().forServers().nodes());
     }
 
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/commandline/ActivateCommand.java b/modules/core/src/main/java/org/apache/ignite/internal/commandline/ActivateCommand.java
index 08cb163..f2817d7 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/commandline/ActivateCommand.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/commandline/ActivateCommand.java
@@ -24,14 +24,17 @@ import org.apache.ignite.internal.client.GridClientConfiguration;
 import org.apache.ignite.internal.client.GridClientException;
 
 import static org.apache.ignite.internal.commandline.CommandList.ACTIVATE;
+import static org.apache.ignite.internal.commandline.CommandList.SET_STATE;
 
 /**
  * Activate cluster command.
+ * @deprecated Use {@link ClusterStateChangeCommand} instead.
  */
+@Deprecated
 public class ActivateCommand implements Command<Void> {
     /** {@inheritDoc} */
     @Override public void printUsage(Logger logger) {
-        Command.usage(logger, "Activate cluster:", ACTIVATE);
+        Command.usage(logger, "Activate cluster (deprecated. Use " + SET_STATE.toString() + " instead):", ACTIVATE);
     }
 
     /**
@@ -41,6 +44,8 @@ public class ActivateCommand implements Command<Void> {
      * @throws GridClientException If failed to activate.
      */
     @Override public Object execute(GridClientConfiguration cfg, Logger logger) throws Exception {
+        logger.warning("Command deprecated. Use " + SET_STATE.toString() + " instead.");
+
         try (GridClient client = Command.startClient(cfg)) {
             GridClientClusterState state = client.state();
 
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/commandline/ClusterReadOnlyModeDisableCommand.java b/modules/core/src/main/java/org/apache/ignite/internal/commandline/ClusterReadOnlyModeDisableCommand.java
deleted file mode 100644
index fca9832..0000000
--- a/modules/core/src/main/java/org/apache/ignite/internal/commandline/ClusterReadOnlyModeDisableCommand.java
+++ /dev/null
@@ -1,72 +0,0 @@
-/*
- * 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.logging.Logger;
-import org.apache.ignite.internal.client.GridClient;
-import org.apache.ignite.internal.client.GridClientConfiguration;
-
-import static org.apache.ignite.internal.commandline.CommandList.READ_ONLY_DISABLE;
-import static org.apache.ignite.internal.commandline.CommandLogger.optional;
-import static org.apache.ignite.internal.commandline.CommonArgParser.CMD_AUTO_CONFIRMATION;
-
-/**
- * Command to disable cluster read-only mode.
- */
-public class ClusterReadOnlyModeDisableCommand implements Command<Void> {
-    /** {@inheritDoc} */
-    @Override public Object execute(GridClientConfiguration clientCfg, Logger log) throws Exception {
-        try (GridClient client = Command.startClient(clientCfg)) {
-            client.state().readOnly(false);
-
-            log.info("Cluster read-only mode disabled");
-        }
-        catch (Throwable e) {
-            log.info("Failed to disable read-only mode");
-
-            throw e;
-        }
-
-        return null;
-    }
-
-    /** {@inheritDoc} */
-    @Override public String confirmationPrompt() {
-        return "Warning: the command will disable read-only mode on a cluster.";
-    }
-
-    /** {@inheritDoc} */
-    @Override public Void arg() {
-        return null;
-    }
-
-    /** {@inheritDoc} */
-    @Override public void printUsage(Logger log) {
-        Command.usage(
-            log,
-            "Disable read-only mode on active cluster:",
-            READ_ONLY_DISABLE,
-            optional(CMD_AUTO_CONFIRMATION)
-        );
-    }
-
-    /** {@inheritDoc} */
-    @Override public String name() {
-        return READ_ONLY_DISABLE.toCommandName();
-    }
-}
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/commandline/ClusterReadOnlyModeEnableCommand.java b/modules/core/src/main/java/org/apache/ignite/internal/commandline/ClusterReadOnlyModeEnableCommand.java
deleted file mode 100644
index ca3fe6b..0000000
--- a/modules/core/src/main/java/org/apache/ignite/internal/commandline/ClusterReadOnlyModeEnableCommand.java
+++ /dev/null
@@ -1,72 +0,0 @@
-/*
- * 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.logging.Logger;
-import org.apache.ignite.internal.client.GridClient;
-import org.apache.ignite.internal.client.GridClientConfiguration;
-
-import static org.apache.ignite.internal.commandline.CommandList.READ_ONLY_ENABLE;
-import static org.apache.ignite.internal.commandline.CommandLogger.optional;
-import static org.apache.ignite.internal.commandline.CommonArgParser.CMD_AUTO_CONFIRMATION;
-
-/**
- * Command to enable cluster read-only mode.
- */
-public class ClusterReadOnlyModeEnableCommand implements Command<Void> {
-    /** {@inheritDoc} */
-    @Override public Object execute(GridClientConfiguration clientCfg, Logger log) throws Exception {
-        try (GridClient client = Command.startClient(clientCfg)) {
-            client.state().readOnly(true);
-
-            log.info("Cluster read-only mode enabled");
-        }
-        catch (Throwable e) {
-            log.info("Failed to enable read-only mode");
-
-            throw e;
-        }
-
-        return null;
-    }
-
-    /** {@inheritDoc} */
-    @Override public String confirmationPrompt() {
-        return "Warning: the command will enable read-only mode on a cluster.";
-    }
-
-    /** {@inheritDoc} */
-    @Override public Void arg() {
-        return null;
-    }
-
-    /** {@inheritDoc} */
-    @Override public void printUsage(Logger log) {
-        Command.usage(
-            log,
-            "Enable read-only mode on active cluster:",
-            READ_ONLY_ENABLE,
-            optional(CMD_AUTO_CONFIRMATION)
-        );
-    }
-
-    /** {@inheritDoc} */
-    @Override public String name() {
-        return READ_ONLY_ENABLE.toCommandName();
-    }
-}
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/commandline/ClusterStateChangeCommand.java b/modules/core/src/main/java/org/apache/ignite/internal/commandline/ClusterStateChangeCommand.java
new file mode 100644
index 0000000..d9b8809
--- /dev/null
+++ b/modules/core/src/main/java/org/apache/ignite/internal/commandline/ClusterStateChangeCommand.java
@@ -0,0 +1,105 @@
+/*
+ * 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.LinkedHashMap;
+import java.util.Map;
+import java.util.logging.Logger;
+import org.apache.ignite.cluster.ClusterState;
+import org.apache.ignite.internal.client.GridClient;
+import org.apache.ignite.internal.client.GridClientConfiguration;
+
+import static org.apache.ignite.cluster.ClusterState.ACTIVE;
+import static org.apache.ignite.cluster.ClusterState.INACTIVE;
+import static org.apache.ignite.cluster.ClusterState.ACTIVE_READ_ONLY;
+import static org.apache.ignite.internal.commandline.CommandList.SET_STATE;
+import static org.apache.ignite.internal.commandline.CommandLogger.optional;
+import static org.apache.ignite.internal.commandline.CommandLogger.or;
+import static org.apache.ignite.internal.commandline.CommonArgParser.CMD_AUTO_CONFIRMATION;
+
+/**
+ * Command to change cluster state.
+ */
+public class ClusterStateChangeCommand implements Command<ClusterState> {
+    /** New cluster state */
+    private ClusterState state;
+
+    /** Cluster name. */
+    private String clusterName;
+
+    /** {@inheritDoc} */
+    @Override public void printUsage(Logger log) {
+        Map<String, String> params = new LinkedHashMap<>();
+
+        params.put(ACTIVE.toString(), "Activate cluster. Cache updates are allowed.");
+        params.put(INACTIVE.toString(), "Deactivate cluster.");
+        params.put(ACTIVE_READ_ONLY.toString(), "Activate cluster. Cache updates are denied.");
+
+        Command.usage(log, "Change cluster state:", SET_STATE, params, or((Object[])ClusterState.values()), optional(CMD_AUTO_CONFIRMATION));
+    }
+
+    /** {@inheritDoc} */
+    @Override public void prepareConfirmation(GridClientConfiguration clientCfg) throws Exception {
+        try (GridClient client = Command.startClient(clientCfg)) {
+            clusterName = client.state().clusterName();
+        }
+    }
+
+    /** {@inheritDoc} */
+    @Override public String confirmationPrompt() {
+        return "Warning: the command will change state of cluster with name \"" + clusterName + "\" to " + state + ".";
+    }
+
+    /** {@inheritDoc} */
+    @Override public Object execute(GridClientConfiguration clientCfg, Logger log) throws Exception {
+        try (GridClient client = Command.startClient(clientCfg)) {
+            client.state().state(state);
+
+            log.info("Cluster state changed to " + state);
+
+            return null;
+        }
+        catch (Throwable e) {
+            log.info("Failed to change cluster state to " +  state);
+
+            throw e;
+        }
+    }
+
+    /** {@inheritDoc} */
+    @Override public void parseArguments(CommandArgIterator argIter) {
+        String s = argIter.nextArg("New cluster state not found.");
+
+        try {
+            state = ClusterState.valueOf(s.toUpperCase());
+        }
+        catch (IllegalArgumentException e) {
+            throw new IllegalArgumentException("Can't parse new cluster state. State: " + s, e);
+        }
+    }
+
+    /** {@inheritDoc} */
+    @Override public ClusterState arg() {
+        return state;
+    }
+
+    /** {@inheritDoc} */
+    @Override public String name() {
+        return SET_STATE.toCommandName();
+    }
+}
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/commandline/Command.java b/modules/core/src/main/java/org/apache/ignite/internal/commandline/Command.java
index eae71e6..d65ced1 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/commandline/Command.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/commandline/Command.java
@@ -17,11 +17,16 @@
 
 package org.apache.ignite.internal.commandline;
 
+import java.util.Comparator;
+import java.util.Map;
 import java.util.logging.Logger;
 import org.apache.ignite.IgniteSystemProperties;
 import org.apache.ignite.internal.client.GridClient;
 import org.apache.ignite.internal.client.GridClientConfiguration;
 import org.apache.ignite.internal.client.GridClientFactory;
+import org.apache.ignite.internal.util.typedef.F;
+import org.apache.ignite.internal.util.typedef.internal.SB;
+import org.jetbrains.annotations.Nullable;
 
 import static org.apache.ignite.IgniteSystemProperties.IGNITE_ENABLE_EXPERIMENTAL_COMMAND;
 import static org.apache.ignite.internal.commandline.CommandHandler.UTILITY_NAME;
@@ -57,12 +62,78 @@ public interface Command<T> {
      *
      * @param logger Logger to use.
      * @param desc Command description.
+     * @param cmd Command.
      * @param args Arguments.
      */
     public static void usage(Logger logger, String desc, CommandList cmd, String... args) {
+        usage(logger, desc, cmd, null, args);
+    }
+
+    /**
+     * Print command usage.
+     *
+     * @param logger Logger to use.
+     * @param desc Command description.
+     * @param cmd Command.
+     * @param paramsDesc Description of parameters (optional).
+     * @param args Arguments.
+     */
+    public static void usage(
+        Logger logger,
+        String desc,
+        CommandList cmd,
+        @Nullable Map<String, String> paramsDesc,
+        String... args
+    ) {
+        logger.info("");
         logger.info(INDENT + desc);
         logger.info(DOUBLE_INDENT + CommandLogger.join(" ", UTILITY_NAME, cmd, CommandLogger.join(" ", args)));
-        logger.info("");
+
+        if (!F.isEmpty(paramsDesc)) {
+            logger.info("");
+
+            logger.info(DOUBLE_INDENT + "Parameters:");
+
+            usageParams(paramsDesc, DOUBLE_INDENT + INDENT, logger);
+        }
+    }
+
+    /**
+     * Print cache command arguments usage.
+     *
+     * @param paramsDesc Cache command arguments description.
+     * @param indent Indent string.
+     * @param logger Logger to use.
+     */
+    public static void usageParams(Map<String, String> paramsDesc, String indent, Logger logger) {
+        int maxParamLen = paramsDesc.keySet().stream().max(Comparator.comparingInt(String::length)).get().length();
+
+        for (Map.Entry<String, String> param : paramsDesc.entrySet())
+            logger.info(indent + extendToLen(param.getKey(), maxParamLen) + "  " + "- " + param.getValue());
+    }
+
+    /**
+     * Appends spaces to end of input string for extending to needed length.
+     *
+     * @param s Input string.
+     * @param targetLen Needed length.
+     * @return String with appended spaces on the end.
+     */
+    public static String extendToLen(String s, int targetLen) {
+        assert targetLen >= 0;
+        assert s.length() <= targetLen;
+
+        if (s.length() == targetLen)
+            return s;
+
+        SB sb = new SB(targetLen);
+
+        sb.a(s);
+
+        for (int i = 0; i < targetLen - s.length(); i++)
+            sb.a(" ");
+
+        return sb.toString();
     }
 
     /**
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/commandline/CommandHandler.java b/modules/core/src/main/java/org/apache/ignite/internal/commandline/CommandHandler.java
index e55ecaa..57bdcfa 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/commandline/CommandHandler.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/commandline/CommandHandler.java
@@ -651,6 +651,7 @@ public class CommandHandler {
 
         Arrays.stream(CommandList.values()).forEach(c -> c.command().printUsage(logger));
 
+        logger.info("");
         logger.info("By default commands affecting the cluster require interactive confirmation.");
         logger.info("Use " + CMD_AUTO_CONFIRMATION + " option to disable it.");
         logger.info("");
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 055276b..5bd59f1 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
@@ -34,6 +34,9 @@ public enum CommandList {
     STATE("--state", new StateCommand()),
 
     /** */
+    SET_STATE("--set-state", new ClusterStateChangeCommand()),
+
+    /** */
     BASELINE("--baseline", new BaselineCommand()),
 
     /** */
@@ -46,13 +49,7 @@ public enum CommandList {
     WAL("--wal", new WalCommands()),
 
     /** */
-    DIAGNOSTIC("--diagnostic", new DiagnosticCommand()),
-
-    /** */
-    READ_ONLY_ENABLE("--read-only-on", new ClusterReadOnlyModeEnableCommand()),
-
-    /** */
-    READ_ONLY_DISABLE("--read-only-off", new ClusterReadOnlyModeDisableCommand());
+    DIAGNOSTIC("--diagnostic", new DiagnosticCommand());
 
     /** Private values copy so there's no need in cloning it every time. */
     private static final CommandList[] VALUES = CommandList.values();
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/commandline/DeactivateCommand.java b/modules/core/src/main/java/org/apache/ignite/internal/commandline/DeactivateCommand.java
index b54fdeb..0704fa8 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/commandline/DeactivateCommand.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/commandline/DeactivateCommand.java
@@ -23,19 +23,22 @@ import org.apache.ignite.internal.client.GridClientClusterState;
 import org.apache.ignite.internal.client.GridClientConfiguration;
 
 import static org.apache.ignite.internal.commandline.CommandList.DEACTIVATE;
+import static org.apache.ignite.internal.commandline.CommandList.SET_STATE;
 import static org.apache.ignite.internal.commandline.CommandLogger.optional;
 import static org.apache.ignite.internal.commandline.CommonArgParser.CMD_AUTO_CONFIRMATION;
 
 /**
  * Command to deactivate cluster.
+ * @deprecated Use {@link ClusterStateChangeCommand} instead.
  */
+@Deprecated
 public class DeactivateCommand implements Command<Void> {
     /** Cluster name. */
     private String clusterName;
 
     /** {@inheritDoc} */
     @Override public void printUsage(Logger logger) {
-        Command.usage(logger, "Deactivate cluster:", DEACTIVATE, optional(CMD_AUTO_CONFIRMATION));
+        Command.usage(logger, "Deactivate cluster (deprecated. Use " + SET_STATE.toString() + " instead):", DEACTIVATE, optional(CMD_AUTO_CONFIRMATION));
     }
 
     /** {@inheritDoc} */
@@ -57,6 +60,8 @@ public class DeactivateCommand implements Command<Void> {
      * @throws Exception If failed to deactivate.
      */
     @Override public Object execute(GridClientConfiguration clientCfg, Logger logger) throws Exception {
+        logger.warning("Command deprecated. Use " + SET_STATE.toString() + " instead.");
+
         try (GridClient client = Command.startClient(clientCfg)) {
             GridClientClusterState state = client.state();
 
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 510c31c..1148110 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
@@ -18,6 +18,7 @@
 package org.apache.ignite.internal.commandline;
 
 import java.util.logging.Logger;
+import org.apache.ignite.cluster.ClusterState;
 import org.apache.ignite.internal.client.GridClient;
 import org.apache.ignite.internal.client.GridClientClusterState;
 import org.apache.ignite.internal.client.GridClientConfiguration;
@@ -43,14 +44,27 @@ public class StateCommand implements Command<Void> {
         try (GridClient client = Command.startClient(clientCfg)){
             GridClientClusterState state = client.state();
 
-            if (state.active()) {
-                if (state.readOnly())
-                    log.info("Cluster is active (read-only)");
-                else
+            ClusterState clusterState = state.state();
+
+            switch (clusterState) {
+                case ACTIVE:
                     log.info("Cluster is active");
+
+                    break;
+
+                case INACTIVE:
+                    log.info("Cluster is inactive");
+
+                    break;
+
+                case ACTIVE_READ_ONLY:
+                    log.info("Cluster is active (read-only)");
+
+                    break;
+
+                default:
+                    throw new IllegalStateException("Unknown state: " + clusterState);
             }
-            else
-                log.info("Cluster is inactive");
         }
         catch (Throwable e) {
             if (!CommandHandler.isAuthError(e))
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/commandline/cache/CacheCommands.java b/modules/core/src/main/java/org/apache/ignite/internal/commandline/cache/CacheCommands.java
index bcb30e7..9084445 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/commandline/cache/CacheCommands.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/commandline/cache/CacheCommands.java
@@ -18,7 +18,6 @@
 package org.apache.ignite.internal.commandline.cache;
 
 import java.util.Arrays;
-import java.util.Comparator;
 import java.util.Map;
 import java.util.logging.Logger;
 import org.apache.ignite.internal.client.GridClientConfiguration;
@@ -26,8 +25,8 @@ import org.apache.ignite.internal.commandline.Command;
 import org.apache.ignite.internal.commandline.CommandArgIterator;
 import org.apache.ignite.internal.commandline.CommandLogger;
 import org.apache.ignite.internal.util.typedef.F;
-import org.apache.ignite.internal.util.typedef.internal.SB;
 
+import static org.apache.ignite.internal.commandline.Command.usageParams;
 import static org.apache.ignite.internal.commandline.CommandHandler.UTILITY_NAME;
 import static org.apache.ignite.internal.commandline.CommandList.CACHE;
 import static org.apache.ignite.internal.commandline.CommandLogger.DOUBLE_INDENT;
@@ -55,9 +54,9 @@ public class CacheCommands implements Command<CacheSubcommands> {
 
     /** {@inheritDoc} */
     @Override public void printUsage(Logger logger) {
+        logger.info("");
         logger.info(INDENT + "View caches information in a cluster. For more details type:");
         logger.info(DOUBLE_INDENT + CommandLogger.join(" ", UTILITY_NAME, CACHE, HELP));
-        logger.info("");
     }
 
     /** {@inheritDoc} */
@@ -162,48 +161,10 @@ public class CacheCommands implements Command<CacheSubcommands> {
             logger.info("");
             logger.info(DOUBLE_INDENT + "Parameters:");
 
-            usageCacheParams(paramsDesc, DOUBLE_INDENT + INDENT, logger);
+            usageParams(paramsDesc, DOUBLE_INDENT + INDENT, logger);
         }
     }
 
-    /**
-     * Print cache command arguments usage.
-     *
-     * @param paramsDesc Cache command arguments description.
-     * @param indent Indent string.
-     * @param logger Logger to use.
-     */
-    private static void usageCacheParams(Map<String, String> paramsDesc, String indent, Logger logger) {
-        int maxParamLen = paramsDesc.keySet().stream().max(Comparator.comparingInt(String::length)).get().length();
-
-        for (Map.Entry<String, String> param : paramsDesc.entrySet())
-            logger.info(indent + extendToLen(param.getKey(), maxParamLen) + "  " + "- " + param.getValue());
-    }
-
-    /**
-     * Appends spaces to end of input string for extending to needed length.
-     *
-     * @param s Input string.
-     * @param targetLen Needed length.
-     * @return String with appended spaces on the end.
-     */
-    private static String extendToLen(String s, int targetLen) {
-        assert targetLen >= 0;
-        assert s.length() <= targetLen;
-
-        if (s.length() == targetLen)
-            return s;
-
-        SB sb = new SB(targetLen);
-
-        sb.a(s);
-
-        for (int i = 0; i < targetLen - s.length(); i++)
-            sb.a(" ");
-
-        return sb.toString();
-    }
-
     /** {@inheritDoc} */
     @Override public CacheSubcommands arg() {
         return subcommand;
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/managers/discovery/GridDiscoveryManager.java b/modules/core/src/main/java/org/apache/ignite/internal/managers/discovery/GridDiscoveryManager.java
index f4ba907..01ff5d3 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/managers/discovery/GridDiscoveryManager.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/managers/discovery/GridDiscoveryManager.java
@@ -47,6 +47,7 @@ import org.apache.ignite.cache.CacheMode;
 import org.apache.ignite.cluster.BaselineNode;
 import org.apache.ignite.cluster.ClusterMetrics;
 import org.apache.ignite.cluster.ClusterNode;
+import org.apache.ignite.cluster.ClusterState;
 import org.apache.ignite.configuration.CacheConfiguration;
 import org.apache.ignite.configuration.CommunicationFailureResolver;
 import org.apache.ignite.configuration.DataRegionConfiguration;
@@ -141,6 +142,9 @@ import static org.apache.ignite.IgniteSystemProperties.IGNITE_EVENT_DRIVEN_SERVI
 import static org.apache.ignite.IgniteSystemProperties.IGNITE_OPTIMIZED_MARSHALLER_USE_DEFAULT_SUID;
 import static org.apache.ignite.IgniteSystemProperties.IGNITE_SECURITY_COMPATIBILITY_MODE;
 import static org.apache.ignite.IgniteSystemProperties.getInteger;
+import static org.apache.ignite.cluster.ClusterState.ACTIVE;
+import static org.apache.ignite.cluster.ClusterState.INACTIVE;
+import static org.apache.ignite.cluster.ClusterState.active;
 import static org.apache.ignite.events.EventType.EVT_CLIENT_NODE_DISCONNECTED;
 import static org.apache.ignite.events.EventType.EVT_CLIENT_NODE_RECONNECTED;
 import static org.apache.ignite.events.EventType.EVT_NODE_FAILED;
@@ -1531,7 +1535,12 @@ public class GridDiscoveryManager extends GridManagerAdapter<DiscoverySpi> {
             clo.apply("  ^-- Baseline [id=" + blt.id() + ", size=" + bltSize + ", online=" + bltOnline
                 + ", offline=" + bltOffline + ']');
 
-            if (!state.active() && ctx.config().isAutoActivationEnabled()) {
+            ClusterState targetState = ctx.config().getClusterStateOnStart();
+
+            if (targetState == null)
+                targetState = ctx.config().isAutoActivationEnabled() ? ACTIVE : INACTIVE;
+
+            if (!active(state.state()) && active(targetState)) {
                 String offlineConsistentIds = "";
 
                 if (bltOffline > 0 && bltOffline <= 5) {
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/ExchangeActions.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/ExchangeActions.java
index 6431d0f..5382a91 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/ExchangeActions.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/ExchangeActions.java
@@ -30,6 +30,8 @@ import org.apache.ignite.internal.util.typedef.F;
 import org.apache.ignite.internal.util.typedef.internal.CU;
 import org.jetbrains.annotations.Nullable;
 
+import static org.apache.ignite.cluster.ClusterState.active;
+
 /**
  * Cache change requests to execute when receive {@link DynamicCacheChangeBatch} event.
  */
@@ -202,14 +204,21 @@ public class ExchangeActions {
      * @return {@code True} if has deactivate request.
      */
     public boolean deactivate() {
-        return stateChangeReq != null && stateChangeReq.activeChanged() && !stateChangeReq.activate();
+        return stateChangeReq != null && stateChangeReq.activeChanged() && !active(stateChangeReq.state());
     }
 
     /**
      * @return {@code True} if has activate request.
      */
     public boolean activate() {
-        return stateChangeReq != null && stateChangeReq.activeChanged() && stateChangeReq.activate();
+        return stateChangeReq != null && stateChangeReq.activeChanged() && active(stateChangeReq.state());
+    }
+
+    /**
+     * @return {@code True} if cluster state was changed.
+     */
+    public boolean changedClusterState() {
+        return stateChangeReq != null && stateChangeReq.prevState() != stateChangeReq.state();
     }
 
     /**
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCachePartitionExchangeManager.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCachePartitionExchangeManager.java
index 0f9fba1..67a62ca 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCachePartitionExchangeManager.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCachePartitionExchangeManager.java
@@ -58,10 +58,10 @@ import org.apache.ignite.cluster.ClusterNode;
 import org.apache.ignite.configuration.CacheConfiguration;
 import org.apache.ignite.configuration.IgniteConfiguration;
 import org.apache.ignite.events.ClusterActivationEvent;
+import org.apache.ignite.events.ClusterStateChangeEvent;
 import org.apache.ignite.events.DiscoveryEvent;
-import org.apache.ignite.events.EventType;
+import org.apache.ignite.events.Event;
 import org.apache.ignite.failure.FailureContext;
-import org.apache.ignite.internal.GridKernalContext;
 import org.apache.ignite.internal.IgniteClientDisconnectedCheckedException;
 import org.apache.ignite.internal.IgniteDiagnosticAware;
 import org.apache.ignite.internal.IgniteDiagnosticPrepareContext;
@@ -76,6 +76,7 @@ import org.apache.ignite.internal.managers.discovery.DiscoCache;
 import org.apache.ignite.internal.managers.discovery.DiscoveryCustomMessage;
 import org.apache.ignite.internal.managers.discovery.DiscoveryLocalJoinData;
 import org.apache.ignite.internal.managers.eventstorage.DiscoveryEventListener;
+import org.apache.ignite.internal.managers.eventstorage.GridEventStorageManager;
 import org.apache.ignite.internal.processors.affinity.AffinityTopologyVersion;
 import org.apache.ignite.internal.processors.affinity.GridAffinityAssignmentCache;
 import org.apache.ignite.internal.processors.cache.distributed.dht.GridDhtTopologyFuture;
@@ -126,6 +127,7 @@ import org.apache.ignite.internal.util.typedef.CI2;
 import org.apache.ignite.internal.util.typedef.F;
 import org.apache.ignite.internal.util.typedef.T2;
 import org.apache.ignite.internal.util.typedef.X;
+import org.apache.ignite.internal.util.typedef.internal.A;
 import org.apache.ignite.internal.util.typedef.internal.CU;
 import org.apache.ignite.internal.util.typedef.internal.S;
 import org.apache.ignite.internal.util.typedef.internal.U;
@@ -146,6 +148,9 @@ import static org.apache.ignite.IgniteSystemProperties.IGNITE_IO_DUMP_ON_TIMEOUT
 import static org.apache.ignite.IgniteSystemProperties.IGNITE_PRELOAD_RESEND_TIMEOUT;
 import static org.apache.ignite.IgniteSystemProperties.IGNITE_THREAD_DUMP_ON_EXCHANGE_TIMEOUT;
 import static org.apache.ignite.IgniteSystemProperties.getLong;
+import static org.apache.ignite.events.EventType.EVT_CLUSTER_ACTIVATED;
+import static org.apache.ignite.events.EventType.EVT_CLUSTER_DEACTIVATED;
+import static org.apache.ignite.events.EventType.EVT_CLUSTER_STATE_CHANGED;
 import static org.apache.ignite.events.EventType.EVT_NODE_FAILED;
 import static org.apache.ignite.events.EventType.EVT_NODE_JOINED;
 import static org.apache.ignite.events.EventType.EVT_NODE_LEFT;
@@ -184,9 +189,6 @@ public class GridCachePartitionExchangeManager<K, V> extends GridCacheSharedMana
     /** */
     private static final IgniteProductVersion EXCHANGE_PROTOCOL_2_SINCE = IgniteProductVersion.fromString("2.1.4");
 
-    /** Stripe id for cluster activation event. */
-    private static final int CLUSTER_ACTIVATION_EVT_STRIPE_ID = Integer.MAX_VALUE;
-
     /** Atomic reference for pending partition resend timeout object. */
     private AtomicReference<ResendTimeoutObject> pendingResend = new AtomicReference<>();
 
@@ -577,12 +579,7 @@ public class GridCachePartitionExchangeManager<K, V> extends GridCacheSharedMana
 
                     exchFut = exchangeFuture(exchId, evt, cache, exchActions, null);
 
-                    exchFut.listen(f -> {
-                        if (exchActions.activate())
-                            recordEvent("Cluster activated.", EventType.EVT_CLUSTER_ACTIVATED);
-                        else if (exchActions.deactivate())
-                            recordEvent("Cluster deactivated.", EventType.EVT_CLUSTER_DEACTIVATED);
-                    });
+                    exchFut.listen(f -> onClusterStateChangeFinish(f, exchActions));
                 }
             }
             else if (customMsg instanceof DynamicCacheChangeBatch) {
@@ -664,26 +661,54 @@ public class GridCachePartitionExchangeManager<K, V> extends GridCacheSharedMana
         }
     }
 
-    /**
-     * @param msg Event message.
-     * @param evtType Event type.
-     */
-    private void recordEvent(String msg, int evtType) {
-        GridKernalContext ctx = cctx.kernalContext();
+    /** */
+    private void onClusterStateChangeFinish(IgniteInternalFuture<AffinityTopologyVersion> fut, ExchangeActions exchActions) {
+        A.notNull(exchActions, "exchActions");
 
-        Collection<ClusterNode> srvNodes = ctx.cluster().get().forServers().nodes();
+        GridEventStorageManager evtMngr = cctx.kernalContext().event();
 
-        Collection<BaselineNode> baselineNodes = new ArrayList<>(srvNodes);
+        if (exchActions.activate() && evtMngr.isRecordable(EVT_CLUSTER_ACTIVATED) ||
+            exchActions.deactivate() && evtMngr.isRecordable(EVT_CLUSTER_DEACTIVATED) ||
+            exchActions.changedClusterState() && evtMngr.isRecordable(EVT_CLUSTER_STATE_CHANGED)
+        ) {
+            List<Event> evts = new ArrayList<>(2);
 
-        ctx.getStripedExecutorService().execute(CLUSTER_ACTIVATION_EVT_STRIPE_ID, new Runnable() {
-            @Override public void run() {
-                if (ctx.event().isRecordable(evtType)) {
-                    ClusterActivationEvent evt = new ClusterActivationEvent(ctx.discovery().localNode(), msg, evtType, baselineNodes);
+            ClusterNode locNode = cctx.kernalContext().discovery().localNode();
 
-                    ctx.event().record(evt);
-                }
+            Collection<BaselineNode> bltNodes = cctx.kernalContext().cluster().get().currentBaselineTopology();
+
+            boolean collectionUsed = false;
+
+            if (exchActions.activate() && evtMngr.isRecordable(EVT_CLUSTER_ACTIVATED)) {
+                assert !exchActions.deactivate() : exchActions;
+
+                collectionUsed = true;
+
+                evts.add(new ClusterActivationEvent(locNode, "Cluster activated.", EVT_CLUSTER_ACTIVATED, bltNodes));
+            }
+
+            if (exchActions.deactivate() && evtMngr.isRecordable(EVT_CLUSTER_DEACTIVATED)) {
+                assert !exchActions.activate() : exchActions;
+
+                collectionUsed = true;
+
+                evts.add(new ClusterActivationEvent(locNode, "Cluster deactivated.", EVT_CLUSTER_DEACTIVATED, bltNodes));
+            }
+
+            if (exchActions.changedClusterState() && evtMngr.isRecordable(EVT_CLUSTER_STATE_CHANGED)) {
+                StateChangeRequest req = exchActions.stateChangeRequest();
+
+                if (collectionUsed && bltNodes != null)
+                    bltNodes = new ArrayList<>(bltNodes);
+
+                evts.add(new ClusterStateChangeEvent(req.prevState(), req.state(), bltNodes, locNode, "Cluster state changed."));
             }
-        });
+
+            A.notEmpty(evts, "events " + exchActions);
+
+            cctx.kernalContext().getSystemExecutorService()
+                .submit(() -> evts.forEach(e -> cctx.kernalContext().event().record(e)));
+        }
     }
 
     /**
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/StateChangeRequest.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/StateChangeRequest.java
index cd37016..5d06855 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/StateChangeRequest.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/StateChangeRequest.java
@@ -18,6 +18,7 @@
 package org.apache.ignite.internal.processors.cache;
 
 import java.util.UUID;
+import org.apache.ignite.cluster.ClusterState;
 import org.apache.ignite.internal.processors.affinity.AffinityTopologyVersion;
 import org.apache.ignite.internal.processors.cluster.BaselineTopology;
 import org.apache.ignite.internal.processors.cluster.BaselineTopologyHistoryItem;
@@ -26,6 +27,8 @@ import org.apache.ignite.internal.util.typedef.internal.S;
 import org.apache.ignite.lang.IgniteUuid;
 import org.jetbrains.annotations.Nullable;
 
+import static org.apache.ignite.cluster.ClusterState.active;
+
 /**
  *
  */
@@ -37,22 +40,26 @@ public class StateChangeRequest {
     private final BaselineTopologyHistoryItem prevBltHistItem;
 
     /** */
-    private final boolean activeChanged;
+    private ClusterState prevState;
 
     /** */
     private final AffinityTopologyVersion topVer;
 
     /**
      * @param msg Message.
+     * @param bltHistItem Baseline history item.
+     * @param prevState Previous cluster state.
      * @param topVer State change topology versoin.
      */
-    public StateChangeRequest(ChangeGlobalStateMessage msg,
+    public StateChangeRequest(
+        ChangeGlobalStateMessage msg,
         BaselineTopologyHistoryItem bltHistItem,
-        boolean activeChanged,
-        AffinityTopologyVersion topVer) {
+        ClusterState prevState,
+        AffinityTopologyVersion topVer
+    ) {
         this.msg = msg;
         prevBltHistItem = bltHistItem;
-        this.activeChanged = activeChanged;
+        this.prevState = prevState;
         this.topVer = topVer;
     }
 
@@ -79,23 +86,32 @@ public class StateChangeRequest {
 
     /**
      * @return New state.
+     * @deprecated Use {@link #state()} instead.
      */
+    @Deprecated
     public boolean activate() {
         return msg.activate();
     }
 
     /**
-     * @return Read-only mode flag.
+     * @return New cluster state.
+     */
+    public ClusterState state() {
+        return msg.state();
+    }
+
+    /**
+     * @return Previous cluster state.
      */
-    public boolean readOnly() {
-        return msg.readOnly();
+    public ClusterState prevState() {
+        return prevState;
     }
 
     /**
      * @return {@code True} if active state was changed.
      */
     public boolean activeChanged() {
-        return activeChanged;
+        return active(prevState) && !active(msg.state()) || !active(prevState) && active(msg.state());
     }
 
     /**
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtTopologyFutureAdapter.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtTopologyFutureAdapter.java
index 766c9fe..c57aaff 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtTopologyFutureAdapter.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtTopologyFutureAdapter.java
@@ -24,7 +24,6 @@ import org.apache.ignite.IgniteCheckedException;
 import org.apache.ignite.cache.PartitionLossPolicy;
 import org.apache.ignite.cluster.ClusterNode;
 import org.apache.ignite.configuration.TopologyValidator;
-import org.apache.ignite.internal.cluster.ClusterReadOnlyModeCheckedException;
 import org.apache.ignite.internal.processors.affinity.AffinityTopologyVersion;
 import org.apache.ignite.internal.processors.cache.CacheGroupContext;
 import org.apache.ignite.internal.processors.cache.CacheInvalidStateException;
@@ -101,7 +100,7 @@ public abstract class GridDhtTopologyFutureAdapter extends GridFutureAdapter<Aff
         PartitionLossPolicy lossPlc = grp.config().getPartitionLossPolicy();
 
         if (cctx.shared().readOnlyMode() && opType == WRITE && !isSystemCache(cctx.name()))
-            return new ClusterReadOnlyModeCheckedException("Failed to perform cache operation (cluster is in read-only mode)");
+            return new IgniteClusterReadOnlyException("Failed to perform cache operation (cluster is in read-only mode)");
 
         if (grp.needsRecovery() && !recovery) {
             if (opType == WRITE && (lossPlc == READ_ONLY_SAFE || lossPlc == READ_ONLY_ALL))
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/cluster/ClusterReadOnlyModeCheckedException.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/IgniteClusterReadOnlyException.java
similarity index 61%
rename from modules/core/src/main/java/org/apache/ignite/internal/cluster/ClusterReadOnlyModeCheckedException.java
rename to modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/IgniteClusterReadOnlyException.java
index 2dcde5a..9e8cd87 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/cluster/ClusterReadOnlyModeCheckedException.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/IgniteClusterReadOnlyException.java
@@ -1,4 +1,20 @@
 /*
+ * Copyright 2019 GridGain Systems, Inc. and Contributors.
+ *
+ * Licensed under the GridGain Community Edition License (the "License");
+ * you may not use this file except in compliance with the License.
+ * You may obtain a copy of the License at
+ *
+ *     https://www.gridgain.com/products/software/community-edition/gridgain-community-edition-license
+ *
+ * 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.
+ */
+
+/*
  * 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.
@@ -15,23 +31,22 @@
  * limitations under the License.
  */
 
-package org.apache.ignite.internal.cluster;
+package org.apache.ignite.internal.processors.cache.distributed.dht;
 
 import org.apache.ignite.IgniteCheckedException;
-import org.apache.ignite.IgniteCluster;
 import org.jetbrains.annotations.Nullable;
 
 /**
- * Exception defines modification data error in read-only cluster. See {@link IgniteCluster#readOnly()}
+ * This exception is used to indicate that the cluster is in a read-only state
  */
-public class ClusterReadOnlyModeCheckedException extends IgniteCheckedException {
+public class IgniteClusterReadOnlyException extends IgniteCheckedException {
     /** */
     private static final long serialVersionUID = 0L;
 
     /**
      * Create empty exception.
      */
-    public ClusterReadOnlyModeCheckedException() {
+    public IgniteClusterReadOnlyException() {
         // No-op.
     }
 
@@ -40,7 +55,7 @@ public class ClusterReadOnlyModeCheckedException extends IgniteCheckedException
      *
      * @param msg Error message.
      */
-    public ClusterReadOnlyModeCheckedException(String msg) {
+    public IgniteClusterReadOnlyException(String msg) {
         super(msg);
     }
 
@@ -50,7 +65,7 @@ public class ClusterReadOnlyModeCheckedException extends IgniteCheckedException
      *
      * @param cause Non-null throwable cause.
      */
-    public ClusterReadOnlyModeCheckedException(Throwable cause) {
+    public IgniteClusterReadOnlyException(Throwable cause) {
         super(cause);
     }
 
@@ -62,7 +77,7 @@ public class ClusterReadOnlyModeCheckedException extends IgniteCheckedException
      * @param writableStackTrace whether or not the stack trace should
      *                           be writable
      */
-    public ClusterReadOnlyModeCheckedException(String msg, @Nullable Throwable cause, boolean writableStackTrace) {
+    public IgniteClusterReadOnlyException(String msg, @Nullable Throwable cause, boolean writableStackTrace) {
         super(msg, cause, writableStackTrace);
     }
 
@@ -72,7 +87,7 @@ public class ClusterReadOnlyModeCheckedException extends IgniteCheckedException
      * @param msg Error message.
      * @param cause Optional nested exception (can be {@code null}).
      */
-    public ClusterReadOnlyModeCheckedException(String msg, @Nullable Throwable cause) {
+    public IgniteClusterReadOnlyException(String msg, @Nullable Throwable cause) {
         super(msg, cause);
     }
 }
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/preloader/GridDhtPartitionsExchangeFuture.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/preloader/GridDhtPartitionsExchangeFuture.java
index 8185aa8..46636b1 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/preloader/GridDhtPartitionsExchangeFuture.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/preloader/GridDhtPartitionsExchangeFuture.java
@@ -50,6 +50,7 @@ import org.apache.ignite.IgniteSystemProperties;
 import org.apache.ignite.cache.CacheMode;
 import org.apache.ignite.cache.CacheRebalanceMode;
 import org.apache.ignite.cluster.ClusterNode;
+import org.apache.ignite.cluster.ClusterState;
 import org.apache.ignite.configuration.CacheConfiguration;
 import org.apache.ignite.configuration.IgniteConfiguration;
 import org.apache.ignite.events.DiscoveryEvent;
@@ -129,6 +130,7 @@ import static org.apache.ignite.IgniteSystemProperties.IGNITE_PARTITION_RELEASE_
 import static org.apache.ignite.IgniteSystemProperties.IGNITE_THREAD_DUMP_ON_EXCHANGE_TIMEOUT;
 import static org.apache.ignite.IgniteSystemProperties.getBoolean;
 import static org.apache.ignite.IgniteSystemProperties.getLong;
+import static org.apache.ignite.cluster.ClusterState.active;
 import static org.apache.ignite.events.EventType.EVT_NODE_FAILED;
 import static org.apache.ignite.events.EventType.EVT_NODE_JOINED;
 import static org.apache.ignite.events.EventType.EVT_NODE_LEFT;
@@ -1151,11 +1153,11 @@ public class GridDhtPartitionsExchangeFuture extends GridDhtTopologyFutureAdapte
             exchangeLocE = state.transitionError();
 
         if (req.activeChanged()) {
-            if (req.activate()) {
+            if (active(req.state())) {
                 if (log.isInfoEnabled()) {
                     log.info("Start activation process [nodeId=" + cctx.localNodeId() +
                         ", client=" + kctx.clientNode() +
-                        ", topVer=" + initialVersion() + "]");
+                        ", topVer=" + initialVersion() + "]. New state: " + req.state());
                 }
 
                 try {
@@ -1185,13 +1187,13 @@ public class GridDhtPartitionsExchangeFuture extends GridDhtTopologyFutureAdapte
                     if (log.isInfoEnabled()) {
                         log.info("Successfully activated caches [nodeId=" + cctx.localNodeId() +
                             ", client=" + kctx.clientNode() +
-                            ", topVer=" + initialVersion() + "]");
+                            ", topVer=" + initialVersion() + ", newState=" + req.state() + "]");
                     }
                 }
                 catch (Exception e) {
                     U.error(log, "Failed to activate node components [nodeId=" + cctx.localNodeId() +
                         ", client=" + kctx.clientNode() +
-                        ", topVer=" + initialVersion() + "]", e);
+                        ", topVer=" + initialVersion() + ", newState=" + req.state() + "]", e);
 
                     exchangeLocE = e;
 
@@ -1251,7 +1253,7 @@ public class GridDhtPartitionsExchangeFuture extends GridDhtTopologyFutureAdapte
                 }
             }
         }
-        else if (req.activate()) {
+        else if (active(req.state())) {
             cctx.exchange().exchangerBlockingSectionBegin();
 
             // TODO: BLT changes on inactive cluster can't be handled easily because persistent storage hasn't been initialized yet.
@@ -3756,11 +3758,11 @@ public class GridDhtPartitionsExchangeFuture extends GridDhtTopologyFutureAdapte
                 }
 
                 if (!cctx.kernalContext().state().clusterState().localBaselineAutoAdjustment()) {
-                    boolean active = !stateChangeErr && req.activate();
+                    ClusterState state = stateChangeErr ? ClusterState.INACTIVE : req.state();
 
                     ChangeGlobalStateFinishMessage stateFinishMsg = new ChangeGlobalStateFinishMessage(
                         req.requestId(),
-                        active,
+                        state,
                         !stateChangeErr
                     );
 
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cluster/ChangeGlobalStateFinishMessage.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cluster/ChangeGlobalStateFinishMessage.java
index 25dfadb..93b5457 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cluster/ChangeGlobalStateFinishMessage.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cluster/ChangeGlobalStateFinishMessage.java
@@ -18,6 +18,7 @@
 package org.apache.ignite.internal.processors.cluster;
 
 import java.util.UUID;
+import org.apache.ignite.cluster.ClusterState;
 import org.apache.ignite.internal.managers.discovery.DiscoCache;
 import org.apache.ignite.internal.managers.discovery.DiscoveryCustomMessage;
 import org.apache.ignite.internal.managers.discovery.GridDiscoveryManager;
@@ -40,23 +41,25 @@ public class ChangeGlobalStateFinishMessage implements DiscoveryCustomMessage {
     private final UUID reqId;
 
     /** New cluster state. */
-    private final boolean clusterActive;
+    private final ClusterState state;
 
     /** State change error. */
     private Boolean transitionRes;
 
     /**
      * @param reqId State change request ID.
-     * @param clusterActive New cluster state.
+     * @param state New cluster state.
      */
     public ChangeGlobalStateFinishMessage(
         UUID reqId,
-        boolean clusterActive,
-        Boolean transitionRes) {
+        ClusterState state,
+        Boolean transitionRes
+    ) {
         assert reqId != null;
+        assert state != null;
 
         this.reqId = reqId;
-        this.clusterActive = clusterActive;
+        this.state = state;
         this.transitionRes = transitionRes;
     }
 
@@ -69,16 +72,25 @@ public class ChangeGlobalStateFinishMessage implements DiscoveryCustomMessage {
 
     /**
      * @return New cluster state.
+     * @deprecated Use {@link #state()} instead.
      */
+    @Deprecated
     public boolean clusterActive() {
-        return clusterActive;
+        return ClusterState.active(state);
     }
 
     /**
      * @return Transition success status.
      */
     public boolean success() {
-        return transitionRes == null ? clusterActive : transitionRes;
+        return transitionRes == null ? ClusterState.active(state) : transitionRes;
+    }
+
+    /**
+     * @return New cluster state.
+     */
+    public ClusterState state() {
+        return state;
     }
 
     /** {@inheritDoc} */
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cluster/ChangeGlobalStateMessage.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cluster/ChangeGlobalStateMessage.java
index 8fd2f66..a74917d 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cluster/ChangeGlobalStateMessage.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cluster/ChangeGlobalStateMessage.java
@@ -19,6 +19,7 @@ package org.apache.ignite.internal.processors.cluster;
 
 import java.util.List;
 import java.util.UUID;
+import org.apache.ignite.cluster.ClusterState;
 import org.apache.ignite.internal.managers.discovery.DiscoCache;
 import org.apache.ignite.internal.managers.discovery.DiscoveryCustomMessage;
 import org.apache.ignite.internal.managers.discovery.GridDiscoveryManager;
@@ -47,11 +48,8 @@ public class ChangeGlobalStateMessage implements DiscoveryCustomMessage {
     /** Initiator node ID. */
     private UUID initiatingNodeId;
 
-    /** If true activate else deactivate. */
-    private boolean activate;
-
-    /** If true read-only mode. */
-    private boolean readOnly;
+    /** Cluster state */
+    private ClusterState state;
 
     /** Configurations read from persistent store. */
     private List<StoredCacheData> storedCfgs;
@@ -77,8 +75,7 @@ public class ChangeGlobalStateMessage implements DiscoveryCustomMessage {
      * @param reqId State change request ID.
      * @param initiatingNodeId Node initiated state change.
      * @param storedCfgs Configurations read from persistent store.
-     * @param activate New cluster state.
-     * @param readOnly New read-only mode flag.
+     * @param state New cluster state.
      * @param baselineTopology Baseline topology.
      * @param forceChangeBaselineTopology Force change baseline topology flag.
      * @param timestamp Timestamp.
@@ -87,8 +84,7 @@ public class ChangeGlobalStateMessage implements DiscoveryCustomMessage {
         UUID reqId,
         UUID initiatingNodeId,
         @Nullable List<StoredCacheData> storedCfgs,
-        boolean activate,
-        boolean readOnly,
+        ClusterState state,
         BaselineTopology baselineTopology,
         boolean forceChangeBaselineTopology,
         long timestamp
@@ -99,8 +95,7 @@ public class ChangeGlobalStateMessage implements DiscoveryCustomMessage {
         this.reqId = reqId;
         this.initiatingNodeId = initiatingNodeId;
         this.storedCfgs = storedCfgs;
-        this.activate = activate;
-        this.readOnly = readOnly;
+        this.state = state;
         this.baselineTopology = baselineTopology;
         this.forceChangeBaselineTopology = forceChangeBaselineTopology;
         this.timestamp = timestamp;
@@ -180,17 +175,19 @@ public class ChangeGlobalStateMessage implements DiscoveryCustomMessage {
     }
 
     /**
-     * @return New cluster state.
+     * @return {@code False} if new cluster state is {@link ClusterState#INACTIVE}, and {@code True} otherwise.
+     * @deprecated Use {@link #state()} instead.
      */
+    @Deprecated
     public boolean activate() {
-        return activate;
+        return ClusterState.active(state);
     }
 
     /**
-     * @return Read-only mode flag.
+     * @return New cluster state.
      */
-    public boolean readOnly() {
-        return readOnly;
+    public ClusterState state() {
+        return state;
     }
 
     /**
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cluster/DiscoveryDataClusterState.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cluster/DiscoveryDataClusterState.java
index 8beef24..91e1fb0 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cluster/DiscoveryDataClusterState.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cluster/DiscoveryDataClusterState.java
@@ -20,6 +20,7 @@ package org.apache.ignite.internal.processors.cluster;
 import java.io.Serializable;
 import java.util.Set;
 import java.util.UUID;
+import org.apache.ignite.cluster.ClusterState;
 import org.apache.ignite.internal.processors.affinity.AffinityTopologyVersion;
 import org.apache.ignite.internal.util.tostring.GridToStringExclude;
 import org.apache.ignite.internal.util.tostring.GridToStringInclude;
@@ -28,6 +29,8 @@ import org.apache.ignite.internal.util.typedef.internal.S;
 import org.apache.ignite.internal.util.typedef.internal.U;
 import org.jetbrains.annotations.Nullable;
 
+import static org.apache.ignite.cluster.ClusterState.INACTIVE;
+
 /**
  * A pojo-object representing current cluster global state. The state includes cluster active flag and cluster
  * baseline topology.
@@ -43,14 +46,11 @@ public class DiscoveryDataClusterState implements Serializable {
     /** */
     private static final long serialVersionUID = 0L;
 
-    /** Flag indicating if the cluster in in active state. */
-    private final boolean active;
-
-    /** Flag indicating if the cluster in read-only mode. */
-    private final boolean readOnly;
+    /** Current cluster state. */
+    private final ClusterState state;
 
-    /** Read-only mode change time. Correctly work's only for enabling read-only mode. */
-    private final long readOnlyChangeTime;
+    /** Time of last cluster state change. */
+    private final long lastStateChangeTime;
 
     /** Current cluster baseline topology. */
     @Nullable private final BaselineTopology baselineTopology;
@@ -61,6 +61,9 @@ public class DiscoveryDataClusterState implements Serializable {
      */
     private final UUID transitionReqId;
 
+    /** Previous cluster state. May not null only if cluster in transition. */
+    private final ClusterState prevClusterState;
+
     /**
      * Topology version in the cluster when state change request was received by the coordinator.
      * The exchange fired for the cluster state change will be on version {@code transitionTopVer.nextMinorVersion()}.
@@ -76,7 +79,7 @@ public class DiscoveryDataClusterState implements Serializable {
      * Local flag for state transition active state result (global state is updated asynchronously by custom message),
      * {@code null} means that state change is not completed yet.
      */
-    private transient volatile Boolean transitionRes;
+    private transient volatile ClusterState transitionRes;
 
     /**
      * Previous cluster state if this state is a transition state and it was not received by a joining node.
@@ -90,29 +93,29 @@ public class DiscoveryDataClusterState implements Serializable {
     private transient volatile boolean locBaselineAutoAdjustment;
 
     /**
-     * @param active Current status.
+     * @param state Current cluster state.
+     * @param baselineTopology Baseline topology associated with this state.
      * @return State instance.
      */
     static DiscoveryDataClusterState createState(
-        boolean active,
-        boolean readOnly,
+        ClusterState state,
         @Nullable BaselineTopology baselineTopology
     ) {
-        return new DiscoveryDataClusterState(null, active, readOnly, baselineTopology, null, null, null);
+        return new DiscoveryDataClusterState(null, state, baselineTopology, null, null, null, null);
     }
 
     /**
-     * @param active New status.
-     * @param readOnly New read-only mode.
+     * @param state New cluster state.
+     * @param prevState Previous state.
+     * @param baselineTopology Baseline topology for new cluster state.
      * @param transitionReqId State change request ID.
      * @param transitionTopVer State change topology version.
      * @param transitionNodes Nodes participating in state change exchange.
-     * @return State instance.
+     * @return Discovery cluster state instance.
      */
     static DiscoveryDataClusterState createTransitionState(
+        ClusterState state,
         DiscoveryDataClusterState prevState,
-        boolean active,
-        boolean readOnly,
         @Nullable BaselineTopology baselineTopology,
         UUID transitionReqId,
         AffinityTopologyVersion transitionTopVer,
@@ -125,46 +128,59 @@ public class DiscoveryDataClusterState implements Serializable {
 
         return new DiscoveryDataClusterState(
             prevState,
-            active,
-            readOnly,
+            state,
             baselineTopology,
             transitionReqId,
             transitionTopVer,
-            transitionNodes
+            transitionNodes,
+            prevState.state
         );
     }
 
     /**
      * @param prevState Previous state. May be non-null only for transitional states.
-     * @param active New state.
-     * @param readOnly New read-only mode.
+     * @param state New cluster state.
+     * @param baselineTopology Baseline topology for new cluster state.
      * @param transitionReqId State change request ID.
      * @param transitionTopVer State change topology version.
      * @param transitionNodes Nodes participating in state change exchange.
+     * @param prevClusterState Nodes participating in state change exchange.
      */
     private DiscoveryDataClusterState(
         DiscoveryDataClusterState prevState,
-        boolean active,
-        boolean readOnly,
+        ClusterState state,
         @Nullable BaselineTopology baselineTopology,
         @Nullable UUID transitionReqId,
         @Nullable AffinityTopologyVersion transitionTopVer,
-        @Nullable Set<UUID> transitionNodes
+        @Nullable Set<UUID> transitionNodes,
+        @Nullable ClusterState prevClusterState
     ) {
+        assert state != null;
+
         this.prevState = prevState;
-        this.active = active;
-        this.readOnly = readOnly;
-        this.readOnlyChangeTime = U.currentTimeMillis();
+        this.state = state;
+        this.lastStateChangeTime = U.currentTimeMillis();
         this.baselineTopology = baselineTopology;
         this.transitionReqId = transitionReqId;
         this.transitionTopVer = transitionTopVer;
         this.transitionNodes = transitionNodes;
+        this.prevClusterState = prevClusterState;
+    }
+
+    /**
+     * @return Cluster state before transition if cluster in transition and current cluster state otherwise.
+     */
+    public ClusterState lastState() {
+        if (transition())
+            return prevClusterState;
+        else
+            return state;
     }
 
     /**
      * @return Local flag for state transition result (global state is updated asynchronously by custom message).
      */
-    @Nullable public Boolean transitionResult() {
+    @Nullable public ClusterState transitionResult() {
         return transitionRes;
     }
 
@@ -173,11 +189,11 @@ public class DiscoveryDataClusterState implements Serializable {
      * for public API calls.
      *
      * @param reqId Request ID.
-     * @param active New cluster state.
+     * @param state New cluster state.
      */
-    public void setTransitionResult(UUID reqId, boolean active) {
+    public void setTransitionResult(UUID reqId, ClusterState state) {
         if (reqId.equals(transitionReqId))
-            transitionRes = active;
+            transitionRes = state;
     }
 
     /**
@@ -203,23 +219,25 @@ public class DiscoveryDataClusterState implements Serializable {
 
     /**
      * @return Current cluster state (or new state in case when transition is in progress).
+     * @deprecated Use {@link #state()} instead.
      */
+    @Deprecated
     public boolean active() {
-        return active;
+        return ClusterState.active(state);
     }
 
     /**
-     * @return Read only mode enabled flag.
+     * @return Current cluster state (or new state in case when transition is in progress).
      */
-    public boolean readOnly() {
-        return readOnly;
+    public ClusterState state() {
+        return state;
     }
 
     /**
-     * @return Change time read-only mode.
+     * @return Time of last cluster state change.
      */
-    public long readOnlyModeChangeTime() {
-        return readOnlyChangeTime;
+    public long lastStateChangeTime() {
+        return lastStateChangeTime;
     }
 
     /**
@@ -307,17 +325,10 @@ public class DiscoveryDataClusterState implements Serializable {
      * @return Cluster state that finished transition.
      */
     public DiscoveryDataClusterState finish(boolean success) {
-        return success ?
-            new DiscoveryDataClusterState(
-                null,
-                active,
-                readOnly,
-                baselineTopology,
-                null,
-                null,
-                null
-            ) :
-            prevState != null ? prevState : createState(false, false, null);
+        if(success)
+            return createState(state, baselineTopology);
+        else
+            return prevState != null ? prevState : createState(INACTIVE, null);
     }
 
     /** {@inheritDoc} */
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cluster/GridClusterStateProcessor.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cluster/GridClusterStateProcessor.java
index ecbc2aa..1c7fb55 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cluster/GridClusterStateProcessor.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cluster/GridClusterStateProcessor.java
@@ -18,6 +18,7 @@
 package org.apache.ignite.internal.processors.cluster;
 
 import java.io.Serializable;
+import java.lang.reflect.Field;
 import java.util.ArrayList;
 import java.util.Collection;
 import java.util.HashMap;
@@ -37,7 +38,9 @@ import org.apache.ignite.IgniteException;
 import org.apache.ignite.IgniteLogger;
 import org.apache.ignite.cluster.BaselineNode;
 import org.apache.ignite.cluster.ClusterNode;
+import org.apache.ignite.cluster.ClusterState;
 import org.apache.ignite.configuration.DataStorageConfiguration;
+import org.apache.ignite.configuration.IgniteConfiguration;
 import org.apache.ignite.events.DiscoveryEvent;
 import org.apache.ignite.events.Event;
 import org.apache.ignite.internal.GridKernalContext;
@@ -92,10 +95,16 @@ import org.apache.ignite.spi.IgniteNodeValidationResult;
 import org.apache.ignite.spi.discovery.DiscoveryDataBag;
 import org.jetbrains.annotations.Nullable;
 
+import static org.apache.ignite.cluster.ClusterState.ACTIVE;
+import static org.apache.ignite.cluster.ClusterState.INACTIVE;
+import static org.apache.ignite.cluster.ClusterState.ACTIVE_READ_ONLY;
+import static org.apache.ignite.cluster.ClusterState.lesserOf;
+import static org.apache.ignite.configuration.IgniteConfiguration.DFLT_STATE_ON_START;
 import static org.apache.ignite.events.EventType.EVT_NODE_FAILED;
 import static org.apache.ignite.events.EventType.EVT_NODE_JOINED;
 import static org.apache.ignite.events.EventType.EVT_NODE_LEFT;
 import static org.apache.ignite.internal.GridComponent.DiscoveryDataExchangeType.STATE_PROC;
+import static org.apache.ignite.internal.IgniteFeatures.CLUSTER_READ_ONLY_MODE;
 import static org.apache.ignite.internal.managers.communication.GridIoPolicy.SYSTEM_POOL;
 import static org.apache.ignite.internal.processors.cache.GridCacheUtils.extractDataStorage;
 
@@ -230,22 +239,12 @@ public class GridClusterStateProcessor extends GridProcessorAdapter implements I
     }
 
     /** {@inheritDoc} */
-    @Override public boolean publicApiReadOnlyMode() {
-        return globalState.readOnly();
+    @Override public ClusterState publicApiState(boolean waitForTransition) {
+        return publicApiStateAsync(waitForTransition).get();
     }
 
     /** {@inheritDoc} */
-    @Override public long readOnlyModeStateChangeTime() {
-        return globalState.readOnlyModeChangeTime();
-    }
-
-    /** {@inheritDoc} */
-    @Override public boolean publicApiActiveState(boolean waitForTransition) {
-        return publicApiActiveStateAsync(waitForTransition).get();
-    }
-
-    /** {@inheritDoc} */
-    @Override public IgniteFuture<Boolean> publicApiActiveStateAsync(boolean asyncWaitForTransition) {
+    @Override public IgniteFuture<ClusterState> publicApiStateAsync(boolean asyncWaitForTransition) {
         if (ctx.isDaemon())
             return sendComputeCheckGlobalState();
 
@@ -253,8 +252,8 @@ public class GridClusterStateProcessor extends GridProcessorAdapter implements I
 
         assert globalState != null;
 
-        if (globalState.transition() && globalState.active()) {
-            Boolean transitionRes = globalState.transitionResult();
+        if (globalState.transition() && active(globalState)) {
+            ClusterState transitionRes = globalState.transitionResult();
 
             if (transitionRes != null)
                 return new IgniteFinishedFutureImpl<>(transitionRes);
@@ -263,18 +262,16 @@ public class GridClusterStateProcessor extends GridProcessorAdapter implements I
 
                 if (fut != null) {
                     if (asyncWaitForTransition) {
-                         return new IgniteFutureImpl<>(fut.chain(new C1<IgniteInternalFuture<Void>, Boolean>() {
-                            @Override public Boolean apply(IgniteInternalFuture<Void> fut) {
-                                Boolean res = globalState.transitionResult();
+                        return new IgniteFutureImpl<>(fut.chain((C1<IgniteInternalFuture<Void>, ClusterState>)f -> {
+                            ClusterState res = globalState.transitionResult();
 
-                                assert res != null;
+                            assert res != null;
 
-                                return res;
-                            }
+                            return res;
                         }));
                     }
                     else
-                        return new IgniteFinishedFutureImpl<>(globalState.baselineChanged());
+                        return new IgniteFinishedFutureImpl<>(lesserOf(globalState.lastState(), globalState.state()));
                 }
 
                 transitionRes = globalState.transitionResult();
@@ -285,7 +282,22 @@ public class GridClusterStateProcessor extends GridProcessorAdapter implements I
             }
         }
         else
-            return new IgniteFinishedFutureImpl<>(globalState.active());
+            return new IgniteFinishedFutureImpl<>(globalState.state());
+    }
+
+    /** {@inheritDoc} */
+    @Override public boolean publicApiActiveState(boolean waitForTransition) {
+        return publicApiActiveStateAsync(waitForTransition).get();
+    }
+
+    /** {@inheritDoc} */
+    @Override public IgniteFuture<Boolean> publicApiActiveStateAsync(boolean asyncWaitForTransition) {
+        return publicApiStateAsync(asyncWaitForTransition).chain(f -> ClusterState.active(f.get()));
+    }
+
+    /** {@inheritDoc} */
+    @Override public long lastStateChangeTime() {
+        return globalState.lastStateChangeTime();
     }
 
     /** {@inheritDoc} */
@@ -374,12 +386,35 @@ public class GridClusterStateProcessor extends GridProcessorAdapter implements I
 
     /** {@inheritDoc} */
     @Override public void start() throws IgniteCheckedException {
-        inMemoryMode = !CU.isPersistenceEnabled(ctx.config());
+        IgniteConfiguration cfg = ctx.config();
 
-        // Start first node as inactive if persistence is enabled.
-        boolean activeOnStart = inMemoryMode && ctx.config().isActiveOnStart();
+        inMemoryMode = !CU.isPersistenceEnabled(cfg);
 
-        globalState = DiscoveryDataClusterState.createState(activeOnStart, false, null);
+        ClusterState stateOnStart;
+
+        if (inMemoryMode) {
+            stateOnStart = cfg.getClusterStateOnStart();
+
+            boolean activeOnStartSet = getBooleanFieldFromConfig(cfg, "activeOnStartPropSetFlag", false);
+
+            if (activeOnStartSet) {
+                if (stateOnStart != null)
+                    log.warning("Property `activeOnStart` will be ignored due to the property `clusterStateOnStart` is presented.");
+                else
+                    stateOnStart = cfg.isActiveOnStart() ? ACTIVE : INACTIVE;
+            }
+            else if (stateOnStart == null)
+                stateOnStart = DFLT_STATE_ON_START;
+        }
+        else {
+            // Start first node as inactive if persistence is enabled.
+            stateOnStart = INACTIVE;
+
+            if (cfg.getClusterStateOnStart() != null && getBooleanFieldFromConfig(cfg, "autoActivationPropSetFlag", false))
+                log.warning("Property `autoActivation` will be ignored due to the property `clusterStateOnStart` is presented.");
+        }
+
+        globalState = DiscoveryDataClusterState.createState(stateOnStart, null);
 
         ctx.event().addLocalEventListener(lsr, EVT_NODE_LEFT, EVT_NODE_FAILED);
     }
@@ -406,7 +441,7 @@ public class GridClusterStateProcessor extends GridProcessorAdapter implements I
     @Override @Nullable public IgniteInternalFuture<Boolean> onLocalJoin(DiscoCache discoCache) {
         final DiscoveryDataClusterState state = globalState;
 
-        if (state.active())
+        if (active(state))
             checkLocalNodeInBaseline(state.baselineTopology());
 
         if (state.transition()) {
@@ -414,14 +449,22 @@ public class GridClusterStateProcessor extends GridProcessorAdapter implements I
 
             return joinFut;
         }
-        else if (!ctx.clientNode()
+        else
+        {
+            ClusterState targetState = ctx.config().getClusterStateOnStart();
+
+            if (targetState == null)
+                targetState = ctx.config().isAutoActivationEnabled() ? ACTIVE : INACTIVE;
+
+            if (!ctx.clientNode()
                 && !ctx.isDaemon()
-                && ctx.config().isAutoActivationEnabled()
-                && !state.active()
+                && !active(state)
+                && ClusterState.active(targetState)
                 && !inMemoryMode
                 && isBaselineSatisfied(state.baselineTopology(), discoCache.serverNodes())
-        )
-            changeGlobalState(true, state.baselineTopology().currentBaseline(), false);
+            )
+                changeGlobalState(targetState, state.baselineTopology().currentBaseline(), false);
+        }
 
         return null;
     }
@@ -469,7 +512,7 @@ public class GridClusterStateProcessor extends GridProcessorAdapter implements I
                     "Switching to inactive state.");
 
                 ChangeGlobalStateFinishMessage msg =
-                    new ChangeGlobalStateFinishMessage(globalState.transitionRequestId(), false, false);
+                    new ChangeGlobalStateFinishMessage(globalState.transitionRequestId(), INACTIVE, false);
 
                 onStateFinishMessage(msg);
 
@@ -482,37 +525,35 @@ public class GridClusterStateProcessor extends GridProcessorAdapter implements I
 
     /** {@inheritDoc} */
     @Override public void onStateFinishMessage(ChangeGlobalStateFinishMessage msg) {
-        DiscoveryDataClusterState state = globalState;
+        DiscoveryDataClusterState discoClusterState = globalState;
 
-        if (msg.requestId().equals(state.transitionRequestId())) {
-            log.info("Received state change finish message: " + msg.clusterActive());
+        if (msg.requestId().equals(discoClusterState.transitionRequestId())) {
+            if (log.isInfoEnabled())
+                log.info("Received state change finish message: " + msg.state());
 
-            globalState = state.finish(msg.success());
+            globalState = discoClusterState.finish(msg.success());
 
             afterStateChangeFinished(msg.id(), msg.success());
 
             ctx.cache().onStateChangeFinish(msg);
 
-            boolean prev = ctx.cache().context().readOnlyMode();
+            if (readOnly(discoClusterState.lastState()) || readOnly(globalState.state()))
+                ctx.cache().context().readOnlyMode(readOnly(globalState.state()));
 
-            if (prev != globalState.readOnly()) {
-                ctx.cache().context().readOnlyMode(globalState.readOnly());
+            log.info("Cluster state was changed from " + discoClusterState.lastState() + " to " + globalState.state());
 
-                if (globalState.readOnly())
-                    log.info("Read-only mode is enabled");
-                else
-                    log.info("Read-only mode is disabled");
-            }
+            if (!ClusterState.active(globalState.state()))
+                ctx.cache().context().readOnlyMode(false);
 
             TransitionOnJoinWaitFuture joinFut = this.joinFut;
 
             if (joinFut != null)
                 joinFut.onDone(false);
 
-            GridFutureAdapter<Void> transitionFut = transitionFuts.remove(state.transitionRequestId());
+            GridFutureAdapter<Void> transitionFut = transitionFuts.remove(discoClusterState.transitionRequestId());
 
             if (transitionFut != null) {
-                state.setTransitionResult(msg.requestId(), msg.clusterActive());
+                discoClusterState.setTransitionResult(msg.requestId(), msg.state());
 
                 transitionFut.onDone();
             }
@@ -534,13 +575,16 @@ public class GridClusterStateProcessor extends GridProcessorAdapter implements I
     ) {
         DiscoveryDataClusterState state = globalState;
 
-        U.log(
-            log,
-            "Received " + prettyStr(msg.activate(), msg.readOnly(), readOnlyChanged(state, msg.readOnly())) +
-                " request with BaselineTopology" +
-                (msg.baselineTopology() == null ? ": null" : "[id=" + msg.baselineTopology().id() + "]") +
-                " initiator node ID: " + msg.initiatorNodeId()
-        );
+        if (log.isInfoEnabled()) {
+            String baseline = msg.baselineTopology() == null ? ": null" : "[id=" + msg.baselineTopology().id() + ']';
+
+            U.log(
+                log,
+                "Received " + prettyStr(msg.state()) +
+                    " request with BaselineTopology" + baseline +
+                    " initiator node ID: " + msg.initiatorNodeId()
+            );
+        }
 
         if (msg.baselineTopology() != null)
             compatibilityMode = false;
@@ -550,7 +594,7 @@ public class GridClusterStateProcessor extends GridProcessorAdapter implements I
                 GridChangeGlobalStateFuture fut = changeStateFuture(msg);
 
                 if (fut != null)
-                    fut.onDone(concurrentStateChangeError(msg.activate()));
+                    fut.onDone(concurrentStateChangeError(msg.state(), state.state()));
             }
             else {
                 final GridChangeGlobalStateFuture stateFut = changeStateFuture(msg);
@@ -601,7 +645,7 @@ public class GridClusterStateProcessor extends GridProcessorAdapter implements I
                     fut.setRemaining(nodeIds, topVer.nextMinorVersion());
 
                 if (log.isInfoEnabled())
-                    log.info("Started state transition: " + msg.activate());
+                    log.info("Started state transition: " + prettyStr(msg.state()));
 
                 BaselineTopologyHistoryItem bltHistItem = BaselineTopologyHistoryItem.fromBaseline(
                     globalState.baselineTopology());
@@ -611,21 +655,25 @@ public class GridClusterStateProcessor extends GridProcessorAdapter implements I
                 DiscoveryDataClusterState prevState = globalState;
 
                 globalState = DiscoveryDataClusterState.createTransitionState(
+                    msg.state(),
                     prevState,
-                    msg.activate(),
-                    msg.readOnly(),
-                    msg.activate() ? msg.baselineTopology() : prevState.baselineTopology(),
+                    activate(prevState.state(), msg.state()) || msg.forceChangeBaselineTopology() ? msg.baselineTopology() : prevState.baselineTopology(),
                     msg.requestId(),
                     topVer,
                     nodeIds
                 );
 
                 if (msg.forceChangeBaselineTopology())
-                    globalState.setTransitionResult(msg.requestId(), msg.activate());
+                    globalState.setTransitionResult(msg.requestId(), msg.state());
 
                 AffinityTopologyVersion stateChangeTopVer = topVer.nextMinorVersion();
 
-                StateChangeRequest req = new StateChangeRequest(msg, bltHistItem, msg.activate() != state.active(), stateChangeTopVer);
+                StateChangeRequest req = new StateChangeRequest(
+                    msg,
+                    bltHistItem,
+                    state.state(),
+                    stateChangeTopVer
+                );
 
                 exchangeActions.stateChangeRequest(req);
 
@@ -646,6 +694,21 @@ public class GridClusterStateProcessor extends GridProcessorAdapter implements I
     }
 
     /**
+     * Same behaviour as {@link ClusterState#active(ClusterState)}
+     */
+    private static boolean active(DiscoveryDataClusterState discoDataClusterState) {
+        return ClusterState.active(discoDataClusterState.state());
+    }
+
+    /**
+     * @param state Cluster state
+     * @return {@code True} passed {@code state} is {@link ClusterState#ACTIVE_READ_ONLY}, and {@code False} otherwise.
+     */
+    private static boolean readOnly(ClusterState state) {
+        return state == ACTIVE_READ_ONLY;
+    }
+
+    /**
      * @param msg State change message.
      * @param state Current cluster state.
      * @return {@code True} if state change from message can be applied to the current state.
@@ -660,9 +723,8 @@ public class GridClusterStateProcessor extends GridProcessorAdapter implements I
      * @return {@code True} if states are equivalent.
      */
     protected static boolean isEquivalent(ChangeGlobalStateMessage msg, DiscoveryDataClusterState state) {
-        return msg.activate() == state.active() &&
-            msg.readOnly() == state.readOnly() &&
-            BaselineTopology.equals(msg.baselineTopology(), state.baselineTopology());
+        return msg.state() == state.state()
+            && BaselineTopology.equals(msg.baselineTopology(), state.baselineTopology());
     }
 
     /** {@inheritDoc} */
@@ -672,11 +734,14 @@ public class GridClusterStateProcessor extends GridProcessorAdapter implements I
 
     /** {@inheritDoc} */
     @Override public DiscoveryDataClusterState pendingState(ChangeGlobalStateMessage stateMsg) {
-        return DiscoveryDataClusterState.createState(
-            stateMsg.activate() || stateMsg.forceChangeBaselineTopology(),
-            stateMsg.readOnly(),
-            stateMsg.baselineTopology()
-        );
+        ClusterState state;
+
+        if (ClusterState.active(stateMsg.state()))
+            state = stateMsg.state();
+        else
+            state = stateMsg.forceChangeBaselineTopology() ? ACTIVE : INACTIVE;
+
+        return DiscoveryDataClusterState.createState(state, stateMsg.baselineTopology());
     }
 
     /**
@@ -707,12 +772,13 @@ public class GridClusterStateProcessor extends GridProcessorAdapter implements I
     }
 
     /**
-     * @param activate New state.
+     * @param state New state.
+     * @param transitionState State in transition.
      * @return State change error.
      */
-    protected IgniteCheckedException concurrentStateChangeError(boolean activate) {
-        return new IgniteCheckedException("Failed to " + prettyStr(activate) +
-            ", because another state change operation is currently in progress: " + prettyStr(!activate));
+    protected IgniteCheckedException concurrentStateChangeError(ClusterState state, ClusterState transitionState) {
+        return new IgniteCheckedException("Failed to " + prettyStr(state) +
+            ", because another state change operation is currently in progress: " + prettyStr(transitionState));
     }
 
     /** {@inheritDoc} */
@@ -822,7 +888,10 @@ public class GridClusterStateProcessor extends GridProcessorAdapter implements I
 
             compatibilityMode = true;
 
-            ctx.cache().context().readOnlyMode(globalState.readOnly());
+            ctx.cache().context().readOnlyMode(readOnly(globalState.state()));
+
+            if (readOnly(globalState.state()))
+                ctx.cache().context().database().forceCheckpoint("Cluster read-only mode enabled");
 
             return;
         }
@@ -842,7 +911,12 @@ public class GridClusterStateProcessor extends GridProcessorAdapter implements I
                     bltHist.bufferHistoryItemForStore(item);
             }
 
-            ctx.cache().context().readOnlyMode(globalState.readOnly());
+            final boolean readOnly = readOnly(globalState.state());
+
+            ctx.cache().context().readOnlyMode(readOnly);
+
+            if (readOnly)
+                ctx.cache().context().database().forceCheckpoint("Cluster read-only mode enabled");
         }
     }
 
@@ -857,24 +931,11 @@ public class GridClusterStateProcessor extends GridProcessorAdapter implements I
 
     /** {@inheritDoc} */
     @Override public IgniteInternalFuture<?> changeGlobalState(
-        boolean activate,
-        boolean readOnly,
+        ClusterState state,
         Collection<? extends BaselineNode> baselineNodes,
         boolean forceChangeBaselineTopology
     ) {
-        return changeGlobalState(activate, readOnly, baselineNodes, forceChangeBaselineTopology, false);
-    }
-
-    /** {@inheritDoc} */
-    @Override public IgniteInternalFuture<?> changeGlobalState(boolean readOnly) {
-        if (!publicApiActiveState(false))
-            return new GridFinishedFuture<>(new IgniteException("Cluster not active"));
-
-        DiscoveryDataClusterState state = globalState;
-
-        List<BaselineNode> bltNodes = state.hasBaselineTopology() ? state.baselineTopology().currentBaseline() : null;
-
-        return changeGlobalState(state.active(), readOnly, bltNodes, false, false);
+        return changeGlobalState(state, baselineNodes, forceChangeBaselineTopology, false);
     }
 
     /**
@@ -883,55 +944,52 @@ public class GridClusterStateProcessor extends GridProcessorAdapter implements I
      * @param forceChangeBaselineTopology Force change BLT.
      * @param isAutoAdjust Auto adjusting flag.
      * @return Global change state future.
+     * @deprecated Use {@link #changeGlobalState(ClusterState, Collection, boolean, boolean)} instead.
      */
+    @Deprecated
     public IgniteInternalFuture<?> changeGlobalState(
         final boolean activate,
         Collection<? extends BaselineNode> baselineNodes,
         boolean forceChangeBaselineTopology,
         boolean isAutoAdjust
     ) {
-        boolean readOnly = activate && globalState.readOnly();
-
-        return changeGlobalState(activate, readOnly, baselineNodes, forceChangeBaselineTopology, isAutoAdjust);
+        return changeGlobalState(activate ? ACTIVE : INACTIVE, baselineNodes, forceChangeBaselineTopology, isAutoAdjust);
     }
 
     /**
-     * @param activate New activate state.
-     * @param readOnly Read-only mode.
+     * @param state New activate state.
      * @param baselineNodes New BLT nodes.
      * @param forceChangeBaselineTopology Force change BLT.
      * @param isAutoAdjust Auto adjusting flag.
      * @return Global change state future.
      */
     public IgniteInternalFuture<?> changeGlobalState(
-        final boolean activate,
-        boolean readOnly,
+        ClusterState state,
         Collection<? extends BaselineNode> baselineNodes,
         boolean forceChangeBaselineTopology,
         boolean isAutoAdjust
     ) {
-        BaselineTopology newBlt = (compatibilityMode && !forceChangeBaselineTopology) ? null :
-            calculateNewBaselineTopology(activate, baselineNodes, forceChangeBaselineTopology);
+        BaselineTopology newBlt = (compatibilityMode && !forceChangeBaselineTopology) ?
+            null :
+            calculateNewBaselineTopology(state, baselineNodes, forceChangeBaselineTopology);
 
-        return changeGlobalState0(activate, readOnly, newBlt, forceChangeBaselineTopology, isAutoAdjust);
+        return changeGlobalState0(state, newBlt, forceChangeBaselineTopology, isAutoAdjust);
     }
 
-    /**
-     *
-     */
+    /** */
     private BaselineTopology calculateNewBaselineTopology(
-        final boolean activate,
+        ClusterState state,
         Collection<? extends BaselineNode> baselineNodes,
         boolean forceChangeBaselineTopology
     ) {
         BaselineTopology newBlt;
 
-        BaselineTopology currentBlt = globalState.baselineTopology();
+        BaselineTopology currBlt = globalState.baselineTopology();
 
         int newBltId = 0;
 
-        if (currentBlt != null)
-            newBltId = activate ? currentBlt.id() + 1 : currentBlt.id();
+        if (currBlt != null)
+            newBltId = ClusterState.active(state) ? currBlt.id() + 1 : currBlt.id();
 
         if (baselineNodes != null && !baselineNodes.isEmpty()) {
             List<BaselineNode> baselineNodes0 = new ArrayList<>();
@@ -952,14 +1010,14 @@ public class GridClusterStateProcessor extends GridProcessorAdapter implements I
 
         if (forceChangeBaselineTopology)
             newBlt = BaselineTopology.build(baselineNodes, newBltId);
-        else if (activate) {
+        else if (ClusterState.active(state)) {
             if (baselineNodes == null)
                 baselineNodes = baselineNodes();
 
-            if (currentBlt == null)
+            if (currBlt == null)
                 newBlt = BaselineTopology.build(baselineNodes, newBltId);
             else {
-                newBlt = currentBlt;
+                newBlt = currBlt;
 
                 newBlt.updateHistory(baselineNodes);
             }
@@ -984,8 +1042,7 @@ public class GridClusterStateProcessor extends GridProcessorAdapter implements I
 
     /** */
     private IgniteInternalFuture<?> changeGlobalState0(
-        final boolean activate,
-        boolean readOnly,
+        ClusterState state,
         BaselineTopology blt,
         boolean forceChangeBaselineTopology,
         boolean isAutoAdjust
@@ -995,36 +1052,29 @@ public class GridClusterStateProcessor extends GridProcessorAdapter implements I
         if (forceChangeBaselineTopology && isBaselineAutoAdjustEnabled != isAutoAdjust)
             throw new BaselineAdjustForbiddenException(isBaselineAutoAdjustEnabled);
 
-        if (ctx.isDaemon() || ctx.clientNode()) {
-            GridFutureAdapter<Void> fut = new GridFutureAdapter<>();
-
-            sendComputeChangeGlobalState(activate, readOnly, blt, forceChangeBaselineTopology, fut);
-
-            return fut;
-        }
+        if (ctx.isDaemon() || ctx.clientNode())
+            return sendComputeChangeGlobalState(state, blt, forceChangeBaselineTopology);
 
         if (cacheProc.transactions().tx() != null || sharedCtx.lockedTopologyVersion(null) != null) {
             return new GridFinishedFuture<>(
-                new IgniteCheckedException("Failed to " +
-                    prettyStr(activate, readOnly, readOnlyChanged(globalState, readOnly)) +
+                new IgniteCheckedException("Failed to " + prettyStr(state) +
                     " (must invoke the method outside of an active transaction).")
             );
         }
 
         DiscoveryDataClusterState curState = globalState;
 
-        if (!curState.transition() &&
-            curState.active() == activate &&
-            curState.readOnly() == readOnly
-            && (!activate || BaselineTopology.equals(curState.baselineTopology(), blt)))
-            return new GridFinishedFuture<>();
+        if (!curState.transition() && curState.state() == state) {
+            if (!ClusterState.active(state) || BaselineTopology.equals(curState.baselineTopology(), blt))
+                return new GridFinishedFuture<>();
+        }
 
         GridChangeGlobalStateFuture startedFut = null;
 
         GridChangeGlobalStateFuture fut = stateChangeFut.get();
 
         while (fut == null || fut.isDone()) {
-            fut = new GridChangeGlobalStateFuture(UUID.randomUUID(), activate, readOnly, ctx);
+            fut = new GridChangeGlobalStateFuture(UUID.randomUUID(), state, ctx);
 
             if (stateChangeFut.compareAndSet(null, fut)) {
                 startedFut = fut;
@@ -1036,12 +1086,11 @@ public class GridClusterStateProcessor extends GridProcessorAdapter implements I
         }
 
         if (startedFut == null) {
-            if (fut.activate != activate && fut.readOnly != readOnly) {
+            if (fut.state != state) {
                 return new GridFinishedFuture<>(
                     new IgniteCheckedException(
-                        "Failed to " + prettyStr(activate, readOnly, readOnlyChanged(globalState, readOnly)) +
-                        ", because another state change operation is currently in progress: " +
-                            prettyStr(fut.activate, fut.readOnly, readOnlyChanged(globalState, fut.readOnly))
+                        "Failed to " + prettyStr(state) +
+                        ", because another state change operation is currently in progress: " + prettyStr(fut.state)
                     )
                 );
             }
@@ -1051,7 +1100,7 @@ public class GridClusterStateProcessor extends GridProcessorAdapter implements I
 
         List<StoredCacheData> storedCfgs = null;
 
-        if (activate && !inMemoryMode) {
+        if (activate(curState.state(), state) && !inMemoryMode) {
             try {
                 Map<String, StoredCacheData> cfgs = ctx.cache().context().pageStore().readCacheConfigurations();
 
@@ -1079,11 +1128,11 @@ public class GridClusterStateProcessor extends GridProcessorAdapter implements I
             }
         }
 
-        ChangeGlobalStateMessage msg = new ChangeGlobalStateMessage(startedFut.requestId,
+        ChangeGlobalStateMessage msg = new ChangeGlobalStateMessage(
+            startedFut.requestId,
             ctx.localNodeId(),
             storedCfgs,
-            activate,
-            readOnly,
+            state,
             blt,
             forceChangeBaselineTopology,
             System.currentTimeMillis()
@@ -1092,26 +1141,18 @@ public class GridClusterStateProcessor extends GridProcessorAdapter implements I
         IgniteInternalFuture<?> resFut = wrapStateChangeFuture(startedFut, msg);
 
         try {
-            U.log(
-                log,
-                "Sending " + prettyStr(activate, readOnly, readOnlyChanged(globalState, readOnly)) +
-                    " request with BaselineTopology " + blt
-            );
+            U.log(log, "Sending " + prettyStr(state) + " request with BaselineTopology " + blt);
 
             ctx.discovery().sendCustomEvent(msg);
 
             if (ctx.isStopping()) {
                 startedFut.onDone(
-                    new IgniteCheckedException(
-                        "Failed to execute " +
-                            prettyStr(activate, readOnly, readOnlyChanged(globalState, readOnly)) +
-                            " request , node is stopping."
-                    )
+                    new IgniteCheckedException("Failed to execute " + prettyStr(state) + " request , node is stopping.")
                 );
             }
         }
         catch (IgniteCheckedException e) {
-            U.error(log, "Failed to send global state change request: " + activate, e);
+            U.error(log, "Failed to send global state change request: " + prettyStr(state), e);
 
             startedFut.onDone(e);
         }
@@ -1127,7 +1168,7 @@ public class GridClusterStateProcessor extends GridProcessorAdapter implements I
         if (node.isClient() || node.isDaemon())
             return null;
 
-        if (globalState.readOnly() && !IgniteFeatures.nodeSupports(node, IgniteFeatures.CLUSTER_READ_ONLY_MODE)) {
+        if (globalState.state() == ACTIVE_READ_ONLY && !IgniteFeatures.nodeSupports(node, CLUSTER_READ_ONLY_MODE)) {
             String msg = "Node not supporting cluster read-only mode is not allowed to join the cluster with enabled" +
                 " read-only mode";
 
@@ -1242,22 +1283,20 @@ public class GridClusterStateProcessor extends GridProcessorAdapter implements I
     }
 
     /**
-     * @param activate New cluster state.
-     * @param readOnly New read-only mode.
-     * @param resFut State change future.
+     * @param state New cluster state.
+     * @param blt New cluster state.
+     * @param forceBlt New cluster state.
      */
-    private void sendComputeChangeGlobalState(
-        boolean activate,
-        boolean readOnly,
+    private IgniteInternalFuture<Void> sendComputeChangeGlobalState(
+        ClusterState state,
         BaselineTopology blt,
-        boolean forceBlt,
-        final GridFutureAdapter<Void> resFut
+        boolean forceBlt
     ) {
         AffinityTopologyVersion topVer = ctx.discovery().topologyVersionEx();
 
         U.log(
             log,
-            "Sending " + prettyStr(activate, readOnly, readOnlyChanged(globalState, readOnly)) +
+            "Sending " + prettyStr(state) +
                 " request from node [id=" + ctx.localNodeId() +
                 ", topVer=" + topVer +
                 ", client=" + ctx.clientNode() +
@@ -1266,46 +1305,32 @@ public class GridClusterStateProcessor extends GridProcessorAdapter implements I
 
         IgniteCompute comp = ((ClusterGroupAdapter)ctx.cluster().get().forServers()).compute();
 
-        IgniteFuture<Void> fut =
-            comp.runAsync(new ClientChangeGlobalStateComputeRequest(activate, readOnly, blt, forceBlt));
-
-        fut.listen(new CI1<IgniteFuture>() {
-            @Override public void apply(IgniteFuture fut) {
-                try {
-                    fut.get();
+        IgniteFuture<Void> fut = comp.runAsync(new ClientSetGlobalStateComputeRequest(state, blt, forceBlt));
 
-                    resFut.onDone();
-                }
-                catch (Exception e) {
-                    resFut.onDone(e);
-                }
-            }
-        });
+        return ((IgniteFutureImpl<Void>)fut).internalFuture();
     }
 
     /**
      *  Check cluster state.
      *
-     *  @return Cluster state, {@code True} if cluster active, {@code False} if inactive.
+     *  @return Cluster state.
      */
-    private IgniteFuture<Boolean> sendComputeCheckGlobalState() {
+    private IgniteFuture<ClusterState> sendComputeCheckGlobalState() {
         AffinityTopologyVersion topVer = ctx.discovery().topologyVersionEx();
 
         if (log.isInfoEnabled()) {
             log.info("Sending check cluster state request from node [id=" + ctx.localNodeId() +
                 ", topVer=" + topVer +
                 ", client=" + ctx.clientNode() +
-                ", daemon" + ctx.isDaemon() + "]");
+                ", daemon " + ctx.isDaemon() + "]");
         }
 
         ClusterGroupAdapter clusterGroupAdapter = (ClusterGroupAdapter)ctx.cluster().get().forServers();
 
         if (F.isEmpty(clusterGroupAdapter.nodes()))
-            return new IgniteFinishedFutureImpl<>(false);
-
-        IgniteCompute comp = clusterGroupAdapter.compute();
+            return new IgniteFinishedFutureImpl<>(INACTIVE);
 
-        return comp.callAsync(new CheckGlobalStateComputeRequest());
+        return clusterGroupAdapter.compute().callAsync(new GetClusterStateComputeRequest());
     }
 
     /** {@inheritDoc} */
@@ -1337,11 +1362,7 @@ public class GridClusterStateProcessor extends GridProcessorAdapter implements I
         GridChangeGlobalStateFuture fut = changeStateFuture(req.initiatorNodeId(), req.requestId());
 
         if (fut != null) {
-            IgniteCheckedException e = new IgniteCheckedException(
-                "Failed to " + prettyStr(req.activate(), req.readOnly(), readOnlyChanged(globalState, req.readOnly())),
-                null,
-                false
-            );
+            IgniteCheckedException e = new IgniteCheckedException("Failed to " + prettyStr(req.state()), null, false);
 
             for (Map.Entry<UUID, Exception> entry : errs.entrySet())
                 e.addSuppressed(entry.getValue());
@@ -1396,10 +1417,10 @@ public class GridClusterStateProcessor extends GridProcessorAdapter implements I
     @Override public void onStateChangeExchangeDone(StateChangeRequest req) {
         try {
             if (req.activeChanged()) {
-                if (req.activate())
+                if (ClusterState.active(req.state()))
                     onFinalActivate(req);
 
-                globalState.setTransitionResult(req.requestId(), req.activate());
+                globalState.setTransitionResult(req.requestId(), req.state());
             }
 
             sendChangeGlobalStateResponse(req.requestId(), req.initiatorNodeId(), null);
@@ -1408,7 +1429,7 @@ public class GridClusterStateProcessor extends GridProcessorAdapter implements I
             Exception e = new IgniteCheckedException("Failed to perform final activation steps", ex);
 
             U.error(log, "Failed to perform final activation steps [nodeId=" + ctx.localNodeId() +
-                ", client=" + ctx.clientNode() + ", topVer=" + req.topologyVersion() + "]", ex);
+                ", client=" + ctx.clientNode() + ", topVer=" + req.topologyVersion() + "]. New state: " + req.state(), ex);
 
             sendChangeGlobalStateResponse(req.requestId(), req.initiatorNodeId(), e);
         }
@@ -1500,11 +1521,8 @@ public class GridClusterStateProcessor extends GridProcessorAdapter implements I
     private void onStateRestored(BaselineTopology blt) {
         DiscoveryDataClusterState state = globalState;
 
-        if (!state.active() && !state.transition() && state.baselineTopology() == null) {
-            DiscoveryDataClusterState newState = DiscoveryDataClusterState.createState(false, false, blt);
-
-            globalState = newState;
-        }
+        if (!ClusterState.active(state.state()) && !state.transition() && state.baselineTopology() == null)
+            globalState = DiscoveryDataClusterState.createState(INACTIVE, blt);
     }
 
     /**
@@ -1535,7 +1553,7 @@ public class GridClusterStateProcessor extends GridProcessorAdapter implements I
         );
 
         boolean autoAdjustBaseline = isInMemoryCluster
-            && oldState.active()
+            && ClusterState.active(oldState.state())
             && !oldState.transition()
             && cluster.isBaselineAutoAdjustEnabled()
             && cluster.baselineAutoAdjustTimeout() == 0L;
@@ -1556,8 +1574,7 @@ public class GridClusterStateProcessor extends GridProcessorAdapter implements I
                     nodeId,
                     nodeId,
                     null,
-                    true,
-                    oldState.readOnly(),
+                    oldState.state(),
                     newBlt,
                     true,
                     System.currentTimeMillis()
@@ -1567,7 +1584,7 @@ public class GridClusterStateProcessor extends GridProcessorAdapter implements I
 
                 onStateChangeMessage(ver, changeGlobalStateMsg, discoCache);
 
-                ChangeGlobalStateFinishMessage finishMsg = new ChangeGlobalStateFinishMessage(nodeId, true, true);
+                ChangeGlobalStateFinishMessage finishMsg = new ChangeGlobalStateFinishMessage(nodeId, oldState.state(), true);
 
                 onStateFinishMessage(finishMsg);
 
@@ -1597,8 +1614,7 @@ public class GridClusterStateProcessor extends GridProcessorAdapter implements I
                 UUID.randomUUID(),
                 ctx.localNodeId(),
                 null,
-                true,
-                clusterState.readOnly(),
+                ClusterState.active(clusterState.state()) ? clusterState.state() : ACTIVE,
                 blt,
                 true,
                 System.currentTimeMillis()
@@ -1607,7 +1623,7 @@ public class GridClusterStateProcessor extends GridProcessorAdapter implements I
             StateChangeRequest stateChangeReq = new StateChangeRequest(
                 msg,
                 BaselineTopologyHistoryItem.fromBaseline(blt),
-                false,
+                msg.state(),
                 null
             );
 
@@ -1712,30 +1728,74 @@ public class GridClusterStateProcessor extends GridProcessorAdapter implements I
     }
 
     /**
-     * @param activate Activate.
-     * @return Activate flag string.
+     * @param state Cluster state.
+     * @return Cluster state string representation.
      */
-    private static String prettyStr(boolean activate) {
-        return activate ? "activate" : "deactivate";
+    private static String prettyStr(ClusterState state) {
+        switch (state) {
+            case ACTIVE:
+                return "activate cluster";
+
+            case INACTIVE:
+                return "deactivate cluster";
+
+            case ACTIVE_READ_ONLY:
+                return "activate cluster in read-only mode";
+
+            default:
+                throw new IllegalStateException("Unknown cluster state: " + state);
+        }
     }
 
     /**
-     * @param activate Activate flag.
-     * @param readOnly Read-only flag.
-     * @param readOnlyChanged Read only state changed.
-     * @return Activate or read-only message string.
+     * Checks that activation process is happening now.
+     *
+     * @param state Current cluster state.
+     * @param newState Cluster state after finish transition.
+     * @return {@code True} if activation process is happening now, and {@code False} otherwise.
      */
-    private static String prettyStr(boolean activate, boolean readOnly, boolean readOnlyChanged) {
-        return readOnlyChanged ? prettyStr(readOnly) + " read-only mode" : prettyStr(activate) + " cluster";
+    private boolean activate(ClusterState state, ClusterState newState) {
+        assert state != null;
+        assert newState != null;
+
+        return state == INACTIVE && ClusterState.active(newState);
     }
 
     /**
-     * @param curState Current cluster state.
-     * @param newReadOnly New read-only mode value.
-     * @return {@code True} if read-only mode changed and {@code False} otherwise.
+     * Gets from given config {@code cfg} field with name {@code fieldName} and type boolean.
+     *
+     * @param cfg Config.
+     * @param fieldName Name of field.
+     * @param defaultValue Default value of field, if field is not presented or empty.
+     * @return Value of field, or {@code defaultValue} in case of any errors.
      */
-    private boolean readOnlyChanged(DiscoveryDataClusterState curState, boolean newReadOnly) {
-        return curState.readOnly() != newReadOnly;
+    private boolean getBooleanFieldFromConfig(IgniteConfiguration cfg, String fieldName, boolean defaultValue) {
+        A.notNull(cfg, "cfg");
+        A.notNull(fieldName, "fieldName");
+
+        Field field = U.findField(IgniteConfiguration.class, fieldName);
+
+        try {
+            if (field != null) {
+                field.setAccessible(true);
+
+                boolean val = defaultValue;
+
+                try {
+                    val = field.getBoolean(cfg);
+                }
+                catch (IllegalAccessException | IllegalArgumentException | NullPointerException  e) {
+                    log.error("Can't get value of field with name " + fieldName + " from config: " + cfg, e);
+                }
+
+                return val;
+            }
+        }
+        catch (SecurityException e) {
+            log.error("Can't get field with name " + fieldName + " from config: " + cfg + " due to security reasons", e);
+        }
+
+        return defaultValue;
     }
 
     /** {@inheritDoc} */
@@ -1751,11 +1811,9 @@ public class GridClusterStateProcessor extends GridProcessorAdapter implements I
         @GridToStringInclude
         private final UUID requestId;
 
-        /** Activate. */
-        private final boolean activate;
-
-        /** Read only. */
-        private final boolean readOnly;
+        /** Cluster state. */
+        @GridToStringInclude
+        private final ClusterState state;
 
         /** Nodes. */
         @GridToStringInclude
@@ -1782,14 +1840,13 @@ public class GridClusterStateProcessor extends GridProcessorAdapter implements I
         private final IgniteLogger log;
 
         /**
-         * @param requestId State change request ID.
-         * @param activate New cluster state.
+         * @param reqId State change request ID.
+         * @param state New cluster state.
          * @param ctx Context.
          */
-        GridChangeGlobalStateFuture(UUID requestId, boolean activate, boolean readOnly, GridKernalContext ctx) {
-            this.requestId = requestId;
-            this.activate = activate;
-            this.readOnly = readOnly;
+        GridChangeGlobalStateFuture(UUID reqId, ClusterState state, GridKernalContext ctx) {
+            this.requestId = reqId;
+            this.state = state;
             this.ctx = ctx;
 
             log = ctx.log(getClass());
@@ -1857,9 +1914,7 @@ public class GridClusterStateProcessor extends GridProcessorAdapter implements I
                 onAllReceived();
         }
 
-        /**
-         *
-         */
+        /** */
         private void onAllReceived() {
             IgniteCheckedException e = new IgniteCheckedException();
 
@@ -1902,15 +1957,12 @@ public class GridClusterStateProcessor extends GridProcessorAdapter implements I
      *
      */
     @GridInternal
-    private static class ClientChangeGlobalStateComputeRequest implements IgniteRunnable {
+    private static class ClientSetGlobalStateComputeRequest implements IgniteRunnable {
         /** */
         private static final long serialVersionUID = 0L;
 
         /** */
-        private final boolean activate;
-
-        /** */
-        private final boolean readOnly;
+        private final ClusterState state;
 
         /** */
         private final BaselineTopology baselineTopology;
@@ -1923,16 +1975,16 @@ public class GridClusterStateProcessor extends GridProcessorAdapter implements I
         private IgniteEx ig;
 
         /**
-         * @param activate New cluster state.
+         * @param state New cluster state.
+         * @param blt New baseline topology.
+         * @param forceBlt Force change cluster state.
          */
-        private ClientChangeGlobalStateComputeRequest(
-            boolean activate,
-            boolean readOnly,
+        private ClientSetGlobalStateComputeRequest(
+            ClusterState state,
             BaselineTopology blt,
             boolean forceBlt
         ) {
-            this.activate = activate;
-            this.readOnly = readOnly;
+            this.state = state;
             this.baselineTopology = blt;
             this.forceChangeBaselineTopology = forceBlt;
         }
@@ -1941,8 +1993,7 @@ public class GridClusterStateProcessor extends GridProcessorAdapter implements I
         @Override public void run() {
             try {
                 ig.context().state().changeGlobalState(
-                    activate,
-                    readOnly,
+                    state,
                     baselineTopology != null ? baselineTopology.currentBaseline() : null,
                     forceChangeBaselineTopology
                 ).get();
@@ -1957,7 +2008,7 @@ public class GridClusterStateProcessor extends GridProcessorAdapter implements I
      *
      */
     @GridInternal
-    private static class CheckGlobalStateComputeRequest implements IgniteCallable<Boolean> {
+    private static class GetClusterStateComputeRequest implements IgniteCallable<ClusterState> {
         /** */
         private static final long serialVersionUID = 0L;
 
@@ -1965,8 +2016,9 @@ public class GridClusterStateProcessor extends GridProcessorAdapter implements I
         @IgniteInstanceResource
         private Ignite ig;
 
-        @Override public Boolean call() throws Exception {
-            return ig.active();
+        /** {@inheritDoc} */
+        @Override public ClusterState call() throws Exception {
+            return ig.cluster().state();
         }
     }
 
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cluster/IGridClusterStateProcessor.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cluster/IGridClusterStateProcessor.java
index 37aff52..f4e6850 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cluster/IGridClusterStateProcessor.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cluster/IGridClusterStateProcessor.java
@@ -24,6 +24,7 @@ import java.util.UUID;
 import org.apache.ignite.IgniteCheckedException;
 import org.apache.ignite.cluster.BaselineNode;
 import org.apache.ignite.cluster.ClusterNode;
+import org.apache.ignite.cluster.ClusterState;
 import org.apache.ignite.internal.IgniteInternalFuture;
 import org.apache.ignite.internal.managers.discovery.DiscoCache;
 import org.apache.ignite.internal.processors.GridProcessor;
@@ -38,23 +39,34 @@ import org.jetbrains.annotations.Nullable;
 public interface IGridClusterStateProcessor extends GridProcessor {
     /**
      * @return Cluster state to be used on public API.
+     * @deprecated Use {@link #publicApiState(boolean)} instead.
      */
+    @Deprecated
     boolean publicApiActiveState(boolean waitForTransition);
 
     /**
      * @return Cluster state to be used on public API.
+     * @deprecated Use {@link #publicApiStateAsync(boolean)} instead.
      */
+    @Deprecated
     IgniteFuture<Boolean> publicApiActiveStateAsync(boolean waitForTransition);
 
     /**
-     * @return Grid read only mode to be used on public API.
+     * @param waitForTransition Wait end of transition or not.
+     * @return Current cluster state to be used on public API.
      */
-    boolean publicApiReadOnlyMode();
+    ClusterState publicApiState(boolean waitForTransition);
 
     /**
-     * @return Time change of read only mode to be used on public API.
+     * @param waitForTransition Wait end of transition or not.
+     * @return Current cluster state to be used on public API.
      */
-    long readOnlyModeStateChangeTime();
+    IgniteFuture<ClusterState> publicApiStateAsync(boolean waitForTransition);
+
+    /**
+     * @return Time of last cluster state change to be used on public API.
+     */
+    long lastStateChangeTime();
 
     /**
      * @param discoCache Discovery data cache.
@@ -93,9 +105,7 @@ public interface IGridClusterStateProcessor extends GridProcessor {
      */
     DiscoveryDataClusterState pendingState(ChangeGlobalStateMessage stateMsg);
 
-    /**
-     *
-     */
+    /** */
     void cacheProcessorStarted();
 
     /**
@@ -103,7 +113,9 @@ public interface IGridClusterStateProcessor extends GridProcessor {
      * @param baselineNodes New baseline nodes.
      * @param forceChangeBaselineTopology Force change baseline topology.
      * @return State change future.
+     * @deprecated Use {@link #changeGlobalState(ClusterState, Collection, boolean)} instead.
      */
+    @Deprecated
     IgniteInternalFuture<?> changeGlobalState(
         boolean activate,
         Collection<? extends BaselineNode> baselineNodes,
@@ -111,26 +123,18 @@ public interface IGridClusterStateProcessor extends GridProcessor {
     );
 
     /**
-     * @param activate New cluster state.
-     * @param readOnly Enable read-only mode.
+     * @param state New cluster state.
      * @param baselineNodes New baseline nodes.
      * @param forceChangeBaselineTopology Force change baseline topology.
      * @return State change future.
      */
     IgniteInternalFuture<?> changeGlobalState(
-        boolean activate,
-        boolean readOnly,
+        ClusterState state,
         Collection<? extends BaselineNode> baselineNodes,
         boolean forceChangeBaselineTopology
     );
 
     /**
-     * @param readOnly Enable/disable read-only mode.
-     * @return State change future.
-     */
-    IgniteInternalFuture<?> changeGlobalState(boolean readOnly);
-
-    /**
      * @param errs Errors.
      * @param req State change request.
      */
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/datastreamer/DataStreamerImpl.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/datastreamer/DataStreamerImpl.java
index cdf15f7..318c684 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/datastreamer/DataStreamerImpl.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/datastreamer/DataStreamerImpl.java
@@ -65,7 +65,6 @@ import org.apache.ignite.internal.IgniteFutureTimeoutCheckedException;
 import org.apache.ignite.internal.IgniteInternalFuture;
 import org.apache.ignite.internal.IgniteInterruptedCheckedException;
 import org.apache.ignite.internal.IgniteNodeAttributes;
-import org.apache.ignite.internal.cluster.ClusterReadOnlyModeCheckedException;
 import org.apache.ignite.internal.cluster.ClusterTopologyCheckedException;
 import org.apache.ignite.internal.cluster.ClusterTopologyServerNotFoundException;
 import org.apache.ignite.internal.managers.communication.GridIoMessage;
@@ -89,6 +88,7 @@ import org.apache.ignite.internal.processors.cache.IgniteCacheFutureImpl;
 import org.apache.ignite.internal.processors.cache.IgniteCacheProxy;
 import org.apache.ignite.internal.processors.cache.KeyCacheObject;
 import org.apache.ignite.internal.processors.cache.distributed.dht.GridDhtTopologyFuture;
+import org.apache.ignite.internal.processors.cache.distributed.dht.IgniteClusterReadOnlyException;
 import org.apache.ignite.internal.processors.cache.distributed.dht.preloader.GridDhtPartitionsExchangeFuture;
 import org.apache.ignite.internal.processors.cache.distributed.dht.topology.GridDhtInvalidPartitionException;
 import org.apache.ignite.internal.processors.cache.distributed.dht.topology.GridDhtLocalPartition;
@@ -978,8 +978,8 @@ public class DataStreamerImpl<K, V> implements IgniteDataStreamer<K, V>, Delayed
                                     resFut.onDone(new IgniteCheckedException("Failed to finish operation (too many remaps): "
                                         + remaps, e1));
                                 }
-                                else if (X.hasCause(e1, ClusterReadOnlyModeCheckedException.class)) {
-                                    resFut.onDone(new ClusterReadOnlyModeCheckedException(
+                                else if (X.hasCause(e1, IgniteClusterReadOnlyException.class)) {
+                                    resFut.onDone(new IgniteClusterReadOnlyException(
                                         "Failed to finish operation. Cluster in read-only mode!",
                                         e1
                                     ));
@@ -1220,7 +1220,7 @@ public class DataStreamerImpl<K, V> implements IgniteDataStreamer<K, V>, Delayed
 
                         err = true;
 
-                        if (X.cause(e, ClusterReadOnlyModeCheckedException.class) != null)
+                        if (X.cause(e, IgniteClusterReadOnlyException.class) != null)
                             throw e;
                     }
                 }
@@ -2086,8 +2086,8 @@ public class DataStreamerImpl<K, V> implements IgniteDataStreamer<K, V>, Delayed
 
                     if (cause instanceof ClusterTopologyCheckedException)
                         err = new ClusterTopologyCheckedException(msg, cause);
-                    else if (X.hasCause(cause, ClusterReadOnlyModeCheckedException.class))
-                        err = new ClusterReadOnlyModeCheckedException(msg, cause);
+                    else if (X.hasCause(cause, IgniteClusterReadOnlyException.class))
+                        err = new IgniteClusterReadOnlyException(msg, cause);
                     else
                         err = new IgniteCheckedException(msg, cause);
                 }
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/SqlStateCode.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/SqlStateCode.java
index 0648fb5..2257f95 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/SqlStateCode.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/SqlStateCode.java
@@ -17,7 +17,7 @@
 
 package org.apache.ignite.internal.processors.odbc;
 
-import org.apache.ignite.IgniteCluster;
+import org.apache.ignite.cluster.ClusterState;
 
 /**
  * SQL state codes.
@@ -76,7 +76,7 @@ public final class SqlStateCode {
     public static final String INTERNAL_ERROR = "50000";  // Generic value for custom "50" class.
 
     /**
-     * Read only mode enabled on cluster. {@link IgniteCluster#readOnly()}.
+     * Read only mode enabled on cluster. {@link ClusterState#ACTIVE_READ_ONLY}.
      * Value is equal to {@code org.h2.api.ErrorCode#DATABASE_IS_READ_ONLY} code.
      */
     public static final String CLUSTER_READ_ONLY_MODE_ENABLED = "90097";
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/rest/GridRestCommand.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/rest/GridRestCommand.java
index 102fefa..a927380 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/rest/GridRestCommand.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/rest/GridRestCommand.java
@@ -19,6 +19,7 @@ package org.apache.ignite.internal.processors.rest;
 
 import java.util.HashMap;
 import java.util.Map;
+import org.apache.ignite.cluster.ClusterState;
 import org.jetbrains.annotations.Nullable;
 
 /**
@@ -167,26 +168,26 @@ public enum GridRestCommand {
     @Deprecated
     CLUSTER_INACTIVE("inactive"),
 
-    /** */
+    /** @deprecated Use {@link #CLUSTER_SET_STATE} with {@link ClusterState#ACTIVE} instead. */
+    @Deprecated
     CLUSTER_ACTIVATE("activate"),
 
-    /** */
+    /** @deprecated Use {@link #CLUSTER_SET_STATE} with {@link ClusterState#INACTIVE} instead. */
+    @Deprecated
     CLUSTER_DEACTIVATE("deactivate"),
 
-    /** */
+    /** @deprecated Use {@link #CLUSTER_STATE} instead. */
+    @Deprecated
     CLUSTER_CURRENT_STATE("currentstate"),
 
-    /** Current cluster name. */
-    CLUSTER_NAME("clustername"),
-
     /** */
-    CLUSTER_CURRENT_READ_ONLY_MODE("currentreadonlymode"),
+    CLUSTER_NAME("clustername"),
 
     /** */
-    CLUSTER_READ_ONLY_ENABLE("readonlyenable"),
+    CLUSTER_STATE("state"),
 
     /** */
-    CLUSTER_READ_ONLY_DISABLE("readonlydisable"),
+    CLUSTER_SET_STATE("setstate"),
 
     /** */
     BASELINE_CURRENT_STATE("baseline"),
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/rest/GridRestProcessor.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/rest/GridRestProcessor.java
index e91b4de..111cd98 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/rest/GridRestProcessor.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/rest/GridRestProcessor.java
@@ -50,7 +50,7 @@ import org.apache.ignite.internal.processors.rest.handlers.GridRestCommandHandle
 import org.apache.ignite.internal.processors.rest.handlers.auth.AuthenticationCommandHandler;
 import org.apache.ignite.internal.processors.rest.handlers.cache.GridCacheCommandHandler;
 import org.apache.ignite.internal.processors.rest.handlers.cluster.GridBaselineCommandHandler;
-import org.apache.ignite.internal.processors.rest.handlers.cluster.GridChangeReadOnlyModeCommandHandler;
+import org.apache.ignite.internal.processors.rest.handlers.cluster.GridChangeClusterStateCommandHandler;
 import org.apache.ignite.internal.processors.rest.handlers.cluster.GridChangeStateCommandHandler;
 import org.apache.ignite.internal.processors.rest.handlers.cluster.GridClusterNameCommandHandler;
 import org.apache.ignite.internal.processors.rest.handlers.memory.MemoryMetricsCommandHandler;
@@ -544,7 +544,7 @@ public class GridRestProcessor extends GridProcessorAdapter implements IgniteRes
             addHandler(new QueryCommandHandler(ctx));
             addHandler(new GridLogCommandHandler(ctx));
             addHandler(new GridChangeStateCommandHandler(ctx));
-            addHandler(new GridChangeReadOnlyModeCommandHandler(ctx));
+            addHandler(new GridChangeClusterStateCommandHandler(ctx));
             addHandler(new GridClusterNameCommandHandler(ctx));
             addHandler(new AuthenticationCommandHandler(ctx));
             addHandler(new UserActionCommandHandler(ctx));
@@ -916,6 +916,7 @@ public class GridRestProcessor extends GridProcessorAdapter implements IgniteRes
             case BASELINE_SET:
             case BASELINE_ADD:
             case BASELINE_REMOVE:
+            case CLUSTER_SET_STATE:
                 perm = SecurityPermission.ADMIN_OPS;
 
                 break;
@@ -937,6 +938,7 @@ public class GridRestProcessor extends GridProcessorAdapter implements IgniteRes
             case CLUSTER_CURRENT_STATE:
             case CLUSTER_NAME:
             case BASELINE_CURRENT_STATE:
+            case CLUSTER_STATE:
             case AUTHENTICATE:
             case ADD_USER:
             case REMOVE_USER:
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/rest/client/message/GridClientReadOnlyModeRequest.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/rest/client/message/GridClientClusterStateRequest.java
similarity index 66%
rename from modules/core/src/main/java/org/apache/ignite/internal/processors/rest/client/message/GridClientReadOnlyModeRequest.java
rename to modules/core/src/main/java/org/apache/ignite/internal/processors/rest/client/message/GridClientClusterStateRequest.java
index 093be35..319a436 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/rest/client/message/GridClientReadOnlyModeRequest.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/rest/client/message/GridClientClusterStateRequest.java
@@ -20,19 +20,21 @@ package org.apache.ignite.internal.processors.rest.client.message;
 import java.io.IOException;
 import java.io.ObjectInput;
 import java.io.ObjectOutput;
+import org.apache.ignite.cluster.ClusterState;
+import org.apache.ignite.internal.util.typedef.internal.U;
 
 /**
  *
  */
-public class  GridClientReadOnlyModeRequest extends GridClientAbstractMessage {
+public class GridClientClusterStateRequest extends GridClientAbstractMessage {
     /** */
     private static final long serialVersionUID = 0L;
 
     /** Request current state. */
     private boolean reqCurrentState;
 
-    /** Read only. */
-    private boolean readOnly;
+    /** New cluster state. */
+    private ClusterState state;
 
     /** */
     public boolean isReqCurrentState() {
@@ -40,15 +42,15 @@ public class  GridClientReadOnlyModeRequest extends GridClientAbstractMessage {
     }
 
     /** */
-    public boolean readOnly() {
-        return readOnly;
+    public ClusterState state() {
+        return state;
     }
 
     /**
      * @return Current read-only mode request.
      */
-    public static GridClientReadOnlyModeRequest currentReadOnlyMode() {
-        GridClientReadOnlyModeRequest msg = new GridClientReadOnlyModeRequest();
+    public static GridClientClusterStateRequest currentState() {
+        GridClientClusterStateRequest msg = new GridClientClusterStateRequest();
 
         msg.reqCurrentState = true;
 
@@ -56,23 +58,13 @@ public class  GridClientReadOnlyModeRequest extends GridClientAbstractMessage {
     }
 
     /**
-     * @return Enable read-only mode request.
+     * @param state New cluster state.
+     * @return Cluster state change request.
      */
-    public static GridClientReadOnlyModeRequest enableReadOnly() {
-        GridClientReadOnlyModeRequest msg = new GridClientReadOnlyModeRequest();
+    public static GridClientClusterStateRequest state(ClusterState state) {
+        GridClientClusterStateRequest msg = new GridClientClusterStateRequest();
 
-        msg.readOnly = true;
-
-        return msg;
-    }
-
-    /**
-     * @return Disable read-only mode request.
-     */
-    public static GridClientReadOnlyModeRequest disableReadOnly() {
-        GridClientReadOnlyModeRequest msg = new GridClientReadOnlyModeRequest();
-
-        msg.readOnly = false;
+        msg.state = state;
 
         return msg;
     }
@@ -82,7 +74,7 @@ public class  GridClientReadOnlyModeRequest extends GridClientAbstractMessage {
         super.writeExternal(out);
 
         out.writeBoolean(reqCurrentState);
-        out.writeBoolean(readOnly);
+        U.writeEnum(out, state);
     }
 
     /** {@inheritDoc} */
@@ -90,6 +82,6 @@ public class  GridClientReadOnlyModeRequest extends GridClientAbstractMessage {
         super.readExternal(in);
 
         reqCurrentState = in.readBoolean();
-        readOnly = in.readBoolean();
+        state = ClusterState.fromOrdinal(in.readByte());
     }
 }
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/rest/client/message/GridClientStateRequest.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/rest/client/message/GridClientStateRequest.java
index 43c7df2..1874c88 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/rest/client/message/GridClientStateRequest.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/rest/client/message/GridClientStateRequest.java
@@ -22,8 +22,9 @@ import java.io.ObjectInput;
 import java.io.ObjectOutput;
 
 /**
- *
+ * @deprecated Use {@link GridClientClusterStateRequest} instead.
  */
+@Deprecated
 public class GridClientStateRequest extends GridClientAbstractMessage {
     /** */
     private static final long serialVersionUID = 0L;
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/rest/handlers/cluster/GridChangeReadOnlyModeCommandHandler.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/rest/handlers/cluster/GridChangeClusterStateCommandHandler.java
similarity index 70%
rename from modules/core/src/main/java/org/apache/ignite/internal/processors/rest/handlers/cluster/GridChangeReadOnlyModeCommandHandler.java
rename to modules/core/src/main/java/org/apache/ignite/internal/processors/rest/handlers/cluster/GridChangeClusterStateCommandHandler.java
index acc893e..7de6d14 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/rest/handlers/cluster/GridChangeReadOnlyModeCommandHandler.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/rest/handlers/cluster/GridChangeClusterStateCommandHandler.java
@@ -23,27 +23,25 @@ import org.apache.ignite.internal.IgniteInternalFuture;
 import org.apache.ignite.internal.processors.rest.GridRestCommand;
 import org.apache.ignite.internal.processors.rest.GridRestResponse;
 import org.apache.ignite.internal.processors.rest.handlers.GridRestCommandHandlerAdapter;
-import org.apache.ignite.internal.processors.rest.request.GridRestReadOnlyChangeModeRequest;
+import org.apache.ignite.internal.processors.rest.request.GridRestClusterStateRequest;
 import org.apache.ignite.internal.processors.rest.request.GridRestRequest;
 import org.apache.ignite.internal.util.future.GridFutureAdapter;
 import org.apache.ignite.internal.util.typedef.internal.U;
 
-import static org.apache.ignite.internal.processors.rest.GridRestCommand.CLUSTER_CURRENT_READ_ONLY_MODE;
-import static org.apache.ignite.internal.processors.rest.GridRestCommand.CLUSTER_READ_ONLY_DISABLE;
-import static org.apache.ignite.internal.processors.rest.GridRestCommand.CLUSTER_READ_ONLY_ENABLE;
+import static org.apache.ignite.internal.processors.rest.GridRestCommand.CLUSTER_SET_STATE;
+import static org.apache.ignite.internal.processors.rest.GridRestCommand.CLUSTER_STATE;
 
 /**
  *
  */
-public class GridChangeReadOnlyModeCommandHandler extends GridRestCommandHandlerAdapter {
+public class GridChangeClusterStateCommandHandler extends GridRestCommandHandlerAdapter {
     /** Commands. */
-    private static final Collection<GridRestCommand> COMMANDS =
-        U.sealList(CLUSTER_CURRENT_READ_ONLY_MODE, CLUSTER_READ_ONLY_DISABLE, CLUSTER_READ_ONLY_ENABLE);
+    private static final Collection<GridRestCommand> COMMANDS = U.sealList(CLUSTER_SET_STATE, CLUSTER_STATE);
 
     /**
      * @param ctx Context.
      */
-    public GridChangeReadOnlyModeCommandHandler(GridKernalContext ctx) {
+    public GridChangeClusterStateCommandHandler(GridKernalContext ctx) {
         super(ctx);
     }
 
@@ -54,7 +52,7 @@ public class GridChangeReadOnlyModeCommandHandler extends GridRestCommandHandler
 
     /** {@inheritDoc} */
     @Override public IgniteInternalFuture<GridRestResponse> handleAsync(GridRestRequest restReq) {
-        GridRestReadOnlyChangeModeRequest req = (GridRestReadOnlyChangeModeRequest)restReq;
+        GridRestClusterStateRequest req = (GridRestClusterStateRequest)restReq;
 
         final GridFutureAdapter<GridRestResponse> fut = new GridFutureAdapter<>();
 
@@ -62,18 +60,20 @@ public class GridChangeReadOnlyModeCommandHandler extends GridRestCommandHandler
 
         try {
             switch (req.command()) {
-                case CLUSTER_CURRENT_READ_ONLY_MODE:
-                    res.setResponse(ctx.grid().cluster().readOnly());
+                case CLUSTER_STATE:
+                    assert req.isReqCurrentMode() : req;
+
+                    res.setResponse(ctx.grid().cluster().state());
 
                     break;
 
                 default:
-                    if (req.readOnly())
-                        U.log(log, "Received enable read-only mode request from client node with ID: " + req.clientId());
-                    else
-                        U.log(log, "Received disable read-only mode request from client node with ID: " + req.clientId());
+                    assert req.state() != null : req;
+
+                    U.log(log, "Received cluster state change request to " + req.state() +
+                        " state from client node with ID: " + req.clientId());
 
-                    ctx.grid().cluster().readOnly(req.readOnly());
+                    ctx.grid().cluster().state(req.state());
 
                     res.setResponse(req.command().key() + " done");
 
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/rest/protocols/tcp/GridTcpRestNioListener.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/rest/protocols/tcp/GridTcpRestNioListener.java
index 39caecb..e07c4d1 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/rest/protocols/tcp/GridTcpRestNioListener.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/rest/protocols/tcp/GridTcpRestNioListener.java
@@ -36,11 +36,11 @@ import org.apache.ignite.internal.processors.rest.GridRestResponse;
 import org.apache.ignite.internal.processors.rest.client.message.GridClientAuthenticationRequest;
 import org.apache.ignite.internal.processors.rest.client.message.GridClientCacheRequest;
 import org.apache.ignite.internal.processors.rest.client.message.GridClientClusterNameRequest;
+import org.apache.ignite.internal.processors.rest.client.message.GridClientClusterStateRequest;
 import org.apache.ignite.internal.processors.rest.client.message.GridClientHandshakeRequest;
 import org.apache.ignite.internal.processors.rest.client.message.GridClientHandshakeResponse;
 import org.apache.ignite.internal.processors.rest.client.message.GridClientMessage;
 import org.apache.ignite.internal.processors.rest.client.message.GridClientPingPacket;
-import org.apache.ignite.internal.processors.rest.client.message.GridClientReadOnlyModeRequest;
 import org.apache.ignite.internal.processors.rest.client.message.GridClientResponse;
 import org.apache.ignite.internal.processors.rest.client.message.GridClientStateRequest;
 import org.apache.ignite.internal.processors.rest.client.message.GridClientTaskRequest;
@@ -51,7 +51,7 @@ import org.apache.ignite.internal.processors.rest.protocols.tcp.redis.GridRedisN
 import org.apache.ignite.internal.processors.rest.request.GridRestCacheRequest;
 import org.apache.ignite.internal.processors.rest.request.GridRestChangeStateRequest;
 import org.apache.ignite.internal.processors.rest.request.GridRestClusterNameRequest;
-import org.apache.ignite.internal.processors.rest.request.GridRestReadOnlyChangeModeRequest;
+import org.apache.ignite.internal.processors.rest.request.GridRestClusterStateRequest;
 import org.apache.ignite.internal.processors.rest.request.GridRestRequest;
 import org.apache.ignite.internal.processors.rest.request.GridRestTaskRequest;
 import org.apache.ignite.internal.processors.rest.request.GridRestTopologyRequest;
@@ -73,9 +73,8 @@ import static org.apache.ignite.internal.processors.rest.GridRestCommand.CACHE_P
 import static org.apache.ignite.internal.processors.rest.GridRestCommand.CACHE_REMOVE;
 import static org.apache.ignite.internal.processors.rest.GridRestCommand.CACHE_REMOVE_ALL;
 import static org.apache.ignite.internal.processors.rest.GridRestCommand.CACHE_REPLACE;
-import static org.apache.ignite.internal.processors.rest.GridRestCommand.CLUSTER_CURRENT_READ_ONLY_MODE;
-import static org.apache.ignite.internal.processors.rest.GridRestCommand.CLUSTER_READ_ONLY_DISABLE;
-import static org.apache.ignite.internal.processors.rest.GridRestCommand.CLUSTER_READ_ONLY_ENABLE;
+import static org.apache.ignite.internal.processors.rest.GridRestCommand.CLUSTER_SET_STATE;
+import static org.apache.ignite.internal.processors.rest.GridRestCommand.CLUSTER_STATE;
 import static org.apache.ignite.internal.processors.rest.GridRestCommand.EXE;
 import static org.apache.ignite.internal.processors.rest.GridRestCommand.NODE;
 import static org.apache.ignite.internal.processors.rest.GridRestCommand.NOOP;
@@ -385,18 +384,18 @@ public class GridTcpRestNioListener extends GridNioServerListenerAdapter<GridCli
 
             restReq = restChangeReq;
         }
-        else if (msg instanceof GridClientReadOnlyModeRequest) {
-            GridClientReadOnlyModeRequest req = (GridClientReadOnlyModeRequest)msg;
+        else if (msg instanceof GridClientClusterStateRequest) {
+            GridClientClusterStateRequest req = (GridClientClusterStateRequest)msg;
 
-            GridRestReadOnlyChangeModeRequest restChangeReq = new GridRestReadOnlyChangeModeRequest();
+            GridRestClusterStateRequest restChangeReq = new GridRestClusterStateRequest();
 
             if (req.isReqCurrentState()) {
                 restChangeReq.reqCurrentMode();
-                restChangeReq.command(CLUSTER_CURRENT_READ_ONLY_MODE);
+                restChangeReq.command(CLUSTER_STATE);
             }
             else {
-                restChangeReq.readOnly(req.readOnly());
-                restChangeReq.command(req.readOnly() ? CLUSTER_READ_ONLY_ENABLE : CLUSTER_READ_ONLY_DISABLE);
+                restChangeReq.state(req.state());
+                restChangeReq.command(CLUSTER_SET_STATE);
             }
 
             restReq = restChangeReq;
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/rest/request/GridRestReadOnlyChangeModeRequest.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/rest/request/GridRestClusterStateRequest.java
similarity index 67%
rename from modules/core/src/main/java/org/apache/ignite/internal/processors/rest/request/GridRestReadOnlyChangeModeRequest.java
rename to modules/core/src/main/java/org/apache/ignite/internal/processors/rest/request/GridRestClusterStateRequest.java
index 6d1e7f4..82de9a3 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/rest/request/GridRestReadOnlyChangeModeRequest.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/rest/request/GridRestClusterStateRequest.java
@@ -17,15 +17,17 @@
 
 package org.apache.ignite.internal.processors.rest.request;
 
+import org.apache.ignite.cluster.ClusterState;
+
 /**
  *
  */
-public class GridRestReadOnlyChangeModeRequest extends GridRestRequest {
+public class GridRestClusterStateRequest extends GridRestRequest {
     /** Request current state. */
     private boolean reqCurrentMode;
 
-    /** Read only. */
-    private boolean readOnly;
+    /** New state. */
+    private ClusterState state;
 
     /** */
     public void reqCurrentMode() {
@@ -38,12 +40,20 @@ public class GridRestReadOnlyChangeModeRequest extends GridRestRequest {
     }
 
     /** */
-    public void readOnly(boolean readOnly) {
-        this.readOnly = readOnly;
+    public ClusterState state() {
+        return state;
     }
 
-    /** */
-    public boolean readOnly() {
-        return readOnly;
+    /**
+     * Sets new cluster state to request.
+     *
+     * @param state New cluster state.
+     * @throws NullPointerException If {@code state} is null.
+     */
+    public void state(ClusterState state) {
+        if (state == null)
+            throw new NullPointerException("State can't be null.");
+
+        this.state = state;
     }
 }
diff --git a/modules/core/src/main/java/org/apache/ignite/mxbean/IgniteMXBean.java b/modules/core/src/main/java/org/apache/ignite/mxbean/IgniteMXBean.java
index 8bf9e4b..686c333 100644
--- a/modules/core/src/main/java/org/apache/ignite/mxbean/IgniteMXBean.java
+++ b/modules/core/src/main/java/org/apache/ignite/mxbean/IgniteMXBean.java
@@ -21,6 +21,7 @@ import java.util.List;
 import java.util.Map;
 import java.util.UUID;
 import javax.management.JMException;
+import org.apache.ignite.cluster.ClusterState;
 import org.apache.ignite.internal.processors.metric.GridMetricManager;
 
 /**
@@ -136,6 +137,12 @@ public interface IgniteMXBean {
     public String IS_NODE_BASELINE_DESC = "Baseline node flag.";
 
     /** */
+    public static final String LAST_CLUSTER_STATE_CHANGE_TIME_DESC = "Unix time of last cluster state change operation.";
+
+    /** */
+    public static final String CLUSTER_STATE_DESC = "Checks cluster state.";
+
+    /** */
     public String READ_ONLY_MODE_DESC = "Cluster read-only mode status.";
 
     /** */
@@ -688,34 +695,34 @@ public interface IgniteMXBean {
     public void resetMetrics(String registry);
 
     /**
-     * Gets cluster read-only mode status.
+     * Checks cluster state.
      *
-     * @return {@code true} if cluster active and read-only mode enabled, and {@code false} otherwise.
+     * @return String representation of current cluster state.
+     * See {@link ClusterState}.
      * @deprecated Use {@link GridMetricManager} instead.
      */
     @Deprecated
-    @MXBeanDescription(READ_ONLY_MODE_DESC)
-    boolean readOnlyMode();
+    @MXBeanDescription(CLUSTER_STATE_DESC)
+    public String clusterState();
 
     /**
-     * Enable or disable cluster read-only mode. If {@code readOnly} flag is {@code true} read-only mode will be
-     * enabled. If {@code readOnly} flag is {@code false} read-only mode will be disabled.
+     * Changes current cluster state.
      *
-     * @param readOnly enable/disable cluster read-only mode flag.
+     * @param state String representation of new cluster state.
+     * See {@link ClusterState}
      */
-    @MXBeanDescription("Enable or disable cluster read-only mode.")
-    @MXBeanParametersNames("readOnly")
-    @MXBeanParametersDescriptions("True - enable read-only mode, false - disable read-only mode.")
-    void readOnlyMode(boolean readOnly);
+    @MXBeanDescription("Changes current cluster state.")
+    @MXBeanParametersNames("state")
+    @MXBeanParametersDescriptions("New cluster state.")
+    public void clusterState(String state);
 
     /**
-     * Gets duration of read-only mode enabled on cluster.
+     * Gets last cluster state change operation.
      *
-     * @return {@code 0} if cluster read-only mode disabled, and time in milliseconds since enabling cluster read-only
-     * mode.
+     * @return Unix time of last cluster state change operation.
      * @deprecated Use {@link GridMetricManager} instead.
      */
     @Deprecated
-    @MXBeanDescription(READ_ONLY_MODE_DURATION_DESC)
-    long getReadOnlyModeDuration();
+    @MXBeanDescription(LAST_CLUSTER_STATE_CHANGE_TIME_DESC)
+    public long lastClusterStateChangeTime();
 }
diff --git a/modules/core/src/main/resources/META-INF/classnames.properties b/modules/core/src/main/resources/META-INF/classnames.properties
index 98cdc25..ecc8ef0 100644
--- a/modules/core/src/main/resources/META-INF/classnames.properties
+++ b/modules/core/src/main/resources/META-INF/classnames.properties
@@ -1659,7 +1659,7 @@ org.apache.ignite.internal.processors.rest.client.message.GridClientMessage
 org.apache.ignite.internal.processors.rest.client.message.GridClientNodeBean
 org.apache.ignite.internal.processors.rest.client.message.GridClientNodeMetricsBean
 org.apache.ignite.internal.processors.rest.client.message.GridClientPingPacket
-org.apache.ignite.internal.processors.rest.client.message.GridClientReadOnlyModeRequest
+org.apache.ignite.internal.processors.rest.client.message.GridClientClusterStateRequest
 org.apache.ignite.internal.processors.rest.client.message.GridClientResponse
 org.apache.ignite.internal.processors.rest.client.message.GridClientStateRequest
 org.apache.ignite.internal.processors.rest.client.message.GridClientTaskRequest
diff --git a/modules/core/src/test/java/org/apache/ignite/cluster/GridClusterStateChangeSelfTest.java b/modules/core/src/test/java/org/apache/ignite/cluster/GridClusterStateChangeSelfTest.java
new file mode 100644
index 0000000..05c595e
--- /dev/null
+++ b/modules/core/src/test/java/org/apache/ignite/cluster/GridClusterStateChangeSelfTest.java
@@ -0,0 +1,144 @@
+/*
+ * 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.cluster;
+
+import java.util.Collection;
+import org.apache.ignite.configuration.DataRegionConfiguration;
+import org.apache.ignite.configuration.DataStorageConfiguration;
+import org.apache.ignite.configuration.IgniteConfiguration;
+import org.apache.ignite.internal.IgniteEx;
+import org.apache.ignite.internal.util.typedef.internal.U;
+import org.apache.ignite.testframework.junits.common.GridCommonAbstractTest;
+import org.junit.Test;
+
+/**
+ * Checks {@link ClusterState} change.
+ */
+public class GridClusterStateChangeSelfTest extends GridCommonAbstractTest {
+    /** Names of nodes. */
+    private static final Collection<String> NODES_NAMES = U.sealList("server1", "server2", "client1", "client2", "daemon1", "daemon2");
+
+    /** {@inheritDoc} */
+    @Override protected IgniteConfiguration getConfiguration(String igniteInstanceName) throws Exception {
+        return super.getConfiguration(igniteInstanceName)
+            .setClientMode(igniteInstanceName.startsWith("client"))
+            .setDaemon(igniteInstanceName.startsWith("daemon"))
+            .setDataStorageConfiguration(new DataStorageConfiguration()
+                .setDefaultDataRegionConfiguration(new DataRegionConfiguration().setPersistenceEnabled(true))
+            );
+    }
+
+    /** {@inheritDoc} */
+    @Override protected void beforeTestsStarted() throws Exception {
+        super.beforeTestsStarted();
+
+        stopAllGrids();
+
+        cleanPersistenceDir();
+
+        for (String name : NODES_NAMES)
+            startGrid(name);
+    }
+
+    /** {@inheritDoc} */
+    @Override protected void afterTestsStopped() throws Exception {
+        stopAllGrids();
+
+        cleanPersistenceDir();
+
+        super.afterTestsStopped();
+    }
+
+    /** */
+    @Test
+    public void testInactiveActive() {
+        testStateChanged(ClusterState.INACTIVE, ClusterState.ACTIVE);
+    }
+
+    /** */
+    @Test
+    public void testInactiveReadOnly() {
+        testStateChanged(ClusterState.INACTIVE, ClusterState.ACTIVE_READ_ONLY);
+    }
+
+    /** */
+    @Test
+    public void testActiveReadOnly() {
+        testStateChanged(ClusterState.ACTIVE, ClusterState.ACTIVE_READ_ONLY);
+    }
+
+    /** */
+    @Test
+    public void testSetSameStateActive() {
+        testSetSameState(ClusterState.ACTIVE);
+    }
+
+    /** */
+    @Test
+    public void testSetSameStateInactive() {
+        testSetSameState(ClusterState.INACTIVE);
+    }
+
+    /** */
+    @Test
+    public void testSetSameStateReadOnly() {
+        testSetSameState(ClusterState.ACTIVE_READ_ONLY);
+    }
+
+    /** */
+    private void testSetSameState(ClusterState state) {
+        for (String name : NODES_NAMES) {
+            IgniteEx initiator = grid(name);
+
+            if (initiator.cluster().state() != state)
+                initiator.cluster().state(state);
+
+            NODES_NAMES.forEach(n -> assertEquals(n, state, grid(n).cluster().state()));
+
+            // Set the same state.
+            initiator.cluster().state(state);
+
+            NODES_NAMES.forEach(n -> assertEquals(n, state, grid(n).cluster().state()));
+        }
+    }
+
+    /**
+     * Checks that cluster correctly change state from {@code from} state to {@code to} state.
+     *
+     * @param from Initial state.
+     * @param to Target state.
+     */
+    private void testStateChanged(ClusterState from, ClusterState to) {
+        assertTrue(from.toString(), from != to);
+
+        for (String name : NODES_NAMES) {
+            IgniteEx initiator = grid(name);
+
+            // Needs for checking in both directions.
+            initiator.cluster().state(to);
+
+            initiator.cluster().state(from);
+
+            NODES_NAMES.forEach(n -> assertEquals(n, from, grid(n).cluster().state()));
+
+            initiator.cluster().state(to);
+
+            NODES_NAMES.forEach(n -> assertEquals(n, to, grid(n).cluster().state()));
+        }
+    }
+}
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 468672a..58d1687 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
@@ -25,6 +25,7 @@ import java.util.UUID;
 import java.util.function.Predicate;
 import java.util.logging.Level;
 import java.util.logging.Logger;
+import org.apache.ignite.cluster.ClusterState;
 import org.apache.ignite.internal.commandline.baseline.BaselineArguments;
 import org.apache.ignite.internal.commandline.cache.CacheCommands;
 import org.apache.ignite.internal.commandline.cache.CacheSubcommands;
@@ -47,6 +48,7 @@ import org.junit.rules.TestRule;
 import static java.util.Arrays.asList;
 import static org.apache.ignite.IgniteSystemProperties.IGNITE_ENABLE_EXPERIMENTAL_COMMAND;
 import static org.apache.ignite.internal.commandline.CommandList.CACHE;
+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.TaskExecutor.DFLT_HOST;
 import static org.apache.ignite.internal.commandline.TaskExecutor.DFLT_PORT;
@@ -253,7 +255,7 @@ public class CommandHandlerParsingTest {
     @Test
     public void testParseAndValidateSSLArguments() {
         for (CommandList cmd : CommandList.values()) {
-            if (cmd == CommandList.CACHE || cmd == CommandList.WAL)
+            if (skipCommand(cmd))
                 continue; // --cache subcommand requires its own specific arguments.
 
             assertParseArgsThrows("Expected SSL trust store path", "--truststore");
@@ -275,6 +277,13 @@ public class CommandHandlerParsingTest {
         }
     }
 
+    /** */
+    private boolean skipCommand(CommandList cmd) {
+        return cmd == CommandList.CACHE ||
+            cmd == CommandList.WAL ||
+            cmd == CommandList.SET_STATE;
+    }
+
 
     /**
      * Tests parsing and validation for user and password arguments.
@@ -282,8 +291,8 @@ public class CommandHandlerParsingTest {
     @Test
     public void testParseAndValidateUserAndPassword() {
         for (CommandList cmd : CommandList.values()) {
-            if (cmd == CommandList.CACHE || cmd == CommandList.WAL)
-                continue; // --cache subcommand requires its own specific arguments.
+            if (skipCommand(cmd))
+                continue; // --cache, --wal and --set-state commands requires its own specific arguments.
 
             assertParseArgsThrows("Expected user name", "--user");
             assertParseArgsThrows("Expected password", "--password");
@@ -331,29 +340,48 @@ public class CommandHandlerParsingTest {
      */
     @Test
     public void testParseAutoConfirmationFlag() {
-        for (CommandList cmd : CommandList.values()) {
-            if (cmd.command().confirmationPrompt() == null)
+        for (CommandList cmdL : CommandList.values()) {
+            // SET_STATE command have mandatory argument, which used in confirmation message.
+            Command cmd = cmdL != SET_STATE ? cmdL.command() : parseArgs(asList(cmdL.text(), "ACTIVE")).command();
+
+            if (cmd.confirmationPrompt() == null)
                 continue;
 
-            ConnectionAndSslParameters args = parseArgs(asList(cmd.text()));
+            ConnectionAndSslParameters args;
+
+            if (cmdL == SET_STATE)
+                args = parseArgs(asList(cmdL.text(), "ACTIVE"));
+            else
+                args = parseArgs(asList(cmdL.text()));
 
-            checkCommonParametersCorrectlyParsed(cmd, args, false);
+            checkCommonParametersCorrectlyParsed(cmdL, args, false);
 
-            switch (cmd) {
-                case DEACTIVATE:
-                case READ_ONLY_DISABLE:
-                case READ_ONLY_ENABLE: {
-                    args = parseArgs(asList(cmd.text(), "--yes"));
+            switch (cmdL) {
+                case DEACTIVATE: {
+                    args = parseArgs(asList(cmdL.text(), "--yes"));
 
-                    checkCommonParametersCorrectlyParsed(cmd, args, true);
+                    checkCommonParametersCorrectlyParsed(cmdL, args, true);
+
+                    break;
+                }
+                case SET_STATE: {
+                    for (String newState : asList("ACTIVE_READ_ONLY", "ACTIVE", "INACTIVE")) {
+                        args = parseArgs(asList(cmdL.text(), newState, "--yes"));
+
+                        checkCommonParametersCorrectlyParsed(cmdL, args, true);
+
+                        ClusterState argState = ((ClusterStateChangeCommand)args.command()).arg();
+
+                        assertEquals(newState, argState.toString());
+                    }
 
                     break;
                 }
                 case BASELINE: {
                     for (String baselineAct : asList("add", "remove", "set")) {
-                        args = parseArgs(asList(cmd.text(), baselineAct, "c_id1,c_id2", "--yes"));
+                        args = parseArgs(asList(cmdL.text(), baselineAct, "c_id1,c_id2", "--yes"));
 
-                        checkCommonParametersCorrectlyParsed(cmd, args, true);
+                        checkCommonParametersCorrectlyParsed(cmdL, args, true);
 
                         BaselineArguments arg = ((BaselineCommand)args.command()).arg();
 
@@ -365,9 +393,9 @@ public class CommandHandlerParsingTest {
                 }
 
                 case TX: {
-                    args = parseArgs(asList(cmd.text(), "--xid", "xid1", "--min-duration", "10", "--kill", "--yes"));
+                    args = parseArgs(asList(cmdL.text(), "--xid", "xid1", "--min-duration", "10", "--kill", "--yes"));
 
-                    checkCommonParametersCorrectlyParsed(cmd, args, true);
+                    checkCommonParametersCorrectlyParsed(cmdL, args, true);
 
                     VisorTxTaskArg txTaskArg = ((TxCommands)args.command()).arg();
 
@@ -375,6 +403,9 @@ public class CommandHandlerParsingTest {
                     assertEquals(10_000, txTaskArg.getMinDuration().longValue());
                     assertEquals(VisorTxOperation.KILL, txTaskArg.getOperation());
                 }
+
+                default:
+                    fail("Unknown command: " + cmd);
             }
         }
     }
@@ -398,7 +429,7 @@ public class CommandHandlerParsingTest {
     @Test
     public void testConnectionSettings() {
         for (CommandList cmd : CommandList.values()) {
-            if (cmd == CommandList.CACHE || cmd == CommandList.WAL)
+            if (skipCommand(cmd))
                 continue; // --cache subcommand requires its own specific arguments.
 
             ConnectionAndSslParameters args = parseArgs(asList(cmd.text()));
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/encryption/EncryptionMXBeanTest.java b/modules/core/src/test/java/org/apache/ignite/internal/encryption/EncryptionMXBeanTest.java
index bbc2aa3..2ddedd2 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/encryption/EncryptionMXBeanTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/encryption/EncryptionMXBeanTest.java
@@ -28,6 +28,7 @@ import org.apache.ignite.internal.util.typedef.internal.U;
 import org.apache.ignite.mxbean.EncryptionMXBean;
 import org.junit.Test;
 
+import static org.apache.ignite.cluster.ClusterState.ACTIVE_READ_ONLY;
 import static org.apache.ignite.spi.encryption.keystore.KeystoreEncryptionSpi.DEFAULT_MASTER_KEY_NAME;
 import static org.apache.ignite.testframework.GridTestUtils.assertThrowsWithCause;
 
@@ -107,9 +108,7 @@ public class EncryptionMXBeanTest extends AbstractEncryptionTest {
 
         assertEquals(DEFAULT_MASTER_KEY_NAME, grid0.encryption().getMasterKeyName());
 
-        grid0.cluster().active(true);
-
-        grid0.cluster().readOnly(true);
+        grid0.cluster().state(ACTIVE_READ_ONLY);
 
         mBean.changeMasterKey(MASTER_KEY_NAME_2);
 
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/metric/JmxExporterSpiTest.java b/modules/core/src/test/java/org/apache/ignite/internal/metric/JmxExporterSpiTest.java
index 4bfbf02..a768a29 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/metric/JmxExporterSpiTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/metric/JmxExporterSpiTest.java
@@ -58,6 +58,7 @@ import org.apache.ignite.cache.query.ContinuousQuery;
 import org.apache.ignite.cache.query.QueryCursor;
 import org.apache.ignite.cache.query.ScanQuery;
 import org.apache.ignite.client.IgniteClient;
+import org.apache.ignite.cluster.ClusterState;
 import org.apache.ignite.configuration.CacheConfiguration;
 import org.apache.ignite.configuration.ClientConfiguration;
 import org.apache.ignite.configuration.DataRegionConfiguration;
@@ -803,8 +804,6 @@ public class JmxExporterSpiTest extends AbstractExporterSpiTest {
         assertTrue((boolean)mbn.getAttribute("isNodeInBaseline"));
         assertTrue((boolean)mbn.getAttribute("active"));
 
-        assertFalse((boolean)mbn.getAttribute("readOnlyMode"));
-
         assertTrue((long)mbn.getAttribute("startTimestamp") > 0);
         assertTrue((long)mbn.getAttribute("uptime") > 0);
 
@@ -817,7 +816,10 @@ public class JmxExporterSpiTest extends AbstractExporterSpiTest {
 
         assertEquals(0L, mbn.getAttribute("longJVMPausesCount"));
         assertEquals(0L, mbn.getAttribute("longJVMPausesTotalDuration"));
-        assertEquals(0L, mbn.getAttribute("readOnlyModeDuration"));
+
+        long clusterStateChangeTime = (long)mbn.getAttribute("lastClusterStateChangeTime");
+
+        assertTrue(0 < clusterStateChangeTime && clusterStateChangeTime < System.currentTimeMillis());
 
         assertEquals(String.valueOf(ignite.configuration().getPublicThreadPoolSize()),
                 mbn.getAttribute("executorServiceFormatted"));
@@ -836,6 +838,8 @@ public class JmxExporterSpiTest extends AbstractExporterSpiTest {
 
         assertEquals(ignite.configuration().getMBeanServer().toString(),
                 (String)mbn.getAttribute("mBeanServerFormatted"));
+
+        assertEquals(ClusterState.ACTIVE.toString(), mbn.getAttribute("clusterState"));
     }
 
     /** */
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/ActiveOnStartPropertyTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/ActiveOnStartPropertyTest.java
new file mode 100644
index 0000000..2b0f147
--- /dev/null
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/ActiveOnStartPropertyTest.java
@@ -0,0 +1,149 @@
+/*
+ * 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.cache;
+
+import java.util.HashMap;
+import java.util.Map;
+import org.apache.ignite.configuration.CacheConfiguration;
+import org.apache.ignite.configuration.DataRegionConfiguration;
+import org.apache.ignite.configuration.DataStorageConfiguration;
+import org.apache.ignite.configuration.IgniteConfiguration;
+import org.apache.ignite.internal.util.typedef.internal.CU;
+import org.apache.ignite.testframework.ListeningTestLogger;
+import org.apache.ignite.testframework.LogListener;
+import org.apache.ignite.testframework.junits.common.GridCommonAbstractTest;
+import org.junit.Test;
+
+import static org.apache.ignite.cluster.ClusterState.ACTIVE;
+import static org.apache.ignite.cluster.ClusterState.INACTIVE;
+
+/**
+ * Checks that {@link IgniteConfiguration#isActiveOnStart()} works correctly after deprecation and introduction {@link
+ * IgniteConfiguration#getClusterStateOnStart()}.
+ */
+public class ActiveOnStartPropertyTest extends GridCommonAbstractTest {
+    /** Active on start flag. */
+    private boolean activeOnStart;
+
+    /** Persistence enabled flag. */
+    private boolean persistenceEnabled;
+
+    /** */
+    private Map<String, LogListener> logListeners = new HashMap<>();
+
+    /** {@inheritDoc} */
+    @Override protected IgniteConfiguration getConfiguration(String igniteInstanceName) throws Exception {
+        ListeningTestLogger testLog = new ListeningTestLogger(false, log);
+
+        LogListener lsnr = LogListener.matches(
+            "Property activeOnStart deprecated. Use clusterStateOnStart instead."
+        ).build();
+
+        testLog.registerListener(lsnr);
+
+        logListeners.put(igniteInstanceName, lsnr);
+
+        return super.getConfiguration(igniteInstanceName)
+            .setGridLogger(testLog)
+            .setActiveOnStart(activeOnStart)
+            .setCacheConfiguration(new CacheConfiguration().setName(DEFAULT_CACHE_NAME))
+            .setDataStorageConfiguration(
+                new DataStorageConfiguration().setDefaultDataRegionConfiguration(
+                    new DataRegionConfiguration().setPersistenceEnabled(persistenceEnabled)
+                )
+            );
+    }
+
+    /** {@inheritDoc} */
+    @Override protected void afterTest() throws Exception {
+        stopAllGrids();
+
+        cleanPersistenceDir();
+
+        super.afterTest();
+    }
+
+    /** {@inheritDoc} */
+    @Override protected void beforeTest() throws Exception {
+        super.beforeTest();
+
+        stopAllGrids();
+
+        cleanPersistenceDir();
+    }
+
+    /** */
+    @Test
+    public void testInMemoryActive() throws Exception {
+        activeOnStart = true;
+        persistenceEnabled = false;
+
+        checkProperty();
+    }
+
+    /** */
+    @Test
+    public void testInMemoryInactive() throws Exception {
+        activeOnStart = false;
+        persistenceEnabled = false;
+
+        checkProperty();
+    }
+
+    /** */
+    @Test
+    public void testPersisnentActive() throws Exception {
+        activeOnStart = true;
+        persistenceEnabled = true;
+
+        checkProperty();
+    }
+
+    /** */
+    @Test
+    public void testPersisnentInactive() throws Exception {
+        activeOnStart = false;
+        persistenceEnabled = true;
+
+        checkProperty();
+    }
+
+    /** */
+    private void checkProperty() throws Exception {
+        final int nodeCnt = 2;
+
+        startGrids(nodeCnt);
+
+        for (int i=0; i < nodeCnt; i++) {
+            assertEquals(activeOnStart, grid(i).configuration().isActiveOnStart());
+
+            assertNull(grid(i).configuration().getClusterStateOnStart());
+
+            assertEquals(persistenceEnabled, CU.isPersistenceEnabled(grid(i).configuration()));
+
+            // Active on start must be ignored if persistence enabled.
+            if (persistenceEnabled)
+                assertEquals(INACTIVE, grid(i).cluster().state());
+            else
+                assertEquals(activeOnStart ? ACTIVE : INACTIVE, grid(i).cluster().state());
+        }
+
+        for (String name : logListeners.keySet())
+            assertTrue(name, logListeners.get(name).check());
+    }
+}
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/AutoActivationPropertyTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/AutoActivationPropertyTest.java
new file mode 100644
index 0000000..1838d3a
--- /dev/null
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/AutoActivationPropertyTest.java
@@ -0,0 +1,128 @@
+/*
+ * 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.cache;
+
+import java.util.HashMap;
+import java.util.Map;
+import org.apache.ignite.configuration.CacheConfiguration;
+import org.apache.ignite.configuration.DataRegionConfiguration;
+import org.apache.ignite.configuration.DataStorageConfiguration;
+import org.apache.ignite.configuration.IgniteConfiguration;
+import org.apache.ignite.internal.IgniteEx;
+import org.apache.ignite.testframework.ListeningTestLogger;
+import org.apache.ignite.testframework.LogListener;
+import org.apache.ignite.testframework.junits.common.GridCommonAbstractTest;
+import org.junit.Test;
+
+import static org.apache.ignite.cluster.ClusterState.ACTIVE;
+import static org.apache.ignite.cluster.ClusterState.INACTIVE;
+
+/**
+ * Checks that {@link IgniteConfiguration#autoActivation} works correctly after deprecation and introduction {@link
+ * IgniteConfiguration#clusterStateOnStart}.
+ */
+public class AutoActivationPropertyTest extends GridCommonAbstractTest {
+    /** Auto activation flag. */
+    private boolean autoActivation;
+
+    /** */
+    private Map<String, LogListener> logListeners = new HashMap<>();
+
+    /** {@inheritDoc} */
+    @Override protected IgniteConfiguration getConfiguration(String igniteInstanceName) throws Exception {
+        ListeningTestLogger testLog = new ListeningTestLogger(false, log);
+
+        LogListener lsnr = LogListener.matches(
+            "Property autoActivation deprecated. Use clusterStateOnStart instead."
+        ).build();
+
+        testLog.registerListener(lsnr);
+
+        logListeners.put(igniteInstanceName, lsnr);
+
+        return super.getConfiguration(igniteInstanceName)
+            .setGridLogger(testLog)
+            .setAutoActivationEnabled(autoActivation)
+            .setCacheConfiguration(new CacheConfiguration().setName(DEFAULT_CACHE_NAME))
+            .setDataStorageConfiguration(
+                new DataStorageConfiguration()
+                    .setDefaultDataRegionConfiguration(
+                        new DataRegionConfiguration().setPersistenceEnabled(true)
+                    )
+            );
+    }
+
+    /** {@inheritDoc} */
+    @Override protected void afterTest() throws Exception {
+        stopAllGrids();
+
+        cleanPersistenceDir();
+
+        super.afterTest();
+    }
+
+    /** {@inheritDoc} */
+    @Override protected void beforeTest() throws Exception {
+        super.beforeTest();
+
+        stopAllGrids();
+
+        cleanPersistenceDir();
+    }
+
+    /** */
+    @Test
+    public void testAutoActivationEnabled() throws Exception {
+        autoActivation = true;
+
+        checkProperty();
+    }
+
+    /** */
+    @Test
+    public void testAutoActivationDisabled() throws Exception {
+        autoActivation = false;
+
+        checkProperty();
+    }
+
+    /** */
+    private void checkProperty() throws Exception {
+        final int nodeCnt = 2;
+
+        IgniteEx crd = startGrids(nodeCnt);
+
+        for (int i=0; i < nodeCnt; i++) {
+            assertEquals(autoActivation, grid(i).configuration().isAutoActivationEnabled());
+
+            assertNull(grid(i).configuration().getClusterStateOnStart());
+        }
+
+        crd.cluster().state(ACTIVE);
+
+        stopAllGrids();
+
+        crd = startGrids(nodeCnt);
+
+        for (int i=0; i < nodeCnt; i++)
+            assertEquals(autoActivation ? ACTIVE : INACTIVE, crd.cluster().state());
+
+        for (String name : logListeners.keySet())
+            assertTrue(name, logListeners.get(name).check());
+    }
+}
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/ClusterActiveStateChangeWithNodeOutOfBaselineTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/ClusterActiveStateChangeWithNodeOutOfBaselineTest.java
new file mode 100644
index 0000000..0dfb418
--- /dev/null
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/ClusterActiveStateChangeWithNodeOutOfBaselineTest.java
@@ -0,0 +1,134 @@
+/*
+ * 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.cache;
+
+import java.util.Collection;
+import org.apache.ignite.Ignite;
+import org.apache.ignite.cluster.BaselineNode;
+import org.apache.ignite.cluster.ClusterState;
+import org.apache.ignite.configuration.DataRegionConfiguration;
+import org.apache.ignite.configuration.DataStorageConfiguration;
+import org.apache.ignite.configuration.IgniteConfiguration;
+import org.apache.ignite.testframework.junits.common.GridCommonAbstractTest;
+import org.junit.Test;
+
+import static org.apache.ignite.cluster.ClusterState.ACTIVE;
+import static org.apache.ignite.cluster.ClusterState.ACTIVE_READ_ONLY;
+import static org.apache.ignite.cluster.ClusterState.active;
+import static org.apache.ignite.testframework.GridTestUtils.assertNotContains;
+
+/**
+ * Checks that node out of baseline will no be added to baseline during transition from one active cluster state to
+ * another active cluster state.
+ */
+public class ClusterActiveStateChangeWithNodeOutOfBaselineTest extends GridCommonAbstractTest {
+    /** Nodes count. */
+    private static final int NODES_CNT = 3;
+
+    /** Id of node out of baseline. */
+    private static final int NODE_OUT_OF_BASELINE_ID = NODES_CNT;
+
+    /** {@inheritDoc} */
+    @Override protected void beforeTestsStarted() throws Exception {
+        super.beforeTestsStarted();
+
+        stopAllGrids();
+
+        cleanPersistenceDir();
+
+        Ignite crd = startGridsMultiThreaded(NODES_CNT);
+
+        crd.cluster().state(ClusterState.ACTIVE);
+
+        startGrid(NODE_OUT_OF_BASELINE_ID);
+
+        checkBaseline();
+    }
+
+    /** {@inheritDoc} */
+    @Override protected void afterTestsStopped() throws Exception {
+        stopAllGrids();
+
+        cleanPersistenceDir();
+
+        super.afterTestsStopped();
+    }
+
+
+    /** {@inheritDoc} */
+    @Override protected IgniteConfiguration getConfiguration(String igniteInstanceName) throws Exception {
+        return super.getConfiguration(igniteInstanceName)
+            .setDataStorageConfiguration(
+                new DataStorageConfiguration().setDefaultDataRegionConfiguration(
+                    new DataRegionConfiguration().setPersistenceEnabled(true)
+                )
+            );
+    }
+
+    /** */
+    @Test
+    public void testActiveActive() {
+        check(ACTIVE, ACTIVE);
+    }
+
+    /** */
+    @Test
+    public void testActiveReadOnly() {
+        check(ACTIVE, ACTIVE_READ_ONLY);
+    }
+
+    /** */
+    @Test
+    public void testReadOnlyActive() {
+        check(ACTIVE_READ_ONLY, ACTIVE);
+    }
+
+    /** */
+    @Test
+    public void testReadOnlyReadOnly() {
+        check(ACTIVE_READ_ONLY, ACTIVE_READ_ONLY);
+    }
+
+    /** */
+    private void check(ClusterState initialState, ClusterState targetState) {
+        assertTrue(initialState + "", active(initialState));
+        assertTrue(targetState + "", active(targetState));
+
+        if (grid(0).cluster().state() != initialState)
+            grid(0).cluster().state(initialState);
+
+        assertEquals(initialState, grid(0).cluster().state());
+
+        checkBaseline();
+
+        grid(0).cluster().state(targetState);
+
+        assertEquals(targetState, grid(0).cluster().state());
+
+        checkBaseline();
+    }
+
+    /** */
+    private void checkBaseline() {
+        Collection<BaselineNode> bltNodes = grid(0).cluster().currentBaselineTopology();
+
+        assertEquals(NODES_CNT, bltNodes.size());
+
+        assertNotContains(log, bltNodes, grid(NODE_OUT_OF_BASELINE_ID).cluster().localNode());
+    }
+}
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/ClusterReadOnlyModeAbstractTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/ClusterReadOnlyModeAbstractTest.java
index 5e92b33..ffb23dd 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/ClusterReadOnlyModeAbstractTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/ClusterReadOnlyModeAbstractTest.java
@@ -18,6 +18,7 @@
 package org.apache.ignite.internal.processors.cache;
 
 import java.util.Collection;
+import org.apache.ignite.cluster.ClusterState;
 import org.apache.ignite.configuration.IgniteConfiguration;
 import org.apache.ignite.failure.StopNodeOrHaltFailureHandler;
 import org.apache.ignite.testframework.junits.common.GridCommonAbstractTest;
@@ -66,6 +67,6 @@ public class ClusterReadOnlyModeAbstractTest extends GridCommonAbstractTest {
      * @param readOnly Read only.
      */
     protected void changeClusterReadOnlyMode(boolean readOnly) {
-        grid(0).cluster().readOnly(readOnly);
+        grid(0).cluster().state(readOnly ? ClusterState.ACTIVE_READ_ONLY : ClusterState.ACTIVE);
     }
 }
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/ClusterReadOnlyModeTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/ClusterReadOnlyModeTest.java
index 327a3f5..9c1d2e7 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/ClusterReadOnlyModeTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/ClusterReadOnlyModeTest.java
@@ -25,7 +25,7 @@ import java.util.concurrent.TimeUnit;
 import org.apache.ignite.IgniteCheckedException;
 import org.apache.ignite.IgniteDataStreamer;
 import org.apache.ignite.internal.IgniteInternalFuture;
-import org.apache.ignite.internal.cluster.ClusterReadOnlyModeCheckedException;
+import org.apache.ignite.internal.processors.cache.distributed.dht.IgniteClusterReadOnlyException;
 import org.apache.ignite.internal.util.typedef.X;
 import org.apache.ignite.testframework.GridTestUtils;
 import org.junit.Test;
@@ -162,7 +162,7 @@ public class ClusterReadOnlyModeTest extends ClusterReadOnlyModeAbstractTest {
                 Exception e = eMap.get(cacheName);
 
                 assertNotNull(cacheName, e);
-                assertTrue(cacheName + " " + e, X.hasCause(e, ClusterReadOnlyModeCheckedException.class));
+                assertTrue(cacheName + " " + e, X.hasCause(e, IgniteClusterReadOnlyException.class));
             }
         }
         finally {
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/ClusterStateAbstractTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/ClusterStateAbstractTest.java
index f6853f6..a7c0101 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/ClusterStateAbstractTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/ClusterStateAbstractTest.java
@@ -18,368 +18,175 @@
 
 package org.apache.ignite.internal.processors.cache;
 
-import java.util.Collection;
-import java.util.concurrent.Callable;
-import java.util.concurrent.atomic.AtomicReference;
-import java.util.concurrent.locks.Lock;
-import org.apache.ignite.Ignite;
-import org.apache.ignite.IgniteCache;
-import org.apache.ignite.IgniteException;
-import org.apache.ignite.cluster.ClusterNode;
+import java.util.List;
+import java.util.stream.Collectors;
+import java.util.stream.IntStream;
+import org.apache.ignite.cluster.ClusterState;
 import org.apache.ignite.configuration.CacheConfiguration;
 import org.apache.ignite.configuration.IgniteConfiguration;
-import org.apache.ignite.internal.managers.communication.GridIoMessage;
-import org.apache.ignite.internal.processors.cache.distributed.dht.preloader.GridDhtPartitionDemandMessage;
-import org.apache.ignite.internal.processors.cache.distributed.dht.preloader.GridDhtPartitionSupplyMessage;
-import org.apache.ignite.internal.util.GridConcurrentHashSet;
-import org.apache.ignite.internal.util.lang.GridAbsPredicate;
-import org.apache.ignite.lang.IgniteInClosure;
-import org.apache.ignite.plugin.extensions.communication.Message;
-import org.apache.ignite.spi.IgniteSpiException;
-import org.apache.ignite.spi.communication.tcp.TcpCommunicationSpi;
-import org.apache.ignite.testframework.GridTestUtils;
+import org.apache.ignite.internal.IgniteEx;
 import org.apache.ignite.testframework.junits.common.GridCommonAbstractTest;
-import org.apache.ignite.transactions.Transaction;
-import org.apache.ignite.transactions.TransactionConcurrency;
-import org.apache.ignite.transactions.TransactionIsolation;
 import org.junit.Test;
 
+import static org.apache.ignite.cluster.ClusterState.ACTIVE;
+import static org.apache.ignite.cluster.ClusterState.INACTIVE;
+import static org.apache.ignite.cluster.ClusterState.ACTIVE_READ_ONLY;
+
 /**
  *
  */
 public abstract class ClusterStateAbstractTest extends GridCommonAbstractTest {
-    /** Entry count. */
-    public static final int ENTRY_CNT = 5000;
-
     /** */
-    public static final int GRID_CNT = 4;
+    protected static final int GRID_CNT = 2;
 
     /** */
-    private static final String CACHE_NAME = "cache1";
-
-    /** */
-    private static final Collection<Class> forbidden = new GridConcurrentHashSet<>();
-
-    /** */
-    private static AtomicReference<Exception> errEncountered = new AtomicReference<>();
-
-    /** */
-    private boolean activeOnStart = true;
-
-    /** */
-    private boolean client;
+    protected static final String CACHE_NAME = "cache1";
 
     /** {@inheritDoc} */
     @Override protected IgniteConfiguration getConfiguration(String gridName) throws Exception {
         IgniteConfiguration cfg = super.getConfiguration(gridName);
 
-        cfg.setActiveOnStart(activeOnStart);
+        cfg.setClusterStateOnStart(INACTIVE);
 
         cfg.setCacheConfiguration(cacheConfiguration(CACHE_NAME));
 
-        if (client)
-            cfg.setClientMode(true);
-
-        cfg.setCommunicationSpi(new TestCommunicationSpi());
-
         return cfg;
     }
 
-    /**
-     * @param cacheName Cache name.
-     * @return Cache configuration.
-     */
-    protected abstract CacheConfiguration cacheConfiguration(String cacheName);
-
     /** {@inheritDoc} */
-    @Override protected void afterTest() throws Exception {
-        stopAllGrids();
+    @Override protected void beforeTestsStarted() throws Exception {
+        super.beforeTestsStarted();
 
-        forbidden.clear();
-
-        Exception err = errEncountered.getAndSet(null);
+        stopAllGrids();
 
-        if (err != null)
-            throw err;
+        startGridsMultiThreaded(GRID_CNT);
     }
 
-    /**
-     * @throws Exception if failed.
-     */
-    @Test
-    public void testDynamicCacheStart() throws Exception {
-        activeOnStart = false;
-
-        forbidden.add(GridDhtPartitionSupplyMessage.class);
-        forbidden.add(GridDhtPartitionDemandMessage.class);
-
-        startGrids(GRID_CNT);
-
-        checkInactive(GRID_CNT);
+    /** {@inheritDoc} */
+    @Override protected void afterTestsStopped() throws Exception {
+        stopAllGrids();
 
-        forbidden.clear();
+        super.afterTestsStopped();
+    }
 
-        grid(0).cluster().active(true);
+    /** {@inheritDoc} */
+    @Override protected void beforeTest() throws Exception {
+        super.beforeTest();
 
-        IgniteCache<Object, Object> cache2 = grid(0).createCache(new CacheConfiguration<>("cache2"));
+        grid(0).cluster().state(INACTIVE);
 
-        for (int k = 0; k < ENTRY_CNT; k++)
-            cache2.put(k, k);
+        checkInactive(nodesCount());
+    }
 
-        grid(0).cluster().active(false);
+    /** {@inheritDoc} */
+    @Override protected void afterTest() throws Exception {
+        grid(0).cluster().state(INACTIVE);
 
-        checkInactive(GRID_CNT);
+        checkInactive(nodesCount());
 
-        stopAllGrids();
+        super.afterTest();
     }
 
-    /**
-     * @throws Exception if failed.
-     */
+    /** */
     @Test
-    public void testNoRebalancing() throws Exception {
-        activeOnStart = false;
-
-        forbidden.add(GridDhtPartitionSupplyMessage.class);
-        forbidden.add(GridDhtPartitionDemandMessage.class);
-
-        startGrids(GRID_CNT);
-
-        checkInactive(GRID_CNT);
-
-        forbidden.clear();
-
-        grid(0).cluster().active(true);
-
-        awaitPartitionMapExchange();
-
-        final IgniteCache<Object, Object> cache = grid(0).cache(CACHE_NAME);
-
-        for (int k = 0; k < ENTRY_CNT; k++)
-            cache.put(k, k);
-
-        for (int g = 0; g < GRID_CNT; g++) {
-            // Tests that state changes are propagated to existing and new nodes.
-            assertTrue(grid(g).cluster().active());
-
-            IgniteCache<Object, Object> cache0 = grid(g).cache(CACHE_NAME);
-
-            for (int k = 0; k < ENTRY_CNT; k++)
-                assertEquals(k,  cache0.get(k));
-        }
-
-        // Check that new node startup and shutdown works fine after activation.
-        startGrid(GRID_CNT);
-        startGrid(GRID_CNT + 1);
-
-        for (int g = 0; g < GRID_CNT + 2; g++) {
-            IgniteCache<Object, Object> cache0 = grid(g).cache(CACHE_NAME);
-
-            for (int k = 0; k < ENTRY_CNT; k++)
-                assertEquals("Failed for [grid=" + g + ", key=" + k + ']', k, cache0.get(k));
-        }
-
-        stopGrid(GRID_CNT + 1);
-
-        for (int g = 0; g < GRID_CNT + 1; g++)
-            grid(g).cache(CACHE_NAME).rebalance().get();
-
-        stopGrid(GRID_CNT);
-
-        for (int g = 0; g < GRID_CNT; g++) {
-            IgniteCache<Object, Object> cache0 = grid(g).cache(CACHE_NAME);
-
-            for (int k = 0; k < ENTRY_CNT; k++)
-                assertEquals(k,  cache0.get(k));
-        }
-
-        grid(0).cluster().active(false);
-
-        GridTestUtils.waitForCondition(new GridAbsPredicate() {
-            @Override public boolean apply() {
-                for (int g = 0; g < GRID_CNT; g++) {
-                    if (grid(g).cluster().active())
-                        return false;
-                }
-
-                return true;
-            }
-        }, 5000);
-
-        checkInactive(GRID_CNT);
-
-        forbidden.add(GridDhtPartitionSupplyMessage.class);
-        forbidden.add(GridDhtPartitionDemandMessage.class);
-
-        // Should stop without exchange.
-        stopAllGrids();
+    public void testActivation() {
+        changeStateAndCheckBehaviour(INACTIVE, ACTIVE);
     }
 
-    /**
-     * @throws Exception if failed.
-     */
+    /** */
     @Test
-    public void testActivationFromClient() throws Exception {
-        forbidden.add(GridDhtPartitionSupplyMessage.class);
-        forbidden.add(GridDhtPartitionDemandMessage.class);
-
-        activeOnStart = false;
-
-        startGrids(GRID_CNT);
-
-        client = true;
-
-        startGrid(GRID_CNT);
-
-        checkInactive(GRID_CNT + 1);
-
-        Ignite cl = grid(GRID_CNT);
-
-        forbidden.clear();
-
-        cl.cluster().active(true);
-
-        awaitPartitionMapExchange();
-
-        IgniteCache<Object, Object> cache = cl.cache(CACHE_NAME);
-
-        for (int k = 0; k < ENTRY_CNT; k++)
-            cache.put(k, k);
-
-        for (int g = 0; g < GRID_CNT + 1; g++) {
-            // Tests that state changes are propagated to existing and new nodes.
-            assertTrue(grid(g).cluster().active());
-
-            IgniteCache<Object, Object> cache0 = grid(g).cache(CACHE_NAME);
-
-            for (int k = 0; k < ENTRY_CNT; k++)
-                assertEquals(k,  cache0.get(k));
-        }
+    public void testActivationWithReadOnly() {
+        changeStateAndCheckBehaviour(INACTIVE, ACTIVE_READ_ONLY);
+    }
 
-        cl.cluster().active(false);
+    /** */
+    @Test
+    public void testEnablingReadOnly() {
+        changeStateAndCheckBehaviour(ACTIVE, ACTIVE_READ_ONLY);
+    }
 
-        GridTestUtils.waitForCondition(new GridAbsPredicate() {
-            @Override public boolean apply() {
-                for (int g = 0; g < GRID_CNT + 1; g++) {
-                    if (grid(g).cluster().active())
-                        return false;
-                }
+    /** */
+    @Test
+    public void testDisablingReadOnly() {
+        changeStateAndCheckBehaviour(ACTIVE_READ_ONLY, ACTIVE);
+    }
 
-                return true;
-            }
-        }, 5000);
+    /** */
+    @Test
+    public void testDeactivation() {
+        changeStateAndCheckBehaviour(ACTIVE, INACTIVE);
+    }
 
-        checkInactive(GRID_CNT + 1);
+    /** */
+    @Test
+    public void testDeactivationFromReadOnly() {
+        changeStateAndCheckBehaviour(ACTIVE_READ_ONLY, INACTIVE);
     }
 
     /**
-     * Tests that deactivation is prohibited if explicit lock is held in current thread.
-     *
-     * @throws Exception If fails.
+     * @param cacheName Cache name.
+     * @return Cache configuration.
      */
-    @Test
-    public void testDeactivationWithPendingLock() throws Exception {
-        startGrids(GRID_CNT);
-
-        Lock lock = grid(0).cache(CACHE_NAME).lock(1);
-
-        lock.lock();
-
-        try {
-            //noinspection ThrowableNotThrown
-            GridTestUtils.assertThrowsAnyCause(log, new Callable<Object>() {
-                @Override public Object call() {
-                    grid(0).cluster().active(false);
+    protected abstract CacheConfiguration cacheConfiguration(String cacheName);
 
-                    return null;
-                }
-            }, IgniteException.class,
-                "Failed to deactivate cluster (must invoke the method outside of an active transaction).");
-        }
-        finally {
-            lock.unlock();
-        }
-    }
+    /** */
+    protected abstract void changeState(ClusterState state);
 
     /**
-     * Tests that deactivation is prohibited if transaction is active in current thread.
+     * Changes cluster state from {@code initialState} to {@code targetState}.
      *
-     * @throws Exception If failed.
+     * @param initialState Initial state.
+     * @param targetState Target state.
      */
-    @Test
-    public void testDeactivationWithPendingTransaction() throws Exception {
-        startGrids(GRID_CNT);
+    private void changeStateAndCheckBehaviour(ClusterState initialState, ClusterState targetState) {
+        assertNotSame(initialState, targetState);
 
-        for (TransactionConcurrency concurrency : TransactionConcurrency.values()) {
-            for (TransactionIsolation isolation : TransactionIsolation.values())
-                deactivateWithPendingTransaction(concurrency, isolation);
-        }
-    }
+        IgniteEx crd = grid(0);
 
-    /**
-     */
-    private void deactivateWithPendingTransaction(TransactionConcurrency concurrency,
-        TransactionIsolation isolation) {
-        final Ignite ignite0 = grid(0);
+        checkInactive(nodesCount());
 
-        final IgniteCache<Object, Object> cache0 = ignite0.cache(CACHE_NAME);
+        long timeOnStart = crd.context().state().lastStateChangeTime();
 
-        try (Transaction ignore = ignite0.transactions().txStart(concurrency, isolation)) {
-            cache0.put(1, "1");
+        assertNotSame(0, timeOnStart);
+        assertTrue(String.valueOf(timeOnStart), timeOnStart > 0);
 
-            //noinspection ThrowableNotThrown
-            GridTestUtils.assertThrowsAnyCause(log, new Callable<Object>() {
-                @Override public Object call() {
-                    grid(0).cluster().active(false);
+        crd.cluster().state(initialState);
 
-                    return null;
-                }
-            }, IgniteException.class,
-                "Failed to deactivate cluster (must invoke the method outside of an active transaction).");
-        }
+        if (initialState == INACTIVE)
+            assertEquals(timeOnStart, crd.context().state().lastStateChangeTime());
+        else {
+            long activationTime = crd.context().state().lastStateChangeTime();
 
-        assertNull(cache0.get(1));
-        assertNull(ignite0.transactions().tx());
-    }
+            assertNotSame(timeOnStart, activationTime);
+            assertTrue(activationTime + " " + timeOnStart, activationTime > timeOnStart);
+        }
 
-    /**
-     *
-     */
-    private void checkInactive(int cnt) {
-        for (int g = 0; g < cnt; g++)
-            assertFalse(grid(g).cluster().active());
-    }
+        checkClusterState(nodesCount(), initialState);
 
-    /**
-     *
-     */
-    private static class TestCommunicationSpi extends TcpCommunicationSpi {
-        /** {@inheritDoc} */
-        @Override public void sendMessage(ClusterNode node, Message msg, IgniteInClosure<IgniteException> ackC) throws IgniteSpiException {
-            checkForbidden((GridIoMessage)msg);
+        changeState(targetState);
 
-            super.sendMessage(node, msg, ackC);
-        }
+        checkClusterState(nodesCount(), targetState);
 
-        /** {@inheritDoc} */
-        @Override public void sendMessage(ClusterNode node, Message msg) throws IgniteSpiException {
-            checkForbidden((GridIoMessage)msg);
+        List<IgniteEx> nodes = IntStream.range(0, nodesCount())
+            .mapToObj(this::grid)
+            .collect(Collectors.toList());
 
-            super.sendMessage(node, msg);
-        }
+        ClusterStateTestUtils.putSomeDataAndCheck(log, nodes, CACHE_NAME);
+    }
 
-        /**
-         * @param msg Message to check.
-         */
-        private void checkForbidden(GridIoMessage msg) {
-            if (forbidden.contains(msg.message().getClass())) {
-                IgniteSpiException err = new IgniteSpiException("Message is forbidden for this test: " + msg.message());
+    /** */
+    protected int nodesCount() {
+        return GRID_CNT;
+    }
 
-                // Set error in case if this exception is not visible to the user code.
-                errEncountered.compareAndSet(null, err);
+    /** */
+    private void checkClusterState(int nodesCnt, ClusterState state) {
+        for (int g = 0; g < nodesCnt ; g++)
+            assertEquals(grid(g).name(), state, grid(g).cluster().state());
+    }
 
-                throw err;
-            }
-        }
+    /** */
+    private void checkInactive(int cnt) {
+        checkClusterState(cnt, INACTIVE);
     }
 }
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/ClusterStateClientAbstractTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/ClusterStateClientAbstractTest.java
new file mode 100644
index 0000000..652dc26
--- /dev/null
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/ClusterStateClientAbstractTest.java
@@ -0,0 +1,58 @@
+/*
+ * 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.cache;
+
+import org.apache.ignite.cluster.ClusterState;
+import org.apache.ignite.configuration.IgniteConfiguration;
+import org.apache.ignite.internal.IgniteEx;
+
+/**
+ * Tests that cluster state change works correctly with connected fat client in different situations.
+ */
+public abstract class ClusterStateClientAbstractTest extends ClusterStateAbstractTest {
+    /** */
+    private boolean client;
+
+    /** {@inheritDoc} */
+    @Override protected int nodesCount() {
+        return GRID_CNT + 1;
+    }
+
+    /** {@inheritDoc} */
+    @Override protected IgniteConfiguration getConfiguration(String gridName) throws Exception {
+        return super.getConfiguration(gridName).setClientMode(client);
+    }
+
+    /** {@inheritDoc} */
+    @Override protected void beforeTestsStarted() throws Exception {
+        super.beforeTestsStarted();
+
+        client = true;
+
+        startGrid(GRID_CNT);
+    }
+
+    /** {@inheritDoc} */
+    @Override protected void changeState(ClusterState state) {
+        IgniteEx cl = grid(GRID_CNT);
+
+        assertTrue(cl.configuration().isClientMode());
+
+        cl.cluster().state(state);
+    }
+}
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/ClusterStatePartitionedSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/ClusterStateClientPartitionedSelfTest.java
similarity index 53%
copy from modules/core/src/test/java/org/apache/ignite/internal/processors/cache/ClusterStatePartitionedSelfTest.java
copy to modules/core/src/test/java/org/apache/ignite/internal/processors/cache/ClusterStateClientPartitionedSelfTest.java
index 4d8b188..4ad54a7 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/ClusterStatePartitionedSelfTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/ClusterStateClientPartitionedSelfTest.java
@@ -1,41 +1,30 @@
 /*
  * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
+ * 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
+ * the License. You may obtain a copy of the License at
  *
- *      http://www.apache.org/licenses/LICENSE-2.0
+ * 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.cache;
 
-import org.apache.ignite.cache.CacheAtomicityMode;
-import org.apache.ignite.cache.CacheMode;
-import org.apache.ignite.cache.CacheWriteSynchronizationMode;
 import org.apache.ignite.configuration.CacheConfiguration;
 
 /**
  *
  */
-public class ClusterStatePartitionedSelfTest extends ClusterStateAbstractTest {
+public class ClusterStateClientPartitionedSelfTest extends ClusterStateClientAbstractTest {
     /** {@inheritDoc} */
     @Override protected CacheConfiguration cacheConfiguration(String cacheName) {
-        CacheConfiguration ccfg = new CacheConfiguration(cacheName);
-
-        ccfg.setAtomicityMode(CacheAtomicityMode.TRANSACTIONAL);
-        ccfg.setCacheMode(CacheMode.PARTITIONED);
-        ccfg.setWriteSynchronizationMode(CacheWriteSynchronizationMode.FULL_SYNC);
-        ccfg.setBackups(1);
-
-        return ccfg;
+        return ClusterStateTestUtils.partitionedCache(cacheName);
     }
 }
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/ClusterStatePartitionedSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/ClusterStateClientReplicatedSelfTest.java
similarity index 54%
copy from modules/core/src/test/java/org/apache/ignite/internal/processors/cache/ClusterStatePartitionedSelfTest.java
copy to modules/core/src/test/java/org/apache/ignite/internal/processors/cache/ClusterStateClientReplicatedSelfTest.java
index 4d8b188..5756f5d 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/ClusterStatePartitionedSelfTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/ClusterStateClientReplicatedSelfTest.java
@@ -1,41 +1,30 @@
 /*
  * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
+ * 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
+ * the License. You may obtain a copy of the License at
  *
- *      http://www.apache.org/licenses/LICENSE-2.0
+ * 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.cache;
 
-import org.apache.ignite.cache.CacheAtomicityMode;
-import org.apache.ignite.cache.CacheMode;
-import org.apache.ignite.cache.CacheWriteSynchronizationMode;
 import org.apache.ignite.configuration.CacheConfiguration;
 
 /**
  *
  */
-public class ClusterStatePartitionedSelfTest extends ClusterStateAbstractTest {
+public class ClusterStateClientReplicatedSelfTest extends ClusterStateClientAbstractTest {
     /** {@inheritDoc} */
     @Override protected CacheConfiguration cacheConfiguration(String cacheName) {
-        CacheConfiguration ccfg = new CacheConfiguration(cacheName);
-
-        ccfg.setAtomicityMode(CacheAtomicityMode.TRANSACTIONAL);
-        ccfg.setCacheMode(CacheMode.PARTITIONED);
-        ccfg.setWriteSynchronizationMode(CacheWriteSynchronizationMode.FULL_SYNC);
-        ccfg.setBackups(1);
-
-        return ccfg;
+        return ClusterStateTestUtils.replicatedCache(cacheName);
     }
 }
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/ClusterStateNoRebalanceAbstractTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/ClusterStateNoRebalanceAbstractTest.java
new file mode 100644
index 0000000..ccce702
--- /dev/null
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/ClusterStateNoRebalanceAbstractTest.java
@@ -0,0 +1,219 @@
+/*
+ * 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.cache;
+
+import java.util.Collection;
+import java.util.concurrent.atomic.AtomicReference;
+import java.util.stream.Collectors;
+import java.util.stream.IntStream;
+import org.apache.ignite.IgniteCache;
+import org.apache.ignite.IgniteException;
+import org.apache.ignite.cluster.ClusterNode;
+import org.apache.ignite.cluster.ClusterState;
+import org.apache.ignite.configuration.CacheConfiguration;
+import org.apache.ignite.configuration.IgniteConfiguration;
+import org.apache.ignite.internal.IgniteEx;
+import org.apache.ignite.internal.managers.communication.GridIoMessage;
+import org.apache.ignite.internal.processors.cache.distributed.dht.preloader.GridDhtPartitionDemandMessage;
+import org.apache.ignite.internal.processors.cache.distributed.dht.preloader.GridDhtPartitionSupplyMessage;
+import org.apache.ignite.internal.util.GridConcurrentHashSet;
+import org.apache.ignite.lang.IgniteInClosure;
+import org.apache.ignite.plugin.extensions.communication.Message;
+import org.apache.ignite.spi.IgniteSpiException;
+import org.apache.ignite.spi.communication.tcp.TcpCommunicationSpi;
+import org.apache.ignite.testframework.junits.common.GridCommonAbstractTest;
+import org.junit.Test;
+
+import static com.google.common.base.Functions.identity;
+import static org.apache.ignite.cluster.ClusterState.ACTIVE;
+import static org.apache.ignite.cluster.ClusterState.INACTIVE;
+
+/**
+ * Checks that join node to {@link ClusterState#INACTIVE} cluster doesn't trigger rebalance.
+ */
+public abstract class ClusterStateNoRebalanceAbstractTest extends GridCommonAbstractTest {
+    /** Entry count. */
+    protected static final int ENTRY_CNT = 5000;
+
+    /** */
+    protected static final Collection<Class> forbidden = new GridConcurrentHashSet<>();
+
+    /** */
+    private static AtomicReference<Exception> errEncountered = new AtomicReference<>();
+
+    /** */
+    private static final int GRID_CNT = 2;
+
+    /**
+     * @param cacheName Cache name.
+     * @return Cache configuration.
+     */
+    protected abstract CacheConfiguration cacheConfiguration(String cacheName);
+
+    /** {@inheritDoc} */
+    @Override protected IgniteConfiguration getConfiguration(String gridName) throws Exception {
+        IgniteConfiguration cfg = super.getConfiguration(gridName);
+
+        cfg.setActiveOnStart(false);
+
+        cfg.setCacheConfiguration(cacheConfiguration(DEFAULT_CACHE_NAME));
+
+        cfg.setClientMode(gridName.startsWith("client"));
+
+        cfg.setCommunicationSpi(new TestCommunicationSpi());
+
+        return cfg;
+    }
+
+    /** {@inheritDoc} */
+    @Override protected void afterTest() throws Exception {
+        stopAllGrids();
+
+        forbidden.clear();
+
+        Exception err = errEncountered.getAndSet(null);
+
+        if (err != null)
+            throw err;
+
+        super.afterTest();
+    }
+
+    /**
+     * @throws Exception if failed.
+     */
+    @Test
+    public void testNoRebalancing() throws Exception {
+        forbidden.add(GridDhtPartitionSupplyMessage.class);
+        forbidden.add(GridDhtPartitionDemandMessage.class);
+
+        startGrids(GRID_CNT);
+
+        checkInactive(GRID_CNT);
+
+        forbidden.clear();
+
+        grid(0).cluster().state(ACTIVE);
+
+        awaitPartitionMapExchange();
+
+        final IgniteCache<Object, Object> cache = grid(0).cache(DEFAULT_CACHE_NAME);
+
+        cache.putAll(IntStream.range(0, ENTRY_CNT).boxed().collect(Collectors.toMap(identity(), identity())));
+
+        for (int g = 0; g < GRID_CNT; g++) {
+            // Tests that state changes are propagated to existing and new nodes.
+            assertEquals(ACTIVE, grid(g).cluster().state());
+
+            for (int k = 0; k < ENTRY_CNT; k++)
+                assertEquals(k,  grid(g).cache(DEFAULT_CACHE_NAME).get(k));
+        }
+
+        // Check that new node startup and shutdown works fine after activation.
+        startGrid(GRID_CNT);
+        startGrid(GRID_CNT + 1);
+
+        for (int g = 0; g < GRID_CNT + 2; g++) {
+            for (int k = 0; k < ENTRY_CNT; k++)
+                assertEquals("Failed for [grid=" + g + ", key=" + k + ']', k, grid(g).cache(DEFAULT_CACHE_NAME).get(k));
+        }
+
+        stopGrid(GRID_CNT + 1);
+
+        for (int g = 0; g < GRID_CNT + 1; g++)
+            grid(g).cache(DEFAULT_CACHE_NAME).rebalance().get();
+
+        stopGrid(GRID_CNT);
+
+        for (int g = 0; g < GRID_CNT; g++) {
+            IgniteCache<Object, Object> cache0 = grid(g).cache(DEFAULT_CACHE_NAME);
+
+            for (int k = 0; k < ENTRY_CNT; k++)
+                assertEquals(k,  cache0.get(k));
+        }
+
+        grid(0).cluster().state(INACTIVE);
+
+        checkInactive(GRID_CNT);
+
+        forbidden.add(GridDhtPartitionSupplyMessage.class);
+        forbidden.add(GridDhtPartitionDemandMessage.class);
+
+        // Should stop without exchange.
+        stopAllGrids();
+    }
+
+    /**
+     * @throws Exception if failed.
+     */
+    @Test
+    public void testNoRebalancingWithClient() throws Exception {
+        forbidden.add(GridDhtPartitionSupplyMessage.class);
+        forbidden.add(GridDhtPartitionDemandMessage.class);
+
+        startGrids(GRID_CNT);
+
+        IgniteEx client = startGrid("client");
+
+        assertTrue(client.configuration().isClientMode());
+
+        checkInactive(GRID_CNT);
+
+        assertEquals(INACTIVE, client.cluster().state());
+    }
+
+    /** */
+    void checkInactive(int cnt) {
+        for (int g = 0; g < cnt ; g++)
+            assertEquals(grid(g).name(), INACTIVE, grid(g).cluster().state());
+    }
+
+    /**
+     *
+     */
+    private static class TestCommunicationSpi extends TcpCommunicationSpi {
+        /** {@inheritDoc} */
+        @Override public void sendMessage(
+            ClusterNode node, Message msg, IgniteInClosure<IgniteException> ackC) throws IgniteSpiException {
+            checkForbidden((GridIoMessage)msg);
+
+            super.sendMessage(node, msg, ackC);
+        }
+
+        /** {@inheritDoc} */
+        @Override public void sendMessage(ClusterNode node, Message msg) throws IgniteSpiException {
+            checkForbidden((GridIoMessage)msg);
+
+            super.sendMessage(node, msg);
+        }
+
+        /**
+         * @param msg Message to check.
+         */
+        private void checkForbidden(GridIoMessage msg) {
+            if (forbidden.contains(msg.message().getClass())) {
+                IgniteSpiException err = new IgniteSpiException("Message is forbidden for this test: " + msg.message());
+
+                // Set error in case if this exception is not visible to the user code.
+                errEncountered.compareAndSet(null, err);
+
+                throw err;
+            }
+        }
+    }
+}
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/ClusterStatePartitionedSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/ClusterStateNoRebalancePartitionedTest.java
similarity index 53%
copy from modules/core/src/test/java/org/apache/ignite/internal/processors/cache/ClusterStatePartitionedSelfTest.java
copy to modules/core/src/test/java/org/apache/ignite/internal/processors/cache/ClusterStateNoRebalancePartitionedTest.java
index 4d8b188..4c15731 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/ClusterStatePartitionedSelfTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/ClusterStateNoRebalancePartitionedTest.java
@@ -1,41 +1,30 @@
 /*
  * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
+ * 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
+ * the License. You may obtain a copy of the License at
  *
- *      http://www.apache.org/licenses/LICENSE-2.0
+ * 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.cache;
 
-import org.apache.ignite.cache.CacheAtomicityMode;
-import org.apache.ignite.cache.CacheMode;
-import org.apache.ignite.cache.CacheWriteSynchronizationMode;
 import org.apache.ignite.configuration.CacheConfiguration;
 
 /**
  *
  */
-public class ClusterStatePartitionedSelfTest extends ClusterStateAbstractTest {
+public class ClusterStateNoRebalancePartitionedTest extends ClusterStateNoRebalanceAbstractTest {
     /** {@inheritDoc} */
     @Override protected CacheConfiguration cacheConfiguration(String cacheName) {
-        CacheConfiguration ccfg = new CacheConfiguration(cacheName);
-
-        ccfg.setAtomicityMode(CacheAtomicityMode.TRANSACTIONAL);
-        ccfg.setCacheMode(CacheMode.PARTITIONED);
-        ccfg.setWriteSynchronizationMode(CacheWriteSynchronizationMode.FULL_SYNC);
-        ccfg.setBackups(1);
-
-        return ccfg;
+        return ClusterStateTestUtils.partitionedCache(cacheName);
     }
 }
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/ClusterStatePartitionedSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/ClusterStateNoRebalanceReplicatedTest.java
similarity index 54%
copy from modules/core/src/test/java/org/apache/ignite/internal/processors/cache/ClusterStatePartitionedSelfTest.java
copy to modules/core/src/test/java/org/apache/ignite/internal/processors/cache/ClusterStateNoRebalanceReplicatedTest.java
index 4d8b188..febcb96 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/ClusterStatePartitionedSelfTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/ClusterStateNoRebalanceReplicatedTest.java
@@ -1,41 +1,30 @@
 /*
  * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
+ * 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
+ * the License. You may obtain a copy of the License at
  *
- *      http://www.apache.org/licenses/LICENSE-2.0
+ * 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.cache;
 
-import org.apache.ignite.cache.CacheAtomicityMode;
-import org.apache.ignite.cache.CacheMode;
-import org.apache.ignite.cache.CacheWriteSynchronizationMode;
 import org.apache.ignite.configuration.CacheConfiguration;
 
 /**
  *
  */
-public class ClusterStatePartitionedSelfTest extends ClusterStateAbstractTest {
+public class ClusterStateNoRebalanceReplicatedTest extends ClusterStateNoRebalanceAbstractTest {
     /** {@inheritDoc} */
     @Override protected CacheConfiguration cacheConfiguration(String cacheName) {
-        CacheConfiguration ccfg = new CacheConfiguration(cacheName);
-
-        ccfg.setAtomicityMode(CacheAtomicityMode.TRANSACTIONAL);
-        ccfg.setCacheMode(CacheMode.PARTITIONED);
-        ccfg.setWriteSynchronizationMode(CacheWriteSynchronizationMode.FULL_SYNC);
-        ccfg.setBackups(1);
-
-        return ccfg;
+        return ClusterStateTestUtils.replicatedCache(cacheName);
     }
 }
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/ClusterStateOnStartPropertyTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/ClusterStateOnStartPropertyTest.java
new file mode 100644
index 0000000..df2ff52
--- /dev/null
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/ClusterStateOnStartPropertyTest.java
@@ -0,0 +1,396 @@
+/*
+ * 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.cache;
+
+import java.util.HashMap;
+import java.util.Map;
+import org.apache.ignite.cluster.ClusterState;
+import org.apache.ignite.configuration.CacheConfiguration;
+import org.apache.ignite.configuration.DataRegionConfiguration;
+import org.apache.ignite.configuration.DataStorageConfiguration;
+import org.apache.ignite.configuration.IgniteConfiguration;
+import org.apache.ignite.internal.IgniteEx;
+import org.apache.ignite.internal.util.typedef.internal.CU;
+import org.apache.ignite.testframework.ListeningTestLogger;
+import org.apache.ignite.testframework.LogListener;
+import org.apache.ignite.testframework.junits.common.GridCommonAbstractTest;
+import org.jetbrains.annotations.Nullable;
+import org.junit.Test;
+
+import static org.apache.ignite.cluster.ClusterState.ACTIVE;
+import static org.apache.ignite.cluster.ClusterState.ACTIVE_READ_ONLY;
+import static org.apache.ignite.cluster.ClusterState.INACTIVE;
+import static org.apache.ignite.configuration.IgniteConfiguration.DFLT_ACTIVE_ON_START;
+import static org.apache.ignite.configuration.IgniteConfiguration.DFLT_AUTO_ACTIVATION;
+
+/**
+ * Checks that {@link IgniteConfiguration#getClusterStateOnStart()} works correctly separately and together with
+ * {@link IgniteConfiguration#isActiveOnStart()} and {@link IgniteConfiguration#isAutoActivationEnabled()}.
+ */
+public class ClusterStateOnStartPropertyTest extends GridCommonAbstractTest {
+    /** */
+    private static final int NODES_CNT = 2;
+
+    /** */
+    private Map<String, LogListener> logListeners = new HashMap<>();
+
+    /** */
+    private ClusterState state;
+
+    /** Persistence enabled flag. */
+    private boolean persistence;
+
+    /** Auto activation flag. Null means that property should be skipped. */
+    private Boolean autoActivation;
+
+    /** Active on state flag. Null means that property should be skipped. */
+    private Boolean activeOnStart;
+
+    /** {@inheritDoc} */
+    @Override protected IgniteConfiguration getConfiguration(String igniteInstanceName) throws Exception {
+        IgniteConfiguration cfg = super.getConfiguration(igniteInstanceName)
+            .setClusterStateOnStart(state)
+            .setCacheConfiguration(new CacheConfiguration().setName(DEFAULT_CACHE_NAME))
+            .setConsistentId(igniteInstanceName)
+            .setDataStorageConfiguration(
+                new DataStorageConfiguration()
+                    .setDefaultDataRegionConfiguration(
+                        new DataRegionConfiguration().setPersistenceEnabled(persistence)
+                    )
+            );
+
+        if (autoActivation != null)
+            cfg.setAutoActivationEnabled(autoActivation);
+
+        if (activeOnStart != null)
+            cfg.setActiveOnStart(activeOnStart);
+
+        // Warn messages must be printed only if both options (old and new) are presented.
+        if (autoActivation != null || activeOnStart != null) {
+            ListeningTestLogger testLog = new ListeningTestLogger(false, log);
+
+            LogListener lsnr = LogListener.matches(
+                persistence ?
+                    "Property `autoActivation` will be ignored due to the property `clusterStateOnStart` is presented." :
+                    "Property `activeOnStart` will be ignored due to the property `clusterStateOnStart` is presented."
+            ).build();
+
+            testLog.registerListener(lsnr);
+
+            logListeners.put(igniteInstanceName, lsnr);
+
+            cfg.setGridLogger(testLog);
+        }
+
+        return cfg;
+    }
+
+    /** {@inheritDoc} */
+    @Override protected void beforeTest() throws Exception {
+        super.beforeTest();
+
+        stopAllGrids();
+
+        cleanPersistenceDir();
+    }
+
+    /** {@inheritDoc} */
+    @Override protected void afterTest() throws Exception {
+        stopAllGrids();
+
+        cleanPersistenceDir();
+
+        super.afterTest();
+    }
+
+    // Simple scenarios with in-memory cluster.
+
+    /**
+     * @see #checkPropertyInMemory(ClusterState).
+     */
+    @Test
+    public void testInactiveInMemory() throws Exception {
+        checkPropertyInMemory(INACTIVE);
+    }
+
+    /**
+     * @see #checkPropertyInMemory(ClusterState).
+     */
+    @Test
+    public void testActiveInMemory() throws Exception {
+        checkPropertyInMemory(ACTIVE);
+    }
+
+    /**
+     * @see #checkPropertyInMemory(ClusterState).
+     */
+    @Test
+    public void testReadOnlyInMemory() throws Exception {
+        checkPropertyInMemory(ACTIVE_READ_ONLY);
+    }
+
+    // Simple scenarios with persistent cluster.
+
+    /**
+     * @see #checkPropertyPersistence(ClusterState).
+     */
+    @Test
+    public void testInactivePersistence() throws Exception {
+        checkPropertyPersistence(INACTIVE);
+    }
+
+    /**
+     * @see #checkPropertyPersistence(ClusterState).
+     */
+    @Test
+    public void testActivePersistence() throws Exception {
+        checkPropertyPersistence(ACTIVE);
+    }
+
+    /**
+     * @see #checkPropertyPersistence(ClusterState).
+     */
+    @Test
+    public void testReadOnlyPersistence() throws Exception {
+        checkPropertyPersistence(ACTIVE_READ_ONLY);
+    }
+
+    // Scenarios for both (activeOnStart, clusterStateOnStart) properties with in-memory cluster.
+
+    /**
+     * @see #checkBothPropertiesInMemory(ClusterState, boolean).
+     */
+    @Test
+    public void testInactiveInactiveOnStartInMemory() throws Exception {
+        checkBothPropertiesInMemory(INACTIVE, false);
+    }
+
+    /**
+     * @see #checkBothPropertiesInMemory(ClusterState, boolean).
+     */
+    @Test
+    public void testInactiveActiveOnStartInMemory() throws Exception {
+        checkBothPropertiesInMemory(INACTIVE, true);
+    }
+
+    /**
+     * @see #checkBothPropertiesInMemory(ClusterState, boolean).
+     */
+    @Test
+    public void testActiveInactiveOnStartInMemory() throws Exception {
+        checkBothPropertiesInMemory(ACTIVE, false);
+    }
+
+    /**
+     * @see #checkBothPropertiesInMemory(ClusterState, boolean).
+     */
+    @Test
+    public void testActiveActiveOnStartInMemory() throws Exception {
+        checkBothPropertiesInMemory(ACTIVE, true);
+    }
+
+    /**
+     * @see #checkBothPropertiesInMemory(ClusterState, boolean).
+     */
+    @Test
+    public void testReadOnlyInactiveOnStartInMemory() throws Exception {
+        checkBothPropertiesInMemory(ACTIVE_READ_ONLY, false);
+    }
+
+    /**
+     * @see #checkBothPropertiesInMemory(ClusterState, boolean).
+     */
+    @Test
+    public void testReadOnlyActiveOnStartInMemory() throws Exception {
+        checkBothPropertiesInMemory(ACTIVE_READ_ONLY, true);
+    }
+
+    // Scenarios for both (autoActivation, clusterStateOnStart) properties with persistent cluster.
+
+    /**
+     * @see #checkBothPropertiesPersistent(ClusterState, boolean).
+     */
+    @Test
+    public void testInactiveDisableAutoActivation() throws Exception {
+        checkBothPropertiesPersistent(INACTIVE, false);
+    }
+
+    /**
+     * @see #checkBothPropertiesPersistent(ClusterState, boolean).
+     */
+    @Test
+    public void testInactiveEnableAutoActivation() throws Exception {
+        checkBothPropertiesPersistent(INACTIVE, true);
+    }
+
+    /**
+     * @see #checkBothPropertiesPersistent(ClusterState, boolean).
+     */
+    @Test
+    public void testActiveDisableAutoActivation() throws Exception {
+        checkBothPropertiesPersistent(ACTIVE, false);
+    }
+
+    /**
+     * @see #checkBothPropertiesPersistent(ClusterState, boolean).
+     */
+    @Test
+    public void testActiveEnableAutoActivation() throws Exception {
+        checkBothPropertiesPersistent(ACTIVE, true);
+    }
+
+    /**
+     * @see #checkBothPropertiesPersistent(ClusterState, boolean).
+     */
+    @Test
+    public void testReadOnlyDisableAutoActivation() throws Exception {
+        checkBothPropertiesPersistent(ACTIVE_READ_ONLY, false);
+    }
+
+    /**
+     * @see #checkBothPropertiesPersistent(ClusterState, boolean).
+     */
+    @Test
+    public void testReadOnlyEnableAutoActivation() throws Exception {
+        checkBothPropertiesPersistent(ACTIVE_READ_ONLY, true);
+    }
+
+    /**
+     * Checks that in memory cluster works correctly with given {@code state} value of
+     * {@link IgniteConfiguration#clusterStateOnStart} property and {@code activeOnStart} property together. Only
+     * {@code state} property must be applied.
+     *
+     * @param state Given value of {@link IgniteConfiguration#clusterStateOnStart} property.
+     * @param activeOnStart Given value of {@link IgniteConfiguration#activeOnStart} property.
+     * @throws Exception If failed.
+     */
+    private void checkBothPropertiesInMemory(ClusterState state, boolean activeOnStart) throws Exception {
+        this.state = state;
+        this.activeOnStart = activeOnStart;
+
+        startGrids(NODES_CNT);
+
+        for (int i=0; i < NODES_CNT; i++) {
+            checkNodeConfig(grid(i).configuration(), false, state, null, activeOnStart);
+
+            assertEquals(state, grid(i).cluster().state());
+        }
+
+        for (String name : logListeners.keySet())
+            assertTrue(name, logListeners.get(name).check());
+    }
+
+    /**
+     * Checks that persistent cluster works correctly with given {@code state} value of
+     * {@link IgniteConfiguration#clusterStateOnStart} property and {@code autoActivation} property together. Only
+     * {@code state} property must be applied.
+     *
+     * @param state Given value of {@link IgniteConfiguration#clusterStateOnStart} property.
+     * @param autoActivation Given value of {@link IgniteConfiguration#autoActivation} property.
+     * @throws Exception If failed.
+     */
+    private void checkBothPropertiesPersistent(ClusterState state, boolean autoActivation) throws Exception {
+        persistence = true;
+
+        this.state = state;
+        this.autoActivation = autoActivation;
+
+        IgniteEx crd = startGrids(NODES_CNT);
+
+        crd.cluster().state(ACTIVE);
+
+        for (int i=0; i < NODES_CNT; i++) {
+            checkNodeConfig(grid(i).configuration(), true, state, autoActivation, null);
+
+            assertEquals(ACTIVE, grid(i).cluster().state());
+        }
+
+        for (String name : logListeners.keySet())
+            assertTrue(name, logListeners.get(name).check());
+
+        stopAllGrids();
+
+        startGrids(NODES_CNT);
+
+        for (int i=0; i < NODES_CNT; i++)
+            assertEquals(state, grid(i).cluster().state());
+    }
+
+    /**
+     * Checks that in memory cluster works correctly with given {@code state} value of
+     * {@link IgniteConfiguration#clusterStateOnStart} property.
+     *
+     * @param state Given value.
+     * @throws Exception If failed.
+     */
+    private void checkPropertyInMemory(ClusterState state) throws Exception {
+        this.state = state;
+
+        startGrids(NODES_CNT);
+
+        for (int i=0; i < NODES_CNT; i++) {
+            checkNodeConfig(grid(i).configuration(), false, state, null, null);
+
+            assertEquals(state, grid(i).cluster().state());
+        }
+    }
+
+    /**
+     * Checks that persistent cluster works correctly with given {@code state} value of
+     * {@link IgniteConfiguration#clusterStateOnStart} property.
+     *
+     * @param state Given value.
+     * @throws Exception If failed.
+     */
+    private void checkPropertyPersistence(ClusterState state) throws Exception {
+        persistence = true;
+
+        this.state = state;
+
+        IgniteEx crd = startGrids(NODES_CNT);
+
+        crd.cluster().state(ACTIVE);
+
+        for (int i=0; i < NODES_CNT; i++) {
+            checkNodeConfig(grid(i).configuration(), true, state, null, null);
+
+            assertEquals(ACTIVE, grid(i).cluster().state());
+        }
+
+        stopAllGrids();
+
+        startGrids(NODES_CNT);
+
+        for (int i=0; i < NODES_CNT; i++)
+            assertEquals(state, grid(i).cluster().state());
+    }
+
+    /** */
+    private static void checkNodeConfig(
+        IgniteConfiguration cfg,
+        boolean persistenceEnabled,
+        ClusterState state,
+        @Nullable Boolean autoActivation,
+        @Nullable Boolean activeOnStart
+    ) {
+        assertEquals(persistenceEnabled, CU.isPersistenceEnabled(cfg));
+        assertEquals(state, cfg.getClusterStateOnStart());
+        assertEquals(autoActivation == null ? DFLT_AUTO_ACTIVATION : autoActivation, cfg.isAutoActivationEnabled());
+        assertEquals(activeOnStart == null ? DFLT_ACTIVE_ON_START : activeOnStart, cfg.isActiveOnStart());
+    }
+}
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/ClusterStatePartitionedSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/ClusterStatePartitionedSelfTest.java
index 4d8b188..03eefad 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/ClusterStatePartitionedSelfTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/ClusterStatePartitionedSelfTest.java
@@ -18,24 +18,14 @@
 
 package org.apache.ignite.internal.processors.cache;
 
-import org.apache.ignite.cache.CacheAtomicityMode;
-import org.apache.ignite.cache.CacheMode;
-import org.apache.ignite.cache.CacheWriteSynchronizationMode;
 import org.apache.ignite.configuration.CacheConfiguration;
 
 /**
  *
  */
-public class ClusterStatePartitionedSelfTest extends ClusterStateAbstractTest {
+public class ClusterStatePartitionedSelfTest extends ClusterStateServerAbstractTest {
     /** {@inheritDoc} */
     @Override protected CacheConfiguration cacheConfiguration(String cacheName) {
-        CacheConfiguration ccfg = new CacheConfiguration(cacheName);
-
-        ccfg.setAtomicityMode(CacheAtomicityMode.TRANSACTIONAL);
-        ccfg.setCacheMode(CacheMode.PARTITIONED);
-        ccfg.setWriteSynchronizationMode(CacheWriteSynchronizationMode.FULL_SYNC);
-        ccfg.setBackups(1);
-
-        return ccfg;
+        return ClusterStateTestUtils.partitionedCache(cacheName);
     }
 }
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/ClusterStateReplicatedSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/ClusterStateReplicatedSelfTest.java
index 23a608f..5bae574 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/ClusterStateReplicatedSelfTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/ClusterStateReplicatedSelfTest.java
@@ -18,25 +18,14 @@
 
 package org.apache.ignite.internal.processors.cache;
 
-import org.apache.ignite.cache.CacheAtomicityMode;
-import org.apache.ignite.cache.CacheMode;
-import org.apache.ignite.cache.CacheRebalanceMode;
-import org.apache.ignite.cache.CacheWriteSynchronizationMode;
 import org.apache.ignite.configuration.CacheConfiguration;
 
 /**
  *
  */
-public class ClusterStateReplicatedSelfTest extends ClusterStateAbstractTest {
+public class ClusterStateReplicatedSelfTest extends ClusterStateServerAbstractTest {
     /** {@inheritDoc} */
     @Override protected CacheConfiguration cacheConfiguration(String cacheName) {
-        CacheConfiguration ccfg = new CacheConfiguration(cacheName);
-
-        ccfg.setAtomicityMode(CacheAtomicityMode.TRANSACTIONAL);
-        ccfg.setCacheMode(CacheMode.REPLICATED);
-        ccfg.setWriteSynchronizationMode(CacheWriteSynchronizationMode.FULL_SYNC);
-        ccfg.setRebalanceMode(CacheRebalanceMode.SYNC);
-
-        return ccfg;
+        return ClusterStateTestUtils.replicatedCache(cacheName);
     }
 }
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/ClusterStateServerAbstractTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/ClusterStateServerAbstractTest.java
new file mode 100644
index 0000000..5879a2b
--- /dev/null
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/ClusterStateServerAbstractTest.java
@@ -0,0 +1,206 @@
+/*
+ * 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.cache;
+
+import java.util.concurrent.Callable;
+import java.util.concurrent.locks.Lock;
+import org.apache.ignite.IgniteCache;
+import org.apache.ignite.IgniteException;
+import org.apache.ignite.cluster.ClusterState;
+import org.apache.ignite.configuration.CacheConfiguration;
+import org.apache.ignite.transactions.Transaction;
+import org.apache.ignite.transactions.TransactionConcurrency;
+import org.apache.ignite.transactions.TransactionIsolation;
+import org.junit.Test;
+
+import static org.apache.ignite.cluster.ClusterState.ACTIVE;
+import static org.apache.ignite.cluster.ClusterState.INACTIVE;
+import static org.apache.ignite.cluster.ClusterState.ACTIVE_READ_ONLY;
+import static org.apache.ignite.internal.processors.cache.ClusterStateTestUtils.ENTRY_CNT;
+import static org.apache.ignite.testframework.GridTestUtils.assertThrowsAnyCause;
+
+/**
+ * Tests that cluster state change works correctly in different situations.
+ */
+public abstract class ClusterStateServerAbstractTest extends ClusterStateAbstractTest {
+    /** */
+    private static final String FAILED_DEACTIVATE_MSG =
+        "Failed to deactivate cluster (must invoke the method outside of an active transaction).";
+
+    /** */
+    private static final String FAILED_ACTIVATE_MSG =
+        "Failed to activate cluster (must invoke the method outside of an active transaction).";
+
+    /** */
+    private static final String FAILED_READ_ONLY_MSG =
+        "Failed to activate cluster in read-only mode (must invoke the method outside of an active transaction).";
+
+    /**
+     * Tests that deactivation is prohibited if explicit lock is held in current thread.
+     */
+    @Test
+    public void testDeactivationWithPendingLock() {
+        changeClusterStateWithPendingLock(INACTIVE, FAILED_DEACTIVATE_MSG);
+    }
+
+    /**
+     * Tests that enabling read-only mode is prohibited if explicit lock is held in current thread.
+     */
+    @Test
+    public void testReadOnlyWithPendingLock() {
+        changeClusterStateWithPendingLock(ACTIVE_READ_ONLY, FAILED_READ_ONLY_MSG);
+    }
+
+    /**
+     * Tests that change cluster mode from {@link ClusterState#ACTIVE} to {@link ClusterState#INACTIVE} is prohibited if
+     * transaction is active in current thread.
+     */
+    @Test
+    public void testDeactivationWithPendingTransaction() {
+        grid(0).cluster().state(ACTIVE);
+
+        for (TransactionConcurrency concurrency : TransactionConcurrency.values()) {
+            for (TransactionIsolation isolation : TransactionIsolation.values())
+                changeStateWithPendingTransaction(INACTIVE, concurrency, isolation, FAILED_DEACTIVATE_MSG);
+        }
+    }
+
+    /**
+     * Tests that change cluster mode from {@link ClusterState#ACTIVE_READ_ONLY} to {@link ClusterState#INACTIVE} is prohibited
+     * if transaction is active in current thread.
+     */
+    @Test
+    public void testDeactivateFromReadonlyWithPendingTransaction() {
+        grid(0).cluster().state(ACTIVE_READ_ONLY);
+
+        for (TransactionConcurrency concurrency : TransactionConcurrency.values()) {
+            for (TransactionIsolation isolation : TransactionIsolation.values())
+                changeStateWithPendingTransaction(INACTIVE, concurrency, isolation, FAILED_DEACTIVATE_MSG);
+        }
+    }
+
+    /**
+     * Tests that change cluster mode from {@link ClusterState#ACTIVE} to {@link ClusterState#ACTIVE_READ_ONLY} is prohibited
+     * if transaction is active in current thread.
+     */
+    @Test
+    public void testReadOnlyWithPendingTransaction() {
+        grid(0).cluster().state(ACTIVE);
+
+        for (TransactionConcurrency concurrency : TransactionConcurrency.values()) {
+            for (TransactionIsolation isolation : TransactionIsolation.values())
+                changeStateWithPendingTransaction(ACTIVE_READ_ONLY, concurrency, isolation, FAILED_READ_ONLY_MSG);
+        }
+    }
+
+    /**
+     * Tests that change cluster mode from {@link ClusterState#ACTIVE_READ_ONLY} to {@link ClusterState#ACTIVE} is prohibited
+     * if transaction is active in current thread.
+     */
+    @Test
+    public void testDisableReadonlyWithPendingTransaction() {
+        grid(0).cluster().state(ACTIVE_READ_ONLY);
+
+        for (TransactionConcurrency concurrency : TransactionConcurrency.values()) {
+            for (TransactionIsolation isolation : TransactionIsolation.values())
+                changeStateWithPendingTransaction(ACTIVE, concurrency, isolation, FAILED_ACTIVATE_MSG);
+        }
+    }
+
+    /** */
+    @Test
+    public void testDynamicCacheStart() {
+        grid(0).cluster().state(ACTIVE);
+
+        try {
+            IgniteCache<Object, Object> cache2 = grid(0).createCache(new CacheConfiguration<>("cache2"));
+
+            for (int k = 0; k < ENTRY_CNT; k++)
+                cache2.put(k, k);
+        }
+        finally {
+            grid(0).destroyCache("cache2");
+        }
+    }
+
+    /** {@inheritDoc} */
+    @Override protected void changeState(ClusterState state) {
+        grid(0).cluster().state(state);
+    }
+
+    /** */
+    private void changeStateWithPendingTransaction(
+        ClusterState state,
+        TransactionConcurrency concurrency,
+        TransactionIsolation isolation,
+        String exceptionMsg
+    ) {
+        final IgniteCache<Object, Object> cache0 = grid(0).cache(CACHE_NAME);
+
+        assertNotSame(state, grid(0).cluster().state());
+
+        try (Transaction ignore = grid(0).transactions().txStart(concurrency, isolation)) {
+            if (grid(0).cluster().state() != ACTIVE_READ_ONLY)
+                cache0.put(1, "1");
+
+            //noinspection ThrowableNotThrown
+            assertThrowsAnyCause(log, changeStateClo(state), IgniteException.class, exceptionMsg);
+        }
+
+        assertNotSame(state, grid(0).cluster().state());
+
+        if (grid(0).cluster().state() != ACTIVE_READ_ONLY)
+            assertNull(cache0.get(1));
+
+        assertNull(grid(0).transactions().tx());
+    }
+
+    /**
+     * Tests that cluster state change to {@code newState} is prohibited if explicit lock is held in current thread.
+     *
+     * @param newState New cluster state.
+     * @param exceptionMsg Exception message.
+     */
+    private void changeClusterStateWithPendingLock(ClusterState newState, String exceptionMsg) {
+        grid(0).cluster().state(ACTIVE);
+
+        Lock lock = grid(0).cache(CACHE_NAME).lock(1);
+
+        lock.lock();
+
+        try {
+            //noinspection ThrowableNotThrown
+            assertThrowsAnyCause(log, changeStateClo(newState), IgniteException.class, exceptionMsg);
+        }
+        finally {
+            lock.unlock();
+        }
+    }
+
+    /**
+     * @param state New cluster state.
+     * @return Callable which tries to change cluster state to {@code state} from {@code ignite} node.
+     */
+    Callable<Object> changeStateClo(ClusterState state) {
+        return () -> {
+            grid(0).cluster().state(state);
+
+            return null;
+        };
+    }
+}
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/ClusterStateTestUtils.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/ClusterStateTestUtils.java
new file mode 100644
index 0000000..6a3fe1b
--- /dev/null
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/ClusterStateTestUtils.java
@@ -0,0 +1,124 @@
+/*
+ * 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.cache;
+
+import java.util.List;
+import org.apache.ignite.Ignite;
+import org.apache.ignite.IgniteException;
+import org.apache.ignite.IgniteLogger;
+import org.apache.ignite.cache.CacheAtomicityMode;
+import org.apache.ignite.cache.CacheMode;
+import org.apache.ignite.cache.CacheRebalanceMode;
+import org.apache.ignite.cache.CacheWriteSynchronizationMode;
+import org.apache.ignite.configuration.CacheConfiguration;
+import org.apache.ignite.internal.IgniteEx;
+import org.apache.ignite.internal.processors.cache.distributed.dht.IgniteClusterReadOnlyException;
+import org.apache.ignite.internal.util.typedef.F;
+
+import static org.apache.ignite.testframework.GridTestUtils.assertThrows;
+import static org.apache.ignite.testframework.GridTestUtils.assertThrowsWithCause;
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertFalse;
+import static org.junit.Assert.assertNotNull;
+import static org.junit.Assert.assertNull;
+
+/**
+ * Utility class for ClusterState* tests.
+ */
+public class ClusterStateTestUtils {
+    /** */
+    public static final int ENTRY_CNT = 50;
+
+    /**
+     * @param cacheName Cache name.
+     * @return Partitioned cache configuration with 1 backup.
+     */
+    public static CacheConfiguration partitionedCache(String cacheName) {
+        CacheConfiguration ccfg = new CacheConfiguration(cacheName);
+
+        ccfg.setAtomicityMode(CacheAtomicityMode.TRANSACTIONAL);
+        ccfg.setCacheMode(CacheMode.PARTITIONED);
+        ccfg.setWriteSynchronizationMode(CacheWriteSynchronizationMode.FULL_SYNC);
+        ccfg.setBackups(1);
+
+        return ccfg;
+    }
+
+    /**
+     * @param cacheName Cache name.
+     * @return Replicated cache configuration.
+     */
+    public static CacheConfiguration replicatedCache(String cacheName) {
+        CacheConfiguration ccfg = new CacheConfiguration(cacheName);
+
+        ccfg.setAtomicityMode(CacheAtomicityMode.TRANSACTIONAL);
+        ccfg.setCacheMode(CacheMode.REPLICATED);
+        ccfg.setWriteSynchronizationMode(CacheWriteSynchronizationMode.FULL_SYNC);
+        ccfg.setRebalanceMode(CacheRebalanceMode.SYNC);
+
+        return ccfg;
+    }
+
+    /** */
+    public static void putSomeDataAndCheck(IgniteLogger log, List<IgniteEx> nodes, String... cacheNames) {
+        assertFalse(F.isEmpty(nodes));
+        assertFalse(F.isEmpty(cacheNames));
+
+        IgniteEx crd = nodes.get(0);
+
+        for (String cacheName : cacheNames) {
+            switch (crd.cluster().state()) {
+                case INACTIVE:
+                    assertNotNull(assertThrows(
+                        log,
+                        () -> crd.cache(cacheName),
+                        IgniteException.class,
+                        "Can not perform the operation because the cluster is inactive. Note, that the cluster is considered inactive by default if Ignite Persistent Store is used to let all the nodes join the cluster. To activate the cluster call Ignite.active(true)."
+                    ));
+
+                    break;
+
+                case ACTIVE:
+                    for (int k = 0; k < ENTRY_CNT; k++)
+                        crd.cache(cacheName).put(k, k);
+
+                    for (Ignite node : nodes) {
+                        for (int k = 0; k < ENTRY_CNT; k++)
+                            assertEquals(k, node.cache(cacheName).get(k));
+                    }
+
+                    break;
+
+                case ACTIVE_READ_ONLY:
+                    assertNotNull(assertThrowsWithCause(
+                        () -> crd.cache(cacheName).put(0, 0),
+                        IgniteClusterReadOnlyException.class
+                    ));
+
+                    for (Ignite node : nodes)
+                        assertNull(node.cache(cacheName).get(0));
+
+                    break;
+            }
+        }
+    }
+
+    /** */
+    private ClusterStateTestUtils() {
+    }
+}
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/ClusterStateThinClientAbstractTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/ClusterStateThinClientAbstractTest.java
new file mode 100644
index 0000000..56e4ab2
--- /dev/null
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/ClusterStateThinClientAbstractTest.java
@@ -0,0 +1,80 @@
+/*
+ * 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.cache;
+
+import java.util.Collections;
+import org.apache.ignite.cluster.ClusterState;
+import org.apache.ignite.configuration.ConnectorConfiguration;
+import org.apache.ignite.configuration.IgniteConfiguration;
+import org.apache.ignite.internal.client.GridClient;
+import org.apache.ignite.internal.client.GridClientConfiguration;
+import org.apache.ignite.internal.client.GridClientException;
+import org.apache.ignite.internal.client.GridClientFactory;
+import org.apache.ignite.internal.client.GridClientProtocol;
+
+import static org.apache.ignite.configuration.ConnectorConfiguration.DFLT_TCP_PORT;
+
+/**
+ * Tests that cluster state change works correctly with connected thin client in different situations.
+ */
+public abstract class ClusterStateThinClientAbstractTest extends ClusterStateAbstractTest {
+    /** */
+    private static final String HOST = "127.0.0.1";
+
+    /** */
+    private static GridClient gridClient;
+
+    /** */
+    private int port = DFLT_TCP_PORT;
+
+    /** {@inheritDoc} */
+    @Override protected IgniteConfiguration getConfiguration(String gridName) throws Exception {
+        return super.getConfiguration(gridName)
+            .setConnectorConfiguration(new ConnectorConfiguration().setPort(port++).setHost(HOST));
+    }
+
+    /** {@inheritDoc} */
+    @Override protected void beforeTestsStarted() throws Exception {
+        super.beforeTestsStarted();
+
+        GridClientConfiguration cfg = new GridClientConfiguration();
+
+        cfg.setProtocol(GridClientProtocol.TCP);
+        cfg.setServers(Collections.singletonList(HOST + ":" + DFLT_TCP_PORT));
+
+        gridClient = GridClientFactory.start(cfg);
+    }
+
+    /** {@inheritDoc} */
+    @Override protected void afterTestsStopped() throws Exception {
+        if (gridClient != null)
+            gridClient.close();
+
+        super.afterTestsStopped();
+    }
+
+    /** {@inheritDoc} */
+    @Override protected void changeState(ClusterState state) {
+        try {
+            gridClient.state().state(state);
+        }
+        catch (GridClientException e) {
+            throw new RuntimeException("Can't change state to " + state, e);
+        }
+    }
+}
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/ClusterStatePartitionedSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/ClusterStateThinClientPartitionedSelfTest.java
similarity index 53%
copy from modules/core/src/test/java/org/apache/ignite/internal/processors/cache/ClusterStatePartitionedSelfTest.java
copy to modules/core/src/test/java/org/apache/ignite/internal/processors/cache/ClusterStateThinClientPartitionedSelfTest.java
index 4d8b188..6eab02a 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/ClusterStatePartitionedSelfTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/ClusterStateThinClientPartitionedSelfTest.java
@@ -1,41 +1,30 @@
 /*
  * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
+ * 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
+ * the License. You may obtain a copy of the License at
  *
- *      http://www.apache.org/licenses/LICENSE-2.0
+ * 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.cache;
 
-import org.apache.ignite.cache.CacheAtomicityMode;
-import org.apache.ignite.cache.CacheMode;
-import org.apache.ignite.cache.CacheWriteSynchronizationMode;
 import org.apache.ignite.configuration.CacheConfiguration;
 
 /**
  *
  */
-public class ClusterStatePartitionedSelfTest extends ClusterStateAbstractTest {
+public class ClusterStateThinClientPartitionedSelfTest extends ClusterStateThinClientAbstractTest {
     /** {@inheritDoc} */
     @Override protected CacheConfiguration cacheConfiguration(String cacheName) {
-        CacheConfiguration ccfg = new CacheConfiguration(cacheName);
-
-        ccfg.setAtomicityMode(CacheAtomicityMode.TRANSACTIONAL);
-        ccfg.setCacheMode(CacheMode.PARTITIONED);
-        ccfg.setWriteSynchronizationMode(CacheWriteSynchronizationMode.FULL_SYNC);
-        ccfg.setBackups(1);
-
-        return ccfg;
+        return ClusterStateTestUtils.partitionedCache(cacheName);
     }
 }
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/ClusterStatePartitionedSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/ClusterStateThinClientReplicatedSelfTest.java
similarity index 54%
copy from modules/core/src/test/java/org/apache/ignite/internal/processors/cache/ClusterStatePartitionedSelfTest.java
copy to modules/core/src/test/java/org/apache/ignite/internal/processors/cache/ClusterStateThinClientReplicatedSelfTest.java
index 4d8b188..9e92dcc 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/ClusterStatePartitionedSelfTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/ClusterStateThinClientReplicatedSelfTest.java
@@ -1,41 +1,30 @@
 /*
  * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
+ * 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
+ * the License. You may obtain a copy of the License at
  *
- *      http://www.apache.org/licenses/LICENSE-2.0
+ * 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.cache;
 
-import org.apache.ignite.cache.CacheAtomicityMode;
-import org.apache.ignite.cache.CacheMode;
-import org.apache.ignite.cache.CacheWriteSynchronizationMode;
 import org.apache.ignite.configuration.CacheConfiguration;
 
 /**
  *
  */
-public class ClusterStatePartitionedSelfTest extends ClusterStateAbstractTest {
+public class ClusterStateThinClientReplicatedSelfTest extends ClusterStateThinClientAbstractTest {
     /** {@inheritDoc} */
     @Override protected CacheConfiguration cacheConfiguration(String cacheName) {
-        CacheConfiguration ccfg = new CacheConfiguration(cacheName);
-
-        ccfg.setAtomicityMode(CacheAtomicityMode.TRANSACTIONAL);
-        ccfg.setCacheMode(CacheMode.PARTITIONED);
-        ccfg.setWriteSynchronizationMode(CacheWriteSynchronizationMode.FULL_SYNC);
-        ccfg.setBackups(1);
-
-        return ccfg;
+        return ClusterStateTestUtils.replicatedCache(cacheName);
     }
 }
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/IgniteClusterActivateDeactivateTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/IgniteClusterActivateDeactivateTest.java
index 2346047..05b7ddf 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/IgniteClusterActivateDeactivateTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/IgniteClusterActivateDeactivateTest.java
@@ -18,6 +18,7 @@
 package org.apache.ignite.internal.processors.cache;
 
 import java.util.ArrayList;
+import java.util.Arrays;
 import java.util.List;
 import java.util.concurrent.Callable;
 import java.util.concurrent.CyclicBarrier;
@@ -29,6 +30,7 @@ import org.apache.ignite.IgniteCache;
 import org.apache.ignite.IgniteCheckedException;
 import org.apache.ignite.cache.CacheAtomicityMode;
 import org.apache.ignite.cluster.ClusterNode;
+import org.apache.ignite.cluster.ClusterState;
 import org.apache.ignite.configuration.CacheConfiguration;
 import org.apache.ignite.configuration.DataRegionConfiguration;
 import org.apache.ignite.configuration.DataStorageConfiguration;
@@ -37,9 +39,11 @@ import org.apache.ignite.configuration.WALMode;
 import org.apache.ignite.internal.IgniteClientReconnectAbstractTest;
 import org.apache.ignite.internal.IgniteEx;
 import org.apache.ignite.internal.IgniteInternalFuture;
+import org.apache.ignite.internal.IgniteInterruptedCheckedException;
 import org.apache.ignite.internal.TestRecordingCommunicationSpi;
 import org.apache.ignite.internal.managers.discovery.IgniteDiscoverySpi;
 import org.apache.ignite.internal.processors.affinity.AffinityTopologyVersion;
+import org.apache.ignite.internal.processors.cache.distributed.dht.IgniteClusterReadOnlyException;
 import org.apache.ignite.internal.processors.cache.distributed.dht.preloader.GridDhtPartitionsFullMessage;
 import org.apache.ignite.internal.processors.cache.distributed.dht.preloader.GridDhtPartitionsSingleMessage;
 import org.apache.ignite.internal.processors.cluster.DiscoveryDataClusterState;
@@ -52,12 +56,20 @@ import org.apache.ignite.plugin.extensions.communication.Message;
 import org.apache.ignite.spi.discovery.tcp.TcpDiscoverySpi;
 import org.apache.ignite.testframework.GridTestUtils;
 import org.apache.ignite.testframework.junits.common.GridCommonAbstractTest;
-import org.junit.Assert;
 import org.junit.Test;
 
 import static org.apache.ignite.cache.CacheAtomicityMode.ATOMIC;
 import static org.apache.ignite.cache.CacheAtomicityMode.TRANSACTIONAL;
 import static org.apache.ignite.cache.CacheWriteSynchronizationMode.FULL_SYNC;
+import static org.apache.ignite.cluster.ClusterState.ACTIVE;
+import static org.apache.ignite.cluster.ClusterState.ACTIVE_READ_ONLY;
+import static org.apache.ignite.cluster.ClusterState.INACTIVE;
+import static org.apache.ignite.cluster.ClusterState.lesserOf;
+import static org.apache.ignite.testframework.GridTestUtils.assertActive;
+import static org.apache.ignite.testframework.GridTestUtils.assertInactive;
+import static org.apache.ignite.testframework.GridTestUtils.assertThrowsWithCause;
+import static org.apache.ignite.testframework.GridTestUtils.runAsync;
+import static org.apache.ignite.testframework.GridTestUtils.waitForCondition;
 
 /**
  *
@@ -76,7 +88,7 @@ public class IgniteClusterActivateDeactivateTest extends GridCommonAbstractTest
     boolean client;
 
     /** */
-    private boolean active = true;
+    private ClusterState stateOnStart;
 
     /** */
     CacheConfiguration[] ccfgs;
@@ -108,7 +120,8 @@ public class IgniteClusterActivateDeactivateTest extends GridCommonAbstractTest
 
         cfg.setClientMode(client);
 
-        cfg.setActiveOnStart(active);
+        if (stateOnStart != null)
+            cfg.setClusterStateOnStart(stateOnStart);
 
         if (ccfgs != null) {
             cfg.setCacheConfiguration(ccfgs);
@@ -164,99 +177,209 @@ public class IgniteClusterActivateDeactivateTest extends GridCommonAbstractTest
      * @throws Exception If failed.
      */
     @Test
-    public void testActivateSimple_SingleNode() throws Exception {
-        activateSimple(1, 0, 0);
+    public void testEnableReadOnlyFromActivateSimple_SingleNode() throws Exception {
+        changeActiveClusterStateSimple(1, 0, 0, ACTIVE, ACTIVE_READ_ONLY);
     }
 
     /**
      * @throws Exception If failed.
      */
     @Test
-    public void testActivateSimple_5_Servers() throws Exception {
-        activateSimple(5, 0, 0);
+    public void testEnableReadOnlyFromActivateSimple_5_Servers() throws Exception {
+        changeActiveClusterStateSimple(5, 0, 0, ACTIVE, ACTIVE_READ_ONLY);
     }
 
     /**
      * @throws Exception If failed.
      */
     @Test
-    public void testActivateSimple_5_Servers2() throws Exception {
-        activateSimple(5, 0, 4);
+    public void testEnableReadOnlyFromActivateSimple_5_Servers2() throws Exception {
+        changeActiveClusterStateSimple(5, 0, 4, ACTIVE, ACTIVE_READ_ONLY);
     }
 
     /**
      * @throws Exception If failed.
      */
     @Test
-    public void testActivateSimple_5_Servers_5_Clients() throws Exception {
-        activateSimple(5, 4, 0);
+    public void testEnableReadOnlyFromActivateSimple_5_Servers_5_Clients() throws Exception {
+        changeActiveClusterStateSimple(5, 4, 0, ACTIVE, ACTIVE_READ_ONLY);
     }
 
     /**
      * @throws Exception If failed.
      */
     @Test
-    public void testActivateSimple_5_Servers_5_Clients_FromClient() throws Exception {
-        activateSimple(5, 4, 6);
+    public void testEnableReadOnlyFromActivateSimple_5_Servers_5_Clients_FromClient() throws Exception {
+        changeActiveClusterStateSimple(5, 4, 6, ACTIVE, ACTIVE_READ_ONLY);
     }
 
     /**
-     * @param srvs Number of servers.
-     * @param clients Number of clients.
-     * @param activateFrom Index of node stating activation.
      * @throws Exception If failed.
      */
-    private void activateSimple(int srvs, int clients, int activateFrom) throws Exception {
-        active = false;
+    @Test
+    public void testDisableReadOnlyFromActivateSimple_SingleNode() throws Exception {
+        changeActiveClusterStateSimple(1, 0, 0, ACTIVE_READ_ONLY, ACTIVE);
+    }
 
-        final int CACHES = 2;
+    /**
+     * @throws Exception If failed.
+     */
+    @Test
+    public void testDisableReadOnlyFromActivateSimple_5_Servers() throws Exception {
+        changeActiveClusterStateSimple(5, 0, 0, ACTIVE_READ_ONLY, ACTIVE);
+    }
 
-        for (int i = 0; i < srvs + clients; i++) {
-            client = i >= srvs;
+    /**
+     * @throws Exception If failed.
+     */
+    @Test
+    public void testDisableReadOnlyFromActivateSimple_5_Servers2() throws Exception {
+        changeActiveClusterStateSimple(5, 0, 4, ACTIVE_READ_ONLY, ACTIVE);
+    }
 
-            ccfgs = cacheConfigurations1();
+    /**
+     * @throws Exception If failed.
+     */
+    @Test
+    public void testDisableReadOnlyFromActivateSimple_5_Servers_5_Clients() throws Exception {
+        changeActiveClusterStateSimple(5, 4, 0, ACTIVE_READ_ONLY, ACTIVE);
+    }
 
-            startGrid(i);
+    /**
+     * @throws Exception If failed.
+     */
+    @Test
+    public void testDisableReadOnlyFromActivateSimple_5_Servers_5_Clients_FromClient() throws Exception {
+        changeActiveClusterStateSimple(5, 4, 6, ACTIVE_READ_ONLY, ACTIVE);
+    }
 
-            checkNoCaches(i);
-        }
+    /**
+     * @throws Exception If failed.
+     */
+    @Test
+    public void testActivateSimple_SingleNode() throws Exception {
+        activateSimple(1, 0, 0, ACTIVE);
+    }
 
-        for (int i = 0; i < srvs + clients; i++)
-            assertFalse(ignite(i).cluster().active());
+    /**
+     * @throws Exception If failed.
+     */
+    @Test
+    public void testActivateInReadOnlySimple_SingleNode() throws Exception {
+        activateSimple(1, 0, 0, ACTIVE_READ_ONLY);
+    }
 
-        ignite(activateFrom).cluster().active(false); // Should be no-op.
+    /**
+     * @throws Exception If failed.
+     */
+    @Test
+    public void testActivateSimple_5_Servers() throws Exception {
+        activateSimple(5, 0, 0, ACTIVE);
+    }
 
-        ignite(activateFrom).cluster().active(true);
+    /**
+     * @throws Exception If failed.
+     */
+    @Test
+    public void testActivateInReadOnlySimple_5_Servers() throws Exception {
+        activateSimple(5, 0, 0, ACTIVE_READ_ONLY);
+    }
 
-        for (int i = 0; i < srvs + clients; i++)
-            assertTrue(ignite(i).cluster().active());
+    /**
+     * @throws Exception If failed.
+     */
+    @Test
+    public void testActivateSimple_5_Servers2() throws Exception {
+        activateSimple(5, 0, 4, ACTIVE);
+    }
 
-        for (int i = 0; i < srvs + clients; i++) {
-            for (int c = 0; c < DEFAULT_CACHES_COUNT; c++)
-                checkCache(ignite(i), CACHE_NAME_PREFIX + c, true);
+    /**
+     * @throws Exception If failed.
+     */
+    @Test
+    public void testActivateInReadOnlySimple_5_Servers2() throws Exception {
+        activateSimple(5, 0, 4, ACTIVE_READ_ONLY);
+    }
 
-            checkCache(ignite(i), CU.UTILITY_CACHE_NAME, true);
-        }
+    /**
+     * @throws Exception If failed.
+     */
+    @Test
+    public void testActivateSimple_5_Servers_5_Clients() throws Exception {
+        activateSimple(5, 4, 0, ACTIVE);
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    @Test
+    public void testActivateInReadOnlySimple_5_Servers_5_Clients() throws Exception {
+        activateSimple(5, 4, 0, ACTIVE_READ_ONLY);
+    }
 
-        checkCaches(srvs + clients, CACHES);
+    /**
+     * @throws Exception If failed.
+     */
+    @Test
+    public void testActivateSimple_5_Servers_5_Clients_FromClient() throws Exception {
+        activateSimple(5, 4, 6, ACTIVE);
+    }
 
-        client = false;
+    /**
+     * @throws Exception If failed.
+     */
+    @Test
+    public void testActivateInReadOnlySimple_5_Servers_5_Clients_FromClient() throws Exception {
+        activateSimple(5, 4, 6, ACTIVE_READ_ONLY);
+    }
 
-        startGrid(srvs + clients);
+    /**
+     * @param srvs Number of servers.
+     * @param clients Number of clients.
+     * @param changeFrom Index of node stating activation.
+     * @param state Activation state.
+     * @throws Exception If failed.
+     */
+    private void activateSimple(int srvs, int clients, int changeFrom, ClusterState state) throws Exception {
+        assertActive(state);
 
-        for (int c = 0; c < DEFAULT_CACHES_COUNT; c++)
-            checkCache(ignite(srvs + clients), CACHE_NAME_PREFIX + c, true);
+        changeStateSimple(srvs, clients, changeFrom, INACTIVE, state);
+    }
 
-        checkCaches(srvs + clients + 1, CACHES);
+    /**
+     * @param srvs Number of servers.
+     * @param clients Number of clients.
+     * @param changeFrom Index of node stating deactivation.
+     * @param initialState Initial cluster state.
+     * @throws Exception If failed.
+     */
+    private void deactivateSimple(int srvs, int clients, int changeFrom, ClusterState initialState) throws Exception {
+        assertActive(initialState);
 
-        client = true;
+        changeStateSimple(srvs, clients, changeFrom, initialState, INACTIVE);
+    }
 
-        startGrid(srvs + clients + 1);
+    /**
+     * @param srvs Number of servers.
+     * @param clients Number of clients.
+     * @param deactivateFrom Index of node stating deactivation.
+     * @param initialState Initial cluster state.
+     * @param targetState Targer cluster state.
+     * @throws Exception If failed.
+     */
+    private void changeActiveClusterStateSimple(
+        int srvs,
+        int clients,
+        int deactivateFrom,
+        ClusterState initialState,
+        ClusterState targetState
+    ) throws Exception {
+        assertActive(initialState);
+        assertActive(targetState);
 
-        for (int c = 0; c < DEFAULT_CACHES_COUNT; c++)
-            checkCache(ignite(srvs + clients + 1), CACHE_NAME_PREFIX + c, false);
+        assertNotSame(initialState, targetState);
 
-        checkCaches(srvs + clients + 2, CACHES);
+        changeStateSimple(srvs, clients, deactivateFrom, initialState, targetState);
     }
 
     /**
@@ -264,7 +387,15 @@ public class IgniteClusterActivateDeactivateTest extends GridCommonAbstractTest
      */
     @Test
     public void testReActivateSimple_5_Servers_4_Clients_FromClient() throws Exception {
-        reactivateSimple(5, 4, 6);
+        reactivateSimple(5, 4, 6, ACTIVE);
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    @Test
+    public void testReActivateInReadOnlySimple_5_Servers_4_Clients_FromClient() throws Exception {
+        reactivateSimple(5, 4, 6, ACTIVE_READ_ONLY);
     }
 
     /**
@@ -272,39 +403,41 @@ public class IgniteClusterActivateDeactivateTest extends GridCommonAbstractTest
      */
     @Test
     public void testReActivateSimple_5_Servers_4_Clients_FromServer() throws Exception {
-        reactivateSimple(5, 4, 0);
+        reactivateSimple(5, 4, 0, ACTIVE);
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    @Test
+    public void testReActivateInReadOnlySimple_5_Servers_4_Clients_FromServer() throws Exception {
+        reactivateSimple(5, 4, 0, ACTIVE_READ_ONLY);
     }
 
     /**
      * @param srvs Number of servers.
      * @param clients Number of clients.
      * @param activateFrom Index of node stating activation.
+     * @param state Activation state.
      * @throws Exception If failed.
      */
-    public void reactivateSimple(int srvs, int clients, int activateFrom) throws Exception {
-        activateSimple(srvs, clients, activateFrom);
+    private void reactivateSimple(int srvs, int clients, int activateFrom, ClusterState state) throws Exception {
+        activateSimple(srvs, clients, activateFrom, state);
 
-        rolloverSegmentAtLeastTwice(activateFrom);
+        if (state == ACTIVE)
+            rolloverSegmentAtLeastTwice(activateFrom);
 
-        for (int i = 0; i < srvs + clients; i++) {
-            for (int c = 0; c < DEFAULT_CACHES_COUNT; c++)
-                checkCache(ignite(i), CACHE_NAME_PREFIX + c, true);
-
-            checkCache(ignite(i), CU.UTILITY_CACHE_NAME, true);
-        }
-
-        ignite(activateFrom).cluster().active(false);
-
-        ignite(activateFrom).cluster().active(true);
+        for (int i = 0; i < srvs + clients; i++)
+            checkCachesOnNode(i, DEFAULT_CACHES_COUNT);
 
-        rolloverSegmentAtLeastTwice(activateFrom);
+        ignite(activateFrom).cluster().state(INACTIVE);
+        ignite(activateFrom).cluster().state(state);
 
-        for (int i = 0; i < srvs + clients; i++) {
-            for (int c = 0; c < DEFAULT_CACHES_COUNT; c++)
-                checkCache(ignite(i), CACHE_NAME_PREFIX + c, true);
+        if (state == ACTIVE)
+            rolloverSegmentAtLeastTwice(activateFrom);
 
-            checkCache(ignite(i), CU.UTILITY_CACHE_NAME, true);
-        }
+        for (int i = 0; i < srvs + clients; i++)
+            checkCachesOnNode(i, DEFAULT_CACHES_COUNT);
     }
 
     /**
@@ -313,6 +446,7 @@ public class IgniteClusterActivateDeactivateTest extends GridCommonAbstractTest
     private void rolloverSegmentAtLeastTwice(int activateFrom) {
         for (int c = 0; c < DEFAULT_CACHES_COUNT; c++) {
             IgniteCache<Object, Object> cache = ignite(activateFrom).cache(CACHE_NAME_PREFIX + c);
+
             //this should be enough including free-,meta- page and etc.
             for (int i = 0; i < 1000; i++)
                 cache.put(i, i);
@@ -335,18 +469,24 @@ public class IgniteClusterActivateDeactivateTest extends GridCommonAbstractTest
         if (awaitExchange)
             awaitPartitionMapExchange();
 
+        ClusterState state = ignite(0).cluster().state();
+
+        assertActive(state);
+
         for (int i = 0; i < nodes; i++) {
             for (int c = 0; c < caches; c++) {
                 IgniteCache<Integer, Integer> cache = ignite(i).cache(CACHE_NAME_PREFIX + c);
 
                 for (int j = 0; j < 10; j++) {
-                    ThreadLocalRandom rnd = ThreadLocalRandom.current();
+                    Integer key = ThreadLocalRandom.current().nextInt(1000);
+                    Integer value = j;
 
-                    Integer key = rnd.nextInt(1000);
+                    if (state == ACTIVE)
+                        cache.put(key, j);
+                    else
+                        assertThrowsWithCause(() -> cache.put(key, value), IgniteClusterReadOnlyException.class);
 
-                    cache.put(key, j);
-
-                    assertEquals((Integer)j, cache.get(key));
+                    assertEquals(state == ACTIVE ? value : null, cache.get(key));
                 }
             }
         }
@@ -357,82 +497,55 @@ public class IgniteClusterActivateDeactivateTest extends GridCommonAbstractTest
      */
     @Test
     public void testJoinWhileActivate1_Server() throws Exception {
-        joinWhileActivate1(false, false);
+        joinWhileActivate1(false, false, ACTIVE);
     }
 
     /**
      * @throws Exception If failed.
      */
     @Test
-    public void testJoinWhileActivate1_WithCache_Server() throws Exception {
-        joinWhileActivate1(false, true);
+    public void testJoinWhileActivateInReadOnly1_Server() throws Exception {
+        joinWhileActivate1(false, false, ACTIVE_READ_ONLY);
     }
 
     /**
      * @throws Exception If failed.
      */
     @Test
-    public void testJoinWhileActivate1_Client() throws Exception {
-        joinWhileActivate1(true, false);
+    public void testJoinWhileActivate1_WithCache_Server() throws Exception {
+        joinWhileActivate1(false, true, ACTIVE);
     }
 
     /**
-     * @param startClient If {@code true} joins client node, otherwise server.
-     * @param withNewCache If {@code true} joining node has new cache in configuration.
      * @throws Exception If failed.
      */
-    private void joinWhileActivate1(final boolean startClient, final boolean withNewCache) throws Exception {
-        IgniteInternalFuture<?> activeFut = startNodesAndBlockStatusChange(2, 0, 0, false);
-
-        IgniteInternalFuture<?> startFut = GridTestUtils.runAsync((Callable<Void>)() -> {
-            client = startClient;
-
-            ccfgs = withNewCache ? cacheConfigurations2() : cacheConfigurations1();
-
-            startGrid(2);
-
-            return null;
-        });
-
-        TestRecordingCommunicationSpi spi1 = TestRecordingCommunicationSpi.spi(ignite(1));
-
-        spi1.stopBlock();
-
-        activeFut.get();
-        startFut.get();
-
-        for (int c = 0; c < DEFAULT_CACHES_COUNT; c++)
-            checkCache(ignite(2), CACHE_NAME_PREFIX + c, true);
-
-        if (withNewCache) {
-            for (int i = 0; i < 3; i++) {
-                for (int c = 0; c < 4; c++)
-                    checkCache(ignite(i), CACHE_NAME_PREFIX + c, true);
-            }
-        }
-
-        awaitPartitionMapExchange();
-
-        checkCaches(3, withNewCache ? 4 : 2);
-
-        client = false;
-
-        startGrid(3);
-
-        checkCaches(4, withNewCache ? 4 : 2);
-
-        client = true;
+    @Test
+    public void testJoinWhileActivateInReadOnly1_WithCache_Server() throws Exception {
+        joinWhileActivate1(false, true, ACTIVE_READ_ONLY);
+    }
 
-        startGrid(4);
+    /**
+     * @throws Exception If failed.
+     */
+    @Test
+    public void testJoinWhileActivate1_Client() throws Exception {
+        joinWhileActivate1(true, false, ACTIVE);
+    }
 
-        checkCaches(5, withNewCache ? 4 : 2);
+    /**
+     * @throws Exception If failed.
+     */
+    @Test
+    public void testJoinWhileActivateInReadOnly1_Client() throws Exception {
+        joinWhileActivate1(true, false, ACTIVE_READ_ONLY);
     }
 
     /**
      * @param srvs Number of servers.
      * @param clients Number of clients.
      * @param stateChangeFrom Index of node initiating changes.
-     * @param initiallyActive If {@code true} start cluster in active state (otherwise in inactive).
+     * @param initialState Cluster state on start nodes.
+     * @param targetState State of started cluster.
      * @param blockMsgNodes Nodes whcis block exchange messages.
      * @return State change future.
      * @throws Exception If failed.
@@ -441,28 +554,35 @@ public class IgniteClusterActivateDeactivateTest extends GridCommonAbstractTest
         int srvs,
         int clients,
         final int stateChangeFrom,
-        final boolean initiallyActive,
+        final ClusterState initialState,
+        final ClusterState targetState,
         int... blockMsgNodes
     ) throws Exception {
-        active = initiallyActive;
+        assertNotSame(initialState, targetState);
+
+        if (!persistenceEnabled())
+            stateOnStart = initialState;
+
         testSpi = true;
 
         startWithCaches1(srvs, clients);
 
-        int minorVer = 1;
+        AffinityTopologyVersion affTopVer = new AffinityTopologyVersion(srvs + clients);
 
-        if (initiallyActive) {
-            ignite(0).cluster().active(true);
+        if (ClusterState.active(initialState)) {
+            ignite(0).cluster().state(initialState);
 
             awaitPartitionMapExchange();
 
-            minorVer++;
+            affTopVer = grid(0).cachex(CU.UTILITY_CACHE_NAME).context().topology().readyTopologyVersion();
+
+            assertEquals(srvs + clients, affTopVer.topologyVersion());
         }
 
         if (blockMsgNodes.length == 0)
             blockMsgNodes = new int[] {1};
 
-        final AffinityTopologyVersion STATE_CHANGE_TOP_VER = new AffinityTopologyVersion(srvs + clients, minorVer);
+        final AffinityTopologyVersion STATE_CHANGE_TOP_VER = affTopVer.nextMinorVersion();
 
         List<TestRecordingCommunicationSpi> spis = new ArrayList<>();
 
@@ -474,9 +594,7 @@ public class IgniteClusterActivateDeactivateTest extends GridCommonAbstractTest
             blockExchangeSingleMessage(spi, STATE_CHANGE_TOP_VER);
         }
 
-        IgniteInternalFuture<?> stateChangeFut = GridTestUtils.runAsync(() ->
-            ignite(stateChangeFrom).cluster().active(!initiallyActive)
-        );
+        IgniteInternalFuture<?> stateChangeFut = runAsync(() -> ignite(stateChangeFrom).cluster().state(targetState));
 
         for (TestRecordingCommunicationSpi spi : spis)
             spi.waitForBlocked();
@@ -510,7 +628,15 @@ public class IgniteClusterActivateDeactivateTest extends GridCommonAbstractTest
      */
     @Test
     public void testJoinWhileDeactivate1_Server() throws Exception {
-        joinWhileDeactivate1(false, false);
+        joinWhileDeactivate1(false, false, ACTIVE);
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    @Test
+    public void testJoinWhileDeactivateFromReadOnly1_Server() throws Exception {
+        joinWhileDeactivate1(false, false, ACTIVE_READ_ONLY);
     }
 
     /**
@@ -518,7 +644,15 @@ public class IgniteClusterActivateDeactivateTest extends GridCommonAbstractTest
      */
     @Test
     public void testJoinWhileDeactivate1_WithCache_Server() throws Exception {
-        joinWhileDeactivate1(false, true);
+        joinWhileDeactivate1(false, true, ACTIVE);
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    @Test
+    public void testJoinWhileDeactivateFromReadOnly1_WithCache_Server() throws Exception {
+        joinWhileDeactivate1(false, true, ACTIVE_READ_ONLY);
     }
 
     /**
@@ -526,63 +660,94 @@ public class IgniteClusterActivateDeactivateTest extends GridCommonAbstractTest
      */
     @Test
     public void testJoinWhileDeactivate1_Client() throws Exception {
-        joinWhileDeactivate1(true, false);
+        joinWhileDeactivate1(true, false, ACTIVE);
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    @Test
+    public void testJoinWhileDeactivateFromReadOnly1_Client() throws Exception {
+        joinWhileDeactivate1(true, false, ACTIVE_READ_ONLY);
+    }
+
+    /**
+     * @param startClient If {@code true} joins client node, otherwise server.
+     * @param withNewCache If {@code true} joining node has new cache in configuration.
+     * @param state Target cluster state.
+     * @throws Exception If failed.
+     */
+    private void joinWhileActivate1(boolean startClient, boolean withNewCache, ClusterState state) throws Exception {
+        joinWhileClusterStateChange(startClient, withNewCache, INACTIVE, state);
+    }
+
+    /**
+     * @param startClient If {@code true} joins client node, otherwise server.
+     * @param withNewCache If {@code true} joining node has new cache in configuration.
+     * @param state Initial cluster state.
+     * @throws Exception If failed.
+     */
+    private void joinWhileDeactivate1(boolean startClient, boolean withNewCache, ClusterState state) throws Exception {
+        joinWhileClusterStateChange(startClient, withNewCache, state, INACTIVE);
     }
 
     /**
      * @param startClient If {@code true} joins client node, otherwise server.
      * @param withNewCache If {@code true} joining node has new cache in configuration.
+     * @param initialState Initial cluster state.
+     * @param targetState Target cluster state.
      * @throws Exception If failed.
      */
-    private void joinWhileDeactivate1(final boolean startClient, final boolean withNewCache) throws Exception {
-        IgniteInternalFuture<?> activeFut = startNodesAndBlockStatusChange(2, 0, 0, true);
+    private void joinWhileClusterStateChange(
+        boolean startClient,
+        boolean withNewCache,
+        ClusterState initialState,
+        ClusterState targetState
+    ) throws Exception {
+        checkStatesAreDifferent(initialState, targetState);
 
-        IgniteInternalFuture<?> startFut = GridTestUtils.runAsync((Callable<Void>)() -> {
-            client = startClient;
+        int nodesCnt = 2;
 
-            ccfgs = withNewCache ? cacheConfigurations2() : cacheConfigurations1();
+        IgniteInternalFuture<?> activeFut = startNodesAndBlockStatusChange(nodesCnt, 0, 0, initialState, targetState);
 
-            startGrid(2);
+        ccfgs = withNewCache ? cacheConfigurations2() : cacheConfigurations1();
 
-            return null;
-        });
+        final int numberOfCaches = ccfgs.length;
 
-        TestRecordingCommunicationSpi spi1 = TestRecordingCommunicationSpi.spi(ignite(1));
+        IgniteInternalFuture<?> startFut = startNodeAsync(nodesCnt++, startClient);
 
-        spi1.stopBlock();
+        TestRecordingCommunicationSpi.spi(ignite(1)).stopBlock();
 
         activeFut.get();
         startFut.get();
 
-        checkNoCaches(3);
+        if (ClusterState.active(targetState))
+            checkCachesOnNode(nodesCnt - 1, DEFAULT_CACHES_COUNT);
+        else {
+            checkNoCaches(nodesCnt);
 
-        ignite(2).cluster().active(true);
+            ignite(nodesCnt - 1).cluster().state(initialState);
 
-        for (int c = 0; c < DEFAULT_CACHES_COUNT; c++)
-            checkCache(ignite(2), CACHE_NAME_PREFIX + c, true);
+            for (int c = 0; c < DEFAULT_CACHES_COUNT; c++)
+                checkCache(ignite(nodesCnt - 1), CACHE_NAME_PREFIX + c, true);
+        }
 
         if (withNewCache) {
-            for (int i = 0; i < 3; i++) {
-                for (int c = 0; c < 4; c++)
-                    checkCache(ignite(i), CACHE_NAME_PREFIX + c, true);
-            }
+            for (int i = 0; i < nodesCnt; i++)
+                checkCachesOnNode(i, numberOfCaches);
         }
 
         awaitPartitionMapExchange();
 
-        checkCaches(3, withNewCache ? 4 : 2);
-
-        client = false;
+        checkCaches(nodesCnt, numberOfCaches);
 
-        startGrid(3);
+        startGrid(nodesCnt++, false);
 
-        checkCaches(4, withNewCache ? 4 : 2);
+        checkCaches(nodesCnt, numberOfCaches);
 
-        client = true;
+        startGrid(nodesCnt++, true);
 
-        startGrid(4);
-
-        checkCaches(5, withNewCache ? 4 : 2);
+        checkCaches(nodesCnt, numberOfCaches);
     }
 
     /**
@@ -590,32 +755,45 @@ public class IgniteClusterActivateDeactivateTest extends GridCommonAbstractTest
      */
     @Test
     public void testConcurrentJoinAndActivate() throws Exception {
+        testConcurrentJoinAndActivate(ACTIVE);
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    @Test
+    public void testConcurrentJoinAndActivateInReadOnly() throws Exception {
+        testConcurrentJoinAndActivate(ACTIVE_READ_ONLY);
+    }
+
+    /** */
+    private void testConcurrentJoinAndActivate(ClusterState activateState) throws Exception {
+        assertActive(activateState);
+
         for (int iter = 0; iter < 3; iter++) {
             log.info("Iteration: " + iter);
 
-            active = false;
+            stateOnStart = INACTIVE;
 
-            for (int i = 0; i < 3; i++) {
-                ccfgs = cacheConfigurations1();
+            final int START_NODES = 3;
 
-                startGrid(i);
-            }
+            startWithCaches1(START_NODES, 0);
 
-            final int START_NODES = 3;
+            final int numberOfCaches = cacheConfigurations1().length;
 
             final CyclicBarrier b = new CyclicBarrier(START_NODES + 1);
 
-            IgniteInternalFuture<Void> fut1 = GridTestUtils.runAsync(() -> {
+            IgniteInternalFuture<Void> fut1 = runAsync(() -> {
                 b.await();
 
                 U.sleep(ThreadLocalRandom.current().nextLong(100) + 1);
 
-                ignite(0).cluster().active(true);
+                ignite(0).cluster().state(activateState);
 
                 return null;
             });
 
-            final AtomicInteger nodeIdx = new AtomicInteger(3);
+            final AtomicInteger nodeIdx = new AtomicInteger(START_NODES);
 
             IgniteInternalFuture<Long> fut2 = GridTestUtils.runMultiThreadedAsync((Callable<Void>)() -> {
                 int idx = nodeIdx.getAndIncrement();
@@ -630,7 +808,7 @@ public class IgniteClusterActivateDeactivateTest extends GridCommonAbstractTest
             fut1.get();
             fut2.get();
 
-            checkCaches(6, 2);
+            checkCaches(2 * START_NODES, numberOfCaches);
 
             afterTest();
         }
@@ -641,103 +819,161 @@ public class IgniteClusterActivateDeactivateTest extends GridCommonAbstractTest
      */
     @Test
     public void testDeactivateSimple_SingleNode() throws Exception {
-        deactivateSimple(1, 0, 0);
+        deactivateSimple(1, 0, 0, ACTIVE);
     }
 
     /**
      * @throws Exception If failed.
      */
     @Test
-    public void testDeactivateSimple_5_Servers() throws Exception {
-        deactivateSimple(5, 0, 0);
+    public void testDeactivateFromReadOnlySimple_SingleNode() throws Exception {
+        deactivateSimple(1, 0, 0, ACTIVE_READ_ONLY);
     }
 
     /**
      * @throws Exception If failed.
      */
     @Test
-    public void testDeactivateSimple_5_Servers2() throws Exception {
-        deactivateSimple(5, 0, 4);
+    public void testDeactivateSimple_5_Servers() throws Exception {
+        deactivateSimple(5, 0, 0, ACTIVE);
     }
 
     /**
      * @throws Exception If failed.
      */
     @Test
-    public void testDeactivateSimple_5_Servers_5_Clients() throws Exception {
-        deactivateSimple(5, 4, 0);
+    public void testDeactivateFromReadOnlySimple_5_Servers() throws Exception {
+        deactivateSimple(5, 0, 0, ACTIVE_READ_ONLY);
     }
 
     /**
      * @throws Exception If failed.
      */
     @Test
-    public void testDeactivateSimple_5_Servers_5_Clients_FromClient() throws Exception {
-        deactivateSimple(5, 4, 6);
+    public void testDeactivateSimple_5_Servers2() throws Exception {
+        deactivateSimple(5, 0, 4, ACTIVE);
     }
 
     /**
-     * @param srvs Number of servers.
-     * @param clients Number of clients.
-     * @param deactivateFrom Index of node stating deactivation.
      * @throws Exception If failed.
      */
-    private void deactivateSimple(int srvs, int clients, int deactivateFrom) throws Exception {
-        active = true;
+    @Test
+    public void testDeactivateFromReadOnlySimple_5_Servers2() throws Exception {
+        deactivateSimple(5, 0, 4, ACTIVE_READ_ONLY);
+    }
 
-        final int CACHES = 2;
+    /**
+     * @throws Exception If failed.
+     */
+    @Test
+    public void testDeactivateSimple_5_Servers_5_Clients() throws Exception {
+        deactivateSimple(5, 4, 0, ACTIVE);
+    }
 
-        for (int i = 0; i < srvs + clients; i++) {
-            client = i >= srvs;
+    /**
+     * @throws Exception If failed.
+     */
+    @Test
+    public void testDeactivateFromReadOnlySimple_5_Servers_5_Clients() throws Exception {
+        deactivateSimple(5, 4, 0, ACTIVE_READ_ONLY);
+    }
 
-            ccfgs = cacheConfigurations1();
+    /**
+     * @throws Exception If failed.
+     */
+    @Test
+    public void testDeactivateSimple_5_Servers_5_Clients_FromClient() throws Exception {
+        deactivateSimple(5, 4, 6, ACTIVE);
+    }
 
-            startGrid(i);
-        }
+    /**
+     * @throws Exception If failed.
+     */
+    @Test
+    public void testDeactivateFromReadOnlySimple_5_Servers_5_Clients_FromClient() throws Exception {
+        deactivateSimple(5, 4, 6, ACTIVE_READ_ONLY);
+    }
+
+    /**
+     * @param srvs Number of servers.
+     * @param clients Number of clients.
+     * @param changeFrom Index of node starting cluster state change from {@code initialState} to {@code targetState}.
+     * @param initialState Initial cluster state.
+     * @param targetState Target cluster state.
+     * @throws Exception If failed.
+     */
+    private void changeStateSimple(
+        int srvs,
+        int clients,
+        int changeFrom,
+        ClusterState initialState,
+        ClusterState targetState
+    ) throws Exception {
+        assertNotSame(initialState, targetState);
 
-        ignite(deactivateFrom).cluster().active(true); // Should be no-op.
+        stateOnStart = initialState;
 
-        checkCaches(srvs + clients, CACHES);
+        int nodesCnt = srvs + clients;
 
-        for (int i = 0; i < srvs + clients; i++)
-            assertTrue(ignite(i).cluster().active());
+        startWithCaches1(srvs, clients);
 
-        ignite(deactivateFrom).cluster().active(false);
+        if (persistenceEnabled() && ClusterState.active(initialState))
+            grid(0).cluster().state(initialState);
 
-        for (int i = 0; i < srvs + clients; i++)
-            assertFalse(ignite(i).cluster().active());
+        checkClusterState(nodesCnt, initialState);
+
+        if (!ClusterState.active(initialState))
+            checkNoCaches(nodesCnt);
 
-        checkNoCaches(srvs + clients);
+        ignite(changeFrom).cluster().state(initialState); // Should be no-op.
 
-        client = false;
+        checkClusterState(nodesCnt, initialState);
 
-        startGrid(srvs + clients);
+        ignite(changeFrom).cluster().state(targetState);
 
-        checkNoCaches(srvs + clients + 1);
+        checkClusterState(nodesCnt, targetState);
 
-        client = true;
+        if (ClusterState.active(targetState)) {
+            for (int i = 0; i < nodesCnt; i++)
+                checkCachesOnNode(i, DEFAULT_CACHES_COUNT);
 
-        startGrid(srvs + clients + 1);
+            checkCaches(nodesCnt, DEFAULT_CACHES_COUNT);
+        }
+        else
+            checkNoCaches(nodesCnt);
 
-        checkNoCaches(srvs + clients + 2);
+        startNodeAndCheckCaches(nodesCnt++, false, DEFAULT_CACHES_COUNT);
+        startNodeAndCheckCaches(nodesCnt++, true, DEFAULT_CACHES_COUNT);
 
-        for (int i = 0; i < srvs + clients + 2; i++)
-            assertFalse(ignite(i).cluster().active());
+        if (!ClusterState.active(targetState)) {
+            checkNoCaches(nodesCnt);
 
-        ignite(deactivateFrom).cluster().active(true);
+            checkClusterState(nodesCnt, targetState);
 
-        for (int i = 0; i < srvs + clients + 2; i++) {
-            assertTrue(ignite(i).cluster().active());
+            ignite(changeFrom).cluster().state(initialState);
 
-            checkCache(ignite(i), CU.UTILITY_CACHE_NAME, true);
-        }
+            checkClusterState(nodesCnt, initialState);
 
-        for (int i = 0; i < srvs; i++) {
-            for (int c = 0; c < DEFAULT_CACHES_COUNT; c++)
-                checkCache(ignite(i), CACHE_NAME_PREFIX + c, true);
+            for (int i = 0; i < nodesCnt; i++) {
+                if (ignite(i).configuration().isClientMode())
+                    checkCache(ignite(i), CU.UTILITY_CACHE_NAME, true);
+                else
+                    checkCachesOnNode(i, DEFAULT_CACHES_COUNT);
+            }
         }
+    }
+
+    /** */
+    private void startNodeAndCheckCaches(int nodeIdx, boolean client, int cachesCount) throws Exception {
+        startGrid(nodeIdx, client);
+
+        ClusterState state = grid(0).cluster().state();
+
+        if (ClusterState.active(state)) {
+            checkCachesOnNode(nodeIdx, cachesCount, !client);
 
-        checkCaches(srvs + clients + 2);
+            checkCaches(nodeIdx + 1, cachesCount);
+        }
     }
 
     /**
@@ -749,9 +985,7 @@ public class IgniteClusterActivateDeactivateTest extends GridCommonAbstractTest
         for (int i = 0; i < srvs + clients; i++) {
             ccfgs = cacheConfigurations1();
 
-            client = i >= srvs;
-
-            startGrid(i);
+            startGrid(i, i >= srvs);
         }
     }
 
@@ -760,38 +994,15 @@ public class IgniteClusterActivateDeactivateTest extends GridCommonAbstractTest
      */
     @Test
     public void testClientReconnectClusterActive() throws Exception {
-        testReconnectSpi = true;
-
-        ccfgs = cacheConfigurations1();
-
-        final int SRVS = 3;
-        final int CLIENTS = 3;
-
-        startWithCaches1(SRVS, CLIENTS);
-
-        if (persistenceEnabled())
-            ignite(0).cluster().active(true);
-
-        Ignite srv = ignite(0);
-        Ignite client = ignite(SRVS);
-
-        checkCache(client, CU.UTILITY_CACHE_NAME, true);
-
-        checkCaches(SRVS + CLIENTS);
-
-        IgniteClientReconnectAbstractTest.reconnectClientNode(log, client, srv, null);
-
-        checkCaches(SRVS + CLIENTS);
-
-        this.client = false;
-
-        startGrid(SRVS + CLIENTS);
-
-        this.client = true;
-
-        startGrid(SRVS + CLIENTS + 1);
+        testClientReconnect(ACTIVE);
+    }
 
-        checkCaches(SRVS + CLIENTS + 2);
+    /**
+     * @throws Exception If failed.
+     */
+    @Test
+    public void testClientReconnectClusterActiveReadOnly() throws Exception {
+        testClientReconnect(ACTIVE_READ_ONLY);
     }
 
     /**
@@ -799,39 +1010,53 @@ public class IgniteClusterActivateDeactivateTest extends GridCommonAbstractTest
      */
     @Test
     public void testClientReconnectClusterInactive() throws Exception {
+        testClientReconnect(INACTIVE);
+    }
+
+    /** */
+    private void testClientReconnect(ClusterState initialState) throws Exception {
         testReconnectSpi = true;
 
-        active = false;
+        stateOnStart = initialState;
+
+        ccfgs = cacheConfigurations1();
 
         final int SRVS = 3;
         final int CLIENTS = 3;
+        int nodesCnt = SRVS + CLIENTS;
 
         startWithCaches1(SRVS, CLIENTS);
 
+        if (persistenceEnabled() && ClusterState.active(initialState))
+            ignite(0).cluster().state(initialState);
+
         Ignite srv = ignite(0);
         Ignite client = ignite(SRVS);
 
-        checkNoCaches(SRVS + CLIENTS);
-
-        IgniteClientReconnectAbstractTest.reconnectClientNode(log, client, srv, null);
-
-        checkNoCaches(SRVS + CLIENTS);
+        if (ClusterState.active(initialState)) {
+            checkCache(client, CU.UTILITY_CACHE_NAME, true);
 
-        ignite(0).cluster().active(true);
+            checkCaches(nodesCnt);
+        }
+        else
+            checkNoCaches(nodesCnt);
 
-        checkCache(client, CU.UTILITY_CACHE_NAME, true);
+        IgniteClientReconnectAbstractTest.reconnectClientNode(log, client, srv, null);
 
-        checkCaches(SRVS + CLIENTS);
+        if (!ClusterState.active(initialState)) {
+            checkNoCaches(nodesCnt);
 
-        this.client = false;
+            srv.cluster().state(ACTIVE);
 
-        startGrid(SRVS + CLIENTS);
+            checkCache(client, CU.UTILITY_CACHE_NAME, true);
+        }
 
-        this.client = true;
+        checkCaches(nodesCnt);
 
-        startGrid(SRVS + CLIENTS + 1);
+        startGrid(nodesCnt++, false);
+        startGrid(nodesCnt++, true);
 
-        checkCaches(SRVS + CLIENTS);
+        checkCaches(nodesCnt);
     }
 
     /**
@@ -839,101 +1064,31 @@ public class IgniteClusterActivateDeactivateTest extends GridCommonAbstractTest
      */
     @Test
     public void testClientReconnectClusterDeactivated() throws Exception {
-        clientReconnectClusterDeactivated(false);
+        clientReconnectClusterState(ACTIVE, INACTIVE, false);
     }
 
     /**
      * @throws Exception If failed.
      */
     @Test
-    public void testClientReconnectClusterDeactivateInProgress() throws Exception {
-        clientReconnectClusterDeactivated(true);
+    public void testClientReconnectClusterDeactivatedFromReadOnly() throws Exception {
+        clientReconnectClusterState(ACTIVE_READ_ONLY, INACTIVE, false);
     }
 
     /**
-     * @param transition If {@code true} client reconnects while cluster state transition is in progress.
      * @throws Exception If failed.
      */
-    private void clientReconnectClusterDeactivated(final boolean transition) throws Exception {
-        testReconnectSpi = true;
-        testSpi = transition;
-
-        final int SRVS = 3;
-        final int CLIENTS = 3;
-
-        startWithCaches1(SRVS, CLIENTS);
-
-        final Ignite srv = ignite(0);
-        IgniteEx client = grid(SRVS);
-
-        if (persistenceEnabled())
-            ignite(0).cluster().active(true);
-
-        checkCache(client, CU.UTILITY_CACHE_NAME, true);
-
-        checkCaches(SRVS + CLIENTS);
-
-        // Wait for late affinity assignment to finish.
-        awaitPartitionMapExchange();
-
-        final AffinityTopologyVersion STATE_CHANGE_TOP_VER = new AffinityTopologyVersion(SRVS + CLIENTS + 1, 1);
-
-        final TestRecordingCommunicationSpi spi1 = transition ? TestRecordingCommunicationSpi.spi(ignite(1)) : null;
-
-        final AtomicReference<IgniteInternalFuture> stateFut = new AtomicReference<>();
-
-        IgniteClientReconnectAbstractTest.reconnectClientNode(log, client, srv, () -> {
-            if (transition) {
-                blockExchangeSingleMessage(spi1, STATE_CHANGE_TOP_VER);
-
-                stateFut.set(GridTestUtils.runAsync(() -> srv.cluster().active(false),
-                    "deactivate"));
-
-                try {
-                    U.sleep(500);
-                }
-                catch (Exception e) {
-                    e.printStackTrace();
-                }
-            }
-            else
-                srv.cluster().active(false);
-        });
-
-        if (transition) {
-            assertFalse(stateFut.get().isDone());
-
-            // Public API method would block forever because we blocked the exchange message.
-            assertFalse(client.context().state().publicApiActiveState(false));
-
-            spi1.waitForBlocked();
-
-            spi1.stopBlock();
-
-            stateFut.get().get();
-        }
-
-        checkNoCaches(SRVS + CLIENTS);
-
-        ignite(0).cluster().active(true);
-
-        checkCache(client, CU.UTILITY_CACHE_NAME, true);
-
-        assertTrue(client.cluster().active());
-
-        checkCaches(SRVS + CLIENTS);
-
-        checkCache(client, CACHE_NAME_PREFIX + 0, true);
-
-        this.client = false;
-
-        startGrid(SRVS + CLIENTS);
-
-        this.client = true;
-
-        startGrid(SRVS + CLIENTS + 1);
+    @Test
+    public void testClientReconnectClusterDeactivateInProgress() throws Exception {
+        clientReconnectClusterState(ACTIVE, INACTIVE, true);
+    }
 
-        checkCaches(SRVS + CLIENTS + 2);
+    /**
+     * @throws Exception If failed.
+     */
+    @Test
+    public void testClientReconnectClusterDeactivateFromReadOnlyInProgress() throws Exception {
+        clientReconnectClusterState(ACTIVE_READ_ONLY, INACTIVE, true);
     }
 
     /**
@@ -941,7 +1096,15 @@ public class IgniteClusterActivateDeactivateTest extends GridCommonAbstractTest
      */
     @Test
     public void testClientReconnectClusterActivated() throws Exception {
-        clientReconnectClusterActivated(false);
+        clientReconnectClusterState(INACTIVE, ACTIVE, false);
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    @Test
+    public void testClientReconnectClusterActivatedReadOnly() throws Exception {
+        clientReconnectClusterState(INACTIVE, ACTIVE_READ_ONLY, false);
     }
 
     /**
@@ -949,30 +1112,58 @@ public class IgniteClusterActivateDeactivateTest extends GridCommonAbstractTest
      */
     @Test
     public void testClientReconnectClusterActivateInProgress() throws Exception {
-        clientReconnectClusterActivated(true);
+        clientReconnectClusterState(INACTIVE, ACTIVE, true);
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    @Test
+    public void testClientReconnectClusterActivateReadOnlyInProgress() throws Exception {
+        clientReconnectClusterState(INACTIVE, ACTIVE_READ_ONLY, true);
     }
 
     /**
+     * @param initialState Initial cluster state.
+     * @param targetState Cluster state after transition.
      * @param transition If {@code true} client reconnects while cluster state transition is in progress.
      * @throws Exception If failed.
      */
-    private void clientReconnectClusterActivated(final boolean transition) throws Exception {
+    private void clientReconnectClusterState(
+        ClusterState initialState,
+        ClusterState targetState,
+        final boolean transition
+    ) throws Exception {
+        assertNotSame(initialState, targetState);
+
         testReconnectSpi = true;
         testSpi = transition;
-
-        active = false;
+        stateOnStart = initialState;
 
         final int SRVS = 3;
         final int CLIENTS = 3;
+        int nodesCnt = SRVS + CLIENTS;
 
         startWithCaches1(SRVS, CLIENTS);
 
         final Ignite srv = ignite(0);
         IgniteEx client = grid(SRVS);
 
-        checkNoCaches(SRVS + CLIENTS);
+        if (persistenceEnabled() && ClusterState.active(initialState))
+            ignite(0).cluster().state(initialState);
+
+        if (ClusterState.active(initialState)) {
+            checkCache(client, CU.UTILITY_CACHE_NAME, true);
+
+            checkCaches(nodesCnt);
+
+            // Wait for late affinity assignment to finish.
+            awaitPartitionMapExchange();
+        }
+        else
+            checkNoCaches(nodesCnt);
 
-        final AffinityTopologyVersion STATE_CHANGE_TOP_VER = new AffinityTopologyVersion(SRVS + CLIENTS + 1, 1);
+        final AffinityTopologyVersion STATE_CHANGE_TOP_VER = new AffinityTopologyVersion(nodesCnt + 1, 1);
 
         final TestRecordingCommunicationSpi spi1 = transition ? TestRecordingCommunicationSpi.spi(ignite(1)) : null;
 
@@ -982,18 +1173,17 @@ public class IgniteClusterActivateDeactivateTest extends GridCommonAbstractTest
             if (transition) {
                 blockExchangeSingleMessage(spi1, STATE_CHANGE_TOP_VER);
 
-                stateFut.set(GridTestUtils.runAsync(() -> srv.cluster().active(true),
-                    "activate"));
+                stateFut.set(runAsync(() -> srv.cluster().state(targetState), initialState + "->" + targetState));
 
                 try {
                     U.sleep(500);
                 }
-                catch (Exception e) {
-                    e.printStackTrace();
+                catch (IgniteInterruptedCheckedException e) {
+                    U.error(log, e);
                 }
             }
             else
-                srv.cluster().active(true);
+                srv.cluster().state(targetState);
         });
 
         if (transition) {
@@ -1001,6 +1191,9 @@ public class IgniteClusterActivateDeactivateTest extends GridCommonAbstractTest
 
             assertTrue(client.context().state().clusterState().transition());
 
+            // Public API method would block forever because we blocked the exchange message.
+            assertEquals(lesserOf(initialState, targetState), client.context().state().publicApiState(false));
+
             spi1.waitForBlocked();
 
             spi1.stopBlock();
@@ -1008,21 +1201,24 @@ public class IgniteClusterActivateDeactivateTest extends GridCommonAbstractTest
             stateFut.get().get();
         }
 
-        checkCache(client, CU.UTILITY_CACHE_NAME, true);
+        if (!ClusterState.active(targetState)) {
+            checkNoCaches(nodesCnt);
 
-        checkCaches(SRVS + CLIENTS);
+            ignite(0).cluster().state(initialState);
 
-        checkCache(client, CACHE_NAME_PREFIX + 0, true);
+            checkClusterState(nodesCnt, initialState);
+        }
 
-        this.client = false;
+        checkCache(client, CU.UTILITY_CACHE_NAME, true);
 
-        startGrid(SRVS + CLIENTS);
+        checkCaches(nodesCnt);
 
-        this.client = true;
+        checkCache(client, CACHE_NAME_PREFIX + 0, true);
 
-        startGrid(SRVS + CLIENTS + 1);
+        startGrid(nodesCnt++, false);
+        startGrid(nodesCnt++, true);
 
-        checkCaches(SRVS + CLIENTS + 2);
+        checkCaches(nodesCnt);
     }
 
     /**
@@ -1030,14 +1226,30 @@ public class IgniteClusterActivateDeactivateTest extends GridCommonAbstractTest
      */
     @Test
     public void testInactiveTopologyChanges() throws Exception {
+        checkInactiveTopologyChanges(ACTIVE);
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    @Test
+    public void testInactiveTopologyChangesReadOnly() throws Exception {
+        checkInactiveTopologyChanges(ACTIVE_READ_ONLY);
+    }
+
+    /** */
+    private void checkInactiveTopologyChanges(ClusterState state) throws Exception {
+        assertActive(state);
+
         testSpi = true;
 
         testSpiRecord = new Class[] {GridDhtPartitionsSingleMessage.class, GridDhtPartitionsFullMessage.class};
 
-        active = false;
+        stateOnStart = INACTIVE;
 
         final int SRVS = 4;
         final int CLIENTS = 4;
+        int nodesCnt = SRVS + CLIENTS;
 
         startWithCaches1(SRVS, CLIENTS);
 
@@ -1046,9 +1258,7 @@ public class IgniteClusterActivateDeactivateTest extends GridCommonAbstractTest
         for (int i = 0; i < 2; i++) {
             stopGrid(i);
 
-            client = false;
-
-            startGrid(i);
+            startGrid(i, false);
         }
 
         checkRecordedMessages(false);
@@ -1056,30 +1266,23 @@ public class IgniteClusterActivateDeactivateTest extends GridCommonAbstractTest
         for (int i = 0; i < 2; i++) {
             stopGrid(SRVS + i);
 
-            client = true;
-
-            startGrid(SRVS + i);
+            startGrid(SRVS + i, true);
         }
 
         checkRecordedMessages(false);
 
-        ignite(0).cluster().active(true);
+        ignite(0).cluster().state(state);
 
-        checkCaches(SRVS + CLIENTS);
+        checkCaches(nodesCnt);
 
         checkRecordedMessages(true);
 
-        client = false;
-
-        startGrid(SRVS + CLIENTS);
-
-        client = true;
-
-        startGrid(SRVS + CLIENTS + 1);
+        startGrid(nodesCnt++, false);
+        startGrid(nodesCnt++, true);
 
         checkRecordedMessages(true);
 
-        checkCaches(SRVS + CLIENTS + 2);
+        checkCaches(nodesCnt);
     }
 
     /**
@@ -1087,58 +1290,47 @@ public class IgniteClusterActivateDeactivateTest extends GridCommonAbstractTest
      */
     @Test
     public void testActivateFailover1() throws Exception {
-        stateChangeFailover1(true);
+        stateChangeFailover1(INACTIVE, ACTIVE);
     }
 
     /**
      * @throws Exception If failed.
      */
     @Test
-    public void testDeactivateFailover1() throws Exception {
-        stateChangeFailover1(false);
+    public void testActivateWithReadOnlyFailover1() throws Exception {
+        stateChangeFailover1(INACTIVE, ACTIVE_READ_ONLY);
     }
 
     /**
-     * @param activate If {@code true} tests activation, otherwise deactivation.
      * @throws Exception If failed.
      */
-    private void stateChangeFailover1(boolean activate) throws Exception {
-        // Nodes 1 and 4 do not reply to coordinator.
-        IgniteInternalFuture<?> fut = startNodesAndBlockStatusChange(4, 4, 3, !activate, 1, 4);
-
-        client = false;
-
-        // Start one more node while transition is in progress.
-        IgniteInternalFuture<Void> startFut = GridTestUtils.runAsync(() -> {
-            startGrid(8);
-
-            return null;
-        }, "start-node");
-
-        U.sleep(500);
-
-        stopGrid(getTestIgniteInstanceName(1), true, false);
-        stopGrid(getTestIgniteInstanceName(4), true, false);
-
-        fut.get();
-
-        startFut.get();
-
-        client = false;
-
-        startGrid(1);
-
-        client = true;
-
-        startGrid(4);
+    @Test
+    public void testDeactivateFailover1() throws Exception {
+        stateChangeFailover1(ACTIVE, INACTIVE);
+    }
 
-        if (!activate) {
-            checkNoCaches(9);
+    /**
+     * @throws Exception If failed.
+     */
+    @Test
+    public void testDeactivateFromReadOnlyFailover1() throws Exception {
+        stateChangeFailover1(ACTIVE_READ_ONLY, INACTIVE);
+    }
 
-            ignite(0).cluster().active(true);
-        }
+    /**
+     * @throws Exception If failed.
+     */
+    @Test
+    public void testEnableReadOnlyFailover1() throws Exception {
+        stateChangeFailover1(ACTIVE, ACTIVE_READ_ONLY);
+    }
 
-        checkCaches(9);
+    /**
+     * @throws Exception If failed.
+     */
+    @Test
+    public void testDisableReadOnlyFailover1() throws Exception {
+        stateChangeFailover1(ACTIVE_READ_ONLY, ACTIVE);
     }
 
     /**
@@ -1146,138 +1338,217 @@ public class IgniteClusterActivateDeactivateTest extends GridCommonAbstractTest
      */
     @Test
     public void testActivateFailover2() throws Exception {
-        stateChangeFailover2(true);
+        stateChangeFailover2(INACTIVE, ACTIVE);
     }
 
     /**
      * @throws Exception If failed.
      */
     @Test
-    public void testDeactivateFailover2() throws Exception {
-        stateChangeFailover2(false);
+    public void testActivateWithReadOnlyFailover2() throws Exception {
+        stateChangeFailover2(INACTIVE, ACTIVE_READ_ONLY);
     }
 
     /**
-     * @param activate If {@code true} tests activation, otherwise deactivation.
      * @throws Exception If failed.
      */
-    private void stateChangeFailover2(boolean activate) throws Exception {
-        // Nodes 1 and 4 do not reply to coordinator.
-        IgniteInternalFuture<?> fut = startNodesAndBlockStatusChange(4, 4, 3, !activate, 1, 4);
-
-        client = false;
-
-        // Start more nodes while transition is in progress.
-        IgniteInternalFuture<Void> startFut1 = GridTestUtils.runAsync(() -> {
-            startGrid(8);
-
-            return null;
-        }, "start-node1");
-
-        IgniteInternalFuture<Void> startFut2 = GridTestUtils.runAsync(() -> {
-            startGrid(9);
-
-            return null;
-        }, "start-node2");
-
-        U.sleep(500);
-
-        // Stop coordinator.
-        stopGrid(getTestIgniteInstanceName(0), true, false);
-
-        stopGrid(getTestIgniteInstanceName(1), true, false);
-        stopGrid(getTestIgniteInstanceName(4), true, false);
-
-        fut.get();
-
-        startFut1.get();
-        startFut2.get();
+    @Test
+    public void testDeactivateFailover2() throws Exception {
+        stateChangeFailover2(ACTIVE, INACTIVE);
+    }
 
-        client = false;
+    /**
+     * @throws Exception If failed.
+     */
+    @Test
+    public void testDeactivateFromReadOnlyFailover2() throws Exception {
+        stateChangeFailover2(ACTIVE_READ_ONLY, INACTIVE);
+    }
 
-        startGrid(0);
-        startGrid(1);
+    /**
+     * @throws Exception If failed.
+     */
+    @Test
+    public void testEnableReadOnlyFailover2() throws Exception {
+        stateChangeFailover2(ACTIVE, ACTIVE_READ_ONLY);
+    }
 
-        client = true;
+    /**
+     * @throws Exception If failed.
+     */
+    @Test
+    public void testDisableReadOnlyFailover2() throws Exception {
+        stateChangeFailover2(ACTIVE_READ_ONLY, ACTIVE);
+    }
 
-        startGrid(4);
+    /**
+     * @throws Exception If failed.
+     */
+    @Test
+    public void testActivateFailover3() throws Exception {
+        stateChangeFailover3(INACTIVE, ACTIVE);
+    }
 
-        if (!activate) {
-            checkNoCaches(10);
+    /**
+     * @throws Exception If failed.
+     */
+    @Test
+    public void testActivateWithReadOnlyFailover3() throws Exception {
+        stateChangeFailover3(INACTIVE, ACTIVE_READ_ONLY);
+    }
 
-            ignite(0).cluster().active(true);
-        }
+    /**
+     * @throws Exception If failed.
+     */
+    @Test
+    public void testDeactivateFailover3() throws Exception {
+        stateChangeFailover3(ACTIVE, INACTIVE);
+    }
 
-        checkCaches(10);
+    /**
+     * @throws Exception If failed.
+     */
+    @Test
+    public void testDeactivateFromReadOnlyFailover3() throws Exception {
+        stateChangeFailover3(ACTIVE_READ_ONLY, INACTIVE);
     }
 
     /**
      * @throws Exception If failed.
      */
     @Test
-    public void testActivateFailover3() throws Exception {
-        stateChangeFailover3(true);
+    public void testEnableReadOnlyFailover3() throws Exception {
+        stateChangeFailover3(ACTIVE, ACTIVE_READ_ONLY);
     }
 
     /**
      * @throws Exception If failed.
      */
     @Test
-    public void testDeactivateFailover3() throws Exception {
-        stateChangeFailover3(false);
+    public void testDisableReadOnlyFailover3() throws Exception {
+        stateChangeFailover3(ACTIVE_READ_ONLY, ACTIVE);
     }
 
     /**
-     * @param activate If {@code true} tests activation, otherwise deactivation.
+     * @param initialState Initial cluster state.
+     * @param targetState Target cluster state.
      * @throws Exception If failed.
      */
-    private void stateChangeFailover3(boolean activate) throws Exception {
-        testReconnectSpi = true;
+    private void stateChangeFailover1(ClusterState initialState, ClusterState targetState) throws Exception {
+        stateChangeFailover(initialState, targetState, 1, 1, 4);
+    }
+
+    /**
+     * @param initialState Initial cluster state.
+     * @param targetState Target cluster state.
+     * @throws Exception If failed.
+     */
+    private void stateChangeFailover2(ClusterState initialState, ClusterState targetState) throws Exception {
+        stateChangeFailover(initialState, targetState, 2, 0, 1, 4);
+    }
 
-        startNodesAndBlockStatusChange(4, 0, 0, !activate);
+    /**
+     * @param initialState Initial cluster state.
+     * @param targetState Target cluster state.
+     * @throws Exception If failed.
+     */
+    private void stateChangeFailover3(ClusterState initialState, ClusterState targetState) throws Exception {
+        assertNotSame(initialState, targetState);
 
-        client = false;
+        testReconnectSpi = true;
 
-        IgniteInternalFuture<?> startFut1 = GridTestUtils.runAsync(() -> {
-            startGrid(4);
+        final int servers = 4;
+        final int clients = 0;
+        int nodesCnt = servers + clients;
 
-            return null;
-        }, "start-node1");
+        startNodesAndBlockStatusChange(servers, clients, 0, initialState, targetState);
 
-        IgniteInternalFuture<?> startFut2 = GridTestUtils.runAsync(() -> {
-            startGrid(5);
+        IgniteInternalFuture<?> startFut1 = startNodeAsync(nodesCnt++, false);
+        IgniteInternalFuture<?> startFut2 = startNodeAsync(nodesCnt++, false);
 
-            return null;
-        }, "start-node2");
+        final int expNodesCnt = nodesCnt;
 
-        U.sleep(1000);
+        assertTrue(waitForCondition(() -> grid(0).cluster().nodes().size() == expNodesCnt, 30000L));
 
         // Stop all nodes participating in state change and not allow last node to finish exchange.
-        for (int i = 0; i < 4; i++)
+        for (int i = 0; i < servers; i++)
             ((IgniteDiscoverySpi)ignite(i).configuration().getDiscoverySpi()).simulateNodeFailure();
 
-        for (int i = 0; i < 4; i++)
+        for (int i = 0; i < servers; i++)
             stopGrid(getTestIgniteInstanceName(i), true, false);
 
         startFut1.get();
         startFut2.get();
 
-        assertFalse(ignite(4).cluster().active());
-        assertFalse(ignite(5).cluster().active());
+        for (int i = servers; i < nodesCnt; i++)
+            assertEquals(ignite(i).name(), INACTIVE, ignite(i).cluster().state());
 
-        ignite(4).cluster().active(true);
+        ignite(servers).cluster().state(ClusterState.active(initialState) ? initialState : targetState);
 
-        doFinalChecks();
+        doFinalChecks(servers, nodesCnt);
     }
 
     /**
-     * Verifies correctness of cache operations when working in in-memory mode.
+     * @param initialState Initial cluster state.
+     * @param targetState Target cluster state.
+     * @param startExtraNodes Number of started server nodes during blocked status change.
+     * @param restartNodes Indexes of ignite instances for restart.
+     * @throws Exception If failed.
      */
-    protected void doFinalChecks() throws Exception {
-        for (int i = 0; i < 4; i++)
+    private void stateChangeFailover(
+        ClusterState initialState,
+        ClusterState targetState,
+        int startExtraNodes,
+        int... restartNodes
+    ) throws Exception {
+        assertNotSame(initialState, targetState);
+
+        assertTrue(Arrays.toString(restartNodes) + " doesn't contain element 1", U.containsIntArray(restartNodes, 1));
+        assertTrue(Arrays.toString(restartNodes) + " doesn't contain element 4", U.containsIntArray(restartNodes, 4));
+
+        final int servers = 4;
+        final int clients = 4;
+        int nodesCnt = servers + clients;
+
+        // Nodes 1 and 4 do not reply to coordinator.
+        IgniteInternalFuture<?> fut = startNodesAndBlockStatusChange(servers, clients, 3, initialState, targetState, 1, 4);
+
+        List<IgniteInternalFuture<?>> startFuts = new ArrayList<>();
+
+        // Start more nodes while transition is in progress.
+        for (int i = 0; i < startExtraNodes; i++)
+            startFuts.add(startNodeAsync(nodesCnt++, false));
+
+        final int exceptedNodesCnt = nodesCnt;
+
+        assertTrue(waitForCondition(() -> grid(0).cluster().nodes().size() == exceptedNodesCnt, 30000L));
+
+        for (int idx : restartNodes)
+            stopGrid(getTestIgniteInstanceName(idx), true, false);
+
+        fut.get();
+
+        for (IgniteInternalFuture<?> startFut : startFuts)
+            startFut.get();
+
+        for (int idx : restartNodes)
+            startGrid(idx, idx >= servers & idx < (servers + clients));
+
+        if (!ClusterState.active(targetState)) {
+            checkNoCaches(nodesCnt);
+
+            ignite(0).cluster().state(initialState);
+        }
+
+        checkCaches(nodesCnt);
+    }
+
+    /** */
+    protected void doFinalChecks(int startNodes, int nodesCnt) throws Exception {
+        for (int i = 0; i < startNodes; i++)
             startGrid(i);
 
-        checkCaches(6);
+        checkCaches(nodesCnt);
     }
 
     /**
@@ -1285,13 +1556,28 @@ public class IgniteClusterActivateDeactivateTest extends GridCommonAbstractTest
      */
     @Test
     public void testClusterStateNotWaitForDeactivation() throws Exception {
+        checkClusterStateNotWaitForDeactivation(ACTIVE);
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    @Test
+    public void testReadOnlyClusterStateNotWaitForDeactivation() throws Exception {
+        checkClusterStateNotWaitForDeactivation(ACTIVE_READ_ONLY);
+    }
+
+    /** */
+    private void checkClusterStateNotWaitForDeactivation(ClusterState initialState) throws Exception {
+        assertActive(initialState);
+
         testSpi = true;
 
         final int nodes = 2;
 
-        IgniteEx crd = (IgniteEx)startGrids(nodes);
+        IgniteEx crd = startGrids(nodes);
 
-        crd.cluster().active(true);
+        crd.cluster().state(initialState);
 
         AffinityTopologyVersion curTopVer = crd.context().discovery().topologyVersionEx();
 
@@ -1300,29 +1586,28 @@ public class IgniteClusterActivateDeactivateTest extends GridCommonAbstractTest
             curTopVer.minorTopologyVersion() + 1
         );
 
-        for (int gridIdx = 0; gridIdx < nodes; gridIdx++) {
-            TestRecordingCommunicationSpi spi = TestRecordingCommunicationSpi.spi(grid(gridIdx));
+        for (int gridIdx = 0; gridIdx < nodes; gridIdx++)
+            blockExchangeSingleMessage(TestRecordingCommunicationSpi.spi(grid(gridIdx)), deactivationTopVer);
 
-            blockExchangeSingleMessage(spi, deactivationTopVer);
-        }
-
-        IgniteInternalFuture deactivationFut = GridTestUtils.runAsync(() -> crd.cluster().active(false));
+        IgniteInternalFuture deactivationFut = runAsync(() -> crd.cluster().state(INACTIVE));
 
         // Wait for deactivation start.
-        GridTestUtils.waitForCondition(() -> {
-            DiscoveryDataClusterState clusterState = crd.context().state().clusterState();
+        assertTrue(GridTestUtils.waitForCondition(
+            () -> {
+                DiscoveryDataClusterState clusterState = crd.context().state().clusterState();
 
-            return clusterState.transition() && !clusterState.active();
-        }, getTestTimeout());
+                return clusterState.transition() && !ClusterState.active(clusterState.state());
+            },
+            getTestTimeout()
+        ));
 
         // Check that deactivation transition wait is not happened.
-        Assert.assertFalse(crd.context().state().publicApiActiveState(true));
+        ClusterState state = crd.context().state().publicApiState(true);
 
-        for (int gridIdx = 0; gridIdx < nodes; gridIdx++) {
-            TestRecordingCommunicationSpi spi = TestRecordingCommunicationSpi.spi(grid(gridIdx));
+        assertInactive(state);
 
-            spi.stopBlock();
-        }
+        for (int gridIdx = 0; gridIdx < nodes; gridIdx++)
+            TestRecordingCommunicationSpi.spi(grid(gridIdx)).stopBlock();
 
         deactivationFut.get();
     }
@@ -1332,8 +1617,7 @@ public class IgniteClusterActivateDeactivateTest extends GridCommonAbstractTest
      */
     private void checkRecordedMessages(boolean exp) {
         for (Ignite node : G.allGrids()) {
-            List<Object> recorded =
-                TestRecordingCommunicationSpi.spi(node).recordedMessages(false);
+            List<Object> recorded = TestRecordingCommunicationSpi.spi(node).recordedMessages(false);
 
             if (exp)
                 assertFalse(F.isEmpty(recorded));
@@ -1343,13 +1627,6 @@ public class IgniteClusterActivateDeactivateTest extends GridCommonAbstractTest
     }
 
     /**
-     * @param nodes Expected nodes number.
-     */
-    private void checkCaches(int nodes) throws InterruptedException {
-        checkCaches(nodes, 2, false);
-    }
-
-    /**
      * @return Cache configurations.
      */
     final CacheConfiguration[] cacheConfigurations1() {
@@ -1407,7 +1684,7 @@ public class IgniteClusterActivateDeactivateTest extends GridCommonAbstractTest
 
         ((IgniteEx)node).context().cache().context().exchange().lastTopologyFuture().get();
 
-        ((IgniteEx)node).context().state().publicApiActiveState(true);
+        ((IgniteEx)node).context().state().publicApiState(true);
 
         GridCacheAdapter cache = ((IgniteEx)node).context().cache().internalCache(cacheName);
 
@@ -1422,12 +1699,69 @@ public class IgniteClusterActivateDeactivateTest extends GridCommonAbstractTest
      */
     final void checkNoCaches(int nodes) {
         for (int i = 0; i < nodes; i++) {
-            assertFalse(grid(i).context().state().publicApiActiveState(true));
+            assertEquals(INACTIVE, grid(i).context().state().publicApiState(true));
 
-            GridCacheProcessor cache = ((IgniteEx)ignite(i)).context().cache();
+            GridCacheProcessor cache = ignite(i).context().cache();
 
             assertTrue(cache.caches().isEmpty());
             assertTrue(cache.internalCaches().stream().allMatch(c -> c.context().isRecoveryMode()));
         }
     }
+
+
+    /** */
+    private void checkClusterState(int nodesCnt, ClusterState state) {
+        for (int i = 0; i < nodesCnt; i++)
+            assertEquals(ignite(i).name(), state, ignite(i).cluster().state());
+    }
+
+    /** */
+    protected void checkCachesOnNode(int nodeNumber, int cachesCnt) throws IgniteCheckedException {
+        checkCachesOnNode(nodeNumber, cachesCnt, true);
+    }
+
+    /** */
+    protected void checkCachesOnNode(int nodeNumber, int cachesCnt, boolean expUserCaches) throws IgniteCheckedException {
+        for (int c = 0; c < cachesCnt; c++)
+            checkCache(ignite(nodeNumber), CACHE_NAME_PREFIX + c, expUserCaches);
+
+        checkCache(ignite(nodeNumber), CU.UTILITY_CACHE_NAME, true);
+    }
+
+    /**
+     * @param nodes Expected nodes number.
+     */
+    private void checkCaches(int nodes) throws InterruptedException {
+        checkCaches(nodes, 2, false);
+    }
+
+    /** */
+    private static void checkStatesAreDifferent(ClusterState state1, ClusterState state2) {
+        assertTrue(state1 + " " + state2, ClusterState.active(state1) != ClusterState.active(state2));
+    }
+
+    /** */
+    protected void startGrid(int nodeNumber, boolean client) throws Exception {
+        startGrid(nodeNumber, client, null);
+    }
+
+    /** */
+    protected void startGrid(int nodeNumber, boolean client, CacheConfiguration[] cacheConfigs) throws Exception {
+        if (cacheConfigs != null)
+            this.ccfgs = cacheConfigs;
+
+        this.client = client;
+
+        startGrid(nodeNumber);
+    }
+
+    /** */
+    private IgniteInternalFuture<?> startNodeAsync(int nodeNumber, boolean client) {
+        this.client = client;
+
+        return runAsync(
+            () -> startGrid(nodeNumber),
+            "start" + "-" + (client ? "client" : "server") + "-node" + nodeNumber
+        );
+    }
 }
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/IgniteClusterActivateDeactivateTestWithPersistence.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/IgniteClusterActivateDeactivateTestWithPersistence.java
index 49bf9e5..21d93d7 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/IgniteClusterActivateDeactivateTestWithPersistence.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/IgniteClusterActivateDeactivateTestWithPersistence.java
@@ -31,14 +31,15 @@ import org.apache.ignite.IgniteDataStreamer;
 import org.apache.ignite.cache.CacheAtomicityMode;
 import org.apache.ignite.cache.CacheWriteSynchronizationMode;
 import org.apache.ignite.cache.affinity.rendezvous.RendezvousAffinityFunction;
+import org.apache.ignite.cluster.ClusterState;
 import org.apache.ignite.configuration.CacheConfiguration;
 import org.apache.ignite.configuration.DataStorageConfiguration;
 import org.apache.ignite.configuration.IgniteConfiguration;
 import org.apache.ignite.internal.IgniteEx;
 import org.apache.ignite.internal.IgniteInternalFuture;
+import org.apache.ignite.internal.processors.cache.distributed.dht.IgniteClusterReadOnlyException;
 import org.apache.ignite.internal.util.GridConcurrentHashSet;
 import org.apache.ignite.internal.util.typedef.G;
-import org.apache.ignite.internal.util.typedef.X;
 import org.apache.ignite.internal.util.typedef.internal.CU;
 import org.apache.ignite.spi.IgniteSpiException;
 import org.apache.ignite.testframework.GridTestUtils;
@@ -48,7 +49,12 @@ import org.junit.Assume;
 import org.junit.Test;
 
 import static org.apache.ignite.IgniteSystemProperties.IGNITE_BASELINE_AUTO_ADJUST_ENABLED;
+import static org.apache.ignite.cluster.ClusterState.ACTIVE;
+import static org.apache.ignite.cluster.ClusterState.INACTIVE;
+import static org.apache.ignite.cluster.ClusterState.ACTIVE_READ_ONLY;
+import static org.apache.ignite.testframework.GridTestUtils.assertActive;
 import static org.apache.ignite.testframework.GridTestUtils.assertThrowsAnyCause;
+import static org.apache.ignite.testframework.GridTestUtils.assertThrowsWithCause;
 
 /**
  *
@@ -75,9 +81,9 @@ public class IgniteClusterActivateDeactivateTestWithPersistence extends IgniteCl
 
     /** {@inheritDoc} */
     @Override protected void beforeTest() throws Exception {
-        super.beforeTest();
-
         cleanPersistenceDir();
+
+        super.beforeTest();
     }
 
     /** {@inheritDoc} */
@@ -97,7 +103,15 @@ public class IgniteClusterActivateDeactivateTestWithPersistence extends IgniteCl
      */
     @Test
     public void testActivateCachesRestore_SingleNode() throws Exception {
-        activateCachesRestore(1, false);
+        activateCachesRestore(1, false, ACTIVE);
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    @Test
+    public void testActivateReadOnlyCachesRestore_SingleNode() throws Exception {
+        activateCachesRestore(1, false, ACTIVE_READ_ONLY);
     }
 
     /**
@@ -105,7 +119,15 @@ public class IgniteClusterActivateDeactivateTestWithPersistence extends IgniteCl
      */
     @Test
     public void testActivateCachesRestore_SingleNode_WithNewCaches() throws Exception {
-        activateCachesRestore(1, true);
+        activateCachesRestore(1, true, ACTIVE);
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    @Test
+    public void testActivateReadOnlyCachesRestore_SingleNode_WithNewCaches() throws Exception {
+        activateCachesRestore(1, true, ACTIVE_READ_ONLY);
     }
 
     /**
@@ -113,7 +135,15 @@ public class IgniteClusterActivateDeactivateTestWithPersistence extends IgniteCl
      */
     @Test
     public void testActivateCachesRestore_5_Servers() throws Exception {
-        activateCachesRestore(5, false);
+        activateCachesRestore(5, false, ACTIVE);
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    @Test
+    public void testActivateReadOnlyCachesRestore_5_Servers() throws Exception {
+        activateCachesRestore(5, false, ACTIVE_READ_ONLY);
     }
 
     /**
@@ -121,7 +151,15 @@ public class IgniteClusterActivateDeactivateTestWithPersistence extends IgniteCl
      */
     @Test
     public void testActivateCachesRestore_5_Servers_WithNewCaches() throws Exception {
-        activateCachesRestore(5, true);
+        activateCachesRestore(5, true, ACTIVE);
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    @Test
+    public void testActivateReadOnlyCachesRestore_5_Servers_WithNewCaches() throws Exception {
+        activateCachesRestore(5, true, ACTIVE_READ_ONLY);
     }
 
     /**
@@ -131,6 +169,23 @@ public class IgniteClusterActivateDeactivateTestWithPersistence extends IgniteCl
      */
     @Test
     public void testDeactivateInactiveCluster() throws Exception {
+        checkDeactivateInactiveCluster(ACTIVE);
+    }
+
+    /**
+     * Test deactivation on cluster that is not yet activated.
+     *
+     * @throws Exception If failed.
+     */
+    @Test
+    public void testDeactivateInactiveClusterReadOnly() throws Exception {
+        checkDeactivateInactiveCluster(ACTIVE_READ_ONLY);
+    }
+
+    /** */
+    private void checkDeactivateInactiveCluster(ClusterState activationMode) throws Exception {
+        assertActive(activationMode);
+
         ccfgs = new CacheConfiguration[] {
             new CacheConfiguration<>("test_cache_1")
                 .setGroupName("test_cache")
@@ -142,43 +197,57 @@ public class IgniteClusterActivateDeactivateTestWithPersistence extends IgniteCl
 
         Ignite ignite = startGrids(3);
 
-        ignite.cluster().active(true);
+        ignite.cluster().state(activationMode);
 
-        ignite.cache("test_cache_1")
-            .put("key1", "val1");
-        ignite.cache("test_cache_2")
-            .put("key1", "val1");
+        if (activationMode == ACTIVE) {
+            ignite.cache("test_cache_1")
+                .put("key1", "val1");
+            ignite.cache("test_cache_2")
+                .put("key1", "val1");
+        }
 
-        ignite.cluster().active(false);
+        ignite.cluster().state(INACTIVE);
 
-        assertFalse(ignite.cluster().active());
+        assertEquals(INACTIVE, ignite.cluster().state());
 
         stopAllGrids();
 
         ignite = startGrids(2);
 
-        assertFalse(ignite.cluster().active());
+        assertEquals(INACTIVE, ignite.cluster().state());
 
-        ignite.cluster().active(false);
+        ignite.cluster().state(INACTIVE);
 
-        assertFalse(ignite.cluster().active());
+        assertEquals(INACTIVE, ignite.cluster().state());
     }
 
     /** */
-    private Map<Integer, Integer> startGridsAndLoadData(int srvs) throws Exception {
+    private Map<Integer, Integer> startGridsAndLoadData(int srvs, ClusterState activationMode) throws Exception {
+        assertActive(activationMode);
+
         Ignite srv = startGrids(srvs);
 
-        srv.active(true);
+        srv.cluster().state(activationMode);
 
         srv.createCaches(Arrays.asList(cacheConfigurations1()));
 
         Map<Integer, Integer> cacheData = new LinkedHashMap<>();
 
-        for (int i = 1; i <= 100; i++) {
-            for (CacheConfiguration ccfg : cacheConfigurations1()) {
-                srv.cache(ccfg.getName()).put(-i, i);
+        for (CacheConfiguration ccfg : cacheConfigurations1()) {
+            for (int i = 1; i <= 100; i++) {
+                int key = -i;
+                int val = i;
 
-                cacheData.put(-i, i);
+                if (activationMode == ACTIVE) {
+                    srv.cache(ccfg.getName()).put(key, val);
+
+                    cacheData.put(key, val);
+                }
+                else {
+                    assertThrowsWithCause(() -> srv.cache(ccfg.getName()).put(key, val), IgniteClusterReadOnlyException.class);
+
+                    cacheData.put(key, null);
+                }
             }
         }
 
@@ -188,10 +257,13 @@ public class IgniteClusterActivateDeactivateTestWithPersistence extends IgniteCl
     /**
      * @param srvs Number of server nodes.
      * @param withNewCaches If {@code true} then after restart has new caches in configuration.
+     * @param activationMode Cluster activation mode.
      * @throws Exception If failed.
      */
-    private void activateCachesRestore(int srvs, boolean withNewCaches) throws Exception {
-        Map<Integer, Integer> cacheData = startGridsAndLoadData(srvs);
+    private void activateCachesRestore(int srvs, boolean withNewCaches, ClusterState activationMode) throws Exception {
+        assertActive(activationMode);
+
+        Map<Integer, Integer> cacheData = startGridsAndLoadData(srvs, activationMode);
 
         stopAllGrids();
 
@@ -206,14 +278,12 @@ public class IgniteClusterActivateDeactivateTestWithPersistence extends IgniteCl
 
         checkNoCaches(srvs);
 
-        srv.cluster().active(true);
+        srv.cluster().state(activationMode);
 
         final int CACHES = withNewCaches ? 4 : 2;
 
-        for (int i = 0; i < srvs; i++) {
-            for (int c = 0; c < CACHES; c++)
-                checkCache(ignite(i), CACHE_NAME_PREFIX + c, true);
-        }
+        for (int i = 0; i < srvs; i++)
+            checkCachesOnNode(i, CACHES);
 
         DataStorageConfiguration dsCfg = srv.configuration().getDataStorageConfiguration();
 
@@ -223,46 +293,32 @@ public class IgniteClusterActivateDeactivateTestWithPersistence extends IgniteCl
 
         int nodes = srvs;
 
-        client = false;
-
-        startGrid(nodes++);
+        startGrid(nodes++, false);
 
-        for (int i = 0; i < nodes; i++) {
-            for (int c = 0; c < CACHES; c++)
-                checkCache(ignite(i), CACHE_NAME_PREFIX + c, true);
-        }
+        for (int i = 0; i < nodes; i++)
+            checkCachesOnNode(i, CACHES);
 
         checkCaches(nodes, CACHES);
 
-        client = true;
+        startGrid(nodes++, true);
 
-        startGrid(nodes++);
-
-        for (int c = 0; c < CACHES; c++)
-            checkCache(ignite(nodes - 1), CACHE_NAME_PREFIX + c, false);
+        checkCachesOnNode(nodes - 1, CACHES, false);
 
         checkCaches(nodes, CACHES);
 
-        for (int i = 0; i < nodes; i++) {
-            for (int c = 0; c < CACHES; c++)
-                checkCache(ignite(i), CACHE_NAME_PREFIX + c, true);
-        }
+        for (int i = 0; i < nodes; i++)
+            checkCachesOnNode(i, CACHES);
 
         checkCachesData(cacheData, dsCfg);
     }
 
-    /**
-     * Verifies correctness of BaselineTopology checks when working in persistent mode.
-     */
-    @Override protected void doFinalChecks() {
-        for (int i = 0; i < 4; i++) {
-            int j = i;
 
-            assertThrowsAnyCause(log, () -> {
-                startGrid(j);
+    /** {@inheritDoc} */
+    @Override protected void doFinalChecks(int startNodes, int nodesCnt) {
+        for (int i = 0; i < startNodes; i++) {
+            int j = i;
 
-                return null;
-            }, IgniteSpiException.class, "not compatible");
+            assertThrowsAnyCause(log, () -> startGrid(j), IgniteSpiException.class, "not compatible");
         }
     }
 
@@ -271,7 +327,20 @@ public class IgniteClusterActivateDeactivateTestWithPersistence extends IgniteCl
      */
     @Test
     public void testClientJoinsWhenActivationIsInProgress() throws Exception {
-        startGridsAndLoadData(5);
+        checkClientJoinsWhenActivationIsInProgress(ACTIVE);
+    }
+
+    /** */
+    @Test
+    public void testClientJoinsWhenActivationReanOnlyIsInProgress() throws Exception {
+        checkClientJoinsWhenActivationIsInProgress(ACTIVE_READ_ONLY);
+    }
+
+    /** */
+    private void checkClientJoinsWhenActivationIsInProgress(ClusterState state) throws Exception {
+        assertActive(state);
+
+        startGridsAndLoadData(5, state);
 
         stopAllGrids();
 
@@ -279,8 +348,8 @@ public class IgniteClusterActivateDeactivateTestWithPersistence extends IgniteCl
 
         final CountDownLatch clientStartLatch = new CountDownLatch(1);
 
-        IgniteInternalFuture clStartFut = GridTestUtils.runAsync(new Runnable() {
-            @Override public void run() {
+        IgniteInternalFuture clStartFut = GridTestUtils.runAsync(
+            () -> {
                 try {
                     clientStartLatch.await();
 
@@ -293,17 +362,20 @@ public class IgniteClusterActivateDeactivateTestWithPersistence extends IgniteCl
                     IgniteCache<Object, Object> atomicCache = cl.cache(CACHE_NAME_PREFIX + '0');
                     IgniteCache<Object, Object> txCache = cl.cache(CACHE_NAME_PREFIX + '1');
 
-                    assertEquals(100, atomicCache.size());
-                    assertEquals(100, txCache.size());
+                    assertEquals(state == ACTIVE ? 100 : 0, atomicCache.size());
+                    assertEquals(state == ACTIVE ? 100 : 0, txCache.size());
                 }
                 catch (Exception e) {
                     log.error("Error occurred", e);
+
+                    fail("Error occurred in client thread. Msg: " + e.getMessage());
                 }
-            }
-        }, "client-starter-thread");
+            },
+            "client-starter-thread"
+        );
 
         clientStartLatch.countDown();
-        srv.cluster().active(true);
+        srv.cluster().state(state);
 
         clStartFut.get();
     }
@@ -330,11 +402,26 @@ public class IgniteClusterActivateDeactivateTestWithPersistence extends IgniteCl
      */
     @Test
     public void testActivateCacheRestoreConfigurationConflict() throws Exception {
+        checkActivateCacheRestoreConfigurationConflict(ACTIVE);
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    @Test
+    public void testActivateReadOnlyCacheRestoreConfigurationConflict() throws Exception {
+        checkActivateCacheRestoreConfigurationConflict(ACTIVE_READ_ONLY);
+    }
+
+    /** */
+    private void checkActivateCacheRestoreConfigurationConflict(ClusterState state) throws Exception {
+        assertActive(state);
+
         final int SRVS = 3;
 
         Ignite srv = startGrids(SRVS);
 
-        srv.cluster().active(true);
+        srv.cluster().state(state);
 
         CacheConfiguration ccfg = new CacheConfiguration(DEFAULT_CACHE_NAME)
             .setAtomicityMode(CacheAtomicityMode.TRANSACTIONAL);
@@ -350,15 +437,7 @@ public class IgniteClusterActivateDeactivateTestWithPersistence extends IgniteCl
 
         ccfgs = new CacheConfiguration[] {ccfg};
 
-        try {
-            startGrids(SRVS);
-
-            fail();
-        }
-        catch (Exception e) {
-            assertTrue(
-                X.cause(e, IgniteCheckedException.class).getMessage().contains("Failed to start configured cache."));
-        }
+        assertThrowsAnyCause(log, () -> startGrids(SRVS), IgniteCheckedException.class, "Failed to start configured cache.");
     }
 
     /**
@@ -371,9 +450,9 @@ public class IgniteClusterActivateDeactivateTestWithPersistence extends IgniteCl
     public void testDeactivateDuringEvictionAndRebalance() throws Exception {
         Assume.assumeFalse("https://issues.apache.org/jira/browse/IGNITE-7384", MvccFeatureChecker.forcedMvcc());
 
-        IgniteEx srv = (IgniteEx) startGrids(3);
+        IgniteEx srv = startGrids(3);
 
-        srv.cluster().active(true);
+        srv.cluster().state(ACTIVE);
 
         CacheConfiguration ccfg = new CacheConfiguration(DEFAULT_CACHE_NAME)
             .setBackups(1)
@@ -406,19 +485,23 @@ public class IgniteClusterActivateDeactivateTestWithPersistence extends IgniteCl
 
         Set<Integer> addedKeys = new GridConcurrentHashSet<>();
 
-        IgniteInternalFuture cacheLoadFuture = GridTestUtils.runMultiThreadedAsync(() -> {
-            while (!stop.get()) {
-                int key = keyCounter.incrementAndGet();
-                try {
-                    cache.put(key, key);
+        IgniteInternalFuture cacheLoadFuture = GridTestUtils.runMultiThreadedAsync(
+            () -> {
+                while (!stop.get()) {
+                    int key = keyCounter.incrementAndGet();
+                    try {
+                        cache.put(key, key);
 
-                    addedKeys.add(key);
+                        addedKeys.add(key);
 
-                    Thread.sleep(10);
+                        Thread.sleep(10);
+                    }
+                    catch (Exception ignored) { }
                 }
-                catch (Exception ignored) { }
-            }
-        }, 2, "cache-load");
+            },
+            2,
+            "cache-load"
+        );
 
         stopGrid(2);
 
@@ -434,9 +517,9 @@ public class IgniteClusterActivateDeactivateTestWithPersistence extends IgniteCl
         cacheLoadFuture.get();
 
         // Deactivate and activate again.
-        srv.cluster().active(false);
+        srv.cluster().state(INACTIVE);
 
-        srv.cluster().active(true);
+        srv.cluster().state(ACTIVE);
 
         awaitPartitionMapExchange();
 
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/baseline/ClusterActivationEventTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/baseline/ClusterActivationEventTest.java
index d30f2ad..323b385 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/baseline/ClusterActivationEventTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/baseline/ClusterActivationEventTest.java
@@ -18,24 +18,119 @@
 package org.apache.ignite.internal.processors.cache.persistence.baseline;
 
 import java.util.Collection;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.Map;
+import java.util.stream.Collectors;
+import java.util.stream.IntStream;
 import org.apache.ignite.Ignite;
 import org.apache.ignite.IgniteCluster;
-import org.apache.ignite.IgniteInterruptedException;
+import org.apache.ignite.cluster.ClusterState;
+import org.apache.ignite.configuration.CacheConfiguration;
 import org.apache.ignite.configuration.IgniteConfiguration;
 import org.apache.ignite.events.Event;
 import org.apache.ignite.events.EventType;
+import org.apache.ignite.failure.StopNodeOrHaltFailureHandler;
+import org.apache.ignite.internal.IgniteEx;
+import org.apache.ignite.internal.IgniteInterruptedCheckedException;
 import org.apache.ignite.internal.util.typedef.F;
+import org.apache.ignite.internal.util.typedef.G;
+import org.apache.ignite.internal.util.typedef.internal.U;
+import org.apache.ignite.lang.IgniteFuture;
 import org.apache.ignite.lang.IgnitePredicate;
 import org.apache.ignite.testframework.junits.common.GridCommonAbstractTest;
 import org.junit.Test;
 
+import static java.util.Comparator.comparingLong;
+import static org.apache.ignite.cluster.ClusterState.ACTIVE;
+import static org.apache.ignite.cluster.ClusterState.INACTIVE;
+import static org.apache.ignite.events.EventType.EVT_CLUSTER_ACTIVATED;
+import static org.apache.ignite.events.EventType.EVT_CLUSTER_DEACTIVATED;
+
 /**
  * Tests cluster activation events.
  */
 public class ClusterActivationEventTest extends GridCommonAbstractTest {
+    /** Nodes count. */
+    private static final int NODES_CNT = 2;
+
+    /** Listener delay. */
+    private static final long DELAY = 1000L;
+
+    /** Logger message format. */
+    private static final String LOG_MESSAGE_FORMAT = "Received event [id=%s, type=%s], msg=%s";
+
+    /** */
+    private final IgnitePredicate<? extends Event> lsnr = (evt) -> {
+        log.info(String.format(LOG_MESSAGE_FORMAT, evt.id(), evt.type(), evt.message()));
+
+        return true;
+    };
+
+    /** */
+    private final IgnitePredicate<? extends Event> delayLsnr = (evt) -> {
+        log.info(String.format(LOG_MESSAGE_FORMAT, evt.id(), evt.type(), evt.message()));
+
+        try {
+            U.sleep(DELAY);
+        }
+        catch (IgniteInterruptedCheckedException e) {
+            log.error("Sleep interrupted", e);
+        }
+
+        return true;
+    };
+
     /** {@inheritDoc} */
     @Override protected IgniteConfiguration getConfiguration(String igniteInstanceName) throws Exception {
-        return super.getConfiguration(igniteInstanceName).setIncludeEventTypes(EventType.EVTS_ALL);
+        return super.getConfiguration(igniteInstanceName)
+            .setIncludeEventTypes(EventType.EVTS_ALL)
+            .setCacheConfiguration(new CacheConfiguration(DEFAULT_CACHE_NAME))
+            .setFailureHandler(new StopNodeOrHaltFailureHandler());
+    }
+
+    /** {@inheritDoc} */
+    @Override protected void beforeTestsStarted() throws Exception {
+        super.beforeTestsStarted();
+
+        startGrids(NODES_CNT);
+
+        startClientGrid(NODES_CNT);
+    }
+
+    /** {@inheritDoc} */
+    @Override protected void afterTestsStopped() throws Exception {
+        stopAllGrids();
+
+        super.afterTestsStopped();
+    }
+
+    /** {@inheritDoc} */
+    @Override protected void beforeTest() throws Exception {
+        super.beforeTest();
+
+        for (Ignite ignite : G.allGrids()) {
+            ignite.events().stopLocalListen(lsnr);
+            ignite.events().stopLocalListen(delayLsnr);
+        }
+
+        grid(0).cluster().state(ACTIVE);
+
+        grid(0).cache(DEFAULT_CACHE_NAME).removeAll();
+
+        Map<Integer, Integer> vals = IntStream.range(0, 100).boxed().collect(Collectors.toMap(i -> i, i -> i));
+
+        grid(0).cachex(DEFAULT_CACHE_NAME).putAll(vals);
+    }
+
+    /** {@inheritDoc} */
+    @Override protected void afterTest() throws Exception {
+        for (Ignite ignite : G.allGrids()) {
+            ignite.events().stopLocalListen(lsnr);
+            ignite.events().stopLocalListen(delayLsnr);
+        }
+
+        super.afterTest();
     }
 
     /**
@@ -43,20 +138,7 @@ public class ClusterActivationEventTest extends GridCommonAbstractTest {
      */
     @Test
     public void testClusterActivation() throws Exception {
-        ClusterActivationTestTask task = new ClusterActivationTestTask() {
-            @Override public void execute(IgniteCluster cluster) throws Exception {
-                deactivateCluster(cluster);
-
-                activateCluster(cluster);
-            }
-        };
-
-        IgnitePredicate<? extends Event> lsnr = (evt) -> {
-            System.out.println("Received event [id=" + evt.id().toString() + ", type=" + evt.type() + ']' + ", msg=" + evt.message() + ']');
-            return true;
-        };
-
-        checkClusterActivation(task, lsnr, EventType.EVT_CLUSTER_ACTIVATED, 1);
+        clusterChangeState(INACTIVE, ACTIVE, EVT_CLUSTER_ACTIVATED);
     }
 
     /**
@@ -64,18 +146,7 @@ public class ClusterActivationEventTest extends GridCommonAbstractTest {
      */
     @Test
     public void testClusterDeactivation() throws Exception {
-        ClusterActivationTestTask task = new ClusterActivationTestTask() {
-            @Override public void execute(IgniteCluster cluster) throws Exception {
-                deactivateCluster(cluster);
-            }
-        };
-
-        IgnitePredicate<? extends Event> lsnr = (evt) -> {
-            System.out.println("Received event [id=" + evt.id().toString() + ", type=" + evt.type() + ']' + ", msg=" + evt.message() + ']');
-            return true;
-        };
-
-        checkClusterActivation(task, lsnr, EventType.EVT_CLUSTER_DEACTIVATED, 1);
+        clusterChangeState(ACTIVE, INACTIVE, EVT_CLUSTER_DEACTIVATED);
     }
 
     /**
@@ -83,22 +154,15 @@ public class ClusterActivationEventTest extends GridCommonAbstractTest {
      */
     @Test
     public void testClusterDoubleActivation() throws Exception {
-        ClusterActivationTestTask task = new ClusterActivationTestTask() {
-            @Override public void execute(IgniteCluster cluster) throws Exception {
-                deactivateCluster(cluster);
-
-                activateCluster(cluster);
-
-                activateCluster(cluster);
-            }
-        };
-
-        IgnitePredicate<? extends Event> lsnr = (evt) -> {
-            System.out.println("Received event [id=" + evt.id().toString() + ", type=" + evt.type() + ']' + ", msg=" + evt.message() + ']');
-            return true;
-        };
+        clusterChangeStateTwice(INACTIVE, ACTIVE, EVT_CLUSTER_ACTIVATED);
+    }
 
-        checkClusterActivation(task, lsnr, EventType.EVT_CLUSTER_ACTIVATED, 1);
+    /**
+     * @throws Exception If failed.
+     */
+    @Test
+    public void testClusterDoubleDeactivation() throws Exception {
+        clusterChangeStateTwice(ACTIVE, INACTIVE, EVT_CLUSTER_DEACTIVATED);
     }
 
     /**
@@ -106,98 +170,115 @@ public class ClusterActivationEventTest extends GridCommonAbstractTest {
      */
     @Test
     public void testClusterActivationListenerSleep() throws Exception {
-        ClusterActivationTestTask task = new ClusterActivationTestTask() {
-            @Override public void execute(IgniteCluster cluster) throws Exception {
-                deactivateCluster(cluster);
+        clusterChangeStateWithDelay(INACTIVE, ACTIVE, EVT_CLUSTER_ACTIVATED);
+    }
 
-                activateCluster(cluster);
-            }
-        };
+    /**
+     * @throws Exception If failed.
... 2146 lines suppressed ...