You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@ignite.apache.org by sb...@apache.org on 2017/07/05 11:23:43 UTC

[01/33] ignite git commit: Pass io policy in GridMessageListener.

Repository: ignite
Updated Branches:
  refs/heads/ignite-2.1.2-exchange 377cc9d7a -> 5b2400ab4


Pass io policy in GridMessageListener.


Project: http://git-wip-us.apache.org/repos/asf/ignite/repo
Commit: http://git-wip-us.apache.org/repos/asf/ignite/commit/bdd31af7
Tree: http://git-wip-us.apache.org/repos/asf/ignite/tree/bdd31af7
Diff: http://git-wip-us.apache.org/repos/asf/ignite/diff/bdd31af7

Branch: refs/heads/ignite-2.1.2-exchange
Commit: bdd31af760f745c9ab05f75d100649a30dfe4f1e
Parents: 7db925c
Author: sboikov <sb...@gridgain.com>
Authored: Mon Jul 3 15:45:04 2017 +0300
Committer: sboikov <sb...@gridgain.com>
Committed: Mon Jul 3 15:45:04 2017 +0300

----------------------------------------------------------------------
 .../checkpoint/GridCheckpointManager.java       |  2 +-
 .../managers/communication/GridIoManager.java   | 10 +--
 .../communication/GridMessageListener.java      |  3 +-
 .../deployment/GridDeploymentCommunication.java |  4 +-
 .../eventstorage/GridEventStorageManager.java   |  4 +-
 .../processors/cache/GridCacheIoManager.java    | 77 +++++++++++---------
 .../cache/binary/BinaryMetadataTransport.java   |  4 +-
 .../cache/transactions/IgniteTxManager.java     |  2 +-
 .../processors/cluster/ClusterProcessor.java    |  2 +-
 .../continuous/GridContinuousProcessor.java     |  4 +-
 .../datastreamer/DataStreamProcessor.java       |  2 +-
 .../datastreamer/DataStreamerImpl.java          |  2 +-
 .../processors/igfs/IgfsDataManager.java        |  2 +-
 .../igfs/IgfsFragmentizerManager.java           |  4 +-
 .../processors/job/GridJobProcessor.java        |  8 +-
 .../GridMarshallerMappingProcessor.java         |  4 +-
 .../processors/query/GridQueryProcessor.java    |  2 +-
 .../handlers/task/GridTaskCommandHandler.java   |  4 +-
 .../processors/task/GridTaskProcessor.java      |  6 +-
 .../jobstealing/JobStealingCollisionSpi.java    |  2 +-
 ...idCommunicationManagerListenersSelfTest.java |  2 +-
 .../GridCommunicationSendMessageSelfTest.java   |  2 +-
 .../cache/GridCachePartitionedGetSelfTest.java  |  2 +-
 ...lerCacheClientRequestsMappingOnMissTest.java |  6 +-
 ...naryObjectMetadataExchangeMultinodeTest.java |  6 +-
 ...DeadlockDetectionMessageMarshallingTest.java |  2 +-
 .../communication/GridIoManagerBenchmark.java   |  4 +-
 .../communication/GridIoManagerBenchmark0.java  | 12 +--
 .../communication/GridCacheMessageSelfTest.java |  2 +-
 .../testframework/GridSpiTestContext.java       |  5 +-
 .../hadoop/shuffle/HadoopShuffle.java           |  2 +-
 .../query/h2/opt/GridH2IndexBase.java           |  2 +-
 .../query/h2/twostep/GridMapQueryExecutor.java  |  2 +-
 .../h2/twostep/GridReduceQueryExecutor.java     |  2 +-
 34 files changed, 106 insertions(+), 93 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/bdd31af7/modules/core/src/main/java/org/apache/ignite/internal/managers/checkpoint/GridCheckpointManager.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/managers/checkpoint/GridCheckpointManager.java b/modules/core/src/main/java/org/apache/ignite/internal/managers/checkpoint/GridCheckpointManager.java
index 782ee5e..aae1a3b 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/managers/checkpoint/GridCheckpointManager.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/managers/checkpoint/GridCheckpointManager.java
@@ -464,7 +464,7 @@ public class GridCheckpointManager extends GridManagerAdapter<CheckpointSpi> {
          * @param msg Received message.
          */
         @SuppressWarnings({"MismatchedQueryAndUpdateOfCollection"})
-        @Override public void onMessage(UUID nodeId, Object msg) {
+        @Override public void onMessage(UUID nodeId, Object msg, byte plc) {
             GridCheckpointRequest req = (GridCheckpointRequest)msg;
 
             if (log.isDebugEnabled())

http://git-wip-us.apache.org/repos/asf/ignite/blob/bdd31af7/modules/core/src/main/java/org/apache/ignite/internal/managers/communication/GridIoManager.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/managers/communication/GridIoManager.java b/modules/core/src/main/java/org/apache/ignite/internal/managers/communication/GridIoManager.java
index a1ddaf4..3ff44a0 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/managers/communication/GridIoManager.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/managers/communication/GridIoManager.java
@@ -337,7 +337,7 @@ public class GridIoManager extends GridManagerAdapter<CommunicationSpi<Serializa
             log.debug(startInfo());
 
         addMessageListener(GridTopic.TOPIC_IO_TEST, new GridMessageListener() {
-            @Override public void onMessage(UUID nodeId, Object msg) {
+            @Override public void onMessage(UUID nodeId, Object msg, byte plc) {
                 ClusterNode node = ctx.discovery().node(nodeId);
 
                 if (node == null)
@@ -1553,7 +1553,7 @@ public class GridIoManager extends GridManagerAdapter<CommunicationSpi<Serializa
             CUR_PLC.set(plc);
 
         try {
-            lsnr.onMessage(nodeId, msg);
+            lsnr.onMessage(nodeId, msg, plc);
         }
         finally {
             if (change)
@@ -2323,14 +2323,14 @@ public class GridIoManager extends GridManagerAdapter<CommunicationSpi<Serializa
          * @param nodeId Node ID.
          * @param msg Message.
          */
-        @Override public void onMessage(UUID nodeId, Object msg) {
+        @Override public void onMessage(UUID nodeId, Object msg, byte plc) {
             GridMessageListener[] arr0 = arr;
 
             if (arr0 == null)
                 return;
 
             for (GridMessageListener l : arr0)
-                l.onMessage(nodeId, msg);
+                l.onMessage(nodeId, msg, plc);
         }
 
         /**
@@ -2430,7 +2430,7 @@ public class GridIoManager extends GridManagerAdapter<CommunicationSpi<Serializa
         /** {@inheritDoc} */
         @SuppressWarnings({"SynchronizationOnLocalVariableOrMethodParameter", "ConstantConditions",
             "OverlyStrongTypeCast"})
-        @Override public void onMessage(UUID nodeId, Object msg) {
+        @Override public void onMessage(UUID nodeId, Object msg, byte plc) {
             if (!(msg instanceof GridIoUserMessage)) {
                 U.error(log, "Received unknown message (potentially fatal problem): " + msg);
 

http://git-wip-us.apache.org/repos/asf/ignite/blob/bdd31af7/modules/core/src/main/java/org/apache/ignite/internal/managers/communication/GridMessageListener.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/managers/communication/GridMessageListener.java b/modules/core/src/main/java/org/apache/ignite/internal/managers/communication/GridMessageListener.java
index 3993591..c7de57c 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/managers/communication/GridMessageListener.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/managers/communication/GridMessageListener.java
@@ -30,6 +30,7 @@ public interface GridMessageListener extends EventListener {
      * @param nodeId ID of node that sent the message. Note that may have already
      *      left topology by the time this message is received.
      * @param msg Message received.
+     * @param plc Message policy (pool).
      */
-    public void onMessage(UUID nodeId, Object msg);
+    public void onMessage(UUID nodeId, Object msg, byte plc);
 }
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/ignite/blob/bdd31af7/modules/core/src/main/java/org/apache/ignite/internal/managers/deployment/GridDeploymentCommunication.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/managers/deployment/GridDeploymentCommunication.java b/modules/core/src/main/java/org/apache/ignite/internal/managers/deployment/GridDeploymentCommunication.java
index 23d186a..2a5f7ca 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/managers/deployment/GridDeploymentCommunication.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/managers/deployment/GridDeploymentCommunication.java
@@ -83,7 +83,7 @@ class GridDeploymentCommunication {
         this.log = log.getLogger(getClass());
 
         peerLsnr = new GridMessageListener() {
-            @Override public void onMessage(UUID nodeId, Object msg) {
+            @Override public void onMessage(UUID nodeId, Object msg, byte plc) {
                 processDeploymentRequest(nodeId, msg);
             }
         };
@@ -422,7 +422,7 @@ class GridDeploymentCommunication {
         };
 
         GridMessageListener resLsnr = new GridMessageListener() {
-            @Override public void onMessage(UUID nodeId, Object msg) {
+            @Override public void onMessage(UUID nodeId, Object msg, byte plc) {
                 assert nodeId != null;
                 assert msg != null;
 

http://git-wip-us.apache.org/repos/asf/ignite/blob/bdd31af7/modules/core/src/main/java/org/apache/ignite/internal/managers/eventstorage/GridEventStorageManager.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/managers/eventstorage/GridEventStorageManager.java b/modules/core/src/main/java/org/apache/ignite/internal/managers/eventstorage/GridEventStorageManager.java
index bd43e43..7836004 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/managers/eventstorage/GridEventStorageManager.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/managers/eventstorage/GridEventStorageManager.java
@@ -1008,7 +1008,7 @@ public class GridEventStorageManager extends GridManagerAdapter<EventStorageSpi>
 
         GridMessageListener resLsnr = new GridMessageListener() {
             @SuppressWarnings("deprecation")
-            @Override public void onMessage(UUID nodeId, Object msg) {
+            @Override public void onMessage(UUID nodeId, Object msg, byte plc) {
                 assert nodeId != null;
                 assert msg != null;
 
@@ -1185,7 +1185,7 @@ public class GridEventStorageManager extends GridManagerAdapter<EventStorageSpi>
      */
     private class RequestListener implements GridMessageListener {
         /** {@inheritDoc} */
-        @Override public void onMessage(UUID nodeId, Object msg) {
+        @Override public void onMessage(UUID nodeId, Object msg, byte plc) {
             assert nodeId != null;
             assert msg != null;
 

http://git-wip-us.apache.org/repos/asf/ignite/blob/bdd31af7/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheIoManager.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheIoManager.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheIoManager.java
index a920bd0..49cfcdd 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheIoManager.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheIoManager.java
@@ -155,7 +155,7 @@ public class GridCacheIoManager extends GridCacheSharedManagerAdapter {
 
     /** Message listener. */
     private GridMessageListener lsnr = new GridMessageListener() {
-        @Override public void onMessage(final UUID nodeId, final Object msg) {
+        @Override public void onMessage(final UUID nodeId, final Object msg, final byte plc) {
             if (log.isDebugEnabled())
                 log.debug("Received unordered cache communication message [nodeId=" + nodeId +
                     ", locId=" + cctx.localNodeId() + ", msg=" + msg + ']');
@@ -196,7 +196,7 @@ public class GridCacheIoManager extends GridCacheSharedManagerAdapter {
                             @Override public void apply(IgniteInternalFuture<?> fut) {
                                 cctx.kernalContext().closure().runLocalSafe(new Runnable() {
                                     @Override public void run() {
-                                        handleMessage(nodeId, cacheMsg);
+                                        handleMessage(nodeId, cacheMsg, plc);
                                     }
                                 });
                             }
@@ -269,40 +269,48 @@ public class GridCacheIoManager extends GridCacheSharedManagerAdapter {
                                     log.debug(msg0.toString());
                                 }
 
-                                handleMessage(nodeId, cacheMsg);
+                                handleMessage(nodeId, cacheMsg, plc);
                             }
                         };
 
                         if (stripe >= 0)
                             cctx.kernalContext().getStripedExecutorService().execute(stripe, c);
-                        else
-                            cctx.kernalContext().closure().runLocalSafe(c);
+                        else {
+                            try {
+                                cctx.kernalContext().pools().poolForPolicy(plc).execute(c);
+                            }
+                            catch (IgniteCheckedException e) {
+                                U.error(cacheMsg.messageLogger(cctx), "Failed to get pool for policy: " + plc, e);
+                            }
+                        }
                     }
                 });
 
                 return;
             }
 
-            handleMessage(nodeId, cacheMsg);
+            handleMessage(nodeId, cacheMsg, plc);
         }
     };
 
     /**
      * @param nodeId Sender node ID.
      * @param cacheMsg Message.
+     * @param plc Message policy.
      */
     @SuppressWarnings("unchecked")
-    private void handleMessage(UUID nodeId, GridCacheMessage cacheMsg) {
-        handleMessage(nodeId, cacheMsg, cacheMsg.cacheGroupMessage() ? grpHandlers : cacheHandlers);
+    private void handleMessage(UUID nodeId, GridCacheMessage cacheMsg, byte plc) {
+        handleMessage(nodeId, cacheMsg, cacheMsg.cacheGroupMessage() ? grpHandlers : cacheHandlers, plc);
     }
 
     /**
      * @param nodeId Sender node ID.
      * @param cacheMsg Message.
      * @param msgHandlers Message handlers.
+     * @param plc Message policy.
      */
     @SuppressWarnings("unchecked")
-    private void handleMessage(UUID nodeId, GridCacheMessage cacheMsg, MessageHandlers msgHandlers) {
+    private void handleMessage(UUID nodeId, GridCacheMessage cacheMsg, MessageHandlers msgHandlers, byte plc) {
         Lock lock = rw.readLock();
 
         lock.lock();
@@ -356,7 +364,7 @@ public class GridCacheIoManager extends GridCacheSharedManagerAdapter {
                 return;
             }
 
-            onMessage0(nodeId, cacheMsg, c);
+            onMessage0(nodeId, cacheMsg, c, plc);
         }
         finally {
             lock.unlock();
@@ -517,10 +525,11 @@ public class GridCacheIoManager extends GridCacheSharedManagerAdapter {
      * @param nodeId Node ID.
      * @param cacheMsg Cache message.
      * @param c Handler closure.
+     * @param plc Message policy.
      */
     @SuppressWarnings({"unchecked", "ConstantConditions", "ThrowableResultOfMethodCallIgnored"})
     private void onMessage0(final UUID nodeId, final GridCacheMessage cacheMsg,
-        final IgniteBiInClosure<UUID, GridCacheMessage> c) {
+        final IgniteBiInClosure<UUID, GridCacheMessage> c, byte plc) {
         try {
             if (stopping) {
                 if (log.isDebugEnabled())
@@ -536,7 +545,7 @@ public class GridCacheIoManager extends GridCacheSharedManagerAdapter {
             unmarshall(nodeId, cacheMsg);
 
             if (cacheMsg.classError() != null)
-                processFailedMessage(nodeId, cacheMsg, c);
+                processFailedMessage(nodeId, cacheMsg, c, plc);
             else
                 processMessage(nodeId, cacheMsg, c);
         }
@@ -669,15 +678,17 @@ public class GridCacheIoManager extends GridCacheSharedManagerAdapter {
      *
      * @param nodeId Node ID.
      * @param msg Message.
+     * @param c Closure.
+     * @param plc Message policy.
      * @throws IgniteCheckedException If failed.
      */
-    private void processFailedMessage(UUID nodeId, GridCacheMessage msg, IgniteBiInClosure<UUID, GridCacheMessage> c)
+    private void processFailedMessage(UUID nodeId,
+        GridCacheMessage msg,
+        IgniteBiInClosure<UUID, GridCacheMessage> c,
+        byte plc)
         throws IgniteCheckedException {
         assert msg != null;
 
-        GridCacheContext ctx = msg instanceof GridCacheIdMessage ?
-            cctx.cacheContext(((GridCacheIdMessage)msg).cacheId()) : null;
-
         switch (msg.directType()) {
             case 30: {
                 GridDhtLockRequest req = (GridDhtLockRequest)msg;
@@ -688,9 +699,9 @@ public class GridCacheIoManager extends GridCacheSharedManagerAdapter {
                     req.futureId(),
                     req.miniId(),
                     0,
-                    ctx.deploymentEnabled());
+                    false);
 
-                sendResponseOnFailedMessage(nodeId, res, cctx, ctx.ioPolicy());
+                sendResponseOnFailedMessage(nodeId, res, cctx, plc);
             }
 
             break;
@@ -723,7 +734,7 @@ public class GridCacheIoManager extends GridCacheSharedManagerAdapter {
 
                 res.onError(req.classError());
 
-                sendResponseOnFailedMessage(nodeId, res, cctx, ctx.ioPolicy());
+                sendResponseOnFailedMessage(nodeId, res, cctx, plc);
 
                 if (req.nearNodeId() != null) {
                     GridDhtAtomicNearResponse nearRes = new GridDhtAtomicNearResponse(req.cacheId(),
@@ -734,7 +745,7 @@ public class GridCacheIoManager extends GridCacheSharedManagerAdapter {
 
                     nearRes.errors(new UpdateErrors(req.classError()));
 
-                    sendResponseOnFailedMessage(req.nearNodeId(), nearRes, cctx, ctx.ioPolicy());
+                    sendResponseOnFailedMessage(req.nearNodeId(), nearRes, cctx, plc);
                 }
             }
 
@@ -753,7 +764,7 @@ public class GridCacheIoManager extends GridCacheSharedManagerAdapter {
 
                 res.error(req.classError());
 
-                sendResponseOnFailedMessage(nodeId, res, cctx, ctx.ioPolicy());
+                sendResponseOnFailedMessage(nodeId, res, cctx, plc);
             }
 
             break;
@@ -770,7 +781,7 @@ public class GridCacheIoManager extends GridCacheSharedManagerAdapter {
 
                 res.error(req.classError());
 
-                sendResponseOnFailedMessage(nodeId, res, cctx, ctx.ioPolicy());
+                sendResponseOnFailedMessage(nodeId, res, cctx, plc);
             }
 
             break;
@@ -793,7 +804,7 @@ public class GridCacheIoManager extends GridCacheSharedManagerAdapter {
 
                 res.error(req.classError());
 
-                sendResponseOnFailedMessage(nodeId, res, cctx, ctx.ioPolicy());
+                sendResponseOnFailedMessage(nodeId, res, cctx, plc);
             }
 
             break;
@@ -831,7 +842,7 @@ public class GridCacheIoManager extends GridCacheSharedManagerAdapter {
                     null,
                     false);
 
-                sendResponseOnFailedMessage(nodeId, res, cctx, ctx.ioPolicy());
+                sendResponseOnFailedMessage(nodeId, res, cctx, plc);
             }
 
             break;
@@ -872,7 +883,7 @@ public class GridCacheIoManager extends GridCacheSharedManagerAdapter {
                     cctx.node(nodeId),
                     TOPIC_CACHE.topic(QUERY_TOPIC_PREFIX, nodeId, req.id()),
                     res,
-                    ctx.ioPolicy(),
+                    plc,
                     Long.MAX_VALUE);
             }
 
@@ -897,7 +908,7 @@ public class GridCacheIoManager extends GridCacheSharedManagerAdapter {
 
                 res.error(req.classError());
 
-                sendResponseOnFailedMessage(nodeId, res, cctx, ctx.ioPolicy());
+                sendResponseOnFailedMessage(nodeId, res, cctx, plc);
             }
 
             break;
@@ -935,7 +946,7 @@ public class GridCacheIoManager extends GridCacheSharedManagerAdapter {
 
                 res.error(req.classError());
 
-                sendResponseOnFailedMessage(nodeId, res, cctx, ctx.ioPolicy());
+                sendResponseOnFailedMessage(nodeId, res, cctx, plc);
             }
 
             break;
@@ -953,7 +964,7 @@ public class GridCacheIoManager extends GridCacheSharedManagerAdapter {
 
                 res.error(req.classError());
 
-                sendResponseOnFailedMessage(nodeId, res, cctx, ctx.ioPolicy());
+                sendResponseOnFailedMessage(nodeId, res, cctx, plc);
             }
 
             break;
@@ -971,7 +982,7 @@ public class GridCacheIoManager extends GridCacheSharedManagerAdapter {
 
                 res.error(req.classError());
 
-                sendResponseOnFailedMessage(nodeId, res, cctx, ctx.ioPolicy());
+                sendResponseOnFailedMessage(nodeId, res, cctx, plc);
             }
 
             break;
@@ -987,7 +998,7 @@ public class GridCacheIoManager extends GridCacheSharedManagerAdapter {
 
                 res.onError(req.classError());
 
-                sendResponseOnFailedMessage(nodeId, res, cctx, ctx.ioPolicy());
+                sendResponseOnFailedMessage(nodeId, res, cctx, plc);
 
                 if (req.nearNodeId() != null) {
                     GridDhtAtomicNearResponse nearRes = new GridDhtAtomicNearResponse(req.cacheId(),
@@ -998,7 +1009,7 @@ public class GridCacheIoManager extends GridCacheSharedManagerAdapter {
 
                     nearRes.errors(new UpdateErrors(req.classError()));
 
-                    sendResponseOnFailedMessage(req.nearNodeId(), nearRes, cctx, ctx.ioPolicy());
+                    sendResponseOnFailedMessage(req.nearNodeId(), nearRes, cctx, plc);
                 }
             }
 
@@ -1540,7 +1551,7 @@ public class GridCacheIoManager extends GridCacheSharedManagerAdapter {
 
         /** {@inheritDoc} */
         @SuppressWarnings({"CatchGenericClass", "unchecked"})
-        @Override public void onMessage(final UUID nodeId, Object msg) {
+        @Override public void onMessage(final UUID nodeId, Object msg, byte plc) {
             if (log.isDebugEnabled())
                 log.debug("Received cache ordered message [nodeId=" + nodeId + ", msg=" + msg + ']');
 
@@ -1551,7 +1562,7 @@ public class GridCacheIoManager extends GridCacheSharedManagerAdapter {
             try {
                 GridCacheMessage cacheMsg = (GridCacheMessage)msg;
 
-                onMessage0(nodeId, cacheMsg, c);
+                onMessage0(nodeId, cacheMsg, c, plc);
             }
             finally {
                 lock.unlock();

http://git-wip-us.apache.org/repos/asf/ignite/blob/bdd31af7/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/binary/BinaryMetadataTransport.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/binary/BinaryMetadataTransport.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/binary/BinaryMetadataTransport.java
index 00c760f..5fd7295 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/binary/BinaryMetadataTransport.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/binary/BinaryMetadataTransport.java
@@ -561,7 +561,7 @@ final class BinaryMetadataTransport {
         }
 
         /** {@inheritDoc} */
-        @Override public void onMessage(UUID nodeId, Object msg) {
+        @Override public void onMessage(UUID nodeId, Object msg, byte plc) {
             assert msg instanceof MetadataRequestMessage : msg;
 
             MetadataRequestMessage msg0 = (MetadataRequestMessage) msg;
@@ -606,7 +606,7 @@ final class BinaryMetadataTransport {
     private final class MetadataResponseListener implements GridMessageListener {
 
         /** {@inheritDoc} */
-        @Override public void onMessage(UUID nodeId, Object msg) {
+        @Override public void onMessage(UUID nodeId, Object msg, byte plc) {
             assert msg instanceof MetadataResponseMessage : msg;
 
             MetadataResponseMessage msg0 = (MetadataResponseMessage) msg;

http://git-wip-us.apache.org/repos/asf/ignite/blob/bdd31af7/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/transactions/IgniteTxManager.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/transactions/IgniteTxManager.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/transactions/IgniteTxManager.java
index a9aa13d..3a3b766 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/transactions/IgniteTxManager.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/transactions/IgniteTxManager.java
@@ -2433,7 +2433,7 @@ public class IgniteTxManager extends GridCacheSharedManagerAdapter {
     private class DeadlockDetectionListener implements GridMessageListener {
         /** {@inheritDoc} */
         @SuppressWarnings("unchecked")
-        @Override public void onMessage(UUID nodeId, Object msg) {
+        @Override public void onMessage(UUID nodeId, Object msg, byte plc) {
             GridCacheMessage cacheMsg = (GridCacheMessage)msg;
 
             Throwable err = null;

http://git-wip-us.apache.org/repos/asf/ignite/blob/bdd31af7/modules/core/src/main/java/org/apache/ignite/internal/processors/cluster/ClusterProcessor.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cluster/ClusterProcessor.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cluster/ClusterProcessor.java
index ed83650..0bd2370 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cluster/ClusterProcessor.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cluster/ClusterProcessor.java
@@ -146,7 +146,7 @@ public class ClusterProcessor extends GridProcessorAdapter {
             EVT_NODE_FAILED, EVT_NODE_LEFT);
 
         ctx.io().addMessageListener(TOPIC_INTERNAL_DIAGNOSTIC, new GridMessageListener() {
-            @Override public void onMessage(UUID nodeId, Object msg) {
+            @Override public void onMessage(UUID nodeId, Object msg, byte plc) {
                 if (msg instanceof IgniteDiagnosticMessage) {
                     IgniteDiagnosticMessage msg0 = (IgniteDiagnosticMessage)msg;
 

http://git-wip-us.apache.org/repos/asf/ignite/blob/bdd31af7/modules/core/src/main/java/org/apache/ignite/internal/processors/continuous/GridContinuousProcessor.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/continuous/GridContinuousProcessor.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/continuous/GridContinuousProcessor.java
index f641399..b1d3442 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/continuous/GridContinuousProcessor.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/continuous/GridContinuousProcessor.java
@@ -289,7 +289,7 @@ public class GridContinuousProcessor extends GridProcessorAdapter {
             });
 
         ctx.io().addMessageListener(TOPIC_CONTINUOUS, new GridMessageListener() {
-            @Override public void onMessage(UUID nodeId, Object obj) {
+            @Override public void onMessage(UUID nodeId, Object obj, byte plc) {
                 GridContinuousMessage msg = (GridContinuousMessage)obj;
 
                 if (msg.data() == null && msg.dataBytes() != null) {
@@ -771,7 +771,7 @@ public class GridContinuousProcessor extends GridProcessorAdapter {
     private void registerMessageListener(GridContinuousHandler hnd) {
         if (hnd.orderedTopic() != null) {
             ctx.io().addMessageListener(hnd.orderedTopic(), new GridMessageListener() {
-                @Override public void onMessage(UUID nodeId, Object obj) {
+                @Override public void onMessage(UUID nodeId, Object obj, byte plc) {
                     GridContinuousMessage msg = (GridContinuousMessage)obj;
 
                     // Only notification can be ordered.

http://git-wip-us.apache.org/repos/asf/ignite/blob/bdd31af7/modules/core/src/main/java/org/apache/ignite/internal/processors/datastreamer/DataStreamProcessor.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/datastreamer/DataStreamProcessor.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/datastreamer/DataStreamProcessor.java
index c52f7ac..31ae1e4 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/datastreamer/DataStreamProcessor.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/datastreamer/DataStreamProcessor.java
@@ -82,7 +82,7 @@ public class DataStreamProcessor<K, V> extends GridProcessorAdapter {
 
         if (!ctx.clientNode()) {
             ctx.io().addMessageListener(TOPIC_DATASTREAM, new GridMessageListener() {
-                @Override public void onMessage(UUID nodeId, Object msg) {
+                @Override public void onMessage(UUID nodeId, Object msg, byte plc) {
                     assert msg instanceof DataStreamerRequest;
 
                     processRequest(nodeId, (DataStreamerRequest)msg);

http://git-wip-us.apache.org/repos/asf/ignite/blob/bdd31af7/modules/core/src/main/java/org/apache/ignite/internal/processors/datastreamer/DataStreamerImpl.java
----------------------------------------------------------------------
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 40988d3..ae441de 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
@@ -319,7 +319,7 @@ public class DataStreamerImpl<K, V> implements IgniteDataStreamer<K, V>, Delayed
         topic = TOPIC_DATASTREAM.topic(IgniteUuid.fromUuid(ctx.localNodeId()));
 
         ctx.io().addMessageListener(topic, new GridMessageListener() {
-            @Override public void onMessage(UUID nodeId, Object msg) {
+            @Override public void onMessage(UUID nodeId, Object msg, byte plc) {
                 assert msg instanceof DataStreamerResponse;
 
                 DataStreamerResponse res = (DataStreamerResponse)msg;

http://git-wip-us.apache.org/repos/asf/ignite/blob/bdd31af7/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/IgfsDataManager.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/IgfsDataManager.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/IgfsDataManager.java
index 8ae6db8..a4ea337 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/IgfsDataManager.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/IgfsDataManager.java
@@ -158,7 +158,7 @@ public class IgfsDataManager extends IgfsManager {
         topic = F.isEmpty(igfsName) ? TOPIC_IGFS : TOPIC_IGFS.topic(igfsName);
 
         igfsCtx.kernalContext().io().addMessageListener(topic, new GridMessageListener() {
-            @Override public void onMessage(UUID nodeId, Object msg) {
+            @Override public void onMessage(UUID nodeId, Object msg, byte plc) {
                 if (msg instanceof IgfsBlocksMessage)
                     processBlocksMessage(nodeId, (IgfsBlocksMessage)msg);
                 else if (msg instanceof IgfsAckMessage)

http://git-wip-us.apache.org/repos/asf/ignite/blob/bdd31af7/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/IgfsFragmentizerManager.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/IgfsFragmentizerManager.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/IgfsFragmentizerManager.java
index 7797f89..0c75bc3 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/IgfsFragmentizerManager.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/IgfsFragmentizerManager.java
@@ -453,7 +453,7 @@ public class IgfsFragmentizerManager extends IgfsManager {
         }
 
         /** {@inheritDoc} */
-        @Override public void onMessage(UUID nodeId, Object msg) {
+        @Override public void onMessage(UUID nodeId, Object msg, byte plc) {
             if (msg instanceof IgfsFragmentizerResponse) {
                 IgfsFragmentizerResponse res = (IgfsFragmentizerResponse)msg;
 
@@ -673,7 +673,7 @@ public class IgfsFragmentizerManager extends IgfsManager {
         }
 
         /** {@inheritDoc} */
-        @Override public void onMessage(UUID nodeId, Object msg) {
+        @Override public void onMessage(UUID nodeId, Object msg, byte plc) {
             if (msg instanceof IgfsFragmentizerRequest ||
                 msg instanceof IgfsSyncMessage) {
                 if (log.isDebugEnabled())

http://git-wip-us.apache.org/repos/asf/ignite/blob/bdd31af7/modules/core/src/main/java/org/apache/ignite/internal/processors/job/GridJobProcessor.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/job/GridJobProcessor.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/job/GridJobProcessor.java
index 408396a..5ae4da6 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/job/GridJobProcessor.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/job/GridJobProcessor.java
@@ -447,7 +447,7 @@ public class GridJobProcessor extends GridProcessorAdapter {
         final Condition cond = lock.newCondition();
 
         GridMessageListener msgLsnr = new GridMessageListener() {
-            @Override public void onMessage(UUID nodeId, Object msg) {
+            @Override public void onMessage(UUID nodeId, Object msg, byte plc) {
                 String err = null;
                 GridJobSiblingsResponse res = null;
 
@@ -1856,7 +1856,7 @@ public class GridJobProcessor extends GridProcessorAdapter {
      */
     private class JobSessionListener implements GridMessageListener {
         /** {@inheritDoc} */
-        @Override public void onMessage(UUID nodeId, Object msg) {
+        @Override public void onMessage(UUID nodeId, Object msg, byte plc) {
             assert nodeId != null;
             assert msg != null;
 
@@ -1872,7 +1872,7 @@ public class GridJobProcessor extends GridProcessorAdapter {
      */
     private class JobCancelListener implements GridMessageListener {
         /** {@inheritDoc} */
-        @Override public void onMessage(UUID nodeId, Object msg) {
+        @Override public void onMessage(UUID nodeId, Object msg, byte plc) {
             assert nodeId != null;
             assert msg != null;
 
@@ -1890,7 +1890,7 @@ public class GridJobProcessor extends GridProcessorAdapter {
      */
     private class JobExecutionListener implements GridMessageListener {
         /** {@inheritDoc} */
-        @Override public void onMessage(UUID nodeId, Object msg) {
+        @Override public void onMessage(UUID nodeId, Object msg, byte plc) {
             assert nodeId != null;
             assert msg != null;
 

http://git-wip-us.apache.org/repos/asf/ignite/blob/bdd31af7/modules/core/src/main/java/org/apache/ignite/internal/processors/marshaller/GridMarshallerMappingProcessor.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/marshaller/GridMarshallerMappingProcessor.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/marshaller/GridMarshallerMappingProcessor.java
index 8de6c49..2543042 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/marshaller/GridMarshallerMappingProcessor.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/marshaller/GridMarshallerMappingProcessor.java
@@ -168,7 +168,7 @@ public class GridMarshallerMappingProcessor extends GridProcessorAdapter {
         }
 
         /** {@inheritDoc} */
-        @Override public void onMessage(UUID nodeId, Object msg) {
+        @Override public void onMessage(UUID nodeId, Object msg, byte plc) {
             assert msg instanceof MissingMappingRequestMessage : msg;
 
             MissingMappingRequestMessage msg0 = (MissingMappingRequestMessage) msg;
@@ -200,7 +200,7 @@ public class GridMarshallerMappingProcessor extends GridProcessorAdapter {
      */
     private final class MissingMappingResponseListener implements GridMessageListener {
         /** {@inheritDoc} */
-        @Override public void onMessage(UUID nodeId, Object msg) {
+        @Override public void onMessage(UUID nodeId, Object msg, byte plc) {
             assert msg instanceof MissingMappingResponseMessage : msg;
 
             MissingMappingResponseMessage msg0 = (MissingMappingResponseMessage) msg;

http://git-wip-us.apache.org/repos/asf/ignite/blob/bdd31af7/modules/core/src/main/java/org/apache/ignite/internal/processors/query/GridQueryProcessor.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/query/GridQueryProcessor.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/query/GridQueryProcessor.java
index dd07584..d55a129 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/query/GridQueryProcessor.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/query/GridQueryProcessor.java
@@ -207,7 +207,7 @@ public class GridQueryProcessor extends GridProcessorAdapter {
         valCtx = new CacheQueryObjectValueContext(ctx);
 
         ioLsnr = new GridMessageListener() {
-            @Override public void onMessage(UUID nodeId, Object msg) {
+            @Override public void onMessage(UUID nodeId, Object msg, byte plc) {
                 if (msg instanceof SchemaOperationStatusMessage) {
                     SchemaOperationStatusMessage msg0 = (SchemaOperationStatusMessage)msg;
 

http://git-wip-us.apache.org/repos/asf/ignite/blob/bdd31af7/modules/core/src/main/java/org/apache/ignite/internal/processors/rest/handlers/task/GridTaskCommandHandler.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/rest/handlers/task/GridTaskCommandHandler.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/rest/handlers/task/GridTaskCommandHandler.java
index 99ba335..d9b49cb 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/rest/handlers/task/GridTaskCommandHandler.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/rest/handlers/task/GridTaskCommandHandler.java
@@ -109,7 +109,7 @@ public class GridTaskCommandHandler extends GridRestCommandHandlerAdapter {
         super(ctx);
 
         ctx.io().addMessageListener(TOPIC_REST, new GridMessageListener() {
-            @Override public void onMessage(UUID nodeId, Object msg) {
+            @Override public void onMessage(UUID nodeId, Object msg, byte plc) {
                 if (!(msg instanceof GridTaskResultRequest)) {
                     U.warn(log, "Received unexpected message instead of task result request: " + msg);
 
@@ -425,7 +425,7 @@ public class GridTaskCommandHandler extends GridRestCommandHandlerAdapter {
         final Condition cond = lock.newCondition();
 
         GridMessageListener msgLsnr = new GridMessageListener() {
-            @Override public void onMessage(UUID nodeId, Object msg) {
+            @Override public void onMessage(UUID nodeId, Object msg, byte plc) {
                 String err = null;
                 GridTaskResultResponse res = null;
 

http://git-wip-us.apache.org/repos/asf/ignite/blob/bdd31af7/modules/core/src/main/java/org/apache/ignite/internal/processors/task/GridTaskProcessor.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/task/GridTaskProcessor.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/task/GridTaskProcessor.java
index 6ae97dd..d7a022e 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/task/GridTaskProcessor.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/task/GridTaskProcessor.java
@@ -1282,7 +1282,7 @@ public class GridTaskProcessor extends GridProcessorAdapter {
         }
 
         /** {@inheritDoc} */
-        @Override public void onMessage(UUID nodeId, Object msg) {
+        @Override public void onMessage(UUID nodeId, Object msg, byte plc) {
             if (msg instanceof GridJobExecuteResponse)
                 processJobExecuteResponse(nodeId, (GridJobExecuteResponse)msg);
             else if (jobResOnly)
@@ -1326,7 +1326,7 @@ public class GridTaskProcessor extends GridProcessorAdapter {
      */
     private class JobSiblingsMessageListener implements GridMessageListener {
         /** {@inheritDoc} */
-        @Override public void onMessage(UUID nodeId, Object msg) {
+        @Override public void onMessage(UUID nodeId, Object msg, byte plc) {
             if (!(msg instanceof GridJobSiblingsRequest)) {
                 U.warn(log, "Received unexpected message instead of siblings request: " + msg);
 
@@ -1398,7 +1398,7 @@ public class GridTaskProcessor extends GridProcessorAdapter {
      */
     private class TaskCancelMessageListener implements GridMessageListener {
         /** {@inheritDoc} */
-        @Override public void onMessage(UUID nodeId, Object msg) {
+        @Override public void onMessage(UUID nodeId, Object msg, byte plc) {
             assert msg != null;
 
             if (!(msg instanceof GridTaskCancelRequest)) {

http://git-wip-us.apache.org/repos/asf/ignite/blob/bdd31af7/modules/core/src/main/java/org/apache/ignite/spi/collision/jobstealing/JobStealingCollisionSpi.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/spi/collision/jobstealing/JobStealingCollisionSpi.java b/modules/core/src/main/java/org/apache/ignite/spi/collision/jobstealing/JobStealingCollisionSpi.java
index 6f2c099..39a8e8c 100644
--- a/modules/core/src/main/java/org/apache/ignite/spi/collision/jobstealing/JobStealingCollisionSpi.java
+++ b/modules/core/src/main/java/org/apache/ignite/spi/collision/jobstealing/JobStealingCollisionSpi.java
@@ -648,7 +648,7 @@ public class JobStealingCollisionSpi extends IgniteSpiAdapter implements Collisi
 
         spiCtx.addMessageListener(
             msgLsnr = new GridMessageListener() {
-                @Override public void onMessage(UUID nodeId, Object msg) {
+                @Override public void onMessage(UUID nodeId, Object msg, byte plc) {
                     MessageInfo info = rcvMsgMap.get(nodeId);
 
                     if (info == null) {

http://git-wip-us.apache.org/repos/asf/ignite/blob/bdd31af7/modules/core/src/test/java/org/apache/ignite/internal/managers/communication/GridCommunicationManagerListenersSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/managers/communication/GridCommunicationManagerListenersSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/managers/communication/GridCommunicationManagerListenersSelfTest.java
index 1738da8..03b7921 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/managers/communication/GridCommunicationManagerListenersSelfTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/managers/communication/GridCommunicationManagerListenersSelfTest.java
@@ -159,7 +159,7 @@ public class GridCommunicationManagerListenersSelfTest extends GridCommonAbstrac
         }
 
         /** {@inheritDoc} */
-        @Override public void onMessage(UUID nodeId, Object msg) {
+        @Override public void onMessage(UUID nodeId, Object msg, byte plc) {
             // No-op.
         }
     }

http://git-wip-us.apache.org/repos/asf/ignite/blob/bdd31af7/modules/core/src/test/java/org/apache/ignite/internal/managers/communication/GridCommunicationSendMessageSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/managers/communication/GridCommunicationSendMessageSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/managers/communication/GridCommunicationSendMessageSelfTest.java
index 29b7847..3563c77 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/managers/communication/GridCommunicationSendMessageSelfTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/managers/communication/GridCommunicationSendMessageSelfTest.java
@@ -143,7 +143,7 @@ public class GridCommunicationSendMessageSelfTest extends GridCommonAbstractTest
         final CountDownLatch latch = new CountDownLatch(SAMPLE_CNT);
 
         mgr1.addMessageListener(topic, new GridMessageListener() {
-            @Override public void onMessage(UUID nodeId, Object msg) {
+            @Override public void onMessage(UUID nodeId, Object msg, byte plc) {
                 if (msgCls.isInstance(msg))
                     latch.countDown();
             }

http://git-wip-us.apache.org/repos/asf/ignite/blob/bdd31af7/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridCachePartitionedGetSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridCachePartitionedGetSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridCachePartitionedGetSelfTest.java
index 308f2b4..d5988f1 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridCachePartitionedGetSelfTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridCachePartitionedGetSelfTest.java
@@ -224,7 +224,7 @@ public class GridCachePartitionedGetSelfTest extends GridCommonAbstractTest {
                 ((IgniteKernal)g).context().io().addMessageListener(
                     TOPIC_CACHE,
                     new GridMessageListener() {
-                        @Override public void onMessage(UUID nodeId, Object msg) {
+                        @Override public void onMessage(UUID nodeId, Object msg, byte plc) {
                             info("Received message from node [nodeId=" + nodeId + ", msg=" + msg + ']');
 
                             if (msg instanceof GridNearSingleGetRequest) {

http://git-wip-us.apache.org/repos/asf/ignite/blob/bdd31af7/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/IgniteMarshallerCacheClientRequestsMappingOnMissTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/IgniteMarshallerCacheClientRequestsMappingOnMissTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/IgniteMarshallerCacheClientRequestsMappingOnMissTest.java
index f5f2512..057b970 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/IgniteMarshallerCacheClientRequestsMappingOnMissTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/IgniteMarshallerCacheClientRequestsMappingOnMissTest.java
@@ -300,10 +300,10 @@ public class IgniteMarshallerCacheClientRequestsMappingOnMissTest extends GridCo
         final GridMessageListener delegate = lsnrs[GridTopic.TOPIC_MAPPING_MARSH.ordinal()];
 
         GridMessageListener wrapper = new GridMessageListener() {
-            @Override public void onMessage(UUID nodeId, Object msg) {
+            @Override public void onMessage(UUID nodeId, Object msg, byte plc) {
                 mappingReqsCounter.incrementAndGet();
 
-                delegate.onMessage(nodeId, msg);
+                delegate.onMessage(nodeId, msg, plc);
             }
         };
 
@@ -321,7 +321,7 @@ public class IgniteMarshallerCacheClientRequestsMappingOnMissTest extends GridCo
         ioMgr.removeMessageListener(GridTopic.TOPIC_MAPPING_MARSH);
 
         ioMgr.addMessageListener(GridTopic.TOPIC_MAPPING_MARSH, new GridMessageListener() {
-            @Override public void onMessage(UUID nodeId, Object msg) {
+            @Override public void onMessage(UUID nodeId, Object msg, byte plc) {
                 new Thread(new Runnable() {
                     @Override public void run() {
                         mappingReqsCounter.incrementAndGet();

http://git-wip-us.apache.org/repos/asf/ignite/blob/bdd31af7/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/binary/GridCacheBinaryObjectMetadataExchangeMultinodeTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/binary/GridCacheBinaryObjectMetadataExchangeMultinodeTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/binary/GridCacheBinaryObjectMetadataExchangeMultinodeTest.java
index 9370e27..6ff703e 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/binary/GridCacheBinaryObjectMetadataExchangeMultinodeTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/binary/GridCacheBinaryObjectMetadataExchangeMultinodeTest.java
@@ -396,7 +396,7 @@ public class GridCacheBinaryObjectMetadataExchangeMultinodeTest extends GridComm
         ioMgr.removeMessageListener(GridTopic.TOPIC_METADATA_REQ);
 
         ioMgr.addMessageListener(GridTopic.TOPIC_METADATA_REQ, new GridMessageListener() {
-            @Override public void onMessage(UUID nodeId, Object msg) {
+            @Override public void onMessage(UUID nodeId, Object msg, byte plc) {
                 new Thread(new Runnable() {
                     @Override public void run() {
                         metadataReqsCounter.incrementAndGet();
@@ -416,9 +416,9 @@ public class GridCacheBinaryObjectMetadataExchangeMultinodeTest extends GridComm
         final GridMessageListener delegate = lsnrs[GridTopic.TOPIC_METADATA_REQ.ordinal()];
 
         GridMessageListener wrapper = new GridMessageListener() {
-            @Override public void onMessage(UUID nodeId, Object msg) {
+            @Override public void onMessage(UUID nodeId, Object msg, byte plc) {
                 metadataReqsCounter.incrementAndGet();
-                delegate.onMessage(nodeId, msg);
+                delegate.onMessage(nodeId, msg, plc);
             }
         };
 

http://git-wip-us.apache.org/repos/asf/ignite/blob/bdd31af7/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/transactions/TxDeadlockDetectionMessageMarshallingTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/transactions/TxDeadlockDetectionMessageMarshallingTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/transactions/TxDeadlockDetectionMessageMarshallingTest.java
index 9126053..1a48cec 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/transactions/TxDeadlockDetectionMessageMarshallingTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/transactions/TxDeadlockDetectionMessageMarshallingTest.java
@@ -76,7 +76,7 @@ public class TxDeadlockDetectionMessageMarshallingTest extends GridCommonAbstrac
             final AtomicBoolean res = new AtomicBoolean();
 
             clientCtx.gridIO().addMessageListener(TOPIC, new GridMessageListener() {
-                @Override public void onMessage(UUID nodeId, Object msg) {
+                @Override public void onMessage(UUID nodeId, Object msg, byte plc) {
                     if (msg instanceof TxLocksResponse) {
                         try {
                             ((TxLocksResponse)msg).finishUnmarshal(clientCtx, clientCtx.deploy().globalLoader());

http://git-wip-us.apache.org/repos/asf/ignite/blob/bdd31af7/modules/core/src/test/java/org/apache/ignite/loadtests/communication/GridIoManagerBenchmark.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/loadtests/communication/GridIoManagerBenchmark.java b/modules/core/src/test/java/org/apache/ignite/loadtests/communication/GridIoManagerBenchmark.java
index 03bbb00..5671158 100644
--- a/modules/core/src/test/java/org/apache/ignite/loadtests/communication/GridIoManagerBenchmark.java
+++ b/modules/core/src/test/java/org/apache/ignite/loadtests/communication/GridIoManagerBenchmark.java
@@ -240,7 +240,7 @@ public class GridIoManagerBenchmark {
         GridMessageListener lsnr = new GridMessageListener() {
             private ClusterNode node;
 
-            @Override public void onMessage(UUID nodeId, Object msg) {
+            @Override public void onMessage(UUID nodeId, Object msg, byte plc) {
                 if (node == null)
                     node = g.context().discovery().node(nodeId);
 
@@ -336,7 +336,7 @@ public class GridIoManagerBenchmark {
      */
     private static class SenderMessageListener implements GridMessageListener {
         /** {@inheritDoc} */
-        @Override public void onMessage(UUID nodeId, Object msg) {
+        @Override public void onMessage(UUID nodeId, Object msg, byte plc) {
             msgCntr.increment();
 
             if (testLatency)

http://git-wip-us.apache.org/repos/asf/ignite/blob/bdd31af7/modules/core/src/test/java/org/apache/ignite/loadtests/communication/GridIoManagerBenchmark0.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/loadtests/communication/GridIoManagerBenchmark0.java b/modules/core/src/test/java/org/apache/ignite/loadtests/communication/GridIoManagerBenchmark0.java
index 0f0332f..7b1d972 100644
--- a/modules/core/src/test/java/org/apache/ignite/loadtests/communication/GridIoManagerBenchmark0.java
+++ b/modules/core/src/test/java/org/apache/ignite/loadtests/communication/GridIoManagerBenchmark0.java
@@ -130,7 +130,7 @@ public class GridIoManagerBenchmark0 extends GridCommonAbstractTest {
         rcv.addMessageListener(
             topic,
             new GridMessageListener() {
-                @Override public void onMessage(UUID nodeId, Object msg) {
+                @Override public void onMessage(UUID nodeId, Object msg, byte plc) {
                     try {
                         rcv.sendToCustomTopic(sndNode, topic, (Message)msg, PUBLIC_POOL);
                     }
@@ -141,7 +141,7 @@ public class GridIoManagerBenchmark0 extends GridCommonAbstractTest {
             });
 
         snd.addMessageListener(topic, new GridMessageListener() {
-            @Override public void onMessage(UUID nodeId, Object msg) {
+            @Override public void onMessage(UUID nodeId, Object msg, byte plc) {
                 msgCntr.increment();
 
                 sem.release();
@@ -224,7 +224,7 @@ public class GridIoManagerBenchmark0 extends GridCommonAbstractTest {
         rcv.addMessageListener(
             topic,
             new GridMessageListener() {
-                @Override public void onMessage(UUID nodeId, Object msg) {
+                @Override public void onMessage(UUID nodeId, Object msg, byte plc) {
                     try {
                         rcv.sendToCustomTopic(sndNode, topic, (Message)msg, PUBLIC_POOL);
                     }
@@ -235,7 +235,7 @@ public class GridIoManagerBenchmark0 extends GridCommonAbstractTest {
             });
 
         snd.addMessageListener(topic, new GridMessageListener() {
-            @Override public void onMessage(UUID nodeId, Object msg) {
+            @Override public void onMessage(UUID nodeId, Object msg, byte plc) {
                 map.get(((GridTestMessage)msg).id()).countDown();
             }
         });
@@ -324,7 +324,7 @@ public class GridIoManagerBenchmark0 extends GridCommonAbstractTest {
         rcv.addMessageListener(
             topic,
             new GridMessageListener() {
-                @Override public void onMessage(UUID nodeId, Object msg) {
+                @Override public void onMessage(UUID nodeId, Object msg, byte plc) {
                     try {
                         rcv.sendToCustomTopic(sndNode, topic, (Message)msg, PUBLIC_POOL);
                     }
@@ -335,7 +335,7 @@ public class GridIoManagerBenchmark0 extends GridCommonAbstractTest {
             });
 
         snd.addMessageListener(topic, new GridMessageListener() {
-            @Override public void onMessage(UUID nodeId, Object msg) {
+            @Override public void onMessage(UUID nodeId, Object msg, byte plc) {
                 msgCntr.increment();
 
                 sem.release();

http://git-wip-us.apache.org/repos/asf/ignite/blob/bdd31af7/modules/core/src/test/java/org/apache/ignite/spi/communication/GridCacheMessageSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/spi/communication/GridCacheMessageSelfTest.java b/modules/core/src/test/java/org/apache/ignite/spi/communication/GridCacheMessageSelfTest.java
index 4a6b765..435af8f 100644
--- a/modules/core/src/test/java/org/apache/ignite/spi/communication/GridCacheMessageSelfTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/spi/communication/GridCacheMessageSelfTest.java
@@ -139,7 +139,7 @@ public class GridCacheMessageSelfTest extends GridCommonAbstractTest {
         final CountDownLatch latch = new CountDownLatch(SAMPLE_CNT);
 
         mgr1.addMessageListener(topic, new GridMessageListener() {
-            @Override public void onMessage(UUID nodeId, Object msg) {
+            @Override public void onMessage(UUID nodeId, Object msg, byte plc) {
                 try {
                     latch.countDown();
 

http://git-wip-us.apache.org/repos/asf/ignite/blob/bdd31af7/modules/core/src/test/java/org/apache/ignite/testframework/GridSpiTestContext.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/testframework/GridSpiTestContext.java b/modules/core/src/test/java/org/apache/ignite/testframework/GridSpiTestContext.java
index 7d3c5d6..93cd911 100644
--- a/modules/core/src/test/java/org/apache/ignite/testframework/GridSpiTestContext.java
+++ b/modules/core/src/test/java/org/apache/ignite/testframework/GridSpiTestContext.java
@@ -41,6 +41,7 @@ import org.apache.ignite.internal.direct.DirectMessageReader;
 import org.apache.ignite.internal.direct.DirectMessageWriter;
 import org.apache.ignite.internal.managers.communication.GridIoManager;
 import org.apache.ignite.internal.managers.communication.GridIoMessageFactory;
+import org.apache.ignite.internal.managers.communication.GridIoPolicy;
 import org.apache.ignite.internal.managers.communication.GridIoUserMessage;
 import org.apache.ignite.internal.managers.communication.GridMessageListener;
 import org.apache.ignite.internal.managers.eventstorage.GridLocalEventListener;
@@ -328,7 +329,7 @@ public class GridSpiTestContext implements IgniteSpiContext {
     @SuppressWarnings("deprecation")
     public void triggerMessage(ClusterNode node, Object msg) {
         for (GridMessageListener lsnr : msgLsnrs)
-            lsnr.onMessage(node.id(), msg);
+            lsnr.onMessage(node.id(), msg, GridIoPolicy.SYSTEM_POOL);
     }
 
     /** {@inheritDoc} */
@@ -667,7 +668,7 @@ public class GridSpiTestContext implements IgniteSpiContext {
         @SuppressWarnings({
             "SynchronizationOnLocalVariableOrMethodParameter", "ConstantConditions",
             "OverlyStrongTypeCast"})
-        @Override public void onMessage(UUID nodeId, Object msg) {
+        @Override public void onMessage(UUID nodeId, Object msg, byte plc) {
             GridIoUserMessage ioMsg = (GridIoUserMessage)msg;
 
             Object msgBody = ioMsg.body();

http://git-wip-us.apache.org/repos/asf/ignite/blob/bdd31af7/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/shuffle/HadoopShuffle.java
----------------------------------------------------------------------
diff --git a/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/shuffle/HadoopShuffle.java b/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/shuffle/HadoopShuffle.java
index 3296993..cd1c93c 100644
--- a/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/shuffle/HadoopShuffle.java
+++ b/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/shuffle/HadoopShuffle.java
@@ -63,7 +63,7 @@ public class HadoopShuffle extends HadoopComponent {
         super.start(ctx);
 
         ctx.kernalContext().io().addMessageListener(GridTopic.TOPIC_HADOOP_MSG, new GridMessageListener() {
-            @Override public void onMessage(UUID nodeId, Object msg) {
+            @Override public void onMessage(UUID nodeId, Object msg, byte plc) {
                 onMessageReceived(nodeId, (HadoopMessage)msg);
             }
         });

http://git-wip-us.apache.org/repos/asf/ignite/blob/bdd31af7/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/opt/GridH2IndexBase.java
----------------------------------------------------------------------
diff --git a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/opt/GridH2IndexBase.java b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/opt/GridH2IndexBase.java
index 3dabc58..542adf0 100644
--- a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/opt/GridH2IndexBase.java
+++ b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/opt/GridH2IndexBase.java
@@ -135,7 +135,7 @@ public abstract class GridH2IndexBase extends BaseIndex {
             msgTopic = new IgniteBiTuple<>(GridTopic.TOPIC_QUERY, tbl.identifierString() + '.' + getName());
 
             msgLsnr = new GridMessageListener() {
-                @Override public void onMessage(UUID nodeId, Object msg) {
+                @Override public void onMessage(UUID nodeId, Object msg, byte plc) {
                     GridSpinBusyLock l = desc.indexing().busyLock();
 
                     if (!l.enterBusy())

http://git-wip-us.apache.org/repos/asf/ignite/blob/bdd31af7/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/twostep/GridMapQueryExecutor.java
----------------------------------------------------------------------
diff --git a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/twostep/GridMapQueryExecutor.java b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/twostep/GridMapQueryExecutor.java
index 6b7ba75..fcf5f10 100644
--- a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/twostep/GridMapQueryExecutor.java
+++ b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/twostep/GridMapQueryExecutor.java
@@ -172,7 +172,7 @@ public class GridMapQueryExecutor {
         }, EventType.EVT_NODE_FAILED, EventType.EVT_NODE_LEFT);
 
         ctx.io().addMessageListener(GridTopic.TOPIC_QUERY, new GridMessageListener() {
-            @Override public void onMessage(UUID nodeId, Object msg) {
+            @Override public void onMessage(UUID nodeId, Object msg, byte plc) {
                 if (!busyLock.enterBusy())
                     return;
 

http://git-wip-us.apache.org/repos/asf/ignite/blob/bdd31af7/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/twostep/GridReduceQueryExecutor.java
----------------------------------------------------------------------
diff --git a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/twostep/GridReduceQueryExecutor.java b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/twostep/GridReduceQueryExecutor.java
index b85fa61..85a7e0b 100644
--- a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/twostep/GridReduceQueryExecutor.java
+++ b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/twostep/GridReduceQueryExecutor.java
@@ -169,7 +169,7 @@ public class GridReduceQueryExecutor {
         log = ctx.log(GridReduceQueryExecutor.class);
 
         ctx.io().addMessageListener(GridTopic.TOPIC_QUERY, new GridMessageListener() {
-            @Override public void onMessage(UUID nodeId, Object msg) {
+            @Override public void onMessage(UUID nodeId, Object msg, byte plc) {
                 if (!busyLock.enterBusy())
                     return;
 


[06/33] ignite git commit: IGNITE-5683 Fixed missing fully qualified class names for generated indexed types on Models screen.

Posted by sb...@apache.org.
IGNITE-5683 Fixed missing fully qualified class names for generated indexed types on Models screen.


Project: http://git-wip-us.apache.org/repos/asf/ignite/repo
Commit: http://git-wip-us.apache.org/repos/asf/ignite/commit/d8461977
Tree: http://git-wip-us.apache.org/repos/asf/ignite/tree/d8461977
Diff: http://git-wip-us.apache.org/repos/asf/ignite/diff/d8461977

Branch: refs/heads/ignite-2.1.2-exchange
Commit: d84619775e3960f30890a467b897315deed20ab7
Parents: 211caf1
Author: Alexey Kuznetsov <ak...@apache.org>
Authored: Tue Jul 4 15:44:27 2017 +0700
Committer: Alexey Kuznetsov <ak...@apache.org>
Committed: Tue Jul 4 15:44:27 2017 +0700

----------------------------------------------------------------------
 .../configuration/generator/ConfigurationGenerator.js       | 9 ++++++---
 1 file changed, 6 insertions(+), 3 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/d8461977/modules/web-console/frontend/app/modules/configuration/generator/ConfigurationGenerator.js
----------------------------------------------------------------------
diff --git a/modules/web-console/frontend/app/modules/configuration/generator/ConfigurationGenerator.js b/modules/web-console/frontend/app/modules/configuration/generator/ConfigurationGenerator.js
index a903ec4..f850dce 100644
--- a/modules/web-console/frontend/app/modules/configuration/generator/ConfigurationGenerator.js
+++ b/modules/web-console/frontend/app/modules/configuration/generator/ConfigurationGenerator.js
@@ -1653,8 +1653,11 @@ export default class IgniteConfigurationGenerator {
     static domainModelGeneral(domain, cfg = this.domainConfigurationBean(domain)) {
         switch (cfg.valueOf('queryMetadata')) {
             case 'Annotations':
-                if (_.nonNil(domain.keyType) && _.nonNil(domain.valueType))
-                    cfg.varArgProperty('indexedTypes', 'indexedTypes', [domain.keyType, domain.valueType], 'java.lang.Class');
+                if (_.nonNil(domain.keyType) && _.nonNil(domain.valueType)) {
+                    cfg.varArgProperty('indexedTypes', 'indexedTypes',
+                        [javaTypes.fullClassName(domain.keyType), javaTypes.fullClassName(domain.valueType)],
+                        'java.lang.Class');
+                }
 
                 break;
             case 'Configuration':
@@ -1864,7 +1867,7 @@ export default class IgniteConfigurationGenerator {
     static cacheQuery(cache, domains, available, ccfg = this.cacheConfigurationBean(cache)) {
         const indexedTypes = _.reduce(domains, (acc, domain) => {
             if (domain.queryMetadata === 'Annotations')
-                acc.push(domain.keyType, domain.valueType);
+                acc.push(javaTypes.fullClassName(domain.keyType), javaTypes.fullClassName(domain.valueType));
 
             return acc;
         }, []);


[24/33] ignite git commit: Reworked cluster activation/deactivation.

Posted by sb...@apache.org.
http://git-wip-us.apache.org/repos/asf/ignite/blob/1337901f/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/standbycluster/IgniteChangeGlobalStateFailOverTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/standbycluster/IgniteChangeGlobalStateFailOverTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/standbycluster/IgniteChangeGlobalStateFailOverTest.java
index d0b2d56..92d1f21 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/standbycluster/IgniteChangeGlobalStateFailOverTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/standbycluster/IgniteChangeGlobalStateFailOverTest.java
@@ -25,6 +25,7 @@ import org.apache.ignite.Ignite;
 import org.apache.ignite.IgniteCache;
 import org.apache.ignite.configuration.CacheConfiguration;
 import org.apache.ignite.internal.IgniteInternalFuture;
+import org.apache.ignite.internal.util.typedef.internal.U;
 
 import static java.lang.Thread.sleep;
 import static org.apache.ignite.testframework.GridTestUtils.runAsync;
@@ -54,7 +55,7 @@ public class IgniteChangeGlobalStateFailOverTest extends IgniteChangeGlobalState
     }
 
     /**
-     *
+     * @throws Exception If failed.
      */
     public void testActivateDeActivateOnFixTopology() throws Exception {
         final Ignite igB1 = backUp(0);
@@ -140,7 +141,7 @@ public class IgniteChangeGlobalStateFailOverTest extends IgniteChangeGlobalState
     }
 
     /**
-     *
+     * @throws Exception If failed.
      */
     public void testActivateDeActivateOnJoiningNode() throws Exception {
         fail("https://issues.apache.org/jira/browse/IGNITE-5520");
@@ -168,9 +169,9 @@ public class IgniteChangeGlobalStateFailOverTest extends IgniteChangeGlobalState
             final IgniteInternalFuture<Void> af = runAsync(new Callable<Void>() {
                 @Override public Void call() throws Exception {
                     while (!stop.get()) {
-                        Ignite ig = randomBackUp(false);
-
                         if (canAct.get()) {
+                            Ignite ig = randomBackUp(false);
+
                             long start = System.currentTimeMillis();
 
                             ig.active(true);
@@ -184,6 +185,8 @@ public class IgniteChangeGlobalStateFailOverTest extends IgniteChangeGlobalState
 
                             canAct.set(false);
                         }
+                        else
+                            U.sleep(100);
 
                     }
                     return null;
@@ -193,9 +196,9 @@ public class IgniteChangeGlobalStateFailOverTest extends IgniteChangeGlobalState
             final IgniteInternalFuture<Void> df = runAsync(new Callable<Void>() {
                 @Override public Void call() throws Exception {
                     while (!stop.get()) {
-                        Ignite ig = randomBackUp(false);
-
                         if (!canAct.get()) {
+                            Ignite ig = randomBackUp(false);
+
                             long start = System.currentTimeMillis();
 
                             ig.active(false);
@@ -209,7 +212,8 @@ public class IgniteChangeGlobalStateFailOverTest extends IgniteChangeGlobalState
 
                             canAct.set(true);
                         }
-
+                        else
+                            U.sleep(100);
                     }
                     return null;
                 }
@@ -243,16 +247,16 @@ public class IgniteChangeGlobalStateFailOverTest extends IgniteChangeGlobalState
             jf2.get();
         }
         finally {
-            log.info("total started nodes: " + (seqIdx.get() - backUpNodes()));
+            log.info("Total started nodes: " + (seqIdx.get() - backUpNodes()));
 
-            log.info("total activate/deactivate:" + cntA.get() + "/" + cntD.get() + " aTime/dTime: "
+            log.info("Total activate/deactivate:" + cntA.get() + "/" + cntD.get() + " aTime/dTime: "
                 + (timeA.get() / cntA.get() + "/" + (timeD.get() / cntD.get()))
             );
         }
     }
 
     /**
-     *
+     * @throws Exception If failed.
      */
     public void testActivateDeActivateOnFixTopologyWithPutValues() throws Exception {
         final Ignite igB1 = backUp(0);
@@ -348,7 +352,7 @@ public class IgniteChangeGlobalStateFailOverTest extends IgniteChangeGlobalState
             df.get();
         }
         finally {
-            log.info("total activate/deactivate:" + cntA.get() + "/" + cntD.get() + " aTime/dTime:"
+            log.info("Total activate/deactivate:" + cntA.get() + "/" + cntD.get() + " aTime/dTime:"
                 + (timeA.get() / cntA.get() + "/" + (timeD.get() / cntD.get()) + " nodes: " + backUpNodes()));
         }
     }

http://git-wip-us.apache.org/repos/asf/ignite/blob/1337901f/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/standbycluster/IgniteChangeGlobalStateTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/standbycluster/IgniteChangeGlobalStateTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/standbycluster/IgniteChangeGlobalStateTest.java
index 9151c24..80bf1fb 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/standbycluster/IgniteChangeGlobalStateTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/standbycluster/IgniteChangeGlobalStateTest.java
@@ -128,74 +128,65 @@ public class IgniteChangeGlobalStateTest extends IgniteChangeGlobalStateAbstract
 
         stopAllPrimary();
 
-        final AtomicInteger exCnt = new AtomicInteger();
-
         final CyclicBarrier barrier = new CyclicBarrier(backUpNodes() + backUpClientNodes());
 
         IgniteInternalFuture<Void> f1 = runAsync(new Callable<Void>() {
             @Override public Void call() throws Exception {
-                try {
-                    barrier.await();
-                    ig1B.active(true);
-                }catch (Exception e){
-                    exCnt.incrementAndGet();
-                }
+                barrier.await();
+
+                ig1B.active(true);
+
                 return null;
             }
         });
 
-        IgniteInternalFuture<Void> f2 = runAsync(new Runnable() {
-            @Override public void run() {
-                try {
-                    barrier.await();
-                    ig2B.active(true);
-                }catch (Exception e){
-                    exCnt.incrementAndGet();
-                }
+        IgniteInternalFuture<?> f2 = runAsync(new Callable<Void>() {
+            @Override public Void call() throws Exception {
+                barrier.await();
+
+                ig2B.active(true);
+
+                return null;
             }
         });
 
-        IgniteInternalFuture<Void> f3 = runAsync(new Runnable() {
-            @Override public void run() {
-                try {
-                    barrier.await();
-                    ig3B.active(true);
-                }catch (Exception e){
-                    exCnt.incrementAndGet();
-                }
+        IgniteInternalFuture<?> f3 = runAsync(new Callable<Void>() {
+            @Override public Void call() throws Exception {
+                barrier.await();
+
+                ig3B.active(true);
+
+                return null;
             }
         });
 
-        IgniteInternalFuture<Void> f4 = runAsync(new Runnable() {
-            @Override public void run() {
-                try {
-                    barrier.await();
-                    ig1C.active(true);
-                }catch (Exception e){
-                    exCnt.incrementAndGet();
-                }
+        IgniteInternalFuture<?> f4 = runAsync(new Callable<Void>() {
+            @Override public Void call() throws Exception {
+                barrier.await();
+
+                ig1C.active(true);
+
+                return null;
             }
         });
 
-        IgniteInternalFuture<Void> f5 = runAsync(new Runnable() {
-            @Override public void run() {
-                try {
-                    barrier.await();
-                    ig2C.active(true);
-                }catch (Exception e){
-                    exCnt.incrementAndGet();
-                }
+        IgniteInternalFuture<?> f5 = runAsync(new Callable<Void>() {
+            @Override public Void call() throws Exception {
+                barrier.await();
+
+                ig2C.active(true);
+
+                return null;
             }
         });
 
-        IgniteInternalFuture<Void> f6 = runAsync(new Runnable() {
-            @Override public void run() {
-                try {
-                    barrier.await();
-                    ig3C.active(true);
-                }catch (Exception e){
-                    exCnt.incrementAndGet();
-                }
+        IgniteInternalFuture<?> f6 = runAsync(new Callable<Void>() {
+            @Override public Void call() throws Exception {
+                barrier.await();
+
+                ig3C.active(true);
+
+                return null;
             }
         });
 
@@ -207,8 +198,6 @@ public class IgniteChangeGlobalStateTest extends IgniteChangeGlobalStateAbstract
         f5.get();
         f6.get();
 
-        assertTrue(exCnt.get() > 0);
-
         assertTrue(ig1B.active());
         assertTrue(ig2B.active());
         assertTrue(ig3B.active());
@@ -232,40 +221,35 @@ public class IgniteChangeGlobalStateTest extends IgniteChangeGlobalStateAbstract
 
         stopAllPrimary();
 
-        final AtomicInteger exCnt = new AtomicInteger();
-
         final CyclicBarrier barrier = new CyclicBarrier(3);
 
-        IgniteInternalFuture<Void> act1 = runAsync(new Runnable() {
-            @Override public void run() {
-                try {
-                    barrier.await();
-                    ig1B.active(true);
-                }catch (Exception e){
-                    exCnt.incrementAndGet();
-                }
+        IgniteInternalFuture<?> act1 = runAsync(new Callable<Void>() {
+            @Override public Void call() throws Exception {
+                barrier.await();
+
+                ig1B.active(true);
+
+                return null;
             }
         });
 
-        IgniteInternalFuture<Void> act2 = runAsync(new Runnable() {
-            @Override public void run() {
-                try {
-                    barrier.await();
-                    ig2B.active(true);
-                }catch (Exception e){
-                    exCnt.incrementAndGet();
-                }
+        IgniteInternalFuture<?> act2 = runAsync(new Callable<Void>() {
+            @Override public Void call() throws Exception {
+                barrier.await();
+
+                ig2B.active(true);
+
+                return null;
             }
         });
 
-        IgniteInternalFuture<Void> act3 = runAsync(new Runnable() {
-            @Override public void run() {
-                try {
-                    barrier.await();
-                    ig3B.active(true);
-                }catch (Exception e){
-                    exCnt.incrementAndGet();
-                }
+        IgniteInternalFuture<?> act3 = runAsync(new Callable<Void>() {
+            @Override public Void call() throws Exception {
+                barrier.await();
+
+                ig3B.active(true);
+
+                return null;
             }
         });
 
@@ -273,15 +257,13 @@ public class IgniteChangeGlobalStateTest extends IgniteChangeGlobalStateAbstract
         act2.get();
         act3.get();
 
-        assertEquals(2, exCnt.get());
-
         assertTrue(ig1B.active());
         assertTrue(ig2B.active());
         assertTrue(ig3B.active());
     }
 
     /**
-     *
+     * @throws Exception If failed.
      */
     public void testActiveAndInActiveAtTheSameTimeCluster() throws Exception {
         Ignite ig1P = primary(0);
@@ -318,7 +300,7 @@ public class IgniteChangeGlobalStateTest extends IgniteChangeGlobalStateAbstract
     }
 
     /**
-     *
+     * @throws Exception If failed.
      */
     public void testActivateOnAlreadyActivatedCluster() throws Exception {
         Ignite ig1P = primary(0);
@@ -375,7 +357,7 @@ public class IgniteChangeGlobalStateTest extends IgniteChangeGlobalStateAbstract
     }
 
     /**
-     *
+     * @throws Exception If failed.
      */
     public void testTryUseCacheInActiveCluster() throws Exception {
         Ignite ig1B = backUp(0);
@@ -410,13 +392,14 @@ public class IgniteChangeGlobalStateTest extends IgniteChangeGlobalStateAbstract
         assertThrows(log, new Callable<Void>() {
             @Override public Void call() throws Exception {
                 IgniteCache cache = ig.cache("cache");
+
                 return null;
             }
         }, IgniteException.class, "Can not perform the operation because the cluster is inactive.");
     }
 
     /**
-     *
+     * @throws Exception If failed.
      */
     public void testTryUseServiceInActiveCluster() throws Exception {
         Ignite ig1B = backUp(0);
@@ -445,7 +428,7 @@ public class IgniteChangeGlobalStateTest extends IgniteChangeGlobalStateAbstract
     }
 
     /**
-     *
+     * @param ig Node to check.
      */
     private void checkExceptionTryUseService(final Ignite ig) {
         assertThrows(log, new Callable<Void>() {
@@ -458,7 +441,7 @@ public class IgniteChangeGlobalStateTest extends IgniteChangeGlobalStateAbstract
     }
 
     /**
-     *
+     * @throws Exception If failed.
      */
     public void testTryUseDataStructureInActiveCluster() throws Exception {
         Ignite ig1B = backUp(0);
@@ -487,19 +470,20 @@ public class IgniteChangeGlobalStateTest extends IgniteChangeGlobalStateAbstract
     }
 
     /**
-     *
+     * @param ig Node.
      */
     private void checkExceptionTryUseDataStructure(final Ignite ig){
         assertThrows(log, new Callable<Void>() {
             @Override public Void call() throws Exception {
                 IgniteAtomicSequence seq = ig.atomicSequence("seq", 0, true);
+
                 return null;
             }
         }, IgniteException.class, "Can not perform the operation because the cluster is inactive.");
     }
 
     /**
-     *
+     * @throws Exception If failed.
      */
     public void testFailGetLock() throws Exception {
         Ignite ig1P = primary(0);
@@ -556,7 +540,7 @@ public class IgniteChangeGlobalStateTest extends IgniteChangeGlobalStateAbstract
     }
 
     /**
-     *
+     * @throws Exception If failed.
      */
     public void testActivateAfterFailGetLock() throws Exception {
         Ignite ig1P = primary(0);

http://git-wip-us.apache.org/repos/asf/ignite/blob/1337901f/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/standbycluster/IgniteStandByClusterTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/standbycluster/IgniteStandByClusterTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/standbycluster/IgniteStandByClusterTest.java
index c3bdcda..2678e51 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/standbycluster/IgniteStandByClusterTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/standbycluster/IgniteStandByClusterTest.java
@@ -60,7 +60,7 @@ public class IgniteStandByClusterTest extends GridCommonAbstractTest {
     /**
      * @throws Exception if fail.
      */
-    public void testNotStartDynamicCachesOnClientAfterActivation() throws Exception {
+    public void testStartDynamicCachesAfterActivation() throws Exception {
         final String cacheName0 = "cache0";
         final String cacheName = "cache";
 
@@ -114,13 +114,10 @@ public class IgniteStandByClusterTest extends GridCommonAbstractTest {
         Map<String, GridCacheAdapter<?, ?>> caches = U.field(ig3.context().cache(), "caches");
 
         // Only system cache and cache0
-        assertTrue(caches.size() == 2);
+        assertEquals("Unexpected caches: " + caches.keySet(), 3, caches.size());
         assertTrue(caches.containsKey(CU.UTILITY_CACHE_NAME));
         assertTrue(caches.containsKey(cacheName0));
-
-        assertNull(caches.get(cacheName));
-
-        assertNotNull(caches.get(cacheName0));
+        assertTrue(caches.containsKey(cacheName));
 
         assertNotNull(ig3.cache(cacheName));
     }
@@ -156,12 +153,6 @@ public class IgniteStandByClusterTest extends GridCommonAbstractTest {
         assertTrue(!ig2.active());
         assertTrue(!ig3.active());
 
-        for (IgniteEx ig : Arrays.asList(ig1, ig2, ig3)) {
-            Map<String, DynamicCacheDescriptor> desc = U.field(U.field(ig.context().cache(), "cachesInfo"), "registeredCaches");
-
-            assertEquals(0, desc.size());
-        }
-
         ig3.active(true);
 
         assertTrue(ig1.active());
@@ -286,7 +277,7 @@ public class IgniteStandByClusterTest extends GridCommonAbstractTest {
         private final String name;
 
         /**
-         * @param name
+         * @param name Node name.
          */
         private NodeFilterIgnoreByName(String name) {
             this.name = name;

http://git-wip-us.apache.org/repos/asf/ignite/blob/1337901f/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/standbycluster/join/JoinActiveNodeToActiveCluster.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/standbycluster/join/JoinActiveNodeToActiveCluster.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/standbycluster/join/JoinActiveNodeToActiveCluster.java
index 9c5f017..8e90e78 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/standbycluster/join/JoinActiveNodeToActiveCluster.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/standbycluster/join/JoinActiveNodeToActiveCluster.java
@@ -29,9 +29,7 @@ import org.junit.Assert;
  *
  */
 public class JoinActiveNodeToActiveCluster extends AbstractNodeJoinTemplate {
-    /**
-     *
-     */
+    /** {@inheritDoc} */
     @Override public JoinNodeTestPlanBuilder withOutConfigurationTemplate() throws Exception {
         JoinNodeTestPlanBuilder b = builder();
 
@@ -56,9 +54,7 @@ public class JoinActiveNodeToActiveCluster extends AbstractNodeJoinTemplate {
         return b;
     }
 
-    /**
-     *
-     */
+    /** {@inheritDoc} */
     @Override public JoinNodeTestPlanBuilder staticCacheConfigurationOnJoinTemplate() throws Exception {
         JoinNodeTestPlanBuilder b = builder();
 
@@ -85,9 +81,7 @@ public class JoinActiveNodeToActiveCluster extends AbstractNodeJoinTemplate {
         return b;
     }
 
-    /**
-     *
-     */
+    /** {@inheritDoc} */
     @Override public JoinNodeTestPlanBuilder staticCacheConfigurationInClusterTemplate() throws Exception {
         JoinNodeTestPlanBuilder b = builder();
 
@@ -115,9 +109,7 @@ public class JoinActiveNodeToActiveCluster extends AbstractNodeJoinTemplate {
         return b;
     }
 
-    /**
-     *
-     */
+    /** {@inheritDoc} */
     @Override public JoinNodeTestPlanBuilder staticCacheConfigurationSameOnBothTemplate() throws Exception {
         JoinNodeTestPlanBuilder b = builder();
 
@@ -146,9 +138,7 @@ public class JoinActiveNodeToActiveCluster extends AbstractNodeJoinTemplate {
         return b;
     }
 
-    /**
-     *
-     */
+    /** {@inheritDoc} */
     @Override public JoinNodeTestPlanBuilder staticCacheConfigurationDifferentOnBothTemplate() throws Exception {
         JoinNodeTestPlanBuilder b = builder();
 
@@ -207,62 +197,46 @@ public class JoinActiveNodeToActiveCluster extends AbstractNodeJoinTemplate {
         withOutConfigurationTemplate().execute();
     }
 
-    /**
-     *
-     */
+    /** {@inheritDoc} */
     @Override public void testStaticCacheConfigurationOnJoin() throws Exception {
         staticCacheConfigurationOnJoinTemplate().execute();
     }
 
-    /**
-     *
-     */
+    /** {@inheritDoc} */
     @Override public void testStaticCacheConfigurationInCluster() throws Exception {
         staticCacheConfigurationInClusterTemplate().execute();
     }
 
-    /**
-     *
-     */
+    /** {@inheritDoc} */
     @Override public void testStaticCacheConfigurationSameOnBoth() throws Exception {
         staticCacheConfigurationSameOnBothTemplate().execute();
     }
 
-    /**
-     *
-     */
+    /** {@inheritDoc} */
     @Override public void testStaticCacheConfigurationDifferentOnBoth() throws Exception {
         staticCacheConfigurationDifferentOnBothTemplate().execute();
     }
 
     // Client node join.
 
-    /**
-     *
-     */
+    /** {@inheritDoc} */
     @Override public void testJoinClientWithOutConfiguration() throws Exception {
         joinClientWithOutConfigurationTemplate().execute();
     }
 
-    /**
-     *
-     */
+    /** {@inheritDoc} */
     @Override public void testJoinClientStaticCacheConfigurationOnJoin() throws Exception {
         joinClientStaticCacheConfigurationOnJoinTemplate().execute();
     }
 
-    /**
-     *
-     */
+    /** {@inheritDoc} */
     @Override public void testJoinClientStaticCacheConfigurationInCluster() throws Exception {
         fail("https://issues.apache.org/jira/browse/IGNITE-5518");
 
         joinClientStaticCacheConfigurationInClusterTemplate().execute();
     }
 
-    /**
-     *
-     */
+    /** {@inheritDoc} */
     @Override public void testJoinClientStaticCacheConfigurationSameOnBoth() throws Exception {
         joinClientStaticCacheConfigurationSameOnBothTemplate().execute();
     }
@@ -276,14 +250,17 @@ public class JoinActiveNodeToActiveCluster extends AbstractNodeJoinTemplate {
         joinClientStaticCacheConfigurationDifferentOnBothTemplate().execute();
     }
 
+    /** {@inheritDoc} */
     @Override public JoinNodeTestPlanBuilder joinClientWithOutConfigurationTemplate() throws Exception {
         return withOutConfigurationTemplate().nodeConfiguration(setClient);
     }
 
+    /** {@inheritDoc} */
     @Override public JoinNodeTestPlanBuilder joinClientStaticCacheConfigurationOnJoinTemplate() throws Exception {
         return staticCacheConfigurationOnJoinTemplate().nodeConfiguration(setClient);
     }
 
+    /** {@inheritDoc} */
     @Override public JoinNodeTestPlanBuilder joinClientStaticCacheConfigurationInClusterTemplate() throws Exception {
         return staticCacheConfigurationInClusterTemplate()
             .nodeConfiguration(setClient)
@@ -360,8 +337,10 @@ public class JoinActiveNodeToActiveCluster extends AbstractNodeJoinTemplate {
             });
     }
 
-    @Override
-    public JoinNodeTestPlanBuilder joinClientStaticCacheConfigurationDifferentOnBothTemplate() throws Exception {
+    /** {@inheritDoc} */
+    @Override public JoinNodeTestPlanBuilder joinClientStaticCacheConfigurationDifferentOnBothTemplate()
+        throws Exception
+    {
         return staticCacheConfigurationDifferentOnBothTemplate()
             .nodeConfiguration(setClient)
             .afterActivate(new Runnable() {
@@ -436,6 +415,7 @@ public class JoinActiveNodeToActiveCluster extends AbstractNodeJoinTemplate {
             });
     }
 
+    /** {@inheritDoc} */
     @Override public JoinNodeTestPlanBuilder joinClientStaticCacheConfigurationSameOnBothTemplate() throws Exception {
         return staticCacheConfigurationSameOnBothTemplate().nodeConfiguration(setClient);
     }

http://git-wip-us.apache.org/repos/asf/ignite/blob/1337901f/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/standbycluster/join/persistence/JoinActiveNodeToActiveClusterWithPersistence.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/standbycluster/join/persistence/JoinActiveNodeToActiveClusterWithPersistence.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/standbycluster/join/persistence/JoinActiveNodeToActiveClusterWithPersistence.java
index 60388fa..1ccfb7d 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/standbycluster/join/persistence/JoinActiveNodeToActiveClusterWithPersistence.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/standbycluster/join/persistence/JoinActiveNodeToActiveClusterWithPersistence.java
@@ -30,6 +30,10 @@ public class JoinActiveNodeToActiveClusterWithPersistence extends JoinActiveNode
         return persistentCfg(super.cfg(name));
     }
 
+    /**
+     * @param b Builder.
+     * @return Builder.
+     */
     private AbstractNodeJoinTemplate.JoinNodeTestPlanBuilder persistent(AbstractNodeJoinTemplate.JoinNodeTestPlanBuilder b) {
         b.afterClusterStarted(
             b.checkCacheEmpty()
@@ -44,6 +48,7 @@ public class JoinActiveNodeToActiveClusterWithPersistence extends JoinActiveNode
         return b;
     }
 
+    /** {@inheritDoc} */
     @Override public AbstractNodeJoinTemplate.JoinNodeTestPlanBuilder withOutConfigurationTemplate() throws Exception {
         AbstractNodeJoinTemplate.JoinNodeTestPlanBuilder b = persistent(super.withOutConfigurationTemplate());
 
@@ -52,6 +57,7 @@ public class JoinActiveNodeToActiveClusterWithPersistence extends JoinActiveNode
         return b;
     }
 
+    /** {@inheritDoc} */
     @Override public AbstractNodeJoinTemplate.JoinNodeTestPlanBuilder joinClientWithOutConfigurationTemplate() throws Exception {
         AbstractNodeJoinTemplate.JoinNodeTestPlanBuilder b = persistent(super.joinClientWithOutConfigurationTemplate());
 
@@ -60,46 +66,57 @@ public class JoinActiveNodeToActiveClusterWithPersistence extends JoinActiveNode
         return b;
     }
 
+    /** {@inheritDoc} */
     @Override public void testJoinWithOutConfiguration() throws Exception {
         withOutConfigurationTemplate().execute();
     }
 
+    /** {@inheritDoc} */
     @Override public void testJoinClientWithOutConfiguration() throws Exception {
         joinClientWithOutConfigurationTemplate().execute();
     }
 
+    /** {@inheritDoc} */
     @Override public void testJoinClientStaticCacheConfigurationDifferentOnBoth() throws Exception {
         staticCacheConfigurationDifferentOnBothTemplate().execute();
     }
 
+    /** {@inheritDoc} */
     @Override public void testJoinClientStaticCacheConfigurationInCluster() throws Exception {
         staticCacheConfigurationInClusterTemplate().execute();
     }
 
+    /** {@inheritDoc} */
     @Override public AbstractNodeJoinTemplate.JoinNodeTestPlanBuilder staticCacheConfigurationOnJoinTemplate() throws Exception {
         return persistent(super.staticCacheConfigurationOnJoinTemplate());
     }
 
+    /** {@inheritDoc} */
     @Override public AbstractNodeJoinTemplate.JoinNodeTestPlanBuilder staticCacheConfigurationInClusterTemplate() throws Exception {
         return persistent(super.staticCacheConfigurationInClusterTemplate());
     }
 
+    /** {@inheritDoc} */
     @Override public AbstractNodeJoinTemplate.JoinNodeTestPlanBuilder staticCacheConfigurationSameOnBothTemplate() throws Exception {
         return persistent(super.staticCacheConfigurationSameOnBothTemplate());
     }
 
+    /** {@inheritDoc} */
     @Override public AbstractNodeJoinTemplate.JoinNodeTestPlanBuilder staticCacheConfigurationDifferentOnBothTemplate() throws Exception {
         return persistent(super.staticCacheConfigurationDifferentOnBothTemplate());
     }
 
+    /** {@inheritDoc} */
     @Override public AbstractNodeJoinTemplate.JoinNodeTestPlanBuilder joinClientStaticCacheConfigurationOnJoinTemplate() throws Exception {
         return persistent(super.joinClientStaticCacheConfigurationOnJoinTemplate());
     }
 
+    /** {@inheritDoc} */
     @Override public AbstractNodeJoinTemplate.JoinNodeTestPlanBuilder joinClientStaticCacheConfigurationInClusterTemplate() throws Exception {
         return persistent(super.joinClientStaticCacheConfigurationInClusterTemplate());
     }
 
+    /** {@inheritDoc} */
     @Override public AbstractNodeJoinTemplate.JoinNodeTestPlanBuilder joinClientStaticCacheConfigurationDifferentOnBothTemplate() throws Exception {
         return persistent(super.joinClientStaticCacheConfigurationDifferentOnBothTemplate());
     }

http://git-wip-us.apache.org/repos/asf/ignite/blob/1337901f/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/standbycluster/reconnect/IgniteStandByClientReconnectTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/standbycluster/reconnect/IgniteStandByClientReconnectTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/standbycluster/reconnect/IgniteStandByClientReconnectTest.java
index 02f7d00..548dba2 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/standbycluster/reconnect/IgniteStandByClientReconnectTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/standbycluster/reconnect/IgniteStandByClientReconnectTest.java
@@ -24,7 +24,9 @@ import org.apache.ignite.internal.IgniteEx;
  *
  */
 public class IgniteStandByClientReconnectTest extends IgniteAbstractStandByClientReconnectTest {
-
+    /**
+     * @throws Exception If failed.
+     */
     public void testActiveClientReconnectToActiveCluster() throws Exception {
         CountDownLatch activateLatch = new CountDownLatch(1);
 
@@ -103,6 +105,9 @@ public class IgniteStandByClientReconnectTest extends IgniteAbstractStandByClien
         checkAllCaches();
     }
 
+    /**
+     * @throws Exception If failed.
+     */
     public void testActiveClientReconnectToInActiveCluster() throws Exception {
         CountDownLatch activateLatch = new CountDownLatch(1);
 
@@ -180,6 +185,9 @@ public class IgniteStandByClientReconnectTest extends IgniteAbstractStandByClien
         checkAllCaches();
     }
 
+    /**
+     * @throws Exception If failed.
+     */
     public void testInActiveClientReconnectToActiveCluster() throws Exception {
         CountDownLatch activateLatch = new CountDownLatch(1);
 
@@ -234,6 +242,9 @@ public class IgniteStandByClientReconnectTest extends IgniteAbstractStandByClien
         checkAllCaches();
     }
 
+    /**
+     * @throws Exception If failed.
+     */
     public void testInActiveClientReconnectToInActiveCluster() throws Exception {
         startNodes(null);
 

http://git-wip-us.apache.org/repos/asf/ignite/blob/1337901f/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/standbycluster/reconnect/IgniteStandByClientReconnectToNewClusterTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/standbycluster/reconnect/IgniteStandByClientReconnectToNewClusterTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/standbycluster/reconnect/IgniteStandByClientReconnectToNewClusterTest.java
index 2bcc177..2da32e3 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/standbycluster/reconnect/IgniteStandByClientReconnectToNewClusterTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/standbycluster/reconnect/IgniteStandByClientReconnectToNewClusterTest.java
@@ -27,7 +27,9 @@ import org.apache.ignite.internal.IgniteEx;
  *
  */
 public class IgniteStandByClientReconnectToNewClusterTest extends IgniteAbstractStandByClientReconnectTest {
-
+    /**
+     * @throws Exception If failed.
+     */
     public void testActiveClientReconnectToActiveCluster() throws Exception {
         CountDownLatch activateLatch = new CountDownLatch(1);
 
@@ -104,6 +106,9 @@ public class IgniteStandByClientReconnectToNewClusterTest extends IgniteAbstract
         checkAllCaches();
     }
 
+    /**
+     * @throws Exception If failed.
+     */
     public void testActiveClientReconnectToInActiveCluster() throws Exception {
         startNodes(null);
 
@@ -181,6 +186,9 @@ public class IgniteStandByClientReconnectToNewClusterTest extends IgniteAbstract
         checkAllCaches();
     }
 
+    /**
+     * @throws Exception If failed.
+     */
     public void testInActiveClientReconnectToActiveCluster() throws Exception {
         CountDownLatch activateLatch = new CountDownLatch(1);
 
@@ -236,6 +244,9 @@ public class IgniteStandByClientReconnectToNewClusterTest extends IgniteAbstract
         checkDescriptors(client, exp2);
     }
 
+    /**
+     * @throws Exception If failed.
+     */
     public void testInActiveClientReconnectToInActiveCluster() throws Exception {
         startNodes(null);
 

http://git-wip-us.apache.org/repos/asf/ignite/blob/1337901f/modules/core/src/test/java/org/apache/ignite/spi/communication/tcp/GridTcpCommunicationSpiMultithreadedSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/spi/communication/tcp/GridTcpCommunicationSpiMultithreadedSelfTest.java b/modules/core/src/test/java/org/apache/ignite/spi/communication/tcp/GridTcpCommunicationSpiMultithreadedSelfTest.java
index 90af25e..bc07028 100644
--- a/modules/core/src/test/java/org/apache/ignite/spi/communication/tcp/GridTcpCommunicationSpiMultithreadedSelfTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/spi/communication/tcp/GridTcpCommunicationSpiMultithreadedSelfTest.java
@@ -479,7 +479,7 @@ public class GridTcpCommunicationSpiMultithreadedSelfTest extends GridSpiAbstrac
 
         timeoutProcessor.start();
 
-        timeoutProcessor.onKernalStart();
+        timeoutProcessor.onKernalStart(true);
 
         for (int i = 0; i < getSpiCount(); i++) {
             CommunicationSpi<Message> spi = newCommunicationSpi();

http://git-wip-us.apache.org/repos/asf/ignite/blob/1337901f/modules/core/src/test/java/org/apache/ignite/testframework/junits/GridAbstractTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/testframework/junits/GridAbstractTest.java b/modules/core/src/test/java/org/apache/ignite/testframework/junits/GridAbstractTest.java
index a9a870e..d6d241c 100755
--- a/modules/core/src/test/java/org/apache/ignite/testframework/junits/GridAbstractTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/testframework/junits/GridAbstractTest.java
@@ -2099,13 +2099,13 @@ public abstract class GridAbstractTest extends TestCase {
         for (Ignite g : G.allGrids()) {
             final GridKernalContext ctx = ((IgniteKernal)g).context();
 
-            if (ctx.isStopping())
+            if (ctx.isStopping() || !g.active())
                 continue;
 
             AffinityTopologyVersion topVer = ctx.discovery().topologyVersionEx();
             AffinityTopologyVersion exchVer = ctx.cache().context().exchange().readyAffinityVersion();
 
-            if (! topVer.equals(exchVer)) {
+            if (!topVer.equals(exchVer)) {
                 info("Topology version mismatch [node="  + g.name() +
                     ", exchVer=" + exchVer +
                     ", topVer=" + topVer + ']');

http://git-wip-us.apache.org/repos/asf/ignite/blob/1337901f/modules/core/src/test/java/org/apache/ignite/testframework/junits/common/GridCommonAbstractTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/testframework/junits/common/GridCommonAbstractTest.java b/modules/core/src/test/java/org/apache/ignite/testframework/junits/common/GridCommonAbstractTest.java
index 3f87c76..dc7e89d 100755
--- a/modules/core/src/test/java/org/apache/ignite/testframework/junits/common/GridCommonAbstractTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/testframework/junits/common/GridCommonAbstractTest.java
@@ -596,6 +596,9 @@ public abstract class GridCommonAbstractTest extends GridAbstractTest {
             else
                 startTime = g0.context().discovery().gridStartTime();
 
+            if (g.cluster().localNode().isDaemon())
+                continue;
+
             IgniteInternalFuture<?> exchFut =
                 g0.context().cache().context().exchange().affinityReadyFuture(waitTopVer);
 

http://git-wip-us.apache.org/repos/asf/ignite/blob/1337901f/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteStandByClusterSuite.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteStandByClusterSuite.java b/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteStandByClusterSuite.java
index 6fa158d..f016e39 100644
--- a/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteStandByClusterSuite.java
+++ b/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteStandByClusterSuite.java
@@ -18,6 +18,7 @@
 package org.apache.ignite.testsuites;
 
 import junit.framework.TestSuite;
+import org.apache.ignite.internal.processors.cache.IgniteClusterActivateDeactivateTest;
 import org.apache.ignite.internal.processors.cache.persistence.standbycluster.IgniteChangeGlobalStateCacheTest;
 import org.apache.ignite.internal.processors.cache.persistence.standbycluster.IgniteChangeGlobalStateDataStreamerTest;
 import org.apache.ignite.internal.processors.cache.persistence.standbycluster.IgniteChangeGlobalStateDataStructureTest;
@@ -41,11 +42,13 @@ import org.apache.ignite.internal.processors.cache.persistence.standbycluster.re
  */
 public class IgniteStandByClusterSuite extends TestSuite {
     /**
-     *
+     * @return Test suite.
      */
     public static TestSuite suite() {
         TestSuite suite = new TestSuite("Ignite Activate/DeActivate Cluster Test Suit");
 
+        suite.addTestSuite(IgniteClusterActivateDeactivateTest.class);
+
         suite.addTestSuite(IgniteStandByClusterTest.class);
         suite.addTestSuite(IgniteStandByClientReconnectTest.class);
         suite.addTestSuite(IgniteStandByClientReconnectToNewClusterTest.class);

http://git-wip-us.apache.org/repos/asf/ignite/blob/1337901f/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/HadoopProcessor.java
----------------------------------------------------------------------
diff --git a/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/HadoopProcessor.java b/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/HadoopProcessor.java
index 329d67f..231fc22 100644
--- a/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/HadoopProcessor.java
+++ b/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/HadoopProcessor.java
@@ -95,8 +95,8 @@ public class HadoopProcessor extends HadoopProcessorAdapter {
     }
 
     /** {@inheritDoc} */
-    @Override public void onKernalStart() throws IgniteCheckedException {
-        super.onKernalStart();
+    @Override public void onKernalStart(boolean active) throws IgniteCheckedException {
+        super.onKernalStart(active);
 
         if (hctx == null)
             return;


[11/33] ignite git commit: IGNITE-5629 .NET: CacheConfiguration copy constructor

Posted by sb...@apache.org.
IGNITE-5629 .NET: CacheConfiguration copy constructor


Project: http://git-wip-us.apache.org/repos/asf/ignite/repo
Commit: http://git-wip-us.apache.org/repos/asf/ignite/commit/b69f53e0
Tree: http://git-wip-us.apache.org/repos/asf/ignite/tree/b69f53e0
Diff: http://git-wip-us.apache.org/repos/asf/ignite/diff/b69f53e0

Branch: refs/heads/ignite-2.1.2-exchange
Commit: b69f53e0ccf9d3da42a99d4423fb3d8bdd60a7bb
Parents: 6c9d222
Author: Pavel Tupitsyn <pt...@apache.org>
Authored: Tue Jul 4 17:42:31 2017 +0300
Committer: Pavel Tupitsyn <pt...@apache.org>
Committed: Tue Jul 4 17:42:31 2017 +0300

----------------------------------------------------------------------
 .../utils/PlatformConfigurationUtils.java       |  8 ++-
 .../Cache/CacheConfigurationTest.cs             | 21 ++++++
 .../Cache/Configuration/CacheConfiguration.cs   | 75 ++++++++++++++++++--
 .../Apache.Ignite.Core/IgniteConfiguration.cs   |  7 +-
 4 files changed, 99 insertions(+), 12 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/b69f53e0/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/utils/PlatformConfigurationUtils.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/utils/PlatformConfigurationUtils.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/utils/PlatformConfigurationUtils.java
index 23106ba..92db41a 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/utils/PlatformConfigurationUtils.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/utils/PlatformConfigurationUtils.java
@@ -885,8 +885,10 @@ public class PlatformConfigurationUtils {
             writer.writeInt(cnt);
 
             for (CachePluginConfiguration cfg : plugins) {
-                if (cfg instanceof PlatformCachePluginConfiguration)
-                    writer.writeObject(((PlatformCachePluginConfiguration)cfg).nativeCfg());
+                if (cfg instanceof PlatformCachePluginConfiguration) {
+                    writer.writeBoolean(false);  // Pure platform plugin.
+                    writer.writeObject(((PlatformCachePluginConfiguration) cfg).nativeCfg());
+                }
             }
         }
     }
@@ -1317,6 +1319,8 @@ public class PlatformConfigurationUtils {
     private static void readCachePluginConfiguration(CacheConfiguration cfg, BinaryRawReader in) {
         int plugCfgFactoryId = in.readInt();
 
+        in.readInt(); // skip size.
+
         PlatformCachePluginConfigurationClosure plugCfg = cachePluginConfiguration(plugCfgFactoryId);
 
         plugCfg.apply(cfg, in);

http://git-wip-us.apache.org/repos/asf/ignite/blob/b69f53e0/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Cache/CacheConfigurationTest.cs
----------------------------------------------------------------------
diff --git a/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Cache/CacheConfigurationTest.cs b/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Cache/CacheConfigurationTest.cs
index 1f6dbcf..435e65f 100644
--- a/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Cache/CacheConfigurationTest.cs
+++ b/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Cache/CacheConfigurationTest.cs
@@ -190,6 +190,27 @@ namespace Apache.Ignite.Core.Tests.Cache
         }
 
         /// <summary>
+        /// Tests the copy constructor.
+        /// </summary>
+        [Test]
+        public void TestCopyConstructor()
+        {
+            foreach (var cfg in new[]
+                {new CacheConfiguration(), GetCustomCacheConfiguration(), GetCustomCacheConfiguration2()})
+            {
+                // Check direct copy.
+                AssertConfigsAreEqual(cfg, cfg);
+                AssertConfigsAreEqual(cfg, new CacheConfiguration(cfg));
+
+                // Check copy via Ignite config.
+                var igniteCfg = new IgniteConfiguration {CacheConfiguration = new[] {cfg}};
+                var igniteCfgCopy = new IgniteConfiguration(igniteCfg);
+
+                AssertConfigsAreEqual(cfg, igniteCfgCopy.CacheConfiguration.Single());
+            }
+        }
+
+        /// <summary>
         /// Asserts the configuration is default.
         /// </summary>
         private static void AssertConfigIsDefault(CacheConfiguration cfg)

http://git-wip-us.apache.org/repos/asf/ignite/blob/b69f53e0/modules/platforms/dotnet/Apache.Ignite.Core/Cache/Configuration/CacheConfiguration.cs
----------------------------------------------------------------------
diff --git a/modules/platforms/dotnet/Apache.Ignite.Core/Cache/Configuration/CacheConfiguration.cs b/modules/platforms/dotnet/Apache.Ignite.Core/Cache/Configuration/CacheConfiguration.cs
index ec0ac40..f5a5179 100644
--- a/modules/platforms/dotnet/Apache.Ignite.Core/Cache/Configuration/CacheConfiguration.cs
+++ b/modules/platforms/dotnet/Apache.Ignite.Core/Cache/Configuration/CacheConfiguration.cs
@@ -25,6 +25,7 @@ namespace Apache.Ignite.Core.Cache.Configuration
     using System.ComponentModel;
     using System.Diagnostics;
     using System.Diagnostics.CodeAnalysis;
+    using System.IO;
     using System.Linq;
     using Apache.Ignite.Core.Binary;
     using Apache.Ignite.Core.Cache;
@@ -34,11 +35,14 @@ namespace Apache.Ignite.Core.Cache.Configuration
     using Apache.Ignite.Core.Cache.Expiry;
     using Apache.Ignite.Core.Cache.Store;
     using Apache.Ignite.Core.Common;
+    using Apache.Ignite.Core.Impl;
     using Apache.Ignite.Core.Impl.Binary;
     using Apache.Ignite.Core.Impl.Cache.Affinity;
     using Apache.Ignite.Core.Impl.Cache.Expiry;
     using Apache.Ignite.Core.Log;
     using Apache.Ignite.Core.Plugin.Cache;
+    using BinaryReader = Apache.Ignite.Core.Impl.Binary.BinaryReader;
+    using BinaryWriter = Apache.Ignite.Core.Impl.Binary.BinaryWriter;
 
     /// <summary>
     /// Defines grid cache configuration.
@@ -199,10 +203,43 @@ namespace Apache.Ignite.Core.Cache.Configuration
         }
 
         /// <summary>
+        /// Initializes a new instance of the <see cref="CacheConfiguration"/> class,
+        /// performing a deep copy of specified cache configuration.
+        /// </summary>
+        /// <param name="other">The other configuration to perfrom deep copy from.</param>
+        public CacheConfiguration(CacheConfiguration other)
+        {
+            if (other != null)
+            {
+                using (var stream = IgniteManager.Memory.Allocate().GetStream())
+                {
+                    other.Write(BinaryUtils.Marshaller.StartMarshal(stream));
+
+                    stream.SynchronizeOutput();
+                    stream.Seek(0, SeekOrigin.Begin);
+
+                    Read(BinaryUtils.Marshaller.StartUnmarshal(stream));
+                }
+
+                // Plugins should be copied directly.
+                PluginConfigurations = other.PluginConfigurations;
+            }
+        }
+
+        /// <summary>
         /// Initializes a new instance of the <see cref="CacheConfiguration"/> class.
         /// </summary>
         /// <param name="reader">The reader.</param>
-        internal CacheConfiguration(IBinaryRawReader reader)
+        internal CacheConfiguration(BinaryReader reader)
+        {
+            Read(reader);
+        }
+
+        /// <summary>
+        /// Reads data into this instance from the specified reader.
+        /// </summary>
+        /// <param name="reader">The reader.</param>
+        private void Read(BinaryReader reader)
         {
             // Make sure system marshaller is used.
             Debug.Assert(((BinaryReader) reader).Marshaller == BinaryUtils.Marshaller);
@@ -244,7 +281,9 @@ namespace Apache.Ignite.Core.Cache.Configuration
             CacheStoreFactory = reader.ReadObject<IFactory<ICacheStore>>();
 
             var count = reader.ReadInt();
-            QueryEntities = count == 0 ? null : Enumerable.Range(0, count).Select(x => new QueryEntity(reader)).ToList();
+            QueryEntities = count == 0
+                ? null
+                : Enumerable.Range(0, count).Select(x => new QueryEntity(reader)).ToList();
 
             NearConfiguration = reader.ReadBoolean() ? new NearCacheConfiguration(reader) : null;
 
@@ -253,19 +292,35 @@ namespace Apache.Ignite.Core.Cache.Configuration
             ExpiryPolicyFactory = ExpiryPolicySerializer.ReadPolicyFactory(reader);
 
             count = reader.ReadInt();
-            PluginConfigurations = count == 0
-                ? null
-                : Enumerable.Range(0, count).Select(x => reader.ReadObject<ICachePluginConfiguration>()).ToList();
+
+            if (count > 0)
+            {
+                PluginConfigurations = new List<ICachePluginConfiguration>(count);
+                for (int i = 0; i < count; i++)
+                {
+                    if (reader.ReadBoolean())
+                    {
+                        // FactoryId-based plugin: skip.
+                        var size = reader.ReadInt();
+                        reader.Stream.Seek(size, SeekOrigin.Current);
+                    }
+                    else
+                    {
+                        // Pure .NET plugin.
+                        PluginConfigurations.Add(reader.ReadObject<ICachePluginConfiguration>());
+                    }
+                }
+            }
         }
 
         /// <summary>
         /// Writes this instance to the specified writer.
         /// </summary>
         /// <param name="writer">The writer.</param>
-        internal void Write(IBinaryRawWriter writer)
+        internal void Write(BinaryWriter writer)
         {
             // Make sure system marshaller is used.
-            Debug.Assert(((BinaryWriter) writer).Marshaller == BinaryUtils.Marshaller);
+            Debug.Assert(writer.Marshaller == BinaryUtils.Marshaller);
 
             writer.WriteInt((int) AtomicityMode);
             writer.WriteInt(Backups);
@@ -344,7 +399,13 @@ namespace Apache.Ignite.Core.Cache.Configuration
                     {
                         writer.WriteBoolean(true);
                         writer.WriteInt(cachePlugin.CachePluginConfigurationClosureFactoryId.Value);
+
+                        int pos = writer.Stream.Position;
+                        writer.WriteInt(0);  // Reserve size.
+
                         cachePlugin.WriteBinary(writer);
+
+                        writer.Stream.WriteInt(pos, writer.Stream.Position - pos);  // Write size.
                     }
                     else
                     {

http://git-wip-us.apache.org/repos/asf/ignite/blob/b69f53e0/modules/platforms/dotnet/Apache.Ignite.Core/IgniteConfiguration.cs
----------------------------------------------------------------------
diff --git a/modules/platforms/dotnet/Apache.Ignite.Core/IgniteConfiguration.cs b/modules/platforms/dotnet/Apache.Ignite.Core/IgniteConfiguration.cs
index 8173642..4419e2e 100644
--- a/modules/platforms/dotnet/Apache.Ignite.Core/IgniteConfiguration.cs
+++ b/modules/platforms/dotnet/Apache.Ignite.Core/IgniteConfiguration.cs
@@ -48,6 +48,7 @@ namespace Apache.Ignite.Core
     using Apache.Ignite.Core.PersistentStore;
     using Apache.Ignite.Core.Plugin;
     using Apache.Ignite.Core.Transactions;
+    using BinaryReader = Apache.Ignite.Core.Impl.Binary.BinaryReader;
     using BinaryWriter = Apache.Ignite.Core.Impl.Binary.BinaryWriter;
 
     /// <summary>
@@ -242,7 +243,7 @@ namespace Apache.Ignite.Core
         /// </summary>
         /// <param name="binaryReader">The binary reader.</param>
         /// <param name="baseConfig">The base configuration.</param>
-        internal IgniteConfiguration(IBinaryRawReader binaryReader, IgniteConfiguration baseConfig)
+        internal IgniteConfiguration(BinaryReader binaryReader, IgniteConfiguration baseConfig)
         {
             Debug.Assert(binaryReader != null);
             Debug.Assert(baseConfig != null);
@@ -509,7 +510,7 @@ namespace Apache.Ignite.Core
         /// Reads data from specified reader into current instance.
         /// </summary>
         /// <param name="r">The binary reader.</param>
-        private void ReadCore(IBinaryRawReader r)
+        private void ReadCore(BinaryReader r)
         {
             // Simple properties
             _clientMode = r.ReadBooleanNullable();
@@ -630,7 +631,7 @@ namespace Apache.Ignite.Core
         /// Reads data from specified reader into current instance.
         /// </summary>
         /// <param name="binaryReader">The binary reader.</param>
-        private void Read(IBinaryRawReader binaryReader)
+        private void Read(BinaryReader binaryReader)
         {
             ReadCore(binaryReader);
 


[10/33] ignite git commit: IGNITE-5527: Prevent starvation in stripe pool on unstable topology.

Posted by sb...@apache.org.
IGNITE-5527: Prevent starvation in stripe pool on unstable topology.

(cherry picked from commit 97ea507)


Project: http://git-wip-us.apache.org/repos/asf/ignite/repo
Commit: http://git-wip-us.apache.org/repos/asf/ignite/commit/f3912902
Tree: http://git-wip-us.apache.org/repos/asf/ignite/tree/f3912902
Diff: http://git-wip-us.apache.org/repos/asf/ignite/diff/f3912902

Branch: refs/heads/ignite-2.1.2-exchange
Commit: f39129020999561e9def300b9cc136138ea3698e
Parents: 6c9d222
Author: Andrey V. Mashenkov <an...@gmail.com>
Authored: Fri Jun 16 19:54:26 2017 +0300
Committer: Andrey V. Mashenkov <an...@gmail.com>
Committed: Tue Jul 4 17:39:28 2017 +0300

----------------------------------------------------------------------
 .../internal/processors/cache/GridDeferredAckMessageSender.java   | 3 ++-
 1 file changed, 2 insertions(+), 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/f3912902/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridDeferredAckMessageSender.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridDeferredAckMessageSender.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridDeferredAckMessageSender.java
index 89aa725..a8e8226 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridDeferredAckMessageSender.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridDeferredAckMessageSender.java
@@ -173,7 +173,8 @@ public abstract class GridDeferredAckMessageSender<T> {
          * @return {@code True} if request was handled, {@code false} if this buffer is filled and cannot be used.
          */
         public boolean add(T ver) {
-            readLock().lock();
+            if(!readLock().tryLock())
+                return false; // Here, writeLock is help by another thread and guard is already true.
 
             boolean snd = false;
 


[28/33] ignite git commit: Reworked cluster activation/deactivation.

Posted by sb...@apache.org.
http://git-wip-us.apache.org/repos/asf/ignite/blob/1337901f/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/ClusterState.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/ClusterState.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/ClusterState.java
deleted file mode 100644
index 1e1ef71..0000000
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/ClusterState.java
+++ /dev/null
@@ -1,38 +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.processors.cache;
-
-/**
- *
- */
-public enum ClusterState {
-    /**
-     * Cache is inactive. No operations are allowed, no partition assignments or rebalancing is performed.
-     */
-    INACTIVE,
-
-    /**
-     * Cache is active and operations. There are no lost partitions.
-     */
-    ACTIVE,
-
-    /**
-     * Cache is inactive. But process of it activation in progress.
-     */
-    TRANSITION
-}

http://git-wip-us.apache.org/repos/asf/ignite/blob/1337901f/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/DynamicCacheChangeRequest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/DynamicCacheChangeRequest.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/DynamicCacheChangeRequest.java
index 6d5eaf3..2fd8780 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/DynamicCacheChangeRequest.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/DynamicCacheChangeRequest.java
@@ -83,15 +83,15 @@ public class DynamicCacheChangeRequest implements Serializable {
     /** */
     private UUID rcvdFrom;
 
-    /** Cache state. Set to non-null when global state is changed. */
-    private ClusterState state;
-
     /** Reset lost partitions flag. */
     private boolean resetLostPartitions;
 
     /** Dynamic schema. */
     private QuerySchema schema;
 
+    /** */
+    private transient boolean locallyConfigured;
+
     /**
      * @param reqId Unique request ID.
      * @param cacheName Cache stop name.
@@ -100,7 +100,6 @@ public class DynamicCacheChangeRequest implements Serializable {
     public DynamicCacheChangeRequest(UUID reqId, String cacheName, UUID initiatingNodeId) {
         assert reqId != null;
         assert cacheName != null;
-        assert initiatingNodeId != null;
 
         this.reqId = reqId;
         this.cacheName = cacheName;
@@ -108,21 +107,6 @@ public class DynamicCacheChangeRequest implements Serializable {
     }
 
     /**
-     * @param reqId Unique request ID.
-     * @param state New cluster state.
-     * @param initiatingNodeId Initiating node ID.
-     */
-    public DynamicCacheChangeRequest(UUID reqId, ClusterState state, UUID initiatingNodeId) {
-        assert reqId != null;
-        assert state != null;
-        assert initiatingNodeId != null;
-
-        this.reqId = reqId;
-        this.state = state;
-        this.initiatingNodeId = initiatingNodeId;
-    }
-
-    /**
      * @param ctx Context.
      * @param cacheName Cache name.
      * @return Request to reset lost partitions.
@@ -183,20 +167,6 @@ public class DynamicCacheChangeRequest implements Serializable {
     }
 
     /**
-     * @return State.
-     */
-    public ClusterState state() {
-        return state;
-    }
-
-    /**
-     * @return {@code True} if global caches state is changes.
-     */
-    public boolean globalStateChange() {
-        return state != null;
-    }
-
-    /**
      * @param template {@code True} if this is request for adding template configuration.
      */
     public void template(boolean template) {
@@ -253,7 +223,7 @@ public class DynamicCacheChangeRequest implements Serializable {
     }
 
     /**
-     *
+     * @return Destroy flag.
      */
     public boolean destroy(){
         return destroy;
@@ -420,6 +390,20 @@ public class DynamicCacheChangeRequest implements Serializable {
         this.schema = schema != null ? schema.copy() : null;
     }
 
+    /**
+     * @return Locally configured flag.
+     */
+    public boolean locallyConfigured() {
+        return locallyConfigured;
+    }
+
+    /**
+     * @param locallyConfigured Locally configured flag.
+     */
+    public void locallyConfigured(boolean locallyConfigured) {
+        this.locallyConfigured = locallyConfigured;
+    }
+
     /** {@inheritDoc} */
     @Override public String toString() {
         return "DynamicCacheChangeRequest [cacheName=" + cacheName() +

http://git-wip-us.apache.org/repos/asf/ignite/blob/1337901f/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/ExchangeActions.java
----------------------------------------------------------------------
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 9caf9aa..e9ece5a 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
@@ -50,7 +50,7 @@ public class ExchangeActions {
     private Map<String, ActionData> cachesToResetLostParts;
 
     /** */
-    private ClusterState newState;
+    private StateChangeRequest stateChangeReq;
 
     /**
      * @param grpId Group ID.
@@ -89,7 +89,7 @@ public class ExchangeActions {
     /**
      * @return New caches start requests.
      */
-    Collection<ActionData> cacheStartRequests() {
+    public Collection<ActionData> cacheStartRequests() {
         return cachesToStart != null ? cachesToStart.values() : Collections.<ActionData>emptyList();
     }
 
@@ -184,19 +184,31 @@ public class ExchangeActions {
     }
 
     /**
-     * @param state New cluster state.
+     * @param stateChange Cluster state change request.
      */
-    void newClusterState(ClusterState state) {
-        assert state != null;
+    public void stateChangeRequest(StateChangeRequest stateChange) {
+        this.stateChangeReq = stateChange;
+    }
+
+    /**
+     * @return {@code True} if has deactivate request.
+     */
+    public boolean deactivate() {
+        return stateChangeReq != null && !stateChangeReq.activate();
+    }
 
-        newState = state;
+    /**
+     * @return {@code True} if has activate request.
+     */
+    public boolean activate() {
+        return stateChangeReq != null && stateChangeReq.activate();
     }
 
     /**
-     * @return New cluster state if state change was requested.
+     * @return Cluster state change request.
      */
-    @Nullable public ClusterState newClusterState() {
-        return newState;
+    @Nullable public StateChangeRequest stateChangeRequest() {
+        return stateChangeReq;
     }
 
     /**
@@ -328,13 +340,14 @@ public class ExchangeActions {
             F.isEmpty(cachesToStop) &&
             F.isEmpty(cacheGrpsToStart) &&
             F.isEmpty(cacheGrpsToStop) &&
-            F.isEmpty(cachesToResetLostParts);
+            F.isEmpty(cachesToResetLostParts) &&
+            stateChangeReq == null;
     }
 
     /**
      *
      */
-    static class ActionData {
+    public static class ActionData {
         /** */
         private final DynamicCacheChangeRequest req;
 
@@ -429,6 +442,6 @@ public class ExchangeActions {
             ", startGrps=" + startGrps +
             ", stopGrps=" + stopGrps +
             ", resetParts=" + (cachesToResetLostParts != null ? cachesToResetLostParts.keySet() : null) +
-            ", newState=" + newState + ']';
+            ", stateChangeRequest=" + stateChangeReq + ']';
     }
 }

http://git-wip-us.apache.org/repos/asf/ignite/blob/1337901f/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheEventManager.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheEventManager.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheEventManager.java
index a967305..a9692f8 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheEventManager.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheEventManager.java
@@ -21,7 +21,6 @@ import java.util.Collection;
 import java.util.UUID;
 import org.apache.ignite.cluster.ClusterNode;
 import org.apache.ignite.events.CacheEvent;
-import org.apache.ignite.events.CacheRebalancingEvent;
 import org.apache.ignite.internal.managers.eventstorage.GridLocalEventListener;
 import org.apache.ignite.internal.processors.cache.transactions.IgniteInternalTx;
 import org.apache.ignite.internal.util.typedef.F;
@@ -32,7 +31,6 @@ import org.apache.ignite.lang.IgniteUuid;
 import org.jetbrains.annotations.Nullable;
 
 import static org.apache.ignite.events.EventType.EVT_CACHE_OBJECT_READ;
-import static org.apache.ignite.events.EventType.EVT_CACHE_REBALANCE_PART_UNLOADED;
 import static org.apache.ignite.events.EventType.EVT_CACHE_STARTED;
 import static org.apache.ignite.events.EventType.EVT_CACHE_STOPPED;
 

http://git-wip-us.apache.org/repos/asf/ignite/blob/1337901f/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheEvictionManager.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheEvictionManager.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheEvictionManager.java
index 8ba10a2..7735f74 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheEvictionManager.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheEvictionManager.java
@@ -18,7 +18,6 @@
 package org.apache.ignite.internal.processors.cache;
 
 import java.util.Collection;
-import java.util.Set;
 import org.apache.ignite.IgniteCheckedException;
 import org.apache.ignite.cache.eviction.EvictionFilter;
 import org.apache.ignite.cache.eviction.EvictionPolicy;

http://git-wip-us.apache.org/repos/asf/ignite/blob/1337901f/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheIoManager.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheIoManager.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheIoManager.java
index 2de3808..f9d1114 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheIoManager.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheIoManager.java
@@ -1405,30 +1405,33 @@ public class GridCacheIoManager extends GridCacheSharedManagerAdapter {
     }
 
     /**
+     * @param cctx Context.
      * @param topic Topic.
      * @param c Handler.
      */
-    public void addOrderedCacheHandler(Object topic, IgniteBiInClosure<UUID, ? extends GridCacheIdMessage> c) {
-        addOrderedHandler(false, topic, c);
+    public void addOrderedCacheHandler(GridCacheSharedContext cctx, Object topic, IgniteBiInClosure<UUID, ? extends GridCacheIdMessage> c) {
+        addOrderedHandler(cctx, false, topic, c);
     }
 
     /**
+     * @param cctx Context.
      * @param topic Topic.
      * @param c Handler.
      */
-    public void addOrderedCacheGroupHandler(Object topic, IgniteBiInClosure<UUID, ? extends GridCacheGroupIdMessage> c) {
-        addOrderedHandler(true, topic, c);
+    public void addOrderedCacheGroupHandler(GridCacheSharedContext cctx, Object topic, IgniteBiInClosure<UUID, ? extends GridCacheGroupIdMessage> c) {
+        addOrderedHandler(cctx, true, topic, c);
     }
 
     /**
      * Adds ordered message handler.
      *
+     * @param cctx Context.
      * @param cacheGrp {@code True} if cache group message, {@code false} if cache message.
      * @param topic Topic.
      * @param c Handler.
      */
     @SuppressWarnings({"unchecked"})
-    private void addOrderedHandler(boolean cacheGrp, Object topic, IgniteBiInClosure<UUID, ? extends GridCacheMessage> c) {
+    private void addOrderedHandler(GridCacheSharedContext cctx, boolean cacheGrp, Object topic, IgniteBiInClosure<UUID, ? extends GridCacheMessage> c) {
         MessageHandlers msgHandlers = cacheGrp ? grpHandlers : cacheHandlers;
 
         IgniteLogger log0 = log;

http://git-wip-us.apache.org/repos/asf/ignite/blob/1337901f/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheMvccManager.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheMvccManager.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheMvccManager.java
index 24433de..a6907b9 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheMvccManager.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheMvccManager.java
@@ -274,10 +274,11 @@ public class GridCacheMvccManager extends GridCacheSharedManagerAdapter {
         pendingExplicit = GridConcurrentFactory.newMap();
     }
 
-    /** {@inheritDoc} */
-    @Override protected void onKernalStart0(boolean reconnect) throws IgniteCheckedException {
-        if (!reconnect)
-            cctx.gridEvents().addLocalEventListener(discoLsnr, EVT_NODE_FAILED, EVT_NODE_LEFT);
+    /**
+     * Cache futures listener must be registered after communication listener.
+     */
+    public void registerEventListener() {
+        cctx.gridEvents().addLocalEventListener(discoLsnr, EVT_NODE_FAILED, EVT_NODE_LEFT);
     }
 
     /** {@inheritDoc} */

http://git-wip-us.apache.org/repos/asf/ignite/blob/1337901f/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCachePartitionExchangeManager.java
----------------------------------------------------------------------
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 93310e3..22345d2 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
@@ -59,6 +59,7 @@ import org.apache.ignite.internal.cluster.ClusterTopologyCheckedException;
 import org.apache.ignite.internal.events.DiscoveryCustomEvent;
 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.pagemem.snapshot.StartSnapshotOperationAckDiscoveryMessage;
 import org.apache.ignite.internal.processors.affinity.AffinityTopologyVersion;
@@ -81,6 +82,8 @@ import org.apache.ignite.internal.processors.cache.distributed.dht.preloader.Ign
 import org.apache.ignite.internal.processors.cache.transactions.IgniteInternalTx;
 import org.apache.ignite.internal.processors.cache.transactions.IgniteTxManager;
 import org.apache.ignite.internal.processors.cache.version.GridCacheVersion;
+import org.apache.ignite.internal.processors.cluster.ChangeGlobalStateFinishMessage;
+import org.apache.ignite.internal.processors.cluster.ChangeGlobalStateMessage;
 import org.apache.ignite.internal.processors.query.schema.SchemaNodeLeaveExchangeWorkerTask;
 import org.apache.ignite.internal.processors.timeout.GridTimeoutObject;
 import org.apache.ignite.internal.util.GridListSet;
@@ -192,6 +195,9 @@ public class GridCachePartitionExchangeManager<K, V> extends GridCacheSharedMana
     /** */
     private DateFormat dateFormat = new SimpleDateFormat("HH:mm:ss.SSS");
 
+    /** Events received while cluster state transition was in progress. */
+    private final List<PendingDiscoveryEvent> pendingEvts = new ArrayList<>();
+
     /** Discovery listener. */
     private final DiscoveryEventListener discoLsnr = new DiscoveryEventListener() {
         @Override public void onEvent(DiscoveryEvent evt, DiscoCache cache) {
@@ -199,109 +205,53 @@ public class GridCachePartitionExchangeManager<K, V> extends GridCacheSharedMana
                 return;
 
             try {
-                ClusterNode loc = cctx.localNode();
-
-                assert evt.type() == EVT_NODE_JOINED || evt.type() == EVT_NODE_LEFT || evt.type() == EVT_NODE_FAILED ||
-                    evt.type() == EVT_DISCOVERY_CUSTOM_EVT;
-
-                final ClusterNode n = evt.eventNode();
-
-                GridDhtPartitionExchangeId exchId = null;
-                GridDhtPartitionsExchangeFuture exchFut = null;
-
-                if (evt.type() != EVT_DISCOVERY_CUSTOM_EVT) {
-                    assert !loc.id().equals(n.id());
-
-                    if (evt.type() == EVT_NODE_LEFT || evt.type() == EVT_NODE_FAILED) {
-                        assert cctx.discovery().node(n.id()) == null;
-
-                        // Avoid race b/w initial future add and discovery event.
-                        GridDhtPartitionsExchangeFuture initFut = null;
-
-                        if (readyTopVer.get().equals(AffinityTopologyVersion.NONE)) {
-                            initFut = exchangeFuture(initialExchangeId(), null, null, null, null);
-
-                            initFut.onNodeLeft(n);
-                        }
-
-                        for (GridDhtPartitionsExchangeFuture f : exchFuts.values()) {
-                            if (f != initFut)
-                                f.onNodeLeft(n);
-                        }
-                    }
+                if (evt.type() == EVT_DISCOVERY_CUSTOM_EVT &&
+                    (((DiscoveryCustomEvent)evt).customMessage() instanceof ChangeGlobalStateMessage)) {
+                    ChangeGlobalStateMessage stateChangeMsg =
+                        (ChangeGlobalStateMessage)((DiscoveryCustomEvent)evt).customMessage();
 
-                    assert evt.type() != EVT_NODE_JOINED || n.order() > loc.order() :
-                        "Node joined with smaller-than-local " +
-                        "order [newOrder=" + n.order() + ", locOrder=" + loc.order() + ']';
+                    if (stateChangeMsg.exchangeActions() == null)
+                        return;
 
-                    exchId = exchangeId(n.id(),
-                        affinityTopologyVersion(evt),
-                        evt.type());
+                    onDiscoveryEvent(evt, cache);
 
-                    exchFut = exchangeFuture(exchId, evt, cache,null, null);
+                    return;
                 }
-                else {
-                    DiscoveryCustomMessage customMsg = ((DiscoveryCustomEvent)evt).customMessage();
+                if (evt.type() == EVT_DISCOVERY_CUSTOM_EVT &&
+                    (((DiscoveryCustomEvent)evt).customMessage() instanceof ChangeGlobalStateFinishMessage)) {
+                    ChangeGlobalStateFinishMessage stateFinishMsg =
+                        (ChangeGlobalStateFinishMessage)((DiscoveryCustomEvent)evt).customMessage();
 
-                    if (customMsg instanceof DynamicCacheChangeBatch) {
-                        DynamicCacheChangeBatch batch = (DynamicCacheChangeBatch)customMsg;
-
-                        ExchangeActions exchActions = batch.exchangeActions();
-
-                        if (exchActions != null) {
-                            exchId = exchangeId(n.id(), affinityTopologyVersion(evt), evt.type());
+                    if (stateFinishMsg.clusterActive()) {
+                        for (PendingDiscoveryEvent pendingEvt : pendingEvts) {
+                            if (log.isDebugEnabled())
+                                log.debug("Process pending event: " + pendingEvt.event());
 
-                            exchFut = exchangeFuture(exchId, evt, cache, exchActions, null);
+                            onDiscoveryEvent(pendingEvt.event(), pendingEvt.discoCache());
                         }
                     }
-                    else if (customMsg instanceof CacheAffinityChangeMessage) {
-                        CacheAffinityChangeMessage msg = (CacheAffinityChangeMessage)customMsg;
-
-                        if (msg.exchangeId() == null) {
-                            if (msg.exchangeNeeded()) {
-                                exchId = exchangeId(n.id(), affinityTopologyVersion(evt), evt.type());
-
-                                exchFut = exchangeFuture(exchId, evt, cache, null, msg);
-                            }
-                        }
-                        else if (msg.exchangeId().topologyVersion().topologyVersion() >= cctx.discovery().localJoinEvent().topologyVersion())
-                            exchangeFuture(msg.exchangeId(), null, null, null, null)
-                                .onAffinityChangeMessage(evt.eventNode(), msg);
+                    else {
+                        for (PendingDiscoveryEvent pendingEvt : pendingEvts)
+                            processEventInactive(pendingEvt.event(), pendingEvt.discoCache());
                     }
-                    else if (customMsg instanceof StartSnapshotOperationAckDiscoveryMessage
-                        && ((StartSnapshotOperationAckDiscoveryMessage)customMsg).needExchange()) {
-                        exchId = exchangeId(n.id(), affinityTopologyVersion(evt), evt.type());
 
-                        exchFut = exchangeFuture(exchId, evt, null, null, null);
-                    }
-                    else {
-                        // Process event as custom discovery task if needed.
-                        CachePartitionExchangeWorkerTask task =
-                            cctx.cache().exchangeTaskForCustomDiscoveryMessage(customMsg);
+                    pendingEvts.clear();
 
-                        if (task != null)
-                            exchWorker.addCustomTask(task);
-                    }
+                    return;
                 }
 
-                if (exchId != null) {
+                if (cache.state().transition()) {
                     if (log.isDebugEnabled())
-                        log.debug("Discovery event (will start exchange): " + exchId);
-
-                    // Event callback - without this callback future will never complete.
-                    exchFut.onEvent(exchId, evt, cache);
+                        log.debug("Add pending event: " + evt);
 
-                    // Start exchange process.
-                    addFuture(exchFut);
-                }
-                else {
-                    if (log.isDebugEnabled())
-                        log.debug("Do not start exchange for discovery event: " + evt);
+                    pendingEvts.add(new PendingDiscoveryEvent(evt, cache));
                 }
+                else if (cache.state().active())
+                    onDiscoveryEvent(evt, cache);
+                else
+                    processEventInactive(evt, cache);
 
-                // Notify indexing engine about node leave so that we can re-map coordinator accordingly.
-                if (evt.type() == EVT_NODE_LEFT || evt.type() == EVT_NODE_FAILED)
-                    exchWorker.addCustomTask(new SchemaNodeLeaveExchangeWorkerTask(evt.eventNode()));
+                notifyNodeFail(evt);
             }
             finally {
                 leaveBusy();
@@ -309,6 +259,29 @@ public class GridCachePartitionExchangeManager<K, V> extends GridCacheSharedMana
         }
     };
 
+    /**
+     * @param evt Event.
+     */
+    private void notifyNodeFail(DiscoveryEvent evt) {
+        if (evt.type() == EVT_NODE_LEFT || evt.type() == EVT_NODE_FAILED) {
+            final ClusterNode n = evt.eventNode();
+
+            assert cctx.discovery().node(n.id()) == null;
+
+            for (GridDhtPartitionsExchangeFuture f : exchFuts.values())
+                f.onNodeLeft(n);
+        }
+    }
+
+    /**
+     * @param evt Event.
+     * @param cache Discovery data cache.
+     */
+    private void processEventInactive(DiscoveryEvent evt, DiscoCache cache) {
+        if (log.isDebugEnabled())
+            log.debug("Ignore event, cluster is inactive: " + evt);
+   }
+
     /** {@inheritDoc} */
     @Override protected void start0() throws IgniteCheckedException {
         super.start0();
@@ -338,12 +311,158 @@ public class GridCachePartitionExchangeManager<K, V> extends GridCacheSharedMana
                     processSinglePartitionRequest(node, msg);
                 }
             });
+
+        if (!cctx.kernalContext().clientNode()) {
+            for (int cnt = 0; cnt < cctx.gridConfig().getRebalanceThreadPoolSize(); cnt++) {
+                final int idx = cnt;
+
+                cctx.io().addOrderedCacheGroupHandler(cctx, rebalanceTopic(cnt), new CI2<UUID, GridCacheGroupIdMessage>() {
+                    @Override public void apply(final UUID id, final GridCacheGroupIdMessage m) {
+                        if (!enterBusy())
+                            return;
+
+                        try {
+                            CacheGroupContext grp = cctx.cache().cacheGroup(m.groupId());
+
+                            if (grp != null) {
+                                if (m instanceof GridDhtPartitionSupplyMessage) {
+                                    grp.preloader().handleSupplyMessage(idx, id, (GridDhtPartitionSupplyMessage) m);
+
+                                    return;
+                                }
+                                else if (m instanceof GridDhtPartitionDemandMessage) {
+                                    grp.preloader().handleDemandMessage(idx, id, (GridDhtPartitionDemandMessage) m);
+
+                                    return;
+                                }
+                            }
+
+                            U.error(log, "Unsupported message type: " + m.getClass().getName());
+                        }
+                        finally {
+                            leaveBusy();
+                        }
+                    }
+                });
+            }
+        }
+    }
+
+    /**
+     * Callback for local join event (needed since regular event for local join is not generated).
+     *
+     * @param evt Event.
+     * @param cache Cache.
+     */
+    public void onLocalJoin(DiscoveryEvent evt, DiscoCache cache) {
+        discoLsnr.onEvent(evt, cache);
+    }
+
+    /**
+     * @param evt Event.
+     * @param cache Discovery data cache.
+     */
+    private void onDiscoveryEvent(DiscoveryEvent evt, DiscoCache cache) {
+        ClusterNode loc = cctx.localNode();
+
+        assert evt.type() == EVT_NODE_JOINED || evt.type() == EVT_NODE_LEFT || evt.type() == EVT_NODE_FAILED ||
+            evt.type() == EVT_DISCOVERY_CUSTOM_EVT;
+
+        final ClusterNode n = evt.eventNode();
+
+        GridDhtPartitionExchangeId exchId = null;
+        GridDhtPartitionsExchangeFuture exchFut = null;
+
+        if (evt.type() != EVT_DISCOVERY_CUSTOM_EVT) {
+            assert evt.type() != EVT_NODE_JOINED || n.isLocal() || n.order() > loc.order() :
+                "Node joined with smaller-than-local " +
+                    "order [newOrder=" + n.order() + ", locOrder=" + loc.order() + ']';
+
+            exchId = exchangeId(n.id(), affinityTopologyVersion(evt), evt.type());
+
+            exchFut = exchangeFuture(exchId, evt, cache,null, null);
+        }
+        else {
+            DiscoveryCustomMessage customMsg = ((DiscoveryCustomEvent)evt).customMessage();
+
+            if (customMsg instanceof ChangeGlobalStateMessage) {
+                ChangeGlobalStateMessage stateChangeMsg = (ChangeGlobalStateMessage)customMsg;
+
+                ExchangeActions exchActions = stateChangeMsg.exchangeActions();
+
+                if (exchActions != null) {
+                    exchId = exchangeId(n.id(), affinityTopologyVersion(evt), evt.type());
+
+                    exchFut = exchangeFuture(exchId, evt, cache, exchActions, null);
+                }
+            }
+            else if (customMsg instanceof DynamicCacheChangeBatch) {
+                DynamicCacheChangeBatch batch = (DynamicCacheChangeBatch)customMsg;
+
+                ExchangeActions exchActions = batch.exchangeActions();
+
+                if (exchActions != null) {
+                    exchId = exchangeId(n.id(), affinityTopologyVersion(evt), evt.type());
+
+                    exchFut = exchangeFuture(exchId, evt, cache, exchActions, null);
+                }
+            }
+            else if (customMsg instanceof CacheAffinityChangeMessage) {
+                CacheAffinityChangeMessage msg = (CacheAffinityChangeMessage)customMsg;
+
+                if (msg.exchangeId() == null) {
+                    if (msg.exchangeNeeded()) {
+                        exchId = exchangeId(n.id(), affinityTopologyVersion(evt), evt.type());
+
+                        exchFut = exchangeFuture(exchId, evt, cache, null, msg);
+                    }
+                }
+                else if (msg.exchangeId().topologyVersion().topologyVersion() >= cctx.discovery().localJoinEvent().topologyVersion())
+                    exchangeFuture(msg.exchangeId(), null, null, null, null)
+                        .onAffinityChangeMessage(evt.eventNode(), msg);
+            }
+            else if (customMsg instanceof StartSnapshotOperationAckDiscoveryMessage
+                && ((StartSnapshotOperationAckDiscoveryMessage)customMsg).needExchange()) {
+                exchId = exchangeId(n.id(), affinityTopologyVersion(evt), evt.type());
+
+                exchFut = exchangeFuture(exchId, evt, null, null, null);
+            }
+            else {
+                // Process event as custom discovery task if needed.
+                CachePartitionExchangeWorkerTask task =
+                    cctx.cache().exchangeTaskForCustomDiscoveryMessage(customMsg);
+
+                if (task != null)
+                    exchWorker.addCustomTask(task);
+            }
+        }
+
+        if (exchId != null) {
+            if (log.isDebugEnabled())
+                log.debug("Discovery event (will start exchange): " + exchId);
+
+            // Event callback - without this callback future will never complete.
+            exchFut.onEvent(exchId, evt, cache);
+
+            // Start exchange process.
+            addFuture(exchFut);
+        }
+        else {
+            if (log.isDebugEnabled())
+                log.debug("Do not start exchange for discovery event: " + evt);
+        }
+
+        notifyNodeFail(evt);
+
+        // Notify indexing engine about node leave so that we can re-map coordinator accordingly.
+        if (evt.type() == EVT_NODE_LEFT || evt.type() == EVT_NODE_FAILED)
+            exchWorker.addCustomTask(new SchemaNodeLeaveExchangeWorkerTask(evt.eventNode()));
     }
 
     /**
      * @param task Task to run in exchange worker thread.
      */
-    public void addCustomTask(CachePartitionExchangeWorkerTask task) {
+    void addCustomTask(CachePartitionExchangeWorkerTask task) {
         assert task != null;
 
         exchWorker.addCustomTask(task);
@@ -371,9 +490,14 @@ public class GridCachePartitionExchangeManager<K, V> extends GridCacheSharedMana
         return exchangeId(cctx.localNode().id(), startTopVer, EVT_NODE_JOINED);
     }
 
-    /** {@inheritDoc} */
-    @Override protected void onKernalStart0(boolean reconnect) throws IgniteCheckedException {
-        super.onKernalStart0(reconnect);
+    /**
+     * @param active Cluster state.
+     * @param reconnect Reconnect flag.
+     * @throws IgniteCheckedException If failed.
+     */
+    public void onKernalStart(boolean active, boolean reconnect) throws IgniteCheckedException {
+        for (ClusterNode n : cctx.discovery().remoteNodes())
+            cctx.versions().onReceived(n.id(), n.metrics().getLastDataVersion());
 
         ClusterNode loc = cctx.localNode();
 
@@ -381,79 +505,49 @@ public class GridCachePartitionExchangeManager<K, V> extends GridCacheSharedMana
 
         assert startTime > 0;
 
-        // Generate dummy discovery event for local node joining.
-        T2<DiscoveryEvent, DiscoCache> locJoin = cctx.discovery().localJoin();
-
-        DiscoveryEvent discoEvt = locJoin.get1();
-        DiscoCache discoCache = locJoin.get2();
-
-        GridDhtPartitionExchangeId exchId = initialExchangeId();
+        DiscoveryLocalJoinData locJoin = cctx.discovery().localJoin();
 
-        GridDhtPartitionsExchangeFuture fut = exchangeFuture(exchId, discoEvt, discoCache, null, null);
+        GridDhtPartitionsExchangeFuture fut = null;
 
         if (reconnect)
             reconnectExchangeFut = new GridFutureAdapter<>();
 
-        exchWorker.addFirstExchangeFuture(fut);
-
-        if (!cctx.kernalContext().clientNode()) {
-            for (int cnt = 0; cnt < cctx.gridConfig().getRebalanceThreadPoolSize(); cnt++) {
-                final int idx = cnt;
-
-                cctx.io().addOrderedCacheGroupHandler(rebalanceTopic(cnt), new CI2<UUID, GridCacheGroupIdMessage>() {
-                    @Override public void apply(final UUID id, final GridCacheGroupIdMessage m) {
-                        if (!enterBusy())
-                            return;
-
-                        try {
-                            CacheGroupContext grp = cctx.cache().cacheGroup(m.groupId());
-
-                            if (grp != null) {
-                                if (m instanceof GridDhtPartitionSupplyMessage) {
-                                    grp.preloader().handleSupplyMessage(idx, id, (GridDhtPartitionSupplyMessage) m);
-
-                                    return;
-                                }
-                                else if (m instanceof GridDhtPartitionDemandMessage) {
-                                    grp.preloader().handleDemandMessage(idx, id, (GridDhtPartitionDemandMessage) m);
+        if (active) {
+            DiscoveryEvent discoEvt = locJoin.event();
+            DiscoCache discoCache = locJoin.discoCache();
 
-                                    return;
-                                }
-                            }
+            GridDhtPartitionExchangeId exchId = initialExchangeId();
 
-                            U.error(log, "Unsupported message type: " + m.getClass().getName());
-                        }
-                        finally {
-                            leaveBusy();
-                        }
-                    }
-                });
-            }
+            fut = exchangeFuture(exchId, discoEvt, discoCache, null, null);
         }
+        else if (reconnect)
+            reconnectExchangeFut.onDone();
 
         new IgniteThread(cctx.igniteInstanceName(), "exchange-worker", exchWorker).start();
 
         if (reconnect) {
-            fut.listen(new CI1<IgniteInternalFuture<AffinityTopologyVersion>>() {
-                @Override public void apply(IgniteInternalFuture<AffinityTopologyVersion> fut) {
-                    try {
-                        fut.get();
+            if (fut != null) {
+                fut.listen(new CI1<IgniteInternalFuture<AffinityTopologyVersion>>() {
+                    @Override public void apply(IgniteInternalFuture<AffinityTopologyVersion> fut) {
+                        try {
+                            fut.get();
 
-                        for (CacheGroupContext grp : cctx.cache().cacheGroups())
-                            grp.preloader().onInitialExchangeComplete(null);
+                            for (CacheGroupContext grp : cctx.cache().cacheGroups())
+                                grp.preloader().onInitialExchangeComplete(null);
 
-                        reconnectExchangeFut.onDone();
-                    }
-                    catch (IgniteCheckedException e) {
-                        for (CacheGroupContext grp : cctx.cache().cacheGroups())
-                            grp.preloader().onInitialExchangeComplete(e);
+                            reconnectExchangeFut.onDone();
+                        }
+                        catch (IgniteCheckedException e) {
+                            for (CacheGroupContext grp : cctx.cache().cacheGroups())
+                                grp.preloader().onInitialExchangeComplete(e);
 
-                        reconnectExchangeFut.onDone(e);
+                            reconnectExchangeFut.onDone(e);
+                        }
                     }
-                }
-            });
+                });
+            }
         }
-        else {
+        else if (fut != null) {
             if (log.isDebugEnabled())
                 log.debug("Beginning to wait on local exchange future: " + fut);
 
@@ -489,10 +583,8 @@ public class GridCachePartitionExchangeManager<K, V> extends GridCacheSharedMana
                 }
             }
 
-            AffinityTopologyVersion nodeStartVer = new AffinityTopologyVersion(discoEvt.topologyVersion(), 0);
-
             for (CacheGroupContext grp : cctx.cache().cacheGroups()) {
-                if (nodeStartVer.equals(grp.localStartVersion()))
+                if (locJoin.joinTopologyVersion().equals(grp.localStartVersion()))
                     grp.preloader().onInitialExchangeComplete(null);
             }
 
@@ -1669,28 +1761,6 @@ public class GridCachePartitionExchangeManager<K, V> extends GridCacheSharedMana
     }
 
     /**
-     * @param deque Deque to poll from.
-     * @param time Time to wait.
-     * @param w Worker.
-     * @return Polled item.
-     * @throws InterruptedException If interrupted.
-     */
-    @Nullable private <T> T poll(BlockingQueue<T> deque, long time, GridWorker w) throws InterruptedException {
-        assert w != null;
-
-        // There is currently a case where {@code interrupted}
-        // flag on a thread gets flipped during stop which causes the pool to hang.  This check
-        // will always make sure that interrupted flag gets reset before going into wait conditions.
-        // The true fix should actually make sure that interrupted flag does not get reset or that
-        // interrupted exception gets propagated. Until we find a real fix, this method should
-        // always work to make sure that there is no hanging during stop.
-        if (w.isCancelled())
-            Thread.currentThread().interrupt();
-
-        return deque.poll(time, MILLISECONDS);
-    }
-
-    /**
      * Exchange future thread. All exchanges happen only by one thread and next
      * exchange will not start until previous one completes.
      */
@@ -1710,15 +1780,6 @@ public class GridCachePartitionExchangeManager<K, V> extends GridCacheSharedMana
         }
 
         /**
-         * Add first exchange future.
-         *
-         * @param exchFut Exchange future.
-         */
-        void addFirstExchangeFuture(GridDhtPartitionsExchangeFuture exchFut) {
-            futQ.addFirst(exchFut);
-        }
-
-        /**
          * @param exchFut Exchange future.
          */
         void addExchangeFuture(GridDhtPartitionsExchangeFuture exchFut) {
@@ -1946,7 +2007,7 @@ public class GridCachePartitionExchangeManager<K, V> extends GridCacheSharedMana
                             }
                         }
 
-                        if (!exchFut.skipPreload() && cctx.kernalContext().state().active()) {
+                        if (!exchFut.skipPreload() ) {
                             assignsMap = new HashMap<>();
 
                             for (CacheGroupContext grp : cctx.cache().cacheGroups()) {

http://git-wip-us.apache.org/repos/asf/ignite/blob/1337901f/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheProcessor.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheProcessor.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheProcessor.java
index 0f859eb..624dec0 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheProcessor.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheProcessor.java
@@ -107,6 +107,9 @@ import org.apache.ignite.internal.processors.cache.transactions.IgniteTransactio
 import org.apache.ignite.internal.processors.cache.transactions.IgniteTxManager;
 import org.apache.ignite.internal.processors.cache.version.GridCacheVersionManager;
 import org.apache.ignite.internal.processors.cacheobject.IgniteCacheObjectProcessor;
+import org.apache.ignite.internal.processors.cluster.ChangeGlobalStateFinishMessage;
+import org.apache.ignite.internal.processors.cluster.ChangeGlobalStateMessage;
+import org.apache.ignite.internal.processors.cluster.DiscoveryDataClusterState;
 import org.apache.ignite.internal.processors.datastructures.DataStructuresProcessor;
 import org.apache.ignite.internal.processors.plugin.CachePluginManager;
 import org.apache.ignite.internal.processors.query.QuerySchema;
@@ -692,36 +695,27 @@ public class GridCacheProcessor extends GridProcessorAdapter {
         for (GridCacheSharedManager mgr : sharedCtx.managers())
             mgr.start(sharedCtx);
 
-        if (ctx.config().isDaemon()) {
-            ctx.state().cacheProcessorStarted(new CacheJoinNodeDiscoveryData(
-                IgniteUuid.randomUuid(),
-                Collections.<String, CacheInfo>emptyMap(),
-                Collections.<String, CacheInfo>emptyMap(),
-                false
-            ));
-
-            return;
-        }
-
-        Map<String, CacheInfo> caches = new HashMap<>();
+        if (!ctx.isDaemon()) {
+            Map<String, CacheInfo> caches = new HashMap<>();
 
-        Map<String, CacheInfo> templates = new HashMap<>();
+            Map<String, CacheInfo> templates = new HashMap<>();
 
-        addCacheOnJoinFromConfig(caches, templates);
+            addCacheOnJoinFromConfig(caches, templates);
 
-        CacheJoinNodeDiscoveryData discoData = new CacheJoinNodeDiscoveryData(
-            IgniteUuid.randomUuid(),
-            caches,
-            templates,
-            startAllCachesOnClientStart()
-        );
+            CacheJoinNodeDiscoveryData discoData = new CacheJoinNodeDiscoveryData(
+                IgniteUuid.randomUuid(),
+                caches,
+                templates,
+                startAllCachesOnClientStart()
+            );
 
-        cachesInfo.onStart(discoData);
+            cachesInfo.onStart(discoData);
 
-        if (log.isDebugEnabled())
-            log.debug("Started cache processor.");
+            if (log.isDebugEnabled())
+                log.debug("Started cache processor.");
+        }
 
-        ctx.state().cacheProcessorStarted(discoData);
+        ctx.state().cacheProcessorStarted();
     }
 
     /**
@@ -830,51 +824,38 @@ public class GridCacheProcessor extends GridProcessorAdapter {
 
     /** {@inheritDoc} */
     @SuppressWarnings("unchecked")
-    @Override public void onKernalStart() throws IgniteCheckedException {
-        boolean active = ctx.state().active();
+    @Override public void onKernalStart(boolean active) throws IgniteCheckedException {
+        if (ctx.isDaemon())
+            return;
 
         try {
-            boolean checkConsistency =
-                !ctx.config().isDaemon() && !getBoolean(IGNITE_SKIP_CONFIGURATION_CONSISTENCY_CHECK);
+            boolean checkConsistency = !getBoolean(IGNITE_SKIP_CONFIGURATION_CONSISTENCY_CHECK);
 
             if (checkConsistency)
                 checkConsistency();
 
-            if (active && cachesInfo.onJoinCacheException() != null)
-                throw new IgniteCheckedException(cachesInfo.onJoinCacheException());
-
             cachesInfo.onKernalStart(checkConsistency);
 
-            if (active && !ctx.clientNode() && !ctx.isDaemon())
-                sharedCtx.database().lock();
-
-            // Must start database before start first cache.
-            sharedCtx.database().onKernalStart(false);
-
             ctx.query().onCacheKernalStart();
 
-            // In shared context, we start exchange manager and wait until processed local join
-            // event, all caches which we get on join will be start.
-            for (GridCacheSharedManager<?, ?> mgr : sharedCtx.managers()) {
-                if (sharedCtx.database() != mgr)
-                    mgr.onKernalStart(false);
-            }
+            sharedCtx.mvcc().registerEventListener();
+
+            sharedCtx.exchange().onKernalStart(active, false);
         }
         finally {
             cacheStartedLatch.countDown();
         }
 
+        if (!ctx.clientNode())
+            addRemovedItemsCleanupTask(Long.getLong(IGNITE_CACHE_REMOVED_ENTRIES_TTL, 10_000));
+
         // Escape if cluster inactive.
         if (!active)
             return;
 
-        if (!ctx.config().isDaemon())
-            ctx.cacheObjects().onUtilityCacheStarted();
-
         ctx.service().onUtilityCacheStarted();
 
-        final AffinityTopologyVersion startTopVer =
-            new AffinityTopologyVersion(ctx.discovery().localJoinEvent().topologyVersion(), 0);
+        final AffinityTopologyVersion startTopVer = ctx.discovery().localJoin().joinTopologyVersion();
 
         final List<IgniteInternalFuture> syncFuts = new ArrayList<>(caches.size());
 
@@ -894,15 +875,8 @@ public class GridCacheProcessor extends GridProcessorAdapter {
             }
         });
 
-        // Avoid iterator creation.
-        //noinspection ForLoopReplaceableByForEach
         for (int i = 0, size = syncFuts.size(); i < size; i++)
             syncFuts.get(i).get();
-
-        assert ctx.config().isDaemon() || caches.containsKey(CU.UTILITY_CACHE_NAME) : "Utility cache should be started";
-
-        if (!ctx.clientNode() && !ctx.isDaemon())
-            addRemovedItemsCleanupTask(Long.getLong(IGNITE_CACHE_REMOVED_ENTRIES_TTL, 10_000));
     }
 
     /**
@@ -969,8 +943,6 @@ public class GridCacheProcessor extends GridProcessorAdapter {
 
         for (CacheGroupContext grp : cacheGrps.values())
             stopCacheGroup(grp.groupId());
-
-        cachesInfo.clearCaches();
     }
 
     /**
@@ -1097,7 +1069,11 @@ public class GridCacheProcessor extends GridProcessorAdapter {
     @Override public IgniteInternalFuture<?> onReconnected(boolean clusterRestarted) throws IgniteCheckedException {
         List<GridCacheAdapter> reconnected = new ArrayList<>(caches.size());
 
-        ClusterCachesReconnectResult reconnectRes = cachesInfo.onReconnected();
+        DiscoveryDataClusterState state = ctx.state().clusterState();
+
+        boolean active = state.active() && !state.transition();
+
+        ClusterCachesReconnectResult reconnectRes = cachesInfo.onReconnected(active, state.transition());
 
         final List<GridCacheAdapter> stoppedCaches = new ArrayList<>();
 
@@ -1135,7 +1111,7 @@ public class GridCacheProcessor extends GridProcessorAdapter {
                 grp.onReconnected();
         }
 
-        sharedCtx.onReconnected();
+        sharedCtx.onReconnected(active);
 
         for (GridCacheAdapter cache : reconnected)
             cache.context().gate().reconnected(false);
@@ -1750,17 +1726,26 @@ public class GridCacheProcessor extends GridProcessorAdapter {
     }
 
     /**
+     * @return Caches to be started when this node starts.
+     */
+    public List<T2<DynamicCacheDescriptor, NearCacheConfiguration>> cachesToStartOnLocalJoin() {
+        return cachesInfo.cachesToStartOnLocalJoin();
+    }
+
+    /**
+     * @param caches Caches to start.
      * @param exchTopVer Current exchange version.
      * @throws IgniteCheckedException If failed.
      */
-    public void startCachesOnLocalJoin(AffinityTopologyVersion exchTopVer) throws IgniteCheckedException {
-        List<T2<DynamicCacheDescriptor, NearCacheConfiguration>> caches = cachesInfo.cachesToStartOnLocalJoin();
-
+    public void startCachesOnLocalJoin(List<T2<DynamicCacheDescriptor, NearCacheConfiguration>> caches,
+        AffinityTopologyVersion exchTopVer)
+        throws IgniteCheckedException {
         if (!F.isEmpty(caches)) {
             for (T2<DynamicCacheDescriptor, NearCacheConfiguration> t : caches) {
                 DynamicCacheDescriptor desc = t.get1();
 
                 prepareCacheStart(
+                    desc.cacheConfiguration(),
                     desc,
                     t.get2(),
                     exchTopVer
@@ -1787,6 +1772,7 @@ public class GridCacheProcessor extends GridProcessorAdapter {
 
                 if (CU.affinityNode(ctx.discovery().localNode(), filter)) {
                     prepareCacheStart(
+                        desc.cacheConfiguration(),
                         desc,
                         null,
                         exchTopVer
@@ -1799,17 +1785,18 @@ public class GridCacheProcessor extends GridProcessorAdapter {
     }
 
     /**
+     * @param startCfg Cache configuration to use.
      * @param desc Cache descriptor.
      * @param reqNearCfg Near configuration if specified for client cache start request.
      * @param exchTopVer Current exchange version.
      * @throws IgniteCheckedException If failed.
      */
-    public void prepareCacheStart(
+    void prepareCacheStart(
+        CacheConfiguration startCfg,
         DynamicCacheDescriptor desc,
         @Nullable NearCacheConfiguration reqNearCfg,
         AffinityTopologyVersion exchTopVer
     ) throws IgniteCheckedException {
-        CacheConfiguration startCfg = desc.cacheConfiguration();
         assert !caches.containsKey(startCfg.getName()) : startCfg.getName();
 
         CacheConfiguration ccfg = new CacheConfiguration(startCfg);
@@ -2003,7 +1990,7 @@ public class GridCacheProcessor extends GridProcessorAdapter {
 
             sharedCtx.removeCacheContext(ctx);
 
-            onKernalStop(cache, destroy);
+            onKernalStop(cache, true);
 
             stopCache(cache, true, destroy);
 
@@ -2017,9 +2004,7 @@ public class GridCacheProcessor extends GridProcessorAdapter {
      * @param startTopVer Cache start version.
      * @param err Cache start error if any.
      */
-    void initCacheProxies(
-        AffinityTopologyVersion startTopVer, @Nullable
-        Throwable err) {
+    void initCacheProxies(AffinityTopologyVersion startTopVer, @Nullable Throwable err) {
         for (GridCacheAdapter<?, ?> cache : caches.values()) {
             GridCacheContext<?, ?> cacheCtx = cache.context();
 
@@ -2122,7 +2107,7 @@ public class GridCacheProcessor extends GridProcessorAdapter {
         if (exchActions == null)
             return;
 
-        if (exchActions.systemCachesStarting() && exchActions.newClusterState() == null)
+        if (exchActions.systemCachesStarting() && exchActions.stateChangeRequest() == null)
             ctx.dataStructures().restoreStructuresState(ctx);
 
         if (err == null) {
@@ -2143,9 +2128,6 @@ public class GridCacheProcessor extends GridProcessorAdapter {
 
                 try {
                     prepareCacheStop(action.request().cacheName(), action.request().destroy());
-
-                    if (exchActions.newClusterState() == null)
-                        ctx.state().onCacheStop(action.request());
                 }
                 finally {
                     sharedCtx.database().checkpointReadUnlock();
@@ -2166,6 +2148,9 @@ public class GridCacheProcessor extends GridProcessorAdapter {
 
             if (!sharedCtx.kernalContext().clientNode())
                 sharedCtx.database().onCacheGroupsStopped(stoppedGroups);
+
+            if (exchActions.deactivate())
+                sharedCtx.deactivate();
         }
     }
 
@@ -2204,10 +2189,11 @@ public class GridCacheProcessor extends GridProcessorAdapter {
 
     /**
      * @param req Request to complete future for.
+     * @param success Future result.
      * @param err Error if any.
      */
     void completeCacheStartFuture(DynamicCacheChangeRequest req, boolean success, @Nullable Throwable err) {
-        if (req.initiatingNodeId().equals(ctx.localNodeId())) {
+        if (ctx.localNodeId().equals(req.initiatingNodeId())) {
             DynamicCacheStartFuture fut = (DynamicCacheStartFuture)pendingFuts.get(req.requestId());
 
             if (fut != null)
@@ -2304,30 +2290,35 @@ public class GridCacheProcessor extends GridProcessorAdapter {
 
     /** {@inheritDoc} */
     @Override public void collectGridNodeData(DiscoveryDataBag dataBag) {
-        if (ctx.state().active())
-            cachesInfo.collectGridNodeData(dataBag);
-        else
-            ctx.state().collectGridNodeData0(dataBag);
+        cachesInfo.collectGridNodeData(dataBag);
     }
 
     /** {@inheritDoc} */
     @Override public void onJoiningNodeDataReceived(JoiningNodeDiscoveryData data) {
-        if (ctx.state().active())
-            cachesInfo.onJoiningNodeDataReceived(data);
-
-        ctx.state().onJoiningNodeDataReceived0(data);
+        cachesInfo.onJoiningNodeDataReceived(data);
     }
 
     /** {@inheritDoc} */
     @Override public void onGridDataReceived(GridDiscoveryData data) {
-        if (ctx.state().active()) {
-            if (!cachesInfo.disconnectedState())
-                cachesInfo.addJoinInfo();
+        cachesInfo.onGridDataReceived(data);
+    }
 
-            cachesInfo.onGridDataReceived(data);
-        }
+    /**
+     * @param msg Message.
+     */
+    public void onStateChangeFinish(ChangeGlobalStateFinishMessage msg) {
+        cachesInfo.onStateChangeFinish(msg);
+    }
 
-        ctx.state().onGridDataReceived0(data);
+    /**
+     * @param msg Message.
+     * @param topVer Current topology version.
+     * @throws IgniteCheckedException If configuration validation failed.
+     * @return Exchange actions.
+     */
+    public ExchangeActions onStateChangeRequest(ChangeGlobalStateMessage msg, AffinityTopologyVersion topVer)
+        throws IgniteCheckedException {
+        return cachesInfo.onStateChangeRequest(msg, topVer);
     }
 
     /**
@@ -2929,13 +2920,19 @@ public class GridCacheProcessor extends GridProcessorAdapter {
 
     /**
      * @param type Event type.
+     * @param customMsg Custom message instance.
      * @param node Event node.
      * @param topVer Topology version.
+     * @param state Cluster state.
      */
-    public void onDiscoveryEvent(int type, ClusterNode node, AffinityTopologyVersion topVer) {
+    public void onDiscoveryEvent(int type,
+        @Nullable DiscoveryCustomMessage customMsg,
+        ClusterNode node,
+        AffinityTopologyVersion topVer,
+        DiscoveryDataClusterState state) {
         cachesInfo.onDiscoveryEvent(type, node, topVer);
 
-        sharedCtx.affinity().onDiscoveryEvent(type, node, topVer);
+        sharedCtx.affinity().onDiscoveryEvent(type, customMsg, node, topVer, state);
     }
 
     /**
@@ -3214,7 +3211,7 @@ public class GridCacheProcessor extends GridProcessorAdapter {
                     proxy = new IgniteCacheProxy(cacheAdapter.context(), cacheAdapter, null, false);
             }
 
-            assert proxy != null;
+            assert proxy != null : name;
 
             return proxy.internalProxy();
         }

http://git-wip-us.apache.org/repos/asf/ignite/blob/1337901f/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheSharedContext.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheSharedContext.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheSharedContext.java
index 75d03d7..9adca8d 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheSharedContext.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheSharedContext.java
@@ -17,6 +17,7 @@
 
 package org.apache.ignite.internal.processors.cache;
 
+import java.util.ArrayList;
 import java.util.Collection;
 import java.util.LinkedList;
 import java.util.List;
@@ -45,7 +46,6 @@ import org.apache.ignite.internal.processors.cache.distributed.dht.GridDhtTopolo
 import org.apache.ignite.internal.processors.cache.persistence.IgniteCacheDatabaseSharedManager;
 import org.apache.ignite.internal.processors.cache.persistence.IgniteCacheSnapshotManager;
 import org.apache.ignite.internal.processors.cache.distributed.dht.GridDhtPartitionTopology;
-import org.apache.ignite.internal.processors.cache.distributed.dht.GridDhtTopologyFuture;
 import org.apache.ignite.internal.processors.cache.distributed.near.GridNearTxLocal;
 import org.apache.ignite.internal.processors.cache.jta.CacheJtaManagerAdapter;
 import org.apache.ignite.internal.processors.cache.store.CacheStoreManager;
@@ -54,6 +54,7 @@ import org.apache.ignite.internal.processors.cache.transactions.IgniteTxManager;
 import org.apache.ignite.internal.processors.cache.transactions.TransactionMetricsAdapter;
 import org.apache.ignite.internal.processors.cache.version.GridCacheVersion;
 import org.apache.ignite.internal.processors.cache.version.GridCacheVersionManager;
+import org.apache.ignite.internal.processors.cluster.IgniteChangeGlobalStateSupport;
 import org.apache.ignite.internal.processors.timeout.GridTimeoutProcessor;
 import org.apache.ignite.internal.util.GridIntList;
 import org.apache.ignite.internal.util.future.GridCompoundFuture;
@@ -156,6 +157,9 @@ public class GridCacheSharedContext<K, V> {
     /** Concurrent DHT atomic updates counters. */
     private AtomicIntegerArray dhtAtomicUpdCnt;
 
+    /** */
+    private final List<IgniteChangeGlobalStateSupport> stateAwareMgrs;
+
     /**
      * @param kernalCtx  Context.
      * @param txMgr Transaction manager.
@@ -207,6 +211,49 @@ public class GridCacheSharedContext<K, V> {
         txFinishMsgLog = kernalCtx.log(CU.TX_MSG_FINISH_LOG_CATEGORY);
         txLockMsgLog = kernalCtx.log(CU.TX_MSG_LOCK_LOG_CATEGORY);
         txRecoveryMsgLog = kernalCtx.log(CU.TX_MSG_RECOVERY_LOG_CATEGORY);
+
+        stateAwareMgrs = new ArrayList<>();
+
+        if (pageStoreMgr != null)
+            stateAwareMgrs.add(pageStoreMgr);
+
+        if (walMgr != null)
+            stateAwareMgrs.add(walMgr);
+
+        stateAwareMgrs.add(dbMgr);
+
+        stateAwareMgrs.add(snpMgr);
+    }
+
+    /**
+     * @throws IgniteCheckedException If failed.
+     */
+    public void activate() throws IgniteCheckedException {
+        if (!kernalCtx.clientNode())
+            dbMgr.lock();
+
+        boolean success = false;
+
+        try {
+            for (IgniteChangeGlobalStateSupport mgr : stateAwareMgrs)
+                mgr.onActivate(kernalCtx);
+
+            success = true;
+        }
+        finally {
+            if (!success) {
+                if (!kernalCtx.clientNode())
+                    dbMgr.unLock();
+            }
+        }
+    }
+
+    /**
+     *
+     */
+    public void deactivate() {
+        for (int i = stateAwareMgrs.size() - 1; i >= 0; i--)
+            stateAwareMgrs.get(i).onDeActivate(kernalCtx);
     }
 
     /**
@@ -272,12 +319,15 @@ public class GridCacheSharedContext<K, V> {
             if (restartOnDisconnect(mgr))
                 mgr.stop(true);
         }
+
+        deactivate();
     }
 
     /**
+     * @param active Active flag.
      * @throws IgniteCheckedException If failed.
      */
-    void onReconnected() throws IgniteCheckedException {
+    void onReconnected(boolean active) throws IgniteCheckedException {
         List<GridCacheSharedManager<K, V>> mgrs = new LinkedList<>();
 
         setManagers(mgrs, txMgr,
@@ -303,8 +353,10 @@ public class GridCacheSharedContext<K, V> {
 
         kernalCtx.query().onCacheReconnect();
 
-        for (GridCacheSharedManager<?, ?> mgr : mgrs)
-            mgr.onKernalStart(true);
+        if (!active)
+            affinity().removeAllCacheInfo();
+
+        exchMgr.onKernalStart(active, true);
     }
 
     /**

http://git-wip-us.apache.org/repos/asf/ignite/blob/1337901f/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheSharedManager.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheSharedManager.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheSharedManager.java
index e0e4090..bc1bbb9 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheSharedManager.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheSharedManager.java
@@ -40,12 +40,6 @@ public interface GridCacheSharedManager<K, V> {
     public void stop(boolean cancel);
 
     /**
-     * @param reconnect {@code True} if manager restarted after client reconnect.
-     * @throws IgniteCheckedException If failed.
-     */
-    public void onKernalStart(boolean reconnect) throws IgniteCheckedException;
-
-    /**
      * @param cancel Cancel flag.
      */
     public void onKernalStop(boolean cancel);

http://git-wip-us.apache.org/repos/asf/ignite/blob/1337901f/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheSharedManagerAdapter.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheSharedManagerAdapter.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheSharedManagerAdapter.java
index f6f79e4..90ae670 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheSharedManagerAdapter.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheSharedManagerAdapter.java
@@ -112,14 +112,6 @@ public class GridCacheSharedManagerAdapter<K, V> implements GridCacheSharedManag
     }
 
     /** {@inheritDoc} */
-    @Override public final void onKernalStart(boolean reconnect) throws IgniteCheckedException {
-        onKernalStart0(reconnect);
-
-        if (!reconnect && log != null && log.isDebugEnabled())
-            log.debug(kernalStartInfo());
-    }
-
-    /** {@inheritDoc} */
     @Override public final void onKernalStop(boolean cancel) {
         if (!starting.get())
             // Ignoring attempt to stop manager that has never been started.
@@ -132,14 +124,6 @@ public class GridCacheSharedManagerAdapter<K, V> implements GridCacheSharedManag
     }
 
     /**
-     * @param reconnect {@code True} if manager restarted after client reconnect.
-     * @throws IgniteCheckedException If failed.
-     */
-    protected void onKernalStart0(boolean reconnect) throws IgniteCheckedException {
-        // No-op.
-    }
-
-    /**
      * @param cancel Cancel flag.
      */
     protected void onKernalStop0(boolean cancel) {

http://git-wip-us.apache.org/repos/asf/ignite/blob/1337901f/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/PendingDiscoveryEvent.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/PendingDiscoveryEvent.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/PendingDiscoveryEvent.java
new file mode 100644
index 0000000..b4274f7
--- /dev/null
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/PendingDiscoveryEvent.java
@@ -0,0 +1,61 @@
+/*
+ * 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.events.DiscoveryEvent;
+import org.apache.ignite.internal.managers.discovery.DiscoCache;
+import org.apache.ignite.internal.util.typedef.internal.S;
+
+/**
+ *
+ */
+public class PendingDiscoveryEvent {
+    /** */
+    private final DiscoveryEvent evt;
+
+    /** */
+    private final DiscoCache cache;
+
+    /**
+     * @param evt Event.
+     * @param cache Discovery data cache.
+     */
+    public PendingDiscoveryEvent(DiscoveryEvent evt, DiscoCache cache) {
+        this.evt = evt;
+        this.cache = cache;
+    }
+
+    /**
+     * @return Event.
+     */
+    public DiscoveryEvent event() {
+        return evt;
+    }
+
+    /**
+     * @return Discovery data cache.
+     */
+    public DiscoCache discoCache() {
+        return cache;
+    }
+
+    /** {@inheritDoc} */
+    @Override public String toString() {
+        return S.toString(PendingDiscoveryEvent.class, this);
+    }
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/1337901f/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/StateChangeRequest.java
----------------------------------------------------------------------
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
new file mode 100644
index 0000000..2d35e81
--- /dev/null
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/StateChangeRequest.java
@@ -0,0 +1,77 @@
+/*
+ * 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.UUID;
+import org.apache.ignite.internal.processors.affinity.AffinityTopologyVersion;
+import org.apache.ignite.internal.processors.cluster.ChangeGlobalStateMessage;
+import org.apache.ignite.internal.util.typedef.internal.S;
+
+/**
+ *
+ */
+public class StateChangeRequest {
+    /** */
+    private final ChangeGlobalStateMessage msg;
+
+    /** */
+    private final AffinityTopologyVersion topVer;
+
+    /**
+     * @param msg Message.
+     * @param topVer State change topology versoin.
+     */
+    public StateChangeRequest(ChangeGlobalStateMessage msg,
+        AffinityTopologyVersion topVer) {
+        this.msg = msg;
+        this.topVer = topVer;
+    }
+
+    /**
+     * @return State change exchange version.
+     */
+    public AffinityTopologyVersion topologyVersion() {
+        return topVer;
+    }
+
+    /**
+     * @return State change request ID.
+     */
+    public UUID requestId() {
+        return msg.requestId();
+    }
+
+    /**
+     * @return New state.
+     */
+    public boolean activate() {
+        return msg.activate();
+    }
+
+    /**
+     * @return Node initiated state change process.
+     */
+    public UUID initiatorNodeId() {
+        return msg.initiatorNodeId();
+    }
+
+    /** {@inheritDoc} */
+    @Override public String toString() {
+        return S.toString(StateChangeRequest.class, this);
+    }
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/1337901f/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/binary/CacheObjectBinaryProcessorImpl.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/binary/CacheObjectBinaryProcessorImpl.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/binary/CacheObjectBinaryProcessorImpl.java
index c2c71ea..0065e41 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/binary/CacheObjectBinaryProcessorImpl.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/binary/CacheObjectBinaryProcessorImpl.java
@@ -258,8 +258,8 @@ public class CacheObjectBinaryProcessorImpl extends IgniteCacheObjectProcessorIm
     }
 
     /** {@inheritDoc} */
-    @Override public void onKernalStart() throws IgniteCheckedException {
-        super.onKernalStart();
+    @Override public void onKernalStart(boolean active) throws IgniteCheckedException {
+        super.onKernalStart(active);
 
         discoveryStarted = true;
     }

http://git-wip-us.apache.org/repos/asf/ignite/blob/1337901f/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/GridCacheTxRecoveryFuture.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/GridCacheTxRecoveryFuture.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/GridCacheTxRecoveryFuture.java
index 1c97de2..d6b45b3 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/GridCacheTxRecoveryFuture.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/GridCacheTxRecoveryFuture.java
@@ -29,7 +29,6 @@ import org.apache.ignite.cluster.ClusterNode;
 import org.apache.ignite.internal.IgniteInternalFuture;
 import org.apache.ignite.internal.cluster.ClusterTopologyCheckedException;
 import org.apache.ignite.internal.processors.cache.GridCacheCompoundIdentityFuture;
-import org.apache.ignite.internal.processors.cache.GridCacheFuture;
 import org.apache.ignite.internal.processors.cache.GridCacheSharedContext;
 import org.apache.ignite.internal.processors.cache.transactions.IgniteInternalTx;
 import org.apache.ignite.internal.util.GridLeanMap;

http://git-wip-us.apache.org/repos/asf/ignite/blob/1337901f/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtCacheAdapter.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtCacheAdapter.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtCacheAdapter.java
index 38d0108..960b91a 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtCacheAdapter.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtCacheAdapter.java
@@ -50,7 +50,6 @@ import org.apache.ignite.internal.processors.cache.GridCacheEntryEx;
 import org.apache.ignite.internal.processors.cache.GridCacheEntryInfo;
 import org.apache.ignite.internal.processors.cache.GridCacheEntryRemovedException;
 import org.apache.ignite.internal.processors.cache.GridCacheMapEntry;
-import org.apache.ignite.internal.processors.cache.GridCacheMapEntryFactory;
 import org.apache.ignite.internal.processors.cache.GridCachePreloader;
 import org.apache.ignite.internal.processors.cache.IgniteCacheExpiryPolicy;
 import org.apache.ignite.internal.processors.cache.KeyCacheObject;

http://git-wip-us.apache.org/repos/asf/ignite/blob/1337901f/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtGetFuture.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtGetFuture.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtGetFuture.java
index 123d26b..0ea48e3 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtGetFuture.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtGetFuture.java
@@ -27,7 +27,6 @@ import java.util.UUID;
 import java.util.concurrent.atomic.AtomicReference;
 import org.apache.ignite.IgniteCheckedException;
 import org.apache.ignite.IgniteLogger;
-import org.apache.ignite.cluster.ClusterNode;
 import org.apache.ignite.internal.IgniteInternalFuture;
 import org.apache.ignite.internal.processors.affinity.AffinityTopologyVersion;
 import org.apache.ignite.internal.processors.cache.CacheObject;

http://git-wip-us.apache.org/repos/asf/ignite/blob/1337901f/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtGetSingleFuture.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtGetSingleFuture.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtGetSingleFuture.java
index 6392d0a..439bb9d 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtGetSingleFuture.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtGetSingleFuture.java
@@ -19,13 +19,11 @@ package org.apache.ignite.internal.processors.cache.distributed.dht;
 
 import java.util.Collection;
 import java.util.Collections;
-import java.util.HashSet;
 import java.util.Map;
 import java.util.UUID;
 import java.util.concurrent.atomic.AtomicReference;
 import org.apache.ignite.IgniteCheckedException;
 import org.apache.ignite.IgniteLogger;
-import org.apache.ignite.cluster.ClusterNode;
 import org.apache.ignite.internal.IgniteInternalFuture;
 import org.apache.ignite.internal.NodeStoppingException;
 import org.apache.ignite.internal.processors.affinity.AffinityTopologyVersion;

http://git-wip-us.apache.org/repos/asf/ignite/blob/1337901f/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtPartitionTopologyImpl.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtPartitionTopologyImpl.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtPartitionTopologyImpl.java
index c205c3b..57ce323 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtPartitionTopologyImpl.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtPartitionTopologyImpl.java
@@ -41,7 +41,6 @@ import org.apache.ignite.internal.managers.discovery.DiscoCache;
 import org.apache.ignite.internal.processors.affinity.AffinityAssignment;
 import org.apache.ignite.internal.processors.affinity.AffinityTopologyVersion;
 import org.apache.ignite.internal.processors.cache.CacheGroupContext;
-import org.apache.ignite.internal.processors.cache.ClusterState;
 import org.apache.ignite.internal.processors.cache.GridCacheSharedContext;
 import org.apache.ignite.internal.processors.cache.distributed.dht.preloader.GridDhtPartitionExchangeId;
 import org.apache.ignite.internal.processors.cache.distributed.dht.preloader.GridDhtPartitionFullMap;
@@ -424,11 +423,8 @@ public class GridDhtPartitionTopologyImpl implements GridDhtPartitionTopology {
         throws IgniteCheckedException {
         DiscoveryEvent discoEvt = exchFut.discoveryEvent();
 
-        ClusterState newState = exchFut.newClusterState();
-
-        treatAllPartAsLoc = (newState != null && newState == ClusterState.ACTIVE)
-            || (ctx.kernalContext().state().active()
-            && discoEvt.type() == EventType.EVT_NODE_JOINED
+        treatAllPartAsLoc = exchFut.activateCluster()
+            || (discoEvt.type() == EventType.EVT_NODE_JOINED
             && discoEvt.eventNode().isLocal()
             && !ctx.kernalContext().clientNode()
         );
@@ -611,7 +607,7 @@ public class GridDhtPartitionTopologyImpl implements GridDhtPartitionTopology {
                     if (locPart != null) {
                         GridDhtPartitionState state = locPart.state();
 
-                        if (state == MOVING && ctx.kernalContext().state().active()) {
+                        if (state == MOVING) {
                             locPart.rent(false);
 
                             updateSeq = updateLocal(p, locPart.state(), updateSeq);
@@ -1773,9 +1769,6 @@ public class GridDhtPartitionTopologyImpl implements GridDhtPartitionTopology {
      * @return Checks if any of the local partitions need to be evicted.
      */
     private boolean checkEvictions(long updateSeq, List<List<ClusterNode>> aff) {
-        if (!ctx.kernalContext().state().active())
-            return false;
-
         boolean changed = false;
 
         UUID locId = ctx.localNodeId();

http://git-wip-us.apache.org/repos/asf/ignite/blob/1337901f/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtTopologyFutureAdapter.java
----------------------------------------------------------------------
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 e70f383..d04870a 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
@@ -80,7 +80,7 @@ public abstract class GridDhtTopologyFutureAdapter extends GridFutureAdapter<Aff
         if (err != null)
             return err;
 
-        if (!cctx.shared().kernalContext().state().active())
+        if (!cctx.shared().kernalContext().state().publicApiActiveState())
             return new CacheInvalidStateException(
                 "Failed to perform cache operation (cluster is not activated): " + cctx.name());
 

http://git-wip-us.apache.org/repos/asf/ignite/blob/1337901f/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridPartitionedSingleGetFuture.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridPartitionedSingleGetFuture.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridPartitionedSingleGetFuture.java
index cfecb1c..d66afca 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridPartitionedSingleGetFuture.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridPartitionedSingleGetFuture.java
@@ -18,16 +18,13 @@
 package org.apache.ignite.internal.processors.cache.distributed.dht;
 
 import java.util.Collection;
-import java.util.Collections;
 import java.util.List;
-import java.util.Map;
 import java.util.UUID;
 import java.util.concurrent.atomic.AtomicReference;
 import org.apache.ignite.IgniteCheckedException;
 import org.apache.ignite.IgniteLogger;
 import org.apache.ignite.cluster.ClusterNode;
 import org.apache.ignite.internal.IgniteDiagnosticAware;
-import org.apache.ignite.internal.IgniteDiagnosticMessage;
 import org.apache.ignite.internal.IgniteDiagnosticPrepareContext;
 import org.apache.ignite.internal.IgniteInternalFuture;
 import org.apache.ignite.internal.cluster.ClusterTopologyCheckedException;

http://git-wip-us.apache.org/repos/asf/ignite/blob/1337901f/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/atomic/GridNearAtomicAbstractUpdateFuture.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/atomic/GridNearAtomicAbstractUpdateFuture.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/atomic/GridNearAtomicAbstractUpdateFuture.java
index 1bd8ec5..6fe96a4 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/atomic/GridNearAtomicAbstractUpdateFuture.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/atomic/GridNearAtomicAbstractUpdateFuture.java
@@ -46,7 +46,6 @@ import org.apache.ignite.internal.processors.cache.GridCacheReturn;
 import org.apache.ignite.internal.processors.cache.KeyCacheObject;
 import org.apache.ignite.internal.util.future.GridFutureAdapter;
 import org.apache.ignite.internal.util.tostring.GridToStringInclude;
-import org.apache.ignite.internal.util.typedef.F;
 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;

http://git-wip-us.apache.org/repos/asf/ignite/blob/1337901f/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/preloader/GridDhtForceKeysFuture.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/preloader/GridDhtForceKeysFuture.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/preloader/GridDhtForceKeysFuture.java
index 763b43b..fe216a0 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/preloader/GridDhtForceKeysFuture.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/preloader/GridDhtForceKeysFuture.java
@@ -23,7 +23,6 @@ import java.util.Collections;
 import java.util.List;
 import java.util.Map;
 import java.util.Set;
-import java.util.UUID;
 import java.util.concurrent.CountDownLatch;
 import java.util.concurrent.atomic.AtomicInteger;
 import java.util.concurrent.atomic.AtomicReference;

http://git-wip-us.apache.org/repos/asf/ignite/blob/1337901f/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/preloader/GridDhtPartitionDemander.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/preloader/GridDhtPartitionDemander.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/preloader/GridDhtPartitionDemander.java
index 774f0ce..e7e95b2 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/preloader/GridDhtPartitionDemander.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/preloader/GridDhtPartitionDemander.java
@@ -161,6 +161,8 @@ public class GridDhtPartitionDemander {
         lastExchangeFut = null;
 
         lastTimeoutObj.set(null);
+
+        syncFut.onDone();
     }
 
     /**


[18/33] ignite git commit: IGNITE-5558 - Added ability to read WAL in standalone mode - Fixes #2174.

Posted by sb...@apache.org.
IGNITE-5558 - Added ability to read WAL in standalone mode - Fixes #2174.

Signed-off-by: Alexey Goncharuk <al...@gmail.com>


Project: http://git-wip-us.apache.org/repos/asf/ignite/repo
Commit: http://git-wip-us.apache.org/repos/asf/ignite/commit/44f3fac2
Tree: http://git-wip-us.apache.org/repos/asf/ignite/tree/44f3fac2
Diff: http://git-wip-us.apache.org/repos/asf/ignite/diff/44f3fac2

Branch: refs/heads/ignite-2.1.2-exchange
Commit: 44f3fac27bec89b5e70e87564c527e48565ddd2a
Parents: ee7566b
Author: dpavlov <dp...@gridgain.com>
Authored: Tue Jul 4 20:23:40 2017 +0300
Committer: Alexey Goncharuk <al...@gmail.com>
Committed: Tue Jul 4 20:23:40 2017 +0300

----------------------------------------------------------------------
 .../PersistentStoreConfiguration.java           |  39 +-
 .../org/apache/ignite/events/EventType.java     |  12 +
 .../ignite/events/WalSegmentArchivedEvent.java  |  62 ++
 .../internal/pagemem/wal/record/WALRecord.java  |  11 +-
 .../IgniteCacheDatabaseSharedManager.java       |  10 +-
 .../wal/AbstractWalRecordsIterator.java         | 289 +++++++++
 .../cache/persistence/wal/FileInput.java        |  16 +-
 .../cache/persistence/wal/FileWALPointer.java   |   4 +-
 .../wal/FileWriteAheadLogManager.java           | 586 +++++++++----------
 .../cache/persistence/wal/RecordSerializer.java |   5 +
 .../persistence/wal/SegmentArchiveResult.java   |  61 ++
 .../persistence/wal/SegmentEofException.java    |   3 +-
 .../wal/reader/IgniteWalIteratorFactory.java    | 102 ++++
 .../wal/reader/StandaloneGridKernalContext.java | 499 ++++++++++++++++
 ...ndaloneIgniteCacheDatabaseSharedManager.java |  30 +
 .../reader/StandaloneWalRecordsIterator.java    | 258 ++++++++
 .../wal/serializer/RecordV1Serializer.java      |  45 +-
 ...IgnitePersistentStoreDataStructuresTest.java |   2 +
 .../wal/IgniteWalHistoryReservationsTest.java   |   2 +-
 .../db/wal/reader/IgniteWalReaderTest.java      | 385 ++++++++++++
 .../db/wal/reader/MockWalIteratorFactory.java   | 114 ++++
 .../ignite/testsuites/IgnitePdsTestSuite2.java  |   9 +-
 22 files changed, 2194 insertions(+), 350 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/44f3fac2/modules/core/src/main/java/org/apache/ignite/configuration/PersistentStoreConfiguration.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/configuration/PersistentStoreConfiguration.java b/modules/core/src/main/java/org/apache/ignite/configuration/PersistentStoreConfiguration.java
index 1d41d41..b531f9d 100644
--- a/modules/core/src/main/java/org/apache/ignite/configuration/PersistentStoreConfiguration.java
+++ b/modules/core/src/main/java/org/apache/ignite/configuration/PersistentStoreConfiguration.java
@@ -55,7 +55,7 @@ public class PersistentStoreConfiguration implements Serializable {
     /** */
     public static final int DFLT_WAL_SEGMENTS = 10;
 
-    /** */
+    /** Default WAL file segment size, 64MBytes */
     public static final int DFLT_WAL_SEGMENT_SIZE = 64 * 1024 * 1024;
 
     /** Default wal mode. */
@@ -103,10 +103,10 @@ public class PersistentStoreConfiguration implements Serializable {
     /** Number of work WAL segments. */
     private int walSegments = DFLT_WAL_SEGMENTS;
 
-    /** Number of WAL segments to keep. */
+    /** Size of one WAL segment in bytes. 64 Mb is used by default.  Maximum value is 2Gb */
     private int walSegmentSize = DFLT_WAL_SEGMENT_SIZE;
 
-    /** WAL persistence path. */
+    /** Directory where WAL is stored (work directory) */
     private String walStorePath = DFLT_WAL_STORE_PATH;
 
     /** WAL archive path. */
@@ -121,7 +121,7 @@ public class PersistentStoreConfiguration implements Serializable {
     /** WAl thread local buffer size. */
     private int tlbSize = DFLT_TLB_SIZE;
 
-    /** Wal flush frequency. */
+    /** Wal flush frequency in milliseconds. */
     private int walFlushFreq = DFLT_WAL_FLUSH_FREQ;
 
     /** Wal fsync delay. */
@@ -147,6 +147,11 @@ public class PersistentStoreConfiguration implements Serializable {
     private long rateTimeInterval = DFLT_RATE_TIME_INTERVAL_MILLIS;
 
     /**
+     *  Time interval (in milliseconds) for running auto archiving for incompletely WAL segment
+     */
+    private long walAutoArchiveAfterInactivity = -1;
+
+    /**
      * Returns a path the root directory where the Persistent Store will persist data and indexes.
      */
     public String getPersistentStorePath() {
@@ -297,7 +302,7 @@ public class PersistentStoreConfiguration implements Serializable {
     }
 
     /**
-     * Gets size of a WAL segment.
+     * Gets size of a WAL segment in bytes.
      *
      * @return WAL segment size.
      */
@@ -308,7 +313,7 @@ public class PersistentStoreConfiguration implements Serializable {
     /**
      * Sets size of a WAL segment.
      *
-     * @param walSegmentSize WAL segment size. 64 MB is used by default.
+     * @param walSegmentSize WAL segment size. 64 MB is used by default.  Maximum value is 2Gb
      * @return {@code this} for chaining.
      */
     public PersistentStoreConfiguration setWalSegmentSize(int walSegmentSize) {
@@ -533,6 +538,28 @@ public class PersistentStoreConfiguration implements Serializable {
         return this;
     }
 
+    /**
+     * <b>Note:</b> setting this value with {@link WALMode#DEFAULT} may generate file size overhead for WAL segments in case
+     * grid is used rarely.
+     *
+     * @param walAutoArchiveAfterInactivity time in millis to run auto archiving segment (even if incomplete) after last
+     * record logging. <br> Positive value enables incomplete segment archiving after timeout (inactivity). <br> Zero or
+     * negative  value disables auto archiving.
+     * @return current configuration instance for chaining
+     */
+    public PersistentStoreConfiguration setWalAutoArchiveAfterInactivity(long walAutoArchiveAfterInactivity) {
+        this.walAutoArchiveAfterInactivity = walAutoArchiveAfterInactivity;
+
+        return this;
+    }
+
+    /**
+     * @return time in millis to run auto archiving WAL segment (even if incomplete) after last record log
+     */
+    public long getWalAutoArchiveAfterInactivity() {
+        return walAutoArchiveAfterInactivity;
+    }
+
     /** {@inheritDoc} */
     @Override public String toString() {
         return S.toString(PersistentStoreConfiguration.class, this);

http://git-wip-us.apache.org/repos/asf/ignite/blob/44f3fac2/modules/core/src/main/java/org/apache/ignite/events/EventType.java
----------------------------------------------------------------------
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 1960692..47b4089 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
@@ -767,6 +767,18 @@ public interface EventType {
     public static final int EVT_IGFS_FILE_PURGED = 127;
 
     /**
+     * Built-in event type: WAL segment movement to archive folder completed
+     * <p>
+     * Fired for each completed WAL segment which was moved to archive
+     * <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 WalSegmentArchivedEvent
+     */
+    public static final int EVT_WAL_SEGMENT_ARCHIVED = 128;
+
+    /**
      * All checkpoint events. This array can be directly passed into
      * {@link IgniteEvents#localListen(IgnitePredicate, int...)} method to
      * subscribe to all checkpoint events.

http://git-wip-us.apache.org/repos/asf/ignite/blob/44f3fac2/modules/core/src/main/java/org/apache/ignite/events/WalSegmentArchivedEvent.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/events/WalSegmentArchivedEvent.java b/modules/core/src/main/java/org/apache/ignite/events/WalSegmentArchivedEvent.java
new file mode 100644
index 0000000..2fc1715
--- /dev/null
+++ b/modules/core/src/main/java/org/apache/ignite/events/WalSegmentArchivedEvent.java
@@ -0,0 +1,62 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.ignite.events;
+
+import java.io.File;
+import org.apache.ignite.cluster.ClusterNode;
+import org.jetbrains.annotations.NotNull;
+
+/**
+ * Event indicates there was movement of WAL segment file to archive has been completed
+ */
+public class WalSegmentArchivedEvent extends EventAdapter {
+    /** */
+    private static final long serialVersionUID = 0L;
+
+    /** Absolute WAL segment file index. */
+    private long absWalSegmentIdx;
+
+    /** Destination archive file. This file is completed and closed archive segment */
+    private final File archiveFile;
+
+    /**
+     * Creates WAL segment event
+     *
+     * @param node Node.
+     * @param absWalSegmentIdx Absolute wal segment index.
+     * @param archiveFile Archive file.
+     */
+    public WalSegmentArchivedEvent(
+        @NotNull final ClusterNode node,
+        final long absWalSegmentIdx,
+        final File archiveFile) {
+        super(node, "", EventType.EVT_WAL_SEGMENT_ARCHIVED);
+        this.absWalSegmentIdx = absWalSegmentIdx;
+        this.archiveFile = archiveFile;
+    }
+
+    /** @return {@link #archiveFile} */
+    public File getArchiveFile() {
+        return archiveFile;
+    }
+
+    /** @return {@link #absWalSegmentIdx} */
+    public long getAbsWalSegmentIdx() {
+        return absWalSegmentIdx;
+    }
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/44f3fac2/modules/core/src/main/java/org/apache/ignite/internal/pagemem/wal/record/WALRecord.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/pagemem/wal/record/WALRecord.java b/modules/core/src/main/java/org/apache/ignite/internal/pagemem/wal/record/WALRecord.java
index 678e1fa..89f3c86 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/pagemem/wal/record/WALRecord.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/pagemem/wal/record/WALRecord.java
@@ -17,6 +17,7 @@
 
 package org.apache.ignite.internal.pagemem.wal.record;
 
+import org.apache.ignite.configuration.WALMode;
 import org.apache.ignite.internal.pagemem.wal.WALPointer;
 import org.apache.ignite.internal.util.tostring.GridToStringExclude;
 import org.apache.ignite.internal.util.typedef.internal.S;
@@ -26,7 +27,8 @@ import org.apache.ignite.internal.util.typedef.internal.S;
  */
 public abstract class WALRecord {
     /**
-     * Record type.
+     * Record type. Ordinal of this record will be written to file. <br>
+     * <b>Note:</b> Do not change order of elements <br>
      */
     public enum RecordType {
         /** */
@@ -171,6 +173,13 @@ public abstract class WALRecord {
         public static RecordType fromOrdinal(int ord) {
             return ord < 0 || ord >= VALS.length ? null : VALS[ord];
         }
+
+        /**
+         * Fake record type, causes stop iterating and indicates segment EOF
+         * <b>Note:</b> regular record type is incremented by 1 and minimal value written to file is also 1
+         * For {@link WALMode#DEFAULT} this value is at least came from padding
+         */
+        public static final int STOP_ITERATION_RECORD_TYPE = 0;
     }
 
     /** */

http://git-wip-us.apache.org/repos/asf/ignite/blob/44f3fac2/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/IgniteCacheDatabaseSharedManager.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/IgniteCacheDatabaseSharedManager.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/IgniteCacheDatabaseSharedManager.java
index ec0e895..f04c278 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/IgniteCacheDatabaseSharedManager.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/IgniteCacheDatabaseSharedManager.java
@@ -92,7 +92,7 @@ public class IgniteCacheDatabaseSharedManager extends GridCacheSharedManagerAdap
     /** */
     private FreeListImpl dfltFreeList;
 
-    /** */
+    /** Page size from memory configuration, may be set only for fake(standalone) IgniteCacheDataBaseSharedManager */
     private int pageSize;
 
     /** {@inheritDoc} */
@@ -961,4 +961,12 @@ public class IgniteCacheDatabaseSharedManager extends GridCacheSharedManagerAdap
     public String systemMemoryPolicyName() {
         return SYSTEM_MEMORY_POLICY_NAME;
     }
+
+    /**
+     * Method for fake (standalone) context initialization. Not to be called in production code
+     * @param pageSize configured page size
+     */
+    protected void setPageSize(int pageSize) {
+        this.pageSize = pageSize;
+    }
 }

http://git-wip-us.apache.org/repos/asf/ignite/blob/44f3fac2/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/wal/AbstractWalRecordsIterator.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/wal/AbstractWalRecordsIterator.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/wal/AbstractWalRecordsIterator.java
new file mode 100644
index 0000000..7dc0a28
--- /dev/null
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/wal/AbstractWalRecordsIterator.java
@@ -0,0 +1,289 @@
+/*
+ * 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.persistence.wal;
+
+import java.io.EOFException;
+import java.io.File;
+import java.io.FileNotFoundException;
+import java.io.IOException;
+import java.io.RandomAccessFile;
+import java.nio.ByteBuffer;
+import java.nio.ByteOrder;
+import java.nio.channels.FileChannel;
+import org.apache.ignite.IgniteCheckedException;
+import org.apache.ignite.IgniteLogger;
+import org.apache.ignite.internal.pagemem.wal.WALIterator;
+import org.apache.ignite.internal.pagemem.wal.WALPointer;
+import org.apache.ignite.internal.pagemem.wal.record.WALRecord;
+import org.apache.ignite.internal.processors.cache.GridCacheSharedContext;
+import org.apache.ignite.internal.processors.cache.persistence.wal.record.HeaderRecord;
+import org.apache.ignite.internal.util.GridCloseableIteratorAdapter;
+import org.apache.ignite.lang.IgniteBiTuple;
+import org.jetbrains.annotations.NotNull;
+import org.jetbrains.annotations.Nullable;
+
+/**
+ * Iterator over WAL segments. This abstract class provides most functionality for reading records in log.
+ * Subclasses are to override segment switching functionality
+ */
+public abstract class AbstractWalRecordsIterator extends GridCloseableIteratorAdapter<IgniteBiTuple<WALPointer, WALRecord>>
+    implements WALIterator {
+    /** */
+    private static final long serialVersionUID = 0L;
+
+    /**
+     * Current record preloaded, to be returned on next()<br>
+     * Normally this should be not null because advance() method should already prepare some value<br>
+     */
+    protected IgniteBiTuple<WALPointer, WALRecord> curRec;
+
+    /**
+     * Current WAL segment absolute index. <br>
+     * Determined as lowest number of file at start, is changed during advance segment
+     */
+    protected long curWalSegmIdx = -1;
+
+    /**
+     * Current WAL segment read file handle. To be filled by subclass advanceSegment
+     */
+    private FileWriteAheadLogManager.ReadFileHandle currWalSegment;
+
+    /** Logger */
+    @NotNull protected final IgniteLogger log;
+
+    /** Shared context for creating serializer of required version and grid name access */
+    @NotNull private final GridCacheSharedContext sharedCtx;
+
+    /** Serializer of current version to read headers. */
+    @NotNull private final RecordSerializer serializer;
+
+    /** Utility buffer for reading records */
+    private final ByteBuffer buf;
+
+    /**
+     * @param log Logger
+     * @param sharedCtx Shared context
+     * @param serializer Serializer of current version to read headers.
+     * @param bufSize buffer for reading records size
+     */
+    protected AbstractWalRecordsIterator(
+        @NotNull final IgniteLogger log,
+        @NotNull final GridCacheSharedContext sharedCtx,
+        @NotNull final RecordSerializer serializer,
+        final int bufSize) {
+        this.log = log;
+        this.sharedCtx = sharedCtx;
+        this.serializer = serializer;
+
+        // Do not allocate direct buffer for iterator.
+        buf = ByteBuffer.allocate(bufSize);
+        buf.order(ByteOrder.nativeOrder());
+
+    }
+
+    /**
+     * Scans provided folder for a WAL segment files
+     * @param walFilesDir directory to scan
+     * @return found WAL file descriptors
+     */
+    protected static FileWriteAheadLogManager.FileDescriptor[] loadFileDescriptors(@NotNull final File walFilesDir) throws IgniteCheckedException {
+        final File[] files = walFilesDir.listFiles(FileWriteAheadLogManager.WAL_SEGMENT_FILE_FILTER);
+
+        if (files == null) {
+            throw new IgniteCheckedException("WAL files directory does not not denote a " +
+                "directory, or if an I/O error occurs: [" + walFilesDir.getAbsolutePath() + "]");
+        }
+        return FileWriteAheadLogManager.scan(files);
+    }
+
+    /** {@inheritDoc} */
+    @Override protected IgniteBiTuple<WALPointer, WALRecord> onNext() throws IgniteCheckedException {
+        IgniteBiTuple<WALPointer, WALRecord> ret = curRec;
+
+        advance();
+
+        return ret;
+    }
+
+    /** {@inheritDoc} */
+    @Override protected boolean onHasNext() throws IgniteCheckedException {
+        return curRec != null;
+    }
+
+    /**
+     * Switches records iterator to the next record.
+     * <ul>
+     * <li>{@link #curRec} will be updated.</li>
+     * <li> If end of segment reached, switch to new segment is called. {@link #currWalSegment} will be updated.</li>
+     * </ul>
+     *
+     * {@code advance()} runs a step ahead {@link #next()}
+     *
+     * @throws IgniteCheckedException If failed.
+     */
+    protected void advance() throws IgniteCheckedException {
+        while (true) {
+            curRec = advanceRecord(currWalSegment);
+
+            if (curRec != null)
+                return;
+            else {
+                currWalSegment = advanceSegment(currWalSegment);
+
+                if (currWalSegment == null)
+                    return;
+            }
+        }
+    }
+
+    /**
+     * Closes and returns WAL segment (if any)
+     * @return closed handle
+     * @throws IgniteCheckedException if IO failed
+     */
+    @Nullable protected FileWriteAheadLogManager.ReadFileHandle closeCurrentWalSegment() throws IgniteCheckedException {
+        final FileWriteAheadLogManager.ReadFileHandle walSegmentClosed = currWalSegment;
+
+        if (walSegmentClosed != null) {
+            walSegmentClosed.close();
+            currWalSegment = null;
+        }
+        return walSegmentClosed;
+    }
+
+    /**
+     * Switches records iterator to the next WAL segment
+     * as result of this method, new reference to segment should be returned.
+     * Null for current handle means stop of iteration
+     * @throws IgniteCheckedException if reading failed
+     * @param curWalSegment current open WAL segment or null if there is no open segment yet
+     * @return new WAL segment to read or null for stop iteration
+     */
+    protected abstract FileWriteAheadLogManager.ReadFileHandle advanceSegment(
+        @Nullable final FileWriteAheadLogManager.ReadFileHandle curWalSegment) throws IgniteCheckedException;
+
+    /**
+     * Switches to new record
+     * @param hnd currently opened read handle
+     * @return next advanced record
+     */
+    private IgniteBiTuple<WALPointer, WALRecord> advanceRecord(
+        @Nullable final FileWriteAheadLogManager.ReadFileHandle hnd) {
+        if (hnd == null)
+            return null;
+
+        final FileWALPointer ptr = new FileWALPointer(
+            hnd.idx,
+            (int)hnd.in.position(),
+            0);
+
+        try {
+            final WALRecord rec = hnd.ser.readRecord(hnd.in, ptr);
+
+            ptr.length(rec.size());
+
+            // cast using diamond operator here can break compile for 7
+            return new IgniteBiTuple<>((WALPointer)ptr, rec);
+        }
+        catch (IOException | IgniteCheckedException e) {
+            if (!(e instanceof SegmentEofException))
+                handleRecordException(e, ptr);
+            return null;
+        }
+    }
+
+    /**
+     * Handler for record deserialization exception
+     * @param e problem from records reading
+     * @param ptr file pointer was accessed
+     */
+    protected void handleRecordException(
+        @NotNull final Exception e,
+        @Nullable final FileWALPointer ptr) {
+        if (log.isInfoEnabled())
+            log.info("Stopping WAL iteration due to an exception: " + e.getMessage());
+    }
+
+    /**
+     * @param desc File descriptor.
+     * @param start Optional start pointer. Null means read from the beginning
+     * @return Initialized file handle.
+     * @throws FileNotFoundException If segment file is missing.
+     * @throws IgniteCheckedException If initialized failed due to another unexpected error.
+     */
+    protected FileWriteAheadLogManager.ReadFileHandle initReadHandle(
+        @NotNull final FileWriteAheadLogManager.FileDescriptor desc,
+        @Nullable final FileWALPointer start)
+        throws IgniteCheckedException, FileNotFoundException {
+        try {
+            RandomAccessFile rf = new RandomAccessFile(desc.file, "r");
+
+            try {
+                FileChannel ch = rf.getChannel();
+                FileInput in = new FileInput(ch, buf);
+
+                // Header record must be agnostic to the serializer version.
+                WALRecord rec = serializer.readRecord(in,
+                    new FileWALPointer(desc.idx, (int)ch.position(), 0));
+
+                if (rec == null)
+                    return null;
+
+                if (rec.type() != WALRecord.RecordType.HEADER_RECORD)
+                    throw new IOException("Missing file header record: " + desc.file.getAbsoluteFile());
+
+                int ver = ((HeaderRecord)rec).version();
+
+                RecordSerializer ser = FileWriteAheadLogManager.forVersion(sharedCtx, ver);
+
+                if (start != null && desc.idx == start.index())
+                    in.seek(start.fileOffset());
+
+                return new FileWriteAheadLogManager.ReadFileHandle(rf, desc.idx, sharedCtx.igniteInstanceName(), ser, in);
+            }
+            catch (SegmentEofException | EOFException ignore) {
+                try {
+                    rf.close();
+                }
+                catch (IOException ce) {
+                    throw new IgniteCheckedException(ce);
+                }
+
+                return null;
+            }
+            catch (IOException | IgniteCheckedException e) {
+                try {
+                    rf.close();
+                }
+                catch (IOException ce) {
+                    e.addSuppressed(ce);
+                }
+
+                throw e;
+            }
+        }
+        catch (FileNotFoundException e) {
+            throw e;
+        }
+        catch (IOException e) {
+            throw new IgniteCheckedException(
+                "Failed to initialize WAL segment: " + desc.file.getAbsolutePath(), e);
+        }
+    }
+
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/44f3fac2/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/wal/FileInput.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/wal/FileInput.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/wal/FileInput.java
index be1e477..e2d7cba 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/wal/FileInput.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/wal/FileInput.java
@@ -26,21 +26,25 @@ import org.apache.ignite.internal.processors.cache.persistence.wal.crc.PureJavaC
 import org.jetbrains.annotations.NotNull;
 
 /**
- * File input.
+ * File input, backed by byte buffer file input.
+ * This class allows to read data by chunks from file and then read primitives
  */
 public final class FileInput implements ByteBufferBackedDataInput {
-    /** */
+    /**
+     * Buffer for reading blocks of data into.
+     * <b>Note:</b> biggest block requested from this input can't be longer than buffer capacity
+     */
     private ByteBuffer buf;
 
-    /** */
+    /** File channel to read chunks from */
     private FileChannel ch;
 
     /** */
     private long pos;
 
     /**
-     * @param ch  Channel.
-     * @param buf Buffer.
+     * @param ch  Channel to read from
+     * @param buf Buffer for reading blocks of data into
      */
     public FileInput(FileChannel ch, ByteBuffer buf) throws IOException {
         assert ch != null;
@@ -101,7 +105,7 @@ public final class FileInput implements ByteBufferBackedDataInput {
             int read = ch.read(buf);
 
             if (read == -1)
-                throw new EOFException();
+                throw new EOFException("EOF at position [" + ch.position() + "] expected to read [" + requested + "] bytes");
 
             available += read;
 

http://git-wip-us.apache.org/repos/asf/ignite/blob/44f3fac2/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/wal/FileWALPointer.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/wal/FileWALPointer.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/wal/FileWALPointer.java
index b6ddfb8..3716de2 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/wal/FileWALPointer.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/wal/FileWALPointer.java
@@ -46,7 +46,7 @@ public class FileWALPointer implements WALPointer, Comparable<FileWALPointer> {
     }
 
     /**
-     * @param idx File timestamp index.
+     * @param idx Absolute WAL segment file index .
      * @param fileOffset Offset in file, from the beginning.
      * @param len Record length.
      * @param forceFlush Force flush flag.
@@ -59,7 +59,7 @@ public class FileWALPointer implements WALPointer, Comparable<FileWALPointer> {
     }
 
     /**
-     * @return Timestamp index.
+     * @return Absolute WAL segment file index .
      */
     public long index() {
         return idx;

http://git-wip-us.apache.org/repos/asf/ignite/blob/44f3fac2/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/wal/FileWriteAheadLogManager.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/wal/FileWriteAheadLogManager.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/wal/FileWriteAheadLogManager.java
index 5918141..f877a14 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/wal/FileWriteAheadLogManager.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/wal/FileWriteAheadLogManager.java
@@ -27,6 +27,7 @@ import java.nio.ByteBuffer;
 import java.nio.ByteOrder;
 import java.nio.channels.FileChannel;
 import java.nio.file.Files;
+import java.sql.Time;
 import java.util.Arrays;
 import java.util.HashMap;
 import java.util.Map;
@@ -34,11 +35,11 @@ import java.util.NavigableMap;
 import java.util.TreeMap;
 import java.util.concurrent.TimeUnit;
 import java.util.concurrent.atomic.AtomicBoolean;
+import java.util.concurrent.atomic.AtomicLong;
 import java.util.concurrent.atomic.AtomicReference;
 import java.util.concurrent.atomic.AtomicReferenceFieldUpdater;
 import java.util.concurrent.locks.Condition;
 import java.util.concurrent.locks.Lock;
-import java.util.concurrent.locks.LockSupport;
 import java.util.concurrent.locks.ReentrantLock;
 import java.util.regex.Pattern;
 import org.apache.ignite.IgniteCheckedException;
@@ -46,8 +47,10 @@ import org.apache.ignite.IgniteLogger;
 import org.apache.ignite.configuration.IgniteConfiguration;
 import org.apache.ignite.configuration.PersistentStoreConfiguration;
 import org.apache.ignite.configuration.WALMode;
+import org.apache.ignite.events.EventType;
 import org.apache.ignite.internal.GridKernalContext;
 import org.apache.ignite.internal.IgniteInterruptedCheckedException;
+import org.apache.ignite.internal.managers.eventstorage.GridEventStorageManager;
 import org.apache.ignite.internal.pagemem.wal.IgniteWriteAheadLogManager;
 import org.apache.ignite.internal.pagemem.wal.StorageException;
 import org.apache.ignite.internal.pagemem.wal.WALIterator;
@@ -58,9 +61,11 @@ import org.apache.ignite.internal.processors.cache.GridCacheSharedContext;
 import org.apache.ignite.internal.processors.cache.GridCacheSharedManagerAdapter;
 import org.apache.ignite.internal.processors.cache.persistence.GridCacheDatabaseSharedManager;
 import org.apache.ignite.internal.processors.cache.persistence.PersistenceMetricsImpl;
+import org.apache.ignite.events.WalSegmentArchivedEvent;
 import org.apache.ignite.internal.processors.cache.persistence.wal.record.HeaderRecord;
 import org.apache.ignite.internal.processors.cache.persistence.wal.serializer.RecordV1Serializer;
-import org.apache.ignite.internal.util.GridCloseableIteratorAdapter;
+import org.apache.ignite.internal.processors.timeout.GridTimeoutObject;
+import org.apache.ignite.internal.processors.timeout.GridTimeoutProcessor;
 import org.apache.ignite.internal.util.GridUnsafe;
 import org.apache.ignite.internal.util.typedef.F;
 import org.apache.ignite.internal.util.typedef.G;
@@ -69,6 +74,7 @@ import org.apache.ignite.internal.util.typedef.internal.SB;
 import org.apache.ignite.internal.util.typedef.internal.U;
 import org.apache.ignite.lang.IgniteBiTuple;
 import org.apache.ignite.lang.IgnitePredicate;
+import org.apache.ignite.lang.IgniteUuid;
 import org.jetbrains.annotations.NotNull;
 import org.jetbrains.annotations.Nullable;
 
@@ -85,14 +91,14 @@ public class FileWriteAheadLogManager extends GridCacheSharedManagerAdapter impl
     /** */
     private static final byte[] FILL_BUF = new byte[1024 * 1024];
 
-    /** */
+    /** Pattern for segment file names */
     private static final Pattern WAL_NAME_PATTERN = Pattern.compile("\\d{16}\\.wal");
 
     /** */
     private static final Pattern WAL_TEMP_NAME_PATTERN = Pattern.compile("\\d{16}\\.wal\\.tmp");
 
-    /** */
-    private static final FileFilter WAL_SEGMENT_FILE_FILTER = new FileFilter() {
+    /** WAL segment file filter, see {@link #WAL_NAME_PATTERN} */
+    public static final FileFilter WAL_SEGMENT_FILE_FILTER = new FileFilter() {
         @Override public boolean accept(File file) {
             return !file.isDirectory() && WAL_NAME_PATTERN.matcher(file.getName()).matches();
         }
@@ -118,7 +124,7 @@ public class FileWriteAheadLogManager extends GridCacheSharedManagerAdapter impl
     private final int tlbSize;
 
     /** WAL flush frequency. Makes sense only for {@link WALMode#BACKGROUND} log WALMode. */
-    public final int flushFreq;
+    private final int flushFreq;
 
     /** Fsync delay. */
     private final long fsyncDelay;
@@ -126,6 +132,9 @@ public class FileWriteAheadLogManager extends GridCacheSharedManagerAdapter impl
     /** */
     private final PersistentStoreConfiguration psCfg;
 
+    /** Events service */
+    private final GridEventStorageManager evt;
+
     /** */
     private IgniteConfiguration igCfg;
 
@@ -135,10 +144,10 @@ public class FileWriteAheadLogManager extends GridCacheSharedManagerAdapter impl
     /** */
     private File walWorkDir;
 
-    /** */
+    /** WAL archive directory (including consistent ID as subfolder) */
     private File walArchiveDir;
 
-    /** */
+    /** Serializer of current version, used to read header record and for write records */
     private RecordSerializer serializer;
 
     /** */
@@ -167,18 +176,41 @@ public class FileWriteAheadLogManager extends GridCacheSharedManagerAdapter impl
     private volatile FileArchiver archiver;
 
     /** */
-    private QueueFlusher flusher;
-
-    /** */
     private final ThreadLocal<WALPointer> lastWALPtr = new ThreadLocal<>();
 
     /** Current log segment handle */
     private volatile FileWriteHandle currentHnd;
 
     /**
+     * Positive (non-0) value indicates WAL can be archived even if not complete<br>
+     * See {@link PersistentStoreConfiguration#setWalAutoArchiveAfterInactivity(long)}<br>
+     */
+    private final long walAutoArchiveAfterInactivity;
+
+    /**
+     * Container with last WAL record logged timestamp.<br>
+     * Zero value means there was no records logged to current segment, skip possible archiving for this case<br>
+     * Value is filled only for case {@link #walAutoArchiveAfterInactivity} > 0<br>
+     */
+    private AtomicLong lastRecordLoggedMs = new AtomicLong();
+
+    /**
+     * Cancellable task for {@link WALMode#BACKGROUND}, should be cancelled at shutdown
+     * Null for non background modes
+     */
+    @Nullable private volatile GridTimeoutProcessor.CancelableTask backgroundFlushSchedule;
+
+    /**
+     * Reference to the last added next archive timeout check object.
+     * Null if mode is not enabled.
+     * Should be cancelled at shutdown
+     */
+    @Nullable private volatile GridTimeoutObject nextAutoArchiveTimeoutObj;
+
+    /**
      * @param ctx Kernal context.
      */
-    public FileWriteAheadLogManager(GridKernalContext ctx) {
+    public FileWriteAheadLogManager(@NotNull final GridKernalContext ctx) {
         igCfg = ctx.config();
 
         PersistentStoreConfiguration psCfg = igCfg.getPersistentStoreConfiguration();
@@ -193,6 +225,8 @@ public class FileWriteAheadLogManager extends GridCacheSharedManagerAdapter impl
         flushFreq = psCfg.getWalFlushFrequency();
         fsyncDelay = psCfg.getWalFsyncDelay();
         alwaysWriteFullPages = psCfg.isAlwaysWriteFullPages();
+        walAutoArchiveAfterInactivity = psCfg.getWalAutoArchiveAfterInactivity();
+        evt = ctx.event();
     }
 
     /** {@inheritDoc} */
@@ -248,8 +282,8 @@ public class FileWriteAheadLogManager extends GridCacheSharedManagerAdapter impl
         if (psCfg.getWalStorePath() == null ^ psCfg.getWalArchivePath() == null) {
             throw new IgniteCheckedException(
                 "Properties should be either both specified or both null " +
-                "[walStorePath = " + psCfg.getWalStorePath() +
-                ", walArchivePath = " + psCfg.getWalArchivePath() + "]"
+                    "[walStorePath = " + psCfg.getWalStorePath() +
+                    ", walArchivePath = " + psCfg.getWalArchivePath() + "]"
             );
         }
     }
@@ -271,26 +305,32 @@ public class FileWriteAheadLogManager extends GridCacheSharedManagerAdapter impl
 
     /** {@inheritDoc} */
     @Override protected void stop0(boolean cancel) {
-        FileWriteHandle currentHnd = currentHandle();
+        final GridTimeoutProcessor.CancelableTask schedule = backgroundFlushSchedule;
 
-        try {
-            QueueFlusher flusher0 = flusher;
+        if (schedule != null)
+            schedule.close();
 
-            if (flusher0 != null) {
-                flusher0.shutdown();
+        final GridTimeoutObject timeoutObj = nextAutoArchiveTimeoutObj;
 
-                if (currentHnd != null)
-                    currentHnd.flush((FileWALPointer)null);
+        if (timeoutObj != null)
+            cctx.time().removeTimeoutObject(timeoutObj);
+
+        final FileWriteHandle currHnd = currentHandle();
+
+        try {
+            if (mode == WALMode.BACKGROUND) {
+                if (currHnd != null)
+                    currHnd.flush((FileWALPointer)null);
             }
 
-            if (currentHnd != null)
-                currentHnd.close(false);
+            if (currHnd != null)
+                currHnd.close(false);
 
             if (archiver != null)
                 archiver.shutdown();
         }
         catch (Exception e) {
-            U.error(log, "Failed to gracefully close WAL segment: " + currentHnd.file, e);
+            U.error(log, "Failed to gracefully close WAL segment: " + currHnd.file, e);
         }
     }
 
@@ -350,39 +390,114 @@ public class FileWriteAheadLogManager extends GridCacheSharedManagerAdapter impl
             }
 
             if (mode == WALMode.BACKGROUND) {
-                flusher = new QueueFlusher(cctx.igniteInstanceName());
-
-                flusher.start();
+                backgroundFlushSchedule = cctx.time().schedule(new Runnable() {
+                    @Override public void run() {
+                        doFlush();
+                    }
+                }, flushFreq, flushFreq);
             }
+
+            if (walAutoArchiveAfterInactivity > 0)
+                scheduleNextInactivityPeriodElapsedCheck();
         }
         catch (StorageException e) {
             throw new IgniteCheckedException(e);
         }
     }
 
+    /**
+     * Schedules next check of inactivity period expired. Based on current record update timestamp.
+     * At timeout method does check of inactivity period and schedules new launch.
+     */
+    private void scheduleNextInactivityPeriodElapsedCheck() {
+        final long lastRecMs = lastRecordLoggedMs.get();
+        final long nextPossibleAutoArchive = (lastRecMs <= 0 ? U.currentTimeMillis() : lastRecMs) + walAutoArchiveAfterInactivity;
+
+        if (log.isDebugEnabled())
+            log.debug("Schedule WAL rollover check at " + new Time(nextPossibleAutoArchive).toString());
+
+        nextAutoArchiveTimeoutObj = new GridTimeoutObject() {
+            private final IgniteUuid id = IgniteUuid.randomUuid();
+
+            @Override public IgniteUuid timeoutId() {
+                return id;
+            }
+
+            @Override public long endTime() {
+                return nextPossibleAutoArchive;
+            }
+
+            @Override public void onTimeout() {
+                if (log.isDebugEnabled())
+                    log.debug("Checking if WAL rollover required (" + new Time(U.currentTimeMillis()).toString() + ")");
+
+                checkWalRolloverRequiredDuringInactivityPeriod();
+
+                scheduleNextInactivityPeriodElapsedCheck();
+            }
+        };
+        cctx.time().addTimeoutObject(nextAutoArchiveTimeoutObj);
+    }
+
+    /**
+     * Checks if there was elapsed significant period of inactivity.
+     * If WAL auto-archive is enabled using {@link #walAutoArchiveAfterInactivity} > 0 this method will activate
+     * roll over by timeout<br>
+     */
+    private void checkWalRolloverRequiredDuringInactivityPeriod() {
+        if (walAutoArchiveAfterInactivity <= 0)
+            return; // feature not configured, nothing to do
+
+        final long lastRecMs = lastRecordLoggedMs.get();
+
+        if (lastRecMs == 0)
+            return; //no records were logged to current segment, does not consider inactivity
+
+        final long elapsedMs = U.currentTimeMillis() - lastRecMs;
+
+        if (elapsedMs <= walAutoArchiveAfterInactivity)
+            return; // not enough time elapsed since last write
+
+        if (!lastRecordLoggedMs.compareAndSet(lastRecMs, 0))
+            return; // record write occurred concurrently
+
+        final FileWriteHandle handle = currentHandle();
+
+        try {
+            rollOver(handle);
+        }
+        catch (IgniteCheckedException e) {
+            U.error(log, "Unable to perform segment rollover: " + e.getMessage(), e);
+            handle.invalidateEnvironment(e);
+        }
+    }
+
     /** {@inheritDoc} */
     @SuppressWarnings("TooBroadScope")
     @Override public WALPointer log(WALRecord record) throws IgniteCheckedException, StorageException {
         if (serializer == null || mode == WALMode.NONE)
             return null;
 
-        FileWriteHandle current = currentHandle();
+        FileWriteHandle currWrHandle = currentHandle();
 
         // Logging was not resumed yet.
-        if (current == null)
+        if (currWrHandle == null)
             return null;
 
         // Need to calculate record size first.
         record.size(serializer.size(record));
 
-        for (; ; current = rollOver(current)) {
-            WALPointer ptr = current.addRecord(record);
+        for (; ; currWrHandle = rollOver(currWrHandle)) {
+            WALPointer ptr = currWrHandle.addRecord(record);
 
             if (ptr != null) {
                 metrics.onWalRecordLogged();
 
                 lastWALPtr.set(ptr);
 
+                if (walAutoArchiveAfterInactivity > 0)
+                    lastRecordLoggedMs.set(U.currentTimeMillis());
+
                 return ptr;
             }
 
@@ -665,6 +780,9 @@ public class FileWriteAheadLogManager extends GridCacheSharedManagerAdapter impl
 
             assert swapped : "Concurrent updates on rollover are not allowed";
 
+            if (walAutoArchiveAfterInactivity > 0)
+                lastRecordLoggedMs.set(0);
+
             // Let other threads to proceed with new segment.
             hnd.signalNextAvailable();
         }
@@ -888,7 +1006,7 @@ public class FileWriteAheadLogManager extends GridCacheSharedManagerAdapter impl
      * @param ver Serializer version.
      * @return Entry serializer.
      */
-    private static RecordSerializer forVersion(GridCacheSharedContext cctx, int ver) throws IgniteCheckedException {
+    static RecordSerializer forVersion(GridCacheSharedContext cctx, int ver) throws IgniteCheckedException {
         if (ver <= 0)
             throw new IgniteCheckedException("Failed to create a serializer (corrupted WAL file).");
 
@@ -905,7 +1023,7 @@ public class FileWriteAheadLogManager extends GridCacheSharedManagerAdapter impl
     /**
      * @return Sorted WAL files descriptors.
      */
-    private static FileDescriptor[] scan(File[] allFiles) {
+    public static FileDescriptor[] scan(File[] allFiles) {
         if (allFiles == null)
             return EMPTY_DESCRIPTORS;
 
@@ -931,11 +1049,11 @@ public class FileWriteAheadLogManager extends GridCacheSharedManagerAdapter impl
      *
      * Monitor of current object is used for notify on:
      * <ul>
-     *     <li>exception occurred ({@link FileArchiver#cleanException}!=null)</li>
-     *     <li>stopping thread ({@link FileArchiver#stopped}==true)</li>
-     *     <li>current file index changed ({@link FileArchiver#curAbsWalIdx})</li>
-     *     <li>last archived file index was changed ({@link FileArchiver#lastAbsArchivedIdx})</li>
-     *     <li>some WAL index was removed from {@link FileArchiver#locked} map</li>
+     * <li>exception occurred ({@link FileArchiver#cleanException}!=null)</li>
+     * <li>stopping thread ({@link FileArchiver#stopped}==true)</li>
+     * <li>current file index changed ({@link FileArchiver#curAbsWalIdx})</li>
+     * <li>last archived file index was changed ({@link FileArchiver#lastAbsArchivedIdx})</li>
+     * <li>some WAL index was removed from {@link FileArchiver#locked} map</li>
      * </ul>
      */
     private class FileArchiver extends Thread {
@@ -1017,6 +1135,7 @@ public class FileWriteAheadLogManager extends GridCacheSharedManagerAdapter impl
 
         /**
          * Check if WAL segment locked or reserved
+         *
          * @param absIdx Index for check reservation.
          * @return {@code True} if index is reserved.
          */
@@ -1080,7 +1199,7 @@ public class FileWriteAheadLogManager extends GridCacheSharedManagerAdapter impl
                         break;
 
                     try {
-                        File workFile = archiveSegment(toArchive);
+                        final SegmentArchiveResult res = archiveSegment(toArchive);
 
                         synchronized (this) {
                             while (locked.containsKey(toArchive) && !stopped)
@@ -1088,13 +1207,16 @@ public class FileWriteAheadLogManager extends GridCacheSharedManagerAdapter impl
 
                             // Firstly, format working file
                             if (!stopped)
-                                formatFile(workFile);
+                                formatFile(res.getOrigWorkFile());
 
                             // Then increase counter to allow rollover on clean working file
                             lastAbsArchivedIdx = toArchive;
 
                             notifyAll();
                         }
+                        if (evt.isRecordable(EventType.EVT_WAL_SEGMENT_ARCHIVED))
+                            evt.record(new WalSegmentArchivedEvent(cctx.discovery().localNode(),
+                                res.getAbsIdx(), res.getDstArchiveFile()));
                     }
                     catch (IgniteCheckedException e) {
                         synchronized (this) {
@@ -1115,7 +1237,7 @@ public class FileWriteAheadLogManager extends GridCacheSharedManagerAdapter impl
          * Blocks till there are available file to write
          *
          * @param curIdx Current absolute index that we want to increment.
-         * @return Next index (curIdx+1) when it is ready to be written.
+         * @return Next index (curWalSegmIdx+1) when it is ready to be written.
          * @throws IgniteCheckedException If failed (if interrupted or if exception occurred in the archiver thread).
          */
         private long nextAbsoluteSegmentIndex(long curIdx) throws IgniteCheckedException {
@@ -1195,9 +1317,12 @@ public class FileWriteAheadLogManager extends GridCacheSharedManagerAdapter impl
         }
 
         /**
+         * Moves WAL segment from work folder to archive folder.
+         * Temp file is used to do movement
+         *
          * @param absIdx Absolute index to archive.
          */
-        private File archiveSegment(long absIdx) throws IgniteCheckedException {
+        private SegmentArchiveResult archiveSegment(long absIdx) throws IgniteCheckedException {
             long segIdx = absIdx % psCfg.getWalSegments();
 
             File origFile = new File(walWorkDir, FileDescriptor.fileName(segIdx));
@@ -1235,7 +1360,7 @@ public class FileWriteAheadLogManager extends GridCacheSharedManagerAdapter impl
                 log.debug("Copied file [src=" + origFile.getAbsolutePath() +
                     ", dst=" + dstFile.getAbsolutePath() + ']');
 
-            return origFile;
+            return new SegmentArchiveResult(absIdx, origFile, dstFile);
         }
 
         /**
@@ -1316,7 +1441,7 @@ public class FileWriteAheadLogManager extends GridCacheSharedManagerAdapter impl
     /**
      * WAL file descriptor.
      */
-    private static class FileDescriptor implements Comparable<FileDescriptor> {
+    public static class FileDescriptor implements Comparable<FileDescriptor> {
         /** */
         protected final File file;
 
@@ -1324,9 +1449,11 @@ public class FileWriteAheadLogManager extends GridCacheSharedManagerAdapter impl
         protected final long idx;
 
         /**
-         * @param file File.
+         * Creates file descriptor. Index is restored from file name
+         *
+         * @param file WAL segment file.
          */
-        private FileDescriptor(File file) {
+        public FileDescriptor(@NotNull File file) {
             this(file, null);
         }
 
@@ -1334,7 +1461,7 @@ public class FileWriteAheadLogManager extends GridCacheSharedManagerAdapter impl
          * @param file WAL segment file.
          * @param idx Absolute WAL segment file index. For null value index is restored from file name
          */
-        private FileDescriptor(@NotNull File file, @Nullable Long idx) {
+        public FileDescriptor(@NotNull File file, @Nullable Long idx) {
             this.file = file;
 
             String fileName = file.getName();
@@ -1350,7 +1477,7 @@ public class FileWriteAheadLogManager extends GridCacheSharedManagerAdapter impl
          * @param segment Segment index.
          * @return Segment file name.
          */
-        private static String fileName(long segment) {
+        public static String fileName(long segment) {
             SB b = new SB();
 
             String segmentStr = Long.toString(segment);
@@ -1402,6 +1529,20 @@ public class FileWriteAheadLogManager extends GridCacheSharedManagerAdapter impl
         @Override public int hashCode() {
             return (int)(idx ^ (idx >>> 32));
         }
+
+        /**
+         * @return Absolute WAL segment file index
+         */
+        public long getIdx() {
+            return idx;
+        }
+
+        /**
+         * @return absolute pathname string of this file descriptor pathname.
+         */
+        public String getAbsolutePath() {
+            return file.getAbsolutePath();
+        }
     }
 
     /**
@@ -1438,14 +1579,17 @@ public class FileWriteAheadLogManager extends GridCacheSharedManagerAdapter impl
     /**
      *
      */
-    private static class ReadFileHandle extends FileHandle {
+    public static class ReadFileHandle extends FileHandle {
         /** Entry serializer. */
-        private RecordSerializer ser;
+        RecordSerializer ser;
 
         /** */
-        private FileInput in;
+        FileInput in;
 
-        /** */
+        /**
+         * <code>true</code> if this file handle came from work directory.
+         * <code>false</code> if this file handle came from archive directory.
+         */
         private boolean workDir;
 
         /**
@@ -1454,7 +1598,7 @@ public class FileWriteAheadLogManager extends GridCacheSharedManagerAdapter impl
          * @param ser Entry serializer.
          * @param in File input.
          */
-        private ReadFileHandle(
+        ReadFileHandle(
             RandomAccessFile file,
             long idx,
             String gridName,
@@ -1499,7 +1643,10 @@ public class FileWriteAheadLogManager extends GridCacheSharedManagerAdapter impl
          */
         private final AtomicReference<WALRecord> head = new AtomicReference<>();
 
-        /** Position in current file after the end of last written record (incremented after file channel write operation) */
+        /**
+         * Position in current file after the end of last written record (incremented after file channel write
+         * operation)
+         */
         private volatile long written;
 
         /** */
@@ -1508,7 +1655,7 @@ public class FileWriteAheadLogManager extends GridCacheSharedManagerAdapter impl
         /** Environment failure. */
         private volatile Throwable envFailed;
 
-        /** Stop guard to provide warranty that only one thread will be successful in calling {@link #close(boolean)}*/
+        /** Stop guard to provide warranty that only one thread will be successful in calling {@link #close(boolean)} */
         private final AtomicBoolean stop = new AtomicBoolean(false);
 
         /** */
@@ -1754,6 +1901,7 @@ public class FileWriteAheadLogManager extends GridCacheSharedManagerAdapter impl
 
         /**
          * Serializes WAL records chain to provided byte buffer
+         *
          * @param buf Buffer, will be filled with records chain from end to beginning
          * @param head Head of the chain to write to the buffer.
          * @return Position in file for this buffer.
@@ -1886,11 +2034,15 @@ public class FileWriteAheadLogManager extends GridCacheSharedManagerAdapter impl
                     flushOrWait(null);
 
                 try {
-                    if (rollOver && written < (maxSegmentSize - 1)) {
-                        ByteBuffer allocate = ByteBuffer.allocate(1);
-                        allocate.put((byte) WALRecord.RecordType.SWITCH_SEGMENT_RECORD.ordinal());
-
-                        ch.write(allocate, written);
+                    int switchSegmentRecSize = RecordV1Serializer.REC_TYPE_SIZE + RecordV1Serializer.FILE_WAL_POINTER_SIZE;
+                    if (rollOver && written < (maxSegmentSize - switchSegmentRecSize)) {
+                        //it is expected there is sufficient space for this record because rollover should run early
+                        final ByteBuffer buf = ByteBuffer.allocate(switchSegmentRecSize);
+                        buf.put((byte)(WALRecord.RecordType.SWITCH_SEGMENT_RECORD.ordinal() + 1));
+                        final FileWALPointer pointer = new FileWALPointer(idx, (int)ch.position(), -1);
+                        RecordV1Serializer.putPosition(buf, pointer);
+                        buf.rewind();
+                        ch.write(buf, written);
 
                         if (mode == WALMode.DEFAULT)
                             ch.force(false);
@@ -1951,8 +2103,8 @@ public class FileWriteAheadLogManager extends GridCacheSharedManagerAdapter impl
         }
 
         /**
-         * @param pos Position in file to start write from.
-         * May be checked against actual position to wait previous writes to complete
+         * @param pos Position in file to start write from. May be checked against actual position to wait previous
+         * writes to complete
          * @param buf Buffer to write to file
          * @throws StorageException If failed.
          * @throws IgniteCheckedException If failed.
@@ -2133,8 +2285,7 @@ public class FileWriteAheadLogManager extends GridCacheSharedManagerAdapter impl
     /**
      * Iterator over WAL-log.
      */
-    private static class RecordsIterator extends GridCloseableIteratorAdapter<IgniteBiTuple<WALPointer, WALRecord>>
-        implements WALIterator {
+    private static class RecordsIterator extends AbstractWalRecordsIterator {
         /** */
         private static final long serialVersionUID = 0L;
         /** */
@@ -2149,33 +2300,14 @@ public class FileWriteAheadLogManager extends GridCacheSharedManagerAdapter impl
         /** */
         private final PersistentStoreConfiguration psCfg;
 
-        /** */
-        private final RecordSerializer serializer;
-
-        /** */
-        private final GridCacheSharedContext cctx;
-
-        /** */
+        /** Optional start pointer. */
+        @Nullable
         private FileWALPointer start;
 
-        /** */
+        /** Optional end pointer. */
+        @Nullable
         private FileWALPointer end;
 
-        /** */
-        private IgniteBiTuple<WALPointer, WALRecord> curRec;
-
-        /** */
-        private long curIdx = -1;
-
-        /** */
-        private ReadFileHandle curHandle;
-
-        /** */
-        private ByteBuffer buf;
-
-        /** */
-        private IgniteLogger log;
-
         /**
          * @param cctx Shared context.
          * @param walWorkDir WAL work dir.
@@ -2183,37 +2315,33 @@ public class FileWriteAheadLogManager extends GridCacheSharedManagerAdapter impl
          * @param start Optional start pointer.
          * @param end Optional end pointer.
          * @param psCfg Database configuration.
-         * @param serializer Serializer.
+         * @param serializer Serializer of current version to read headers.
          * @param archiver Archiver.
+         * @param log Logger
          * @throws IgniteCheckedException If failed to initialize WAL segment.
          */
         private RecordsIterator(
             GridCacheSharedContext cctx,
             File walWorkDir,
             File walArchiveDir,
-            FileWALPointer start,
-            FileWALPointer end,
+            @Nullable FileWALPointer start,
+            @Nullable FileWALPointer end,
             PersistentStoreConfiguration psCfg,
-            RecordSerializer serializer,
+            @NotNull RecordSerializer serializer,
             FileArchiver archiver,
             IgniteLogger log,
             int tlbSize
         ) throws IgniteCheckedException {
-            this.cctx = cctx;
+            super(log,
+                cctx,
+                serializer,
+                Math.min(16 * tlbSize, psCfg.getWalRecordIteratorBufferSize()));
             this.walWorkDir = walWorkDir;
             this.walArchiveDir = walArchiveDir;
             this.psCfg = psCfg;
-            this.serializer = serializer;
             this.archiver = archiver;
             this.start = start;
             this.end = end;
-            this.log = log;
-
-            int buffSize = Math.min(16 * tlbSize, psCfg.getWalRecordIteratorBufferSize());
-
-            // Do not allocate direct buffer for iterator.
-            buf = ByteBuffer.allocate(buffSize);
-            buf.order(ByteOrder.nativeOrder());
 
             init();
 
@@ -2221,40 +2349,21 @@ public class FileWriteAheadLogManager extends GridCacheSharedManagerAdapter impl
         }
 
         /** {@inheritDoc} */
-        @Override protected IgniteBiTuple<WALPointer, WALRecord> onNext() throws IgniteCheckedException {
-            IgniteBiTuple<WALPointer, WALRecord> ret = curRec;
-
-            advance();
-
-            return ret;
-        }
-
-        /** {@inheritDoc} */
-        @Override protected boolean onHasNext() throws IgniteCheckedException {
-            return curRec != null;
-        }
-
-        /** {@inheritDoc} */
         @Override protected void onClose() throws IgniteCheckedException {
             curRec = null;
 
-            if (curHandle != null) {
-                curHandle.close();
+            final ReadFileHandle handle = closeCurrentWalSegment();
+            if (handle != null && handle.workDir)
+                releaseWorkSegment(curWalSegmIdx);
 
-                if (curHandle.workDir)
-                    releaseWorkSegment(curIdx);
-
-                curHandle = null;
-            }
-
-            curIdx = Integer.MAX_VALUE;
+            curWalSegmIdx = Integer.MAX_VALUE;
         }
 
         /**
          * @throws IgniteCheckedException If failed to initialize first file handle.
          */
         private void init() throws IgniteCheckedException {
-            FileDescriptor[] descs = scan(walArchiveDir.listFiles(WAL_SEGMENT_FILE_FILTER));
+            FileDescriptor[] descs = loadFileDescriptors(walArchiveDir);
 
             if (start != null) {
                 if (!F.isEmpty(descs)) {
@@ -2264,13 +2373,13 @@ public class FileWriteAheadLogManager extends GridCacheSharedManagerAdapter impl
 
                     for (FileDescriptor desc : descs) {
                         if (desc.idx == start.index()) {
-                            curIdx = start.index();
+                            curWalSegmIdx = start.index();
 
                             break;
                         }
                     }
 
-                    if (curIdx == -1) {
+                    if (curWalSegmIdx == -1) {
                         long lastArchived = descs[descs.length - 1].idx;
 
                         if (lastArchived > start.index())
@@ -2278,203 +2387,86 @@ public class FileWriteAheadLogManager extends GridCacheSharedManagerAdapter impl
 
                         // This pointer may be in work files because archiver did not
                         // copy the file yet, check that it is not too far forward.
-                        curIdx = start.index();
+                        curWalSegmIdx = start.index();
                     }
                 }
                 else {
                     // This means that whole checkpoint history fits in one segment in WAL work directory.
                     // Will start from this index right away.
-                    curIdx = start.index();
+                    curWalSegmIdx = start.index();
                 }
             }
             else
-                curIdx = !F.isEmpty(descs) ? descs[0].idx : 0;
+                curWalSegmIdx = !F.isEmpty(descs) ? descs[0].idx : 0;
 
-            curIdx--;
+            curWalSegmIdx--;
 
             if (log.isDebugEnabled())
-                log.debug("Initialized WAL cursor [start=" + start + ", end=" + end + ", curIdx=" + curIdx + ']');
-        }
-
-        /**
-         * @throws IgniteCheckedException If failed.
-         */
-        private void advance() throws IgniteCheckedException {
-            while (true) {
-                advanceRecord();
-
-                if (curRec != null)
-                    return;
-                else {
-                    advanceSegment();
-
-                    if (curHandle == null)
-                        return;
-                }
-            }
-        }
-
-        /**
-         *
-         */
-        private void advanceRecord() {
-            try {
-                ReadFileHandle hnd = curHandle;
-
-                if (hnd != null) {
-                    RecordSerializer ser = hnd.ser;
-
-                    int pos = (int)hnd.in.position();
-
-                    FileWALPointer ptr = new FileWALPointer(hnd.idx, pos, 0);
-
-                    WALRecord rec = ser.readRecord(hnd.in, ptr);
-
-                    ptr.length(rec.size());
-
-                    curRec = new IgniteBiTuple<WALPointer, WALRecord>(ptr, rec);
-                }
-            }
-            catch (IOException | IgniteCheckedException e) {
-                if (!(e instanceof SegmentEofException)) {
-                    if (log.isInfoEnabled())
-                        log.info("Stopping WAL iteration due to an exception: " + e.getMessage());
-                }
-
-                curRec = null;
-            }
+                log.debug("Initialized WAL cursor [start=" + start + ", end=" + end + ", curWalSegmIdx=" + curWalSegmIdx + ']');
         }
 
-        /**
-         * @throws IgniteCheckedException If failed.
-         */
-        private void advanceSegment() throws IgniteCheckedException {
-            ReadFileHandle cur0 = curHandle;
-
-            if (cur0 != null) {
-                cur0.close();
+        /** {@inheritDoc} */
+        @Override protected ReadFileHandle advanceSegment(
+            @Nullable final ReadFileHandle curWalSegment) throws IgniteCheckedException {
+            if (curWalSegment != null) {
+                curWalSegment.close();
 
-                if (cur0.workDir)
-                    releaseWorkSegment(cur0.idx);
+                if (curWalSegment.workDir)
+                    releaseWorkSegment(curWalSegment.idx);
 
-                curHandle = null;
             }
 
             // We are past the end marker.
-            if (end != null && curIdx + 1 > end.index())
-                return;
+            if (end != null && curWalSegmIdx + 1 > end.index())
+                return null; //stop iteration
 
-            curIdx++;
+            curWalSegmIdx++;
 
             FileDescriptor fd;
 
-            boolean readArchive = canReadArchiveOrReserveWork(curIdx);
+            boolean readArchive = canReadArchiveOrReserveWork(curWalSegmIdx);
 
             if (readArchive) {
                 fd = new FileDescriptor(new File(walArchiveDir,
-                    FileDescriptor.fileName(curIdx)));
+                    FileDescriptor.fileName(curWalSegmIdx)));
             }
             else {
-                long workIdx = curIdx % psCfg.getWalSegments();
+                long workIdx = curWalSegmIdx % psCfg.getWalSegments();
 
                 fd = new FileDescriptor(
                     new File(walWorkDir, FileDescriptor.fileName(workIdx)),
-                    curIdx);
+                    curWalSegmIdx);
             }
 
             if (log.isDebugEnabled())
-                log.debug("Reading next file [absIdx=" + curIdx + ", file=" + fd.file.getAbsolutePath() + ']');
+                log.debug("Reading next file [absIdx=" + curWalSegmIdx + ", file=" + fd.file.getAbsolutePath() + ']');
 
             assert fd != null;
 
+            ReadFileHandle nextHandle;
             try {
-                curHandle = initReadHandle(fd, start != null && curIdx == start.index() ? start : null);
+                nextHandle = initReadHandle(fd, start != null && curWalSegmIdx == start.index() ? start : null);
             }
             catch (FileNotFoundException e) {
                 if (readArchive)
                     throw new IgniteCheckedException("Missing WAL segment in the archive", e);
                 else
-                    curHandle = null;
+                    nextHandle = null;
             }
 
-            if (curHandle != null)
-                curHandle.workDir = !readArchive;
+            if (nextHandle != null)
+                nextHandle.workDir = !readArchive;
             else
-                releaseWorkSegment(curIdx);
+                releaseWorkSegment(curWalSegmIdx);
 
             curRec = null;
-        }
-
-        /**
-         * @param desc File descriptor.
-         * @param start Optional start pointer.
-         * @return Initialized file handle.
-         * @throws FileNotFoundException If segment file is missing.
-         * @throws IgniteCheckedException If initialized failed due to another unexpected error.
-         */
-        private ReadFileHandle initReadHandle(FileDescriptor desc, FileWALPointer start)
-            throws IgniteCheckedException, FileNotFoundException {
-            try {
-                RandomAccessFile rf = new RandomAccessFile(desc.file, "r");
-
-                try {
-                    FileChannel channel = rf.getChannel();
-                    FileInput in = new FileInput(channel, buf);
-
-                    // Header record must be agnostic to the serializer version.
-                    WALRecord rec = serializer.readRecord(in,
-                        new FileWALPointer(desc.idx, (int)channel.position(), 0));
-
-                    if (rec == null)
-                        return null;
-
-                    if (rec.type() != WALRecord.RecordType.HEADER_RECORD)
-                        throw new IOException("Missing file header record: " + desc.file.getAbsoluteFile());
-
-                    int ver = ((HeaderRecord)rec).version();
-
-                    RecordSerializer ser = forVersion(cctx, ver);
-
-                    if (start != null && desc.idx == start.index())
-                        in.seek(start.fileOffset());
-
-                    return new ReadFileHandle(rf, desc.idx, cctx.igniteInstanceName(), ser, in);
-                }
-                catch (SegmentEofException | EOFException ignore) {
-                    try {
-                        rf.close();
-                    }
-                    catch (IOException ce) {
-                        throw new IgniteCheckedException(ce);
-                    }
-
-                    return null;
-                }
-                catch (IOException | IgniteCheckedException e) {
-                    try {
-                        rf.close();
-                    }
-                    catch (IOException ce) {
-                        e.addSuppressed(ce);
-                    }
-
-                    throw e;
-                }
-            }
-            catch (FileNotFoundException e) {
-                throw e;
-            }
-            catch (IOException e) {
-                throw new IgniteCheckedException(
-                    "Failed to initialize WAL segment: " + desc.file.getAbsolutePath(), e);
-            }
+            return nextHandle;
         }
 
         /**
          * @param absIdx Absolute index to check.
-         * @return {@code True} if we can safely read the archive, {@code false} if the segment has not been
-         *      archived yet. In this case the corresponding work segment is reserved (will not be deleted until
-         *      release).
+         * @return {@code True} if we can safely read the archive, {@code false} if the segment has not been archived
+         * yet. In this case the corresponding work segment is reserved (will not be deleted until release).
          */
         private boolean canReadArchiveOrReserveWork(long absIdx) {
             return archiver != null && archiver.checkCanReadArchiveOrReserveWorkSegment(absIdx);
@@ -2490,51 +2482,17 @@ public class FileWriteAheadLogManager extends GridCacheSharedManagerAdapter impl
     }
 
     /**
-     * Periodically flushes current file handle for {@link WALMode#BACKGROUND} WALMode.
+     * Flushes current file handle for {@link WALMode#BACKGROUND} WALMode.
+     * Called periodically from scheduler.
      */
-    private class QueueFlusher extends Thread {
-        /** */
-        private volatile boolean stopped;
+    private void doFlush() {
+        final FileWriteHandle hnd = currentHandle();
 
-        /**
-         * @param gridName Grid name.
-         */
-        private QueueFlusher(String gridName) {
-            super("wal-queue-flusher-#" + gridName);
-        }
-
-        /** {@inheritDoc} */
-        @Override public void run() {
-            while (!stopped) {
-                long wakeup = U.currentTimeMillis() + flushFreq;
-
-                LockSupport.parkUntil(wakeup);
-
-                FileWriteHandle hnd = currentHandle();
-
-                try {
-                    hnd.flush(hnd.head.get());
-                }
-                catch (IgniteCheckedException e) {
-                    U.warn(log, "Failed to flush WAL record queue", e);
-                }
-            }
+        try {
+            hnd.flush(hnd.head.get());
         }
-
-        /**
-         * Signals stop, wakes up thread and waiting until completion.
-         */
-        private void shutdown() {
-            stopped = true;
-
-            LockSupport.unpark(this);
-
-            try {
-                join();
-            }
-            catch (InterruptedException ignore) {
-                // Got interrupted while waiting for flusher to shutdown.
-            }
+        catch (IgniteCheckedException e) {
+            U.warn(log, "Failed to flush WAL record queue", e);
         }
     }
 }

http://git-wip-us.apache.org/repos/asf/ignite/blob/44f3fac2/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/wal/RecordSerializer.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/wal/RecordSerializer.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/wal/RecordSerializer.java
index 75a62a9..1ea7fa6 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/wal/RecordSerializer.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/wal/RecordSerializer.java
@@ -33,6 +33,8 @@ public interface RecordSerializer {
     public int version();
 
     /**
+     * Calculates record size in byte including expected wal pointer, CRC and type field
+     *
      * @param record Record.
      * @return Size in bytes.
      */
@@ -45,7 +47,10 @@ public interface RecordSerializer {
     public void writeRecord(WALRecord record, ByteBuffer buf) throws IgniteCheckedException;
 
     /**
+     * Loads record from input
+     *
      * @param in Data input to read data from.
+     * @param expPtr expected WAL pointer for record. Used to validate actual position against expected from the file
      * @return Read entry.
      */
     public WALRecord readRecord(FileInput in, WALPointer expPtr) throws IOException, IgniteCheckedException;

http://git-wip-us.apache.org/repos/asf/ignite/blob/44f3fac2/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/wal/SegmentArchiveResult.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/wal/SegmentArchiveResult.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/wal/SegmentArchiveResult.java
new file mode 100644
index 0000000..5b65970
--- /dev/null
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/wal/SegmentArchiveResult.java
@@ -0,0 +1,61 @@
+/*
+ * 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.persistence.wal;
+
+import java.io.File;
+
+/**
+ * Result of archiving (movement) operation
+ * Replacement of generic T3-Tuple
+ */
+class SegmentArchiveResult {
+    /** Absolute WAL segment file index. */
+    private final long absWalIdx;
+
+    /** Original work file. May and most likely to be used for new WAL round */
+    private final File origWorkFile;
+
+    /** Destination archive file. This file is completed and closed archive segment */
+    private final File dstArchiveFile;
+
+    /**
+     * Creates result
+     * @param absWalIdx Absolute wal index.
+     * @param origWorkFile Orig work file.
+     * @param dstArchiveFile Dst archive file.
+     */
+    SegmentArchiveResult(long absWalIdx, File origWorkFile, File dstArchiveFile) {
+        this.absWalIdx = absWalIdx;
+        this.origWorkFile = origWorkFile;
+        this.dstArchiveFile = dstArchiveFile;
+    }
+
+    /** @return {@link #absWalIdx} */
+    long getAbsIdx() {
+        return absWalIdx;
+    }
+
+    /** @return {@link #origWorkFile} */
+    File getOrigWorkFile() {
+        return origWorkFile;
+    }
+
+    /** @return {@link #dstArchiveFile} */
+    File getDstArchiveFile() {
+        return dstArchiveFile;
+    }
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/44f3fac2/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/wal/SegmentEofException.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/wal/SegmentEofException.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/wal/SegmentEofException.java
index 80c375e..2f58e3d 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/wal/SegmentEofException.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/wal/SegmentEofException.java
@@ -21,7 +21,8 @@ import org.apache.ignite.IgniteCheckedException;
 
 /**
  * This exception is thrown either when we reach the end of file of WAL segment, or when we encounter
- * a record with type equal to {@code 0}.
+ * a record with type equal to
+ * {@link org.apache.ignite.internal.pagemem.wal.record.WALRecord.RecordType#STOP_ITERATION_RECORD_TYPE}
  */
 public class SegmentEofException extends IgniteCheckedException {
     /** */

http://git-wip-us.apache.org/repos/asf/ignite/blob/44f3fac2/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/wal/reader/IgniteWalIteratorFactory.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/wal/reader/IgniteWalIteratorFactory.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/wal/reader/IgniteWalIteratorFactory.java
new file mode 100644
index 0000000..8ea0585
--- /dev/null
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/wal/reader/IgniteWalIteratorFactory.java
@@ -0,0 +1,102 @@
+/*
+ * 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.persistence.wal.reader;
+
+import java.io.File;
+import org.apache.ignite.IgniteCheckedException;
+import org.apache.ignite.IgniteLogger;
+import org.apache.ignite.configuration.MemoryConfiguration;
+import org.apache.ignite.internal.GridKernalContext;
+import org.apache.ignite.internal.pagemem.wal.WALIterator;
+import org.apache.ignite.internal.processors.cache.GridCacheSharedContext;
+import org.jetbrains.annotations.NotNull;
+
+/**
+ * Factory for creating iterator over WAL files
+ */
+public class IgniteWalIteratorFactory {
+    /** Logger. */
+    private final IgniteLogger log;
+    /** Page size, in standalone iterator mode this value can't be taken from memory configuration */
+    private final int pageSize;
+
+    /**
+     * Creates WAL files iterator factory
+     * @param log Logger.
+     * @param pageSize Page size, size is validated
+     */
+    public IgniteWalIteratorFactory(@NotNull final IgniteLogger log, final int pageSize) {
+        this.log = log;
+        this.pageSize = pageSize;
+        new MemoryConfiguration().setPageSize(pageSize); // just for validate
+    }
+
+    /**
+     * Creates iterator for (archive) directory scan mode.
+     * Note in this mode total scanned files at end of iteration may be wider that initial files in directory.
+     * This mode does not support work directory scan because work directory contains unpredictable number in file name.
+     * Such file may broke iteration.
+     *
+     * @param walDirWithConsistentId directory with WAL files. Should already contain node consistent ID as subfolder
+     * @return closable WAL records iterator, should be closed when non needed
+     * @throws IgniteCheckedException if failed to read folder
+     */
+    public WALIterator iteratorArchiveDirectory(@NotNull final File walDirWithConsistentId) throws IgniteCheckedException {
+        return new StandaloneWalRecordsIterator(walDirWithConsistentId, log, prepareSharedCtx());
+    }
+
+    /**
+     * Creates iterator for file by file scan mode.
+     * This method may be used only for archive folder (not for work).
+     * In this mode only provided WAL segments will be scanned. New WAL files created during iteration will be ignored
+     * @param files files to scan. Order it not important, but is significant to provide all segments without omissions
+     * @return closable WAL records iterator, should be closed when non needed
+     * @throws IgniteCheckedException if failed to read files
+     */
+    public WALIterator iteratorArchiveFiles(@NotNull final File ...files) throws IgniteCheckedException {
+        return new StandaloneWalRecordsIterator(log, prepareSharedCtx(), false, files);
+    }
+
+    /**
+     * Creates iterator for file by file scan mode.
+     * This method may be used for work folder, file indexes are scanned from the file context.
+     * In this mode only provided WAL segments will be scanned. New WAL files created during iteration will be ignored.
+     * @param files files to scan. Order it not important, but is significant to provide all segments without omissions
+     * @return closable WAL records iterator, should be closed when non needed
+     * @throws IgniteCheckedException if failed to read files
+     */
+    public WALIterator iteratorWorkFiles(@NotNull final File ...files) throws IgniteCheckedException {
+        return new StandaloneWalRecordsIterator(log, prepareSharedCtx(), true, files);
+    }
+
+    /**
+     * @return fake shared context required for create minimal services for record reading
+     */
+    @NotNull private GridCacheSharedContext prepareSharedCtx() {
+        final GridKernalContext kernalCtx = new StandaloneGridKernalContext(log);
+
+        final StandaloneIgniteCacheDatabaseSharedManager dbMgr = new StandaloneIgniteCacheDatabaseSharedManager();
+
+        dbMgr.setPageSize(pageSize);
+        return new GridCacheSharedContext<>(
+            kernalCtx, null, null, null,
+            null, null, dbMgr, null,
+            null, null, null, null,
+            null, null, null);
+    }
+}


[30/33] ignite git commit: Merge remote-tracking branch 'origin/ignite-2.1' into ignite-2.1

Posted by sb...@apache.org.
Merge remote-tracking branch 'origin/ignite-2.1' into ignite-2.1


Project: http://git-wip-us.apache.org/repos/asf/ignite/repo
Commit: http://git-wip-us.apache.org/repos/asf/ignite/commit/b67b8c44
Tree: http://git-wip-us.apache.org/repos/asf/ignite/tree/b67b8c44
Diff: http://git-wip-us.apache.org/repos/asf/ignite/diff/b67b8c44

Branch: refs/heads/ignite-2.1.2-exchange
Commit: b67b8c4438f240038b61ca65b947394106b0e43b
Parents: 1337901 44fad24
Author: sboikov <sb...@gridgain.com>
Authored: Wed Jul 5 11:20:54 2017 +0300
Committer: sboikov <sb...@gridgain.com>
Committed: Wed Jul 5 11:20:54 2017 +0300

----------------------------------------------------------------------
 .../apache/ignite/internal/jdbc/thin/JdbcThinConnection.java  | 7 ++++++-
 1 file changed, 6 insertions(+), 1 deletion(-)
----------------------------------------------------------------------



[05/33] ignite git commit: Reduced amount of debug logging.

Posted by sb...@apache.org.
Reduced amount of debug logging.


Project: http://git-wip-us.apache.org/repos/asf/ignite/repo
Commit: http://git-wip-us.apache.org/repos/asf/ignite/commit/211caf15
Tree: http://git-wip-us.apache.org/repos/asf/ignite/tree/211caf15
Diff: http://git-wip-us.apache.org/repos/asf/ignite/diff/211caf15

Branch: refs/heads/ignite-2.1.2-exchange
Commit: 211caf15fa4d2f01ac09c7de414272b8c0d8d908
Parents: ae5ec94
Author: sboikov <sb...@gridgain.com>
Authored: Tue Jul 4 11:21:49 2017 +0300
Committer: sboikov <sb...@gridgain.com>
Committed: Tue Jul 4 11:21:49 2017 +0300

----------------------------------------------------------------------
 .../processors/affinity/GridAffinityAssignmentCache.java     | 8 +++++++-
 .../processors/cache/GridCachePartitionExchangeManager.java  | 8 ++++++--
 2 files changed, 13 insertions(+), 3 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/211caf15/modules/core/src/main/java/org/apache/ignite/internal/processors/affinity/GridAffinityAssignmentCache.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/affinity/GridAffinityAssignmentCache.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/affinity/GridAffinityAssignmentCache.java
index 71ec3ea..a8c6c59 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/affinity/GridAffinityAssignmentCache.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/affinity/GridAffinityAssignmentCache.java
@@ -431,8 +431,10 @@ public class GridAffinityAssignmentCache {
 
     /**
      * Dumps debug information.
+     *
+     * @return {@code True} if there are pending futures.
      */
-    public void dumpDebugInfo() {
+    public boolean dumpDebugInfo() {
         if (!readyFuts.isEmpty()) {
             U.warn(log, "First 3 pending affinity ready futures [grp=" + cacheOrGrpName +
                 ", total=" + readyFuts.size() +
@@ -446,7 +448,11 @@ public class GridAffinityAssignmentCache {
                 if (++cnt == 3)
                     break;
             }
+
+            return true;
         }
+
+        return false;
     }
 
     /**

http://git-wip-us.apache.org/repos/asf/ignite/blob/211caf15/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCachePartitionExchangeManager.java
----------------------------------------------------------------------
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 2b5a4ff..f07119a 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
@@ -1636,6 +1636,8 @@ public class GridCachePartitionExchangeManager<K, V> extends GridCacheSharedMana
             }
         }
 
+        int affDumpCnt = 0;
+
         for (CacheGroupContext grp : cctx.cache().cacheGroups()) {
             if (grp.isLocal())
                 continue;
@@ -1647,8 +1649,10 @@ public class GridCachePartitionExchangeManager<K, V> extends GridCacheSharedMana
 
             GridAffinityAssignmentCache aff = grp.affinity();
 
-            if (aff != null)
-                aff.dumpDebugInfo();
+            if (aff != null && affDumpCnt < 5) {
+                if (aff.dumpDebugInfo())
+                    affDumpCnt++;
+            }
         }
     }
 


[20/33] ignite git commit: Muted hanging tests.

Posted by sb...@apache.org.
Muted hanging tests.


Project: http://git-wip-us.apache.org/repos/asf/ignite/repo
Commit: http://git-wip-us.apache.org/repos/asf/ignite/commit/50bb0901
Tree: http://git-wip-us.apache.org/repos/asf/ignite/tree/50bb0901
Diff: http://git-wip-us.apache.org/repos/asf/ignite/diff/50bb0901

Branch: refs/heads/ignite-2.1.2-exchange
Commit: 50bb0901255707dc9601159803363a36aeafecab
Parents: 90b67fa
Author: Ilya Lantukh <il...@gridgain.com>
Authored: Tue Jul 4 18:10:25 2017 +0300
Committer: Alexey Goncharuk <al...@gmail.com>
Committed: Tue Jul 4 20:53:09 2017 +0300

----------------------------------------------------------------------
 .../IgniteCommunicationBalanceMultipleConnectionsTest.java      | 5 +++++
 .../IgniteHadoopFileSystemShmemExternalDualAsyncSelfTest.java   | 5 +++++
 .../org/apache/ignite/spark/JavaEmbeddedIgniteRDDSelfTest.java  | 5 +++++
 3 files changed, 15 insertions(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/50bb0901/modules/core/src/test/java/org/apache/ignite/internal/managers/communication/IgniteCommunicationBalanceMultipleConnectionsTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/managers/communication/IgniteCommunicationBalanceMultipleConnectionsTest.java b/modules/core/src/test/java/org/apache/ignite/internal/managers/communication/IgniteCommunicationBalanceMultipleConnectionsTest.java
index e95b1ec..444f086 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/managers/communication/IgniteCommunicationBalanceMultipleConnectionsTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/managers/communication/IgniteCommunicationBalanceMultipleConnectionsTest.java
@@ -25,4 +25,9 @@ public class IgniteCommunicationBalanceMultipleConnectionsTest extends IgniteCom
     @Override protected int connectionsPerNode() {
         return 5;
     }
+
+    /** {@inheritDoc} */
+    @Override protected void beforeTest() throws Exception {
+        fail("https://issues.apache.org/jira/browse/IGNITE-5689");
+    }
 }

http://git-wip-us.apache.org/repos/asf/ignite/blob/50bb0901/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/impl/igfs/IgniteHadoopFileSystemShmemExternalDualAsyncSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/impl/igfs/IgniteHadoopFileSystemShmemExternalDualAsyncSelfTest.java b/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/impl/igfs/IgniteHadoopFileSystemShmemExternalDualAsyncSelfTest.java
index 5154642..edf0279 100644
--- a/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/impl/igfs/IgniteHadoopFileSystemShmemExternalDualAsyncSelfTest.java
+++ b/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/impl/igfs/IgniteHadoopFileSystemShmemExternalDualAsyncSelfTest.java
@@ -30,4 +30,9 @@ public class IgniteHadoopFileSystemShmemExternalDualAsyncSelfTest
     public IgniteHadoopFileSystemShmemExternalDualAsyncSelfTest() {
         super(DUAL_ASYNC, true);
     }
+
+    /** {@inheritDoc} */
+    @Override protected void beforeTest() throws Exception {
+        fail("https://issues.apache.org/jira/browse/IGNITE-5691");
+    }
 }
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/ignite/blob/50bb0901/modules/spark/src/test/java/org/apache/ignite/spark/JavaEmbeddedIgniteRDDSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/spark/src/test/java/org/apache/ignite/spark/JavaEmbeddedIgniteRDDSelfTest.java b/modules/spark/src/test/java/org/apache/ignite/spark/JavaEmbeddedIgniteRDDSelfTest.java
index 49bb1ac..5477d43 100644
--- a/modules/spark/src/test/java/org/apache/ignite/spark/JavaEmbeddedIgniteRDDSelfTest.java
+++ b/modules/spark/src/test/java/org/apache/ignite/spark/JavaEmbeddedIgniteRDDSelfTest.java
@@ -100,6 +100,11 @@ public class JavaEmbeddedIgniteRDDSelfTest extends GridCommonAbstractTest {
     }
 
     /** {@inheritDoc} */
+    @Override protected void beforeTest() throws Exception {
+        fail("https://issues.apache.org/jira/browse/IGNITE-5690");
+    }
+
+    /** {@inheritDoc} */
     @Override protected void afterTest() throws Exception {
         stopAllGrids();
     }


[02/33] ignite git commit: Remove some unused parameters.

Posted by sb...@apache.org.
Remove some unused parameters.


Project: http://git-wip-us.apache.org/repos/asf/ignite/repo
Commit: http://git-wip-us.apache.org/repos/asf/ignite/commit/3cc13eae
Tree: http://git-wip-us.apache.org/repos/asf/ignite/tree/3cc13eae
Diff: http://git-wip-us.apache.org/repos/asf/ignite/diff/3cc13eae

Branch: refs/heads/ignite-2.1.2-exchange
Commit: 3cc13eae99b2f4f86a9c679efd31a4e87a37d4fd
Parents: bdd31af
Author: sboikov <sb...@gridgain.com>
Authored: Mon Jul 3 16:03:37 2017 +0300
Committer: sboikov <sb...@gridgain.com>
Committed: Mon Jul 3 16:03:37 2017 +0300

----------------------------------------------------------------------
 .../cache/GridCachePartitionExchangeManager.java     | 15 ++++++---------
 .../preloader/GridDhtPartitionsExchangeFuture.java   | 11 ++++-------
 2 files changed, 10 insertions(+), 16 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/3cc13eae/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCachePartitionExchangeManager.java
----------------------------------------------------------------------
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 cbb07f9..2d1aca0 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
@@ -860,7 +860,7 @@ public class GridCachePartitionExchangeManager<K, V> extends GridCacheSharedMana
      * @param nodes Nodes.
      */
     private void sendAllPartitions(Collection<ClusterNode> nodes) {
-        GridDhtPartitionsFullMessage m = createPartitionsFullMessage(nodes, null, null, true);
+        GridDhtPartitionsFullMessage m = createPartitionsFullMessage(null, null);
 
         if (log.isDebugEnabled())
             log.debug("Sending all partitions [nodeIds=" + U.nodeIds(nodes) + ", msg=" + m + ']');
@@ -883,21 +883,18 @@ public class GridCachePartitionExchangeManager<K, V> extends GridCacheSharedMana
     }
 
     /**
-     * @param nodes Target nodes.
      * @param exchId Non-null exchange ID if message is created for exchange.
      * @param lastVer Last version.
-     * @param compress {@code True} if it is possible to use compression for message.
      * @return Message.
      */
-    public GridDhtPartitionsFullMessage createPartitionsFullMessage(Collection<ClusterNode> nodes,
+    public GridDhtPartitionsFullMessage createPartitionsFullMessage(
         final @Nullable GridDhtPartitionExchangeId exchId,
-        @Nullable GridCacheVersion lastVer,
-        final boolean compress) {
+        @Nullable GridCacheVersion lastVer) {
         final GridDhtPartitionsFullMessage m = new GridDhtPartitionsFullMessage(exchId,
             lastVer,
             exchId != null ? exchId.topologyVersion() : AffinityTopologyVersion.NONE);
 
-        m.compress(compress);
+        m.compress(true);
 
         final Map<Object, T2<Integer, GridDhtPartitionFullMap>> dupData = new HashMap<>();
 
@@ -917,7 +914,7 @@ public class GridCachePartitionExchangeManager<K, V> extends GridCacheSharedMana
                 if (locMap != null) {
                     addFullPartitionsMap(m,
                         dupData,
-                        compress,
+                        true,
                         grp.groupId(),
                         locMap,
                         affCache.similarAffinityKey());
@@ -935,7 +932,7 @@ public class GridCachePartitionExchangeManager<K, V> extends GridCacheSharedMana
             if (map != null) {
                 addFullPartitionsMap(m,
                     dupData,
-                    compress,
+                    true,
                     top.groupId(),
                     map,
                     top.similarAffinityKey());

http://git-wip-us.apache.org/repos/asf/ignite/blob/3cc13eae/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/preloader/GridDhtPartitionsExchangeFuture.java
----------------------------------------------------------------------
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 3dc2242..7471855 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
@@ -1083,17 +1083,14 @@ public class GridDhtPartitionsExchangeFuture extends GridDhtTopologyFutureAdapte
     }
 
     /**
-     * @param compress {@code True} if it is possible to use compression for message.
      * @return Message.
      */
-    private GridDhtPartitionsFullMessage createPartitionsMessage(Collection<ClusterNode> nodes, boolean compress) {
+    private GridDhtPartitionsFullMessage createPartitionsMessage() {
         GridCacheVersion last = lastVer.get();
 
         GridDhtPartitionsFullMessage m = cctx.exchange().createPartitionsFullMessage(
-            nodes,
             exchangeId(),
-            last != null ? last : cctx.versions().last(),
-            compress);
+            last != null ? last : cctx.versions().last());
 
         if (exchangeOnChangeGlobalState && !F.isEmpty(changeGlobalStateExceptions))
             m.setExceptionsMap(changeGlobalStateExceptions);
@@ -1106,7 +1103,7 @@ public class GridDhtPartitionsExchangeFuture extends GridDhtTopologyFutureAdapte
      * @throws IgniteCheckedException If failed.
      */
     private void sendAllPartitions(Collection<ClusterNode> nodes) throws IgniteCheckedException {
-        GridDhtPartitionsFullMessage m = createPartitionsMessage(nodes, true);
+        GridDhtPartitionsFullMessage m = createPartitionsMessage();
 
         assert !nodes.contains(cctx.localNode());
 
@@ -1390,7 +1387,7 @@ public class GridDhtPartitionsExchangeFuture extends GridDhtTopologyFutureAdapte
 
             Map<Integer, Map<Integer, List<UUID>>> assignmentChange = fut.get();
 
-            GridDhtPartitionsFullMessage m = createPartitionsMessage(null, false);
+            GridDhtPartitionsFullMessage m = createPartitionsMessage();
 
             CacheAffinityChangeMessage msg = new CacheAffinityChangeMessage(exchId, m, assignmentChange);
 


[09/33] ignite git commit: ignite-2.1 Added persistent store metrics.

Posted by sb...@apache.org.
ignite-2.1 Added persistent store metrics.


Project: http://git-wip-us.apache.org/repos/asf/ignite/repo
Commit: http://git-wip-us.apache.org/repos/asf/ignite/commit/6c9d2228
Tree: http://git-wip-us.apache.org/repos/asf/ignite/tree/6c9d2228
Diff: http://git-wip-us.apache.org/repos/asf/ignite/diff/6c9d2228

Branch: refs/heads/ignite-2.1.2-exchange
Commit: 6c9d2228d2aba6ba5be64bd20ef244e634aeaa5a
Parents: a5abdd5
Author: Alexey Kuznetsov <ak...@apache.org>
Authored: Tue Jul 4 19:30:46 2017 +0700
Committer: Alexey Kuznetsov <ak...@apache.org>
Committed: Tue Jul 4 19:30:46 2017 +0700

----------------------------------------------------------------------
 .../visor/node/VisorNodeDataCollectorJob.java   |  32 ++-
 .../node/VisorNodeDataCollectorJobResult.java   |  73 +++++--
 .../visor/node/VisorNodeDataCollectorTask.java  |  14 +-
 .../node/VisorNodeDataCollectorTaskResult.java  |  28 ++-
 .../visor/node/VisorPersistenceMetrics.java     | 214 +++++++++++++++++++
 .../visor/service/VisorServiceTask.java         |  10 +-
 .../frontend/app/filters/duration.filter.js     |   2 +-
 7 files changed, 341 insertions(+), 32 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/6c9d2228/modules/core/src/main/java/org/apache/ignite/internal/visor/node/VisorNodeDataCollectorJob.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/visor/node/VisorNodeDataCollectorJob.java b/modules/core/src/main/java/org/apache/ignite/internal/visor/node/VisorNodeDataCollectorJob.java
index f5ea567..3fd7b0d 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/visor/node/VisorNodeDataCollectorJob.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/visor/node/VisorNodeDataCollectorJob.java
@@ -39,6 +39,7 @@ import org.apache.ignite.internal.visor.cache.VisorMemoryMetrics;
 import org.apache.ignite.internal.visor.compute.VisorComputeMonitoringHolder;
 import org.apache.ignite.internal.visor.igfs.VisorIgfs;
 import org.apache.ignite.internal.visor.igfs.VisorIgfsEndpoint;
+import org.apache.ignite.internal.visor.util.VisorExceptionWrapper;
 import org.apache.ignite.lang.IgniteProductVersion;
 
 import static org.apache.ignite.internal.processors.cache.GridCacheUtils.isIgfsCache;
@@ -119,7 +120,7 @@ public class VisorNodeDataCollectorJob extends VisorJob<VisorNodeDataCollectorTa
             events0(res, arg.getEventsOrderKey(), arg.getEventsThrottleCounterKey(), arg.isTaskMonitoringEnabled());
         }
         catch (Exception e) {
-            res.setEventsEx(e);
+            res.setEventsEx(new VisorExceptionWrapper(e));
         }
     }
 
@@ -160,9 +161,9 @@ public class VisorNodeDataCollectorJob extends VisorJob<VisorNodeDataCollectorTa
                 memoryMetrics.add(new VisorMemoryMetrics(m));
         }
         catch (Exception e) {
-            res.setMemoryMetricsEx(e);
+            res.setMemoryMetricsEx(new VisorExceptionWrapper(e));
         }
-}
+    }
 
     /**
      * Collect caches.
@@ -205,7 +206,7 @@ public class VisorNodeDataCollectorJob extends VisorJob<VisorNodeDataCollectorTa
             }
         }
         catch (Exception e) {
-            res.setCachesEx(e);
+            res.setCachesEx(new VisorExceptionWrapper(e));
         }
     }
 
@@ -245,7 +246,21 @@ public class VisorNodeDataCollectorJob extends VisorJob<VisorNodeDataCollectorTa
             }
         }
         catch (Exception e) {
-            res.setIgfssEx(e);
+            res.setIgfssEx(new VisorExceptionWrapper(e));
+        }
+    }
+
+    /**
+     * Collect persistence metrics.
+     *
+     * @param res Job result.
+     */
+    protected void persistenceMetrics(VisorNodeDataCollectorJobResult res) {
+        try {
+            res.setPersistenceMetrics(new VisorPersistenceMetrics(ignite.persistentStoreMetrics()));
+        }
+        catch (Exception e) {
+            res.setPersistenceMetricsEx(new VisorExceptionWrapper(e));
         }
     }
 
@@ -292,7 +307,12 @@ public class VisorNodeDataCollectorJob extends VisorJob<VisorNodeDataCollectorTa
         igfs(res);
 
         if (debug)
-            log(ignite.log(), "Collected igfs", getClass(), start0);
+            start0 = log(ignite.log(), "Collected igfs", getClass(), start0);
+
+        persistenceMetrics(res);
+
+        if (debug)
+            log(ignite.log(), "Collected persistence metrics", getClass(), start0);
 
         res.setErrorCount(ignite.context().exceptionRegistry().errorCount());
 

http://git-wip-us.apache.org/repos/asf/ignite/blob/6c9d2228/modules/core/src/main/java/org/apache/ignite/internal/visor/node/VisorNodeDataCollectorJobResult.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/visor/node/VisorNodeDataCollectorJobResult.java b/modules/core/src/main/java/org/apache/ignite/internal/visor/node/VisorNodeDataCollectorJobResult.java
index ce4f9fc..90ecf6e 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/visor/node/VisorNodeDataCollectorJobResult.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/visor/node/VisorNodeDataCollectorJobResult.java
@@ -30,6 +30,7 @@ import org.apache.ignite.internal.visor.cache.VisorMemoryMetrics;
 import org.apache.ignite.internal.visor.event.VisorGridEvent;
 import org.apache.ignite.internal.visor.igfs.VisorIgfs;
 import org.apache.ignite.internal.visor.igfs.VisorIgfsEndpoint;
+import org.apache.ignite.internal.visor.util.VisorExceptionWrapper;
 
 /**
  * Data collector job result.
@@ -51,19 +52,19 @@ public class VisorNodeDataCollectorJobResult extends VisorDataTransferObject {
     private List<VisorGridEvent> evts = new ArrayList<>();
 
     /** Exception while collecting node events. */
-    private Throwable evtsEx;
+    private VisorExceptionWrapper evtsEx;
 
     /** Node memory metrics. */
     private List<VisorMemoryMetrics> memoryMetrics = new ArrayList<>();
 
     /** Exception while collecting memory metrics. */
-    private Throwable memoryMetricsEx;
+    private VisorExceptionWrapper memoryMetricsEx;
 
     /** Node caches. */
     private List<VisorCache> caches = new ArrayList<>();
 
     /** Exception while collecting node caches. */
-    private Throwable cachesEx;
+    private VisorExceptionWrapper cachesEx;
 
     /** Node IGFSs. */
     private List<VisorIgfs> igfss = new ArrayList<>();
@@ -72,7 +73,7 @@ public class VisorNodeDataCollectorJobResult extends VisorDataTransferObject {
     private List<VisorIgfsEndpoint> igfsEndpoints = new ArrayList<>();
 
     /** Exception while collecting node IGFSs. */
-    private Throwable igfssEx;
+    private VisorExceptionWrapper igfssEx;
 
     /** Errors count. */
     private long errCnt;
@@ -83,6 +84,12 @@ public class VisorNodeDataCollectorJobResult extends VisorDataTransferObject {
     /** Whether pending exchange future exists. */
     private boolean hasPendingExchange;
 
+    /** Persistence metrics. */
+    private VisorPersistenceMetrics persistenceMetrics;
+
+    /** Exception while collecting persistence metrics. */
+    private VisorExceptionWrapper persistenceMetricsEx;
+
     /**
      * Default constructor.
      */
@@ -142,14 +149,14 @@ public class VisorNodeDataCollectorJobResult extends VisorDataTransferObject {
     /**
      * @return Exception caught during collecting events.
      */
-    public Throwable getEventsEx() {
+    public VisorExceptionWrapper getEventsEx() {
         return evtsEx;
     }
 
     /**
      * @param evtsEx Exception caught during collecting events.
      */
-    public void setEventsEx(Throwable evtsEx) {
+    public void setEventsEx(VisorExceptionWrapper evtsEx) {
         this.evtsEx = evtsEx;
     }
 
@@ -163,14 +170,14 @@ public class VisorNodeDataCollectorJobResult extends VisorDataTransferObject {
     /**
      * @return Exception caught during collecting memory metrics.
      */
-    public Throwable getMemoryMetricsEx() {
+    public VisorExceptionWrapper getMemoryMetricsEx() {
         return memoryMetricsEx;
     }
 
     /**
      * @param memoryMetricsEx Exception caught during collecting memory metrics.
      */
-    public void setMemoryMetricsEx(Throwable memoryMetricsEx) {
+    public void setMemoryMetricsEx(VisorExceptionWrapper memoryMetricsEx) {
         this.memoryMetricsEx = memoryMetricsEx;
     }
 
@@ -184,14 +191,14 @@ public class VisorNodeDataCollectorJobResult extends VisorDataTransferObject {
     /**
      * @return Exception caught during collecting caches metrics.
      */
-    public Throwable getCachesEx() {
+    public VisorExceptionWrapper getCachesEx() {
         return cachesEx;
     }
 
     /**
      * @param cachesEx Exception caught during collecting caches metrics.
      */
-    public void setCachesEx(Throwable cachesEx) {
+    public void setCachesEx(VisorExceptionWrapper cachesEx) {
         this.cachesEx = cachesEx;
     }
 
@@ -212,14 +219,14 @@ public class VisorNodeDataCollectorJobResult extends VisorDataTransferObject {
     /**
      * @return Exception caught during collecting IGFSs metrics.
      */
-    public Throwable getIgfssEx() {
+    public VisorExceptionWrapper getIgfssEx() {
         return igfssEx;
     }
 
     /**
      * @param igfssEx Exception caught during collecting IGFSs metrics.
      */
-    public void setIgfssEx(Throwable igfssEx) {
+    public void setIgfssEx(VisorExceptionWrapper igfssEx) {
         this.igfssEx = igfssEx;
     }
 
@@ -265,6 +272,36 @@ public class VisorNodeDataCollectorJobResult extends VisorDataTransferObject {
         this.hasPendingExchange = hasPendingExchange;
     }
 
+    /**
+     * Get persistence metrics.
+     */
+    public VisorPersistenceMetrics getPersistenceMetrics() {
+        return persistenceMetrics;
+    }
+
+    /**
+     * Set persistence metrics.
+     *
+     * @param persistenceMetrics Persistence metrics.
+     */
+    public void setPersistenceMetrics(VisorPersistenceMetrics persistenceMetrics) {
+        this.persistenceMetrics = persistenceMetrics;
+    }
+
+    /**
+     * @return Exception caught during collecting persistence metrics.
+     */
+    public VisorExceptionWrapper getPersistenceMetricsEx() {
+        return persistenceMetricsEx;
+    }
+
+    /**
+     * @param persistenceMetricsEx Exception caught during collecting persistence metrics.
+     */
+    public void setPersistenceMetricsEx(VisorExceptionWrapper persistenceMetricsEx) {
+        this.persistenceMetricsEx = persistenceMetricsEx;
+    }
+
     /** {@inheritDoc} */
     @Override protected void writeExternalData(ObjectOutput out) throws IOException {
         U.writeString(out, gridName);
@@ -282,6 +319,8 @@ public class VisorNodeDataCollectorJobResult extends VisorDataTransferObject {
         out.writeLong(errCnt);
         out.writeObject(readyTopVer);
         out.writeBoolean(hasPendingExchange);
+        out.writeObject(persistenceMetrics);
+        out.writeObject(persistenceMetricsEx);
     }
 
     /** {@inheritDoc} */
@@ -290,17 +329,19 @@ public class VisorNodeDataCollectorJobResult extends VisorDataTransferObject {
         topVer = in.readLong();
         taskMonitoringEnabled = in.readBoolean();
         evts = U.readList(in);
-        evtsEx = (Throwable)in.readObject();
+        evtsEx = (VisorExceptionWrapper)in.readObject();
         memoryMetrics = U.readList(in);
-        memoryMetricsEx = (Throwable)in.readObject();
+        memoryMetricsEx = (VisorExceptionWrapper)in.readObject();
         caches = U.readList(in);
-        cachesEx = (Throwable)in.readObject();
+        cachesEx = (VisorExceptionWrapper)in.readObject();
         igfss = U.readList(in);
         igfsEndpoints = U.readList(in);
-        igfssEx = (Throwable)in.readObject();
+        igfssEx = (VisorExceptionWrapper)in.readObject();
         errCnt = in.readLong();
         readyTopVer = (VisorAffinityTopologyVersion)in.readObject();
         hasPendingExchange = in.readBoolean();
+        persistenceMetrics = (VisorPersistenceMetrics)in.readObject();
+        persistenceMetricsEx = (VisorExceptionWrapper)in.readObject();
     }
 
     /** {@inheritDoc} */

http://git-wip-us.apache.org/repos/asf/ignite/blob/6c9d2228/modules/core/src/main/java/org/apache/ignite/internal/visor/node/VisorNodeDataCollectorTask.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/visor/node/VisorNodeDataCollectorTask.java b/modules/core/src/main/java/org/apache/ignite/internal/visor/node/VisorNodeDataCollectorTask.java
index 56b3718..80664a1 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/visor/node/VisorNodeDataCollectorTask.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/visor/node/VisorNodeDataCollectorTask.java
@@ -97,19 +97,19 @@ public class VisorNodeDataCollectorTask extends VisorMultiNodeTask<VisorNodeData
             taskRes.getEvents().addAll(jobRes.getEvents());
 
         if (jobRes.getEventsEx() != null)
-            taskRes.getEventsEx().put(nid, new VisorExceptionWrapper(jobRes.getEventsEx()));
+            taskRes.getEventsEx().put(nid, jobRes.getEventsEx());
 
         if (!jobRes.getMemoryMetrics().isEmpty())
             taskRes.getMemoryMetrics().put(nid, jobRes.getMemoryMetrics());
 
         if (jobRes.getMemoryMetricsEx() != null)
-            taskRes.getMemoryMetricsEx().put(nid, new VisorExceptionWrapper(jobRes.getMemoryMetricsEx()));
+            taskRes.getMemoryMetricsEx().put(nid, jobRes.getMemoryMetricsEx());
 
         if (!jobRes.getCaches().isEmpty())
             taskRes.getCaches().put(nid, jobRes.getCaches());
 
         if (jobRes.getCachesEx() != null)
-            taskRes.getCachesEx().put(nid, new VisorExceptionWrapper(jobRes.getCachesEx()));
+            taskRes.getCachesEx().put(nid, jobRes.getCachesEx());
 
         if (!jobRes.getIgfss().isEmpty())
             taskRes.getIgfss().put(nid, jobRes.getIgfss());
@@ -118,7 +118,13 @@ public class VisorNodeDataCollectorTask extends VisorMultiNodeTask<VisorNodeData
             taskRes.getIgfsEndpoints().put(nid, jobRes.getIgfsEndpoints());
 
         if (jobRes.getIgfssEx() != null)
-            taskRes.getIgfssEx().put(nid, new VisorExceptionWrapper(jobRes.getIgfssEx()));
+            taskRes.getIgfssEx().put(nid, jobRes.getIgfssEx());
+
+        if (jobRes.getPersistenceMetrics() != null)
+            taskRes.getPersistenceMetrics().put(nid, jobRes.getPersistenceMetrics());
+
+        if (jobRes.getPersistenceMetricsEx() != null)
+            taskRes.getPersistenceMetricsEx().put(nid, jobRes.getPersistenceMetricsEx());
 
         taskRes.getReadyAffinityVersions().put(nid, jobRes.getReadyAffinityVersion());
 

http://git-wip-us.apache.org/repos/asf/ignite/blob/6c9d2228/modules/core/src/main/java/org/apache/ignite/internal/visor/node/VisorNodeDataCollectorTaskResult.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/visor/node/VisorNodeDataCollectorTaskResult.java b/modules/core/src/main/java/org/apache/ignite/internal/visor/node/VisorNodeDataCollectorTaskResult.java
index cef3a29..093e867 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/visor/node/VisorNodeDataCollectorTaskResult.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/visor/node/VisorNodeDataCollectorTaskResult.java
@@ -94,6 +94,12 @@ public class VisorNodeDataCollectorTaskResult extends VisorDataTransferObject {
     /** Whether pending exchange future exists from nodes. */
     private Map<UUID, Boolean> pendingExchanges = new HashMap<>();
 
+    /** All persistence metrics collected from nodes. */
+    private Map<UUID, VisorPersistenceMetrics> persistenceMetrics = new HashMap<>();
+
+    /** Exceptions caught during collecting persistence metrics from nodes. */
+    private Map<UUID, VisorExceptionWrapper> persistenceMetricsEx = new HashMap<>();
+
     /**
      * Default constructor.
      */
@@ -120,7 +126,9 @@ public class VisorNodeDataCollectorTaskResult extends VisorDataTransferObject {
             igfsEndpoints.isEmpty() &&
             igfssEx.isEmpty() &&
             readyTopVers.isEmpty() &&
-            pendingExchanges.isEmpty();
+            pendingExchanges.isEmpty() &&
+            persistenceMetrics.isEmpty() &&
+            persistenceMetricsEx.isEmpty();
     }
 
     /**
@@ -249,6 +257,20 @@ public class VisorNodeDataCollectorTaskResult extends VisorDataTransferObject {
         return pendingExchanges;
     }
 
+    /**
+     * All persistence metrics collected from nodes.
+     */
+    public Map<UUID, VisorPersistenceMetrics> getPersistenceMetrics() {
+        return persistenceMetrics;
+    }
+
+    /**
+     * @return Exceptions caught during collecting persistence metrics from nodes.
+     */
+    public Map<UUID, VisorExceptionWrapper> getPersistenceMetricsEx() {
+        return persistenceMetricsEx;
+    }
+
     /** {@inheritDoc} */
     @Override protected void writeExternalData(ObjectOutput out) throws IOException {
         out.writeBoolean(active);
@@ -268,6 +290,8 @@ public class VisorNodeDataCollectorTaskResult extends VisorDataTransferObject {
         U.writeMap(out, igfssEx);
         U.writeMap(out, readyTopVers);
         U.writeMap(out, pendingExchanges);
+        U.writeMap(out, persistenceMetrics);
+        U.writeMap(out, persistenceMetricsEx);
     }
 
     /** {@inheritDoc} */
@@ -289,6 +313,8 @@ public class VisorNodeDataCollectorTaskResult extends VisorDataTransferObject {
         igfssEx = U.readMap(in);
         readyTopVers = U.readMap(in);
         pendingExchanges = U.readMap(in);
+        persistenceMetrics = U.readMap(in);
+        persistenceMetricsEx = U.readMap(in);
     }
 
     /** {@inheritDoc} */

http://git-wip-us.apache.org/repos/asf/ignite/blob/6c9d2228/modules/core/src/main/java/org/apache/ignite/internal/visor/node/VisorPersistenceMetrics.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/visor/node/VisorPersistenceMetrics.java b/modules/core/src/main/java/org/apache/ignite/internal/visor/node/VisorPersistenceMetrics.java
new file mode 100644
index 0000000..c838161
--- /dev/null
+++ b/modules/core/src/main/java/org/apache/ignite/internal/visor/node/VisorPersistenceMetrics.java
@@ -0,0 +1,214 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.ignite.internal.visor.node;
+
+import java.io.IOException;
+import java.io.ObjectInput;
+import java.io.ObjectOutput;
+import org.apache.ignite.PersistenceMetrics;
+import org.apache.ignite.internal.util.typedef.internal.S;
+import org.apache.ignite.internal.visor.VisorDataTransferObject;
+
+/**
+ * DTO object for {@link PersistenceMetrics}.
+ */
+public class VisorPersistenceMetrics extends VisorDataTransferObject {
+    /** */
+    private static final long serialVersionUID = 0L;
+
+    /** */
+    private float walLoggingRate;
+
+    /** */
+    private float walWritingRate;
+
+    /** */
+    private int walArchiveSegments;
+
+    /** */
+    private float walFsyncTimeAvg;
+
+    /** */
+    private long lastCpDuration;
+
+    /** */
+    private long lastCpLockWaitDuration;
+
+    /** */
+    private long lastCpMmarkDuration;
+
+    /** */
+    private long lastCpPagesWriteDuration;
+
+    /** */
+    private long lastCpFsyncDuration;
+
+    /** */
+    private long lastCpTotalPages;
+
+    /** */
+    private long lastCpDataPages;
+
+    /** */
+    private long lastCpCowPages;
+
+    /**
+     * Default constructor.
+     */
+    public VisorPersistenceMetrics() {
+        // No-op.
+    }
+
+    /**
+     * @param metrics Persistence metrics.
+     */
+    public VisorPersistenceMetrics(PersistenceMetrics metrics) {
+        walLoggingRate = metrics.getWalLoggingRate();
+        walWritingRate = metrics.getWalWritingRate();
+        walArchiveSegments = metrics.getWalArchiveSegments();
+        walFsyncTimeAvg = metrics.getWalFsyncTimeAverage();
+        lastCpDuration = metrics.getLastCheckpointingDuration();
+        lastCpLockWaitDuration = metrics.getLastCheckpointLockWaitDuration();
+        lastCpMmarkDuration = metrics.getLastCheckpointMarkDuration();
+        lastCpPagesWriteDuration = metrics.getLastCheckpointPagesWriteDuration();
+        lastCpFsyncDuration = metrics.getLastCheckpointFsyncDuration();
+        lastCpTotalPages = metrics.getLastCheckpointTotalPagesNumber();
+        lastCpDataPages = metrics.getLastCheckpointDataPagesNumber();
+        lastCpCowPages = metrics.getLastCheckpointCopiedOnWritePagesNumber();
+    }
+
+    /**
+     * @return Average number of WAL records per second written during the last time interval.
+     */
+    public float getWalLoggingRate() {
+        return walLoggingRate;
+    }
+
+    /**
+     * @return Average number of bytes per second written during the last time interval.
+     */
+    public float getWalWritingRate(){
+        return walWritingRate;
+    }
+
+    /**
+     * @return Current number of WAL segments in the WAL archive.
+     */
+    public int getWalArchiveSegments(){
+        return walArchiveSegments;
+    }
+
+    /**
+     * @return Average WAL fsync duration in microseconds over the last time interval.
+     */
+    public float getWalFsyncTimeAverage(){
+        return walFsyncTimeAvg;
+    }
+
+    /**
+     * @return Total checkpoint duration in milliseconds.
+     */
+    public long getLastCheckpointingDuration(){
+        return lastCpDuration;
+    }
+
+    /**
+     * @return Checkpoint lock wait time in milliseconds.
+     */
+    public long getLastCheckpointLockWaitDuration(){
+        return lastCpLockWaitDuration;
+    }
+
+    /**
+     * @return Checkpoint mark duration in milliseconds.
+     */
+    public long getLastCheckpointMarkDuration(){
+        return lastCpMmarkDuration;
+    }
+
+    /**
+     * @return Checkpoint pages write phase in milliseconds.
+     */
+    public long getLastCheckpointPagesWriteDuration(){
+        return lastCpPagesWriteDuration;
+    }
+
+    /**
+     * @return Checkpoint fsync time in milliseconds.
+     */
+    public long getLastCheckpointFsyncDuration(){
+        return lastCpFsyncDuration;
+    }
+
+    /**
+     * @return Total number of pages written during the last checkpoint.
+     */
+    public long getLastCheckpointTotalPagesNumber(){
+        return lastCpTotalPages;
+    }
+
+    /**
+     * @return Total number of data pages written during the last checkpoint.
+     */
+    public long getLastCheckpointDataPagesNumber(){
+        return lastCpDataPages;
+    }
+
+    /**
+     * @return Total number of pages copied to a temporary checkpoint buffer during the last checkpoint.
+     */
+    public long getLastCheckpointCopiedOnWritePagesNumber(){
+        return lastCpCowPages;
+    }
+
+    /** {@inheritDoc} */
+    @Override protected void writeExternalData(ObjectOutput out) throws IOException {
+        out.writeFloat(walLoggingRate);
+        out.writeFloat(walWritingRate);
+        out.writeInt(walArchiveSegments);
+        out.writeFloat(walFsyncTimeAvg);
+        out.writeLong(lastCpDuration);
+        out.writeLong(lastCpLockWaitDuration);
+        out.writeLong(lastCpMmarkDuration);
+        out.writeLong(lastCpPagesWriteDuration);
+        out.writeLong(lastCpFsyncDuration);
+        out.writeLong(lastCpTotalPages);
+        out.writeLong(lastCpDataPages);
+        out.writeLong(lastCpCowPages);
+    }
+
+    /** {@inheritDoc} */
+    @Override protected void readExternalData(byte protoVer, ObjectInput in) throws IOException, ClassNotFoundException {
+        walLoggingRate = in.readFloat();
+        walWritingRate = in.readFloat();
+        walArchiveSegments = in.readInt();
+        walFsyncTimeAvg = in.readFloat();
+        lastCpDuration = in.readLong();
+        lastCpLockWaitDuration = in.readLong();
+        lastCpMmarkDuration = in.readLong();
+        lastCpPagesWriteDuration = in.readLong();
+        lastCpFsyncDuration = in.readLong();
+        lastCpTotalPages = in.readLong();
+        lastCpDataPages = in.readLong();
+        lastCpCowPages = in.readLong();
+    }
+
+    /** {@inheritDoc} */
+    @Override public String toString() {
+        return S.toString(VisorPersistenceMetrics.class, this);
+    }
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/6c9d2228/modules/core/src/main/java/org/apache/ignite/internal/visor/service/VisorServiceTask.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/visor/service/VisorServiceTask.java b/modules/core/src/main/java/org/apache/ignite/internal/visor/service/VisorServiceTask.java
index 1b3495c..f2489bc 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/visor/service/VisorServiceTask.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/visor/service/VisorServiceTask.java
@@ -57,12 +57,14 @@ public class VisorServiceTask extends VisorOneNodeTask<Void, Collection<VisorSer
 
         /** {@inheritDoc} */
         @Override protected Collection<VisorServiceDescriptor> run(final Void arg) {
-            Collection<ServiceDescriptor> services = ignite.services().serviceDescriptors();
+            Collection<VisorServiceDescriptor> res = new ArrayList<>();
 
-            Collection<VisorServiceDescriptor> res = new ArrayList<>(services.size());
+            if (ignite.active()) {
+                Collection<ServiceDescriptor> services = ignite.services().serviceDescriptors();
 
-            for (ServiceDescriptor srvc: services)
-                res.add(new VisorServiceDescriptor(srvc));
+                for (ServiceDescriptor srvc : services)
+                    res.add(new VisorServiceDescriptor(srvc));
+            }
 
             return res;
         }

http://git-wip-us.apache.org/repos/asf/ignite/blob/6c9d2228/modules/web-console/frontend/app/filters/duration.filter.js
----------------------------------------------------------------------
diff --git a/modules/web-console/frontend/app/filters/duration.filter.js b/modules/web-console/frontend/app/filters/duration.filter.js
index 770bca3..46d50d8 100644
--- a/modules/web-console/frontend/app/filters/duration.filter.js
+++ b/modules/web-console/frontend/app/filters/duration.filter.js
@@ -34,7 +34,7 @@ export default [() => {
         const h = Math.floor((t - d * cd) / ch);
         const m = Math.floor((t - d * cd - h * ch) / cm);
         const s = Math.floor((t - d * cd - h * ch - m * cm) / cs);
-        const ms = t % 1000;
+        const ms = Math.round(t % 1000);
 
         return a(d, 'd') + a(h, 'h') + a(m, 'm') + a(s, 's') + (t < cm ? ms + 'ms' : '');
     };


[13/33] ignite git commit: Muted hanging tests.

Posted by sb...@apache.org.
Muted hanging tests.


Project: http://git-wip-us.apache.org/repos/asf/ignite/repo
Commit: http://git-wip-us.apache.org/repos/asf/ignite/commit/9743fa3d
Tree: http://git-wip-us.apache.org/repos/asf/ignite/tree/9743fa3d
Diff: http://git-wip-us.apache.org/repos/asf/ignite/diff/9743fa3d

Branch: refs/heads/ignite-2.1.2-exchange
Commit: 9743fa3d711d0f1bbf0fa7ceec65237879f76d0d
Parents: f391290
Author: Ilya Lantukh <il...@gridgain.com>
Authored: Tue Jul 4 18:10:25 2017 +0300
Committer: Ilya Lantukh <il...@gridgain.com>
Committed: Tue Jul 4 18:11:09 2017 +0300

----------------------------------------------------------------------
 .../IgniteCommunicationBalanceMultipleConnectionsTest.java      | 5 +++++
 .../IgniteHadoopFileSystemShmemExternalDualAsyncSelfTest.java   | 5 +++++
 .../org/apache/ignite/spark/JavaEmbeddedIgniteRDDSelfTest.java  | 5 +++++
 3 files changed, 15 insertions(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/9743fa3d/modules/core/src/test/java/org/apache/ignite/internal/managers/communication/IgniteCommunicationBalanceMultipleConnectionsTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/managers/communication/IgniteCommunicationBalanceMultipleConnectionsTest.java b/modules/core/src/test/java/org/apache/ignite/internal/managers/communication/IgniteCommunicationBalanceMultipleConnectionsTest.java
index e95b1ec..444f086 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/managers/communication/IgniteCommunicationBalanceMultipleConnectionsTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/managers/communication/IgniteCommunicationBalanceMultipleConnectionsTest.java
@@ -25,4 +25,9 @@ public class IgniteCommunicationBalanceMultipleConnectionsTest extends IgniteCom
     @Override protected int connectionsPerNode() {
         return 5;
     }
+
+    /** {@inheritDoc} */
+    @Override protected void beforeTest() throws Exception {
+        fail("https://issues.apache.org/jira/browse/IGNITE-5689");
+    }
 }

http://git-wip-us.apache.org/repos/asf/ignite/blob/9743fa3d/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/impl/igfs/IgniteHadoopFileSystemShmemExternalDualAsyncSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/impl/igfs/IgniteHadoopFileSystemShmemExternalDualAsyncSelfTest.java b/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/impl/igfs/IgniteHadoopFileSystemShmemExternalDualAsyncSelfTest.java
index 5154642..edf0279 100644
--- a/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/impl/igfs/IgniteHadoopFileSystemShmemExternalDualAsyncSelfTest.java
+++ b/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/impl/igfs/IgniteHadoopFileSystemShmemExternalDualAsyncSelfTest.java
@@ -30,4 +30,9 @@ public class IgniteHadoopFileSystemShmemExternalDualAsyncSelfTest
     public IgniteHadoopFileSystemShmemExternalDualAsyncSelfTest() {
         super(DUAL_ASYNC, true);
     }
+
+    /** {@inheritDoc} */
+    @Override protected void beforeTest() throws Exception {
+        fail("https://issues.apache.org/jira/browse/IGNITE-5691");
+    }
 }
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/ignite/blob/9743fa3d/modules/spark/src/test/java/org/apache/ignite/spark/JavaEmbeddedIgniteRDDSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/spark/src/test/java/org/apache/ignite/spark/JavaEmbeddedIgniteRDDSelfTest.java b/modules/spark/src/test/java/org/apache/ignite/spark/JavaEmbeddedIgniteRDDSelfTest.java
index 49bb1ac..5477d43 100644
--- a/modules/spark/src/test/java/org/apache/ignite/spark/JavaEmbeddedIgniteRDDSelfTest.java
+++ b/modules/spark/src/test/java/org/apache/ignite/spark/JavaEmbeddedIgniteRDDSelfTest.java
@@ -100,6 +100,11 @@ public class JavaEmbeddedIgniteRDDSelfTest extends GridCommonAbstractTest {
     }
 
     /** {@inheritDoc} */
+    @Override protected void beforeTest() throws Exception {
+        fail("https://issues.apache.org/jira/browse/IGNITE-5690");
+    }
+
+    /** {@inheritDoc} */
     @Override protected void afterTest() throws Exception {
         stopAllGrids();
     }


[19/33] ignite git commit: IGNITE-4536 two tests unmuted, one test removed as obsolete (IGNITE-5592) - Fixes #2235.

Posted by sb...@apache.org.
IGNITE-4536 two tests unmuted, one test removed as obsolete (IGNITE-5592) - Fixes #2235.

Signed-off-by: Alexey Goncharuk <al...@gmail.com>


Project: http://git-wip-us.apache.org/repos/asf/ignite/repo
Commit: http://git-wip-us.apache.org/repos/asf/ignite/commit/90b67fa1
Tree: http://git-wip-us.apache.org/repos/asf/ignite/tree/90b67fa1
Diff: http://git-wip-us.apache.org/repos/asf/ignite/diff/90b67fa1

Branch: refs/heads/ignite-2.1.2-exchange
Commit: 90b67fa1b72b096943499e26e402988840bdbe97
Parents: 44f3fac
Author: Sergey Chugunov <se...@gmail.com>
Authored: Tue Jul 4 20:47:09 2017 +0300
Committer: Alexey Goncharuk <al...@gmail.com>
Committed: Tue Jul 4 20:47:09 2017 +0300

----------------------------------------------------------------------
 .../cache/GridCacheAbstractMetricsSelfTest.java | 24 --------------------
 1 file changed, 24 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/90b67fa1/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheAbstractMetricsSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheAbstractMetricsSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheAbstractMetricsSelfTest.java
index 2e81b96..6aed380 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheAbstractMetricsSelfTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheAbstractMetricsSelfTest.java
@@ -17,7 +17,6 @@
 
 package org.apache.ignite.internal.processors.cache;
 
-import java.util.Collections;
 import java.util.HashMap;
 import java.util.HashSet;
 import java.util.LinkedHashSet;
@@ -581,8 +580,6 @@ public abstract class GridCacheAbstractMetricsSelfTest extends GridCacheAbstract
      * @throws Exception If failed.
      */
     public void testMisses() throws Exception {
-        fail("https://issues.apache.org/jira/browse/IGNITE-4536");
-
         IgniteCache<Integer, Integer> cache = grid(0).cache(DEFAULT_CACHE_NAME);
 
         int keyCnt = keyCount();
@@ -625,8 +622,6 @@ public abstract class GridCacheAbstractMetricsSelfTest extends GridCacheAbstract
      * @throws Exception If failed.
      */
     public void testMissesOnEmptyCache() throws Exception {
-        fail("https://issues.apache.org/jira/browse/IGNITE-4536");
-
         IgniteCache<Integer, Integer> cache = grid(0).cache(DEFAULT_CACHE_NAME);
 
         assertEquals("Expected 0 read", 0, cache.localMetrics().getCacheGets());
@@ -678,25 +673,6 @@ public abstract class GridCacheAbstractMetricsSelfTest extends GridCacheAbstract
     /**
      * @throws Exception If failed.
      */
-    public void testManualEvictions() throws Exception {
-        fail("https://issues.apache.org/jira/browse/IGNITE-4536");
-
-        IgniteCache<Integer, Integer> cache = grid(0).cache(DEFAULT_CACHE_NAME);
-
-        if (cache.getConfiguration(CacheConfiguration.class).getCacheMode() == CacheMode.PARTITIONED)
-            return;
-
-        cache.put(1, 1);
-
-        cache.localEvict(Collections.singleton(1));
-
-        assertEquals(0L, cache.localMetrics().getCacheRemovals());
-        assertEquals(1L, cache.localMetrics().getCacheEvictions());
-    }
-
-    /**
-     * @throws Exception If failed.
-     */
     public void testTxEvictions() throws Exception {
         if (grid(0).cache(DEFAULT_CACHE_NAME).getConfiguration(CacheConfiguration.class).getAtomicityMode() != CacheAtomicityMode.ATOMIC)
             checkTtl(true);


[21/33] ignite git commit: IGNITE-5693 - Fixed semaphore flaky test - Fixes #2239.

Posted by sb...@apache.org.
IGNITE-5693 - Fixed semaphore flaky test - Fixes #2239.

Signed-off-by: Alexey Goncharuk <al...@gmail.com>


Project: http://git-wip-us.apache.org/repos/asf/ignite/repo
Commit: http://git-wip-us.apache.org/repos/asf/ignite/commit/5b7165ca
Tree: http://git-wip-us.apache.org/repos/asf/ignite/tree/5b7165ca
Diff: http://git-wip-us.apache.org/repos/asf/ignite/diff/5b7165ca

Branch: refs/heads/ignite-2.1.2-exchange
Commit: 5b7165ca717de038249570c4213f050ba9b5aee7
Parents: 50bb090
Author: Ilya Lantukh <il...@gridgain.com>
Authored: Tue Jul 4 20:57:56 2017 +0300
Committer: Alexey Goncharuk <al...@gmail.com>
Committed: Tue Jul 4 20:57:56 2017 +0300

----------------------------------------------------------------------
 .../IgniteSemaphoreAbstractSelfTest.java           | 17 +++++++++++++----
 1 file changed, 13 insertions(+), 4 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/5b7165ca/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/datastructures/IgniteSemaphoreAbstractSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/datastructures/IgniteSemaphoreAbstractSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/datastructures/IgniteSemaphoreAbstractSelfTest.java
index d5f4e0e..445d469 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/datastructures/IgniteSemaphoreAbstractSelfTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/datastructures/IgniteSemaphoreAbstractSelfTest.java
@@ -35,6 +35,7 @@ import org.apache.ignite.configuration.IgniteConfiguration;
 import org.apache.ignite.internal.IgniteEx;
 import org.apache.ignite.internal.IgniteInternalFuture;
 import org.apache.ignite.internal.IgniteKernal;
+import org.apache.ignite.internal.util.lang.GridAbsPredicateX;
 import org.apache.ignite.internal.util.typedef.G;
 import org.apache.ignite.internal.util.typedef.PA;
 import org.apache.ignite.lang.IgniteCallable;
@@ -437,8 +438,7 @@ public abstract class IgniteSemaphoreAbstractSelfTest extends IgniteAtomicsAbstr
      * @param semaphoreName Semaphore name.
      * @throws Exception If failed.
      */
-    private void removeSemaphore(String semaphoreName)
-        throws Exception {
+    private void removeSemaphore(final String semaphoreName) throws Exception {
         IgniteSemaphore semaphore = grid(RND.nextInt(NODES_CNT)).semaphore(semaphoreName, 10, false, true);
 
         assert semaphore != null;
@@ -454,8 +454,17 @@ public abstract class IgniteSemaphoreAbstractSelfTest extends IgniteAtomicsAbstr
         semaphore0.close();
 
         // Ensure semaphore is removed on all nodes.
-        for (Ignite g : G.allGrids())
-            assertNull(((IgniteKernal)g).context().dataStructures().semaphore(semaphoreName, null, 10, true, false));
+        assert GridTestUtils.waitForCondition(new GridAbsPredicateX() {
+            @Override public boolean applyx() throws IgniteCheckedException {
+                for (Ignite g : G.allGrids()) {
+                    if (((IgniteKernal)g).context().dataStructures().semaphore(
+                        semaphoreName, null, 10, true, false) != null)
+                        return false;
+                }
+
+                return true;
+            }
+        }, 5_000);
 
         checkRemovedSemaphore(semaphore);
     }


[17/33] ignite git commit: IGNITE-5558 - Added ability to read WAL in standalone mode - Fixes #2174.

Posted by sb...@apache.org.
http://git-wip-us.apache.org/repos/asf/ignite/blob/44f3fac2/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/wal/reader/StandaloneGridKernalContext.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/wal/reader/StandaloneGridKernalContext.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/wal/reader/StandaloneGridKernalContext.java
new file mode 100644
index 0000000..df932e6
--- /dev/null
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/wal/reader/StandaloneGridKernalContext.java
@@ -0,0 +1,499 @@
+/*
+ * 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.persistence.wal.reader;
+
+import java.util.Iterator;
+import java.util.List;
+import java.util.Map;
+import java.util.UUID;
+import java.util.concurrent.ExecutorService;
+import org.apache.ignite.IgniteLogger;
+import org.apache.ignite.configuration.IgniteConfiguration;
+import org.apache.ignite.internal.GridComponent;
+import org.apache.ignite.internal.GridKernalContext;
+import org.apache.ignite.internal.GridKernalGateway;
+import org.apache.ignite.internal.IgniteEx;
+import org.apache.ignite.internal.MarshallerContextImpl;
+import org.apache.ignite.internal.managers.checkpoint.GridCheckpointManager;
+import org.apache.ignite.internal.managers.collision.GridCollisionManager;
+import org.apache.ignite.internal.managers.communication.GridIoManager;
+import org.apache.ignite.internal.managers.deployment.GridDeploymentManager;
+import org.apache.ignite.internal.managers.discovery.GridDiscoveryManager;
+import org.apache.ignite.internal.managers.eventstorage.GridEventStorageManager;
+import org.apache.ignite.internal.managers.failover.GridFailoverManager;
+import org.apache.ignite.internal.managers.indexing.GridIndexingManager;
+import org.apache.ignite.internal.managers.loadbalancer.GridLoadBalancerManager;
+import org.apache.ignite.internal.processors.affinity.GridAffinityProcessor;
+import org.apache.ignite.internal.processors.cache.GridCacheProcessor;
+import org.apache.ignite.internal.processors.cacheobject.IgniteCacheObjectProcessor;
+import org.apache.ignite.internal.processors.closure.GridClosureProcessor;
+import org.apache.ignite.internal.processors.cluster.ClusterProcessor;
+import org.apache.ignite.internal.processors.cluster.GridClusterStateProcessor;
+import org.apache.ignite.internal.processors.continuous.GridContinuousProcessor;
+import org.apache.ignite.internal.processors.datastreamer.DataStreamProcessor;
+import org.apache.ignite.internal.processors.datastructures.DataStructuresProcessor;
+import org.apache.ignite.internal.processors.hadoop.HadoopHelper;
+import org.apache.ignite.internal.processors.hadoop.HadoopProcessorAdapter;
+import org.apache.ignite.internal.processors.igfs.IgfsHelper;
+import org.apache.ignite.internal.processors.igfs.IgfsProcessorAdapter;
+import org.apache.ignite.internal.processors.job.GridJobProcessor;
+import org.apache.ignite.internal.processors.jobmetrics.GridJobMetricsProcessor;
+import org.apache.ignite.internal.processors.marshaller.GridMarshallerMappingProcessor;
+import org.apache.ignite.internal.processors.odbc.SqlListenerProcessor;
+import org.apache.ignite.internal.processors.platform.PlatformProcessor;
+import org.apache.ignite.internal.processors.plugin.IgnitePluginProcessor;
+import org.apache.ignite.internal.processors.pool.PoolProcessor;
+import org.apache.ignite.internal.processors.port.GridPortProcessor;
+import org.apache.ignite.internal.processors.query.GridQueryProcessor;
+import org.apache.ignite.internal.processors.resource.GridResourceProcessor;
+import org.apache.ignite.internal.processors.rest.GridRestProcessor;
+import org.apache.ignite.internal.processors.schedule.IgniteScheduleProcessorAdapter;
+import org.apache.ignite.internal.processors.security.GridSecurityProcessor;
+import org.apache.ignite.internal.processors.segmentation.GridSegmentationProcessor;
+import org.apache.ignite.internal.processors.service.GridServiceProcessor;
+import org.apache.ignite.internal.processors.session.GridTaskSessionProcessor;
+import org.apache.ignite.internal.processors.task.GridTaskProcessor;
+import org.apache.ignite.internal.processors.timeout.GridTimeoutProcessor;
+import org.apache.ignite.internal.suggestions.GridPerformanceSuggestions;
+import org.apache.ignite.internal.util.IgniteExceptionRegistry;
+import org.apache.ignite.internal.util.StripedExecutor;
+import org.apache.ignite.plugin.PluginNotFoundException;
+import org.apache.ignite.plugin.PluginProvider;
+import org.apache.ignite.thread.IgniteStripedThreadPoolExecutor;
+import org.jetbrains.annotations.NotNull;
+import org.jetbrains.annotations.Nullable;
+
+/**
+ * Dummy grid kernal context
+ */
+public class StandaloneGridKernalContext implements GridKernalContext {
+    private IgniteLogger log;
+
+    /**
+     * @param log Logger.
+     */
+    StandaloneGridKernalContext(IgniteLogger log) {
+        this.log = log;
+    }
+
+    /** {@inheritDoc} */
+    @Override public List<GridComponent> components() {
+        return null;
+    }
+
+    /** {@inheritDoc} */
+    @Override public UUID localNodeId() {
+        return null;
+    }
+
+    /** {@inheritDoc} */
+    @Override public String igniteInstanceName() {
+        return null;
+    }
+
+    /** {@inheritDoc} */
+    @Override public IgniteLogger log(String ctgr) {
+        return log;
+    }
+
+    /** {@inheritDoc} */
+    @Override public IgniteLogger log(Class<?> cls) {
+        return log;
+    }
+
+    /** {@inheritDoc} */
+    @Override public boolean isStopping() {
+        return false;
+    }
+
+    /** {@inheritDoc} */
+    @Override public GridKernalGateway gateway() {
+        return null;
+    }
+
+    /** {@inheritDoc} */
+    @Override public IgniteEx grid() {
+        return null;
+    }
+
+    /** {@inheritDoc} */
+    @Override public IgniteConfiguration config() {
+        return null;
+    }
+
+    /** {@inheritDoc} */
+    @Override public GridTaskProcessor task() {
+        return null;
+    }
+
+    /** {@inheritDoc} */
+    @Override public GridAffinityProcessor affinity() {
+        return null;
+    }
+
+    /** {@inheritDoc} */
+    @Override public GridJobProcessor job() {
+        return null;
+    }
+
+    /** {@inheritDoc} */
+    @Override public GridTimeoutProcessor timeout() {
+        return null;
+    }
+
+    /** {@inheritDoc} */
+    @Override public GridResourceProcessor resource() {
+        return null;
+    }
+
+    /** {@inheritDoc} */
+    @Override public GridJobMetricsProcessor jobMetric() {
+        return null;
+    }
+
+    /** {@inheritDoc} */
+    @Override public GridCacheProcessor cache() {
+        return null;
+    }
+
+    /** {@inheritDoc} */
+    @Override public GridClusterStateProcessor state() {
+        return null;
+    }
+
+    /** {@inheritDoc} */
+    @Override public GridTaskSessionProcessor session() {
+        return null;
+    }
+
+    /** {@inheritDoc} */
+    @Override public GridClosureProcessor closure() {
+        return null;
+    }
+
+    /** {@inheritDoc} */
+    @Override public GridServiceProcessor service() {
+        return null;
+    }
+
+    /** {@inheritDoc} */
+    @Override public GridPortProcessor ports() {
+        return null;
+    }
+
+    /** {@inheritDoc} */
+    @Override public IgniteScheduleProcessorAdapter schedule() {
+        return null;
+    }
+
+    /** {@inheritDoc} */
+    @Override public GridRestProcessor rest() {
+        return null;
+    }
+
+    /** {@inheritDoc} */
+    @Override public GridSegmentationProcessor segmentation() {
+        return null;
+    }
+
+    /** {@inheritDoc} */
+    @Override public <K, V> DataStreamProcessor<K, V> dataStream() {
+        return null;
+    }
+
+    /** {@inheritDoc} */
+    @Override public IgfsProcessorAdapter igfs() {
+        return null;
+    }
+
+    /** {@inheritDoc} */
+    @Override public IgfsHelper igfsHelper() {
+        return null;
+    }
+
+    /** {@inheritDoc} */
+    @Override public GridContinuousProcessor continuous() {
+        return null;
+    }
+
+    /** {@inheritDoc} */
+    @Override public HadoopProcessorAdapter hadoop() {
+        return null;
+    }
+
+    /** {@inheritDoc} */
+    @Override public PoolProcessor pools() {
+        return null;
+    }
+
+    /** {@inheritDoc} */
+    @Override public GridMarshallerMappingProcessor mapping() {
+        return null;
+    }
+
+    /** {@inheritDoc} */
+    @Override public HadoopHelper hadoopHelper() {
+        return null;
+    }
+
+    /** {@inheritDoc} */
+    @Override public ExecutorService utilityCachePool() {
+        return null;
+    }
+
+    /** {@inheritDoc} */
+    @Override public IgniteStripedThreadPoolExecutor asyncCallbackPool() {
+        return null;
+    }
+
+    /** {@inheritDoc} */
+    @Override public IgniteCacheObjectProcessor cacheObjects() {
+        return null;
+    }
+
+    /** {@inheritDoc} */
+    @Override public GridQueryProcessor query() {
+        return null;
+    }
+
+    /** {@inheritDoc} */
+    @Override public SqlListenerProcessor sqlListener() {
+        return null;
+    }
+
+    /** {@inheritDoc} */
+    @Override public IgnitePluginProcessor plugins() {
+        return null;
+    }
+
+    /** {@inheritDoc} */
+    @Override public GridDeploymentManager deploy() {
+        return null;
+    }
+
+    /** {@inheritDoc} */
+    @Override public GridIoManager io() {
+        return null;
+    }
+
+    /** {@inheritDoc} */
+    @Override public GridDiscoveryManager discovery() {
+        return null;
+    }
+
+    /** {@inheritDoc} */
+    @Override public GridCheckpointManager checkpoint() {
+        return null;
+    }
+
+    /** {@inheritDoc} */
+    @Override public GridEventStorageManager event() {
+        return null;
+    }
+
+    /** {@inheritDoc} */
+    @Override public GridFailoverManager failover() {
+        return null;
+    }
+
+    /** {@inheritDoc} */
+    @Override public GridCollisionManager collision() {
+        return null;
+    }
+
+    /** {@inheritDoc} */
+    @Override public GridSecurityProcessor security() {
+        return null;
+    }
+
+    /** {@inheritDoc} */
+    @Override public GridLoadBalancerManager loadBalancing() {
+        return null;
+    }
+
+    /** {@inheritDoc} */
+    @Override public GridIndexingManager indexing() {
+        return null;
+    }
+
+    /** {@inheritDoc} */
+    @Override public DataStructuresProcessor dataStructures() {
+        return null;
+    }
+
+    /** {@inheritDoc} */
+    @Override public void markSegmented() { }
+
+    /** {@inheritDoc} */
+    @Override public boolean segmented() {
+        return false;
+    }
+
+    /** {@inheritDoc} */
+    @Override public void printMemoryStats() { }
+
+    /** {@inheritDoc} */
+    @Override public boolean isDaemon() {
+        return false;
+    }
+
+    /** {@inheritDoc} */
+    @Override public GridPerformanceSuggestions performance() {
+        return null;
+    }
+
+    /** {@inheritDoc} */
+    @Override public String userVersion(ClassLoader ldr) {
+        return null;
+    }
+
+    /** {@inheritDoc} */
+    @Override public PluginProvider pluginProvider(String name) throws PluginNotFoundException {
+        return null;
+    }
+
+    /** {@inheritDoc} */
+    @Override public <T> T createComponent(Class<T> cls) {
+        return null;
+    }
+
+    /** {@inheritDoc} */
+    @Override public ExecutorService getExecutorService() {
+        return null;
+    }
+
+    /** {@inheritDoc} */
+    @Override public ExecutorService getServiceExecutorService() {
+        return null;
+    }
+
+    /** {@inheritDoc} */
+    @Override public ExecutorService getSystemExecutorService() {
+        return null;
+    }
+
+    /** {@inheritDoc} */
+    @Override public StripedExecutor getStripedExecutorService() {
+        return null;
+    }
+
+    /** {@inheritDoc} */
+    @Override public ExecutorService getManagementExecutorService() {
+        return null;
+    }
+
+    /** {@inheritDoc} */
+    @Override public ExecutorService getPeerClassLoadingExecutorService() {
+        return null;
+    }
+
+    /** {@inheritDoc} */
+    @Override public ExecutorService getIgfsExecutorService() {
+        return null;
+    }
+
+    /** {@inheritDoc} */
+    @Override public ExecutorService getDataStreamerExecutorService() {
+        return null;
+    }
+
+    /** {@inheritDoc} */
+    @Override public ExecutorService getRestExecutorService() {
+        return null;
+    }
+
+    /** {@inheritDoc} */
+    @Override public ExecutorService getAffinityExecutorService() {
+        return null;
+    }
+
+    /** {@inheritDoc} */
+    @Nullable @Override public ExecutorService getIndexingExecutorService() {
+        return null;
+    }
+
+    /** {@inheritDoc} */
+    @Override public ExecutorService getQueryExecutorService() {
+        return null;
+    }
+
+    /** {@inheritDoc} */
+    @Nullable @Override public Map<String, ? extends ExecutorService> customExecutors() {
+        return null;
+    }
+
+    /** {@inheritDoc} */
+    @Override public ExecutorService getSchemaExecutorService() {
+        return null;
+    }
+
+    /** {@inheritDoc} */
+    @Override public IgniteExceptionRegistry exceptionRegistry() {
+        return null;
+    }
+
+    /** {@inheritDoc} */
+    @Override public Object nodeAttribute(String key) {
+        return null;
+    }
+
+    /** {@inheritDoc} */
+    @Override public boolean hasNodeAttribute(String key) {
+        return false;
+    }
+
+    /** {@inheritDoc} */
+    @Override public Object addNodeAttribute(String key, Object val) {
+        return null;
+    }
+
+    /** {@inheritDoc} */
+    @Override public Map<String, Object> nodeAttributes() {
+        return null;
+    }
+
+    /** {@inheritDoc} */
+    @Override public ClusterProcessor cluster() {
+        return null;
+    }
+
+    /** {@inheritDoc} */
+    @Override public MarshallerContextImpl marshallerContext() {
+        return null;
+    }
+
+    /** {@inheritDoc} */
+    @Override public boolean clientNode() {
+        return false;
+    }
+
+    /** {@inheritDoc} */
+    @Override public boolean clientDisconnected() {
+        return false;
+    }
+
+    /** {@inheritDoc} */
+    @Override public PlatformProcessor platform() {
+        return null;
+    }
+
+    /** {@inheritDoc} */
+    @NotNull @Override public Iterator<GridComponent> iterator() {
+        return null;
+    }
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/44f3fac2/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/wal/reader/StandaloneIgniteCacheDatabaseSharedManager.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/wal/reader/StandaloneIgniteCacheDatabaseSharedManager.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/wal/reader/StandaloneIgniteCacheDatabaseSharedManager.java
new file mode 100644
index 0000000..85a8724
--- /dev/null
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/wal/reader/StandaloneIgniteCacheDatabaseSharedManager.java
@@ -0,0 +1,30 @@
+/*
+ * 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.persistence.wal.reader;
+
+import org.apache.ignite.internal.processors.cache.persistence.IgniteCacheDatabaseSharedManager;
+
+/**
+ * Fake implementation for publishing setter and for creation in standalone WAL reader tool
+ */
+class StandaloneIgniteCacheDatabaseSharedManager extends IgniteCacheDatabaseSharedManager {
+    /** {@inheritDoc} */
+    @Override public void setPageSize(int pageSize) {
+        super.setPageSize(pageSize);
+    }
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/44f3fac2/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/wal/reader/StandaloneWalRecordsIterator.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/wal/reader/StandaloneWalRecordsIterator.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/wal/reader/StandaloneWalRecordsIterator.java
new file mode 100644
index 0000000..f17c112
--- /dev/null
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/wal/reader/StandaloneWalRecordsIterator.java
@@ -0,0 +1,258 @@
+/*
+ * 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.persistence.wal.reader;
+
+import java.io.DataInput;
+import java.io.File;
+import java.io.FileNotFoundException;
+import java.io.IOException;
+import java.io.RandomAccessFile;
+import java.nio.ByteBuffer;
+import java.nio.ByteOrder;
+import java.nio.channels.FileChannel;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.Collections;
+import java.util.List;
+import org.apache.ignite.IgniteCheckedException;
+import org.apache.ignite.IgniteLogger;
+import org.apache.ignite.internal.pagemem.wal.record.WALRecord;
+import org.apache.ignite.internal.processors.cache.GridCacheSharedContext;
+import org.apache.ignite.internal.processors.cache.persistence.wal.AbstractWalRecordsIterator;
+import org.apache.ignite.internal.processors.cache.persistence.wal.FileInput;
+import org.apache.ignite.internal.processors.cache.persistence.wal.FileWALPointer;
+import org.apache.ignite.internal.processors.cache.persistence.wal.FileWriteAheadLogManager;
+import org.apache.ignite.internal.processors.cache.persistence.wal.SegmentEofException;
+import org.apache.ignite.internal.processors.cache.persistence.wal.serializer.RecordV1Serializer;
+import org.apache.ignite.internal.util.typedef.F;
+import org.jetbrains.annotations.NotNull;
+import org.jetbrains.annotations.Nullable;
+
+import static org.apache.ignite.internal.processors.cache.persistence.wal.serializer.RecordV1Serializer.HEADER_RECORD_SIZE;
+
+/**
+ * WAL reader iterator, for creation in standalone WAL reader tool
+ * Operates over one directory, does not provide start and end boundaries
+ */
+class StandaloneWalRecordsIterator extends AbstractWalRecordsIterator {
+    /** */
+    private static final long serialVersionUID = 0L;
+
+    /** Record buffer size */
+    private static final int BUF_SIZE = 2 * 1024 * 1024;
+
+    /**
+     * WAL files directory. Should already contain 'consistent ID' as subfolder.
+     * <code>null</code> value means file-by-file iteration mode
+     */
+    @Nullable
+    private File walFilesDir;
+
+    /**
+     * File descriptors remained to scan.
+     * <code>null</code> value means directory scan mode
+     */
+    @Nullable
+    private List<FileWriteAheadLogManager.FileDescriptor> walFileDescriptors;
+
+    /**
+     * True if this iterator used for work dir, false for archive.
+     * In work dir mode exceptions come from record reading are ignored (file may be not completed).
+     * Index of file is taken from file itself, not from file name
+     */
+    private boolean workDir;
+
+    /**
+     * Creates iterator in directory scan mode
+     *
+     * @param walFilesDir Wal files directory. Should already contain node consistent ID as subfolder
+     * @param log Logger.
+     * @param sharedCtx Shared context.
+     */
+    StandaloneWalRecordsIterator(
+        @NotNull final File walFilesDir,
+        @NotNull final IgniteLogger log,
+        @NotNull final GridCacheSharedContext sharedCtx) throws IgniteCheckedException {
+        super(log,
+            sharedCtx,
+            new RecordV1Serializer(sharedCtx),
+            BUF_SIZE);
+        init(walFilesDir, false, null);
+        advance();
+    }
+
+    /**
+     * Creates iterator in file-by-file iteration mode. Directory
+     *
+     * @param log Logger.
+     * @param sharedCtx Shared context.
+     * @param workDir Work directory is scanned, false - archive
+     * @param walFiles Wal files.
+     */
+    StandaloneWalRecordsIterator(
+        @NotNull final IgniteLogger log,
+        @NotNull final GridCacheSharedContext sharedCtx,
+        final boolean workDir,
+        @NotNull final File... walFiles) throws IgniteCheckedException {
+        super(log,
+            sharedCtx,
+            new RecordV1Serializer(sharedCtx),
+            BUF_SIZE);
+        this.workDir = workDir;
+        init(null, workDir, walFiles);
+        advance();
+    }
+
+    /**
+     * For directory mode sets oldest file as initial segment,
+     * for file by file mode, converts all files to descriptors and gets oldest as initial.
+     *
+     * @param walFilesDir directory for directory scan mode
+     * @param workDir work directory, only for file-by-file mode
+     * @param walFiles files for file-by-file iteration mode
+     */
+    private void init(
+        @Nullable final File walFilesDir,
+        final boolean workDir,
+        @Nullable final File[] walFiles) throws IgniteCheckedException {
+        if (walFilesDir != null) {
+            FileWriteAheadLogManager.FileDescriptor[] descs = loadFileDescriptors(walFilesDir);
+            curWalSegmIdx = !F.isEmpty(descs) ? descs[0].getIdx() : 0;
+            this.walFilesDir = walFilesDir;
+            this.workDir = false;
+        }
+        else {
+            this.workDir = workDir;
+            if (workDir)
+                walFileDescriptors = scanIndexesFromFileHeaders(walFiles);
+            else
+                walFileDescriptors = new ArrayList<>(Arrays.asList(FileWriteAheadLogManager.scan(walFiles)));
+            curWalSegmIdx = !walFileDescriptors.isEmpty() ? walFileDescriptors.get(0).getIdx() : 0;
+        }
+        curWalSegmIdx--;
+
+        if (log.isDebugEnabled())
+            log.debug("Initialized WAL cursor [curWalSegmIdx=" + curWalSegmIdx + ']');
+    }
+
+    /**
+     * This methods checks all provided files to be correct WAL segment.
+     * Header record and its position is checked. WAL position is used to deremine real index.
+     * File index from file name is ignored.
+     *
+     * @param allFiles files to scan
+     * @return list of file descriptors with checked header records, file index is set
+     * @throws IgniteCheckedException if IO error occurs
+     */
+    private List<FileWriteAheadLogManager.FileDescriptor> scanIndexesFromFileHeaders(
+        @Nullable final File[] allFiles) throws IgniteCheckedException {
+        if (allFiles == null || allFiles.length == 0)
+            return Collections.emptyList();
+
+        final List<FileWriteAheadLogManager.FileDescriptor> resultingDescs = new ArrayList<>();
+
+        for (File file : allFiles) {
+            if (file.length() < HEADER_RECORD_SIZE)
+                continue;
+
+            FileWALPointer ptr;
+
+            try (RandomAccessFile rf = new RandomAccessFile(file, "r");) {
+                final FileChannel ch = rf.getChannel();
+                final ByteBuffer buf = ByteBuffer.allocate(HEADER_RECORD_SIZE);
+
+                buf.order(ByteOrder.nativeOrder());
+
+                final DataInput in = new FileInput(ch, buf);
+                // Header record must be agnostic to the serializer version.
+                final int type = in.readUnsignedByte();
+
+                if (type == WALRecord.RecordType.STOP_ITERATION_RECORD_TYPE)
+                    throw new SegmentEofException("Reached logical end of the segment", null);
+                ptr = RecordV1Serializer.readPosition(in);
+            }
+            catch (IOException e) {
+                throw new IgniteCheckedException("Failed to scan index from file [" + file + "]", e);
+            }
+
+            resultingDescs.add(new FileWriteAheadLogManager.FileDescriptor(file, ptr.index()));
+        }
+        Collections.sort(resultingDescs);
+        return resultingDescs;
+    }
+
+    /** {@inheritDoc} */
+    @Override protected FileWriteAheadLogManager.ReadFileHandle advanceSegment(
+        @Nullable final FileWriteAheadLogManager.ReadFileHandle curWalSegment) throws IgniteCheckedException {
+
+        if (curWalSegment != null)
+            curWalSegment.close();
+
+        curWalSegmIdx++;
+        // curHandle.workDir is false
+        final FileWriteAheadLogManager.FileDescriptor fd;
+
+        if (walFilesDir != null) {
+            fd = new FileWriteAheadLogManager.FileDescriptor(
+                new File(walFilesDir,
+                    FileWriteAheadLogManager.FileDescriptor.fileName(curWalSegmIdx)));
+        }
+        else {
+            if (walFileDescriptors.isEmpty())
+                return null; //no files to read, stop iteration
+
+            fd = walFileDescriptors.remove(0);
+        }
+
+        if (log.isDebugEnabled())
+            log.debug("Reading next file [absIdx=" + curWalSegmIdx + ", file=" + fd.getAbsolutePath() + ']');
+
+        assert fd != null;
+
+        curRec = null;
+        try {
+            return initReadHandle(fd, null);
+        }
+        catch (FileNotFoundException e) {
+            log.info("Missing WAL segment in the archive: " + e.getMessage());
+            return null;
+        }
+    }
+
+    /** {@inheritDoc} */
+    @Override protected void handleRecordException(
+        @NotNull final Exception e,
+        @Nullable final FileWALPointer ptr) {
+        super.handleRecordException(e, ptr);
+        final RuntimeException ex = new RuntimeException("Record reading problem occurred at file pointer [" + ptr + "]:" + e.getMessage(), e);
+
+        ex.printStackTrace();
+        if (!workDir)
+            throw ex;
+    }
+
+    /** {@inheritDoc} */
+    @Override protected void onClose() throws IgniteCheckedException {
+        super.onClose();
+        curRec = null;
+
+        closeCurrentWalSegment();
+
+        curWalSegmIdx = Integer.MAX_VALUE;
+    }
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/44f3fac2/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/wal/serializer/RecordV1Serializer.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/wal/serializer/RecordV1Serializer.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/wal/serializer/RecordV1Serializer.java
index 0ccd3a0..0a7b3dd 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/wal/serializer/RecordV1Serializer.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/wal/serializer/RecordV1Serializer.java
@@ -103,21 +103,37 @@ import static org.apache.ignite.IgniteSystemProperties.IGNITE_PDS_SKIP_CRC;
 
 /**
  * Record V1 serializer.
+ * Stores records in following format:
+ * <ul>
+ *     <li>Record type from {@link RecordType#ordinal()} incremented by 1</li>
+ *     <li>WAL pointer to double check consistency</li>
+ *     <li>Data</li>
+ *     <li>CRC or zero padding</li>
+ * </ul>
  */
 public class RecordV1Serializer implements RecordSerializer {
-    /** */
-    public static final int HEADER_RECORD_SIZE = /*Type*/1 + /*Pointer */12 + /*Magic*/8 + /*Version*/4 + /*CRC*/4;
+    /** Length of Type */
+    public static final int REC_TYPE_SIZE = 1;
+
+    /** Length of WAL Pointer */
+    public static final int FILE_WAL_POINTER_SIZE = 12;
+
+    /** Length of CRC value */
+    private static final int CRC_SIZE = 4;
 
     /** */
+    public static final int HEADER_RECORD_SIZE = REC_TYPE_SIZE + FILE_WAL_POINTER_SIZE + /*Magic*/8 + /*Version*/4 + CRC_SIZE;
+
+    /** Cache shared context */
     private GridCacheSharedContext cctx;
 
-    /** */
+    /** Size of page used for PageMemory regions */
     private int pageSize;
 
-    /** */
+    /** Cache object processor to reading {@link DataEntry DataEntries} */
     private IgniteCacheObjectProcessor co;
 
-    /** */
+    /** Skip CRC calculation/check flag */
     private boolean skipCrc = IgniteSystemProperties.getBoolean(IGNITE_PDS_SKIP_CRC, false);
 
     /**
@@ -658,7 +674,7 @@ public class RecordV1Serializer implements RecordSerializer {
 
             assert res != null;
 
-            res.size((int)(in0.position() - startPos + 4)); // Account for CRC which will be read afterwards.
+            res.size((int)(in0.position() - startPos + CRC_SIZE)); // Account for CRC which will be read afterwards.
 
             return res;
         }
@@ -671,12 +687,16 @@ public class RecordV1Serializer implements RecordSerializer {
     }
 
     /**
-     * @param in In.
+     * Loads record from input, does not read CRC value
+     *
+     * @param in Input to read record from
+     * @param expPtr expected WAL pointer for record. Used to validate actual position against expected from the file
+     * @throws SegmentEofException if end of WAL segment reached
      */
     private WALRecord readRecord(ByteBufferBackedDataInput in, WALPointer expPtr) throws IOException, IgniteCheckedException {
         int type = in.readUnsignedByte();
 
-        if (type == 0)
+        if (type == WALRecord.RecordType.STOP_ITERATION_RECORD_TYPE)
             throw new SegmentEofException("Reached logical end of the segment", null);
 
         FileWALPointer ptr = readPosition(in);
@@ -1212,7 +1232,7 @@ public class RecordV1Serializer implements RecordSerializer {
     /** {@inheritDoc} */
     @SuppressWarnings("CastConflictsWithInstanceof")
     @Override public int size(WALRecord record) throws IgniteCheckedException {
-        int commonFields = /* Type */1 + /* Pointer */12 + /*CRC*/4;
+        int commonFields = REC_TYPE_SIZE + FILE_WAL_POINTER_SIZE + CRC_SIZE;
 
         switch (record.type()) {
             case PAGE_RECORD:
@@ -1371,7 +1391,7 @@ public class RecordV1Serializer implements RecordSerializer {
                 return commonFields + /*cacheId*/ 4 + /*pageId*/ 8;
 
             case SWITCH_SEGMENT_RECORD:
-                return commonFields;
+                return commonFields - CRC_SIZE; //CRC is not loaded for switch segment, exception is thrown instead
 
             default:
                 throw new UnsupportedOperationException("Type: " + record.type());
@@ -1379,10 +1399,11 @@ public class RecordV1Serializer implements RecordSerializer {
     }
 
     /**
+     * Saves position, WAL pointer (requires {@link #FILE_WAL_POINTER_SIZE} bytes)
      * @param buf Byte buffer to serialize version to.
      * @param ptr File WAL pointer to write.
      */
-    private void putPosition(ByteBuffer buf, FileWALPointer ptr) {
+    public static void putPosition(ByteBuffer buf, FileWALPointer ptr) {
         buf.putLong(ptr.index());
         buf.putInt(ptr.fileOffset());
     }
@@ -1392,7 +1413,7 @@ public class RecordV1Serializer implements RecordSerializer {
      * @return Read file WAL pointer.
      * @throws IOException If failed to write.
      */
-    private FileWALPointer readPosition(DataInput in) throws IOException {
+    public static FileWALPointer readPosition(DataInput in) throws IOException {
         long idx = in.readLong();
         int fileOffset = in.readInt();
 

http://git-wip-us.apache.org/repos/asf/ignite/blob/44f3fac2/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/IgnitePersistentStoreDataStructuresTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/IgnitePersistentStoreDataStructuresTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/IgnitePersistentStoreDataStructuresTest.java
index 5561d95..fed8766 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/IgnitePersistentStoreDataStructuresTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/IgnitePersistentStoreDataStructuresTest.java
@@ -181,6 +181,8 @@ public class IgnitePersistentStoreDataStructuresTest extends GridCommonAbstractT
         for (int i = 0; i < 100; i++)
             set.add(i);
 
+        assertEquals(100, set.size());
+
         stopAllGrids();
 
         ignite = startGrids(4);

http://git-wip-us.apache.org/repos/asf/ignite/blob/44f3fac2/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/db/wal/IgniteWalHistoryReservationsTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/db/wal/IgniteWalHistoryReservationsTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/db/wal/IgniteWalHistoryReservationsTest.java
index 793806e..48d8c21 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/db/wal/IgniteWalHistoryReservationsTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/db/wal/IgniteWalHistoryReservationsTest.java
@@ -297,7 +297,7 @@ public class IgniteWalHistoryReservationsTest extends GridCommonAbstractTest {
         final int entryCnt = 10_000;
         final int initGridCnt = 4;
 
-        final IgniteEx ig0 = (IgniteEx)startGrids(initGridCnt);
+        final Ignite ig0 = startGrids(initGridCnt);
 
         ig0.active(true);
 

http://git-wip-us.apache.org/repos/asf/ignite/blob/44f3fac2/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/db/wal/reader/IgniteWalReaderTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/db/wal/reader/IgniteWalReaderTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/db/wal/reader/IgniteWalReaderTest.java
new file mode 100644
index 0000000..06bcf08
--- /dev/null
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/db/wal/reader/IgniteWalReaderTest.java
@@ -0,0 +1,385 @@
+/*
+ * 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.persistence.db.wal.reader;
+
+import java.io.File;
+import java.util.Arrays;
+import java.util.concurrent.CountDownLatch;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.atomic.AtomicBoolean;
+import org.apache.ignite.Ignite;
+import org.apache.ignite.IgniteCache;
+import org.apache.ignite.IgniteCheckedException;
+import org.apache.ignite.IgniteEvents;
+import org.apache.ignite.cache.CacheAtomicityMode;
+import org.apache.ignite.cache.CacheRebalanceMode;
+import org.apache.ignite.cache.affinity.rendezvous.RendezvousAffinityFunction;
+import org.apache.ignite.cache.query.annotations.QuerySqlField;
+import org.apache.ignite.configuration.BinaryConfiguration;
+import org.apache.ignite.configuration.CacheConfiguration;
+import org.apache.ignite.configuration.IgniteConfiguration;
+import org.apache.ignite.configuration.MemoryConfiguration;
+import org.apache.ignite.configuration.MemoryPolicyConfiguration;
+import org.apache.ignite.configuration.PersistentStoreConfiguration;
+import org.apache.ignite.configuration.WALMode;
+import org.apache.ignite.events.Event;
+import org.apache.ignite.events.EventType;
+import org.apache.ignite.internal.pagemem.wal.WALIterator;
+import org.apache.ignite.internal.pagemem.wal.WALPointer;
+import org.apache.ignite.internal.pagemem.wal.record.WALRecord;
+import org.apache.ignite.internal.processors.cache.persistence.wal.FileWriteAheadLogManager;
+import org.apache.ignite.events.WalSegmentArchivedEvent;
+import org.apache.ignite.internal.processors.cache.persistence.wal.reader.IgniteWalIteratorFactory;
+import org.apache.ignite.internal.util.typedef.internal.S;
+import org.apache.ignite.internal.util.typedef.internal.U;
+import org.apache.ignite.lang.IgniteBiTuple;
+import org.apache.ignite.lang.IgnitePredicate;
+import org.apache.ignite.testframework.junits.common.GridCommonAbstractTest;
+
+import static org.apache.ignite.events.EventType.EVT_WAL_SEGMENT_ARCHIVED;
+
+/**
+ * Test suite for WAL segments reader and event generator.
+ */
+public class IgniteWalReaderTest extends GridCommonAbstractTest {
+    /** Wal segments count */
+    private static final int WAL_SEGMENTS = 10;
+
+    /** Cache name. */
+    private static final String CACHE_NAME = "cache0";
+
+    /** Fill wal with some data before iterating. Should be true for non local run */
+    private static final boolean fillWalBeforeTest = true;
+
+    /** Delete DB dir before test. */
+    private static final boolean deleteBefore = true;
+
+    /** Delete DB dir after test. */
+    private static final boolean deleteAfter = true;
+
+    /** Dump records to logger. Should be false for non local run */
+    private static final boolean dumpRecords = false;
+
+    /** Page size to set */
+    public static final int PAGE_SIZE = 4 * 1024;
+
+    /**
+     * Field for transferring setting from test to getConfig method
+     * Archive incomplete segment after inactivity milliseconds.
+     */
+    private int archiveIncompleteSegmentAfterInactivityMs = 0;
+
+    /** {@inheritDoc} */
+    @Override protected IgniteConfiguration getConfiguration(String gridName) throws Exception {
+        final IgniteConfiguration cfg = super.getConfiguration(gridName);
+
+        final CacheConfiguration<Integer, IgniteWalReaderTest.IndexedObject> ccfg = new CacheConfiguration<>(CACHE_NAME);
+
+        ccfg.setAtomicityMode(CacheAtomicityMode.ATOMIC);
+        ccfg.setRebalanceMode(CacheRebalanceMode.SYNC);
+        ccfg.setAffinity(new RendezvousAffinityFunction(false, 32));
+        ccfg.setIndexedTypes(Integer.class, IgniteWalReaderTest.IndexedObject.class);
+
+        cfg.setCacheConfiguration(ccfg);
+
+        cfg.setIncludeEventTypes(EventType.EVT_WAL_SEGMENT_ARCHIVED);
+
+        final MemoryConfiguration dbCfg = new MemoryConfiguration();
+
+        dbCfg.setPageSize(PAGE_SIZE);
+
+        final MemoryPolicyConfiguration memPlcCfg = new MemoryPolicyConfiguration();
+
+        memPlcCfg.setName("dfltMemPlc");
+        memPlcCfg.setInitialSize(1024 * 1024 * 1024);
+        memPlcCfg.setMaxSize(1024 * 1024 * 1024);
+
+        dbCfg.setMemoryPolicies(memPlcCfg);
+        dbCfg.setDefaultMemoryPolicyName("dfltMemPlc");
+
+        cfg.setMemoryConfiguration(dbCfg);
+
+        final PersistentStoreConfiguration pCfg = new PersistentStoreConfiguration();
+        pCfg.setWalHistorySize(1);
+        pCfg.setWalSegmentSize(1024 * 1024);
+        pCfg.setWalSegments(WAL_SEGMENTS);
+        pCfg.setWalMode(WALMode.BACKGROUND);
+
+        if (archiveIncompleteSegmentAfterInactivityMs > 0)
+            pCfg.setWalAutoArchiveAfterInactivity(archiveIncompleteSegmentAfterInactivityMs);
+
+        cfg.setPersistentStoreConfiguration(pCfg);
+
+        final BinaryConfiguration binCfg = new BinaryConfiguration();
+
+        binCfg.setCompactFooter(false);
+
+        cfg.setBinaryConfiguration(binCfg);
+        return cfg;
+    }
+
+    /** {@inheritDoc} */
+    @Override protected void beforeTestsStarted() throws Exception {
+        stopAllGrids();
+
+        if (deleteBefore)
+            deleteWorkFiles();
+    }
+
+    /** {@inheritDoc} */
+    @Override protected void afterTest() throws Exception {
+        stopAllGrids();
+
+        if (deleteAfter)
+            deleteWorkFiles();
+    }
+
+    /**
+     * @throws IgniteCheckedException If failed.
+     */
+    private void deleteWorkFiles() throws IgniteCheckedException {
+        if (fillWalBeforeTest)
+            deleteRecursively(U.resolveWorkDirectory(U.defaultWorkDirectory(), "db", false));
+    }
+
+    /**
+     * @throws Exception if failed.
+     */
+    public void testFillWalAndReadRecords() throws Exception {
+        final int cacheObjectsToWrite = 10000;
+
+        if (fillWalBeforeTest) {
+            final Ignite ignite0 = startGrid("node0");
+
+            ignite0.active(true);
+
+            putDummyRecords(ignite0, cacheObjectsToWrite);
+
+            stopGrid("node0");
+        }
+
+        final File db = U.resolveWorkDirectory(U.defaultWorkDirectory(), "db", false);
+        final File wal = new File(db, "wal");
+        final File walArchive = new File(wal, "archive");
+        final String consistentId = "127_0_0_1_47500";
+        final MockWalIteratorFactory mockItFactory = new MockWalIteratorFactory(log, PAGE_SIZE, consistentId, WAL_SEGMENTS);
+        final WALIterator it = mockItFactory.iterator(wal, walArchive);
+        final int cntUsingMockIter = iterateAndCount(it);
+
+        log.info("Total records loaded " + cntUsingMockIter);
+        assert cntUsingMockIter > 0;
+        assert cntUsingMockIter > cacheObjectsToWrite;
+
+        final File walArchiveDirWithConsistentId = new File(walArchive, consistentId);
+        final File walWorkDirWithConsistentId = new File(wal, consistentId);
+
+        final IgniteWalIteratorFactory factory = new IgniteWalIteratorFactory(log, PAGE_SIZE);
+        final int cntArchiveDir = iterateAndCount(factory.iteratorArchiveDirectory(walArchiveDirWithConsistentId));
+
+        log.info("Total records loaded using directory : " + cntArchiveDir);
+
+        final int cntArchiveFileByFile = iterateAndCount(
+            factory.iteratorArchiveFiles(
+                walArchiveDirWithConsistentId.listFiles(FileWriteAheadLogManager.WAL_SEGMENT_FILE_FILTER)));
+
+        log.info("Total records loaded using archive directory (file-by-file): " + cntArchiveFileByFile);
+
+        assert cntArchiveFileByFile > cacheObjectsToWrite;
+        assert cntArchiveDir > cacheObjectsToWrite;
+        assert cntArchiveDir == cntArchiveFileByFile;
+        //really count2 may be less because work dir correct loading is not supported yet
+        assert cntUsingMockIter >= cntArchiveDir
+            : "Mock based reader loaded " + cntUsingMockIter + " records but standalone has loaded only " + cntArchiveDir;
+
+
+        final File[] workFiles = walWorkDirWithConsistentId.listFiles(FileWriteAheadLogManager.WAL_SEGMENT_FILE_FILTER);
+        int cntWork = 0;
+
+        try (WALIterator stIt = factory.iteratorWorkFiles(workFiles)) {
+            while (stIt.hasNextX()) {
+                IgniteBiTuple<WALPointer, WALRecord> next = stIt.nextX();
+                if (dumpRecords)
+                    log.info("Work. Record: " + next.get2());
+                cntWork++;
+            }
+        }
+        log.info("Total records loaded from work: " + cntWork);
+
+        assert cntWork + cntArchiveFileByFile == cntUsingMockIter
+            : "Work iterator loaded [" + cntWork + "] " +
+            "Archive iterator loaded [" + cntArchiveFileByFile + "]; " +
+            "mock iterator [" + cntUsingMockIter + "]";
+
+    }
+
+    /**
+     * @param walIter iterator to count, will be closed
+     * @return count of records
+     * @throws IgniteCheckedException if failed to iterate
+     */
+    private int iterateAndCount(WALIterator walIter) throws IgniteCheckedException {
+        int cntUsingMockIter = 0;
+
+        try(WALIterator it = walIter) {
+            while (it.hasNextX()) {
+                IgniteBiTuple<WALPointer, WALRecord> next = it.nextX();
+                if (dumpRecords)
+                    log.info("Record: " + next.get2());
+                cntUsingMockIter++;
+            }
+        }
+        return cntUsingMockIter;
+    }
+
+    /**
+     * Tests archive completed event is fired
+     *
+     * @throws Exception if failed
+     */
+    public void testArchiveCompletedEventFired() throws Exception {
+        final AtomicBoolean evtRecorded = new AtomicBoolean();
+
+        final Ignite ignite = startGrid("node0");
+
+        ignite.active(true);
+
+        final IgniteEvents evts = ignite.events();
+
+        if (!evts.isEnabled(EVT_WAL_SEGMENT_ARCHIVED))
+            return; //nothing to test
+
+        evts.localListen(new IgnitePredicate<Event>() {
+            @Override public boolean apply(Event e) {
+                WalSegmentArchivedEvent archComplEvt = (WalSegmentArchivedEvent)e;
+                long idx = archComplEvt.getAbsWalSegmentIdx();
+                log.info("Finished archive for segment [" + idx + ", " +
+                    archComplEvt.getArchiveFile() + "]: [" + e + "]");
+
+                evtRecorded.set(true);
+                return true;
+            }
+        }, EVT_WAL_SEGMENT_ARCHIVED);
+
+        putDummyRecords(ignite, 150);
+
+        stopGrid("node0");
+        assert evtRecorded.get();
+    }
+
+    /**
+     * Puts provided number of records to fill WAL
+     *
+     * @param ignite ignite instance
+     * @param recordsToWrite count
+     */
+    private void putDummyRecords(Ignite ignite, int recordsToWrite) {
+        IgniteCache<Object, Object> cache0 = ignite.cache(CACHE_NAME);
+
+        for (int i = 0; i < recordsToWrite; i++)
+            cache0.put(i, new IndexedObject(i));
+    }
+
+    /**
+     * Tests time out based WAL segment archiving
+     *
+     * @throws Exception if failure occurs
+     */
+    public void testArchiveIncompleteSegmentAfterInactivity() throws Exception {
+        final AtomicBoolean waitingForEvt = new AtomicBoolean();
+        final CountDownLatch archiveSegmentForInactivity = new CountDownLatch(1);
+
+        archiveIncompleteSegmentAfterInactivityMs = 1000;
+
+        final Ignite ignite = startGrid("node0");
+
+        ignite.active(true);
+
+        final IgniteEvents evts = ignite.events();
+
+        evts.localListen(new IgnitePredicate<Event>() {
+            @Override public boolean apply(Event e) {
+                WalSegmentArchivedEvent archComplEvt = (WalSegmentArchivedEvent)e;
+                long idx = archComplEvt.getAbsWalSegmentIdx();
+                log.info("Finished archive for segment [" + idx + ", " +
+                    archComplEvt.getArchiveFile() + "]: [" + e + "]");
+
+                if (waitingForEvt.get())
+                    archiveSegmentForInactivity.countDown();
+                return true;
+            }
+        }, EVT_WAL_SEGMENT_ARCHIVED);
+
+        putDummyRecords(ignite, 100);
+        waitingForEvt.set(true); //flag for skipping regular log() and rollOver()
+
+        log.info("Wait for archiving segment for inactive grid started");
+
+        boolean recordedAfterSleep =
+            archiveSegmentForInactivity.await(archiveIncompleteSegmentAfterInactivityMs + 1001, TimeUnit.MILLISECONDS);
+
+        stopGrid("node0");
+        assert recordedAfterSleep;
+    }
+
+    /** Test object for placing into grid in this test */
+    private static class IndexedObject {
+        /** */
+        @QuerySqlField(index = true)
+        private int iVal;
+
+        /** Data filled with recognizable pattern */
+        private byte[] data;
+
+        /**
+         * @param iVal Integer value.
+         */
+        private IndexedObject(int iVal) {
+            this.iVal = iVal;
+            int sz = 40000;
+            data = new byte[sz];
+            for (int i = 0; i < sz; i++)
+                data[i] = (byte)('A' + (i % 10));
+        }
+
+        /** {@inheritDoc} */
+        @Override public boolean equals(Object o) {
+            if (this == o)
+                return true;
+            if (o == null || getClass() != o.getClass())
+                return false;
+
+            IndexedObject obj = (IndexedObject)o;
+
+            if (iVal != obj.iVal)
+                return false;
+            return Arrays.equals(data, obj.data);
+        }
+
+        /** {@inheritDoc} */
+        @Override public int hashCode() {
+            int res = iVal;
+            res = 31 * res + Arrays.hashCode(data);
+            return res;
+        }
+
+        /** {@inheritDoc} */
+        @Override public String toString() {
+            return S.toString(IgniteWalReaderTest.IndexedObject.class, this);
+        }
+    }
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/44f3fac2/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/db/wal/reader/MockWalIteratorFactory.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/db/wal/reader/MockWalIteratorFactory.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/db/wal/reader/MockWalIteratorFactory.java
new file mode 100644
index 0000000..95079a0
--- /dev/null
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/db/wal/reader/MockWalIteratorFactory.java
@@ -0,0 +1,114 @@
+/*
+ * 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.persistence.db.wal.reader;
+
+import java.io.File;
+import org.apache.ignite.IgniteCheckedException;
+import org.apache.ignite.IgniteLogger;
+import org.apache.ignite.configuration.IgniteConfiguration;
+import org.apache.ignite.configuration.PersistentStoreConfiguration;
+import org.apache.ignite.internal.GridKernalContext;
+import org.apache.ignite.internal.managers.discovery.GridDiscoveryManager;
+import org.apache.ignite.internal.pagemem.wal.IgniteWriteAheadLogManager;
+import org.apache.ignite.internal.pagemem.wal.WALIterator;
+import org.apache.ignite.internal.processors.cache.GridCacheSharedContext;
+import org.apache.ignite.internal.processors.cache.persistence.GridCacheDatabaseSharedManager;
+import org.apache.ignite.internal.processors.cache.persistence.wal.FileWriteAheadLogManager;
+import org.jetbrains.annotations.Nullable;
+import org.mockito.Mockito;
+
+import static org.mockito.Matchers.any;
+import static org.mockito.Mockito.when;
+
+/**
+ * Mockito based WAL iterator provider
+ */
+public class MockWalIteratorFactory {
+    /** Logger. */
+    private final IgniteLogger log;
+
+    /** Page size. */
+    private final int pageSize;
+
+    /** Consistent node id. */
+    private final String consistentId;
+
+    /** Segments count in work dir. */
+    private int segments;
+
+    /**
+     * Creates factory
+     * @param log Logger.
+     * @param pageSize Page size.
+     * @param consistentId Consistent id.
+     * @param segments Segments.
+     */
+    public MockWalIteratorFactory(@Nullable IgniteLogger log, int pageSize, String consistentId, int segments) {
+        this.log = log == null ? Mockito.mock(IgniteLogger.class) : log;
+        this.pageSize = pageSize;
+        this.consistentId = consistentId;
+        this.segments = segments;
+    }
+
+    /**
+     * Creates iterator
+     * @param wal WAL directory without node id
+     * @param walArchive WAL archive without node id
+     * @return iterator
+     * @throws IgniteCheckedException if IO failed
+     */
+    public WALIterator iterator(File wal, File walArchive) throws IgniteCheckedException {
+        final PersistentStoreConfiguration persistentCfg1 = Mockito.mock(PersistentStoreConfiguration.class);
+
+        when(persistentCfg1.getWalStorePath()).thenReturn(wal.getAbsolutePath());
+        when(persistentCfg1.getWalArchivePath()).thenReturn(walArchive.getAbsolutePath());
+        when(persistentCfg1.getWalSegments()).thenReturn(segments);
+        when(persistentCfg1.getTlbSize()).thenReturn(PersistentStoreConfiguration.DFLT_TLB_SIZE);
+        when(persistentCfg1.getWalRecordIteratorBufferSize()).thenReturn(PersistentStoreConfiguration.DFLT_WAL_RECORD_ITERATOR_BUFFER_SIZE);
+
+        final IgniteConfiguration cfg = Mockito.mock(IgniteConfiguration.class);
+
+        when(cfg.getPersistentStoreConfiguration()).thenReturn(persistentCfg1);
+
+        final GridKernalContext ctx = Mockito.mock(GridKernalContext.class);
+
+        when(ctx.config()).thenReturn(cfg);
+        when(ctx.clientNode()).thenReturn(false);
+
+        final GridDiscoveryManager disco = Mockito.mock(GridDiscoveryManager.class);
+
+        when(disco.consistentId()).thenReturn(consistentId);
+        when(ctx.discovery()).thenReturn(disco);
+
+        final IgniteWriteAheadLogManager mgr = new FileWriteAheadLogManager(ctx);
+        final GridCacheSharedContext sctx = Mockito.mock(GridCacheSharedContext.class);
+
+        when(sctx.kernalContext()).thenReturn(ctx);
+        when(sctx.discovery()).thenReturn(disco);
+
+        final GridCacheDatabaseSharedManager database = Mockito.mock(GridCacheDatabaseSharedManager.class);
+
+        when(database.pageSize()).thenReturn(pageSize);
+        when(sctx.database()).thenReturn(database);
+        when(sctx.logger(any(Class.class))).thenReturn(log);
+
+        mgr.start(sctx);
+
+        return mgr.replay(null);
+    }
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/44f3fac2/modules/core/src/test/java/org/apache/ignite/testsuites/IgnitePdsTestSuite2.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/testsuites/IgnitePdsTestSuite2.java b/modules/core/src/test/java/org/apache/ignite/testsuites/IgnitePdsTestSuite2.java
index 351f52e..8018705 100644
--- a/modules/core/src/test/java/org/apache/ignite/testsuites/IgnitePdsTestSuite2.java
+++ b/modules/core/src/test/java/org/apache/ignite/testsuites/IgnitePdsTestSuite2.java
@@ -18,22 +18,18 @@
 package org.apache.ignite.testsuites;
 
 import junit.framework.TestSuite;
-import org.apache.ignite.internal.processors.cache.persistence.IgnitePdsAtomicCacheRebalancingTest;
-import org.apache.ignite.internal.processors.cache.persistence.IgnitePdsTxCacheRebalancingTest;
 import org.apache.ignite.internal.processors.cache.persistence.IgnitePdsContinuousRestartTest;
 import org.apache.ignite.internal.processors.cache.persistence.IgnitePdsPageSizesTest;
 import org.apache.ignite.internal.processors.cache.persistence.IgnitePdsRecoveryAfterFileCorruptionTest;
 import org.apache.ignite.internal.processors.cache.persistence.IgnitePersistenceMetricsSelfTest;
 import org.apache.ignite.internal.processors.cache.persistence.IgnitePersistentStoreDataStructuresTest;
 import org.apache.ignite.internal.processors.cache.persistence.db.IgnitePdsPageEvictionDuringPartitionClearTest;
+import org.apache.ignite.internal.processors.cache.persistence.db.IgnitePdsRebalancingOnNotStableTopologyTest;
 import org.apache.ignite.internal.processors.cache.persistence.db.IgnitePdsTransactionsHangTest;
 import org.apache.ignite.internal.processors.cache.persistence.db.IgnitePdsWholeClusterRestartTest;
-import org.apache.ignite.internal.processors.cache.persistence.db.IgnitePdsRebalancingOnNotStableTopologyTest;
-import org.apache.ignite.internal.processors.cache.persistence.db.file.IgnitePdsNoActualWalHistoryTest;
 import org.apache.ignite.internal.processors.cache.persistence.db.wal.IgniteWalHistoryReservationsTest;
-import org.apache.ignite.internal.processors.cache.persistence.db.wal.IgniteWalRecoveryTest;
-import org.apache.ignite.internal.processors.cache.persistence.db.wal.WalRecoveryTxLogicalRecordsTest;
 import org.apache.ignite.internal.processors.cache.persistence.db.wal.crc.IgniteDataIntegrityTests;
+import org.apache.ignite.internal.processors.cache.persistence.db.wal.reader.IgniteWalReaderTest;
 
 /**
  *
@@ -69,6 +65,7 @@ public class IgnitePdsTestSuite2 extends TestSuite {
 
         suite.addTestSuite(IgnitePersistentStoreDataStructuresTest.class);
 
+        suite.addTestSuite(IgniteWalReaderTest.class);
         return suite;
     }
 }


[03/33] ignite git commit: IGNITE-5613 - Fixed race on local sequence increment and distributed update

Posted by sb...@apache.org.
IGNITE-5613 - Fixed race on local sequence increment and distributed update


Project: http://git-wip-us.apache.org/repos/asf/ignite/repo
Commit: http://git-wip-us.apache.org/repos/asf/ignite/commit/c08849cd
Tree: http://git-wip-us.apache.org/repos/asf/ignite/tree/c08849cd
Diff: http://git-wip-us.apache.org/repos/asf/ignite/diff/c08849cd

Branch: refs/heads/ignite-2.1.2-exchange
Commit: c08849cdb362f1c699afb5d04383fa3200193539
Parents: 3cc13eae
Author: Alexey Goncharuk <al...@gmail.com>
Authored: Mon Jul 3 17:05:48 2017 +0300
Committer: Alexey Goncharuk <al...@gmail.com>
Committed: Mon Jul 3 17:06:38 2017 +0300

----------------------------------------------------------------------
 .../GridCacheAtomicSequenceImpl.java            | 55 ++++++++++++--------
 ...titionedAtomicSequenceMultiThreadedTest.java | 32 ++++++++++++
 2 files changed, 64 insertions(+), 23 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/c08849cd/modules/core/src/main/java/org/apache/ignite/internal/processors/datastructures/GridCacheAtomicSequenceImpl.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/datastructures/GridCacheAtomicSequenceImpl.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/datastructures/GridCacheAtomicSequenceImpl.java
index 31ec16f..0354a17 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/datastructures/GridCacheAtomicSequenceImpl.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/datastructures/GridCacheAtomicSequenceImpl.java
@@ -385,39 +385,48 @@ public final class GridCacheAtomicSequenceImpl implements GridCacheAtomicSequenc
 
                     long newUpBound;
 
-                    curLocVal = locVal;
+                    // Even though we hold a transaction lock here, we must hold the local update lock here as well
+                    // because we mutate multipe variables (locVal and upBound).
+                    localUpdate.lock();
 
-                    // If local range was already reserved in another thread.
-                    if (curLocVal + l <= upBound) {
-                        locVal = curLocVal + l;
+                    try {
+                        curLocVal = locVal;
 
-                        return updated ? curLocVal + l : curLocVal;
-                    }
+                        // If local range was already reserved in another thread.
+                        if (curLocVal + l <= upBound) {
+                            locVal = curLocVal + l;
 
-                    long curGlobalVal = seq.get();
+                            return updated ? curLocVal + l : curLocVal;
+                        }
 
-                    long newLocVal;
+                        long curGlobalVal = seq.get();
 
-                    /* We should use offset because we already reserved left side of range.*/
-                    long off = batchSize > 1 ? batchSize - 1 : 1;
+                        long newLocVal;
 
-                    // Calculate new values for local counter, global counter and upper bound.
-                    if (curLocVal + l >= curGlobalVal) {
-                        newLocVal = curLocVal + l;
+                        /* We should use offset because we already reserved left side of range.*/
+                        long off = batchSize > 1 ? batchSize - 1 : 1;
 
-                        newUpBound = newLocVal + off;
-                    }
-                    else {
-                        newLocVal = curGlobalVal;
+                        // Calculate new values for local counter, global counter and upper bound.
+                        if (curLocVal + l >= curGlobalVal) {
+                            newLocVal = curLocVal + l;
 
-                        newUpBound = newLocVal + off;
-                    }
+                            newUpBound = newLocVal + off;
+                        }
+                        else {
+                            newLocVal = curGlobalVal;
 
-                    locVal = newLocVal;
-                    upBound = newUpBound;
+                            newUpBound = newLocVal + off;
+                        }
 
-                    if (updated)
-                        curLocVal = newLocVal;
+                        locVal = newLocVal;
+                        upBound = newUpBound;
+
+                        if (updated)
+                            curLocVal = newLocVal;
+                    }
+                    finally {
+                        localUpdate.unlock();
+                    }
 
                     // Global counter must be more than reserved upper bound.
                     seq.set(newUpBound + 1);

http://git-wip-us.apache.org/repos/asf/ignite/blob/c08849cd/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/datastructures/partitioned/GridCachePartitionedAtomicSequenceMultiThreadedTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/datastructures/partitioned/GridCachePartitionedAtomicSequenceMultiThreadedTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/datastructures/partitioned/GridCachePartitionedAtomicSequenceMultiThreadedTest.java
index 945650d..4db9bd3 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/datastructures/partitioned/GridCachePartitionedAtomicSequenceMultiThreadedTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/datastructures/partitioned/GridCachePartitionedAtomicSequenceMultiThreadedTest.java
@@ -19,6 +19,7 @@ package org.apache.ignite.internal.processors.cache.datastructures.partitioned;
 
 import java.util.Random;
 import java.util.UUID;
+import java.util.concurrent.Callable;
 import org.apache.ignite.IgniteAtomicSequence;
 import org.apache.ignite.IgniteCheckedException;
 import org.apache.ignite.cache.CacheMode;
@@ -26,6 +27,7 @@ import org.apache.ignite.configuration.AtomicConfiguration;
 import org.apache.ignite.internal.processors.cache.datastructures.IgniteAtomicsAbstractTest;
 import org.apache.ignite.internal.processors.datastructures.GridCacheAtomicSequenceImpl;
 import org.apache.ignite.internal.util.typedef.internal.U;
+import org.apache.ignite.testframework.GridTestUtils;
 
 import static org.apache.ignite.cache.CacheMode.PARTITIONED;
 
@@ -281,6 +283,36 @@ public class GridCachePartitionedAtomicSequenceMultiThreadedTest extends IgniteA
     }
 
     /**
+     * @throws Exception if failed.
+     */
+    public void testMultipleSequences() throws Exception {
+        final int seqCnt = 5;
+        final int threadCnt = 5;
+        final int incCnt = 1_000;
+
+        final IgniteAtomicSequence[] seqs = new IgniteAtomicSequence[seqCnt];
+
+        String seqName = UUID.randomUUID().toString();
+
+        for (int i = 0; i < seqs.length; i++)
+            seqs[i] = grid(0).atomicSequence(seqName, 0, true);
+
+        GridTestUtils.runMultiThreaded(new Callable<Object>() {
+            @Override public Object call() throws Exception {
+                for (int i = 0; i < incCnt; i++) {
+                    for (IgniteAtomicSequence seq : seqs)
+                        seq.incrementAndGet();
+                }
+
+                return null;
+            }
+        }, threadCnt, "load");
+
+        for (IgniteAtomicSequence seq : seqs)
+            assertEquals(seqCnt * threadCnt * incCnt, seq.get());
+    }
+
+    /**
      * Executes given closure in a given number of threads given number of times.
      *
      * @param c Closure to execute.


[04/33] ignite git commit: IGNITE-5628 .NET: Fix jvm.dll lookup paths for JRE

Posted by sb...@apache.org.
IGNITE-5628 .NET: Fix jvm.dll lookup paths for JRE


Project: http://git-wip-us.apache.org/repos/asf/ignite/repo
Commit: http://git-wip-us.apache.org/repos/asf/ignite/commit/5093660d
Tree: http://git-wip-us.apache.org/repos/asf/ignite/tree/5093660d
Diff: http://git-wip-us.apache.org/repos/asf/ignite/diff/5093660d

Branch: refs/heads/ignite-2.1.2-exchange
Commit: 5093660d758ad4149d5cd135f3cad3dfee0ae6e4
Parents: c08849c
Author: Pavel Tupitsyn <pt...@apache.org>
Authored: Mon Jul 3 17:58:47 2017 +0300
Committer: Pavel Tupitsyn <pt...@apache.org>
Committed: Mon Jul 3 17:58:47 2017 +0300

----------------------------------------------------------------------
 .../dotnet/Apache.Ignite.Core/Impl/IgniteUtils.cs       | 12 +++++++++++-
 1 file changed, 11 insertions(+), 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/5093660d/modules/platforms/dotnet/Apache.Ignite.Core/Impl/IgniteUtils.cs
----------------------------------------------------------------------
diff --git a/modules/platforms/dotnet/Apache.Ignite.Core/Impl/IgniteUtils.cs b/modules/platforms/dotnet/Apache.Ignite.Core/Impl/IgniteUtils.cs
index b024345..f3bdd2b 100644
--- a/modules/platforms/dotnet/Apache.Ignite.Core/Impl/IgniteUtils.cs
+++ b/modules/platforms/dotnet/Apache.Ignite.Core/Impl/IgniteUtils.cs
@@ -46,7 +46,17 @@ namespace Apache.Ignite.Core.Impl
         private const string EnvJavaHome = "JAVA_HOME";
 
         /** Lookup paths. */
-        private static readonly string[] JvmDllLookupPaths = {@"jre\bin\server", @"jre\bin\default"};
+        private static readonly string[] JvmDllLookupPaths =
+        {
+            // JRE paths
+            @"bin\server",
+            @"bin\client",
+
+            // JDK paths
+            @"jre\bin\server",
+            @"jre\bin\client",
+            @"jre\bin\default"
+        };
 
         /** Registry lookup paths. */
         private static readonly string[] JreRegistryKeys =


[08/33] ignite git commit: ignite-2.1 Fixed minor issues and typos.

Posted by sb...@apache.org.
ignite-2.1 Fixed minor issues and typos.


Project: http://git-wip-us.apache.org/repos/asf/ignite/repo
Commit: http://git-wip-us.apache.org/repos/asf/ignite/commit/a5abdd57
Tree: http://git-wip-us.apache.org/repos/asf/ignite/tree/a5abdd57
Diff: http://git-wip-us.apache.org/repos/asf/ignite/diff/a5abdd57

Branch: refs/heads/ignite-2.1.2-exchange
Commit: a5abdd579cc7ace5d17ebcc6555e3ce66c777917
Parents: 15613e2
Author: Vasiliy Sisko <vs...@gridgain.com>
Authored: Tue Jul 4 18:44:42 2017 +0700
Committer: Alexey Kuznetsov <ak...@apache.org>
Committed: Tue Jul 4 18:44:42 2017 +0700

----------------------------------------------------------------------
 .../visor/cache/VisorCacheConfiguration.java    | 11 +----
 .../visor/node/VisorBasicConfiguration.java     |  2 +-
 .../node/VisorMemoryPolicyConfiguration.java    |  2 +-
 .../states/configuration/caches/affinity.pug    | 18 +++----
 .../states/configuration/caches/general.pug     | 50 ++++++++++----------
 5 files changed, 38 insertions(+), 45 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/a5abdd57/modules/core/src/main/java/org/apache/ignite/internal/visor/cache/VisorCacheConfiguration.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/visor/cache/VisorCacheConfiguration.java b/modules/core/src/main/java/org/apache/ignite/internal/visor/cache/VisorCacheConfiguration.java
index 7dba829..6fe056c 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/visor/cache/VisorCacheConfiguration.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/visor/cache/VisorCacheConfiguration.java
@@ -320,13 +320,6 @@ public class VisorCacheConfiguration extends VisorDataTransferObject {
     }
 
     /**
-     * @return Default lock acquisition timeout.
-     */
-    public long getDfltLockTimeout() {
-        return dfltLockTimeout;
-    }
-
-    /**
      * @return {@code true} if cache statistics collection enabled.
      */
     public boolean isStatisticsEnabled() {
@@ -455,7 +448,7 @@ public class VisorCacheConfiguration extends VisorDataTransferObject {
     /**
      * @return Listener configurations.
      */
-    public String getLsnrConfigurations() {
+    public String getListenerConfigurations() {
         return lsnrConfigurations;
     }
 
@@ -476,7 +469,7 @@ public class VisorCacheConfiguration extends VisorDataTransferObject {
     /**
      * @return Maximum payload size for offheap indexes.
      */
-    public int getSqlIdxMaxInlineSize() {
+    public int getSqlIndexMaxInlineSize() {
         return sqlIdxMaxInlineSize;
     }
 

http://git-wip-us.apache.org/repos/asf/ignite/blob/a5abdd57/modules/core/src/main/java/org/apache/ignite/internal/visor/node/VisorBasicConfiguration.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/visor/node/VisorBasicConfiguration.java b/modules/core/src/main/java/org/apache/ignite/internal/visor/node/VisorBasicConfiguration.java
index 56d000d..b32e814 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/visor/node/VisorBasicConfiguration.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/visor/node/VisorBasicConfiguration.java
@@ -195,7 +195,7 @@ public class VisorBasicConfiguration extends VisorDataTransferObject {
         addrRslvr = compactClass(c.getAddressResolver());
         cacheSanityCheckEnabled = c.isCacheSanityCheckEnabled();
         clsLdr = compactClass(c.getClassLoader());
-        consistentId = String.valueOf(c.getConsistentId());
+        consistentId = c.getConsistentId() != null ? String.valueOf(c.getConsistentId()) : null;
         failureDetectionTimeout = c.getFailureDetectionTimeout();
         igniteWorkDir = c.getWorkDirectory();
         lateAffAssignment = c.isLateAffinityAssignment();

http://git-wip-us.apache.org/repos/asf/ignite/blob/a5abdd57/modules/core/src/main/java/org/apache/ignite/internal/visor/node/VisorMemoryPolicyConfiguration.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/visor/node/VisorMemoryPolicyConfiguration.java b/modules/core/src/main/java/org/apache/ignite/internal/visor/node/VisorMemoryPolicyConfiguration.java
index ced3124..d3153a6 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/visor/node/VisorMemoryPolicyConfiguration.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/visor/node/VisorMemoryPolicyConfiguration.java
@@ -98,7 +98,7 @@ public class VisorMemoryPolicyConfiguration extends VisorDataTransferObject {
     /**
      * Initial memory region size defined by this memory policy.
      */
-    public long getInitSize() {
+    public long getInitialSize() {
         return initSize;
     }
 

http://git-wip-us.apache.org/repos/asf/ignite/blob/a5abdd57/modules/web-console/frontend/app/modules/states/configuration/caches/affinity.pug
----------------------------------------------------------------------
diff --git a/modules/web-console/frontend/app/modules/states/configuration/caches/affinity.pug b/modules/web-console/frontend/app/modules/states/configuration/caches/affinity.pug
index cb333ac..bc7674f 100644
--- a/modules/web-console/frontend/app/modules/states/configuration/caches/affinity.pug
+++ b/modules/web-console/frontend/app/modules/states/configuration/caches/affinity.pug
@@ -40,20 +40,20 @@ include /app/helpers/jade/mixins
             .col-sm-6
                 .settings-row(ng-if='$ctrl.available(["1.0.0", "2.0.0"])')
                     +dropdown('Function:', `${affModel}.kind`, '"AffinityKind"', 'true', 'Default', 'affinityFunction',
-                        'Key topology resolver to provide mapping from keys to nodes\
+                        'Key topology resolver to provide mapping from keys to nodes<br/>\
                         <ul>\
-                            <li>Rendezvous - Based on Highest Random Weight algorithm<br/></li>\
-                            <li>Fair - Tries to ensure that all nodes get equal number of partitions with minimum amount of reassignments between existing nodes<br/></li>\
-                            <li>Custom - Custom implementation of key affinity fynction<br/></li>\
-                            <li>Default - By default rendezvous affinity function  with 1024 partitions is used<br/></li>\
+                            <li>Rendezvous - Based on Highest Random Weight algorithm</li>\
+                            <li>Fair - Tries to ensure that all nodes get equal number of partitions with minimum amount of reassignments between existing nodes</li>\
+                            <li>Custom - Custom implementation of key affinity fynction</li>\
+                            <li>Default - By default rendezvous affinity function  with 1024 partitions is used</li>\
                         </ul>')
                 .settings-row(ng-if='$ctrl.available("2.0.0")')
                     +dropdown('Function:', `${affModel}.kind`, '"AffinityKind"', 'true', 'Default', 'affinityFunction',
-                        'Key topology resolver to provide mapping from keys to nodes\
+                        'Key topology resolver to provide mapping from keys to nodes<br/>\
                         <ul>\
-                            <li>Rendezvous - Based on Highest Random Weight algorithm<br/></li>\
-                            <li>Custom - Custom implementation of key affinity fynction<br/></li>\
-                            <li>Default - By default rendezvous affinity function  with 1024 partitions is used<br/></li>\
+                            <li>Rendezvous - Based on Highest Random Weight algorithm</li>\
+                            <li>Custom - Custom implementation of key affinity fynction</li>\
+                            <li>Default - By default rendezvous affinity function  with 1024 partitions is used</li>\
                         </ul>')
                 .panel-details(ng-if=rendezvousAff)
                     .details-row

http://git-wip-us.apache.org/repos/asf/ignite/blob/a5abdd57/modules/web-console/frontend/app/modules/states/configuration/caches/general.pug
----------------------------------------------------------------------
diff --git a/modules/web-console/frontend/app/modules/states/configuration/caches/general.pug b/modules/web-console/frontend/app/modules/states/configuration/caches/general.pug
index 33e62b8..87c67f5 100644
--- a/modules/web-console/frontend/app/modules/states/configuration/caches/general.pug
+++ b/modules/web-console/frontend/app/modules/states/configuration/caches/general.pug
@@ -82,31 +82,31 @@ include /app/helpers/jade/mixins
                         'Flag indicating whether data can be read from backup<br/>\
                         If not set then always get data from primary node (never from backup)')
                 //- Since ignite 2.0
-                    .settings-row(ng-if='$ctrl.available("2.0.0")')
-                        +dropdown('Partition loss policy:', `${model}.partitionLossPolicy`, '"partitionLossPolicy"', 'true', 'IGNORE',
-                        '[\
-                            {value: "READ_ONLY_SAFE", label: "READ_ONLY_SAFE"},\
-                            {value: "READ_ONLY_ALL", label: "READ_ONLY_ALL"},\
-                            {value: "READ_WRITE_SAFE", label: "READ_WRITE_SAFE"},\
-                            {value: "READ_WRITE_ALL", label: "READ_WRITE_ALL"},\
-                            {value: "IGNORE", label: "IGNORE"}\
-                        ]',
-                        'Partition loss policies:\
-                        <ul>\
-                            <li>READ_ONLY_SAFE - in this mode all writes to the cache will be failed with an exception,\
-                                reads will only be allowed for keys in  non-lost partitions.\
-                                Reads from lost partitions will be failed with an exception.</li>\
-                            <li>READ_ONLY_ALL - in this mode Ñ„ll writes to the cache will be failed with an exception.\
-                                All reads will proceed as if all partitions were in a consistent state.\
-                                The result of reading from a lost partition is undefined and may be different on different nodes in the cluster.</li>\
-                            <li>READ_WRITE_SAFE - in this mode Aall reads and writes will be allowed for keys in valid partitions.\
-                                All reads and writes for keys in lost partitions will be failed with an exception.</li>\
-                            <li>READ_WRITE_ALL - in this mode all reads and writes will proceed as if all partitions were in a consistent state.\
-                                The result of reading from a lost partition is undefined and may be different on different nodes in the cluster.</li>\
-                            <li>IGNORE - in this mode if partition is lost, reset it state and do not clear intermediate data.\
-                                The result of reading from a previously lost and not cleared partition is undefined and may be different\
-                                on different nodes in the cluster.</li>\
-                        </ul>')
+                .settings-row(ng-if='$ctrl.available("2.0.0")')
+                    +dropdown('Partition loss policy:', `${model}.partitionLossPolicy`, '"partitionLossPolicy"', 'true', 'IGNORE',
+                    '[\
+                        {value: "READ_ONLY_SAFE", label: "READ_ONLY_SAFE"},\
+                        {value: "READ_ONLY_ALL", label: "READ_ONLY_ALL"},\
+                        {value: "READ_WRITE_SAFE", label: "READ_WRITE_SAFE"},\
+                        {value: "READ_WRITE_ALL", label: "READ_WRITE_ALL"},\
+                        {value: "IGNORE", label: "IGNORE"}\
+                    ]',
+                    'Partition loss policies:\
+                    <ul>\
+                        <li>READ_ONLY_SAFE - in this mode all writes to the cache will be failed with an exception,\
+                            reads will only be allowed for keys in  non-lost partitions.\
+                            Reads from lost partitions will be failed with an exception.</li>\
+                        <li>READ_ONLY_ALL - in this mode Ñ„ll writes to the cache will be failed with an exception.\
+                            All reads will proceed as if all partitions were in a consistent state.\
+                            The result of reading from a lost partition is undefined and may be different on different nodes in the cluster.</li>\
+                        <li>READ_WRITE_SAFE - in this mode Aall reads and writes will be allowed for keys in valid partitions.\
+                            All reads and writes for keys in lost partitions will be failed with an exception.</li>\
+                        <li>READ_WRITE_ALL - in this mode all reads and writes will proceed as if all partitions were in a consistent state.\
+                            The result of reading from a lost partition is undefined and may be different on different nodes in the cluster.</li>\
+                        <li>IGNORE - in this mode if partition is lost, reset it state and do not clear intermediate data.\
+                            The result of reading from a previously lost and not cleared partition is undefined and may be different\
+                            on different nodes in the cluster.</li>\
+                    </ul>')
                 .settings-row
                     +checkbox('Copy on read', `${model}.copyOnRead`, '"copyOnRead"',
                         'Flag indicating whether copy of the value stored in cache should be created for cache operation implying return value<br/>\


[22/33] ignite git commit: IGNITE-5340: Improved IgniteCacheAbstractQuerySelfTest to better test non-standard index names. This closes #2206.

Posted by sb...@apache.org.
IGNITE-5340: Improved IgniteCacheAbstractQuerySelfTest to better test non-standard index names. This closes #2206.


Project: http://git-wip-us.apache.org/repos/asf/ignite/repo
Commit: http://git-wip-us.apache.org/repos/asf/ignite/commit/54572c30
Tree: http://git-wip-us.apache.org/repos/asf/ignite/tree/54572c30
Diff: http://git-wip-us.apache.org/repos/asf/ignite/diff/54572c30

Branch: refs/heads/ignite-2.1.2-exchange
Commit: 54572c3023dc03a55621fbb4754888b081791e31
Parents: 5b7165c
Author: tledkov-gridgain <tl...@gridgain.com>
Authored: Wed Jul 5 10:30:48 2017 +0300
Committer: devozerov <vo...@gridgain.com>
Committed: Wed Jul 5 10:30:48 2017 +0300

----------------------------------------------------------------------
 .../processors/cache/IgniteCacheAbstractQuerySelfTest.java        | 3 ++-
 1 file changed, 2 insertions(+), 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/54572c30/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/IgniteCacheAbstractQuerySelfTest.java
----------------------------------------------------------------------
diff --git a/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/IgniteCacheAbstractQuerySelfTest.java b/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/IgniteCacheAbstractQuerySelfTest.java
index f3dbbb1..e58f983 100644
--- a/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/IgniteCacheAbstractQuerySelfTest.java
+++ b/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/IgniteCacheAbstractQuerySelfTest.java
@@ -179,7 +179,8 @@ public abstract class IgniteCacheAbstractQuerySelfTest extends GridCommonAbstrac
         qryEntity.addQueryField("id", Integer.class.getName(), null);
         qryEntity.addQueryField("name", String.class.getName(), null);
         qryEntity.setTableName("Type2");
-        qryEntity.setIndexes(Arrays.asList(new QueryIndex("id")));
+        qryEntity.setIndexes(Arrays.asList(new QueryIndex("id")
+            .setName("index~!@#$%^&*()_=-+;[]{}|?,.*`:nameWithNonLetterSymbols")));
 
         entityList.add(qryEntity);
 


[07/33] ignite git commit: Merge master into ignite-2.1.2

Posted by sb...@apache.org.
Merge master into ignite-2.1.2


Project: http://git-wip-us.apache.org/repos/asf/ignite/repo
Commit: http://git-wip-us.apache.org/repos/asf/ignite/commit/15613e2a
Tree: http://git-wip-us.apache.org/repos/asf/ignite/tree/15613e2a
Diff: http://git-wip-us.apache.org/repos/asf/ignite/diff/15613e2a

Branch: refs/heads/ignite-2.1.2-exchange
Commit: 15613e2af5e0a4a0014bb5c6d6f6915038b1be1a
Parents: d846197 5093660
Author: Alexey Goncharuk <al...@gmail.com>
Authored: Tue Jul 4 12:39:38 2017 +0300
Committer: Alexey Goncharuk <al...@gmail.com>
Committed: Tue Jul 4 12:39:38 2017 +0300

----------------------------------------------------------------------
 .../checkpoint/GridCheckpointManager.java       |  2 +-
 .../managers/communication/GridIoManager.java   | 10 +--
 .../communication/GridMessageListener.java      |  3 +-
 .../deployment/GridDeploymentCommunication.java |  4 +-
 .../eventstorage/GridEventStorageManager.java   |  4 +-
 .../processors/cache/GridCacheIoManager.java    | 85 +++++++++++---------
 .../GridCachePartitionExchangeManager.java      | 16 ++--
 .../cache/binary/BinaryMetadataTransport.java   |  4 +-
 .../GridDhtPartitionsExchangeFuture.java        | 11 +--
 .../cache/transactions/IgniteTxManager.java     |  2 +-
 .../processors/cluster/ClusterProcessor.java    |  2 +-
 .../continuous/GridContinuousProcessor.java     |  4 +-
 .../datastreamer/DataStreamProcessor.java       |  2 +-
 .../datastreamer/DataStreamerImpl.java          |  2 +-
 .../processors/igfs/IgfsDataManager.java        |  2 +-
 .../igfs/IgfsFragmentizerManager.java           |  4 +-
 .../processors/job/GridJobProcessor.java        |  8 +-
 .../GridMarshallerMappingProcessor.java         |  4 +-
 .../processors/query/GridQueryProcessor.java    |  2 +-
 .../handlers/task/GridTaskCommandHandler.java   |  4 +-
 .../processors/task/GridTaskProcessor.java      |  6 +-
 .../jobstealing/JobStealingCollisionSpi.java    |  2 +-
 ...idCommunicationManagerListenersSelfTest.java |  2 +-
 .../GridCommunicationSendMessageSelfTest.java   |  2 +-
 .../cache/GridCachePartitionedGetSelfTest.java  |  2 +-
 ...lerCacheClientRequestsMappingOnMissTest.java |  6 +-
 ...naryObjectMetadataExchangeMultinodeTest.java |  6 +-
 ...DeadlockDetectionMessageMarshallingTest.java |  2 +-
 .../communication/GridIoManagerBenchmark.java   |  4 +-
 .../communication/GridIoManagerBenchmark0.java  | 12 +--
 .../communication/GridCacheMessageSelfTest.java |  2 +-
 .../testframework/GridSpiTestContext.java       |  5 +-
 .../hadoop/shuffle/HadoopShuffle.java           |  2 +-
 .../query/h2/opt/GridH2IndexBase.java           |  2 +-
 .../query/h2/twostep/GridMapQueryExecutor.java  |  2 +-
 .../h2/twostep/GridReduceQueryExecutor.java     |  2 +-
 .../Apache.Ignite.Core/Impl/IgniteUtils.cs      | 12 ++-
 37 files changed, 129 insertions(+), 117 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/15613e2a/modules/core/src/main/java/org/apache/ignite/internal/managers/checkpoint/GridCheckpointManager.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/ignite/blob/15613e2a/modules/core/src/main/java/org/apache/ignite/internal/managers/communication/GridIoManager.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/ignite/blob/15613e2a/modules/core/src/main/java/org/apache/ignite/internal/managers/eventstorage/GridEventStorageManager.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/ignite/blob/15613e2a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheIoManager.java
----------------------------------------------------------------------
diff --cc modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheIoManager.java
index 44eedb1,49cfcdd..2de3808
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheIoManager.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheIoManager.java
@@@ -19,7 -19,7 +19,6 @@@ package org.apache.ignite.internal.proc
  
  import java.util.ArrayList;
  import java.util.Arrays;
--import java.util.Collection;
  import java.util.HashMap;
  import java.util.Iterator;
  import java.util.List;
@@@ -80,17 -80,17 +79,12 @@@ import org.apache.ignite.internal.proce
  import org.apache.ignite.internal.processors.cache.transactions.IgniteTxState;
  import org.apache.ignite.internal.processors.cache.transactions.IgniteTxStateAware;
  import org.apache.ignite.internal.processors.cache.version.GridCacheVersion;
--import org.apache.ignite.internal.util.F0;
--import org.apache.ignite.internal.util.GridLeanSet;
  import org.apache.ignite.internal.util.StripedCompositeReadWriteLock;
  import org.apache.ignite.internal.util.typedef.CI1;
--import org.apache.ignite.internal.util.typedef.F;
--import org.apache.ignite.internal.util.typedef.P1;
  import org.apache.ignite.internal.util.typedef.X;
  import org.apache.ignite.internal.util.typedef.internal.CU;
  import org.apache.ignite.internal.util.typedef.internal.U;
  import org.apache.ignite.lang.IgniteBiInClosure;
--import org.apache.ignite.lang.IgnitePredicate;
  import org.apache.ignite.lang.IgniteUuid;
  import org.apache.ignite.thread.IgniteThread;
  import org.jetbrains.annotations.Nullable;
@@@ -355,19 -358,9 +357,19 @@@ public class GridCacheIoManager extend
                      if (log.isDebugEnabled())
                          log.debug(msg0.toString());
                  }
 -                else
 +                else {
                      U.error(log, msg0.toString());
  
 +                    try {
 +                        cacheMsg.onClassError(new IgniteCheckedException("Failed to find message handler for message: " + cacheMsg));
 +
-                         processFailedMessage(nodeId, cacheMsg, c);
++                        processFailedMessage(nodeId, cacheMsg, c, plc);
 +                    }
 +                    catch (Exception e) {
 +                        U.error(log, "Failed to process failed message: " + e, e);
 +                    }
 +                }
 +
                  return;
              }
  

http://git-wip-us.apache.org/repos/asf/ignite/blob/15613e2a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCachePartitionExchangeManager.java
----------------------------------------------------------------------
diff --cc modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCachePartitionExchangeManager.java
index f07119a,2d1aca0..93310e3
--- 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
@@@ -863,7 -860,7 +863,7 @@@ public class GridCachePartitionExchange
       * @param nodes Nodes.
       */
      private void sendAllPartitions(Collection<ClusterNode> nodes) {
-         GridDhtPartitionsFullMessage m = createPartitionsFullMessage(nodes, null, null, null, null, true);
 -        GridDhtPartitionsFullMessage m = createPartitionsFullMessage(null, null);
++        GridDhtPartitionsFullMessage m = createPartitionsFullMessage(null, null, null, null);
  
          if (log.isDebugEnabled())
              log.debug("Sending all partitions [nodeIds=" + U.nodeIds(nodes) + ", msg=" + m + ']');
@@@ -886,26 -883,18 +886,24 @@@
      }
  
      /**
-      * @param nodes Target nodes.
       * @param exchId Non-null exchange ID if message is created for exchange.
       * @param lastVer Last version.
-      * @param compress {@code True} if it is possible to use compression for message.
       * @return Message.
       */
-     public GridDhtPartitionsFullMessage createPartitionsFullMessage(Collection<ClusterNode> nodes,
+     public GridDhtPartitionsFullMessage createPartitionsFullMessage(
 -        final @Nullable GridDhtPartitionExchangeId exchId,
 -        @Nullable GridCacheVersion lastVer) {
 +        @Nullable final GridDhtPartitionExchangeId exchId,
 +        @Nullable GridCacheVersion lastVer,
 +        @Nullable IgniteDhtPartitionHistorySuppliersMap partHistSuppliers,
-         @Nullable IgniteDhtPartitionsToReloadMap partsToReload,
-         final boolean compress) {
++        @Nullable IgniteDhtPartitionsToReloadMap partsToReload
++    ) {
          final GridDhtPartitionsFullMessage m = new GridDhtPartitionsFullMessage(exchId,
              lastVer,
 -            exchId != null ? exchId.topologyVersion() : AffinityTopologyVersion.NONE);
 +            exchId != null ? exchId.topologyVersion() : AffinityTopologyVersion.NONE,
 +            partHistSuppliers,
 +            partsToReload
 +            );
  
-         m.compress(compress);
+         m.compress(true);
  
          final Map<Object, T2<Integer, GridDhtPartitionFullMap>> dupData = new HashMap<>();
  

http://git-wip-us.apache.org/repos/asf/ignite/blob/15613e2a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/binary/BinaryMetadataTransport.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/ignite/blob/15613e2a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/preloader/GridDhtPartitionsExchangeFuture.java
----------------------------------------------------------------------
diff --cc modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/preloader/GridDhtPartitionsExchangeFuture.java
index 42f60b1,7471855..a1926ee
--- 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
@@@ -1156,12 -1089,8 +1155,10 @@@ public class GridDhtPartitionsExchangeF
          GridCacheVersion last = lastVer.get();
  
          GridDhtPartitionsFullMessage m = cctx.exchange().createPartitionsFullMessage(
-             nodes,
              exchangeId(),
 -            last != null ? last : cctx.versions().last());
 +            last != null ? last : cctx.versions().last(),
 +            partHistSuppliers,
-             partsToReload,
-             compress);
++            partsToReload);
  
          if (exchangeOnChangeGlobalState && !F.isEmpty(changeGlobalStateExceptions))
              m.setExceptionsMap(changeGlobalStateExceptions);

http://git-wip-us.apache.org/repos/asf/ignite/blob/15613e2a/modules/core/src/main/java/org/apache/ignite/internal/processors/cluster/ClusterProcessor.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/ignite/blob/15613e2a/modules/core/src/main/java/org/apache/ignite/internal/processors/continuous/GridContinuousProcessor.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/ignite/blob/15613e2a/modules/core/src/main/java/org/apache/ignite/internal/processors/datastreamer/DataStreamProcessor.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/ignite/blob/15613e2a/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/IgfsDataManager.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/ignite/blob/15613e2a/modules/core/src/main/java/org/apache/ignite/internal/processors/job/GridJobProcessor.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/ignite/blob/15613e2a/modules/core/src/main/java/org/apache/ignite/internal/processors/marshaller/GridMarshallerMappingProcessor.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/ignite/blob/15613e2a/modules/core/src/main/java/org/apache/ignite/internal/processors/query/GridQueryProcessor.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/ignite/blob/15613e2a/modules/core/src/main/java/org/apache/ignite/internal/processors/task/GridTaskProcessor.java
----------------------------------------------------------------------


[16/33] ignite git commit: Minor code cleanup

Posted by sb...@apache.org.
Minor code cleanup


Project: http://git-wip-us.apache.org/repos/asf/ignite/repo
Commit: http://git-wip-us.apache.org/repos/asf/ignite/commit/ee7566b1
Tree: http://git-wip-us.apache.org/repos/asf/ignite/tree/ee7566b1
Diff: http://git-wip-us.apache.org/repos/asf/ignite/diff/ee7566b1

Branch: refs/heads/ignite-2.1.2-exchange
Commit: ee7566b1063de69ca4f261ee2088d69d231b1c2a
Parents: 77c5dc7
Author: Dmitriy Govorukhin <dm...@gmail.com>
Authored: Tue Jul 4 20:00:09 2017 +0300
Committer: Alexey Goncharuk <al...@gmail.com>
Committed: Tue Jul 4 20:00:09 2017 +0300

----------------------------------------------------------------------
 .../processors/cluster/GridClusterStateProcessor.java         | 7 -------
 1 file changed, 7 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/ee7566b1/modules/core/src/main/java/org/apache/ignite/internal/processors/cluster/GridClusterStateProcessor.java
----------------------------------------------------------------------
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 031c596..d57c720 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
@@ -725,13 +725,6 @@ public class GridClusterStateProcessor extends GridProcessorAdapter {
             log.info("Start activation process [nodeId=" + ctx.localNodeId() + ", client=" + client +
                 ", topVer=" + cgsCtx.topVer + "]");
 
-        Collection<StoredCacheData> cfgs = new ArrayList<>();
-
-        for (DynamicCacheChangeRequest req : cgsCtx.batch.requests()) {
-            if (req.startCacheConfiguration() != null)
-                cfgs.add(new StoredCacheData(req.startCacheConfiguration()));
-        }
-
         try {
             if (!client)
                 sharedCtx.database().lock();


[31/33] ignite git commit: IGNITE-5605 .NET: Inject resources into remote event filters

Posted by sb...@apache.org.
IGNITE-5605 .NET: Inject resources into remote event filters


Project: http://git-wip-us.apache.org/repos/asf/ignite/repo
Commit: http://git-wip-us.apache.org/repos/asf/ignite/commit/f9f13cf0
Tree: http://git-wip-us.apache.org/repos/asf/ignite/tree/f9f13cf0
Diff: http://git-wip-us.apache.org/repos/asf/ignite/diff/f9f13cf0

Branch: refs/heads/ignite-2.1.2-exchange
Commit: f9f13cf083b481f004531710ec3835afdf5b7cef
Parents: b67b8c4
Author: Pavel Tupitsyn <pt...@apache.org>
Authored: Wed Jul 5 12:59:43 2017 +0300
Committer: Pavel Tupitsyn <pt...@apache.org>
Committed: Wed Jul 5 12:59:43 2017 +0300

----------------------------------------------------------------------
 .../Apache.Ignite.Core.Tests/EventsTest.cs       | 19 ++++++++++++++-----
 .../Impl/Events/RemoteListenEventFilter.cs       |  3 +++
 2 files changed, 17 insertions(+), 5 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/f9f13cf0/modules/platforms/dotnet/Apache.Ignite.Core.Tests/EventsTest.cs
----------------------------------------------------------------------
diff --git a/modules/platforms/dotnet/Apache.Ignite.Core.Tests/EventsTest.cs b/modules/platforms/dotnet/Apache.Ignite.Core.Tests/EventsTest.cs
index 7578475..c05511c 100644
--- a/modules/platforms/dotnet/Apache.Ignite.Core.Tests/EventsTest.cs
+++ b/modules/platforms/dotnet/Apache.Ignite.Core.Tests/EventsTest.cs
@@ -17,6 +17,8 @@
 
 // ReSharper disable MemberCanBePrivate.Global
 // ReSharper disable UnusedParameter.Global
+// ReSharper disable UnusedAutoPropertyAccessor.Local
+// ReSharper disable UnusedAutoPropertyAccessor.Global
 #pragma warning disable 618
 namespace Apache.Ignite.Core.Tests
 {
@@ -33,6 +35,7 @@ namespace Apache.Ignite.Core.Tests
     using Apache.Ignite.Core.Events;
     using Apache.Ignite.Core.Impl;
     using Apache.Ignite.Core.Impl.Events;
+    using Apache.Ignite.Core.Resource;
     using Apache.Ignite.Core.Tests.Compute;
     using NUnit.Framework;
 
@@ -360,14 +363,14 @@ namespace Apache.Ignite.Core.Tests
                 if (i > 3)
                 {
                     // Filter
-                    waitTask = getWaitTask(new EventFilter<IEvent>(e => e.Type == EventType.TaskReduced), new int[0]);
+                    waitTask = getWaitTask(new LocalEventFilter<IEvent>(e => e.Type == EventType.TaskReduced), new int[0]);
 
                     Assert.IsTrue(waitTask.Wait(timeout));
                     Assert.IsInstanceOf(typeof(TaskEvent), waitTask.Result);
                     Assert.AreEqual(EventType.TaskReduced, waitTask.Result.Type);
 
                     // Filter & types
-                    waitTask = getWaitTask(new EventFilter<IEvent>(e => e.Type == EventType.TaskReduced),
+                    waitTask = getWaitTask(new LocalEventFilter<IEvent>(e => e.Type == EventType.TaskReduced),
                         new[] {EventType.TaskReduced});
 
                     Assert.IsTrue(waitTask.Wait(timeout));
@@ -868,7 +871,7 @@ namespace Apache.Ignite.Core.Tests
         /// <returns>New instance of event listener.</returns>
         public static IEventListener<IEvent> GetListener()
         {
-            return new EventFilter<IEvent>(Listen);
+            return new LocalEventFilter<IEvent>(Listen);
         }
 
         /// <summary>
@@ -917,7 +920,7 @@ namespace Apache.Ignite.Core.Tests
     /// Test event filter.
     /// </summary>
     [Serializable]
-    public class EventFilter<T> : IEventFilter<T>, IEventListener<T> where T : IEvent
+    public class LocalEventFilter<T> : IEventFilter<T>, IEventListener<T> where T : IEvent
     {
         /** */
         private readonly Func<T, bool> _invoke;
@@ -926,7 +929,7 @@ namespace Apache.Ignite.Core.Tests
         /// Initializes a new instance of the <see cref="RemoteListenEventFilter"/> class.
         /// </summary>
         /// <param name="invoke">The invoke delegate.</param>
-        public EventFilter(Func<T, bool> invoke)
+        public LocalEventFilter(Func<T, bool> invoke)
         {
             _invoke = invoke;
         }
@@ -960,6 +963,10 @@ namespace Apache.Ignite.Core.Tests
         /** */
         private readonly int _type;
 
+        /** */
+        [InstanceResource]
+        public IIgnite Ignite { get; set; }
+
         public RemoteEventFilter(int type)
         {
             _type = type;
@@ -968,6 +975,8 @@ namespace Apache.Ignite.Core.Tests
         /** <inheritdoc /> */
         public bool Invoke(IEvent evt)
         {
+            Assert.IsNotNull(Ignite);
+
             return evt.Type == _type;
         }
     }

http://git-wip-us.apache.org/repos/asf/ignite/blob/f9f13cf0/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Events/RemoteListenEventFilter.cs
----------------------------------------------------------------------
diff --git a/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Events/RemoteListenEventFilter.cs b/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Events/RemoteListenEventFilter.cs
index 31bfff1..2e0b66c 100644
--- a/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Events/RemoteListenEventFilter.cs
+++ b/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Events/RemoteListenEventFilter.cs
@@ -22,6 +22,7 @@ namespace Apache.Ignite.Core.Impl.Events
     using Apache.Ignite.Core.Events;
     using Apache.Ignite.Core.Impl.Binary.IO;
     using Apache.Ignite.Core.Impl.Common;
+    using Apache.Ignite.Core.Impl.Resource;
 
     /// <summary>
     /// Event filter/listener holder for RemoteListen.
@@ -75,6 +76,8 @@ namespace Apache.Ignite.Core.Impl.Events
 
                 var pred = reader.ReadObject<object>();
 
+                ResourceProcessor.Inject(pred, grid);
+
                 var func = DelegateTypeDescriptor.GetEventFilter(pred.GetType());
 
                 return new RemoteListenEventFilter(grid, evt => func(pred, evt));


[29/33] ignite git commit: Reworked cluster activation/deactivation.

Posted by sb...@apache.org.
Reworked cluster activation/deactivation.


Project: http://git-wip-us.apache.org/repos/asf/ignite/repo
Commit: http://git-wip-us.apache.org/repos/asf/ignite/commit/1337901f
Tree: http://git-wip-us.apache.org/repos/asf/ignite/tree/1337901f
Diff: http://git-wip-us.apache.org/repos/asf/ignite/diff/1337901f

Branch: refs/heads/ignite-2.1.2-exchange
Commit: 1337901f04c866e20093b20449c0872f089fb64b
Parents: 54572c3
Author: sboikov <sb...@gridgain.com>
Authored: Wed Jul 5 11:19:43 2017 +0300
Committer: sboikov <sb...@gridgain.com>
Committed: Wed Jul 5 11:19:43 2017 +0300

----------------------------------------------------------------------
 .../apache/ignite/internal/GridComponent.java   |    4 +-
 .../ignite/internal/GridPluginComponent.java    |    2 +-
 .../apache/ignite/internal/IgniteKernal.java    |   33 +-
 .../internal/managers/GridManagerAdapter.java   |    2 +-
 .../internal/managers/discovery/DiscoCache.java |   17 +-
 .../discovery/DiscoveryLocalJoinData.java       |  104 ++
 .../discovery/GridDiscoveryManager.java         |  128 +-
 .../pagemem/store/IgnitePageStoreManager.java   |    3 +-
 .../processors/GridProcessorAdapter.java        |    2 +-
 .../cache/CacheAffinitySharedManager.java       |   67 +-
 .../processors/cache/CacheGroupContext.java     |    4 +-
 .../processors/cache/CacheGroupData.java        |    4 +-
 .../cache/ChangeGlobalStateMessage.java         |  120 --
 .../processors/cache/ClusterCachesInfo.java     |  490 +++++--
 .../internal/processors/cache/ClusterState.java |   38 -
 .../cache/DynamicCacheChangeRequest.java        |   52 +-
 .../processors/cache/ExchangeActions.java       |   37 +-
 .../processors/cache/GridCacheEventManager.java |    2 -
 .../cache/GridCacheEvictionManager.java         |    1 -
 .../processors/cache/GridCacheIoManager.java    |   13 +-
 .../processors/cache/GridCacheMvccManager.java  |    9 +-
 .../GridCachePartitionExchangeManager.java      |  423 +++---
 .../processors/cache/GridCacheProcessor.java    |  177 ++-
 .../cache/GridCacheSharedContext.java           |   60 +-
 .../cache/GridCacheSharedManager.java           |    6 -
 .../cache/GridCacheSharedManagerAdapter.java    |   16 -
 .../processors/cache/PendingDiscoveryEvent.java |   61 +
 .../processors/cache/StateChangeRequest.java    |   77 ++
 .../binary/CacheObjectBinaryProcessorImpl.java  |    4 +-
 .../distributed/GridCacheTxRecoveryFuture.java  |    1 -
 .../distributed/dht/GridDhtCacheAdapter.java    |    1 -
 .../cache/distributed/dht/GridDhtGetFuture.java |    1 -
 .../distributed/dht/GridDhtGetSingleFuture.java |    2 -
 .../dht/GridDhtPartitionTopologyImpl.java       |   13 +-
 .../dht/GridDhtTopologyFutureAdapter.java       |    2 +-
 .../dht/GridPartitionedSingleGetFuture.java     |    3 -
 .../GridNearAtomicAbstractUpdateFuture.java     |    1 -
 .../dht/preloader/GridDhtForceKeysFuture.java   |    1 -
 .../dht/preloader/GridDhtPartitionDemander.java |    2 +
 .../GridDhtPartitionsExchangeFuture.java        |  228 +++-
 .../preloader/GridDhtPartitionsFullMessage.java |   44 +-
 .../GridDhtPartitionsSingleMessage.java         |   38 +-
 .../dht/preloader/GridDhtPreloader.java         |    2 +-
 .../distributed/near/GridNearGetFuture.java     |    2 -
 .../near/GridNearTxPrepareRequest.java          |    1 -
 .../GridCacheDatabaseSharedManager.java         |  105 +-
 .../persistence/GridCacheOffheapManager.java    |    5 +-
 .../IgniteCacheDatabaseSharedManager.java       |   64 +-
 .../persistence/IgniteCacheSnapshotManager.java |   12 +-
 .../persistence/file/FilePageStoreManager.java  |   14 +-
 .../wal/FileWriteAheadLogManager.java           |    8 -
 .../query/GridCacheDistributedQueryManager.java |    4 +-
 .../store/GridCacheStoreManagerAdapter.java     |    1 -
 .../cache/version/GridCacheVersionManager.java  |    6 -
 .../cacheobject/IgniteCacheObjectProcessor.java |    5 -
 .../IgniteCacheObjectProcessorImpl.java         |    5 -
 .../cluster/ChangeGlobalStateFinishMessage.java |   86 ++
 .../cluster/ChangeGlobalStateMessage.java       |  140 ++
 .../processors/cluster/ClusterProcessor.java    |    3 +-
 .../cluster/DiscoveryDataClusterState.java      |  157 +++
 .../cluster/GridClusterStateProcessor.java      | 1122 ++++++---------
 .../cluster/IgniteChangeGlobalStateSupport.java |    3 +-
 .../datastructures/DataStructuresProcessor.java |    6 +-
 .../datastructures/GridCacheAtomicLongImpl.java |    2 +-
 .../GridCacheAtomicReferenceImpl.java           |    2 +-
 .../GridCacheAtomicSequenceImpl.java            |    2 +-
 .../GridCacheAtomicStampedImpl.java             |    2 +-
 .../GridCacheCountDownLatchImpl.java            |    2 +-
 .../datastructures/GridCacheLockImpl.java       |    4 +-
 .../datastructures/GridCacheQueueAdapter.java   |    1 -
 .../datastructures/GridCacheSemaphoreImpl.java  |    2 +-
 .../datastructures/GridCacheSetImpl.java        |    1 -
 .../internal/processors/igfs/IgfsImpl.java      |    2 -
 .../internal/processors/igfs/IgfsProcessor.java |    2 +-
 .../processors/query/GridQueryProcessor.java    |    4 +-
 .../processors/rest/GridRestProcessor.java      |    2 +-
 .../cluster/GridChangeStateCommandHandler.java  |    2 +-
 .../service/GridServiceProcessor.java           |    6 +-
 .../processors/task/GridTaskProcessor.java      |    2 +-
 .../ignite/spi/discovery/tcp/ClientImpl.java    |   12 +-
 .../ignite/spi/discovery/tcp/ServerImpl.java    |   10 +-
 .../internal/TestRecordingCommunicationSpi.java |   10 +
 ...GridManagerLocalMessageListenerSelfTest.java |    4 +-
 .../cache/IgniteActiveClusterTest.java          |  182 ---
 .../IgniteClusterActivateDeactivateTest.java    | 1284 ++++++++++++++++++
 ...erActivateDeactivateTestWithPersistence.java |  197 +++
 .../IgniteDaemonNodeMarshallerCacheTest.java    |   10 -
 .../pagemem/NoOpPageStoreManager.java           |   12 +-
 .../persistence/pagemem/NoOpWALManager.java     |   23 +-
 .../AbstractNodeJoinTemplate.java               |  149 +-
 .../IgniteChangeGlobalStateAbstractTest.java    |   65 +-
 .../IgniteChangeGlobalStateCacheTest.java       |    2 +-
 ...IgniteChangeGlobalStateDataStreamerTest.java |    5 +-
 ...gniteChangeGlobalStateDataStructureTest.java |    6 +-
 .../IgniteChangeGlobalStateFailOverTest.java    |   26 +-
 .../IgniteChangeGlobalStateTest.java            |  158 +--
 .../IgniteStandByClusterTest.java               |   17 +-
 .../join/JoinActiveNodeToActiveCluster.java     |   62 +-
 ...ctiveNodeToActiveClusterWithPersistence.java |   17 +
 .../IgniteStandByClientReconnectTest.java       |   13 +-
 ...eStandByClientReconnectToNewClusterTest.java |   13 +-
 ...cpCommunicationSpiMultithreadedSelfTest.java |    2 +-
 .../testframework/junits/GridAbstractTest.java  |    4 +-
 .../junits/common/GridCommonAbstractTest.java   |    3 +
 .../testsuites/IgniteStandByClusterSuite.java   |    5 +-
 .../processors/hadoop/HadoopProcessor.java      |    4 +-
 106 files changed, 4180 insertions(+), 2197 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/1337901f/modules/core/src/main/java/org/apache/ignite/internal/GridComponent.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/GridComponent.java b/modules/core/src/main/java/org/apache/ignite/internal/GridComponent.java
index 0505929..93ffe95 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/GridComponent.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/GridComponent.java
@@ -84,9 +84,11 @@ public interface GridComponent {
      * Callback that notifies that kernal has successfully started,
      * including all managers and processors.
      *
+     * @param active Cluster active flag (note: should be used carefully since state can
+     *     change concurrently).
      * @throws IgniteCheckedException Thrown in case of any errors.
      */
-    public void onKernalStart() throws IgniteCheckedException;
+    public void onKernalStart(boolean active) throws IgniteCheckedException;
 
     /**
      * Callback to notify that kernal is about to stop.

http://git-wip-us.apache.org/repos/asf/ignite/blob/1337901f/modules/core/src/main/java/org/apache/ignite/internal/GridPluginComponent.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/GridPluginComponent.java b/modules/core/src/main/java/org/apache/ignite/internal/GridPluginComponent.java
index cc1ae71..fd59d24 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/GridPluginComponent.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/GridPluginComponent.java
@@ -61,7 +61,7 @@ public class GridPluginComponent implements GridComponent {
     }
 
     /** {@inheritDoc} */
-    @Override public void onKernalStart() throws IgniteCheckedException {
+    @Override public void onKernalStart(boolean active) throws IgniteCheckedException {
         plugin.onIgniteStart();
     }
 

http://git-wip-us.apache.org/repos/asf/ignite/blob/1337901f/modules/core/src/main/java/org/apache/ignite/internal/IgniteKernal.java
----------------------------------------------------------------------
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 31ee3e2..0c17b32 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
@@ -98,6 +98,7 @@ import org.apache.ignite.internal.managers.checkpoint.GridCheckpointManager;
 import org.apache.ignite.internal.managers.collision.GridCollisionManager;
 import org.apache.ignite.internal.managers.communication.GridIoManager;
 import org.apache.ignite.internal.managers.deployment.GridDeploymentManager;
+import org.apache.ignite.internal.managers.discovery.DiscoveryLocalJoinData;
 import org.apache.ignite.internal.managers.discovery.GridDiscoveryManager;
 import org.apache.ignite.internal.managers.eventstorage.GridEventStorageManager;
 import org.apache.ignite.internal.managers.failover.GridFailoverManager;
@@ -207,7 +208,6 @@ import static org.apache.ignite.internal.IgniteComponentType.HADOOP_HELPER;
 import static org.apache.ignite.internal.IgniteComponentType.IGFS;
 import static org.apache.ignite.internal.IgniteComponentType.IGFS_HELPER;
 import static org.apache.ignite.internal.IgniteComponentType.SCHEDULE;
-import static org.apache.ignite.internal.IgniteNodeAttributes.ATTR_ACTIVE_ON_START;
 import static org.apache.ignite.internal.IgniteNodeAttributes.ATTR_BUILD_DATE;
 import static org.apache.ignite.internal.IgniteNodeAttributes.ATTR_BUILD_VER;
 import static org.apache.ignite.internal.IgniteNodeAttributes.ATTR_CLIENT_MODE;
@@ -818,8 +818,6 @@ public class IgniteKernal implements IgniteEx, IgniteMXBean, Externalizable {
 
         List<PluginProvider> plugins = U.allPluginProviders();
 
-        final boolean activeOnStart = cfg.isActiveOnStart();
-
         // Spin out SPIs & managers.
         try {
             ctx = new GridKernalContextImpl(log,
@@ -994,11 +992,28 @@ public class IgniteKernal implements IgniteEx, IgniteMXBean, Externalizable {
             // Suggest Operation System optimizations.
             ctx.performance().addAll(OsConfigurationSuggestions.getSuggestions());
 
+            DiscoveryLocalJoinData joinData = ctx.discovery().localJoin();
+
+            IgniteInternalFuture<Boolean> transitionWaitFut = joinData.transitionWaitFuture();
+
+            boolean active;
+
+            if (transitionWaitFut != null) {
+                if (log.isInfoEnabled()) {
+                    log.info("Join cluster while cluster state transition is in progress, " +
+                        "waiting when transition finish.");
+                }
+
+                active = transitionWaitFut.get();
+            }
+            else
+                active = joinData.active();
+
             // Notify discovery manager the first to make sure that topology is discovered.
-            ctx.discovery().onKernalStart();
+            ctx.discovery().onKernalStart(active);
 
             // Notify IO manager the second so further components can send and receive messages.
-            ctx.io().onKernalStart();
+            ctx.io().onKernalStart(active);
 
             // Start plugins.
             for (PluginProvider provider : ctx.plugins().allProviders())
@@ -1021,7 +1036,7 @@ public class IgniteKernal implements IgniteEx, IgniteMXBean, Externalizable {
 
                 if (!skipDaemon(comp)) {
                     try {
-                        comp.onKernalStart();
+                        comp.onKernalStart(active);
                     }
                     catch (IgniteNeedReconnectException e) {
                         assert ctx.discovery().reconnectSupported();
@@ -1486,7 +1501,6 @@ public class IgniteKernal implements IgniteEx, IgniteMXBean, Externalizable {
         add(ATTR_MARSHALLER_USE_DFLT_SUID,
             getBoolean(IGNITE_OPTIMIZED_MARSHALLER_USE_DEFAULT_SUID, OptimizedMarshaller.USE_DFLT_SUID));
         add(ATTR_LATE_AFFINITY_ASSIGNMENT, cfg.isLateAffinityAssignment());
-        add(ATTR_ACTIVE_ON_START, cfg.isActiveOnStart());
 
         if (cfg.getMarshaller() instanceof BinaryMarshaller) {
             add(ATTR_MARSHALLER_COMPACT_FOOTER, cfg.getBinaryConfiguration() == null ?
@@ -3395,7 +3409,7 @@ public class IgniteKernal implements IgniteEx, IgniteMXBean, Externalizable {
         guard();
 
         try {
-            return context().state().active();
+            return context().state().publicApiActiveState();
         }
         finally {
             unguard();
@@ -3694,10 +3708,11 @@ public class IgniteKernal implements IgniteEx, IgniteMXBean, Externalizable {
      * @throws IgniteException if cluster in inActive state
      */
     private void checkClusterState() throws IgniteException {
-        if (!ctx.state().active())
+        if (!ctx.state().publicApiActiveState()) {
             throw new IgniteException("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.activate(true).");
+        }
     }
 
     /**

http://git-wip-us.apache.org/repos/asf/ignite/blob/1337901f/modules/core/src/main/java/org/apache/ignite/internal/managers/GridManagerAdapter.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/managers/GridManagerAdapter.java b/modules/core/src/main/java/org/apache/ignite/internal/managers/GridManagerAdapter.java
index 7dfeffb..a151eb5 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/managers/GridManagerAdapter.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/managers/GridManagerAdapter.java
@@ -362,7 +362,7 @@ public abstract class GridManagerAdapter<T extends IgniteSpi> implements GridMan
     }
 
     /** {@inheritDoc} */
-    @Override public final void onKernalStart() throws IgniteCheckedException {
+    @Override public final void onKernalStart(boolean active) throws IgniteCheckedException {
         for (final IgniteSpi spi : spis) {
             try {
                 spi.onContextInitialized(new IgniteSpiContext() {

http://git-wip-us.apache.org/repos/asf/ignite/blob/1337901f/modules/core/src/main/java/org/apache/ignite/internal/managers/discovery/DiscoCache.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/managers/discovery/DiscoCache.java b/modules/core/src/main/java/org/apache/ignite/internal/managers/discovery/DiscoCache.java
index 2b3c4fc..4c1077b 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/managers/discovery/DiscoCache.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/managers/discovery/DiscoCache.java
@@ -25,6 +25,7 @@ import java.util.Map;
 import java.util.Set;
 import java.util.UUID;
 import org.apache.ignite.cluster.ClusterNode;
+import org.apache.ignite.internal.processors.cluster.DiscoveryDataClusterState;
 import org.apache.ignite.internal.util.GridConcurrentHashSet;
 import org.apache.ignite.internal.util.tostring.GridToStringInclude;
 import org.apache.ignite.internal.util.typedef.F;
@@ -37,6 +38,9 @@ import org.jetbrains.annotations.Nullable;
  *
  */
 public class DiscoCache {
+    /** */
+    private final DiscoveryDataClusterState state;
+
     /** Local node. */
     private final ClusterNode loc;
 
@@ -78,6 +82,7 @@ public class DiscoCache {
     private final Set<UUID> alives = new GridConcurrentHashSet<>();
 
     /**
+     * @param state Current cluster state.
      * @param loc Local node.
      * @param rmtNodes Remote nodes.
      * @param allNodes All nodes.
@@ -91,7 +96,9 @@ public class DiscoCache {
      * @param nodeMap Node map.
      * @param alives Alive nodes.
      */
-    DiscoCache(ClusterNode loc,
+    DiscoCache(
+        DiscoveryDataClusterState state,
+        ClusterNode loc,
         List<ClusterNode> rmtNodes,
         List<ClusterNode> allNodes,
         List<ClusterNode> srvNodes,
@@ -103,6 +110,7 @@ public class DiscoCache {
         Map<Integer, List<ClusterNode>> cacheGrpAffNodes,
         Map<UUID, ClusterNode> nodeMap,
         Set<UUID> alives) {
+        this.state = state;
         this.loc = loc;
         this.rmtNodes = rmtNodes;
         this.allNodes = allNodes;
@@ -117,6 +125,13 @@ public class DiscoCache {
         this.alives.addAll(alives);
     }
 
+    /**
+     * @return Current cluster state.
+     */
+    public DiscoveryDataClusterState state() {
+        return state;
+    }
+
     /** @return Local node. */
     public ClusterNode localNode() {
         return loc;

http://git-wip-us.apache.org/repos/asf/ignite/blob/1337901f/modules/core/src/main/java/org/apache/ignite/internal/managers/discovery/DiscoveryLocalJoinData.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/managers/discovery/DiscoveryLocalJoinData.java b/modules/core/src/main/java/org/apache/ignite/internal/managers/discovery/DiscoveryLocalJoinData.java
new file mode 100644
index 0000000..a1f2aa7
--- /dev/null
+++ b/modules/core/src/main/java/org/apache/ignite/internal/managers/discovery/DiscoveryLocalJoinData.java
@@ -0,0 +1,104 @@
+/*
+ * 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.managers.discovery;
+
+import org.apache.ignite.events.DiscoveryEvent;
+import org.apache.ignite.internal.IgniteInternalFuture;
+import org.apache.ignite.internal.processors.affinity.AffinityTopologyVersion;
+import org.apache.ignite.internal.util.typedef.internal.S;
+import org.jetbrains.annotations.Nullable;
+
+/**
+ * Information about local join event.
+ */
+public class DiscoveryLocalJoinData {
+    /** */
+    private final DiscoveryEvent evt;
+
+    /** */
+    private final DiscoCache discoCache;
+
+    /** */
+    private final AffinityTopologyVersion joinTopVer;
+
+    /** */
+    private final IgniteInternalFuture<Boolean> transitionWaitFut;
+
+    /** */
+    private final boolean active;
+
+    /**
+     * @param evt Event.
+     * @param discoCache Discovery data cache.
+     * @param transitionWaitFut Future if cluster state transition is in progress.
+     * @param active Cluster active status.
+     */
+    public DiscoveryLocalJoinData(DiscoveryEvent evt,
+        DiscoCache discoCache,
+        @Nullable IgniteInternalFuture<Boolean> transitionWaitFut,
+        boolean active) {
+        assert evt != null && evt.topologyVersion() > 0 : evt;
+
+        this.evt = evt;
+        this.discoCache = discoCache;
+        this.transitionWaitFut = transitionWaitFut;
+        this.active = active;
+
+        joinTopVer = new AffinityTopologyVersion(evt.topologyVersion(), 0);
+    }
+
+    /**
+     * @return Future if cluster state transition is in progress.
+     */
+    @Nullable public IgniteInternalFuture<Boolean> transitionWaitFuture() {
+        return transitionWaitFut;
+    }
+
+    /**
+     * @return Cluster state.
+     */
+    public boolean active() {
+        return active;
+    }
+
+    /**
+     * @return Event.
+     */
+    public DiscoveryEvent event() {
+        return evt;
+    }
+
+    /**
+     * @return Discovery data cache.
+     */
+    public DiscoCache discoCache() {
+        return discoCache;
+    }
+
+    /**
+     * @return Join topology version.
+     */
+    public AffinityTopologyVersion joinTopologyVersion() {
+        return joinTopVer;
+    }
+
+    /** {@inheritDoc} */
+    @Override public String toString() {
+        return S.toString(DiscoveryLocalJoinData.class, this);
+    }
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/1337901f/modules/core/src/main/java/org/apache/ignite/internal/managers/discovery/GridDiscoveryManager.java
----------------------------------------------------------------------
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 c38e37a..9f5bd3f 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
@@ -75,8 +75,11 @@ import org.apache.ignite.internal.processors.cache.ClientCacheChangeDummyDiscove
 import org.apache.ignite.internal.processors.cache.DynamicCacheChangeRequest;
 import org.apache.ignite.internal.processors.cache.DynamicCacheDescriptor;
 import org.apache.ignite.internal.processors.cache.GridCacheAdapter;
-import org.apache.ignite.internal.processors.cluster.GridClusterStateProcessor;
 import org.apache.ignite.internal.processors.cache.GridCacheContext;
+import org.apache.ignite.internal.processors.cluster.ChangeGlobalStateFinishMessage;
+import org.apache.ignite.internal.processors.cluster.ChangeGlobalStateMessage;
+import org.apache.ignite.internal.processors.cluster.DiscoveryDataClusterState;
+import org.apache.ignite.internal.processors.cluster.GridClusterStateProcessor;
 import org.apache.ignite.internal.processors.jobmetrics.GridJobMetrics;
 import org.apache.ignite.internal.processors.security.SecurityContext;
 import org.apache.ignite.internal.processors.timeout.GridTimeoutProcessor;
@@ -90,7 +93,6 @@ import org.apache.ignite.internal.util.typedef.CI1;
 import org.apache.ignite.internal.util.typedef.F;
 import org.apache.ignite.internal.util.typedef.G;
 import org.apache.ignite.internal.util.typedef.P1;
-import org.apache.ignite.internal.util.typedef.T2;
 import org.apache.ignite.internal.util.typedef.internal.CU;
 import org.apache.ignite.internal.util.typedef.internal.LT;
 import org.apache.ignite.internal.util.typedef.internal.S;
@@ -133,7 +135,6 @@ 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.events.EventType.EVT_NODE_METRICS_UPDATED;
 import static org.apache.ignite.events.EventType.EVT_NODE_SEGMENTED;
-import static org.apache.ignite.internal.IgniteNodeAttributes.ATTR_ACTIVE_ON_START;
 import static org.apache.ignite.internal.IgniteNodeAttributes.ATTR_DEPLOYMENT_MODE;
 import static org.apache.ignite.internal.IgniteNodeAttributes.ATTR_LATE_AFFINITY_ASSIGNMENT;
 import static org.apache.ignite.internal.IgniteNodeAttributes.ATTR_MACS;
@@ -144,6 +145,7 @@ import static org.apache.ignite.internal.IgniteNodeAttributes.ATTR_SECURITY_COMP
 import static org.apache.ignite.internal.IgniteNodeAttributes.ATTR_SERVICES_COMPATIBILITY_MODE;
 import static org.apache.ignite.internal.IgniteNodeAttributes.ATTR_USER_NAME;
 import static org.apache.ignite.internal.IgniteVersionUtils.VER;
+import static org.apache.ignite.internal.events.DiscoveryCustomEvent.EVT_DISCOVERY_CUSTOM_EVT;
 import static org.apache.ignite.internal.processors.security.SecurityUtils.SERVICE_PERMISSIONS_SINCE;
 import static org.apache.ignite.internal.processors.security.SecurityUtils.isSecurityCompatibilityMode;
 import static org.apache.ignite.plugin.segmentation.SegmentationPolicy.NOOP;
@@ -238,7 +240,7 @@ public class GridDiscoveryManager extends GridManagerAdapter<DiscoverySpi> {
     private long segChkFreq;
 
     /** Local node join to topology event. */
-    private GridFutureAdapter<T2<DiscoveryEvent, DiscoCache>> locJoin = new GridFutureAdapter<>();
+    private GridFutureAdapter<DiscoveryLocalJoinData> locJoin = new GridFutureAdapter<>();
 
     /** GC CPU load. */
     private volatile double gcCpuLoad;
@@ -570,7 +572,7 @@ public class GridDiscoveryManager extends GridManagerAdapter<DiscoverySpi> {
                     if (type != EVT_NODE_SEGMENTED &&
                         type != EVT_CLIENT_NODE_DISCONNECTED &&
                         type != EVT_CLIENT_NODE_RECONNECTED &&
-                        type != DiscoveryCustomEvent.EVT_DISCOVERY_CUSTOM_EVT) {
+                        type != EVT_DISCOVERY_CUSTOM_EVT) {
                         minorTopVer = 0;
 
                         verChanged = true;
@@ -586,15 +588,50 @@ public class GridDiscoveryManager extends GridManagerAdapter<DiscoverySpi> {
                     updateClientNodes(node.id());
                 }
 
+                DiscoCache discoCache = null;
+
+                boolean locJoinEvt = type == EVT_NODE_JOINED && node.id().equals(locNode.id());
+
+                IgniteInternalFuture<Boolean> transitionWaitFut = null;
+
+                ChangeGlobalStateFinishMessage stateFinishMsg = null;
+
+                if (locJoinEvt) {
+                    discoCache = createDiscoCache(ctx.state().clusterState(), locNode, topSnapshot);
+
+                    transitionWaitFut = ctx.state().onLocalJoin(discoCache);
+                }
+                else if (type == EVT_NODE_FAILED || type == EVT_NODE_LEFT)
+                    stateFinishMsg = ctx.state().onNodeLeft(node);
+
                 final AffinityTopologyVersion nextTopVer;
 
-                if (type == DiscoveryCustomEvent.EVT_DISCOVERY_CUSTOM_EVT) {
+                if (type == EVT_DISCOVERY_CUSTOM_EVT) {
                     assert customMsg != null;
 
-                    boolean incMinorTopVer = ctx.cache().onCustomEvent(
-                        customMsg,
-                        new AffinityTopologyVersion(topVer, minorTopVer),
-                        node);
+                    boolean incMinorTopVer;
+
+                    if (customMsg instanceof ChangeGlobalStateMessage) {
+                        incMinorTopVer = ctx.state().onStateChangeMessage(
+                            new AffinityTopologyVersion(topVer, minorTopVer),
+                            (ChangeGlobalStateMessage)customMsg,
+                            discoCache());
+                    }
+                    else if (customMsg instanceof ChangeGlobalStateFinishMessage) {
+                        ctx.state().onStateFinishMessage((ChangeGlobalStateFinishMessage)customMsg);
+
+                        discoCache = createDiscoCache(ctx.state().clusterState(), locNode, topSnapshot);
+
+                        topSnap.set(new Snapshot(topSnap.get().topVer, discoCache));
+
+                        incMinorTopVer = false;
+                    }
+                    else {
+                        incMinorTopVer = ctx.cache().onCustomEvent(
+                            customMsg,
+                            new AffinityTopologyVersion(topVer, minorTopVer),
+                            node);
+                    }
 
                     if (incMinorTopVer) {
                         minorTopVer++;
@@ -603,17 +640,13 @@ public class GridDiscoveryManager extends GridManagerAdapter<DiscoverySpi> {
                     }
 
                     nextTopVer = new AffinityTopologyVersion(topVer, minorTopVer);
-
-                    if (verChanged)
-                        ctx.cache().onDiscoveryEvent(type, node, nextTopVer);
                 }
-                else {
+                else
                     nextTopVer = new AffinityTopologyVersion(topVer, minorTopVer);
 
-                    ctx.cache().onDiscoveryEvent(type, node, nextTopVer);
-                }
+                ctx.cache().onDiscoveryEvent(type, customMsg, node, nextTopVer, ctx.state().clusterState());
 
-                if (type == DiscoveryCustomEvent.EVT_DISCOVERY_CUSTOM_EVT) {
+                if (type == EVT_DISCOVERY_CUSTOM_EVT) {
                     for (Class cls = customMsg.getClass(); cls != null; cls = cls.getSuperclass()) {
                         List<CustomEventListener<DiscoveryCustomMessage>> list = customEvtLsnrs.get(cls);
 
@@ -630,13 +663,12 @@ public class GridDiscoveryManager extends GridManagerAdapter<DiscoverySpi> {
                     }
                 }
 
-                final DiscoCache discoCache;
-
                 // Put topology snapshot into discovery history.
                 // There is no race possible between history maintenance and concurrent discovery
                 // event notifications, since SPI notifies manager about all events from this listener.
                 if (verChanged) {
-                    discoCache = createDiscoCache(locNode, topSnapshot);
+                    if (discoCache == null)
+                        discoCache = createDiscoCache(ctx.state().clusterState(), locNode, topSnapshot);
 
                     discoCacheHist.put(nextTopVer, discoCache);
 
@@ -650,8 +682,10 @@ public class GridDiscoveryManager extends GridManagerAdapter<DiscoverySpi> {
                     // Current version.
                     discoCache = discoCache();
 
+                final DiscoCache discoCache0 = discoCache;
+
                 // If this is a local join event, just save it and do not notify listeners.
-                if (type == EVT_NODE_JOINED && node.id().equals(locNode.id())) {
+                if (locJoinEvt) {
                     if (gridStartTime == 0)
                         gridStartTime = getSpi().getGridStartTime();
 
@@ -668,7 +702,15 @@ public class GridDiscoveryManager extends GridManagerAdapter<DiscoverySpi> {
 
                     discoEvt.topologySnapshot(topVer, new ArrayList<>(F.view(topSnapshot, FILTER_DAEMON)));
 
-                    locJoin.onDone(new T2<>(discoEvt, discoCache));
+                    discoWrk.discoCache = discoCache;
+
+                    if (!isLocDaemon && !ctx.clientDisconnected())
+                        ctx.cache().context().exchange().onLocalJoin(discoEvt, discoCache);
+
+                    locJoin.onDone(new DiscoveryLocalJoinData(discoEvt,
+                        discoCache,
+                        transitionWaitFut,
+                        ctx.state().clusterState().active()));
 
                     return;
                 }
@@ -697,7 +739,7 @@ public class GridDiscoveryManager extends GridManagerAdapter<DiscoverySpi> {
                     topHist.clear();
 
                     topSnap.set(new Snapshot(AffinityTopologyVersion.ZERO,
-                        createDiscoCache(locNode, Collections.<ClusterNode>emptySet())));
+                        createDiscoCache(ctx.state().clusterState(), locNode, Collections.<ClusterNode>emptySet())));
                 }
                 else if (type == EVT_CLIENT_NODE_RECONNECTED) {
                     assert locNode.isClient() : locNode;
@@ -709,12 +751,14 @@ public class GridDiscoveryManager extends GridManagerAdapter<DiscoverySpi> {
 
                     ((IgniteKernal)ctx.grid()).onReconnected(clusterRestarted);
 
+                    ctx.cache().context().exchange().onLocalJoin(localJoinEvent(), discoCache);
+
                     ctx.cluster().clientReconnectFuture().listen(new CI1<IgniteFuture<?>>() {
                         @Override public void apply(IgniteFuture<?> fut) {
                             try {
                                 fut.get();
 
-                                discoWrk.addEvent(type, nextTopVer, node, discoCache, topSnapshot, null);
+                                discoWrk.addEvent(type, nextTopVer, node, discoCache0, topSnapshot, null);
                             }
                             catch (IgniteException ignore) {
                                 // No-op.
@@ -727,6 +771,9 @@ public class GridDiscoveryManager extends GridManagerAdapter<DiscoverySpi> {
 
                 if (type == EVT_CLIENT_NODE_DISCONNECTED || type == EVT_NODE_SEGMENTED || !ctx.clientDisconnected())
                     discoWrk.addEvent(type, nextTopVer, node, discoCache, topSnapshot, customMsg);
+
+                if (stateFinishMsg != null)
+                    discoWrk.addEvent(EVT_DISCOVERY_CUSTOM_EVT, nextTopVer, node, discoCache, topSnapshot, stateFinishMsg);
             }
         });
 
@@ -826,7 +873,7 @@ public class GridDiscoveryManager extends GridManagerAdapter<DiscoverySpi> {
      * @return {@code True} if should not process message.
      */
     private boolean skipMessage(int type, @Nullable DiscoveryCustomMessage customMsg) {
-        if (type == DiscoveryCustomEvent.EVT_DISCOVERY_CUSTOM_EVT) {
+        if (type == EVT_DISCOVERY_CUSTOM_EVT) {
             assert customMsg != null && customMsg.id() != null : customMsg;
 
             if (rcvdCustomMsgs.contains(customMsg.id())) {
@@ -1157,7 +1204,6 @@ public class GridDiscoveryManager extends GridManagerAdapter<DiscoverySpi> {
             locMarshStrSerVer2;
 
         boolean locDelayAssign = locNode.attribute(ATTR_LATE_AFFINITY_ASSIGNMENT);
-        boolean locActiveOnStart = locNode.attribute(ATTR_ACTIVE_ON_START);
 
         Boolean locSrvcCompatibilityEnabled = locNode.attribute(ATTR_SERVICES_COMPATIBILITY_MODE);
         Boolean locSecurityCompatibilityEnabled = locNode.attribute(ATTR_SECURITY_COMPATIBILITY_MODE);
@@ -1971,7 +2017,7 @@ public class GridDiscoveryManager extends GridManagerAdapter<DiscoverySpi> {
     /** @return Event that represents a local node joined to topology. */
     public DiscoveryEvent localJoinEvent() {
         try {
-            return locJoin.get().get1();
+            return locJoin.get().event();
         }
         catch (IgniteCheckedException e) {
             throw new IgniteException(e);
@@ -1981,7 +2027,7 @@ public class GridDiscoveryManager extends GridManagerAdapter<DiscoverySpi> {
     /**
      * @return Tuple that consists of a local join event and discovery cache at the join time.
      */
-    public T2<DiscoveryEvent, DiscoCache> localJoin() {
+    public DiscoveryLocalJoinData localJoin() {
         try {
             return locJoin.get();
         }
@@ -2016,7 +2062,7 @@ public class GridDiscoveryManager extends GridManagerAdapter<DiscoverySpi> {
     public void clientCacheStartEvent(UUID reqId,
         @Nullable Map<String, DynamicCacheChangeRequest> startReqs,
         @Nullable Set<String> cachesToClose) {
-        discoWrk.addEvent(DiscoveryCustomEvent.EVT_DISCOVERY_CUSTOM_EVT,
+        discoWrk.addEvent(EVT_DISCOVERY_CUSTOM_EVT,
             AffinityTopologyVersion.NONE,
             localNode(),
             null,
@@ -2098,11 +2144,14 @@ public class GridDiscoveryManager extends GridManagerAdapter<DiscoverySpi> {
     /**
      * Called from discovery thread.
      *
+     * @param state Current state.
      * @param loc Local node.
      * @param topSnapshot Topology snapshot.
      * @return Newly created discovery cache.
      */
-    @NotNull private DiscoCache createDiscoCache(ClusterNode loc, Collection<ClusterNode> topSnapshot) {
+    @NotNull private DiscoCache createDiscoCache(DiscoveryDataClusterState state,
+        ClusterNode loc,
+        Collection<ClusterNode> topSnapshot) {
         HashSet<UUID> alives = U.newHashSet(topSnapshot.size());
         HashMap<UUID, ClusterNode> nodeMap = U.newHashMap(topSnapshot.size());
 
@@ -2177,6 +2226,7 @@ public class GridDiscoveryManager extends GridManagerAdapter<DiscoverySpi> {
         }
 
         return new DiscoCache(
+            state,
             loc,
             Collections.unmodifiableList(rmtNodes),
             Collections.unmodifiableList(allNodes),
@@ -2318,7 +2368,7 @@ public class GridDiscoveryManager extends GridManagerAdapter<DiscoverySpi> {
                         discoWrk.addEvent(EVT_NODE_SEGMENTED,
                             AffinityTopologyVersion.NONE,
                             node,
-                            createDiscoCache(node, empty),
+                            createDiscoCache(null, node, empty),
                             empty,
                             null);
 
@@ -2339,6 +2389,9 @@ public class GridDiscoveryManager extends GridManagerAdapter<DiscoverySpi> {
 
     /** Worker for discovery events. */
     private class DiscoveryWorker extends GridWorker {
+        /** */
+        private DiscoCache discoCache;
+
         /** Event queue. */
         private final BlockingQueue<GridTuple6<Integer, AffinityTopologyVersion, ClusterNode,
             DiscoCache, Collection<ClusterNode>, DiscoveryCustomMessage>> evts = new LinkedBlockingQueue<>();
@@ -2457,6 +2510,9 @@ public class GridDiscoveryManager extends GridManagerAdapter<DiscoverySpi> {
 
             boolean segmented = false;
 
+            if (evt.get4() != null)
+                discoCache = evt.get4();
+
             switch (type) {
                 case EVT_NODE_JOINED: {
                     assert !discoOrdered || topVer.topologyVersion() == node.order() : "Invalid topology version [topVer=" + topVer +
@@ -2570,8 +2626,8 @@ public class GridDiscoveryManager extends GridManagerAdapter<DiscoverySpi> {
                     break;
                 }
 
-                case DiscoveryCustomEvent.EVT_DISCOVERY_CUSTOM_EVT: {
-                    if (ctx.event().isRecordable(DiscoveryCustomEvent.EVT_DISCOVERY_CUSTOM_EVT)) {
+                case EVT_DISCOVERY_CUSTOM_EVT: {
+                    if (ctx.event().isRecordable(EVT_DISCOVERY_CUSTOM_EVT)) {
                         DiscoveryCustomEvent customEvt = new DiscoveryCustomEvent();
 
                         customEvt.node(ctx.discovery().localNode());
@@ -2581,6 +2637,12 @@ public class GridDiscoveryManager extends GridManagerAdapter<DiscoverySpi> {
                         customEvt.affinityTopologyVersion(topVer);
                         customEvt.customMessage(evt.get6());
 
+                        if (evt.get4() == null) {
+                            assert discoCache != null : evt.get6();
+
+                            evt.set4(discoCache);
+                        }
+
                         ctx.event().record(customEvt, evt.get4());
                     }
 

http://git-wip-us.apache.org/repos/asf/ignite/blob/1337901f/modules/core/src/main/java/org/apache/ignite/internal/pagemem/store/IgnitePageStoreManager.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/pagemem/store/IgnitePageStoreManager.java b/modules/core/src/main/java/org/apache/ignite/internal/pagemem/store/IgnitePageStoreManager.java
index 468d35d..fa6e9e4 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/pagemem/store/IgnitePageStoreManager.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/pagemem/store/IgnitePageStoreManager.java
@@ -183,11 +183,10 @@ public interface IgnitePageStoreManager extends GridCacheSharedManager, IgniteCh
     public Map<String, StoredCacheData> readCacheConfigurations() throws IgniteCheckedException;
 
     /**
-     * @param grpDesc Cache group descriptor.
      * @param cacheData Cache configuration.
      * @throws IgniteCheckedException If failed.
      */
-    public void storeCacheData(CacheGroupDescriptor grpDesc, StoredCacheData cacheData) throws IgniteCheckedException;
+    public void storeCacheData(StoredCacheData cacheData) throws IgniteCheckedException;
     /**
      * @param grpId Cache group ID.
      * @return {@code True} if index store for given cache group existed before node started.

http://git-wip-us.apache.org/repos/asf/ignite/blob/1337901f/modules/core/src/main/java/org/apache/ignite/internal/processors/GridProcessorAdapter.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/GridProcessorAdapter.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/GridProcessorAdapter.java
index 690ba0e..d6f78ab 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/GridProcessorAdapter.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/GridProcessorAdapter.java
@@ -65,7 +65,7 @@ public abstract class GridProcessorAdapter implements GridProcessor {
     }
 
     /** {@inheritDoc} */
-    @Override public void onKernalStart() throws IgniteCheckedException {
+    @Override public void onKernalStart(boolean active) throws IgniteCheckedException {
         // No-op.
     }
 

http://git-wip-us.apache.org/repos/asf/ignite/blob/1337901f/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/CacheAffinitySharedManager.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/CacheAffinitySharedManager.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/CacheAffinitySharedManager.java
index 9516f84..8d08c3f 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/CacheAffinitySharedManager.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/CacheAffinitySharedManager.java
@@ -41,6 +41,7 @@ import org.apache.ignite.events.Event;
 import org.apache.ignite.internal.IgniteInternalFuture;
 import org.apache.ignite.internal.cluster.ClusterTopologyServerNotFoundException;
 import org.apache.ignite.internal.managers.discovery.DiscoCache;
+import org.apache.ignite.internal.managers.discovery.DiscoveryCustomMessage;
 import org.apache.ignite.internal.managers.eventstorage.GridLocalEventListener;
 import org.apache.ignite.internal.processors.affinity.AffinityTopologyVersion;
 import org.apache.ignite.internal.processors.affinity.GridAffinityAssignmentCache;
@@ -52,6 +53,8 @@ import org.apache.ignite.internal.processors.cache.distributed.dht.GridDhtPartit
 import org.apache.ignite.internal.processors.cache.distributed.dht.GridDhtPartitionTopology;
 import org.apache.ignite.internal.processors.cache.distributed.dht.preloader.GridDhtPartitionFullMap;
 import org.apache.ignite.internal.processors.cache.distributed.dht.preloader.GridDhtPartitionsExchangeFuture;
+import org.apache.ignite.internal.processors.cluster.ChangeGlobalStateFinishMessage;
+import org.apache.ignite.internal.processors.cluster.DiscoveryDataClusterState;
 import org.apache.ignite.internal.util.future.GridCompoundFuture;
 import org.apache.ignite.internal.util.future.GridFinishedFuture;
 import org.apache.ignite.internal.util.future.GridFutureAdapter;
@@ -108,6 +111,9 @@ public class CacheAffinitySharedManager<K, V> extends GridCacheSharedManagerAdap
     /** */
     private final ThreadLocal<ClientCacheChangeDiscoveryMessage> clientCacheChanges = new ThreadLocal<>();
 
+    /** Caches initialized flag (initialized when join activate cluster or after activation. */
+    private boolean cachesInitialized;
+
     /** Discovery listener. */
     private final GridLocalEventListener discoLsnr = new GridLocalEventListener() {
         @Override public void onEvent(Event evt) {
@@ -140,10 +146,19 @@ public class CacheAffinitySharedManager<K, V> extends GridCacheSharedManagerAdap
      * Callback invoked from discovery thread when discovery message is received.
      *
      * @param type Event type.
+     * @param customMsg Custom message instance.
      * @param node Event node.
      * @param topVer Topology version.
+     * @param state Cluster state.
      */
-    void onDiscoveryEvent(int type, ClusterNode node, AffinityTopologyVersion topVer) {
+    void onDiscoveryEvent(int type,
+        @Nullable DiscoveryCustomMessage customMsg,
+        ClusterNode node,
+        AffinityTopologyVersion topVer,
+        DiscoveryDataClusterState state) {
+        if (state.transition() || !state.active())
+            return;
+
         if (type == EVT_NODE_JOINED && node.isLocal()) {
             // Clean-up in case of client reconnect.
             caches.clear();
@@ -153,6 +168,15 @@ public class CacheAffinitySharedManager<K, V> extends GridCacheSharedManagerAdap
             lastAffVer = null;
 
             caches.init(cctx.cache().cacheGroupDescriptors(), cctx.cache().cacheDescriptors());
+
+            cachesInitialized = true;
+        }
+        else if (customMsg instanceof ChangeGlobalStateFinishMessage) {
+            if (!cachesInitialized && ((ChangeGlobalStateFinishMessage)customMsg).clusterActive()) {
+                caches.init(cctx.cache().cacheGroupDescriptors(), cctx.cache().cacheDescriptors());
+
+                cachesInitialized = true;
+            }
         }
 
         if (!CU.clientNode(node) && (type == EVT_NODE_FAILED || type == EVT_NODE_JOINED || type == EVT_NODE_LEFT)) {
@@ -404,7 +428,10 @@ public class CacheAffinitySharedManager<K, V> extends GridCacheSharedManagerAdap
 
                 DynamicCacheChangeRequest startReq = startReqs.get(desc.cacheName());
 
-                cctx.cache().prepareCacheStart(desc, startReq.nearCacheConfiguration(), topVer);
+                cctx.cache().prepareCacheStart(desc.cacheConfiguration(),
+                    desc,
+                    startReq.nearCacheConfiguration(),
+                    topVer);
 
                 startedInfos.put(desc.cacheId(), startReq.nearCacheConfiguration() != null);
 
@@ -683,19 +710,7 @@ public class CacheAffinitySharedManager<K, V> extends GridCacheSharedManagerAdap
 
             NearCacheConfiguration nearCfg = null;
 
-            if (exchActions.newClusterState() == ClusterState.ACTIVE) {
-                if (CU.isSystemCache(req.cacheName()))
-                    startCache = true;
-                else if (!cctx.localNode().isClient()) {
-                    startCache = cctx.cacheContext(action.descriptor().cacheId()) == null &&
-                        CU.affinityNode(cctx.localNode(), req.startCacheConfiguration().getNodeFilter());
-
-                    nearCfg = req.nearCacheConfiguration();
-                }
-                else // Only static cache configured on client must be started.
-                    startCache = cctx.kernalContext().state().isLocallyConfigured(req.cacheName());
-            }
-            else if (cctx.localNodeId().equals(req.initiatingNodeId())) {
+            if (req.locallyConfigured() || (cctx.localNodeId().equals(req.initiatingNodeId()) && !exchActions.activate())) {
                 startCache = true;
 
                 nearCfg = req.nearCacheConfiguration();
@@ -703,7 +718,7 @@ public class CacheAffinitySharedManager<K, V> extends GridCacheSharedManagerAdap
             else {
                 // Cache should not be started
                 assert cctx.cacheContext(cacheDesc.cacheId()) == null
-                        : "Starting cache has not null context: " + cacheDesc.cacheName();
+                    : "Starting cache has not null context: " + cacheDesc.cacheName();
 
                 IgniteCacheProxy cacheProxy = cctx.cache().jcacheProxy(req.cacheName());
 
@@ -711,27 +726,29 @@ public class CacheAffinitySharedManager<K, V> extends GridCacheSharedManagerAdap
                 if (cacheProxy != null) {
                     // Cache should be in restarting mode
                     assert cacheProxy.isRestarting()
-                            : "Cache has non restarting proxy " + cacheProxy;
+                        : "Cache has non restarting proxy " + cacheProxy;
 
                     startCache = true;
                 }
-                else
-                    startCache = CU.affinityNode(cctx.localNode(), cacheDesc.groupDescriptor().config().getNodeFilter());
+                else {
+                    startCache = CU.affinityNode(cctx.localNode(),
+                        cacheDesc.groupDescriptor().config().getNodeFilter());
+                }
             }
 
             try {
                 // Save configuration before cache started.
-                if (cctx.pageStore() != null && !cctx.localNode().isClient())
+                if (cctx.pageStore() != null && !cctx.kernalContext().clientNode()) {
                     cctx.pageStore().storeCacheData(
-                        cacheDesc.groupDescriptor(),
                         new StoredCacheData(req.startCacheConfiguration())
                     );
+                }
 
                 if (startCache) {
-                    cctx.cache().prepareCacheStart(cacheDesc, nearCfg, fut.topologyVersion());
-
-                    if (exchActions.newClusterState() == null)
-                        cctx.kernalContext().state().onCacheStart(req);
+                    cctx.cache().prepareCacheStart(req.startCacheConfiguration(),
+                        cacheDesc,
+                        nearCfg,
+                        fut.topologyVersion());
 
                     if (fut.cacheAddedOnExchange(cacheDesc.cacheId(), cacheDesc.receivedFrom())) {
                         if (fut.discoCache().cacheGroupAffinityNodes(cacheDesc.groupId()).isEmpty())

http://git-wip-us.apache.org/repos/asf/ignite/blob/1337901f/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/CacheGroupContext.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/CacheGroupContext.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/CacheGroupContext.java
index c3ddc5f..14eb362 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/CacheGroupContext.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/CacheGroupContext.java
@@ -683,6 +683,8 @@ public class CacheGroupContext {
 
         aff.cancelFutures(err);
 
+        preldr.onKernalStop();
+
         offheapMgr.stop();
 
         ctx.io().removeCacheGroupHandlers(grpId);
@@ -853,8 +855,6 @@ public class CacheGroupContext {
             preldr = new GridCachePreloaderAdapter(this);
 
         if (ctx.kernalContext().config().getPersistentStoreConfiguration() != null) {
-            ClassLoader clsLdr = U.gridClassLoader();
-
             try {
                 offheapMgr = new GridCacheOffheapManager();
             }

http://git-wip-us.apache.org/repos/asf/ignite/blob/1337901f/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/CacheGroupData.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/CacheGroupData.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/CacheGroupData.java
index a290caf..99b7b1e 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/CacheGroupData.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/CacheGroupData.java
@@ -78,8 +78,8 @@ public class CacheGroupData implements Serializable {
         Map<String, Integer> caches,
         long flags) {
         assert cacheCfg != null;
-        assert grpId != 0;
-        assert deploymentId != null;
+        assert grpId != 0 : cacheCfg.getName();
+        assert deploymentId != null : cacheCfg.getName();
 
         this.cacheCfg = cacheCfg;
         this.grpName = grpName;

http://git-wip-us.apache.org/repos/asf/ignite/blob/1337901f/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/ChangeGlobalStateMessage.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/ChangeGlobalStateMessage.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/ChangeGlobalStateMessage.java
deleted file mode 100644
index 4d1a50b..0000000
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/ChangeGlobalStateMessage.java
+++ /dev/null
@@ -1,120 +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.processors.cache;
-
-import java.util.UUID;
-import org.apache.ignite.internal.managers.discovery.DiscoveryCustomMessage;
-import org.apache.ignite.internal.util.typedef.internal.S;
-import org.apache.ignite.lang.IgniteUuid;
-import org.jetbrains.annotations.Nullable;
-
-/**
- * Message represent request for change cluster global state.
- */
-public class ChangeGlobalStateMessage implements DiscoveryCustomMessage {
-    /** */
-    private static final long serialVersionUID = 0L;
-
-    /** Custom message ID. */
-    private IgniteUuid id = IgniteUuid.randomUuid();
-
-    /** Request ID */
-    private UUID requestId;
-
-    /** Initiator node ID. */
-    private UUID initiatingNodeId;
-
-    /** If true activate else deactivate. */
-    private boolean activate;
-
-    /** Batch contains all requests for start or stop caches. */
-    private DynamicCacheChangeBatch changeGlobalStateBatch;
-
-    /** If happened concurrent activate/deactivate then processed only first message, other message must be skip. */
-    private boolean concurrentChangeState;
-
-    /**
-     *
-     */
-    public ChangeGlobalStateMessage(
-        UUID requestId,
-        UUID initiatingNodeId,
-        boolean activate,
-        DynamicCacheChangeBatch changeGlobalStateBatch
-    ) {
-        this.requestId = requestId;
-        this.initiatingNodeId = initiatingNodeId;
-        this.activate = activate;
-        this.changeGlobalStateBatch = changeGlobalStateBatch;
-    }
-
-    /**
-     *
-     */
-    public DynamicCacheChangeBatch getDynamicCacheChangeBatch() {
-        return changeGlobalStateBatch;
-    }
-
-    /** {@inheritDoc} */
-    @Override public IgniteUuid id() {
-        return id;
-    }
-
-    /** {@inheritDoc} */
-    @Nullable @Override public DiscoveryCustomMessage ackMessage() {
-        return !concurrentChangeState ? changeGlobalStateBatch : null;
-    }
-
-    /** {@inheritDoc} */
-    @Override public boolean isMutable() {
-        return false;
-    }
-
-    /**
-     *
-     */
-    public UUID initiatorNodeId() {
-        return initiatingNodeId;
-    }
-
-    /**
-     *
-     */
-    public boolean activate() {
-        return activate;
-    }
-
-    /**
-     *
-     */
-    public UUID requestId() {
-        return requestId;
-    }
-
-    /**
-     *
-     */
-    public void concurrentChangeState() {
-        this.concurrentChangeState = true;
-    }
-
-    /** {@inheritDoc} */
-    @Override public String toString() {
-        return S.toString(ChangeGlobalStateMessage.class, this);
-    }
-}

http://git-wip-us.apache.org/repos/asf/ignite/blob/1337901f/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/ClusterCachesInfo.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/ClusterCachesInfo.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/ClusterCachesInfo.java
index 8f124b2..5452bd2 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/ClusterCachesInfo.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/ClusterCachesInfo.java
@@ -40,6 +40,9 @@ import org.apache.ignite.configuration.NearCacheConfiguration;
 import org.apache.ignite.internal.GridKernalContext;
 import org.apache.ignite.internal.IgniteInternalFuture;
 import org.apache.ignite.internal.processors.affinity.AffinityTopologyVersion;
+import org.apache.ignite.internal.processors.cluster.ChangeGlobalStateFinishMessage;
+import org.apache.ignite.internal.processors.cluster.ChangeGlobalStateMessage;
+import org.apache.ignite.internal.processors.cluster.DiscoveryDataClusterState;
 import org.apache.ignite.internal.processors.query.QuerySchema;
 import org.apache.ignite.internal.processors.query.QueryUtils;
 import org.apache.ignite.internal.processors.query.schema.SchemaOperationException;
@@ -93,10 +96,13 @@ class ClusterCachesInfo {
     private List<T2<DynamicCacheDescriptor, NearCacheConfiguration>> locJoinStartCaches;
 
     /** */
-    private Map<UUID, CacheClientReconnectDiscoveryData> clientReconnectReqs;
+    private Map<String, T2<CacheConfiguration, NearCacheConfiguration>> locCfgsForActivation;
 
     /** */
-    private volatile Exception onJoinCacheException;
+    private Map<UUID, CacheClientReconnectDiscoveryData> clientReconnectReqs;
+
+    /** {@code True} if joined cluster while cluster state change was in progress. */
+    private boolean joinOnTransition;
 
     /**
      * @param ctx Context.
@@ -113,14 +119,25 @@ class ClusterCachesInfo {
      */
     void onStart(CacheJoinNodeDiscoveryData joinDiscoData) throws IgniteCheckedException {
         this.joinDiscoData = joinDiscoData;
-    }
 
-    /**
-     *
-     * @return Exception if cache has conflict.
-     */
-    Exception onJoinCacheException(){
-        return onJoinCacheException;
+        Map<String, CacheConfiguration> grpCfgs = new HashMap<>();
+
+        for (CacheJoinNodeDiscoveryData.CacheInfo info : joinDiscoData.caches().values()) {
+            if (info.cacheData().config().getGroupName() == null)
+                continue;
+
+            CacheConfiguration ccfg = grpCfgs.get(info.cacheData().config().getGroupName());
+
+            if (ccfg == null)
+                grpCfgs.put(info.cacheData().config().getGroupName(), info.cacheData().config());
+            else
+                validateCacheGroupConfiguration(ccfg, info.cacheData().config());
+        }
+
+        String conflictErr = processJoiningNode(joinDiscoData, ctx.localNodeId(), true);
+
+        if (conflictErr != null)
+            throw new IgniteCheckedException("Failed to start configured cache. " + conflictErr);
     }
 
     /**
@@ -142,7 +159,9 @@ class ClusterCachesInfo {
         if (gridData != null && gridData.conflictErr != null)
             throw new IgniteCheckedException(gridData.conflictErr);
 
-        if (joinDiscoData != null && gridData != null) {
+        if (gridData != null && gridData.joinDiscoData != null) {
+            CacheJoinNodeDiscoveryData joinDiscoData = gridData.joinDiscoData;
+
             for (CacheJoinNodeDiscoveryData.CacheInfo locCacheInfo : joinDiscoData.caches().values()) {
                 CacheConfiguration locCfg = locCacheInfo.cacheData().config();
 
@@ -165,9 +184,9 @@ class ClusterCachesInfo {
             }
         }
 
-        joinDiscoData = null;
         gridData = null;
     }
+
     /**
      * Checks that remote caches has configuration compatible with the local.
      *
@@ -308,22 +327,64 @@ class ClusterCachesInfo {
             }
         }
     }
-
     /**
      * @param batch Cache change request.
      * @param topVer Topology version.
      * @return {@code True} if minor topology version should be increased.
      */
     boolean onCacheChangeRequested(DynamicCacheChangeBatch batch, AffinityTopologyVersion topVer) {
-        ExchangeActions exchangeActions = new ExchangeActions();
+        DiscoveryDataClusterState state = ctx.state().clusterState();
+
+        if (state.active() && !state.transition()) {
+            ExchangeActions exchangeActions = new ExchangeActions();
+
+            CacheChangeProcessResult res = processCacheChangeRequests(exchangeActions,
+                batch.requests(),
+                topVer,
+                false);
 
-        boolean incMinorTopVer = false;
+            if (res.needExchange) {
+                assert !exchangeActions.empty() : exchangeActions;
 
-        List<DynamicCacheDescriptor> addedDescs = new ArrayList<>();
+                batch.exchangeActions(exchangeActions);
+            }
+
+            return res.needExchange;
+        }
+        else {
+            IgniteCheckedException err = new IgniteCheckedException("Failed to start/stop cache, cluster state change " +
+                "is in progress.");
+
+            for (DynamicCacheChangeRequest req : batch.requests()) {
+                if (req.template()) {
+                    ctx.cache().completeTemplateAddFuture(req.startCacheConfiguration().getName(),
+                        req.deploymentId());
+                }
+                else
+                    ctx.cache().completeCacheStartFuture(req, false, err);
+            }
+
+            return false;
+        }
+    }
+
+    /**
+     * @param exchangeActions Exchange actions to update.
+     * @param reqs Requests.
+     * @param topVer Topology version.
+     * @param persistedCfgs {@code True} if process start of persisted caches during cluster activation.
+     * @return Process result.
+     */
+    private CacheChangeProcessResult processCacheChangeRequests(
+        ExchangeActions exchangeActions,
+        Collection<DynamicCacheChangeRequest> reqs,
+        AffinityTopologyVersion topVer,
+        boolean persistedCfgs) {
+        CacheChangeProcessResult res = new CacheChangeProcessResult();
 
         final List<T2<DynamicCacheChangeRequest, AffinityTopologyVersion>> reqsToComplete = new ArrayList<>();
 
-        for (DynamicCacheChangeRequest req : batch.requests()) {
+        for (DynamicCacheChangeRequest req : reqs) {
             if (req.template()) {
                 CacheConfiguration ccfg = req.startCacheConfiguration();
 
@@ -347,17 +408,18 @@ class ClusterCachesInfo {
 
                     assert old == null;
 
-                    addedDescs.add(templateDesc);
+                    res.addedDescs.add(templateDesc);
                 }
 
-                ctx.cache().completeTemplateAddFuture(ccfg.getName(), req.deploymentId());
+                if (!persistedCfgs)
+                    ctx.cache().completeTemplateAddFuture(ccfg.getName(), req.deploymentId());
 
                 continue;
             }
 
             assert !req.clientStartOnly() : req;
 
-            DynamicCacheDescriptor desc = req.globalStateChange() ? null : registeredCaches.get(req.cacheName());
+            DynamicCacheDescriptor desc = registeredCaches.get(req.cacheName());
 
             boolean needExchange = false;
 
@@ -373,22 +435,32 @@ class ClusterCachesInfo {
                     if (conflictErr != null) {
                         U.warn(log, "Ignore cache start request. " + conflictErr);
 
-                        ctx.cache().completeCacheStartFuture(req, false, new IgniteCheckedException("Failed to start " +
-                            "cache. " + conflictErr));
+                        IgniteCheckedException err = new IgniteCheckedException("Failed to start " +
+                            "cache. " + conflictErr);
+
+                        if (persistedCfgs)
+                            res.errs.add(err);
+                        else
+                            ctx.cache().completeCacheStartFuture(req, false, err);
 
                         continue;
                     }
 
                     if (req.clientStartOnly()) {
+                        assert !persistedCfgs;
+
                         ctx.cache().completeCacheStartFuture(req, false, new IgniteCheckedException("Failed to start " +
                             "client cache (a cache with the given name is not started): " + req.cacheName()));
                     }
                     else {
                         SchemaOperationException err = QueryUtils.checkQueryEntityConflicts(
-                            req.startCacheConfiguration(), ctx.cache().cacheDescriptors().values());
+                            req.startCacheConfiguration(), registeredCaches.values());
 
                         if (err != null) {
-                            ctx.cache().completeCacheStartFuture(req, false, err);
+                            if (persistedCfgs)
+                                res.errs.add(err);
+                            else
+                                ctx.cache().completeCacheStartFuture(req, false, err);
 
                             continue;
                         }
@@ -430,11 +502,13 @@ class ClusterCachesInfo {
                             ccfg.getName(),
                             ccfg.getNearConfiguration() != null);
 
-                        ctx.discovery().addClientNode(req.cacheName(),
-                            req.initiatingNodeId(),
-                            req.nearCacheConfiguration() != null);
+                        if (!persistedCfgs) {
+                            ctx.discovery().addClientNode(req.cacheName(),
+                                req.initiatingNodeId(),
+                                req.nearCacheConfiguration() != null);
+                        }
 
-                        addedDescs.add(startDesc);
+                        res.addedDescs.add(startDesc);
 
                         exchangeActions.addCacheToStart(req, startDesc);
 
@@ -442,6 +516,7 @@ class ClusterCachesInfo {
                     }
                 }
                 else {
+                    assert !persistedCfgs;
                     assert req.initiatingNodeId() != null : req;
 
                     if (req.failIfExists()) {
@@ -489,8 +564,6 @@ class ClusterCachesInfo {
                     }
                 }
             }
-            else if (req.globalStateChange())
-                exchangeActions.newClusterState(req.state());
             else if (req.resetLostPartitions()) {
                 if (desc != null) {
                     needExchange = true;
@@ -559,18 +632,18 @@ class ClusterCachesInfo {
                 assert false : req;
 
             if (!needExchange) {
-                if (!clientCacheStart && req.initiatingNodeId().equals(ctx.localNodeId()))
+                if (!clientCacheStart && ctx.localNodeId().equals(req.initiatingNodeId()))
                     reqsToComplete.add(new T2<>(req, waitTopVer));
             }
             else
-                incMinorTopVer = true;
+                res.needExchange = true;
         }
 
-        if (!F.isEmpty(addedDescs)) {
-            AffinityTopologyVersion startTopVer = incMinorTopVer ? topVer.nextMinorVersion() : topVer;
+        if (!F.isEmpty(res.addedDescs)) {
+            AffinityTopologyVersion startTopVer = res.needExchange ? topVer.nextMinorVersion() : topVer;
 
-            for (DynamicCacheDescriptor desc : addedDescs) {
-                assert desc.template() || incMinorTopVer;
+            for (DynamicCacheDescriptor desc : res.addedDescs) {
+                assert desc.template() || res.needExchange;
 
                 desc.startTopologyVersion(startTopVer);
             }
@@ -602,13 +675,7 @@ class ClusterCachesInfo {
             });
         }
 
-        if (incMinorTopVer) {
-            assert !exchangeActions.empty() : exchangeActions;
-
-            batch.exchangeActions(exchangeActions);
-        }
-
-        return incMinorTopVer;
+        return res;
     }
 
     /**
@@ -669,7 +736,7 @@ class ClusterCachesInfo {
             return new CacheClientReconnectDiscoveryData(cacheGrpsInfo, cachesInfo);
         }
         else {
-            assert ctx.config().isDaemon() || joinDiscoData != null || !ctx.state().active();
+            assert ctx.config().isDaemon() || joinDiscoData != null;
 
             return joinDiscoData;
         }
@@ -720,31 +787,6 @@ class ClusterCachesInfo {
         return started != null ? started : Collections.<DynamicCacheDescriptor>emptyList();
     }
 
-    public void addJoinInfo() {
-        try {
-            Map<String, CacheConfiguration> grpCfgs = new HashMap<>();
-
-            for (CacheJoinNodeDiscoveryData.CacheInfo info : joinDiscoData.caches().values()) {
-                if (info.cacheData().config().getGroupName() == null)
-                    continue;
-
-                CacheConfiguration ccfg = grpCfgs.get(info.cacheData().config().getGroupName());
-
-                if (ccfg == null)
-                    grpCfgs.put(info.cacheData().config().getGroupName(), info.cacheData().config());
-                else
-                    validateCacheGroupConfiguration(ccfg, info.cacheData().config());
-            }
-
-            String conflictErr = processJoiningNode(joinDiscoData, ctx.localNodeId(), true);
-
-            if (conflictErr != null)
-                onJoinCacheException = new IgniteCheckedException("Failed to start configured cache. " + conflictErr);
-        }catch (IgniteCheckedException e){
-            onJoinCacheException = e;
-        }
-    }
-
     /**
      * Discovery event callback, executed from discovery thread.
      *
@@ -771,10 +813,7 @@ class ClusterCachesInfo {
 
             if (node.id().equals(ctx.discovery().localNode().id())) {
                 if (gridData == null) { // First node starts.
-                    assert joinDiscoData != null || !ctx.state().active();
-
-                    if (ctx.state().active())
-                        addJoinInfo();
+                    assert joinDiscoData != null;
 
                     initStartCachesForLocalJoin(true);
                 }
@@ -864,7 +903,7 @@ class ClusterCachesInfo {
         if (ctx.isDaemon() || data.commonData() == null)
             return;
 
-        assert joinDiscoData != null || disconnectedState() || !ctx.state().active();
+        assert joinDiscoData != null || disconnectedState();
         assert data.commonData() instanceof CacheNodeCommonDiscoveryData : data;
 
         CacheNodeCommonDiscoveryData cachesData = (CacheNodeCommonDiscoveryData)data.commonData();
@@ -965,7 +1004,7 @@ class ClusterCachesInfo {
             }
         }
 
-        gridData = new GridData(cachesData, conflictErr);
+        gridData = new GridData(joinDiscoData, cachesData, conflictErr);
 
         if (!disconnectedState())
             initStartCachesForLocalJoin(false);
@@ -977,11 +1016,20 @@ class ClusterCachesInfo {
      * @param firstNode {@code True} if first node in cluster starts.
      */
     private void initStartCachesForLocalJoin(boolean firstNode) {
-        assert locJoinStartCaches == null;
+        assert F.isEmpty(locJoinStartCaches) : locJoinStartCaches;
+
+        if (ctx.state().clusterState().transition()) {
+            joinOnTransition = true;
 
-        locJoinStartCaches = new ArrayList<>();
+            return;
+        }
 
         if (joinDiscoData != null) {
+            locJoinStartCaches = new ArrayList<>();
+            locCfgsForActivation = new HashMap<>();
+
+            boolean active = ctx.state().clusterState().active();
+
             for (DynamicCacheDescriptor desc : registeredCaches.values()) {
                 if (firstNode && !joinDiscoData.caches().containsKey(desc.cacheName()))
                     continue;
@@ -997,13 +1045,13 @@ class ClusterCachesInfo {
 
                     DynamicCacheDescriptor desc0 = new DynamicCacheDescriptor(ctx,
                         locCfg.cacheData().config(),
-                            desc.cacheType(),
-                            desc.groupDescriptor(),
-                            desc.template(),
-                            desc.receivedFrom(),
-                            desc.staticallyConfigured(),
-                            desc.sql(),
-                            desc.deploymentId(),
+                        desc.cacheType(),
+                        desc.groupDescriptor(),
+                        desc.template(),
+                        desc.receivedFrom(),
+                        desc.staticallyConfigured(),
+                        desc.sql(),
+                        desc.deploymentId(),
                         new QuerySchema(locCfg.cacheData().queryEntities()));
 
                     desc0.startTopologyVersion(desc.startTopologyVersion());
@@ -1016,14 +1064,126 @@ class ClusterCachesInfo {
                 if (locCfg != null ||
                     joinDiscoData.startCaches() ||
                     CU.affinityNode(ctx.discovery().localNode(), desc.groupDescriptor().config().getNodeFilter())) {
-                    // Move system and internal caches first.
-                    if (desc.cacheType().userCache())
-                        locJoinStartCaches.add(new T2<>(desc, nearCfg));
+                    if (active) {
+                        // Move system and internal caches first.
+                        if (desc.cacheType().userCache())
+                            locJoinStartCaches.add(new T2<>(desc, nearCfg));
+                        else
+                            locJoinStartCaches.add(0, new T2<>(desc, nearCfg));
+                    }
                     else
-                        locJoinStartCaches.add(0, new T2<>(desc, nearCfg));
+                        locCfgsForActivation.put(desc.cacheName(), new T2<>(desc.cacheConfiguration(), nearCfg));
+                }
+            }
+        }
+    }
+
+    /**
+     * @param msg Message.
+     */
+    void onStateChangeFinish(ChangeGlobalStateFinishMessage msg) {
+        if (joinOnTransition) {
+            initStartCachesForLocalJoin(false);
+
+            joinOnTransition = false;
+        }
+    }
+
+    /**
+     * @param msg Message.
+     * @param topVer Current topology version.
+     * @return Exchange action.
+     * @throws IgniteCheckedException If configuration validation failed.
+     */
+    ExchangeActions onStateChangeRequest(ChangeGlobalStateMessage msg, AffinityTopologyVersion topVer)
+        throws IgniteCheckedException {
+        ExchangeActions exchangeActions = new ExchangeActions();
+
+        if (msg.activate()) {
+            for (DynamicCacheDescriptor desc : registeredCaches.values()) {
+                desc.startTopologyVersion(topVer);
+
+                T2<CacheConfiguration, NearCacheConfiguration> locCfg = !F.isEmpty(locCfgsForActivation) ?
+                    locCfgsForActivation.get(desc.cacheName()) : null;
+
+                DynamicCacheChangeRequest req = new DynamicCacheChangeRequest(msg.requestId(),
+                    desc.cacheName(),
+                    msg.initiatorNodeId());
+
+                req.startCacheConfiguration(desc.cacheConfiguration());
+                req.cacheType(desc.cacheType());
+
+                if (locCfg != null) {
+                    if (locCfg.get1() != null)
+                        req.startCacheConfiguration(locCfg.get1());
+
+                    req.nearCacheConfiguration(locCfg.get2());
+
+                    req.locallyConfigured(true);
+                }
+
+                exchangeActions.addCacheToStart(req, desc);
+            }
+
+            for (CacheGroupDescriptor grpDesc : registeredCacheGroups().values())
+                exchangeActions.addCacheGroupToStart(grpDesc);
+
+            List<StoredCacheData> storedCfgs = msg.storedCacheConfigurations();
+
+            if (storedCfgs != null) {
+                List<DynamicCacheChangeRequest> reqs = new ArrayList<>();
+
+                IgniteUuid deplymentId = IgniteUuid.fromUuid(msg.requestId());
+
+                for (StoredCacheData storedCfg : storedCfgs) {
+                    CacheConfiguration ccfg = storedCfg.config();
+
+                    if (!registeredCaches.containsKey(ccfg.getName())) {
+                        DynamicCacheChangeRequest req = new DynamicCacheChangeRequest(msg.requestId(),
+                            ccfg.getName(),
+                            msg.initiatorNodeId());
+
+                        req.deploymentId(deplymentId);
+                        req.startCacheConfiguration(ccfg);
+                        req.cacheType(ctx.cache().cacheType(ccfg.getName()));
+                        req.schema(new QuerySchema(storedCfg.queryEntities()));
+
+                        reqs.add(req);
+                    }
+                }
+
+                CacheChangeProcessResult res = processCacheChangeRequests(exchangeActions, reqs, topVer, true);
+
+                if (!res.errs.isEmpty()) {
+                    IgniteCheckedException err = new IgniteCheckedException("Failed to activate cluster.");
+
+                    for (IgniteCheckedException err0 : res.errs)
+                        err.addSuppressed(err0);
+
+                    throw err;
                 }
             }
         }
+        else {
+            locCfgsForActivation = new HashMap<>();
+
+            for (DynamicCacheDescriptor desc : registeredCaches.values()) {
+                DynamicCacheChangeRequest req = DynamicCacheChangeRequest.stopRequest(ctx,
+                    desc.cacheName(),
+                    desc.sql(),
+                    false);
+
+                exchangeActions.addCacheToStop(req, desc);
+
+                if (ctx.discovery().cacheClientNode(ctx.discovery().localNode(), desc.cacheName()))
+                    locCfgsForActivation.put(desc.cacheName(), new T2<>((CacheConfiguration)null, (NearCacheConfiguration)null));
+            }
+
+            for (CacheGroupDescriptor grpDesc : registeredCacheGroups().values())
+                exchangeActions.addCacheGroupToStop(grpDesc, false);
+        }
+
+        return exchangeActions;
     }
 
     /**
@@ -1053,16 +1213,20 @@ class ClusterCachesInfo {
      * @param clientNodeId Client node ID.
      */
     private void processClientReconnectData(CacheClientReconnectDiscoveryData clientData, UUID clientNodeId) {
-        for (CacheClientReconnectDiscoveryData.CacheInfo cacheInfo : clientData.clientCaches().values()) {
-            String cacheName = cacheInfo.config().getName();
+        DiscoveryDataClusterState state = ctx.state().clusterState();
+
+        if (state.active() && !state.transition()) {
+            for (CacheClientReconnectDiscoveryData.CacheInfo cacheInfo : clientData.clientCaches().values()) {
+                String cacheName = cacheInfo.config().getName();
 
-            if (surviveReconnect(cacheName))
-                ctx.discovery().addClientNode(cacheName, clientNodeId, false);
-            else {
-                DynamicCacheDescriptor desc = registeredCaches.get(cacheName);
+                if (surviveReconnect(cacheName))
+                    ctx.discovery().addClientNode(cacheName, clientNodeId, false);
+                else {
+                    DynamicCacheDescriptor desc = registeredCaches.get(cacheName);
 
-                if (desc != null && desc.deploymentId().equals(cacheInfo.deploymentId()))
-                    ctx.discovery().addClientNode(cacheName, clientNodeId, cacheInfo.nearCache());
+                    if (desc != null && desc.deploymentId().equals(cacheInfo.deploymentId()))
+                        ctx.discovery().addClientNode(cacheName, clientNodeId, cacheInfo.nearCache());
+                }
             }
         }
     }
@@ -1371,6 +1535,7 @@ class ClusterCachesInfo {
      */
     void onDisconnect() {
         cachesOnDisconnect = new CachesOnDisconnect(
+            ctx.state().clusterState(),
             new HashMap<>(registeredCacheGrps),
             new HashMap<>(registeredCaches));
 
@@ -1382,57 +1547,82 @@ class ClusterCachesInfo {
     }
 
     /**
+     * @param active {@code True} if reconnected to active cluster.
+     * @param transition {@code True} if reconnected while state transition in progress.
      * @return Information about stopped caches and cache groups.
      */
-    ClusterCachesReconnectResult onReconnected() {
+    ClusterCachesReconnectResult onReconnected(boolean active, boolean transition) {
         assert disconnectedState();
 
         Set<String> stoppedCaches = new HashSet<>();
         Set<Integer> stoppedCacheGrps = new HashSet<>();
 
-        for (Map.Entry<Integer, CacheGroupDescriptor> e : cachesOnDisconnect.cacheGrps.entrySet()) {
-            CacheGroupDescriptor locDesc = e.getValue();
-
-            CacheGroupDescriptor desc;
-            boolean stopped = true;
+        if (!active) {
+            joinOnTransition = transition;
 
-            if (locDesc.sharedGroup()) {
-                desc = cacheGroupByName(locDesc.groupName());
+            if (F.isEmpty(locCfgsForActivation)) {
+                locCfgsForActivation = new HashMap<>();
 
-                if (desc != null && desc.deploymentId().equals(locDesc.deploymentId()))
-                    stopped = false;
+                for (IgniteInternalCache cache : ctx.cache().caches()) {
+                    locCfgsForActivation.put(cache.name(),
+                        new T2<>((CacheConfiguration)null, cache.configuration().getNearConfiguration()));
+                }
             }
-            else {
-                desc = nonSharedCacheGroupByCacheName(locDesc.config().getName());
 
-                if (desc != null &&
-                    (surviveReconnect(locDesc.config().getName()) || desc.deploymentId().equals(locDesc.deploymentId())))
-                    stopped = false;
-            }
+            for (Map.Entry<Integer, CacheGroupDescriptor> e : cachesOnDisconnect.cacheGrps.entrySet())
+                stoppedCacheGrps.add(e.getValue().groupId());
 
-            if (stopped)
-                stoppedCacheGrps.add(locDesc.groupId());
-            else
-                assert locDesc.groupId() == desc.groupId();
+            for (Map.Entry<String, DynamicCacheDescriptor> e : cachesOnDisconnect.caches.entrySet())
+                stoppedCaches.add(e.getKey());
         }
+        else {
+            for (Map.Entry<Integer, CacheGroupDescriptor> e : cachesOnDisconnect.cacheGrps.entrySet()) {
+                CacheGroupDescriptor locDesc = e.getValue();
 
-        for (Map.Entry<String, DynamicCacheDescriptor> e : cachesOnDisconnect.caches.entrySet()) {
-            DynamicCacheDescriptor desc = e.getValue();
+                CacheGroupDescriptor desc;
+                boolean stopped = true;
 
-            String cacheName = e.getKey();
+                if (locDesc.sharedGroup()) {
+                    desc = cacheGroupByName(locDesc.groupName());
 
-            boolean stopped;
+                    if (desc != null && desc.deploymentId().equals(locDesc.deploymentId()))
+                        stopped = false;
+                }
+                else {
+                    desc = nonSharedCacheGroupByCacheName(locDesc.config().getName());
 
-            if (!surviveReconnect(cacheName) || !ctx.state().active()) {
-                DynamicCacheDescriptor newDesc = registeredCaches.get(cacheName);
+                    if (desc != null &&
+                        (surviveReconnect(locDesc.config().getName()) || desc.deploymentId().equals(locDesc.deploymentId())))
+                        stopped = false;
+                }
 
-                stopped = newDesc == null || !desc.deploymentId().equals(newDesc.deploymentId());
+                if (stopped)
+                    stoppedCacheGrps.add(locDesc.groupId());
+                else
+                    assert locDesc.groupId() == desc.groupId();
             }
-            else
-                stopped = false;
 
-            if (stopped)
-                stoppedCaches.add(cacheName);
+            for (Map.Entry<String, DynamicCacheDescriptor> e : cachesOnDisconnect.caches.entrySet()) {
+                DynamicCacheDescriptor desc = e.getValue();
+
+                String cacheName = e.getKey();
+
+                boolean stopped;
+
+                if (!surviveReconnect(cacheName)) {
+                    DynamicCacheDescriptor newDesc = registeredCaches.get(cacheName);
+
+                    stopped = newDesc == null || !desc.deploymentId().equals(newDesc.deploymentId());
+                }
+                else
+                    stopped = false;
+
+                if (stopped)
+                    stoppedCaches.add(cacheName);
+            }
+
+            if (!cachesOnDisconnect.clusterActive())
+                initStartCachesForLocalJoin(false);
         }
 
         if (clientReconnectReqs != null) {
@@ -1450,7 +1640,7 @@ class ClusterCachesInfo {
     /**
      * @return {@code True} if client node is currently in disconnected state.
      */
-    public boolean disconnectedState() {
+    private boolean disconnectedState() {
         return cachesOnDisconnect != null;
     }
 
@@ -1465,27 +1655,23 @@ class ClusterCachesInfo {
     /**
      *
      */
-    void clearCaches() {
-        registeredCacheGrps.clear();
-
-        registeredCaches.clear();
-    }
-
-    /**
-     *
-     */
     private static class GridData {
         /** */
+        private final CacheJoinNodeDiscoveryData joinDiscoData;
+
+        /** */
         private final CacheNodeCommonDiscoveryData gridData;
 
         /** */
         private final String conflictErr;
 
         /**
+         * @param joinDiscoData Discovery data collected for local node join.
          * @param gridData Grid data.
          * @param conflictErr Cache configuration conflict error.
          */
-        GridData(CacheNodeCommonDiscoveryData gridData, String conflictErr) {
+        GridData(CacheJoinNodeDiscoveryData joinDiscoData, CacheNodeCommonDiscoveryData gridData, String conflictErr) {
+            this.joinDiscoData = joinDiscoData;
             this.gridData = gridData;
             this.conflictErr = conflictErr;
         }
@@ -1496,18 +1682,46 @@ class ClusterCachesInfo {
      */
     private static class CachesOnDisconnect {
         /** */
+        final DiscoveryDataClusterState state;
+
+        /** */
         final Map<Integer, CacheGroupDescriptor> cacheGrps;
 
         /** */
         final Map<String, DynamicCacheDescriptor> caches;
 
         /**
+         * @param state Cluster state.
          * @param cacheGrps Cache groups.
          * @param caches Caches.
          */
-        CachesOnDisconnect(Map<Integer, CacheGroupDescriptor> cacheGrps, Map<String, DynamicCacheDescriptor> caches) {
+        CachesOnDisconnect(DiscoveryDataClusterState state,
+            Map<Integer, CacheGroupDescriptor> cacheGrps,
+            Map<String, DynamicCacheDescriptor> caches) {
+            this.state = state;
             this.cacheGrps = cacheGrps;
             this.caches = caches;
         }
+
+        /**
+         * @return {@code True} if cluster was in active state.
+         */
+        boolean clusterActive() {
+            return state.active() && !state.transition();
+        }
+    }
+
+    /**
+     *
+     */
+    private static class CacheChangeProcessResult {
+        /** */
+        private boolean needExchange;
+
+        /** */
+        private final List<DynamicCacheDescriptor> addedDescs = new ArrayList<>();
+
+        /** */
+        private final List<IgniteCheckedException> errs = new ArrayList<>();
     }
 }


[15/33] ignite git commit: Added onCacheGroupStopped callback

Posted by sb...@apache.org.
Added onCacheGroupStopped callback


Project: http://git-wip-us.apache.org/repos/asf/ignite/repo
Commit: http://git-wip-us.apache.org/repos/asf/ignite/commit/77c5dc74
Tree: http://git-wip-us.apache.org/repos/asf/ignite/tree/77c5dc74
Diff: http://git-wip-us.apache.org/repos/asf/ignite/diff/77c5dc74

Branch: refs/heads/ignite-2.1.2-exchange
Commit: 77c5dc7455b9bf020dee549cc91286a897dbfdbc
Parents: 114c42e
Author: Dmitriy Govorukhin <dm...@gmail.com>
Authored: Tue Jul 4 19:40:21 2017 +0300
Committer: Alexey Goncharuk <al...@gmail.com>
Committed: Tue Jul 4 19:40:21 2017 +0300

----------------------------------------------------------------------
 .../cache/persistence/GridCacheDatabaseSharedManager.java    | 6 +++++-
 .../cache/persistence/IgniteCacheSnapshotManager.java        | 8 ++++++++
 2 files changed, 13 insertions(+), 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/77c5dc74/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/GridCacheDatabaseSharedManager.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/GridCacheDatabaseSharedManager.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/GridCacheDatabaseSharedManager.java
index 990f54c..b3ab1cd 100755
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/GridCacheDatabaseSharedManager.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/GridCacheDatabaseSharedManager.java
@@ -821,7 +821,11 @@ public class GridCacheDatabaseSharedManager extends IgniteCacheDatabaseSharedMan
         Map<PageMemoryEx, Collection<Integer>> destroyed = new HashMap<>();
 
         for (IgniteBiTuple<CacheGroupContext, Boolean> tup : stoppedGrps) {
-            PageMemoryEx pageMem = (PageMemoryEx)tup.get1().memoryPolicy().pageMemory();
+            CacheGroupContext gctx = tup.get1();
+
+            snapshotMgr.onCacheGroupStop(gctx);
+
+            PageMemoryEx pageMem = (PageMemoryEx)gctx.memoryPolicy().pageMemory();
 
             Collection<Integer> grpIds = destroyed.get(pageMem);
 

http://git-wip-us.apache.org/repos/asf/ignite/blob/77c5dc74/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/IgniteCacheSnapshotManager.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/IgniteCacheSnapshotManager.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/IgniteCacheSnapshotManager.java
index 95af487..ad804cb 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/IgniteCacheSnapshotManager.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/IgniteCacheSnapshotManager.java
@@ -26,6 +26,7 @@ import org.apache.ignite.internal.IgniteInternalFuture;
 import org.apache.ignite.internal.pagemem.FullPageId;
 import org.apache.ignite.internal.pagemem.PageMemory;
 import org.apache.ignite.internal.pagemem.snapshot.SnapshotOperation;
+import org.apache.ignite.internal.processors.cache.CacheGroupContext;
 import org.apache.ignite.internal.processors.cache.GridCacheContext;
 import org.apache.ignite.internal.processors.cache.GridCacheSharedManagerAdapter;
 import org.apache.ignite.internal.processors.cluster.IgniteChangeGlobalStateSupport;
@@ -109,6 +110,13 @@ public class IgniteCacheSnapshotManager extends GridCacheSharedManagerAdapter im
     }
 
     /**
+     * @param gctx Cctx.
+     */
+    public void onCacheGroupStop(CacheGroupContext gctx) {
+        // No-op.
+    }
+
+    /**
      *
      */
     public void onChangeTrackerPage(


[27/33] ignite git commit: Reworked cluster activation/deactivation.

Posted by sb...@apache.org.
http://git-wip-us.apache.org/repos/asf/ignite/blob/1337901f/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/preloader/GridDhtPartitionsExchangeFuture.java
----------------------------------------------------------------------
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 a1926ee..cea758a 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
@@ -36,6 +36,7 @@ import java.util.concurrent.locks.ReadWriteLock;
 import org.apache.ignite.IgniteCheckedException;
 import org.apache.ignite.IgniteLogger;
 import org.apache.ignite.cluster.ClusterNode;
+import org.apache.ignite.configuration.NearCacheConfiguration;
 import org.apache.ignite.events.CacheEvent;
 import org.apache.ignite.events.DiscoveryEvent;
 import org.apache.ignite.events.Event;
@@ -57,15 +58,14 @@ import org.apache.ignite.internal.processors.affinity.AffinityTopologyVersion;
 import org.apache.ignite.internal.processors.affinity.GridAffinityAssignmentCache;
 import org.apache.ignite.internal.processors.cache.CacheAffinityChangeMessage;
 import org.apache.ignite.internal.processors.cache.CacheGroupContext;
-import org.apache.ignite.internal.processors.cache.CacheGroupDescriptor;
 import org.apache.ignite.internal.processors.cache.CachePartitionExchangeWorkerTask;
-import org.apache.ignite.internal.processors.cache.ClusterState;
 import org.apache.ignite.internal.processors.cache.DynamicCacheChangeBatch;
 import org.apache.ignite.internal.processors.cache.DynamicCacheDescriptor;
 import org.apache.ignite.internal.processors.cache.ExchangeActions;
 import org.apache.ignite.internal.processors.cache.GridCacheContext;
 import org.apache.ignite.internal.processors.cache.GridCacheMvccCandidate;
 import org.apache.ignite.internal.processors.cache.GridCacheSharedContext;
+import org.apache.ignite.internal.processors.cache.StateChangeRequest;
 import org.apache.ignite.internal.processors.cache.distributed.dht.GridDhtTopologyFutureAdapter;
 import org.apache.ignite.internal.processors.cache.distributed.dht.GridClientPartitionTopology;
 import org.apache.ignite.internal.processors.cache.distributed.dht.GridDhtLocalPartition;
@@ -73,7 +73,8 @@ import org.apache.ignite.internal.processors.cache.distributed.dht.GridDhtPartit
 import org.apache.ignite.internal.processors.cache.distributed.dht.GridDhtPartitionTopology;
 import org.apache.ignite.internal.processors.cache.transactions.IgniteTxKey;
 import org.apache.ignite.internal.processors.cache.version.GridCacheVersion;
-import org.apache.ignite.internal.processors.cluster.GridClusterStateProcessor;
+import org.apache.ignite.internal.processors.cluster.ChangeGlobalStateFinishMessage;
+import org.apache.ignite.internal.processors.cluster.ChangeGlobalStateMessage;
 import org.apache.ignite.internal.processors.timeout.GridTimeoutObjectAdapter;
 import org.apache.ignite.internal.util.future.GridCompoundFuture;
 import org.apache.ignite.internal.util.future.GridFutureAdapter;
@@ -214,9 +215,6 @@ public class GridDhtPartitionsExchangeFuture extends GridDhtTopologyFutureAdapte
     /** Change global state exceptions. */
     private final Map<UUID, Exception> changeGlobalStateExceptions = new ConcurrentHashMap8<>();
 
-    /** This exchange for change global state. */
-    private boolean exchangeOnChangeGlobalState;
-
     /** */
     private ConcurrentMap<UUID, GridDhtPartitionsSingleMessage> msgs = new ConcurrentHashMap8<>();
 
@@ -463,10 +461,24 @@ public class GridDhtPartitionsExchangeFuture extends GridDhtTopologyFutureAdapte
     }
 
     /**
-     *
+     * @return {@code True} if cluster state change exchange.
+     */
+    private boolean stateChangeExchange() {
+        return exchActions != null && exchActions.stateChangeRequest() != null;
+    }
+
+    /**
+     * @return {@code True} if activate cluster exchange.
      */
-    public ClusterState newClusterState() {
-        return exchActions != null ? exchActions.newClusterState() : null;
+    public boolean activateCluster() {
+        return exchActions != null && exchActions.activate();
+    }
+
+    /**
+     * @return {@code True} if deactivate cluster exchange.
+     */
+    boolean deactivateCluster() {
+        return exchActions != null && exchActions.deactivate();
     }
 
     /**
@@ -519,6 +531,8 @@ public class GridDhtPartitionsExchangeFuture extends GridDhtTopologyFutureAdapte
         if (isDone())
             return;
 
+        assert !cctx.kernalContext().isDaemon();
+
         initTs = U.currentTimeMillis();
 
         U.await(evtLatch);
@@ -557,7 +571,12 @@ public class GridDhtPartitionsExchangeFuture extends GridDhtTopologyFutureAdapte
             if (discoEvt.type() == EVT_DISCOVERY_CUSTOM_EVT) {
                 DiscoveryCustomMessage msg = ((DiscoveryCustomEvent)discoEvt).customMessage();
 
-                if (msg instanceof DynamicCacheChangeBatch) {
+                if (msg instanceof ChangeGlobalStateMessage) {
+                    assert exchActions != null && !exchActions.empty();
+
+                    exchange = onClusterStateChangeRequest(crdNode);
+                }
+                else if (msg instanceof DynamicCacheChangeBatch) {
                     assert exchActions != null && !exchActions.empty();
 
                     exchange = onCacheChangeRequest(crdNode);
@@ -582,8 +601,26 @@ public class GridDhtPartitionsExchangeFuture extends GridDhtTopologyFutureAdapte
 
                         cctx.affinity().initStartedCaches(crdNode, this, receivedCaches);
                     }
-                    else
-                        cctx.cache().startCachesOnLocalJoin(topVer);
+                    else {
+                        cctx.activate();
+
+                        List<T2<DynamicCacheDescriptor, NearCacheConfiguration>> caches =
+                            cctx.cache().cachesToStartOnLocalJoin();
+
+                        if (cctx.database().persistenceEnabled() &&
+                            !cctx.kernalContext().clientNode()) {
+                            List<DynamicCacheDescriptor> startDescs = new ArrayList<>();
+
+                            if (caches != null) {
+                                for (T2<DynamicCacheDescriptor, NearCacheConfiguration> c : caches)
+                                    startDescs.add(c.get1());
+                            }
+
+                            cctx.database().readCheckpointAndRestoreMemory(startDescs);
+                        }
+
+                        cctx.cache().startCachesOnLocalJoin(caches, topVer);
+                    }
                 }
 
                 exchange = CU.clientNode(discoEvt.eventNode()) ?
@@ -710,21 +747,94 @@ public class GridDhtPartitionsExchangeFuture extends GridDhtTopologyFutureAdapte
      * @return Exchange type.
      * @throws IgniteCheckedException If failed.
      */
-    private ExchangeType onCacheChangeRequest(boolean crd) throws IgniteCheckedException {
+    private ExchangeType onClusterStateChangeRequest(boolean crd) throws IgniteCheckedException {
         assert exchActions != null && !exchActions.empty() : this;
 
-        GridClusterStateProcessor stateProc = cctx.kernalContext().state();
+        StateChangeRequest req = exchActions.stateChangeRequest();
+
+        assert req != null : exchActions;
+
+        if (req.activate()) {
+            if (log.isInfoEnabled()) {
+                log.info("Start activation process [nodeId=" + cctx.localNodeId() +
+                    ", client=" + cctx.kernalContext().clientNode() +
+                    ", topVer=" + topologyVersion() + "]");
+            }
+
+            try {
+                cctx.activate();
+
+                if (cctx.database().persistenceEnabled() && !cctx.kernalContext().clientNode()) {
+                    List<DynamicCacheDescriptor> startDescs = new ArrayList<>();
+
+                    for (ExchangeActions.ActionData startReq : exchActions.cacheStartRequests())
+                        startDescs.add(startReq.descriptor());
+
+                    cctx.database().readCheckpointAndRestoreMemory(startDescs);
+                }
+
+                cctx.affinity().onCacheChangeRequest(this, crd, exchActions);
 
-        if (exchangeOnChangeGlobalState = stateProc.changeGlobalState(exchActions, topologyVersion())) {
-            changeGlobalStateE = stateProc.onChangeGlobalState();
+                if (log.isInfoEnabled()) {
+                    log.info("Successfully activated caches [nodeId=" + cctx.localNodeId() +
+                        ", client=" + cctx.kernalContext().clientNode() +
+                        ", topVer=" + topologyVersion() + "]");
+                }
+            }
+            catch (Exception e) {
+                U.error(log, "Failed to activate node components [nodeId=" + cctx.localNodeId() +
+                    ", client=" + cctx.kernalContext().clientNode() +
+                    ", topVer=" + topologyVersion() + "]", e);
 
-            if (changeGlobalStateE != null) {
-                if (crd)
-                    changeGlobalStateExceptions.put(cctx.localNodeId(), changeGlobalStateE);
+                changeGlobalStateE = e;
 
-                return cctx.kernalContext().clientNode() ? ExchangeType.CLIENT : ExchangeType.ALL;
+                if (crd) {
+                    synchronized (this) {
+                        changeGlobalStateExceptions.put(cctx.localNodeId(), e);
+                    }
+                }
             }
         }
+        else {
+            if (log.isInfoEnabled()) {
+                log.info("Start deactivation process [nodeId=" + cctx.localNodeId() +
+                    ", client=" + cctx.kernalContext().clientNode() +
+                    ", topVer=" + topologyVersion() + "]");
+            }
+
+            try {
+                cctx.kernalContext().dataStructures().onDeActivate(cctx.kernalContext());
+
+                cctx.kernalContext().service().onDeActivate(cctx.kernalContext());
+
+                cctx.affinity().onCacheChangeRequest(this, crd, exchActions);
+
+                if (log.isInfoEnabled()) {
+                    log.info("Successfully deactivated data structures, services and caches [" +
+                        "nodeId=" + cctx.localNodeId() +
+                        ", client=" + cctx.kernalContext().clientNode() +
+                        ", topVer=" + topologyVersion() + "]");
+                }
+            }
+            catch (Exception e) {
+                U.error(log, "Failed to deactivate node components [nodeId=" + cctx.localNodeId() +
+                    ", client=" + cctx.kernalContext().clientNode() +
+                    ", topVer=" + topologyVersion() + "]", e);
+
+                changeGlobalStateE = e;
+            }
+        }
+
+        return cctx.kernalContext().clientNode() ? ExchangeType.CLIENT : ExchangeType.ALL;
+    }
+
+    /**
+     * @param crd Coordinator flag.
+     * @return Exchange type.
+     * @throws IgniteCheckedException If failed.
+     */
+    private ExchangeType onCacheChangeRequest(boolean crd) throws IgniteCheckedException {
+        assert exchActions != null && !exchActions.empty() : this;
 
         assert !exchActions.clientOnlyExchange() : exchActions;
 
@@ -1133,8 +1243,8 @@ public class GridDhtPartitionsExchangeFuture extends GridDhtTopologyFutureAdapte
         if (partHistReserved0 != null)
             m.partitionHistoryCounters(partHistReserved0);
 
-        if (exchangeOnChangeGlobalState && changeGlobalStateE != null)
-            m.setException(changeGlobalStateE);
+        if (stateChangeExchange() && changeGlobalStateE != null)
+            m.setError(changeGlobalStateE);
 
         if (log.isDebugEnabled())
             log.debug("Sending local partitions [nodeId=" + node.id() + ", exchId=" + exchId + ", msg=" + m + ']');
@@ -1160,8 +1270,8 @@ public class GridDhtPartitionsExchangeFuture extends GridDhtTopologyFutureAdapte
             partHistSuppliers,
             partsToReload);
 
-        if (exchangeOnChangeGlobalState && !F.isEmpty(changeGlobalStateExceptions))
-            m.setExceptionsMap(changeGlobalStateExceptions);
+        if (stateChangeExchange() && !F.isEmpty(changeGlobalStateExceptions))
+            m.setErrorsMap(changeGlobalStateExceptions);
 
         return m;
     }
@@ -1175,9 +1285,10 @@ public class GridDhtPartitionsExchangeFuture extends GridDhtTopologyFutureAdapte
 
         assert !nodes.contains(cctx.localNode());
 
-        if (log.isDebugEnabled())
+        if (log.isDebugEnabled()) {
             log.debug("Sending full partition map [nodeIds=" + F.viewReadOnly(nodes, F.node2id()) +
                 ", exchId=" + exchId + ", msg=" + m + ']');
+        }
 
         for (ClusterNode node : nodes) {
             try {
@@ -1291,8 +1402,8 @@ public class GridDhtPartitionsExchangeFuture extends GridDhtTopologyFutureAdapte
         if (exchActions != null && err == null)
             exchActions.completeRequestFutures(cctx);
 
-        if (exchangeOnChangeGlobalState && err == null)
-            cctx.kernalContext().state().onExchangeDone();
+        if (stateChangeExchange() && err == null)
+            cctx.kernalContext().state().onStateChangeExchangeDone(exchActions.stateChangeRequest());
 
         Map<T2<Integer, Integer>, Long> localReserved = partHistSuppliers.getReservations(cctx.localNodeId());
 
@@ -1368,6 +1479,7 @@ public class GridDhtPartitionsExchangeFuture extends GridDhtTopologyFutureAdapte
         crd = null;
         partReleaseFut = null;
         changeGlobalStateE = null;
+        exchActions = null;
     }
 
     /**
@@ -1444,8 +1556,8 @@ public class GridDhtPartitionsExchangeFuture extends GridDhtTopologyFutureAdapte
 
                     pendingSingleUpdates++;
 
-                    if (exchangeOnChangeGlobalState && msg.getException() != null)
-                        changeGlobalStateExceptions.put(node.id(), msg.getException());
+                    if (stateChangeExchange() && msg.getError() != null)
+                        changeGlobalStateExceptions.put(node.id(), msg.getError());
 
                     allReceived = remaining.isEmpty();
                 }
@@ -1457,7 +1569,7 @@ public class GridDhtPartitionsExchangeFuture extends GridDhtTopologyFutureAdapte
         if (updateSingleMap) {
             try {
                 // Do not update partition map, in case cluster transitioning to inactive state.
-                if (!exchangeOnChangeGlobalState || exchActions.newClusterState() != ClusterState.INACTIVE)
+                if (!deactivateCluster())
                     updatePartitionSingleMap(node, msg);
             }
             finally {
@@ -1735,18 +1847,16 @@ public class GridDhtPartitionsExchangeFuture extends GridDhtTopologyFutureAdapte
                 }
             }
 
-            if (discoEvt.type() == EVT_NODE_JOINED) {
-                if (cctx.kernalContext().state().active())
-                    assignPartitionsStates();
-            }
+            if (discoEvt.type() == EVT_NODE_JOINED)
+                assignPartitionsStates();
             else if (discoEvt.type() == EVT_DISCOVERY_CUSTOM_EVT) {
                 assert discoEvt instanceof DiscoveryCustomEvent;
 
+                if (activateCluster())
+                    assignPartitionsStates();
+
                 if (((DiscoveryCustomEvent)discoEvt).customMessage() instanceof DynamicCacheChangeBatch) {
                     if (exchActions != null) {
-                        if (exchActions.newClusterState() == ClusterState.ACTIVE)
-                            assignPartitionsStates();
-
                         Set<String> caches = exchActions.cachesToResetLostPartitions();
 
                         if (!F.isEmpty(caches))
@@ -1783,13 +1893,34 @@ public class GridDhtPartitionsExchangeFuture extends GridDhtTopologyFutureAdapte
                     nodes = new ArrayList<>(srvNodes);
                 }
 
+                IgniteCheckedException err = null;
+
+                if (stateChangeExchange()) {
+                    StateChangeRequest req = exchActions.stateChangeRequest();
+
+                    assert req != null : exchActions;
+
+                    boolean stateChangeErr = false;
+
+                    if (!F.isEmpty(changeGlobalStateExceptions)) {
+                        stateChangeErr = true;
+
+                        err = new IgniteCheckedException("Cluster state change failed.");
+
+                        cctx.kernalContext().state().onStateChangeError(changeGlobalStateExceptions, req);
+                    }
+
+                    boolean active = !stateChangeErr && req.activate();
+
+                    ChangeGlobalStateFinishMessage msg = new ChangeGlobalStateFinishMessage(req.requestId(), active);
+
+                    cctx.discovery().sendCustomEvent(msg);
+                }
+
                 if (!nodes.isEmpty())
                     sendAllPartitions(nodes);
 
-                if (exchangeOnChangeGlobalState && !F.isEmpty(changeGlobalStateExceptions))
-                    cctx.kernalContext().state().onFullResponseMessage(changeGlobalStateExceptions);
-
-                onDone(exchangeId().topologyVersion());
+                onDone(exchangeId().topologyVersion(), err);
             }
         }
         catch (IgniteCheckedException e) {
@@ -1898,7 +2029,7 @@ public class GridDhtPartitionsExchangeFuture extends GridDhtTopologyFutureAdapte
      * @param msg Message.
      */
     private void processMessage(ClusterNode node, GridDhtPartitionsFullMessage msg) {
-        assert msg.exchangeId().equals(exchId) : msg;
+        assert exchId.equals(msg.exchangeId()) : msg;
         assert msg.lastVersion() != null : msg;
 
         synchronized (this) {
@@ -1919,10 +2050,15 @@ public class GridDhtPartitionsExchangeFuture extends GridDhtTopologyFutureAdapte
 
         updatePartitionFullMap(msg);
 
-        if (exchangeOnChangeGlobalState && !F.isEmpty(msg.getExceptionsMap()))
-            cctx.kernalContext().state().onFullResponseMessage(msg.getExceptionsMap());
+        IgniteCheckedException err = null;
 
-        onDone(exchId.topologyVersion());
+        if (stateChangeExchange() && !F.isEmpty(msg.getErrorsMap())) {
+            err = new IgniteCheckedException("Cluster state change failed");
+
+            cctx.kernalContext().state().onStateChangeError(msg.getErrorsMap(), exchActions.stateChangeRequest());
+        }
+
+        onDone(exchId.topologyVersion(), err);
     }
 
     /**
@@ -2143,7 +2279,7 @@ public class GridDhtPartitionsExchangeFuture extends GridDhtTopologyFutureAdapte
                         }
 
                         if (crd0.isLocal()) {
-                            if (exchangeOnChangeGlobalState && changeGlobalStateE != null)
+                            if (stateChangeExchange() && changeGlobalStateE != null)
                                 changeGlobalStateExceptions.put(crd0.id(), changeGlobalStateE);
 
                             if (allReceived) {

http://git-wip-us.apache.org/repos/asf/ignite/blob/1337901f/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/preloader/GridDhtPartitionsFullMessage.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/preloader/GridDhtPartitionsFullMessage.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/preloader/GridDhtPartitionsFullMessage.java
index 0beb935..75609b8 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/preloader/GridDhtPartitionsFullMessage.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/preloader/GridDhtPartitionsFullMessage.java
@@ -90,10 +90,10 @@ public class GridDhtPartitionsFullMessage extends GridDhtPartitionsAbstractMessa
     /** Exceptions. */
     @GridToStringInclude
     @GridDirectTransient
-    private Map<UUID, Exception> exs;
+    private Map<UUID, Exception> errs;
 
     /**  */
-    private byte[] exsBytes;
+    private byte[] errsBytes;
 
     /** */
     @GridDirectTransient
@@ -224,17 +224,17 @@ public class GridDhtPartitionsFullMessage extends GridDhtPartitionsAbstractMessa
     }
 
     /**
-     *
+     * @return Errors map.
      */
-    public Map<UUID, Exception> getExceptionsMap() {
-        return exs;
+    @Nullable Map<UUID, Exception> getErrorsMap() {
+        return errs;
     }
 
     /**
-     * @param exs Exs.
+     * @param errs Errors map.
      */
-    public void setExceptionsMap(Map<UUID, Exception> exs) {
-        this.exs = new HashMap<>(exs);
+    void setErrorsMap(Map<UUID, Exception> errs) {
+        this.errs = new HashMap<>(errs);
     }
 
     /** {@inheritDoc} */
@@ -245,14 +245,14 @@ public class GridDhtPartitionsFullMessage extends GridDhtPartitionsAbstractMessa
             (partCntrs != null && partCntrsBytes == null) ||
             (partHistSuppliers != null && partHistSuppliersBytes == null) ||
             (partsToReload != null && partsToReloadBytes == null) ||
-            (exs != null && exsBytes == null);
+            (errs != null && errsBytes == null);
 
         if (marshal) {
             byte[] partsBytes0 = null;
             byte[] partCntrsBytes0 = null;
             byte[] partHistSuppliersBytes0 = null;
             byte[] partsToReloadBytes0 = null;
-            byte[] exsBytes0 = null;
+            byte[] errsBytes0 = null;
 
             if (parts != null && partsBytes == null)
                 partsBytes0 = U.marshal(ctx, parts);
@@ -266,8 +266,8 @@ public class GridDhtPartitionsFullMessage extends GridDhtPartitionsAbstractMessa
             if (partsToReload != null && partsToReloadBytes == null)
                 partsToReloadBytes0 = U.marshal(ctx, partsToReload);
 
-            if (exs != null && exsBytes == null)
-                exsBytes0 = U.marshal(ctx, exs);
+            if (errs != null && errsBytes == null)
+                errsBytes0 = U.marshal(ctx, errs);
 
             if (compress) {
                 assert !compressed();
@@ -277,13 +277,13 @@ public class GridDhtPartitionsFullMessage extends GridDhtPartitionsAbstractMessa
                     byte[] partCntrsBytesZip = U.zip(partCntrsBytes0);
                     byte[] partHistSuppliersBytesZip = U.zip(partHistSuppliersBytes0);
                     byte[] partsToReloadBytesZip = U.zip(partsToReloadBytes0);
-                    byte[] exsBytesZip = U.zip(exsBytes0);
+                    byte[] exsBytesZip = U.zip(errsBytes0);
 
                     partsBytes0 = partsBytesZip;
                     partCntrsBytes0 = partCntrsBytesZip;
                     partHistSuppliersBytes0 = partHistSuppliersBytesZip;
                     partsToReloadBytes0 = partsToReloadBytesZip;
-                    exsBytes0 = exsBytesZip;
+                    errsBytes0 = exsBytesZip;
 
                     compressed(true);
                 }
@@ -296,7 +296,7 @@ public class GridDhtPartitionsFullMessage extends GridDhtPartitionsAbstractMessa
             partCntrsBytes = partCntrsBytes0;
             partHistSuppliersBytes = partHistSuppliersBytes0;
             partsToReloadBytes = partsToReloadBytes0;
-            exsBytes = exsBytes0;
+            errsBytes = errsBytes0;
         }
     }
 
@@ -379,15 +379,15 @@ public class GridDhtPartitionsFullMessage extends GridDhtPartitionsAbstractMessa
         if (partCntrs == null)
             partCntrs = new IgniteDhtPartitionCountersMap();
 
-        if (exsBytes != null && exs == null) {
+        if (errsBytes != null && errs == null) {
             if (compressed())
-                exs = U.unmarshalZip(ctx.marshaller(), exsBytes, U.resolveClassLoader(ldr, ctx.gridConfig()));
+                errs = U.unmarshalZip(ctx.marshaller(), errsBytes, U.resolveClassLoader(ldr, ctx.gridConfig()));
             else
-                exs = U.unmarshal(ctx, exsBytes, U.resolveClassLoader(ldr, ctx.gridConfig()));
+                errs = U.unmarshal(ctx, errsBytes, U.resolveClassLoader(ldr, ctx.gridConfig()));
         }
 
-        if (exs == null)
-            exs = new HashMap<>();
+        if (errs == null)
+            errs = new HashMap<>();
     }
 
     /** {@inheritDoc} */
@@ -412,7 +412,7 @@ public class GridDhtPartitionsFullMessage extends GridDhtPartitionsAbstractMessa
                 writer.incrementState();
 
             case 6:
-                if (!writer.writeByteArray("exsBytes", exsBytes))
+                if (!writer.writeByteArray("errsBytes", errsBytes))
                     return false;
 
                 writer.incrementState();
@@ -472,7 +472,7 @@ public class GridDhtPartitionsFullMessage extends GridDhtPartitionsAbstractMessa
                 reader.incrementState();
 
             case 6:
-                exsBytes = reader.readByteArray("exsBytes");
+                errsBytes = reader.readByteArray("errsBytes");
 
                 if (!reader.isLastRead())
                     return false;

http://git-wip-us.apache.org/repos/asf/ignite/blob/1337901f/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/preloader/GridDhtPartitionsSingleMessage.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/preloader/GridDhtPartitionsSingleMessage.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/preloader/GridDhtPartitionsSingleMessage.java
index 1e5ea14..b4d25c4 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/preloader/GridDhtPartitionsSingleMessage.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/preloader/GridDhtPartitionsSingleMessage.java
@@ -76,10 +76,10 @@ public class GridDhtPartitionsSingleMessage extends GridDhtPartitionsAbstractMes
     /** Exception. */
     @GridToStringInclude
     @GridDirectTransient
-    private Exception ex;
+    private Exception err;
 
     /** */
-    private byte[] exBytes;
+    private byte[] errBytes;
 
     /** */
     private boolean client;
@@ -220,15 +220,15 @@ public class GridDhtPartitionsSingleMessage extends GridDhtPartitionsAbstractMes
     /**
      * @param ex Exception.
      */
-    public void setException(Exception ex) {
-        this.ex = ex;
+    public void setError(Exception ex) {
+        this.err = ex;
     }
 
     /**
-     *
+     * @return Not null exception if exchange processing failed.
      */
-    public Exception getException() {
-        return ex;
+    @Nullable public Exception getError() {
+        return err;
     }
 
     /** {@inheritDoc}
@@ -239,13 +239,13 @@ public class GridDhtPartitionsSingleMessage extends GridDhtPartitionsAbstractMes
         boolean marshal = (parts != null && partsBytes == null) ||
             (partCntrs != null && partCntrsBytes == null) ||
             (partHistCntrs != null && partHistCntrsBytes == null) ||
-            (ex != null && exBytes == null);
+            (err != null && errBytes == null);
 
         if (marshal) {
             byte[] partsBytes0 = null;
             byte[] partCntrsBytes0 = null;
             byte[] partHistCntrsBytes0 = null;
-            byte[] exBytes0 = null;
+            byte[] errBytes0 = null;
 
             if (parts != null && partsBytes == null)
                 partsBytes0 = U.marshal(ctx, parts);
@@ -256,8 +256,8 @@ public class GridDhtPartitionsSingleMessage extends GridDhtPartitionsAbstractMes
             if (partHistCntrs != null && partHistCntrsBytes == null)
                 partHistCntrsBytes0 = U.marshal(ctx, partHistCntrs);
 
-            if (ex != null && exBytes == null)
-                exBytes0 = U.marshal(ctx, ex);
+            if (err != null && errBytes == null)
+                errBytes0 = U.marshal(ctx, err);
 
             if (compress) {
                 assert !compressed();
@@ -266,12 +266,12 @@ public class GridDhtPartitionsSingleMessage extends GridDhtPartitionsAbstractMes
                     byte[] partsBytesZip = U.zip(partsBytes0);
                     byte[] partCntrsBytesZip = U.zip(partCntrsBytes0);
                     byte[] partHistCntrsBytesZip = U.zip(partHistCntrsBytes0);
-                    byte[] exBytesZip = U.zip(exBytes0);
+                    byte[] exBytesZip = U.zip(errBytes0);
 
                     partsBytes0 = partsBytesZip;
                     partCntrsBytes0 = partCntrsBytesZip;
                     partHistCntrsBytes0 = partHistCntrsBytesZip;
-                    exBytes0 = exBytesZip;
+                    errBytes0 = exBytesZip;
 
                     compressed(true);
                 }
@@ -283,7 +283,7 @@ public class GridDhtPartitionsSingleMessage extends GridDhtPartitionsAbstractMes
             partsBytes = partsBytes0;
             partCntrsBytes = partCntrsBytes0;
             partHistCntrsBytes = partHistCntrsBytes0;
-            exBytes = exBytes0;
+            errBytes = errBytes0;
         }
     }
 
@@ -312,11 +312,11 @@ public class GridDhtPartitionsSingleMessage extends GridDhtPartitionsAbstractMes
                 partHistCntrs = U.unmarshal(ctx, partHistCntrsBytes, U.resolveClassLoader(ldr, ctx.gridConfig()));
         }
 
-        if (exBytes != null && ex == null) {
+        if (errBytes != null && err == null) {
             if (compressed())
-                ex = U.unmarshalZip(ctx.marshaller(), exBytes, U.resolveClassLoader(ldr, ctx.gridConfig()));
+                err = U.unmarshalZip(ctx.marshaller(), errBytes, U.resolveClassLoader(ldr, ctx.gridConfig()));
             else
-                ex = U.unmarshal(ctx, exBytes, U.resolveClassLoader(ldr, ctx.gridConfig()));
+                err = U.unmarshal(ctx, errBytes, U.resolveClassLoader(ldr, ctx.gridConfig()));
         }
 
         if (dupPartsData != null) {
@@ -368,7 +368,7 @@ public class GridDhtPartitionsSingleMessage extends GridDhtPartitionsAbstractMes
                 writer.incrementState();
 
             case 7:
-                if (!writer.writeByteArray("exBytes", exBytes))
+                if (!writer.writeByteArray("errBytes", errBytes))
                     return false;
 
                 writer.incrementState();
@@ -424,7 +424,7 @@ public class GridDhtPartitionsSingleMessage extends GridDhtPartitionsAbstractMes
                 reader.incrementState();
 
             case 7:
-                exBytes = reader.readByteArray("exBytes");
+                errBytes = reader.readByteArray("errBytes");
 
                 if (!reader.isLastRead())
                     return false;

http://git-wip-us.apache.org/repos/asf/ignite/blob/1337901f/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/preloader/GridDhtPreloader.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/preloader/GridDhtPreloader.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/preloader/GridDhtPreloader.java
index 66b5987..2b18c24 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/preloader/GridDhtPreloader.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/preloader/GridDhtPreloader.java
@@ -183,7 +183,7 @@ public class GridDhtPreloader extends GridCachePreloaderAdapter {
         // No assignments for disabled preloader.
         GridDhtPartitionTopology top = grp.topology();
 
-        if (!grp.rebalanceEnabled() || !grp.shared().kernalContext().state().active())
+        if (!grp.rebalanceEnabled())
             return new GridDhtPreloaderAssignments(exchFut, top.topologyVersion());
 
         int partCnt = grp.affinity().partitions();

http://git-wip-us.apache.org/repos/asf/ignite/blob/1337901f/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/near/GridNearGetFuture.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/near/GridNearGetFuture.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/near/GridNearGetFuture.java
index 062ff6c..a49812e 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/near/GridNearGetFuture.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/near/GridNearGetFuture.java
@@ -45,8 +45,6 @@ import org.apache.ignite.internal.processors.cache.distributed.dht.CacheDistribu
 import org.apache.ignite.internal.processors.cache.distributed.dht.GridDhtCacheAdapter;
 import org.apache.ignite.internal.processors.cache.distributed.dht.GridDhtFuture;
 import org.apache.ignite.internal.processors.cache.distributed.dht.GridDhtInvalidPartitionException;
-import org.apache.ignite.internal.processors.cache.distributed.dht.GridDhtLocalPartition;
-import org.apache.ignite.internal.processors.cache.distributed.dht.GridDhtPartitionState;
 import org.apache.ignite.internal.processors.cache.transactions.IgniteTxLocalEx;
 import org.apache.ignite.internal.processors.cache.version.GridCacheVersion;
 import org.apache.ignite.internal.util.GridLeanMap;

http://git-wip-us.apache.org/repos/asf/ignite/blob/1337901f/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/near/GridNearTxPrepareRequest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/near/GridNearTxPrepareRequest.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/near/GridNearTxPrepareRequest.java
index 41e6d70..29c7aad 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/near/GridNearTxPrepareRequest.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/near/GridNearTxPrepareRequest.java
@@ -26,7 +26,6 @@ import java.util.UUID;
 import org.apache.ignite.internal.processors.affinity.AffinityTopologyVersion;
 import org.apache.ignite.internal.processors.cache.GridCacheContext;
 import org.apache.ignite.internal.processors.cache.distributed.GridDistributedTxPrepareRequest;
-import org.apache.ignite.internal.processors.cache.transactions.IgniteInternalTx;
 import org.apache.ignite.internal.processors.cache.transactions.IgniteTxEntry;
 import org.apache.ignite.internal.util.tostring.GridToStringExclude;
 import org.apache.ignite.internal.util.typedef.F;

http://git-wip-us.apache.org/repos/asf/ignite/blob/1337901f/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/GridCacheDatabaseSharedManager.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/GridCacheDatabaseSharedManager.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/GridCacheDatabaseSharedManager.java
index b3ab1cd..c700ef4 100755
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/GridCacheDatabaseSharedManager.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/GridCacheDatabaseSharedManager.java
@@ -65,7 +65,6 @@ import org.apache.ignite.IgniteException;
 import org.apache.ignite.IgniteLogger;
 import org.apache.ignite.IgniteSystemProperties;
 import org.apache.ignite.PersistenceMetrics;
-import org.apache.ignite.configuration.CacheConfiguration;
 import org.apache.ignite.configuration.DataPageEvictionMode;
 import org.apache.ignite.configuration.IgniteConfiguration;
 import org.apache.ignite.configuration.MemoryConfiguration;
@@ -100,11 +99,13 @@ import org.apache.ignite.internal.pagemem.wal.record.delta.PartitionDestroyRecor
 import org.apache.ignite.internal.pagemem.wal.record.delta.PartitionMetaStateRecord;
 import org.apache.ignite.internal.processors.affinity.AffinityTopologyVersion;
 import org.apache.ignite.internal.processors.cache.CacheGroupContext;
-import org.apache.ignite.internal.processors.cache.ClusterState;
 import org.apache.ignite.internal.processors.cache.DynamicCacheDescriptor;
 import org.apache.ignite.internal.processors.cache.GridCacheContext;
 import org.apache.ignite.internal.processors.cache.GridCacheSharedContext;
 import org.apache.ignite.internal.processors.cache.StoredCacheData;
+import org.apache.ignite.internal.processors.cache.distributed.dht.GridDhtLocalPartition;
+import org.apache.ignite.internal.processors.cache.distributed.dht.GridDhtPartitionState;
+import org.apache.ignite.internal.processors.cache.distributed.dht.preloader.GridDhtPartitionsExchangeFuture;
 import org.apache.ignite.internal.processors.cache.persistence.file.FilePageStoreManager;
 import org.apache.ignite.internal.processors.cache.persistence.pagemem.CheckpointMetricsTracker;
 import org.apache.ignite.internal.processors.cache.persistence.pagemem.PageMemoryEx;
@@ -113,9 +114,6 @@ import org.apache.ignite.internal.processors.cache.persistence.tree.io.PageIO;
 import org.apache.ignite.internal.processors.cache.persistence.tree.io.PagePartitionMetaIO;
 import org.apache.ignite.internal.processors.cache.persistence.wal.FileWALPointer;
 import org.apache.ignite.internal.processors.cache.persistence.wal.crc.PureJavaCrc32;
-import org.apache.ignite.internal.processors.cache.distributed.dht.GridDhtLocalPartition;
-import org.apache.ignite.internal.processors.cache.distributed.dht.GridDhtPartitionState;
-import org.apache.ignite.internal.processors.cache.distributed.dht.preloader.GridDhtPartitionsExchangeFuture;
 import org.apache.ignite.internal.processors.port.GridPortRecord;
 import org.apache.ignite.internal.util.GridConcurrentHashSet;
 import org.apache.ignite.internal.util.GridMultiCollectionWrapper;
@@ -352,9 +350,9 @@ public class GridCacheDatabaseSharedManager extends IgniteCacheDatabaseSharedMan
 
     /** {@inheritDoc} */
     @Override protected void start0() throws IgniteCheckedException {
-        snapshotMgr = cctx.snapshot();
+        super.start0();
 
-        assert !cctx.kernalContext().state().active() : "Cluster with persistent must starting as inactive.";
+        snapshotMgr = cctx.snapshot();
 
         if (!cctx.kernalContext().clientNode()) {
             IgnitePageStoreManager store = cctx.pageStore();
@@ -371,15 +369,13 @@ public class GridCacheDatabaseSharedManager extends IgniteCacheDatabaseSharedMan
             fileLockHolder = new FileLockHolder(storeMgr.workDir().getPath(), cctx.kernalContext(), log);
 
             persStoreMetrics.wal(cctx.wal());
-
-            registrateMetricsMBean();
         }
     }
 
     /**
-     *
+     * @throws IgniteCheckedException If failed.
      */
-    @Override public void initDataBase() throws IgniteCheckedException {
+    private void initDataBase() throws IgniteCheckedException {
         Long cpBufSize = persistenceCfg.getCheckpointingPageBufferSize();
 
         if (persistenceCfg.getCheckpointingThreads() > 1)
@@ -432,8 +428,6 @@ public class GridCacheDatabaseSharedManager extends IgniteCacheDatabaseSharedMan
         }
 
         checkpointPageBufSize = cpBufSize;
-
-        super.start0();
     }
 
     /** {@inheritDoc} */
@@ -442,58 +436,7 @@ public class GridCacheDatabaseSharedManager extends IgniteCacheDatabaseSharedMan
     }
 
     /** {@inheritDoc} */
-    @Override protected void onKernalStart0(boolean reconnect) throws IgniteCheckedException {
-        if (reconnect || cctx.kernalContext().clientNode() || !cctx.kernalContext().state().active())
-            return;
-
-        initDataBase();
-
-        initCachesAndRestoreMemory();
-    }
-
-    /**
-     *
-     */
-    private void initCachesAndRestoreMemory() throws IgniteCheckedException {
-        Collection<String> cacheNames = new HashSet<>();
-
-        // TODO IGNITE-5075 group descriptors.
-        for (CacheConfiguration ccfg : cctx.kernalContext().config().getCacheConfiguration()) {
-            if (CU.isSystemCache(ccfg.getName())) {
-                storeMgr.initializeForCache(
-                    cctx.cache().cacheDescriptors().get(ccfg.getName()).groupDescriptor(),
-                    new StoredCacheData(ccfg)
-                );
-
-                cacheNames.add(ccfg.getName());
-            }
-        }
-
-        for (CacheConfiguration ccfg : cctx.kernalContext().config().getCacheConfiguration())
-            if (!CU.isSystemCache(ccfg.getName())) {
-                DynamicCacheDescriptor cacheDesc = cctx.cache().cacheDescriptors().get(ccfg.getName());
-
-                if (cacheDesc != null)
-                    storeMgr.initializeForCache(
-                        cacheDesc.groupDescriptor(),
-                        new StoredCacheData(ccfg)
-                    );
-
-                cacheNames.add(ccfg.getName());
-            }
-
-        for (StoredCacheData cacheData : cctx.pageStore().readCacheConfigurations().values()) {
-            if (!cacheNames.contains(cacheData.config().getName()))
-                storeMgr.initializeForCache(
-                    cctx.cache().cacheDescriptors().get(
-                        cacheData.config().getName()).groupDescriptor(), cacheData);
-        }
-
-        readCheckpointAndRestoreMemory();
-    }
-
-    /** {@inheritDoc} */
-    @Override public void onActivate(GridKernalContext kctx) throws IgniteCheckedException {
+    @Override public void onActivate(GridKernalContext ctx) throws IgniteCheckedException {
         if (log.isDebugEnabled())
             log.debug("Activate database manager [id=" + cctx.localNodeId() +
                 " topVer=" + cctx.discovery().topologyVersionEx() + " ]");
@@ -504,16 +447,13 @@ public class GridCacheDatabaseSharedManager extends IgniteCacheDatabaseSharedMan
             initDataBase();
 
             registrateMetricsMBean();
-
-            initCachesAndRestoreMemory();
         }
 
-        if (log.isDebugEnabled())
-            log.debug("Restore state after activation [nodeId=" + cctx.localNodeId() + " ]");
+        super.onActivate(ctx);
     }
 
     /** {@inheritDoc} */
-    @Override public void onDeActivate(GridKernalContext kctx) throws IgniteCheckedException {
+    @Override public void onDeActivate(GridKernalContext kctx) {
         if (log.isDebugEnabled())
             log.debug("DeActivate database manager [id=" + cctx.localNodeId() +
                 " topVer=" + cctx.discovery().topologyVersionEx() + " ]");
@@ -530,7 +470,7 @@ public class GridCacheDatabaseSharedManager extends IgniteCacheDatabaseSharedMan
     }
 
     /**
-     *
+     * @throws IgniteCheckedException If failed.
      */
     private void registrateMetricsMBean() throws IgniteCheckedException {
         try {
@@ -564,13 +504,18 @@ public class GridCacheDatabaseSharedManager extends IgniteCacheDatabaseSharedMan
         }
     }
 
-    /**
-     *
-     */
-    private void readCheckpointAndRestoreMemory() throws IgniteCheckedException {
+    /** {@inheritDoc} */
+    @Override public void readCheckpointAndRestoreMemory(List<DynamicCacheDescriptor> cachesToStart) throws IgniteCheckedException {
         checkpointReadLock();
 
         try {
+            if (!F.isEmpty(cachesToStart)) {
+                for (DynamicCacheDescriptor desc : cachesToStart) {
+                    if (CU.affinityNode(cctx.localNode(), desc.cacheConfiguration().getNodeFilter()))
+                        storeMgr.initializeForCache(desc.groupDescriptor(), new StoredCacheData(desc.cacheConfiguration()));
+                }
+            }
+
             CheckpointStatus status = readCheckpointStatus();
 
             // First, bring memory to the last consistent checkpoint state if needed.
@@ -774,13 +719,10 @@ public class GridCacheDatabaseSharedManager extends IgniteCacheDatabaseSharedMan
 
         boolean isSrvNode = !cctx.kernalContext().clientNode();
 
-        boolean clusterStatusActive = cctx.kernalContext().state().active();
-
-        boolean clusterInTransitionStateToActive = fut.newClusterState() == ClusterState.ACTIVE;
+        boolean clusterInTransitionStateToActive = fut.activateCluster();
 
         // Before local node join event.
-        if (clusterInTransitionStateToActive ||
-            (joinEvt && locNode && isSrvNode && clusterStatusActive))
+        if (clusterInTransitionStateToActive || (joinEvt && locNode && isSrvNode))
             restoreState();
 
         if (cctx.kernalContext().query().moduleEnabled()) {
@@ -1579,9 +1521,10 @@ public class GridCacheDatabaseSharedManager extends IgniteCacheDatabaseSharedMan
         Map<T2<Integer, Integer>, T2<Integer, Long>> partStates
     ) throws IgniteCheckedException {
         for (CacheGroupContext grp : cctx.cache().cacheGroups()) {
-            if (grp.isLocal())
+            if (grp.isLocal() || !grp.affinityNode()) {
                 // Local cache has no partitions and its states.
                 continue;
+            }
 
             int grpId = grp.groupId();
 

http://git-wip-us.apache.org/repos/asf/ignite/blob/1337901f/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/GridCacheOffheapManager.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/GridCacheOffheapManager.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/GridCacheOffheapManager.java
index 4e322b9..7a38b61 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/GridCacheOffheapManager.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/GridCacheOffheapManager.java
@@ -135,10 +135,7 @@ public class GridCacheOffheapManager extends IgniteCacheOffheapManagerImpl imple
     /** {@inheritDoc} */
     @Override protected CacheDataStore createCacheDataStore0(final int p)
         throws IgniteCheckedException {
-        GridCacheDatabaseSharedManager dbMgr = (GridCacheDatabaseSharedManager)ctx.database();
-
-        boolean exists = ctx.pageStore() != null
-            && ctx.pageStore().exists(grp.groupId(), p);
+        boolean exists = ctx.pageStore() != null && ctx.pageStore().exists(grp.groupId(), p);
 
         return new GridCacheDataStore(p, exists);
     }

http://git-wip-us.apache.org/repos/asf/ignite/blob/1337901f/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/IgniteCacheDatabaseSharedManager.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/IgniteCacheDatabaseSharedManager.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/IgniteCacheDatabaseSharedManager.java
index f04c278..c5f174c 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/IgniteCacheDatabaseSharedManager.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/IgniteCacheDatabaseSharedManager.java
@@ -22,6 +22,7 @@ import java.util.ArrayList;
 import java.util.Collection;
 import java.util.Collections;
 import java.util.HashSet;
+import java.util.List;
 import java.util.Map;
 import java.util.Set;
 import javax.management.JMException;
@@ -41,8 +42,10 @@ import org.apache.ignite.internal.mem.unsafe.UnsafeMemoryProvider;
 import org.apache.ignite.internal.pagemem.PageMemory;
 import org.apache.ignite.internal.pagemem.impl.PageMemoryNoStoreImpl;
 import org.apache.ignite.internal.processors.cache.CacheGroupContext;
+import org.apache.ignite.internal.processors.cache.DynamicCacheDescriptor;
 import org.apache.ignite.internal.processors.cache.GridCacheMapEntry;
 import org.apache.ignite.internal.processors.cache.GridCacheSharedManagerAdapter;
+import org.apache.ignite.internal.processors.cache.distributed.dht.preloader.GridDhtPartitionsExchangeFuture;
 import org.apache.ignite.internal.processors.cache.persistence.evict.FairFifoPageEvictionTracker;
 import org.apache.ignite.internal.processors.cache.persistence.evict.NoOpPageEvictionTracker;
 import org.apache.ignite.internal.processors.cache.persistence.evict.PageEvictionTracker;
@@ -51,7 +54,6 @@ import org.apache.ignite.internal.processors.cache.persistence.evict.RandomLruPa
 import org.apache.ignite.internal.processors.cache.persistence.freelist.FreeList;
 import org.apache.ignite.internal.processors.cache.persistence.freelist.FreeListImpl;
 import org.apache.ignite.internal.processors.cache.persistence.tree.reuse.ReuseList;
-import org.apache.ignite.internal.processors.cache.distributed.dht.preloader.GridDhtPartitionsExchangeFuture;
 import org.apache.ignite.internal.processors.cluster.IgniteChangeGlobalStateSupport;
 import org.apache.ignite.internal.util.typedef.F;
 import org.apache.ignite.internal.util.typedef.internal.LT;
@@ -100,13 +102,6 @@ public class IgniteCacheDatabaseSharedManager extends GridCacheSharedManagerAdap
         if (cctx.kernalContext().clientNode() && cctx.kernalContext().config().getMemoryConfiguration() == null)
             return;
 
-        init();
-    }
-
-    /**
-     * @throws IgniteCheckedException If failed.
-     */
-    public void init() throws IgniteCheckedException {
         MemoryConfiguration memCfg = cctx.kernalContext().config().getMemoryConfiguration();
 
         assert memCfg != null;
@@ -114,14 +109,6 @@ public class IgniteCacheDatabaseSharedManager extends GridCacheSharedManagerAdap
         validateConfiguration(memCfg);
 
         pageSize = memCfg.getPageSize();
-
-        initPageMemoryPolicies(memCfg);
-
-        registerMetricsMBeans();
-
-        startMemoryPolicies();
-
-        initPageMemoryDataStructures(memCfg);
     }
 
     /**
@@ -149,12 +136,12 @@ public class IgniteCacheDatabaseSharedManager extends GridCacheSharedManagerAdap
     ) {
         try {
             U.registerMBean(
-                    cfg.getMBeanServer(),
-                    cfg.getIgniteInstanceName(),
-                    "MemoryMetrics",
-                    memPlcCfg.getName(),
-                    new MemoryMetricsMXBeanImpl(memMetrics, memPlcCfg),
-                    MemoryMetricsMXBean.class);
+                cfg.getMBeanServer(),
+                cfg.getIgniteInstanceName(),
+                "MemoryMetrics",
+                memPlcCfg.getName(),
+                new MemoryMetricsMXBeanImpl(memMetrics, memPlcCfg),
+                MemoryMetricsMXBean.class);
         }
         catch (JMException e) {
             U.error(log, "Failed to register MBean for MemoryMetrics with name: '" + memMetrics.getName() + "'", e);
@@ -163,6 +150,7 @@ public class IgniteCacheDatabaseSharedManager extends GridCacheSharedManagerAdap
 
     /**
      * @param dbCfg Database config.
+     * @throws IgniteCheckedException If failed.
      */
     protected void initPageMemoryDataStructures(MemoryConfiguration dbCfg) throws IgniteCheckedException {
         freeListMap = U.newHashMap(memPlcMap.size());
@@ -554,13 +542,6 @@ public class IgniteCacheDatabaseSharedManager extends GridCacheSharedManagerAdap
     }
 
     /**
-     * @throws IgniteCheckedException If failed.
-     */
-    public void initDataBase() throws IgniteCheckedException {
-        // No-op.
-    }
-
-    /**
      * @return collection of all configured {@link MemoryPolicy policies}.
      */
     public Collection<MemoryPolicy> memoryPolicies() {
@@ -592,6 +573,14 @@ public class IgniteCacheDatabaseSharedManager extends GridCacheSharedManagerAdap
     }
 
     /**
+     * @param cachesToStart Started caches.
+     * @throws IgniteCheckedException If failed.
+     */
+    public void readCheckpointAndRestoreMemory(List<DynamicCacheDescriptor> cachesToStart) throws IgniteCheckedException {
+        // No-op.
+    }
+
+    /**
      * @param memPlcName Name of {@link MemoryPolicy} to obtain {@link MemoryMetrics} for.
      * @return {@link MemoryMetrics} snapshot for specified {@link MemoryPolicy} or {@code null} if
      * no {@link MemoryPolicy} is configured for specified name.
@@ -947,11 +936,24 @@ public class IgniteCacheDatabaseSharedManager extends GridCacheSharedManagerAdap
 
     /** {@inheritDoc} */
     @Override public void onActivate(GridKernalContext kctx) throws IgniteCheckedException {
-        start0();
+        if (cctx.kernalContext().clientNode() && cctx.kernalContext().config().getMemoryConfiguration() == null)
+            return;
+
+        MemoryConfiguration memCfg = cctx.kernalContext().config().getMemoryConfiguration();
+
+        assert memCfg != null;
+
+        initPageMemoryPolicies(memCfg);
+
+        registerMetricsMBeans();
+
+        startMemoryPolicies();
+
+        initPageMemoryDataStructures(memCfg);
     }
 
     /** {@inheritDoc} */
-    @Override public void onDeActivate(GridKernalContext kctx) throws IgniteCheckedException {
+    @Override public void onDeActivate(GridKernalContext kctx) {
         stop0(false);
     }
 

http://git-wip-us.apache.org/repos/asf/ignite/blob/1337901f/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/IgniteCacheSnapshotManager.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/IgniteCacheSnapshotManager.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/IgniteCacheSnapshotManager.java
index ad804cb..cce6f55 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/IgniteCacheSnapshotManager.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/IgniteCacheSnapshotManager.java
@@ -124,7 +124,7 @@ public class IgniteCacheSnapshotManager extends GridCacheSharedManagerAdapter im
         FullPageId fullId,
         PageMemory pageMem
     ) throws IgniteCheckedException {
-
+        // No-op.
     }
 
     /**
@@ -135,14 +135,16 @@ public class IgniteCacheSnapshotManager extends GridCacheSharedManagerAdapter im
         ByteBuffer pageBuf,
         Integer tag
     ) throws IgniteCheckedException {
-
+        // No-op.
     }
 
+    /** {@inheritDoc} */
     @Override public void onActivate(GridKernalContext kctx) throws IgniteCheckedException {
-
+        // No-op.
     }
 
-    @Override public void onDeActivate(GridKernalContext kctx) throws IgniteCheckedException {
-
+    /** {@inheritDoc} */
+    @Override public void onDeActivate(GridKernalContext kctx) {
+        // No-op.
     }
 }

http://git-wip-us.apache.org/repos/asf/ignite/blob/1337901f/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/file/FilePageStoreManager.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/file/FilePageStoreManager.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/file/FilePageStoreManager.java
index f908512..28bf6e4 100755
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/file/FilePageStoreManager.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/file/FilePageStoreManager.java
@@ -162,7 +162,7 @@ public class FilePageStoreManager extends GridCacheSharedManagerAdapter implemen
     }
 
     /** {@inheritDoc} */
-    @Override public void onDeActivate(GridKernalContext kctx) throws IgniteCheckedException {
+    @Override public void onDeActivate(GridKernalContext kctx) {
         if (log.isDebugEnabled())
             log.debug("DeActivate page store manager [id=" + cctx.localNodeId() +
                 " topVer=" + cctx.discovery().topologyVersionEx() + " ]");
@@ -208,18 +208,17 @@ public class FilePageStoreManager extends GridCacheSharedManagerAdapter implemen
 
     /** {@inheritDoc} */
     @Override public void storeCacheData(
-        CacheGroupDescriptor grpDesc,
         StoredCacheData cacheData
     ) throws IgniteCheckedException {
 
-        File cacheWorkDir = cacheWorkDirectory(grpDesc, cacheData.config());
+        File cacheWorkDir = cacheWorkDirectory(cacheData.config());
         File file;
 
         checkAndInitCacheWorkDir(cacheWorkDir);
 
         assert cacheWorkDir.exists() : "Work directory does not exist: " + cacheWorkDir;
 
-        if (grpDesc.sharedGroup())
+        if (cacheData.config().getGroupName() != null)
             file = new File(cacheWorkDir, cacheData.config().getName() + CACHE_DATA_FILENAME);
         else
             file = new File(cacheWorkDir, CACHE_DATA_FILENAME);
@@ -333,14 +332,13 @@ public class FilePageStoreManager extends GridCacheSharedManagerAdapter implemen
     }
 
     /**
-     * @param grpDesc Cache group descriptor.
      * @param ccfg Cache configuration.
      * @return Cache work directory.
      */
-    private File cacheWorkDirectory(CacheGroupDescriptor grpDesc, CacheConfiguration ccfg) {
+    private File cacheWorkDirectory(CacheConfiguration ccfg) {
         String dirName;
 
-        if (grpDesc.sharedGroup())
+        if (ccfg.getGroupName() != null)
             dirName = CACHE_GRP_DIR_PREFIX + ccfg.getGroupName();
         else
             dirName = CACHE_DIR_PREFIX + ccfg.getName();
@@ -357,7 +355,7 @@ public class FilePageStoreManager extends GridCacheSharedManagerAdapter implemen
     private CacheStoreHolder initForCache(CacheGroupDescriptor grpDesc, CacheConfiguration ccfg) throws IgniteCheckedException {
         assert !grpDesc.sharedGroup() || ccfg.getGroupName() != null : ccfg.getName();
 
-        File cacheWorkDir = cacheWorkDirectory(grpDesc, ccfg);
+        File cacheWorkDir = cacheWorkDirectory(ccfg);
 
         boolean dirExisted = checkAndInitCacheWorkDir(cacheWorkDir);
 

http://git-wip-us.apache.org/repos/asf/ignite/blob/1337901f/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/wal/FileWriteAheadLogManager.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/wal/FileWriteAheadLogManager.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/wal/FileWriteAheadLogManager.java
index f877a14..8993112 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/wal/FileWriteAheadLogManager.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/wal/FileWriteAheadLogManager.java
@@ -288,14 +288,6 @@ public class FileWriteAheadLogManager extends GridCacheSharedManagerAdapter impl
         }
     }
 
-    /** {@inheritDoc} */
-    @Override protected void onKernalStart0(boolean reconnect) throws IgniteCheckedException {
-        super.onKernalStart0(reconnect);
-
-        if (!cctx.kernalContext().clientNode() && cctx.kernalContext().state().active())
-            archiver.start();
-    }
-
     /**
      * @return Consistent ID.
      */

http://git-wip-us.apache.org/repos/asf/ignite/blob/1337901f/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/query/GridCacheDistributedQueryManager.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/query/GridCacheDistributedQueryManager.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/query/GridCacheDistributedQueryManager.java
index 63228a0..7f859a2 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/query/GridCacheDistributedQueryManager.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/query/GridCacheDistributedQueryManager.java
@@ -560,7 +560,7 @@ public class GridCacheDistributedQueryManager<K, V> extends GridCacheQueryManage
 
             final Object topic = topic(cctx.nodeId(), req.id());
 
-            cctx.io().addOrderedCacheHandler(topic, resHnd);
+            cctx.io().addOrderedCacheHandler(cctx.shared(), topic, resHnd);
 
             fut.listen(new CI1<IgniteInternalFuture<?>>() {
                 @Override public void apply(IgniteInternalFuture<?> fut) {
@@ -744,7 +744,7 @@ public class GridCacheDistributedQueryManager<K, V> extends GridCacheQueryManage
 
             final Object topic = topic(cctx.nodeId(), req.id());
 
-            cctx.io().addOrderedCacheHandler(topic, resHnd);
+            cctx.io().addOrderedCacheHandler(cctx.shared(), topic, resHnd);
 
             fut.listen(new CI1<IgniteInternalFuture<?>>() {
                 @Override public void apply(IgniteInternalFuture<?> fut) {

http://git-wip-us.apache.org/repos/asf/ignite/blob/1337901f/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/store/GridCacheStoreManagerAdapter.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/store/GridCacheStoreManagerAdapter.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/store/GridCacheStoreManagerAdapter.java
index 62ead23..8ff2f5a 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/store/GridCacheStoreManagerAdapter.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/store/GridCacheStoreManagerAdapter.java
@@ -34,7 +34,6 @@ import javax.cache.integration.CacheWriterException;
 import org.apache.ignite.IgniteCheckedException;
 import org.apache.ignite.IgniteException;
 import org.apache.ignite.cache.store.CacheStore;
-import org.apache.ignite.cache.store.CacheStore;
 import org.apache.ignite.cache.store.CacheStoreSession;
 import org.apache.ignite.cache.store.CacheStoreSessionListener;
 import org.apache.ignite.cache.store.jdbc.CacheJdbcPojoStore;

http://git-wip-us.apache.org/repos/asf/ignite/blob/1337901f/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/version/GridCacheVersionManager.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/version/GridCacheVersionManager.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/version/GridCacheVersionManager.java
index bc2e49a..269925d 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/version/GridCacheVersionManager.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/version/GridCacheVersionManager.java
@@ -88,12 +88,6 @@ public class GridCacheVersionManager extends GridCacheSharedManagerAdapter {
     }
 
     /** {@inheritDoc} */
-    @Override protected void onKernalStart0(boolean reconnect) throws IgniteCheckedException {
-        for (ClusterNode n : cctx.discovery().remoteNodes())
-            onReceived(n.id(), n.metrics().getLastDataVersion());
-    }
-
-    /** {@inheritDoc} */
     @Override protected void stop0(boolean cancel) {
         cctx.gridEvents().removeLocalEventListener(discoLsnr, EVT_NODE_METRICS_UPDATED);
     }

http://git-wip-us.apache.org/repos/asf/ignite/blob/1337901f/modules/core/src/main/java/org/apache/ignite/internal/processors/cacheobject/IgniteCacheObjectProcessor.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cacheobject/IgniteCacheObjectProcessor.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cacheobject/IgniteCacheObjectProcessor.java
index 1dd47ed..dad6728 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cacheobject/IgniteCacheObjectProcessor.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cacheobject/IgniteCacheObjectProcessor.java
@@ -44,11 +44,6 @@ public interface IgniteCacheObjectProcessor extends GridProcessor {
     public void onContinuousProcessorStarted(GridKernalContext ctx) throws IgniteCheckedException;
 
     /**
-     * @throws IgniteCheckedException If failed.
-     */
-    public void onUtilityCacheStarted() throws IgniteCheckedException;
-
-    /**
      * @param typeName Type name.
      * @return Type ID.
      */

http://git-wip-us.apache.org/repos/asf/ignite/blob/1337901f/modules/core/src/main/java/org/apache/ignite/internal/processors/cacheobject/IgniteCacheObjectProcessorImpl.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cacheobject/IgniteCacheObjectProcessorImpl.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cacheobject/IgniteCacheObjectProcessorImpl.java
index 67e14dc..70711e5 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cacheobject/IgniteCacheObjectProcessorImpl.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cacheobject/IgniteCacheObjectProcessorImpl.java
@@ -307,11 +307,6 @@ public class IgniteCacheObjectProcessorImpl extends GridProcessorAdapter impleme
     }
 
     /** {@inheritDoc} */
-    @Override public void onUtilityCacheStarted() throws IgniteCheckedException {
-        // No-op.
-    }
-
-    /** {@inheritDoc} */
     @Override public int typeId(String typeName) {
         return 0;
     }

http://git-wip-us.apache.org/repos/asf/ignite/blob/1337901f/modules/core/src/main/java/org/apache/ignite/internal/processors/cluster/ChangeGlobalStateFinishMessage.java
----------------------------------------------------------------------
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
new file mode 100644
index 0000000..0771198
--- /dev/null
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cluster/ChangeGlobalStateFinishMessage.java
@@ -0,0 +1,86 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.ignite.internal.processors.cluster;
+
+import java.util.UUID;
+import org.apache.ignite.internal.managers.discovery.DiscoveryCustomMessage;
+import org.apache.ignite.internal.util.typedef.internal.S;
+import org.apache.ignite.lang.IgniteUuid;
+import org.jetbrains.annotations.Nullable;
+
+/**
+ *
+ */
+public class ChangeGlobalStateFinishMessage implements DiscoveryCustomMessage {
+    /** */
+    private static final long serialVersionUID = 0L;
+
+    /** Custom message ID. */
+    private IgniteUuid id = IgniteUuid.randomUuid();
+
+    /** State change request ID. */
+    private UUID reqId;
+
+    /** New cluster state. */
+    private boolean clusterActive;
+
+    /**
+     * @param reqId State change request ID.
+     * @param clusterActive New cluster state.
+     */
+    public ChangeGlobalStateFinishMessage(UUID reqId, boolean clusterActive) {
+        assert reqId != null;
+
+        this.reqId = reqId;
+        this.clusterActive = clusterActive;
+    }
+
+    /**
+     * @return State change request ID.
+     */
+    public UUID requestId() {
+        return reqId;
+    }
+
+    /**
+     * @return New cluster state.
+     */
+    public boolean clusterActive() {
+        return clusterActive;
+    }
+
+    /** {@inheritDoc} */
+    @Override public IgniteUuid id() {
+        return id;
+    }
+
+    /** {@inheritDoc} */
+    @Nullable @Override public DiscoveryCustomMessage ackMessage() {
+        return null;
+    }
+
+    /** {@inheritDoc} */
+    @Override public boolean isMutable() {
+        return false;
+    }
+
+    /** {@inheritDoc} */
+    @Override public String toString() {
+        return S.toString(ChangeGlobalStateFinishMessage.class, this);
+    }
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/1337901f/modules/core/src/main/java/org/apache/ignite/internal/processors/cluster/ChangeGlobalStateMessage.java
----------------------------------------------------------------------
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
new file mode 100644
index 0000000..6579399
--- /dev/null
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cluster/ChangeGlobalStateMessage.java
@@ -0,0 +1,140 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.ignite.internal.processors.cluster;
+
+import java.util.List;
+import java.util.UUID;
+import org.apache.ignite.internal.managers.discovery.DiscoveryCustomMessage;
+import org.apache.ignite.internal.processors.cache.ExchangeActions;
+import org.apache.ignite.internal.processors.cache.StoredCacheData;
+import org.apache.ignite.internal.util.tostring.GridToStringExclude;
+import org.apache.ignite.internal.util.typedef.internal.S;
+import org.apache.ignite.lang.IgniteUuid;
+import org.jetbrains.annotations.Nullable;
+
+/**
+ * Message represent request for change cluster global state.
+ */
+public class ChangeGlobalStateMessage implements DiscoveryCustomMessage {
+    /** */
+    private static final long serialVersionUID = 0L;
+
+    /** Custom message ID. */
+    private IgniteUuid id = IgniteUuid.randomUuid();
+
+    /** Request ID */
+    private UUID reqId;
+
+    /** Initiator node ID. */
+    private UUID initiatingNodeId;
+
+    /** If true activate else deactivate. */
+    private boolean activate;
+
+    /** Configurations read from persistent store. */
+    private List<StoredCacheData> storedCfgs;
+
+    /** */
+    @GridToStringExclude
+    private transient ExchangeActions exchangeActions;
+
+    /**
+     * @param reqId State change request ID.
+     * @param initiatingNodeId Node initiated state change.
+     * @param storedCfgs Configurations read from persistent store.
+     * @param activate New cluster state.
+     */
+    public ChangeGlobalStateMessage(
+        UUID reqId,
+        UUID initiatingNodeId,
+        @Nullable List<StoredCacheData> storedCfgs,
+        boolean activate
+    ) {
+        assert reqId != null;
+        assert initiatingNodeId != null;
+
+        this.reqId = reqId;
+        this.initiatingNodeId = initiatingNodeId;
+        this.storedCfgs = storedCfgs;
+        this.activate = activate;
+    }
+
+    /**
+     * @return Configurations read from persistent store..
+     */
+    @Nullable public List<StoredCacheData> storedCacheConfigurations() {
+        return storedCfgs;
+    }
+
+    /**
+     * @return Cache updates to be executed on exchange. If {@code null} exchange is not needed.
+     */
+    @Nullable public ExchangeActions exchangeActions() {
+        return exchangeActions;
+    }
+
+    /**
+     * @param exchangeActions Cache updates to be executed on exchange.
+     */
+    void exchangeActions(ExchangeActions exchangeActions) {
+        assert exchangeActions != null && !exchangeActions.empty() : exchangeActions;
+
+        this.exchangeActions = exchangeActions;
+    }
+
+    /** {@inheritDoc} */
+    @Override public IgniteUuid id() {
+        return id;
+    }
+
+    /** {@inheritDoc} */
+    @Nullable @Override public DiscoveryCustomMessage ackMessage() {
+        return null;
+    }
+
+    /** {@inheritDoc} */
+    @Override public boolean isMutable() {
+        return false;
+    }
+
+   /**
+    * @return Node initiated state change.
+    */
+    public UUID initiatorNodeId() {
+        return initiatingNodeId;
+    }
+
+    /**
+     * @return New cluster state.
+     */
+    public boolean activate() {
+        return activate;
+    }
+
+    /**
+     * @return State change request ID.
+     */
+    public UUID requestId() {
+        return reqId;
+    }
+
+    /** {@inheritDoc} */
+    @Override public String toString() {
+        return S.toString(ChangeGlobalStateMessage.class, this);
+    }
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/1337901f/modules/core/src/main/java/org/apache/ignite/internal/processors/cluster/ClusterProcessor.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cluster/ClusterProcessor.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cluster/ClusterProcessor.java
index ad95a78..dc503fb 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cluster/ClusterProcessor.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cluster/ClusterProcessor.java
@@ -270,7 +270,6 @@ public class ClusterProcessor extends GridProcessorAdapter {
         dataBag.addNodeSpecificData(CLUSTER_PROC.ordinal(), getDiscoveryData());
     }
 
-
     /**
      * @return Discovery data.
      */
@@ -314,7 +313,7 @@ public class ClusterProcessor extends GridProcessorAdapter {
     }
 
     /** {@inheritDoc} */
-    @Override public void onKernalStart() throws IgniteCheckedException {
+    @Override public void onKernalStart(boolean active) throws IgniteCheckedException {
         if (notifyEnabled.get()) {
             try {
                 verChecker = new GridUpdateNotifier(ctx.igniteInstanceName(),

http://git-wip-us.apache.org/repos/asf/ignite/blob/1337901f/modules/core/src/main/java/org/apache/ignite/internal/processors/cluster/DiscoveryDataClusterState.java
----------------------------------------------------------------------
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
new file mode 100644
index 0000000..71bf90b
--- /dev/null
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cluster/DiscoveryDataClusterState.java
@@ -0,0 +1,157 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.ignite.internal.processors.cluster;
+
+import java.io.Serializable;
+import java.util.Set;
+import java.util.UUID;
+import org.apache.ignite.internal.processors.affinity.AffinityTopologyVersion;
+import org.apache.ignite.internal.util.tostring.GridToStringExclude;
+import org.apache.ignite.internal.util.tostring.GridToStringInclude;
+import org.apache.ignite.internal.util.typedef.F;
+import org.apache.ignite.internal.util.typedef.internal.S;
+import org.jetbrains.annotations.Nullable;
+
+/**
+ * Discovery data related to cluster state.
+ */
+public class DiscoveryDataClusterState implements Serializable {
+    /** */
+    private static final long serialVersionUID = 0L;
+
+    /** */
+    private final boolean active;
+
+    /** */
+    private final UUID transitionReqId;
+
+    /** Topology version for state change exchange. */
+    @GridToStringInclude
+    private final AffinityTopologyVersion transitionTopVer;
+
+    /** Nodes participating in state change exchange. */
+    @GridToStringExclude
+    private final Set<UUID> transitionNodes;
+
+    /** Local flag for state transition result (global state is updated asynchronously by custom message). */
+    private transient volatile Boolean transitionRes;
+
+    /**
+     * @param active Current status.
+     * @return State instance.
+     */
+    static DiscoveryDataClusterState createState(boolean active) {
+        return new DiscoveryDataClusterState(active, null, null, null);
+    }
+
+    /**
+     * @param active New status.
+     * @param transitionReqId State change request ID.
+     * @param transitionTopVer State change topology version.
+     * @param transitionNodes Nodes participating in state change exchange.
+     * @return State instance.
+     */
+    static DiscoveryDataClusterState createTransitionState(boolean active,
+        UUID transitionReqId,
+        AffinityTopologyVersion transitionTopVer,
+        Set<UUID> transitionNodes) {
+        assert transitionReqId != null;
+        assert transitionTopVer != null;
+        assert !F.isEmpty(transitionNodes) : transitionNodes;
+
+        return new DiscoveryDataClusterState(active, transitionReqId, transitionTopVer, transitionNodes);
+    }
+
+    /**
+     * @param active New state.
+     * @param transitionReqId State change request ID.
+     * @param transitionTopVer State change topology version.
+     * @param transitionNodes Nodes participating in state change exchange.
+     */
+    private DiscoveryDataClusterState(boolean active,
+        @Nullable UUID transitionReqId,
+        @Nullable AffinityTopologyVersion transitionTopVer,
+        @Nullable Set<UUID> transitionNodes) {
+        this.active = active;
+        this.transitionReqId = transitionReqId;
+        this.transitionTopVer = transitionTopVer;
+        this.transitionNodes = transitionNodes;
+    }
+
+    /**
+     * @return Local flag for state transition result (global state is updated asynchronously by custom message).
+     */
+    @Nullable public Boolean transitionResult() {
+        return transitionRes;
+    }
+
+    /**
+     * Discovery cluster state is changed asynchronously by discovery message, this methods changes local status
+     * for public API calls.
+     *
+     * @param reqId Request ID.
+     * @param active New cluster state.
+     */
+    public void setTransitionResult(UUID reqId, boolean active) {
+        if (reqId.equals(transitionReqId)) {
+            assert transitionRes == null : this;
+
+            transitionRes = active;
+        }
+    }
+
+    /**
+     * @return State change request ID.
+     */
+    public UUID transitionRequestId() {
+        return transitionReqId;
+    }
+
+    /**
+     * @return {@code True} if state change is in progress.
+     */
+    public boolean transition() {
+        return transitionReqId != null;
+    }
+
+    /**
+     * @return State change exchange version.
+     */
+    public AffinityTopologyVersion transitionTopologyVersion() {
+        return transitionTopVer;
+    }
+
+    /**
+     * @return Current cluster state (or new state in case when transition is in progress).
+     */
+    public boolean active() {
+        return active;
+    }
+
+    /**
+     * @return Nodes participating in state change exchange.
+     */
+    public Set<UUID> transitionNodes() {
+        return transitionNodes;
+    }
+
+    /** {@inheritDoc} */
+    @Override public String toString() {
+        return S.toString(DiscoveryDataClusterState.class, this);
+    }
+}


[26/33] ignite git commit: Reworked cluster activation/deactivation.

Posted by sb...@apache.org.
http://git-wip-us.apache.org/repos/asf/ignite/blob/1337901f/modules/core/src/main/java/org/apache/ignite/internal/processors/cluster/GridClusterStateProcessor.java
----------------------------------------------------------------------
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 d57c720..8cea13f 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,49 +18,34 @@
 package org.apache.ignite.internal.processors.cluster;
 
 import java.util.ArrayList;
-import java.util.Arrays;
-import java.util.Collection;
 import java.util.HashMap;
 import java.util.HashSet;
 import java.util.List;
 import java.util.Map;
 import java.util.Set;
 import java.util.UUID;
-import java.util.concurrent.ConcurrentHashMap;
 import java.util.concurrent.atomic.AtomicReference;
 import org.apache.ignite.Ignite;
 import org.apache.ignite.IgniteCheckedException;
 import org.apache.ignite.IgniteCompute;
-import org.apache.ignite.IgniteException;
 import org.apache.ignite.IgniteLogger;
 import org.apache.ignite.cluster.ClusterNode;
-import org.apache.ignite.configuration.CacheConfiguration;
 import org.apache.ignite.events.DiscoveryEvent;
 import org.apache.ignite.events.Event;
 import org.apache.ignite.internal.GridKernalContext;
 import org.apache.ignite.internal.IgniteInternalFuture;
-import org.apache.ignite.internal.IgniteInterruptedCheckedException;
 import org.apache.ignite.internal.cluster.ClusterGroupAdapter;
-import org.apache.ignite.internal.managers.discovery.CustomEventListener;
+import org.apache.ignite.internal.cluster.ClusterTopologyCheckedException;
+import org.apache.ignite.internal.managers.discovery.DiscoCache;
 import org.apache.ignite.internal.managers.eventstorage.GridLocalEventListener;
-import org.apache.ignite.internal.pagemem.store.IgnitePageStoreManager;
 import org.apache.ignite.internal.processors.GridProcessorAdapter;
 import org.apache.ignite.internal.processors.affinity.AffinityTopologyVersion;
-import org.apache.ignite.internal.processors.cache.CacheClientReconnectDiscoveryData;
-import org.apache.ignite.internal.processors.cache.CacheData;
-import org.apache.ignite.internal.processors.cache.CacheJoinNodeDiscoveryData;
-import org.apache.ignite.internal.processors.cache.CacheJoinNodeDiscoveryData.CacheInfo;
-import org.apache.ignite.internal.processors.cache.CacheNodeCommonDiscoveryData;
-import org.apache.ignite.internal.processors.cache.ChangeGlobalStateMessage;
-import org.apache.ignite.internal.processors.cache.ClusterState;
-import org.apache.ignite.internal.processors.cache.DynamicCacheChangeBatch;
-import org.apache.ignite.internal.processors.cache.DynamicCacheChangeRequest;
 import org.apache.ignite.internal.processors.cache.ExchangeActions;
 import org.apache.ignite.internal.processors.cache.GridCacheProcessor;
 import org.apache.ignite.internal.processors.cache.GridCacheSharedContext;
 import org.apache.ignite.internal.processors.cache.GridChangeGlobalStateMessageResponse;
+import org.apache.ignite.internal.processors.cache.StateChangeRequest;
 import org.apache.ignite.internal.processors.cache.StoredCacheData;
-import org.apache.ignite.internal.processors.query.QuerySchema;
 import org.apache.ignite.internal.util.future.GridFinishedFuture;
 import org.apache.ignite.internal.util.future.GridFutureAdapter;
 import org.apache.ignite.internal.util.tostring.GridToStringExclude;
@@ -72,34 +57,27 @@ import org.apache.ignite.internal.util.typedef.internal.S;
 import org.apache.ignite.internal.util.typedef.internal.U;
 import org.apache.ignite.lang.IgniteFuture;
 import org.apache.ignite.lang.IgniteRunnable;
-import org.apache.ignite.lang.IgniteUuid;
 import org.apache.ignite.resources.IgniteInstanceResource;
 import org.apache.ignite.spi.discovery.DiscoveryDataBag;
-import org.apache.ignite.spi.discovery.DiscoveryDataBag.JoiningNodeDiscoveryData;
 import org.jetbrains.annotations.Nullable;
 
 import static org.apache.ignite.events.EventType.EVT_NODE_FAILED;
 import static org.apache.ignite.events.EventType.EVT_NODE_LEFT;
-import static org.apache.ignite.internal.GridComponent.DiscoveryDataExchangeType.CACHE_PROC;
 import static org.apache.ignite.internal.GridComponent.DiscoveryDataExchangeType.STATE_PROC;
 import static org.apache.ignite.internal.managers.communication.GridIoPolicy.SYSTEM_POOL;
-import static org.apache.ignite.internal.processors.cache.ClusterState.ACTIVE;
-import static org.apache.ignite.internal.processors.cache.ClusterState.INACTIVE;
-import static org.apache.ignite.internal.processors.cache.ClusterState.TRANSITION;
-import static org.apache.ignite.internal.processors.cache.DynamicCacheChangeRequest.stopRequest;
 
 /**
  *
  */
 public class GridClusterStateProcessor extends GridProcessorAdapter {
-    /** Global status. */
-    private volatile ClusterState globalState;
-
-    /** Action context. */
-    private volatile ChangeGlobalStateContext lastCgsCtx;
+    /** */
+    private volatile DiscoveryDataClusterState globalState;
 
     /** Local action future. */
-    private final AtomicReference<GridChangeGlobalStateFuture> cgsLocFut = new AtomicReference<>();
+    private final AtomicReference<GridChangeGlobalStateFuture> stateChangeFut = new AtomicReference<>();
+
+    /** Future initialized if node joins when cluster state change is in progress. */
+    private TransitionOnJoinWaitFuture joinFut;
 
     /** Process. */
     @GridToStringExclude
@@ -109,12 +87,6 @@ public class GridClusterStateProcessor extends GridProcessorAdapter {
     @GridToStringExclude
     private GridCacheSharedContext<?, ?> sharedCtx;
 
-    /** */
-    private final ConcurrentHashMap<String, CacheInfo> cacheData = new ConcurrentHashMap<>();
-
-    /** */
-    private volatile CacheJoinNodeDiscoveryData localCacheData;
-
     /** Listener. */
     private final GridLocalEventListener lsr = new GridLocalEventListener() {
         @Override public void onEvent(Event evt) {
@@ -124,14 +96,15 @@ public class GridClusterStateProcessor extends GridProcessorAdapter {
 
             assert e.type() == EVT_NODE_LEFT || e.type() == EVT_NODE_FAILED : this;
 
-            final GridChangeGlobalStateFuture f = cgsLocFut.get();
+            final GridChangeGlobalStateFuture f = stateChangeFut.get();
 
-            if (f != null)
+            if (f != null) {
                 f.initFut.listen(new CI1<IgniteInternalFuture<?>>() {
                     @Override public void apply(IgniteInternalFuture<?> fut) {
-                        f.onDiscoveryEvent(e);
+                        f.onNodeLeft(e);
                     }
                 });
+            }
         }
     };
 
@@ -142,531 +115,417 @@ public class GridClusterStateProcessor extends GridProcessorAdapter {
         super(ctx);
     }
 
-    /** {@inheritDoc} */
-    @Override public void start() throws IgniteCheckedException {
-        // Start first node as inactive if persistent enable.
-        globalState = ctx.config().isPersistentStoreEnabled() ? INACTIVE :
-            ctx.config().isActiveOnStart() ? ACTIVE : INACTIVE;
-
-        ctx.discovery().setCustomEventListener(
-            ChangeGlobalStateMessage.class, new CustomEventListener<ChangeGlobalStateMessage>() {
-                @Override public void onCustomEvent(
-                    AffinityTopologyVersion topVer, ClusterNode snd, ChangeGlobalStateMessage msg) {
-                    assert topVer != null;
-                    assert snd != null;
-                    assert msg != null;
-
-                    boolean activate = msg.activate();
-
-                    ChangeGlobalStateContext actx = lastCgsCtx;
-
-                    if (actx != null && globalState == TRANSITION) {
-                        GridChangeGlobalStateFuture f = cgsLocFut.get();
-
-                        if (log.isDebugEnabled())
-                            log.debug("Concurrent " + prettyStr(activate) + " [id=" +
-                                ctx.localNodeId() + " topVer=" + topVer + " actx=" + actx + ", msg=" + msg + "]");
-
-                        if (f != null && f.requestId.equals(msg.requestId()))
-                            f.onDone(new IgniteCheckedException(
-                                "Concurrent change state, now in progress=" + (activate)
-                                    + ", initiatingNodeId=" + actx.initiatingNodeId
-                                    + ", you try=" + (prettyStr(activate)) + ", locNodeId=" + ctx.localNodeId()
-                            ));
-
-                        msg.concurrentChangeState();
-                    }
-                    else {
-                        if (log.isInfoEnabled())
-                            log.info("Create " + prettyStr(activate) + " context [id=" +
-                                ctx.localNodeId() + " topVer=" + topVer + ", reqId=" +
-                                msg.requestId() + ", initiatingNodeId=" + msg.initiatorNodeId() + "]");
-
-                        lastCgsCtx = new ChangeGlobalStateContext(
-                            msg.requestId(),
-                            msg.initiatorNodeId(),
-                            msg.getDynamicCacheChangeBatch(),
-                            msg.activate());
-
-                        globalState = TRANSITION;
-                    }
-                }
-            });
-
-        ctx.event().addLocalEventListener(lsr, EVT_NODE_LEFT, EVT_NODE_FAILED);
-    }
-
     /**
-     * @param data Joining node discovery data.
+     * @return Cluster state to be used on public API.
      */
-    public void cacheProcessorStarted(CacheJoinNodeDiscoveryData data) {
-        assert data != null;
+    public boolean publicApiActiveState() {
+        DiscoveryDataClusterState globalState = this.globalState;
 
-        localCacheData = data;
+        assert globalState != null;
 
-        cacheProc = ctx.cache();
-        sharedCtx = cacheProc.context();
+        if (globalState.transition()) {
+            Boolean transitionRes = globalState.transitionResult();
 
-        sharedCtx.io().addCacheHandler(
-            0, GridChangeGlobalStateMessageResponse.class,
-            new CI2<UUID, GridChangeGlobalStateMessageResponse>() {
-                @Override public void apply(UUID nodeId, GridChangeGlobalStateMessageResponse msg) {
-                    processChangeGlobalStateResponse(nodeId, msg);
-                }
-            });
+            if (transitionRes != null)
+                return transitionRes;
+            else
+                return false;
+        }
+        else
+            return globalState.active();
     }
 
     /** {@inheritDoc} */
-    @Override public void stop(boolean cancel) throws IgniteCheckedException {
-        super.stop(cancel);
-
-        sharedCtx.io().removeHandler(false, 0, GridChangeGlobalStateMessageResponse.class);
-        ctx.event().removeLocalEventListener(lsr, EVT_NODE_LEFT, EVT_NODE_FAILED);
-
-        IgniteCheckedException stopErr = new IgniteInterruptedCheckedException(
-            "Node is stopping: " + ctx.igniteInstanceName());
-
-        GridChangeGlobalStateFuture f = cgsLocFut.get();
+    @Override public void start() throws IgniteCheckedException {
+        // Start first node as inactive if persistence is enabled.
+        boolean activeOnStart = !ctx.config().isPersistentStoreEnabled() && ctx.config().isActiveOnStart();
 
-        if (f != null)
-            f.onDone(stopErr);
+        globalState = DiscoveryDataClusterState.createState(activeOnStart);
 
-        cgsLocFut.set(null);
+        ctx.event().addLocalEventListener(lsr, EVT_NODE_LEFT, EVT_NODE_FAILED);
     }
 
     /** {@inheritDoc} */
-    @Override public void onKernalStart() throws IgniteCheckedException {
-        super.onKernalStart();
+    @Override public void onKernalStop(boolean cancel) {
+        GridChangeGlobalStateFuture fut = this.stateChangeFut.get();
 
-        if (ctx.isDaemon())
-            return;
+        if (fut != null)
+            fut.onDone(new IgniteCheckedException("Failed to wait for cluster state change, node is stopping."));
 
-        List<ClusterNode> nodes = ctx.discovery().serverNodes(AffinityTopologyVersion.NONE);
-
-        assert localCacheData != null;
-
-        // First node started (coordinator).
-        if (nodes.isEmpty() || nodes.get(0).isLocal())
-            cacheData.putAll(localCacheData.caches());
-
-        if (globalState == INACTIVE) { // Accept inactivate state after join.
-            if (log != null && log.isInfoEnabled())
-                log.info("Got inactivate state from cluster during node join.");
-
-            // Revert start action if get INACTIVE state on join.
-            sharedCtx.snapshot().onDeActivate(ctx);
-
-            if (sharedCtx.pageStore() != null)
-                sharedCtx.pageStore().onDeActivate(ctx);
-
-            if (sharedCtx.wal() != null)
-                sharedCtx.wal().onDeActivate(ctx);
-
-            sharedCtx.database().onDeActivate(ctx);
-        }
-    }
-
-    /** {@inheritDoc} */
-    @Nullable @Override public DiscoveryDataExchangeType discoveryDataType() {
-        return DiscoveryDataExchangeType.STATE_PROC;
+        super.onKernalStop(cancel);
     }
 
-    /** {@inheritDoc} */
-    @Override public void collectGridNodeData(DiscoveryDataBag dataBag) {
-        if (!dataBag.commonDataCollectedFor(STATE_PROC.ordinal()))
-            dataBag.addGridCommonData(STATE_PROC.ordinal(), globalState);
-    }
+    /**
+     * @param discoCache Discovery data cache.
+     * @return If transition is in progress returns future which is completed when transition finishes.
+     */
+    @Nullable public IgniteInternalFuture<Boolean> onLocalJoin(DiscoCache discoCache) {
+        if (globalState.transition()) {
+            joinFut = new TransitionOnJoinWaitFuture(globalState, discoCache);
 
-    /** {@inheritDoc} */
-    @Override public void onGridDataReceived(DiscoveryDataBag.GridDiscoveryData data) {
-        ClusterState state = (ClusterState)data.commonData();
+            return joinFut;
+        }
 
-        if (state != null)
-            globalState = state;
+        return null;
     }
 
     /**
-     *
+     * @param node Failed node.
+     * @return Message if cluster state changed.
      */
-    public IgniteInternalFuture<?> changeGlobalState(final boolean activate) {
-        if (ctx.isDaemon()) {
-            GridFutureAdapter<Void> fut = new GridFutureAdapter<>();
+    @Nullable public ChangeGlobalStateFinishMessage onNodeLeft(ClusterNode node) {
+        if (globalState.transition()) {
+            Set<UUID> nodes = globalState.transitionNodes();
 
-            sendCompute(activate, fut);
+            if (nodes.remove(node.id()) && nodes.isEmpty()) {
+                U.warn(log, "Failed to change cluster state, all participating nodes failed. " +
+                    "Switching to inactive state.");
 
-            return fut;
-        }
+                ChangeGlobalStateFinishMessage msg =
+                    new ChangeGlobalStateFinishMessage(globalState.transitionRequestId(), false);
 
-        if (cacheProc.transactions().tx() != null || sharedCtx.lockedTopologyVersion(null) != null)
-            throw new IgniteException("Failed to " + prettyStr(activate) + " cluster (must invoke the " +
-                "method outside of an active transaction).");
+                onStateFinishMessage(msg);
 
-        if ((globalState == ACTIVE && activate) || (globalState == INACTIVE && !activate))
-            return new GridFinishedFuture<>();
+                return msg;
+            }
+        }
 
-        final UUID requestId = UUID.randomUUID();
+        return null;
+    }
 
-        final GridChangeGlobalStateFuture cgsFut = new GridChangeGlobalStateFuture(requestId, activate, ctx);
+    /**
+     * @param msg Message.
+     */
+    public void onStateFinishMessage(ChangeGlobalStateFinishMessage msg) {
+        if (msg.requestId().equals(globalState.transitionRequestId())) {
+            log.info("Received state change finish message: " + msg.clusterActive());
 
-        if (!cgsLocFut.compareAndSet(null, cgsFut)) {
-            GridChangeGlobalStateFuture locF = cgsLocFut.get();
+            globalState = DiscoveryDataClusterState.createState(msg.clusterActive());
 
-            if (locF.activate == activate)
-                return locF;
+            ctx.cache().onStateChangeFinish(msg);
 
-            return new GridFinishedFuture<>(new IgniteException(
-                "Failed to " + prettyStr(activate) + ", because another state change operation is currently " +
-                    "in progress: " + prettyStr(locF.activate)));
-        }
+            TransitionOnJoinWaitFuture joinFut = this.joinFut;
 
-        if (globalState == ACTIVE && !activate && ctx.cache().context().snapshot().snapshotOperationInProgress()){
-            return new GridFinishedFuture<>(new IgniteException(
-                "Failed to " + prettyStr(activate) + ", because snapshot operation in progress."));
+            if (joinFut != null)
+                joinFut.onDone(false);
         }
+        else
+            U.warn(log, "Received state finish message with unexpected ID: " + msg);
+    }
 
-        if (ctx.clientNode())
-            sendCompute(activate, cgsFut);
-        else {
-            try {
-                List<DynamicCacheChangeRequest> reqs = new ArrayList<>();
-
-                DynamicCacheChangeRequest changeGlobalStateReq = new DynamicCacheChangeRequest(
-                    requestId, activate ? ACTIVE : INACTIVE, ctx.localNodeId());
-
-                reqs.add(changeGlobalStateReq);
-
-                List<DynamicCacheChangeRequest> cacheReqs = activate ? startAllCachesRequests() : stopAllCachesRequests();
+    /**
+     * @param topVer Current topology version.
+     * @param msg Message.
+     * @param discoCache Current nodes.
+     * @return {@code True} if need start state change process.
+     */
+    public boolean onStateChangeMessage(AffinityTopologyVersion topVer,
+        ChangeGlobalStateMessage msg,
+        DiscoCache discoCache) {
+        if (globalState.transition()) {
+            if (globalState.active() != msg.activate()) {
+                GridChangeGlobalStateFuture fut = changeStateFuture(msg);
+
+                if (fut != null)
+                    fut.onDone(concurrentStateChangeError(msg.activate()));
+            }
+            else {
+                final GridChangeGlobalStateFuture stateFut = changeStateFuture(msg);
 
-                reqs.addAll(cacheReqs);
+                if (stateFut != null) {
+                    IgniteInternalFuture<?> exchFut = ctx.cache().context().exchange().affinityReadyFuture(
+                        globalState.transitionTopologyVersion());
 
-                printCacheInfo(cacheReqs, activate);
+                    if (exchFut == null)
+                        exchFut = new GridFinishedFuture<>();
 
-                ChangeGlobalStateMessage changeGlobalStateMsg = new ChangeGlobalStateMessage(
-                    requestId, ctx.localNodeId(), activate, new DynamicCacheChangeBatch(reqs));
+                    exchFut.listen(new CI1<IgniteInternalFuture<?>>() {
+                        @Override public void apply(IgniteInternalFuture<?> exchFut) {
+                            stateFut.onDone();
+                        }
+                    });
+                }
+            }
+        }
+        else {
+            if (globalState.active() != msg.activate()) {
+                ExchangeActions exchangeActions;
 
                 try {
-                    ctx.discovery().sendCustomEvent(changeGlobalStateMsg);
-
-                    if (ctx.isStopping())
-                        cgsFut.onDone(new IgniteCheckedException("Failed to execute " + prettyStr(activate) + " request, " +
-                            "node is stopping."));
+                    exchangeActions = ctx.cache().onStateChangeRequest(msg, topVer);
                 }
                 catch (IgniteCheckedException e) {
-                    U.error(log, "Failed to create or send global state change request: " + cgsFut, e);
-
-                    cgsFut.onDone(e);
-                }
-            }
-            catch (IgniteCheckedException e) {
-                cgsFut.onDone(e);
-            }
-        }
+                    GridChangeGlobalStateFuture fut = changeStateFuture(msg);
 
-        return cgsFut;
-    }
+                    if (fut != null)
+                        fut.onDone(e);
 
-    /**
-     *
-     */
-    private void sendCompute(boolean activate, final GridFutureAdapter<Void> res) {
-        AffinityTopologyVersion topVer = ctx.discovery().topologyVersionEx();
+                    return false;
+                }
 
-        IgniteCompute comp = ((ClusterGroupAdapter)ctx.cluster().get().forServers()).compute();
+                Set<UUID> nodeIds = U.newHashSet(discoCache.allNodes().size());
 
-        if (log.isInfoEnabled())
-            log.info("Sending " + prettyStr(activate) + " request from node [id=" +
-                ctx.localNodeId() + " topVer=" + topVer + " isClient=" + ctx.isDaemon() +
-                " isDaemon" + ctx.isDaemon() + "]");
+                for (ClusterNode node : discoCache.allNodes())
+                    nodeIds.add(node.id());
 
-        IgniteFuture<Void> fut = comp.runAsync(new ClientChangeGlobalStateComputeRequest(activate));
+                GridChangeGlobalStateFuture fut = changeStateFuture(msg);
 
-        fut.listen(new CI1<IgniteFuture>() {
-            @Override public void apply(IgniteFuture fut) {
-                try {
-                    fut.get();
+                if (fut != null)
+                    fut.setRemaining(nodeIds, topVer.nextMinorVersion());
 
-                    res.onDone();
-                }
-                catch (Exception e) {
-                    res.onDone(e);
-                }
-            }
-        });
-    }
-    /**
-     * @param reqs Requests to print.
-     * @param active Active flag.
-     */
-    private void printCacheInfo(List<DynamicCacheChangeRequest> reqs, boolean active) {
-        assert reqs != null;
+                log.info("Start state transition: " + msg.activate());
 
-        StringBuilder sb = new StringBuilder();
+                globalState = DiscoveryDataClusterState.createTransitionState(msg.activate(),
+                    msg.requestId(),
+                    topVer,
+                    nodeIds);
 
-        sb.append("[");
+                AffinityTopologyVersion stateChangeTopVer = topVer.nextMinorVersion();
 
-        for (int i = 0; i < reqs.size() - 1; i++)
-            sb.append(reqs.get(i).cacheName()).append(", ");
+                StateChangeRequest req = new StateChangeRequest(msg, stateChangeTopVer);
 
-        sb.append(reqs.get(reqs.size() - 1).cacheName());
+                exchangeActions.stateChangeRequest(req);
 
-        sb.append("]");
+                msg.exchangeActions(exchangeActions);
 
-        sb.append(" ").append(reqs.size())
-            .append(" caches will be ")
-            .append(active ? "started" : "stopped");
+                return true;
+            }
+            else {
+                // State already changed.
+                GridChangeGlobalStateFuture stateFut = changeStateFuture(msg);
 
-        if (log.isInfoEnabled())
-            log.info(sb.toString());
-    }
+                if (stateFut != null)
+                    stateFut.onDone();
+            }
+        }
 
-    /**
-     * @param req Cache being started.
-     */
-    public void onCacheStart(DynamicCacheChangeRequest req) {
-        CacheInfo cacheInfo = cacheData.get(req.cacheName());
-
-        if (cacheInfo == null)
-            cacheData.put(req.cacheName(),
-                new CacheInfo(
-                    new StoredCacheData(req.startCacheConfiguration()),
-                    req.cacheType(), req.sql(),
-                    0L)
-            );
+        return false;
     }
 
     /**
-     * @param req Cache being stopped.
+     * @return Current cluster state, should be called only from discovery thread.
      */
-    public void onCacheStop(DynamicCacheChangeRequest req) {
-        CacheInfo cacheInfo = cacheData.get(req.cacheName());
-
-        if (cacheInfo != null)
-            cacheData.remove(req.cacheName());
+    public DiscoveryDataClusterState clusterState() {
+        return globalState;
     }
 
     /**
-     * @return All caches map.
+     * @param msg State change message.
+     * @return Local future for state change process.
      */
-    private Map<String, CacheConfiguration> allCaches() {
-        Map<String, CacheConfiguration> cfgs = new HashMap<>();
-
-        for (Map.Entry<String, CacheInfo> entry : cacheData.entrySet())
-            if (cfgs.get(entry.getKey()) == null)
-                cfgs.put(entry.getKey(), entry.getValue().cacheData().config());
-
-        return cfgs;
+    @Nullable private GridChangeGlobalStateFuture changeStateFuture(ChangeGlobalStateMessage msg) {
+        return changeStateFuture(msg.initiatorNodeId(), msg.requestId());
     }
 
     /**
-     * @return Collection of all caches start requests.
-     * @throws IgniteCheckedException If failed to create requests.
+     * @param initiatorNode Node initiated state change process.
+     * @param reqId State change request ID.
+     * @return Local future for state change process.
      */
-    private List<DynamicCacheChangeRequest> startAllCachesRequests() throws IgniteCheckedException {
-        assert !ctx.config().isDaemon();
-
-        Collection<CacheConfiguration> cacheCfgs = allCaches().values();
-
-        final List<DynamicCacheChangeRequest> reqs = new ArrayList<>();
-
-        if (sharedCtx.pageStore() != null && sharedCtx.database().persistenceEnabled()) {
-            Map<String, StoredCacheData> ccfgs = sharedCtx.pageStore().readCacheConfigurations();
-
-            for (Map.Entry<String, StoredCacheData> entry : ccfgs.entrySet())
-                reqs.add(createRequest(entry.getValue().config()));
+    @Nullable private GridChangeGlobalStateFuture changeStateFuture(UUID initiatorNode, UUID reqId) {
+        assert initiatorNode != null;
+        assert reqId != null;
 
-            for (CacheConfiguration cfg : cacheCfgs)
-                if (!ccfgs.keySet().contains(cfg.getName()))
-                    reqs.add(createRequest(cfg));
+        if (initiatorNode.equals(ctx.localNodeId())) {
+            GridChangeGlobalStateFuture fut = stateChangeFut.get();
 
-            return reqs;
+            if (fut != null && fut.requestId.equals(reqId))
+                return fut;
         }
-        else {
-            for (CacheConfiguration cfg : cacheCfgs)
-                reqs.add(createRequest(cfg));
 
-            return reqs;
-        }
+        return null;
     }
 
     /**
-     * @return Collection of requests to stop caches.
+     * @param activate New state.
+     * @return State change error.
      */
-    private List<DynamicCacheChangeRequest> stopAllCachesRequests() {
-        Collection<CacheConfiguration> cacheCfgs = allCaches().values();
-
-        List<DynamicCacheChangeRequest> reqs = new ArrayList<>(cacheCfgs.size());
-
-        for (CacheConfiguration cfg : cacheCfgs) {
-            DynamicCacheChangeRequest req = stopRequest(ctx, cfg.getName(), false, false);
-
-            reqs.add(req);
-        }
-
-        return reqs;
+    private IgniteCheckedException concurrentStateChangeError(boolean activate) {
+        return new IgniteCheckedException("Failed to " + prettyStr(activate) +
+            ", because another state change operation is currently in progress: " + prettyStr(!activate));
     }
 
     /**
-     * @param cfg Configuration to create request for.
-     * @return Dynamic cache change request.
+     *
      */
-    private DynamicCacheChangeRequest createRequest(CacheConfiguration cfg) {
-        assert cfg != null;
-        assert cfg.getName() != null;
-
-        String cacheName = cfg.getName();
+    public void cacheProcessorStarted() {
+        cacheProc = ctx.cache();
+        sharedCtx = cacheProc.context();
 
-        DynamicCacheChangeRequest req = new DynamicCacheChangeRequest(
-            UUID.randomUUID(), cacheName, ctx.localNodeId());
+        sharedCtx.io().addCacheHandler(
+            0, GridChangeGlobalStateMessageResponse.class,
+            new CI2<UUID, GridChangeGlobalStateMessageResponse>() {
+                @Override public void apply(UUID nodeId, GridChangeGlobalStateMessageResponse msg) {
+                    processChangeGlobalStateResponse(nodeId, msg);
+                }
+            });
+    }
 
-        req.startCacheConfiguration(cfg);
-        req.template(cfg.getName().endsWith("*"));
-        req.nearCacheConfiguration(cfg.getNearConfiguration());
-        req.deploymentId(IgniteUuid.randomUuid());
-        req.schema(new QuerySchema(cfg.getQueryEntities()));
-        req.cacheType(cacheProc.cacheType(cacheName));
+    /** {@inheritDoc} */
+    @Override public void stop(boolean cancel) throws IgniteCheckedException {
+        super.stop(cancel);
 
-        return req;
-    }
+        if (sharedCtx != null)
+            sharedCtx.io().removeHandler(false, 0, GridChangeGlobalStateMessageResponse.class);
 
-    /**
-     *
-     */
-    public boolean active() {
-        ChangeGlobalStateContext actx = lastCgsCtx;
+        ctx.event().removeLocalEventListener(lsr, EVT_NODE_LEFT, EVT_NODE_FAILED);
 
-        if (actx != null && !actx.activate && globalState == TRANSITION)
-            return true;
+        IgniteCheckedException stopErr = new IgniteCheckedException(
+            "Node is stopping: " + ctx.igniteInstanceName());
 
-        if (actx != null && actx.activate && globalState == TRANSITION)
-            return false;
+        GridChangeGlobalStateFuture f = stateChangeFut.get();
 
-        return globalState == ACTIVE;
+        if (f != null)
+            f.onDone(stopErr);
     }
 
-    /**
-     * @param cacheName Cache name to check.
-     * @return Locally configured flag.
-     */
-    public boolean isLocallyConfigured(String cacheName){
-        assert localCacheData != null;
+    /** {@inheritDoc} */
+    @Nullable @Override public DiscoveryDataExchangeType discoveryDataType() {
+        return DiscoveryDataExchangeType.STATE_PROC;
+    }
 
-        return localCacheData.caches().containsKey(cacheName) || localCacheData.templates().containsKey(cacheName);
+    /** {@inheritDoc} */
+    @Override public void collectGridNodeData(DiscoveryDataBag dataBag) {
+        if (!dataBag.commonDataCollectedFor(STATE_PROC.ordinal()))
+            dataBag.addGridCommonData(STATE_PROC.ordinal(), globalState);
     }
 
-    /**
-     * Invoked if cluster is inactive.
-     *
-     * @param dataBag Bag to collect data to.
-     */
-    public void collectGridNodeData0(DiscoveryDataBag dataBag) {
-        if (!dataBag.commonDataCollectedFor(CACHE_PROC.ordinal()))
-            dataBag.addGridCommonData(CACHE_PROC.ordinal(), cacheData);
+    /** {@inheritDoc} */
+    @Override public void onGridDataReceived(DiscoveryDataBag.GridDiscoveryData data) {
+        DiscoveryDataClusterState state = (DiscoveryDataClusterState)data.commonData();
+
+        if (state != null)
+            globalState = state;
     }
 
     /**
-     * @param data Joining node discovery data.
+     * @param activate New cluster state.
+     * @return State change future.
      */
-    public void onJoiningNodeDataReceived0(JoiningNodeDiscoveryData data) {
-        if (data.hasJoiningNodeData()) {
-            if (data.joiningNodeData() instanceof CacheJoinNodeDiscoveryData) {
-                CacheJoinNodeDiscoveryData data0 = (CacheJoinNodeDiscoveryData)data.joiningNodeData();
+    public IgniteInternalFuture<?> changeGlobalState(final boolean activate) {
+        if (ctx.isDaemon() || ctx.clientNode()) {
+            GridFutureAdapter<Void> fut = new GridFutureAdapter<>();
 
-                cacheData.putAll(data0.caches());
-            }
-            else if (data.joiningNodeData() instanceof CacheClientReconnectDiscoveryData) {
-                CacheClientReconnectDiscoveryData data0 = (CacheClientReconnectDiscoveryData)data.joiningNodeData();
+            sendCompute(activate, fut);
 
-                // No-op.
-            }
+            return fut;
         }
-    }
 
-    public void onGridDataReceived0(DiscoveryDataBag.GridDiscoveryData data) {
-        // Receive data from active cluster.
-        if (data.commonData() instanceof CacheNodeCommonDiscoveryData) {
-            CacheNodeCommonDiscoveryData data0 = (CacheNodeCommonDiscoveryData)data.commonData();
+        if (cacheProc.transactions().tx() != null || sharedCtx.lockedTopologyVersion(null) != null) {
+            return new GridFinishedFuture<>(new IgniteCheckedException("Failed to " + prettyStr(activate) +
+                " cluster (must invoke the method outside of an active transaction)."));
+        }
 
-            Map<String, CacheData> caches = data0.caches();
+        DiscoveryDataClusterState curState = globalState;
 
-            Map<String, CacheInfo> cacheInfos = new HashMap<>();
+        if (!curState.transition() && curState.active() == activate)
+            return new GridFinishedFuture<>();
 
-            for (Map.Entry<String, CacheData> entry : caches.entrySet()) {
-                CacheData val = entry.getValue();
+        GridChangeGlobalStateFuture startedFut = null;
 
-                CacheInfo info = new CacheInfo(
-                    new StoredCacheData(val.cacheConfiguration()),
-                    val.cacheType(),
-                    val.sql(),
-                    val.flags()
-                );
+        GridChangeGlobalStateFuture fut = stateChangeFut.get();
 
-                cacheInfos.put(entry.getKey(), info);
-            }
+        while (fut == null) {
+            fut = new GridChangeGlobalStateFuture(UUID.randomUUID(), activate, ctx);
 
-            cacheData.putAll(cacheInfos);
+            if (stateChangeFut.compareAndSet(null, fut)) {
+                startedFut = fut;
 
-        } // Receive data from inactive cluster.
-        else if (data.commonData() instanceof Map) {
-            Map<String, CacheInfo> data0 = (Map<String, CacheInfo>)data.commonData();
+                break;
+            }
+            else
+                fut = stateChangeFut.get();
+        }
 
-            cacheData.putAll(data0);
+        if (startedFut == null) {
+            if (fut.activate != activate) {
+                return new GridFinishedFuture<>(new IgniteCheckedException("Failed to " + prettyStr(activate) +
+                    ", because another state change operation is currently in progress: " + prettyStr(fut.activate)));
+            }
+            else
+                return fut;
         }
 
-        cacheData.putAll(localCacheData.caches());
-    }
+        List<StoredCacheData> storedCfgs = null;
 
-    /**
-     * @param exchActions Requests.
-     * @param topVer Exchange topology version.
-     */
-    public boolean changeGlobalState(
-        ExchangeActions exchActions,
-        AffinityTopologyVersion topVer
-    ) {
-        assert exchActions != null;
-        assert topVer != null;
+        if (activate && sharedCtx.database().persistenceEnabled()) {
+            try {
+                Map<String, StoredCacheData> cfgs = ctx.cache().context().pageStore().readCacheConfigurations();
+
+                if (!F.isEmpty(cfgs))
+                    storedCfgs = new ArrayList<>(cfgs.values());
+            }
+            catch (IgniteCheckedException e) {
+                U.error(log, "Failed to read stored cache configurations: " + e, e);
+
+                startedFut.onDone(e);
 
-        if (exchActions.newClusterState() != null) {
-            ChangeGlobalStateContext cgsCtx = lastCgsCtx;
+                return startedFut;
+            }
+        }
 
-            assert cgsCtx != null : topVer;
+        ChangeGlobalStateMessage msg = new ChangeGlobalStateMessage(startedFut.requestId,
+            ctx.localNodeId(),
+            storedCfgs,
+            activate);
 
-            cgsCtx.topologyVersion(topVer);
+        try {
+            ctx.discovery().sendCustomEvent(msg);
 
-            return true;
+            if (ctx.isStopping())
+                startedFut.onDone(new IgniteCheckedException("Failed to execute " + prettyStr(activate) + " request, " +
+                    "node is stopping."));
         }
+        catch (IgniteCheckedException e) {
+            U.error(log, "Failed to send global state change request: " + activate, e);
 
-        return false;
+            startedFut.onDone(e);
+        }
+
+        return startedFut;
     }
 
     /**
-     * Invoke from exchange future.
+     * @param activate New cluster state.
+     * @param resFut State change future.
      */
-    public Exception onChangeGlobalState() {
-        GridChangeGlobalStateFuture f = cgsLocFut.get();
+    private void sendCompute(boolean activate, final GridFutureAdapter<Void> resFut) {
+        AffinityTopologyVersion topVer = ctx.discovery().topologyVersionEx();
 
-        ChangeGlobalStateContext cgsCtx = lastCgsCtx;
+        IgniteCompute comp = ((ClusterGroupAdapter)ctx.cluster().get().forServers()).compute();
 
-        assert cgsCtx != null;
+        if (log.isInfoEnabled()) {
+            log.info("Sending " + prettyStr(activate) + " request from node [id=" + ctx.localNodeId() +
+                ", topVer=" + topVer +
+                ", client=" + ctx.clientNode() +
+                ", daemon" + ctx.isDaemon() + "]");
+        }
 
-        if (f != null)
-            f.setRemaining(cgsCtx.topVer);
+        IgniteFuture<Void> fut = comp.runAsync(new ClientChangeGlobalStateComputeRequest(activate));
 
-        return cgsCtx.activate ? onActivate(cgsCtx) : onDeActivate(cgsCtx);
+        fut.listen(new CI1<IgniteFuture>() {
+            @Override public void apply(IgniteFuture fut) {
+                try {
+                    fut.get();
+
+                    resFut.onDone();
+                }
+                catch (Exception e) {
+                    resFut.onDone(e);
+                }
+            }
+        });
     }
 
     /**
-     * @param exs Exs.
+     * @param errs Errors.
+     * @param req State change request.
      */
-    public void onFullResponseMessage(Map<UUID, Exception> exs) {
-        assert !F.isEmpty(exs);
-
-        ChangeGlobalStateContext actx = lastCgsCtx;
-
-        actx.setFail();
+    public void onStateChangeError(Map<UUID, Exception> errs, StateChangeRequest req) {
+        assert !F.isEmpty(errs);
 
-        // Revert change if activation request fail.
-        if (actx.activate) {
+        // Revert caches start if activation request fail.
+        if (req.activate()) {
             try {
                 cacheProc.onKernalStopCaches(true);
 
@@ -674,22 +533,10 @@ public class GridClusterStateProcessor extends GridProcessorAdapter {
 
                 sharedCtx.affinity().removeAllCacheInfo();
 
-                ctx.discovery().cleanCachesAndGroups();
-
-                if (!ctx.clientNode()) {
-                    sharedCtx.database().onDeActivate(ctx);
-
-                    if (sharedCtx.pageStore() != null)
-                        sharedCtx.pageStore().onDeActivate(ctx);
-
-                    if (sharedCtx.wal() != null)
-                        sharedCtx.wal().onDeActivate(ctx);
-                }
+                if (!ctx.clientNode())
+                    sharedCtx.deactivate();
             }
             catch (Exception e) {
-                for (Map.Entry<UUID, Exception> entry : exs.entrySet())
-                    e.addSuppressed(entry.getValue());
-
                 U.error(log, "Failed to revert activation request changes", e);
             }
         }
@@ -697,110 +544,33 @@ public class GridClusterStateProcessor extends GridProcessorAdapter {
             //todo https://issues.apache.org/jira/browse/IGNITE-5480
         }
 
-        globalState = actx.activate ? INACTIVE : ACTIVE;
-
-        GridChangeGlobalStateFuture af = cgsLocFut.get();
+        GridChangeGlobalStateFuture fut = changeStateFuture(req.initiatorNodeId(), req.requestId());
 
-        if (af != null && af.requestId.equals(actx.requestId)) {
+        if (fut != null) {
             IgniteCheckedException e = new IgniteCheckedException(
-                "Fail " + prettyStr(actx.activate),
+                "Failed to " + prettyStr(req.activate()) + " cluster",
                 null,
                 false
             );
 
-            for (Map.Entry<UUID, Exception> entry : exs.entrySet())
+            for (Map.Entry<UUID, Exception> entry : errs.entrySet())
                 e.addSuppressed(entry.getValue());
 
-            af.onDone(e);
-        }
-    }
-
-    /**
-     *
-     */
-    private Exception onActivate(ChangeGlobalStateContext cgsCtx) {
-        final boolean client = ctx.clientNode();
-
-        if (log.isInfoEnabled())
-            log.info("Start activation process [nodeId=" + ctx.localNodeId() + ", client=" + client +
-                ", topVer=" + cgsCtx.topVer + "]");
-
-        try {
-            if (!client)
-                sharedCtx.database().lock();
-
-            IgnitePageStoreManager pageStore = sharedCtx.pageStore();
-
-            if (pageStore != null)
-                pageStore.onActivate(ctx);
-
-            if (sharedCtx.wal() != null)
-                sharedCtx.wal().onActivate(ctx);
-
-            sharedCtx.database().onActivate(ctx);
-
-            sharedCtx.snapshot().onActivate(ctx);
-
-            if (log.isInfoEnabled())
-                log.info("Successfully activated persistence managers [nodeId="
-                    + ctx.localNodeId() + ", client=" + client + ", topVer=" + cgsCtx.topVer + "]");
-
-            return null;
-        }
-        catch (Exception e) {
-            U.error(log, "Failed to activate persistence managers [nodeId=" + ctx.localNodeId() + ", client=" + client +
-                ", topVer=" + cgsCtx.topVer + "]", e);
-
-            if (!client)
-                sharedCtx.database().unLock();
-
-            return e;
-        }
-    }
-
-    /**
-     *
-     */
-    public Exception onDeActivate(ChangeGlobalStateContext cgsCtx) {
-        final boolean client = ctx.clientNode();
-
-        if (log.isInfoEnabled())
-            log.info("Starting deactivation [id=" + ctx.localNodeId() + ", client=" +
-                client + ", topVer=" + cgsCtx.topVer + "]");
-
-        try {
-            ctx.dataStructures().onDeActivate(ctx);
-
-            ctx.service().onDeActivate(ctx);
-
-            if (log.isInfoEnabled())
-                log.info("Successfully deactivated persistence processors [id=" + ctx.localNodeId() + ", client=" +
-                    client + ", topVer=" + cgsCtx.topVer + "]");
-
-            return null;
-        }
-        catch (Exception e) {
-            U.error(log, "Failed to execute deactivation callback [nodeId=" + ctx.localNodeId() + ", client=" + client +
-                ", topVer=" + cgsCtx.topVer + "]", e);
-
-            return e;
+            fut.onDone(e);
         }
     }
 
     /**
-     *
+     * @param req State change request.
      */
-    private void onFinalActivate(final ChangeGlobalStateContext cgsCtx) {
-        IgniteInternalFuture<?> asyncActivateFut = ctx.closure().runLocalSafe(new Runnable() {
+    private void onFinalActivate(final StateChangeRequest req) {
+        ctx.closure().runLocalSafe(new Runnable() {
             @Override public void run() {
                 boolean client = ctx.clientNode();
 
                 Exception e = null;
 
                 try {
-                    if (!ctx.config().isDaemon())
-                        ctx.cacheObjects().onUtilityCacheStarted();
-
                     ctx.service().onUtilityCacheStarted();
 
                     ctx.service().onActivate(ctx);
@@ -809,146 +579,114 @@ public class GridClusterStateProcessor extends GridProcessorAdapter {
 
                     if (log.isInfoEnabled())
                         log.info("Successfully performed final activation steps [nodeId="
-                            + ctx.localNodeId() + ", client=" + client + ", topVer=" + cgsCtx.topVer + "]");
+                            + ctx.localNodeId() + ", client=" + client + ", topVer=" + req.topologyVersion() + "]");
                 }
                 catch (Exception ex) {
-                    e = ex;
+                    e = new IgniteCheckedException("Failed to perform final activation steps", ex);
 
                     U.error(log, "Failed to perform final activation steps [nodeId=" + ctx.localNodeId() +
-                        ", client=" + client + ", topVer=" + lastCgsCtx.topVer + "]", ex);
+                        ", client=" + client + ", topVer=" + req.topologyVersion() + "]", ex);
                 }
                 finally {
-                    globalState = ACTIVE;
-
-                    sendChangeGlobalStateResponse(cgsCtx.requestId, cgsCtx.initiatingNodeId, e);
+                    globalState.setTransitionResult(req.requestId(), true);
 
-                    lastCgsCtx = null;
+                    sendChangeGlobalStateResponse(req.requestId(), req.initiatorNodeId(), e);
                 }
             }
         });
-
-        cgsCtx.setAsyncActivateFut(asyncActivateFut);
     }
 
     /**
-     *
+     * @param req State change request.
      */
-    public void onFinalDeActivate(ChangeGlobalStateContext cgsCtx) {
-        final boolean client = ctx.clientNode();
-
-        if (log.isInfoEnabled())
-            log.info("Successfully performed final deactivation steps [nodeId="
-                + ctx.localNodeId() + ", client=" + client + ", topVer=" + cgsCtx.topVer + "]");
-
-        Exception ex = null;
-
-        try {
-            sharedCtx.snapshot().onDeActivate(ctx);
+    private void onFinalDeActivate(final StateChangeRequest req) {
+        globalState.setTransitionResult(req.requestId(), false);
 
-            sharedCtx.database().onDeActivate(ctx);
-
-            if (sharedCtx.pageStore() != null)
-                sharedCtx.pageStore().onDeActivate(ctx);
-
-            if (sharedCtx.wal() != null)
-                sharedCtx.wal().onDeActivate(ctx);
-
-            sharedCtx.affinity().removeAllCacheInfo();
-        }
-        catch (Exception e) {
-            ex = e;
-        }
-        finally {
-            globalState = INACTIVE;
-        }
-
-        sendChangeGlobalStateResponse(cgsCtx.requestId, cgsCtx.initiatingNodeId, ex);
-
-        lastCgsCtx = null;
+        sendChangeGlobalStateResponse(req.requestId(), req.initiatorNodeId(), null);
     }
 
     /**
-     *
+     * @param req State change request.
      */
-    public void onExchangeDone() {
-        ChangeGlobalStateContext cgsCtx = lastCgsCtx;
-
-        assert cgsCtx != null;
-
-        if (!cgsCtx.isFail()) {
-            if (cgsCtx.activate)
-                onFinalActivate(cgsCtx);
-            else
-                onFinalDeActivate(cgsCtx);
-        }
+    public void onStateChangeExchangeDone(StateChangeRequest req) {
+        if (req.activate())
+            onFinalActivate(req);
         else
-            lastCgsCtx = null;
+            onFinalDeActivate(req);
     }
 
     /**
+     * @param reqId Request ID.
      * @param initNodeId Initialize node id.
      * @param ex Exception.
      */
-    private void sendChangeGlobalStateResponse(UUID requestId, UUID initNodeId, Exception ex) {
-        assert requestId != null;
+    private void sendChangeGlobalStateResponse(UUID reqId, UUID initNodeId, Exception ex) {
+        assert reqId != null;
         assert initNodeId != null;
 
-        try {
-            GridChangeGlobalStateMessageResponse actResp = new GridChangeGlobalStateMessageResponse(requestId, ex);
+        GridChangeGlobalStateMessageResponse res = new GridChangeGlobalStateMessageResponse(reqId, ex);
 
+        try {
             if (log.isDebugEnabled())
                 log.debug("Sending global state change response [nodeId=" + ctx.localNodeId() +
-                    ", topVer=" + ctx.discovery().topologyVersionEx() + ", response=" + actResp + "]");
+                    ", topVer=" + ctx.discovery().topologyVersionEx() + ", res=" + res + "]");
 
             if (ctx.localNodeId().equals(initNodeId))
-                processChangeGlobalStateResponse(ctx.localNodeId(), actResp);
+                processChangeGlobalStateResponse(ctx.localNodeId(), res);
             else
-                sharedCtx.io().send(initNodeId, actResp, SYSTEM_POOL);
+                sharedCtx.io().send(initNodeId, res, SYSTEM_POOL);
+        }
+        catch (ClusterTopologyCheckedException e) {
+            if (log.isDebugEnabled()) {
+                log.debug("Failed to send change global state response, node left [node=" + initNodeId +
+                    ", res=" + res + ']');
+            }
         }
         catch (IgniteCheckedException e) {
-            log.error("Fail send change global state response to " + initNodeId, e);
+            U.error(log, "Failed to send change global state response [node=" + initNodeId + ", res=" + res + ']', e);
         }
     }
 
     /**
+     * @param nodeId Node ID.
      * @param msg Message.
      */
     private void processChangeGlobalStateResponse(final UUID nodeId, final GridChangeGlobalStateMessageResponse msg) {
         assert nodeId != null;
         assert msg != null;
 
-        if (log.isDebugEnabled())
+        if (log.isDebugEnabled()) {
             log.debug("Received activation response [requestId=" + msg.getRequestId() +
                 ", nodeId=" + nodeId + "]");
-
-        ClusterNode node = ctx.discovery().node(nodeId);
-
-        if (node == null) {
-            U.warn(log, "Received activation response from unknown node (will ignore) [requestId=" +
-                msg.getRequestId() + ']');
-
-            return;
         }
 
         UUID requestId = msg.getRequestId();
 
-        final GridChangeGlobalStateFuture fut = cgsLocFut.get();
-
-        if (fut != null && !fut.isDone() && requestId.equals(fut.requestId)) {
-            fut.initFut.listen(new CI1<IgniteInternalFuture<?>>() {
-                @Override public void apply(IgniteInternalFuture<?> f) {
-                    fut.onResponse(nodeId, msg);
-                }
-            });
+        final GridChangeGlobalStateFuture fut = stateChangeFut.get();
+
+        if (fut != null && requestId.equals(fut.requestId)) {
+            if (fut.initFut.isDone())
+                fut.onResponse(nodeId, msg);
+            else {
+                fut.initFut.listen(new CI1<IgniteInternalFuture<?>>() {
+                    @Override public void apply(IgniteInternalFuture<?> f) {
+                        // initFut is completed from discovery thread, process response from other thread.
+                        ctx.getSystemExecutorService().execute(new Runnable() {
+                            @Override public void run() {
+                                fut.onResponse(nodeId, msg);
+                            }
+                        });
+                    }
+                });
+            }
         }
     }
 
-
-
     /**
      * @param activate Activate.
+     * @return Activate flag string.
      */
-    private String prettyStr(boolean activate) {
+    private static String prettyStr(boolean activate) {
         return activate ? "activate" : "deactivate";
     }
 
@@ -993,7 +731,9 @@ public class GridClusterStateProcessor extends GridProcessorAdapter {
         private final IgniteLogger log;
 
         /**
-         *
+         * @param requestId State change request ID.
+         * @param activate New cluster state.
+         * @param ctx Context.
          */
         GridChangeGlobalStateFuture(UUID requestId, boolean activate, GridKernalContext ctx) {
             this.requestId = requestId;
@@ -1006,7 +746,7 @@ public class GridClusterStateProcessor extends GridProcessorAdapter {
         /**
          * @param event Event.
          */
-        public void onDiscoveryEvent(DiscoveryEvent event) {
+        void onNodeLeft(DiscoveryEvent event) {
             assert event != null;
 
             if (isDone())
@@ -1024,29 +764,26 @@ public class GridClusterStateProcessor extends GridProcessorAdapter {
         }
 
         /**
-         *
+         * @param nodesIds Node IDs.
+         * @param topVer Current topology version.
          */
-        public void setRemaining(AffinityTopologyVersion topVer) {
-            Collection<ClusterNode> nodes = ctx.discovery().nodes(topVer);
-
-            List<UUID> ids = new ArrayList<>(nodes.size());
-
-            for (ClusterNode n : nodes)
-                ids.add(n.id());
-
-            if (log.isDebugEnabled())
-                log.debug("Setup remaining node [id=" + ctx.localNodeId() + ", client=" +
-                    ctx.clientNode() + ", topVer=" + ctx.discovery().topologyVersionEx() +
-                    ", nodes=" + Arrays.toString(ids.toArray()) + "]");
+        void setRemaining(Set<UUID> nodesIds, AffinityTopologyVersion topVer) {
+            if (log.isDebugEnabled()) {
+                log.debug("Setup remaining node [id=" + ctx.localNodeId() +
+                    ", client=" + ctx.clientNode() +
+                    ", topVer=" + topVer +
+                    ", nodes=" + nodesIds + "]");
+            }
 
             synchronized (mux) {
-                remaining.addAll(ids);
+                remaining.addAll(nodesIds);
             }
 
             initFut.onDone();
         }
 
         /**
+         * @param nodeId Sender node ID.
          * @param msg Activation message response.
          */
         public void onResponse(UUID nodeId, GridChangeGlobalStateMessageResponse msg) {
@@ -1072,7 +809,7 @@ public class GridClusterStateProcessor extends GridProcessorAdapter {
          *
          */
         private void onAllReceived() {
-            Throwable e = new Throwable();
+            IgniteCheckedException e = new IgniteCheckedException();
 
             boolean fail = false;
 
@@ -1094,9 +831,13 @@ public class GridClusterStateProcessor extends GridProcessorAdapter {
 
         /** {@inheritDoc} */
         @Override public boolean onDone(@Nullable Void res, @Nullable Throwable err) {
-            ctx.state().cgsLocFut.set(null);
+            if (super.onDone(res, err)) {
+                ctx.state().stateChangeFut.compareAndSet(this, null);
+
+                return true;
+            }
 
-            return super.onDone(res, err);
+            return false;
         }
 
         /** {@inheritDoc} */
@@ -1107,110 +848,65 @@ public class GridClusterStateProcessor extends GridProcessorAdapter {
 
     /**
      *
-     *
      */
-    private static class ChangeGlobalStateContext {
-        /** Request id. */
-        private final UUID requestId;
-
-        /** Initiating node id. */
-        private final UUID initiatingNodeId;
-
-        /** Batch requests. */
-        private final DynamicCacheChangeBatch batch;
+    private static class ClientChangeGlobalStateComputeRequest implements IgniteRunnable {
+        /** */
+        private static final long serialVersionUID = 0L;
 
-        /** Activate. */
+        /** */
         private final boolean activate;
 
-        /** Topology version. */
-        private AffinityTopologyVersion topVer;
-
-        /** Fail. */
-        private boolean fail;
-
-        /** Async activate future. */
-        private IgniteInternalFuture<?> asyncActivateFut;
+        /** Ignite. */
+        @IgniteInstanceResource
+        private Ignite ignite;
 
         /**
-         *
+         * @param activate New cluster state.
          */
-        ChangeGlobalStateContext(
-            UUID requestId,
-            UUID initiatingNodeId,
-            DynamicCacheChangeBatch batch,
-            boolean activate
-        ) {
-            this.requestId = requestId;
-            this.batch = batch;
+        private ClientChangeGlobalStateComputeRequest(boolean activate) {
             this.activate = activate;
-            this.initiatingNodeId = initiatingNodeId;
-        }
-
-        /**
-         * @param topVer Topology version.
-         */
-        public void topologyVersion(AffinityTopologyVersion topVer) {
-            this.topVer = topVer;
-        }
-
-        /**
-         *
-         */
-        private void setFail() {
-            fail = true;
-        }
-
-        /**
-         *
-         */
-        private boolean isFail() {
-            return fail;
-        }
-
-        /**
-         *
-         */
-        public IgniteInternalFuture<?> getAsyncActivateFut() {
-            return asyncActivateFut;
-        }
-
-        /**
-         * @param asyncActivateFut Async activate future.
-         */
-        public void setAsyncActivateFut(IgniteInternalFuture<?> asyncActivateFut) {
-            this.asyncActivateFut = asyncActivateFut;
         }
 
         /** {@inheritDoc} */
-        @Override public String toString() {
-            return S.toString(ChangeGlobalStateContext.class, this);
+        @Override public void run() {
+            ignite.active(activate);
         }
     }
 
     /**
      *
      */
-    private static class ClientChangeGlobalStateComputeRequest implements IgniteRunnable {
+    class TransitionOnJoinWaitFuture extends GridFutureAdapter<Boolean> {
         /** */
-        private static final long serialVersionUID = 0L;
-
-        /** Activation. */
-        private final boolean activation;
+        private DiscoveryDataClusterState transitionState;
 
-        /** Ignite. */
-        @IgniteInstanceResource
-        private Ignite ignite;
+        /** */
+        private final Set<UUID> transitionNodes;
 
         /**
-         *
+         * @param state Current state.
+         * @param discoCache Discovery data cache.
          */
-        private ClientChangeGlobalStateComputeRequest(boolean activation) {
-            this.activation = activation;
+        TransitionOnJoinWaitFuture(DiscoveryDataClusterState state, DiscoCache discoCache) {
+            assert state.transition() : state;
+
+            transitionNodes = U.newHashSet(state.transitionNodes().size());
+
+            for (UUID nodeId : state.transitionNodes()) {
+                if (discoCache.node(nodeId) != null)
+                    transitionNodes.add(nodeId);
+            }
         }
 
         /** {@inheritDoc} */
-        @Override public void run() {
-            ignite.active(activation);
+        @Override public boolean onDone(@Nullable Boolean res, @Nullable Throwable err) {
+            if (super.onDone(res, err)) {
+                joinFut = null;
+
+                return true;
+            }
+
+            return false;
         }
     }
 }

http://git-wip-us.apache.org/repos/asf/ignite/blob/1337901f/modules/core/src/main/java/org/apache/ignite/internal/processors/cluster/IgniteChangeGlobalStateSupport.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cluster/IgniteChangeGlobalStateSupport.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cluster/IgniteChangeGlobalStateSupport.java
index 3dd9911..5d77f57 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cluster/IgniteChangeGlobalStateSupport.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cluster/IgniteChangeGlobalStateSupport.java
@@ -36,7 +36,6 @@ public interface IgniteChangeGlobalStateSupport {
      * Called when cluster performing deactivation.
      *
      * @param kctx Kernal context.
-     * @throws IgniteCheckedException If failed.
      */
-    public void onDeActivate(GridKernalContext kctx) throws IgniteCheckedException;
+    public void onDeActivate(GridKernalContext kctx);
 }

http://git-wip-us.apache.org/repos/asf/ignite/blob/1337901f/modules/core/src/main/java/org/apache/ignite/internal/processors/datastructures/DataStructuresProcessor.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/datastructures/DataStructuresProcessor.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/datastructures/DataStructuresProcessor.java
index 52cc9e9..4399fe2 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/datastructures/DataStructuresProcessor.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/datastructures/DataStructuresProcessor.java
@@ -175,8 +175,8 @@ public final class DataStructuresProcessor extends GridProcessorAdapter implemen
 
     /** {@inheritDoc} */
     @SuppressWarnings("unchecked")
-    @Override public void onKernalStart() throws IgniteCheckedException {
-        if (ctx.config().isDaemon() || !ctx.state().active())
+    @Override public void onKernalStart(boolean active) throws IgniteCheckedException {
+        if (ctx.config().isDaemon() || !active)
             return;
 
         onKernalStart0();
@@ -278,7 +278,7 @@ public final class DataStructuresProcessor extends GridProcessorAdapter implemen
     }
 
     /** {@inheritDoc} */
-    @Override public void onDeActivate(GridKernalContext ctx) throws IgniteCheckedException {
+    @Override public void onDeActivate(GridKernalContext ctx) {
         if (log.isDebugEnabled())
             log.debug("DeActivate data structure processor [nodeId=" + ctx.localNodeId() +
                 " topVer=" + ctx.discovery().topologyVersionEx() + " ]");

http://git-wip-us.apache.org/repos/asf/ignite/blob/1337901f/modules/core/src/main/java/org/apache/ignite/internal/processors/datastructures/GridCacheAtomicLongImpl.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/datastructures/GridCacheAtomicLongImpl.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/datastructures/GridCacheAtomicLongImpl.java
index c54f801..0bc0c63 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/datastructures/GridCacheAtomicLongImpl.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/datastructures/GridCacheAtomicLongImpl.java
@@ -368,7 +368,7 @@ public final class GridCacheAtomicLongImpl implements GridCacheAtomicLongEx, Ign
     }
 
     /** {@inheritDoc} */
-    @Override public void onDeActivate(GridKernalContext kctx) throws IgniteCheckedException {
+    @Override public void onDeActivate(GridKernalContext kctx) {
         // No-op.
     }
 

http://git-wip-us.apache.org/repos/asf/ignite/blob/1337901f/modules/core/src/main/java/org/apache/ignite/internal/processors/datastructures/GridCacheAtomicReferenceImpl.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/datastructures/GridCacheAtomicReferenceImpl.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/datastructures/GridCacheAtomicReferenceImpl.java
index 64b68e3..42f16f2 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/datastructures/GridCacheAtomicReferenceImpl.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/datastructures/GridCacheAtomicReferenceImpl.java
@@ -299,7 +299,7 @@ public final class GridCacheAtomicReferenceImpl<T> implements GridCacheAtomicRef
     }
 
     /** {@inheritDoc} */
-    @Override public void onDeActivate(GridKernalContext kctx) throws IgniteCheckedException {
+    @Override public void onDeActivate(GridKernalContext kctx) {
         // No-op.
     }
 

http://git-wip-us.apache.org/repos/asf/ignite/blob/1337901f/modules/core/src/main/java/org/apache/ignite/internal/processors/datastructures/GridCacheAtomicSequenceImpl.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/datastructures/GridCacheAtomicSequenceImpl.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/datastructures/GridCacheAtomicSequenceImpl.java
index 47fa49e..019de3c 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/datastructures/GridCacheAtomicSequenceImpl.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/datastructures/GridCacheAtomicSequenceImpl.java
@@ -451,7 +451,7 @@ public final class GridCacheAtomicSequenceImpl implements GridCacheAtomicSequenc
 
     /** {@inheritDoc} */
     @Override public void onDeActivate(GridKernalContext kctx) {
-
+        // No-op.
     }
 
     /** {@inheritDoc} */

http://git-wip-us.apache.org/repos/asf/ignite/blob/1337901f/modules/core/src/main/java/org/apache/ignite/internal/processors/datastructures/GridCacheAtomicStampedImpl.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/datastructures/GridCacheAtomicStampedImpl.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/datastructures/GridCacheAtomicStampedImpl.java
index ac171a6..ed7a225 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/datastructures/GridCacheAtomicStampedImpl.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/datastructures/GridCacheAtomicStampedImpl.java
@@ -343,7 +343,7 @@ public final class GridCacheAtomicStampedImpl<T, S> implements GridCacheAtomicSt
     }
 
     /** {@inheritDoc} */
-    @Override public void onDeActivate(GridKernalContext kctx) throws IgniteCheckedException {
+    @Override public void onDeActivate(GridKernalContext kctx) {
         // No-op.
     }
 

http://git-wip-us.apache.org/repos/asf/ignite/blob/1337901f/modules/core/src/main/java/org/apache/ignite/internal/processors/datastructures/GridCacheCountDownLatchImpl.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/datastructures/GridCacheCountDownLatchImpl.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/datastructures/GridCacheCountDownLatchImpl.java
index 585cb20..7f331c3 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/datastructures/GridCacheCountDownLatchImpl.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/datastructures/GridCacheCountDownLatchImpl.java
@@ -340,7 +340,7 @@ public final class GridCacheCountDownLatchImpl implements GridCacheCountDownLatc
     }
 
     /** {@inheritDoc} */
-    @Override public void onDeActivate(GridKernalContext kctx) throws IgniteCheckedException {
+    @Override public void onDeActivate(GridKernalContext kctx) {
         // No-op.
     }
 

http://git-wip-us.apache.org/repos/asf/ignite/blob/1337901f/modules/core/src/main/java/org/apache/ignite/internal/processors/datastructures/GridCacheLockImpl.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/datastructures/GridCacheLockImpl.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/datastructures/GridCacheLockImpl.java
index 8d3a770..b798670 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/datastructures/GridCacheLockImpl.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/datastructures/GridCacheLockImpl.java
@@ -1477,8 +1477,8 @@ public final class GridCacheLockImpl implements GridCacheLockEx, IgniteChangeGlo
     }
 
     /** {@inheritDoc} */
-    @Override public void onDeActivate(GridKernalContext kctx) throws IgniteCheckedException {
-
+    @Override public void onDeActivate(GridKernalContext kctx) {
+        // No-op.
     }
 
     /** {@inheritDoc} */

http://git-wip-us.apache.org/repos/asf/ignite/blob/1337901f/modules/core/src/main/java/org/apache/ignite/internal/processors/datastructures/GridCacheQueueAdapter.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/datastructures/GridCacheQueueAdapter.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/datastructures/GridCacheQueueAdapter.java
index 2f6abb6..c567ac4 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/datastructures/GridCacheQueueAdapter.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/datastructures/GridCacheQueueAdapter.java
@@ -37,7 +37,6 @@ import org.apache.ignite.IgniteException;
 import org.apache.ignite.IgniteInterruptedException;
 import org.apache.ignite.IgniteLogger;
 import org.apache.ignite.IgniteQueue;
-import org.apache.ignite.cache.affinity.AffinityKeyMapped;
 import org.apache.ignite.internal.processors.cache.GridCacheAdapter;
 import org.apache.ignite.internal.processors.cache.GridCacheContext;
 import org.apache.ignite.internal.util.tostring.GridToStringExclude;

http://git-wip-us.apache.org/repos/asf/ignite/blob/1337901f/modules/core/src/main/java/org/apache/ignite/internal/processors/datastructures/GridCacheSemaphoreImpl.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/datastructures/GridCacheSemaphoreImpl.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/datastructures/GridCacheSemaphoreImpl.java
index c76aec4..4abefc9 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/datastructures/GridCacheSemaphoreImpl.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/datastructures/GridCacheSemaphoreImpl.java
@@ -968,7 +968,7 @@ public final class GridCacheSemaphoreImpl implements GridCacheSemaphoreEx, Ignit
     }
 
     /** {@inheritDoc} */
-    @Override public void onDeActivate(GridKernalContext kctx) throws IgniteCheckedException {
+    @Override public void onDeActivate(GridKernalContext kctx) {
         // No-op.
     }
 

http://git-wip-us.apache.org/repos/asf/ignite/blob/1337901f/modules/core/src/main/java/org/apache/ignite/internal/processors/datastructures/GridCacheSetImpl.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/datastructures/GridCacheSetImpl.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/datastructures/GridCacheSetImpl.java
index e336474..c27770f 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/datastructures/GridCacheSetImpl.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/datastructures/GridCacheSetImpl.java
@@ -35,7 +35,6 @@ import org.apache.ignite.IgniteCompute;
 import org.apache.ignite.IgniteException;
 import org.apache.ignite.IgniteLogger;
 import org.apache.ignite.IgniteSet;
-import org.apache.ignite.cache.affinity.AffinityKeyMapped;
 import org.apache.ignite.cluster.ClusterNode;
 import org.apache.ignite.internal.processors.affinity.AffinityTopologyVersion;
 import org.apache.ignite.internal.processors.cache.CacheIteratorConverter;

http://git-wip-us.apache.org/repos/asf/ignite/blob/1337901f/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/IgfsImpl.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/IgfsImpl.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/IgfsImpl.java
index 8712756..7eb61d1 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/IgfsImpl.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/IgfsImpl.java
@@ -101,8 +101,6 @@ import static org.apache.ignite.events.EventType.EVT_IGFS_DIR_DELETED;
 import static org.apache.ignite.events.EventType.EVT_IGFS_FILE_DELETED;
 import static org.apache.ignite.events.EventType.EVT_IGFS_FILE_OPENED_READ;
 import static org.apache.ignite.events.EventType.EVT_IGFS_META_UPDATED;
-import static org.apache.ignite.igfs.IgfsMode.DUAL_ASYNC;
-import static org.apache.ignite.igfs.IgfsMode.DUAL_SYNC;
 import static org.apache.ignite.igfs.IgfsMode.PRIMARY;
 import static org.apache.ignite.igfs.IgfsMode.PROXY;
 

http://git-wip-us.apache.org/repos/asf/ignite/blob/1337901f/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/IgfsProcessor.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/IgfsProcessor.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/IgfsProcessor.java
index 3c2f64d..244820f 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/IgfsProcessor.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/IgfsProcessor.java
@@ -177,7 +177,7 @@ public class IgfsProcessor extends IgfsProcessorAdapter {
     }
 
     /** {@inheritDoc} */
-    @Override public void onKernalStart() throws IgniteCheckedException {
+    @Override public void onKernalStart(boolean active) throws IgniteCheckedException {
         if (ctx.config().isDaemon())
             return;
 

http://git-wip-us.apache.org/repos/asf/ignite/blob/1337901f/modules/core/src/main/java/org/apache/ignite/internal/processors/query/GridQueryProcessor.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/query/GridQueryProcessor.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/query/GridQueryProcessor.java
index 23ad63d..ce6c9fe 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/query/GridQueryProcessor.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/query/GridQueryProcessor.java
@@ -512,10 +512,8 @@ public class GridQueryProcessor extends GridProcessorAdapter {
 
                     cacheData.queryEntities(cacheDesc.schema().entities());
 
-                    CacheGroupDescriptor grpDesc = ctx.cache().cacheDescriptors().get(cacheData.config().getName()).groupDescriptor();
-
                     try {
-                        ctx.cache().context().pageStore().storeCacheData(grpDesc, cacheData);
+                        ctx.cache().context().pageStore().storeCacheData(cacheData);
                     }
                     catch (IgniteCheckedException e) {
                         throw new IllegalStateException("Failed to persist cache data: " + cacheData.config().getName(), e);

http://git-wip-us.apache.org/repos/asf/ignite/blob/1337901f/modules/core/src/main/java/org/apache/ignite/internal/processors/rest/GridRestProcessor.java
----------------------------------------------------------------------
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 716adf7..f528184 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
@@ -503,7 +503,7 @@ public class GridRestProcessor extends GridProcessorAdapter {
     }
 
     /** {@inheritDoc} */
-    @Override public void onKernalStart() throws IgniteCheckedException {
+    @Override public void onKernalStart(boolean active) throws IgniteCheckedException {
         if (isRestEnabled()) {
             for (GridRestProtocol proto : protos)
                 proto.onKernalStart();

http://git-wip-us.apache.org/repos/asf/ignite/blob/1337901f/modules/core/src/main/java/org/apache/ignite/internal/processors/rest/handlers/cluster/GridChangeStateCommandHandler.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/rest/handlers/cluster/GridChangeStateCommandHandler.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/rest/handlers/cluster/GridChangeStateCommandHandler.java
index 909b524..6236026 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/rest/handlers/cluster/GridChangeStateCommandHandler.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/rest/handlers/cluster/GridChangeStateCommandHandler.java
@@ -64,7 +64,7 @@ public class GridChangeStateCommandHandler extends GridRestCommandHandlerAdapter
 
         try {
             if (req.command().equals(CLUSTER_CURRENT_STATE)) {
-                Boolean currentState = ctx.state().active();
+                Boolean currentState = ctx.state().publicApiActiveState();
 
                 res.setResponse(currentState);
             }

http://git-wip-us.apache.org/repos/asf/ignite/blob/1337901f/modules/core/src/main/java/org/apache/ignite/internal/processors/service/GridServiceProcessor.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/service/GridServiceProcessor.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/service/GridServiceProcessor.java
index 12be63b..2eeee1b 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/service/GridServiceProcessor.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/service/GridServiceProcessor.java
@@ -211,8 +211,8 @@ public class GridServiceProcessor extends GridProcessorAdapter implements Ignite
 
     /** {@inheritDoc} */
     @SuppressWarnings("unchecked")
-    @Override public void onKernalStart() throws IgniteCheckedException {
-        if (ctx.isDaemon() || !ctx.state().active())
+    @Override public void onKernalStart(boolean active) throws IgniteCheckedException {
+        if (ctx.isDaemon() || !active)
             return;
 
         onKernalStart0();
@@ -363,7 +363,7 @@ public class GridServiceProcessor extends GridProcessorAdapter implements Ignite
     }
 
     /** {@inheritDoc} */
-    @Override public void onDeActivate(GridKernalContext kctx) throws IgniteCheckedException {
+    @Override public void onDeActivate(GridKernalContext kctx) {
         if (log.isDebugEnabled())
             log.debug("DeActivate service processor [nodeId=" + ctx.localNodeId() +
                 " topVer=" + ctx.discovery().topologyVersionEx() + " ]");

http://git-wip-us.apache.org/repos/asf/ignite/blob/1337901f/modules/core/src/main/java/org/apache/ignite/internal/processors/task/GridTaskProcessor.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/task/GridTaskProcessor.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/task/GridTaskProcessor.java
index 7ac7b64..d0b88d8 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/task/GridTaskProcessor.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/task/GridTaskProcessor.java
@@ -153,7 +153,7 @@ public class GridTaskProcessor extends GridProcessorAdapter {
     }
 
     /** {@inheritDoc} */
-    @Override public void onKernalStart() throws IgniteCheckedException {
+    @Override public void onKernalStart(boolean active) throws IgniteCheckedException {
         tasksMetaCache = ctx.security().enabled() && !ctx.isDaemon() ?
             ctx.cache().<GridTaskNameHashKey, String>utilityCache() : null;
 

http://git-wip-us.apache.org/repos/asf/ignite/blob/1337901f/modules/core/src/main/java/org/apache/ignite/spi/discovery/tcp/ClientImpl.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/spi/discovery/tcp/ClientImpl.java b/modules/core/src/main/java/org/apache/ignite/spi/discovery/tcp/ClientImpl.java
index d5bacdb..5dbfe6e 100644
--- a/modules/core/src/main/java/org/apache/ignite/spi/discovery/tcp/ClientImpl.java
+++ b/modules/core/src/main/java/org/apache/ignite/spi/discovery/tcp/ClientImpl.java
@@ -582,7 +582,8 @@ class ClientImpl extends TcpDiscoveryImpl {
      * @param addr Address.
      * @return Socket, connect response and client acknowledge support flag.
      */
-    @Nullable private T3<SocketStream, Integer, Boolean> sendJoinRequest(boolean recon, InetSocketAddress addr) {
+    @Nullable private T3<SocketStream, Integer, Boolean> sendJoinRequest(boolean recon,
+        InetSocketAddress addr) {
         assert addr != null;
 
         if (log.isDebugEnabled())
@@ -603,6 +604,8 @@ class ClientImpl extends TcpDiscoveryImpl {
 
         IgniteSpiOperationTimeoutHelper timeoutHelper = new IgniteSpiOperationTimeoutHelper(spi, true);
 
+        DiscoveryDataPacket discoveryData = null;
+
         while (true) {
             boolean openSock = false;
 
@@ -645,9 +648,10 @@ class ClientImpl extends TcpDiscoveryImpl {
                         marshalCredentials(node);
                     }
 
-                    msg = new TcpDiscoveryJoinRequestMessage(
-                            node,
-                            spi.collectExchangeData(new DiscoveryDataPacket(getLocalNodeId())));
+                    if (discoveryData == null)
+                        discoveryData = spi.collectExchangeData(new DiscoveryDataPacket(getLocalNodeId()));
+
+                    msg = new TcpDiscoveryJoinRequestMessage(node, discoveryData);
                 }
                 else
                     msg = new TcpDiscoveryClientReconnectMessage(getLocalNodeId(), rmtNodeId, lastMsgId);

http://git-wip-us.apache.org/repos/asf/ignite/blob/1337901f/modules/core/src/main/java/org/apache/ignite/spi/discovery/tcp/ServerImpl.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/spi/discovery/tcp/ServerImpl.java b/modules/core/src/main/java/org/apache/ignite/spi/discovery/tcp/ServerImpl.java
index 03afff5..c2d9b7e 100644
--- a/modules/core/src/main/java/org/apache/ignite/spi/discovery/tcp/ServerImpl.java
+++ b/modules/core/src/main/java/org/apache/ignite/spi/discovery/tcp/ServerImpl.java
@@ -846,8 +846,10 @@ class ServerImpl extends TcpDiscoveryImpl {
         // Marshal credentials for backward compatibility and security.
         marshalCredentials(locNode, locCred);
 
+        DiscoveryDataPacket discoveryData = spi.collectExchangeData(new DiscoveryDataPacket(getLocalNodeId()));
+
         while (true) {
-            if (!sendJoinRequestMessage()) {
+            if (!sendJoinRequestMessage(discoveryData)) {
                 if (log.isDebugEnabled())
                     log.debug("Join request message has not been sent (local node is the first in the topology).");
 
@@ -973,13 +975,13 @@ class ServerImpl extends TcpDiscoveryImpl {
      * Address is provided by {@link org.apache.ignite.spi.discovery.tcp.ipfinder.TcpDiscoveryIpFinder} and message is
      * sent to first node connection succeeded to.
      *
+     * @param discoveryData Discovery data.
      * @return {@code true} if send succeeded.
      * @throws IgniteSpiException If any error occurs.
      */
     @SuppressWarnings({"BusyWait"})
-    private boolean sendJoinRequestMessage() throws IgniteSpiException {
-        TcpDiscoveryAbstractMessage joinReq = new TcpDiscoveryJoinRequestMessage(locNode,
-            spi.collectExchangeData(new DiscoveryDataPacket(getLocalNodeId())));
+    private boolean sendJoinRequestMessage(DiscoveryDataPacket discoveryData) throws IgniteSpiException {
+        TcpDiscoveryAbstractMessage joinReq = new TcpDiscoveryJoinRequestMessage(locNode, discoveryData);
 
         // Time when it has been detected, that addresses from IP finder do not respond.
         long noResStart = 0;

http://git-wip-us.apache.org/repos/asf/ignite/blob/1337901f/modules/core/src/test/java/org/apache/ignite/internal/TestRecordingCommunicationSpi.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/TestRecordingCommunicationSpi.java b/modules/core/src/test/java/org/apache/ignite/internal/TestRecordingCommunicationSpi.java
index 98d2553..ab61687 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/TestRecordingCommunicationSpi.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/TestRecordingCommunicationSpi.java
@@ -179,6 +179,16 @@ public class TestRecordingCommunicationSpi extends TcpCommunicationSpi {
     /**
      * @throws InterruptedException If interrupted.
      */
+    public void waitForBlocked() throws InterruptedException {
+        synchronized (this) {
+            while (blockedMsgs.isEmpty())
+                wait();
+        }
+    }
+
+    /**
+     * @throws InterruptedException If interrupted.
+     */
     public void waitForRecorded() throws InterruptedException {
         synchronized (this) {
             while (recordedMsgs.isEmpty())

http://git-wip-us.apache.org/repos/asf/ignite/blob/1337901f/modules/core/src/test/java/org/apache/ignite/internal/managers/GridManagerLocalMessageListenerSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/managers/GridManagerLocalMessageListenerSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/managers/GridManagerLocalMessageListenerSelfTest.java
index 5e85b62..b88eef9 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/managers/GridManagerLocalMessageListenerSelfTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/managers/GridManagerLocalMessageListenerSelfTest.java
@@ -128,7 +128,7 @@ public class GridManagerLocalMessageListenerSelfTest extends GridCommonAbstractT
 
         mgr.start();
 
-        mgr.onKernalStart();
+        mgr.onKernalStart(true);
 
         assertTrue(mgr.enabled());
     }
@@ -143,7 +143,7 @@ public class GridManagerLocalMessageListenerSelfTest extends GridCommonAbstractT
 
         assertTrue(mgr.enabled());
 
-        mgr.onKernalStart();
+        mgr.onKernalStart(true);
 
         mgr.onKernalStop(false);
 


[32/33] ignite git commit: Merge branch 'ignite-2.1' into ignite-2.1.2

Posted by sb...@apache.org.
Merge branch 'ignite-2.1' into ignite-2.1.2


Project: http://git-wip-us.apache.org/repos/asf/ignite/repo
Commit: http://git-wip-us.apache.org/repos/asf/ignite/commit/aa21a9b5
Tree: http://git-wip-us.apache.org/repos/asf/ignite/tree/aa21a9b5
Diff: http://git-wip-us.apache.org/repos/asf/ignite/diff/aa21a9b5

Branch: refs/heads/ignite-2.1.2-exchange
Commit: aa21a9b59ec0266b4fee16e02d2796e6e0d1a9b5
Parents: 9743fa3 f9f13cf
Author: devozerov <vo...@gridgain.com>
Authored: Wed Jul 5 14:18:32 2017 +0300
Committer: devozerov <vo...@gridgain.com>
Committed: Wed Jul 5 14:18:32 2017 +0300

----------------------------------------------------------------------
 .../jdbc2/JdbcPreparedStatementSelfTest.java    |   35 +
 .../jdbc/JdbcPreparedStatementSelfTest.java     |   35 +
 .../thin/JdbcThinPreparedStatementSelfTest.java |   35 +
 .../PersistentStoreConfiguration.java           |   39 +-
 .../org/apache/ignite/events/EventType.java     |   12 +
 .../ignite/events/WalSegmentArchivedEvent.java  |   62 +
 .../apache/ignite/internal/GridComponent.java   |    4 +-
 .../ignite/internal/GridPluginComponent.java    |    2 +-
 .../apache/ignite/internal/IgniteKernal.java    |   33 +-
 .../internal/jdbc/JdbcPreparedStatement.java    |    6 +-
 .../internal/jdbc/thin/JdbcThinConnection.java  |    7 +-
 .../jdbc/thin/JdbcThinPreparedStatement.java    |    2 -
 .../internal/jdbc2/JdbcPreparedStatement.java   |   12 +-
 .../internal/managers/GridManagerAdapter.java   |    2 +-
 .../internal/managers/discovery/DiscoCache.java |   17 +-
 .../discovery/DiscoveryLocalJoinData.java       |  104 ++
 .../discovery/GridDiscoveryManager.java         |  128 +-
 .../pagemem/store/IgnitePageStoreManager.java   |    3 +-
 .../internal/pagemem/wal/record/WALRecord.java  |   11 +-
 .../processors/GridProcessorAdapter.java        |    2 +-
 .../cache/CacheAffinitySharedManager.java       |   67 +-
 .../processors/cache/CacheGroupContext.java     |    4 +-
 .../processors/cache/CacheGroupData.java        |    4 +-
 .../cache/ChangeGlobalStateMessage.java         |  120 --
 .../processors/cache/ClusterCachesInfo.java     |  493 +++++--
 .../internal/processors/cache/ClusterState.java |   38 -
 .../cache/DynamicCacheChangeRequest.java        |   52 +-
 .../processors/cache/ExchangeActions.java       |   37 +-
 .../processors/cache/GridCacheEventManager.java |    2 -
 .../cache/GridCacheEvictionManager.java         |    1 -
 .../processors/cache/GridCacheIoManager.java    |   13 +-
 .../processors/cache/GridCacheMvccManager.java  |    9 +-
 .../GridCachePartitionExchangeManager.java      |  423 +++---
 .../processors/cache/GridCacheProcessor.java    |  193 +--
 .../cache/GridCacheSharedContext.java           |   60 +-
 .../cache/GridCacheSharedManager.java           |    6 -
 .../cache/GridCacheSharedManagerAdapter.java    |   16 -
 .../processors/cache/PendingDiscoveryEvent.java |   61 +
 .../processors/cache/StateChangeRequest.java    |   77 ++
 .../binary/CacheObjectBinaryProcessorImpl.java  |    4 +-
 .../distributed/GridCacheTxRecoveryFuture.java  |    1 -
 .../distributed/dht/GridDhtCacheAdapter.java    |    1 -
 .../cache/distributed/dht/GridDhtGetFuture.java |    1 -
 .../distributed/dht/GridDhtGetSingleFuture.java |    2 -
 .../dht/GridDhtPartitionTopologyImpl.java       |   13 +-
 .../dht/GridDhtTopologyFutureAdapter.java       |    2 +-
 .../dht/GridPartitionedSingleGetFuture.java     |    3 -
 .../GridNearAtomicAbstractUpdateFuture.java     |    1 -
 .../dht/preloader/GridDhtForceKeysFuture.java   |    1 -
 .../dht/preloader/GridDhtPartitionDemander.java |    2 +
 .../GridDhtPartitionsExchangeFuture.java        |  228 +++-
 .../preloader/GridDhtPartitionsFullMessage.java |   44 +-
 .../GridDhtPartitionsSingleMessage.java         |   38 +-
 .../dht/preloader/GridDhtPreloader.java         |    2 +-
 .../distributed/near/GridNearGetFuture.java     |    2 -
 .../near/GridNearTxPrepareRequest.java          |    1 -
 .../GridCacheDatabaseSharedManager.java         |  111 +-
 .../persistence/GridCacheOffheapManager.java    |    5 +-
 .../IgniteCacheDatabaseSharedManager.java       |   74 +-
 .../persistence/IgniteCacheSnapshotManager.java |   20 +-
 .../persistence/file/FilePageStoreManager.java  |   14 +-
 .../persistence/tree/io/TrackingPageIO.java     |   12 +-
 .../wal/AbstractWalRecordsIterator.java         |  289 ++++
 .../cache/persistence/wal/FileInput.java        |   16 +-
 .../cache/persistence/wal/FileWALPointer.java   |    4 +-
 .../wal/FileWriteAheadLogManager.java           |  594 ++++----
 .../cache/persistence/wal/RecordSerializer.java |    5 +
 .../persistence/wal/SegmentArchiveResult.java   |   61 +
 .../persistence/wal/SegmentEofException.java    |    3 +-
 .../wal/reader/IgniteWalIteratorFactory.java    |  102 ++
 .../wal/reader/StandaloneGridKernalContext.java |  499 +++++++
 ...ndaloneIgniteCacheDatabaseSharedManager.java |   30 +
 .../reader/StandaloneWalRecordsIterator.java    |  258 ++++
 .../wal/serializer/RecordV1Serializer.java      |   45 +-
 .../query/GridCacheDistributedQueryManager.java |    4 +-
 .../store/GridCacheStoreManagerAdapter.java     |    1 -
 .../cache/version/GridCacheVersionManager.java  |    6 -
 .../cacheobject/IgniteCacheObjectProcessor.java |    5 -
 .../IgniteCacheObjectProcessorImpl.java         |    5 -
 .../cluster/ChangeGlobalStateFinishMessage.java |   86 ++
 .../cluster/ChangeGlobalStateMessage.java       |  140 ++
 .../processors/cluster/ClusterProcessor.java    |    3 +-
 .../cluster/DiscoveryDataClusterState.java      |  157 +++
 .../cluster/GridClusterStateProcessor.java      | 1129 ++++++---------
 .../cluster/IgniteChangeGlobalStateSupport.java |    3 +-
 .../datastructures/DataStructuresProcessor.java |    6 +-
 .../datastructures/GridCacheAtomicLongImpl.java |    2 +-
 .../GridCacheAtomicReferenceImpl.java           |    2 +-
 .../GridCacheAtomicSequenceImpl.java            |    2 +-
 .../GridCacheAtomicStampedImpl.java             |    2 +-
 .../GridCacheCountDownLatchImpl.java            |    2 +-
 .../datastructures/GridCacheLockImpl.java       |    4 +-
 .../datastructures/GridCacheQueueAdapter.java   |    1 -
 .../datastructures/GridCacheSemaphoreImpl.java  |    2 +-
 .../datastructures/GridCacheSetImpl.java        |    1 -
 .../internal/processors/igfs/IgfsImpl.java      |    2 -
 .../internal/processors/igfs/IgfsProcessor.java |    2 +-
 .../utils/PlatformConfigurationUtils.java       |    8 +-
 .../processors/query/GridQueryProcessor.java    |    4 +-
 .../processors/rest/GridRestProcessor.java      |    2 +-
 .../cluster/GridChangeStateCommandHandler.java  |    2 +-
 .../service/GridServiceProcessor.java           |    6 +-
 .../processors/task/GridTaskProcessor.java      |    2 +-
 .../ignite/spi/discovery/tcp/ClientImpl.java    |   12 +-
 .../ignite/spi/discovery/tcp/ServerImpl.java    |   10 +-
 .../internal/TestRecordingCommunicationSpi.java |   10 +
 ...GridManagerLocalMessageListenerSelfTest.java |    4 +-
 .../cache/GridCacheAbstractMetricsSelfTest.java |   24 -
 .../cache/IgniteActiveClusterTest.java          |  182 ---
 .../IgniteClusterActivateDeactivateTest.java    | 1284 ++++++++++++++++++
 ...erActivateDeactivateTestWithPersistence.java |  197 +++
 .../IgniteDaemonNodeMarshallerCacheTest.java    |   10 -
 .../IgniteSemaphoreAbstractSelfTest.java        |   17 +-
 ...IgnitePersistentStoreDataStructuresTest.java |    2 +
 .../wal/IgniteWalHistoryReservationsTest.java   |    2 +-
 .../db/wal/reader/IgniteWalReaderTest.java      |  385 ++++++
 .../db/wal/reader/MockWalIteratorFactory.java   |  114 ++
 .../pagemem/NoOpPageStoreManager.java           |   12 +-
 .../persistence/pagemem/NoOpWALManager.java     |   23 +-
 .../AbstractNodeJoinTemplate.java               |  149 +-
 .../IgniteChangeGlobalStateAbstractTest.java    |   65 +-
 .../IgniteChangeGlobalStateCacheTest.java       |    2 +-
 ...IgniteChangeGlobalStateDataStreamerTest.java |    5 +-
 ...gniteChangeGlobalStateDataStructureTest.java |    6 +-
 .../IgniteChangeGlobalStateFailOverTest.java    |   26 +-
 .../IgniteChangeGlobalStateTest.java            |  158 +--
 .../IgniteStandByClusterTest.java               |   17 +-
 .../join/JoinActiveNodeToActiveCluster.java     |   62 +-
 ...ctiveNodeToActiveClusterWithPersistence.java |   17 +
 .../IgniteStandByClientReconnectTest.java       |   13 +-
 ...eStandByClientReconnectToNewClusterTest.java |   13 +-
 ...cpCommunicationSpiMultithreadedSelfTest.java |    2 +-
 .../testframework/junits/GridAbstractTest.java  |    4 +-
 .../junits/common/GridCommonAbstractTest.java   |    3 +
 .../ignite/testsuites/IgnitePdsTestSuite2.java  |    9 +-
 .../testsuites/IgniteStandByClusterSuite.java   |    5 +-
 .../processors/hadoop/HadoopProcessor.java      |    4 +-
 .../cache/IgniteCacheAbstractQuerySelfTest.java |    3 +-
 .../Cache/CacheConfigurationTest.cs             |   21 +
 .../Apache.Ignite.Core.Tests/EventsTest.cs      |   19 +-
 .../Cache/Configuration/CacheConfiguration.cs   |   75 +-
 .../Apache.Ignite.Core/IgniteConfiguration.cs   |    7 +-
 .../Impl/Events/RemoteListenEventFilter.cs      |    3 +
 143 files changed, 6651 insertions(+), 2631 deletions(-)
----------------------------------------------------------------------



[14/33] ignite git commit: Cosmetic changes

Posted by sb...@apache.org.
Cosmetic changes


Project: http://git-wip-us.apache.org/repos/asf/ignite/repo
Commit: http://git-wip-us.apache.org/repos/asf/ignite/commit/114c42e1
Tree: http://git-wip-us.apache.org/repos/asf/ignite/tree/114c42e1
Diff: http://git-wip-us.apache.org/repos/asf/ignite/diff/114c42e1

Branch: refs/heads/ignite-2.1.2-exchange
Commit: 114c42e14013fe5b7bd6f186f7db59a533d61f68
Parents: 2e7adbf
Author: Alexey Goncharuk <al...@gmail.com>
Authored: Tue Jul 4 19:35:06 2017 +0300
Committer: Alexey Goncharuk <al...@gmail.com>
Committed: Tue Jul 4 19:35:06 2017 +0300

----------------------------------------------------------------------
 .../processors/cache/ClusterCachesInfo.java         |  3 ++-
 .../processors/cache/GridCacheProcessor.java        | 16 +++++++++++-----
 .../cache/persistence/tree/io/TrackingPageIO.java   | 12 ++++++------
 3 files changed, 19 insertions(+), 12 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/114c42e1/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/ClusterCachesInfo.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/ClusterCachesInfo.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/ClusterCachesInfo.java
index 4e9dcf1..8f124b2 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/ClusterCachesInfo.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/ClusterCachesInfo.java
@@ -43,6 +43,7 @@ import org.apache.ignite.internal.processors.affinity.AffinityTopologyVersion;
 import org.apache.ignite.internal.processors.query.QuerySchema;
 import org.apache.ignite.internal.processors.query.QueryUtils;
 import org.apache.ignite.internal.processors.query.schema.SchemaOperationException;
+import org.apache.ignite.internal.util.GridConcurrentHashSet;
 import org.apache.ignite.internal.util.typedef.F;
 import org.apache.ignite.internal.util.typedef.T2;
 import org.apache.ignite.internal.util.typedef.internal.CU;
@@ -74,7 +75,7 @@ class ClusterCachesInfo {
     private final ConcurrentMap<String, DynamicCacheDescriptor> registeredTemplates = new ConcurrentHashMap<>();
 
     /** Caches currently being restarted. */
-    private final Collection<String> restartingCaches = new HashSet<>();
+    private final Collection<String> restartingCaches = new GridConcurrentHashSet<>();
 
     /** */
     private final IgniteLogger log;

http://git-wip-us.apache.org/repos/asf/ignite/blob/114c42e1/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheProcessor.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheProcessor.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheProcessor.java
index 402d874..0f859eb 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheProcessor.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheProcessor.java
@@ -831,8 +831,6 @@ public class GridCacheProcessor extends GridProcessorAdapter {
     /** {@inheritDoc} */
     @SuppressWarnings("unchecked")
     @Override public void onKernalStart() throws IgniteCheckedException {
-        ClusterNode locNode = ctx.discovery().localNode();
-
         boolean active = ctx.state().active();
 
         try {
@@ -881,7 +879,7 @@ public class GridCacheProcessor extends GridProcessorAdapter {
         final List<IgniteInternalFuture> syncFuts = new ArrayList<>(caches.size());
 
         sharedCtx.forAllCaches(new CIX1<GridCacheContext>() {
-            @Override public void applyx(GridCacheContext cctx) throws IgniteCheckedException {
+            @Override public void applyx(GridCacheContext cctx) {
                 CacheConfiguration cfg = cctx.config();
 
                 if (cctx.affinityNode() &&
@@ -3015,13 +3013,20 @@ public class GridCacheProcessor extends GridProcessorAdapter {
     }
 
     /**
+     * Reset restarting caches.
+     */
+    public void resetRestartingCaches(){
+        cachesInfo.restartingCaches().clear();
+    }
+
+    /**
      * @param node Joining node to validate.
      * @return Node validation result if there was an issue with the joining node, {@code null} otherwise.
      */
     private IgniteNodeValidationResult validateRestartingCaches(ClusterNode node) {
         if (cachesInfo.hasRestartingCaches()) {
             String msg = "Joining node during caches restart is not allowed [joiningNodeId=" + node.id() +
-                ", restartingCaches=" + new HashSet<String>(cachesInfo.restartingCaches()) + ']';
+                ", restartingCaches=" + new HashSet<>(cachesInfo.restartingCaches()) + ']';
 
             return new IgniteNodeValidationResult(node.id(), msg, msg);
         }
@@ -3984,7 +3989,8 @@ public class GridCacheProcessor extends GridProcessorAdapter {
          */
         RemovedItemsCleanupTask(long timeout) {
             this.timeout = timeout;
-            this.endTime = U.currentTimeMillis() + timeout;
+
+            endTime = U.currentTimeMillis() + timeout;
         }
 
         /** {@inheritDoc} */

http://git-wip-us.apache.org/repos/asf/ignite/blob/114c42e1/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/tree/io/TrackingPageIO.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/tree/io/TrackingPageIO.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/tree/io/TrackingPageIO.java
index bbf452d..2263130 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/tree/io/TrackingPageIO.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/tree/io/TrackingPageIO.java
@@ -110,12 +110,12 @@ public class TrackingPageIO extends PageIO {
     /**
      * @param buf Buffer.
      * @param nextSnapshotTag Next snapshot id.
-     * @param lastSuccessfulSnapshotId Last successful snapshot id.
+     * @param lastSuccessfulSnapshotTag Last successful snapshot id.
      * @param pageSize Page size.
      */
-    private void validateSnapshotId(ByteBuffer buf, long nextSnapshotTag, long lastSuccessfulSnapshotId, int pageSize) {
-        assert nextSnapshotTag != lastSuccessfulSnapshotId : "nextSnapshotTag = " + nextSnapshotTag +
-            ", lastSuccessfulSnapshotId = " + lastSuccessfulSnapshotId;
+    private void validateSnapshotId(ByteBuffer buf, long nextSnapshotTag, long lastSuccessfulSnapshotTag, int pageSize) {
+        assert nextSnapshotTag != lastSuccessfulSnapshotTag : "nextSnapshotTag = " + nextSnapshotTag +
+            ", lastSuccessfulSnapshotId = " + lastSuccessfulSnapshotTag;
 
         long last = getLastSnapshotTag(buf);
 
@@ -126,7 +126,7 @@ public class TrackingPageIO extends PageIO {
 
         int cntOfPage = countOfPageToTrack(pageSize);
 
-        if (last <= lastSuccessfulSnapshotId) { //we can drop our data
+        if (last <= lastSuccessfulSnapshotTag) { //we can drop our data
             buf.putLong(LAST_SNAPSHOT_TAG_OFFSET, nextSnapshotTag);
 
             PageHandler.zeroMemory(buf, SIZE_FIELD_OFFSET, buf.capacity() - SIZE_FIELD_OFFSET);
@@ -136,7 +136,7 @@ public class TrackingPageIO extends PageIO {
             int sizeOff = useLeftHalf(nextSnapshotTag) ? SIZE_FIELD_OFFSET : BITMAP_OFFSET + len;
             int sizeOff2 = !useLeftHalf(nextSnapshotTag) ? SIZE_FIELD_OFFSET : BITMAP_OFFSET + len;
 
-            if (last - lastSuccessfulSnapshotId == 1) { //we should keep only data in last half
+            if (last - lastSuccessfulSnapshotTag == 1) { //we should keep only data in last half
                 //new data will be written in the same half, we should move old data to another half
                 if ((nextSnapshotTag - last) % 2 == 0)
                     PageHandler.copyMemory(buf, sizeOff, buf, sizeOff2, len + SIZE_FIELD_SIZE);


[12/33] ignite git commit: ignite-5685 JDBC prepared statement shouldn't clear parameters after execution

Posted by sb...@apache.org.
ignite-5685 JDBC prepared statement shouldn't clear parameters after execution


Project: http://git-wip-us.apache.org/repos/asf/ignite/repo
Commit: http://git-wip-us.apache.org/repos/asf/ignite/commit/2e7adbfb
Tree: http://git-wip-us.apache.org/repos/asf/ignite/tree/2e7adbfb
Diff: http://git-wip-us.apache.org/repos/asf/ignite/diff/2e7adbfb

Branch: refs/heads/ignite-2.1.2-exchange
Commit: 2e7adbfbbab62f3b98e0409d2f4ceabad89b4120
Parents: b69f53e
Author: agura <ag...@gridgain.com>
Authored: Tue Jul 4 16:56:40 2017 +0300
Committer: agura <ag...@gridgain.com>
Committed: Tue Jul 4 17:47:46 2017 +0300

----------------------------------------------------------------------
 .../jdbc2/JdbcPreparedStatementSelfTest.java    | 35 ++++++++++++++++++++
 .../jdbc/JdbcPreparedStatementSelfTest.java     | 35 ++++++++++++++++++++
 .../thin/JdbcThinPreparedStatementSelfTest.java | 35 ++++++++++++++++++++
 .../internal/jdbc/JdbcPreparedStatement.java    |  6 +---
 .../jdbc/thin/JdbcThinPreparedStatement.java    |  2 --
 .../internal/jdbc2/JdbcPreparedStatement.java   | 12 ++-----
 6 files changed, 108 insertions(+), 17 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/2e7adbfb/modules/clients/src/test/java/org/apache/ignite/internal/jdbc2/JdbcPreparedStatementSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/clients/src/test/java/org/apache/ignite/internal/jdbc2/JdbcPreparedStatementSelfTest.java b/modules/clients/src/test/java/org/apache/ignite/internal/jdbc2/JdbcPreparedStatementSelfTest.java
index e2939e6..7df0e02 100644
--- a/modules/clients/src/test/java/org/apache/ignite/internal/jdbc2/JdbcPreparedStatementSelfTest.java
+++ b/modules/clients/src/test/java/org/apache/ignite/internal/jdbc2/JdbcPreparedStatementSelfTest.java
@@ -159,6 +159,41 @@ public class JdbcPreparedStatementSelfTest extends GridCommonAbstractTest {
     /**
      * @throws Exception If failed.
      */
+    public void testRepeatableUsage() throws Exception {
+        stmt = conn.prepareStatement("select * from TestObject where id = ?");
+
+        stmt.setInt(1, 1);
+
+        ResultSet rs = stmt.executeQuery();
+
+        int cnt = 0;
+
+        while (rs.next()) {
+            if (cnt == 0)
+                assertEquals(1, rs.getInt(1));
+
+            cnt++;
+        }
+
+        assertEquals(1, cnt);
+
+        cnt = 0;
+
+        rs = stmt.executeQuery();
+
+        while (rs.next()) {
+            if (cnt == 0)
+                assertEquals(1, rs.getInt(1));
+
+            cnt++;
+        }
+
+        assertEquals(1, cnt);
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
     public void testBoolean() throws Exception {
         stmt = conn.prepareStatement("select * from TestObject where boolVal is not distinct from ?");
 

http://git-wip-us.apache.org/repos/asf/ignite/blob/2e7adbfb/modules/clients/src/test/java/org/apache/ignite/jdbc/JdbcPreparedStatementSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/clients/src/test/java/org/apache/ignite/jdbc/JdbcPreparedStatementSelfTest.java b/modules/clients/src/test/java/org/apache/ignite/jdbc/JdbcPreparedStatementSelfTest.java
index 0dfa0fd..384036a 100644
--- a/modules/clients/src/test/java/org/apache/ignite/jdbc/JdbcPreparedStatementSelfTest.java
+++ b/modules/clients/src/test/java/org/apache/ignite/jdbc/JdbcPreparedStatementSelfTest.java
@@ -156,6 +156,41 @@ public class JdbcPreparedStatementSelfTest extends GridCommonAbstractTest {
     /**
      * @throws Exception If failed.
      */
+    public void testRepeatableUsage() throws Exception {
+        stmt = conn.prepareStatement("select * from TestObject where id = ?");
+
+        stmt.setInt(1, 1);
+
+        ResultSet rs = stmt.executeQuery();
+
+        int cnt = 0;
+
+        while (rs.next()) {
+            if (cnt == 0)
+                assertEquals(1, rs.getInt(1));
+
+            cnt++;
+        }
+
+        assertEquals(1, cnt);
+
+        cnt = 0;
+
+        rs = stmt.executeQuery();
+
+        while (rs.next()) {
+            if (cnt == 0)
+                assertEquals(1, rs.getInt(1));
+
+            cnt++;
+        }
+
+        assertEquals(1, cnt);
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
     public void testBoolean() throws Exception {
         stmt = conn.prepareStatement("select * from TestObject where boolVal is not distinct from ?");
 

http://git-wip-us.apache.org/repos/asf/ignite/blob/2e7adbfb/modules/clients/src/test/java/org/apache/ignite/jdbc/thin/JdbcThinPreparedStatementSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/clients/src/test/java/org/apache/ignite/jdbc/thin/JdbcThinPreparedStatementSelfTest.java b/modules/clients/src/test/java/org/apache/ignite/jdbc/thin/JdbcThinPreparedStatementSelfTest.java
index 6f18c75..841a0af 100644
--- a/modules/clients/src/test/java/org/apache/ignite/jdbc/thin/JdbcThinPreparedStatementSelfTest.java
+++ b/modules/clients/src/test/java/org/apache/ignite/jdbc/thin/JdbcThinPreparedStatementSelfTest.java
@@ -164,6 +164,41 @@ public class JdbcThinPreparedStatementSelfTest extends JdbcThinAbstractSelfTest
     /**
      * @throws Exception If failed.
      */
+    public void testRepeatableUsage() throws Exception {
+        stmt = conn.prepareStatement(SQL_PART + " where id = ?");
+
+        stmt.setInt(1, 1);
+
+        ResultSet rs = stmt.executeQuery();
+
+        int cnt = 0;
+
+        while (rs.next()) {
+            if (cnt == 0)
+                assertEquals(1, rs.getInt(1));
+
+            cnt++;
+        }
+
+        assertEquals(1, cnt);
+
+        cnt = 0;
+
+        rs = stmt.executeQuery();
+
+        while (rs.next()) {
+            if (cnt == 0)
+                assertEquals(1, rs.getInt(1));
+
+            cnt++;
+        }
+
+        assertEquals(1, cnt);
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
     public void testQueryExecuteException() throws Exception {
         stmt = conn.prepareStatement(SQL_PART + " where boolVal is not distinct from ?");
 

http://git-wip-us.apache.org/repos/asf/ignite/blob/2e7adbfb/modules/core/src/main/java/org/apache/ignite/internal/jdbc/JdbcPreparedStatement.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/jdbc/JdbcPreparedStatement.java b/modules/core/src/main/java/org/apache/ignite/internal/jdbc/JdbcPreparedStatement.java
index 7e5358b..93cda1e 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/jdbc/JdbcPreparedStatement.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/jdbc/JdbcPreparedStatement.java
@@ -69,11 +69,7 @@ public class JdbcPreparedStatement extends JdbcStatement implements PreparedStat
 
     /** {@inheritDoc} */
     @Override public ResultSet executeQuery() throws SQLException {
-        ResultSet rs = executeQuery(sql);
-
-        args = null;
-
-        return rs;
+        return executeQuery(sql);
     }
 
     /** {@inheritDoc} */

http://git-wip-us.apache.org/repos/asf/ignite/blob/2e7adbfb/modules/core/src/main/java/org/apache/ignite/internal/jdbc/thin/JdbcThinPreparedStatement.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/jdbc/thin/JdbcThinPreparedStatement.java b/modules/core/src/main/java/org/apache/ignite/internal/jdbc/thin/JdbcThinPreparedStatement.java
index 49a78b6..0c78a13 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/jdbc/thin/JdbcThinPreparedStatement.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/jdbc/thin/JdbcThinPreparedStatement.java
@@ -219,8 +219,6 @@ public class JdbcThinPreparedStatement extends JdbcThinStatement implements Prep
      */
     private void executeWithArguments() throws SQLException {
         execute0(sql, args);
-
-        args = null;
     }
 
     /** {@inheritDoc} */

http://git-wip-us.apache.org/repos/asf/ignite/blob/2e7adbfb/modules/core/src/main/java/org/apache/ignite/internal/jdbc2/JdbcPreparedStatement.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/jdbc2/JdbcPreparedStatement.java b/modules/core/src/main/java/org/apache/ignite/internal/jdbc2/JdbcPreparedStatement.java
index 1a66ced..16030f7 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/jdbc2/JdbcPreparedStatement.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/jdbc2/JdbcPreparedStatement.java
@@ -73,22 +73,14 @@ public class JdbcPreparedStatement extends JdbcStatement implements PreparedStat
     @Override public ResultSet executeQuery() throws SQLException {
         ensureNotClosed();
 
-        ResultSet rs = executeQuery(sql);
-
-        args = null;
-
-        return rs;
+        return executeQuery(sql);
     }
 
     /** {@inheritDoc} */
     @Override public int executeUpdate() throws SQLException {
         ensureNotClosed();
 
-        int res = executeUpdate(sql);
-
-        args = null;
-
-        return res;
+        return executeUpdate(sql);
     }
 
     /** {@inheritDoc} */


[25/33] ignite git commit: Reworked cluster activation/deactivation.

Posted by sb...@apache.org.
http://git-wip-us.apache.org/repos/asf/ignite/blob/1337901f/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/IgniteActiveClusterTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/IgniteActiveClusterTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/IgniteActiveClusterTest.java
deleted file mode 100644
index cf68767..0000000
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/IgniteActiveClusterTest.java
+++ /dev/null
@@ -1,182 +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.processors.cache;
-
-import java.util.concurrent.Callable;
-import java.util.concurrent.CyclicBarrier;
-import java.util.concurrent.ThreadLocalRandom;
-import java.util.concurrent.atomic.AtomicInteger;
-import org.apache.ignite.IgniteCache;
-import org.apache.ignite.configuration.CacheConfiguration;
-import org.apache.ignite.configuration.IgniteConfiguration;
-import org.apache.ignite.internal.IgniteInternalFuture;
-import org.apache.ignite.spi.discovery.tcp.TcpDiscoverySpi;
-import org.apache.ignite.spi.discovery.tcp.ipfinder.TcpDiscoveryIpFinder;
-import org.apache.ignite.spi.discovery.tcp.ipfinder.vm.TcpDiscoveryVmIpFinder;
-import org.apache.ignite.testframework.GridTestUtils;
-import org.apache.ignite.testframework.junits.common.GridCommonAbstractTest;
-
-import static org.apache.ignite.cache.CacheAtomicityMode.TRANSACTIONAL;
-import static org.apache.ignite.cache.CacheWriteSynchronizationMode.FULL_SYNC;
-
-/**
- *
- */
-public class IgniteActiveClusterTest extends GridCommonAbstractTest {
-    /** */
-    private static final TcpDiscoveryIpFinder IP_FINDER = new TcpDiscoveryVmIpFinder(true);
-
-    /** */
-    private boolean client;
-
-    /** */
-    private boolean active = true;
-
-    /** */
-    private CacheConfiguration ccfg;
-
-    /** {@inheritDoc} */
-    @Override protected IgniteConfiguration getConfiguration(String igniteInstanceName) throws Exception {
-        IgniteConfiguration cfg = super.getConfiguration(igniteInstanceName);
-
-        ((TcpDiscoverySpi)cfg.getDiscoverySpi()).setIpFinder(IP_FINDER);
-
-        cfg.setClientMode(client);
-
-        cfg.setActiveOnStart(active);
-
-        if (ccfg != null) {
-            cfg.setCacheConfiguration(ccfg);
-
-            ccfg = null;
-        }
-
-        return cfg;
-    }
-
-    /**
-     * @throws Exception If failed.
-     */
-    public void testActivate() throws Exception {
-        active = false;
-
-        for (int i = 0; i < 3; i++) {
-            ccfg = cacheConfiguration(DEFAULT_CACHE_NAME);
-
-            startGrid(i);
-        }
-
-        ignite(0).active(true);
-
-        startGrid(3);
-
-        for (int i  = 0; i < 4; i++) {
-            IgniteCache<Integer, Integer> cache = ignite(i).cache(DEFAULT_CACHE_NAME);
-
-            for (int j = 0; j < 10; j++) {
-                ThreadLocalRandom rnd = ThreadLocalRandom.current();
-
-                Integer key = rnd.nextInt(1000);
-
-                cache.put(key, j);
-
-                assertEquals((Integer)j, cache.get(key));
-            }
-        }
-    }
-
-    /**
-     * @throws Exception If failed.
-     */
-    public void testJoinAndActivate() throws Exception {
-        for (int iter = 0; iter < 3; iter++) {
-            log.info("Iteration: " + iter);
-
-            active = false;
-
-            for (int i = 0; i < 3; i++) {
-                ccfg = cacheConfiguration(DEFAULT_CACHE_NAME);
-
-                startGrid(i);
-            }
-
-            final int START_NODES = 3;
-
-            final CyclicBarrier b = new CyclicBarrier(START_NODES + 1);
-
-            IgniteInternalFuture<?> fut1 = GridTestUtils.runAsync(new Callable<Void>() {
-                @Override public Void call() throws Exception {
-                    b.await();
-
-                    Thread.sleep(ThreadLocalRandom.current().nextLong(100) + 1);
-
-                    ignite(0).active(true);
-
-                    return null;
-                }
-            });
-
-            final AtomicInteger nodeIdx = new AtomicInteger(3);
-
-            IgniteInternalFuture<?> fut2 = GridTestUtils.runMultiThreadedAsync(new Callable<Void>() {
-                @Override public Void call() throws Exception {
-                    int idx = nodeIdx.getAndIncrement();
-
-                    b.await();
-
-                    startGrid(idx);
-
-                    return null;
-                }
-            }, START_NODES, "start-node");
-
-            fut1.get();
-            fut2.get();
-
-            for (int i  = 0; i < 6; i++) {
-                IgniteCache<Integer, Integer> cache = ignite(i).cache(DEFAULT_CACHE_NAME);
-
-                for (int j = 0; j < 10; j++) {
-                    ThreadLocalRandom rnd = ThreadLocalRandom.current();
-
-                    Integer key = rnd.nextInt(1000);
-
-                    cache.put(key, j);
-
-                    assertEquals((Integer)j, cache.get(key));
-                }
-            }
-
-            stopAllGrids();
-        }
-    }
-
-    /**
-     * @param name Cache name.
-     * @return Cache configuration.
-     */
-    private CacheConfiguration cacheConfiguration(String name) {
-        CacheConfiguration ccfg = new CacheConfiguration(name);
-
-        ccfg.setWriteSynchronizationMode(FULL_SYNC);
-        ccfg.setAtomicityMode(TRANSACTIONAL);
-        ccfg.setBackups(3);
-
-        return ccfg;
-    }
-}

http://git-wip-us.apache.org/repos/asf/ignite/blob/1337901f/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/IgniteClusterActivateDeactivateTest.java
----------------------------------------------------------------------
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
new file mode 100644
index 0000000..8a604be
--- /dev/null
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/IgniteClusterActivateDeactivateTest.java
@@ -0,0 +1,1284 @@
+/*
+ * 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.ArrayList;
+import java.util.List;
+import java.util.concurrent.Callable;
+import java.util.concurrent.CyclicBarrier;
+import java.util.concurrent.ThreadLocalRandom;
+import java.util.concurrent.atomic.AtomicInteger;
+import java.util.concurrent.atomic.AtomicReference;
+import org.apache.ignite.Ignite;
+import org.apache.ignite.IgniteCache;
+import org.apache.ignite.cache.CacheAtomicityMode;
+import org.apache.ignite.cluster.ClusterNode;
+import org.apache.ignite.configuration.CacheConfiguration;
+import org.apache.ignite.configuration.IgniteConfiguration;
+import org.apache.ignite.configuration.MemoryConfiguration;
+import org.apache.ignite.configuration.PersistentStoreConfiguration;
+import org.apache.ignite.configuration.WALMode;
+import org.apache.ignite.internal.IgniteClientReconnectAbstractTest;
+import org.apache.ignite.internal.IgniteInternalFuture;
+import org.apache.ignite.internal.IgniteKernal;
+import org.apache.ignite.internal.TestRecordingCommunicationSpi;
+import org.apache.ignite.internal.processors.affinity.AffinityTopologyVersion;
+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.util.typedef.F;
+import org.apache.ignite.internal.util.typedef.G;
+import org.apache.ignite.internal.util.typedef.internal.CU;
+import org.apache.ignite.internal.util.typedef.internal.U;
+import org.apache.ignite.lang.IgniteBiPredicate;
+import org.apache.ignite.plugin.extensions.communication.Message;
+import org.apache.ignite.spi.discovery.tcp.TcpDiscoverySpi;
+import org.apache.ignite.spi.discovery.tcp.TestTcpDiscoverySpi;
+import org.apache.ignite.spi.discovery.tcp.ipfinder.TcpDiscoveryIpFinder;
+import org.apache.ignite.spi.discovery.tcp.ipfinder.vm.TcpDiscoveryVmIpFinder;
+import org.apache.ignite.testframework.GridTestUtils;
+import org.apache.ignite.testframework.junits.common.GridCommonAbstractTest;
+
+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;
+
+/**
+ *
+ */
+public class IgniteClusterActivateDeactivateTest extends GridCommonAbstractTest {
+    /** */
+    private static final TcpDiscoveryIpFinder IP_FINDER = new TcpDiscoveryVmIpFinder(true);
+
+    /** */
+    static final String CACHE_NAME_PREFIX = "cache-";
+
+    /** */
+    boolean client;
+
+    /** */
+    private boolean active = true;
+
+    /** */
+    CacheConfiguration[] ccfgs;
+
+    /** */
+    private boolean testSpi;
+
+    /** */
+    private boolean testDiscoSpi;
+
+    /** */
+    private boolean testReconnectSpi;
+
+    /** */
+    private Class[] testSpiRecord;
+
+    /** {@inheritDoc} */
+    @Override protected IgniteConfiguration getConfiguration(String igniteInstanceName) throws Exception {
+        IgniteConfiguration cfg = super.getConfiguration(igniteInstanceName);
+
+        if (testReconnectSpi) {
+            TcpDiscoverySpi spi = new IgniteClientReconnectAbstractTest.TestTcpDiscoverySpi();
+
+            cfg.setDiscoverySpi(spi);
+
+            spi.setJoinTimeout(2 * 60_000);
+        }
+        else if (testDiscoSpi)
+            cfg.setDiscoverySpi(new TestTcpDiscoverySpi());
+
+        ((TcpDiscoverySpi)cfg.getDiscoverySpi()).setIpFinder(IP_FINDER);
+
+        cfg.setConsistentId(igniteInstanceName);
+
+        cfg.setClientMode(client);
+
+        cfg.setActiveOnStart(active);
+
+        if (ccfgs != null) {
+            cfg.setCacheConfiguration(ccfgs);
+
+            ccfgs = null;
+        }
+
+        MemoryConfiguration memCfg = new MemoryConfiguration();
+        memCfg.setPageSize(1024);
+        memCfg.setDefaultMemoryPolicySize(10 * 1024 * 1024);
+
+        cfg.setMemoryConfiguration(memCfg);
+
+        if (persistenceEnabled()) {
+            PersistentStoreConfiguration pCfg = new PersistentStoreConfiguration();
+
+            pCfg.setWalMode(WALMode.LOG_ONLY);
+
+            cfg.setPersistentStoreConfiguration(pCfg);
+        }
+
+        if (testSpi) {
+            TestRecordingCommunicationSpi spi = new TestRecordingCommunicationSpi();
+
+            if (testSpiRecord != null)
+                spi.record(testSpiRecord);
+
+            cfg.setCommunicationSpi(spi);
+        }
+
+        return cfg;
+    }
+
+    /** {@inheritDoc} */
+    @Override protected void afterTest() throws Exception {
+        stopAllGrids();
+
+        super.afterTest();
+    }
+
+    /**
+     * @return {@code True} if test with persistence.
+     */
+    protected boolean persistenceEnabled() {
+        return false;
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    public void testActivateSimple_SingleNode() throws Exception {
+        activateSimple(1, 0, 0);
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    public void testActivateSimple_5_Servers() throws Exception {
+        activateSimple(5, 0, 0);
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    public void testActivateSimple_5_Servers2() throws Exception {
+        activateSimple(5, 0, 4);
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    public void testActivateSimple_5_Servers_5_Clients() throws Exception {
+        activateSimple(5, 4, 0);
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    public void testActivateSimple_5_Servers_5_Clients_FromClient() throws Exception {
+        activateSimple(5, 4, 6);
+    }
+
+    /**
+     * @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;
+
+        final int CACHES = 2;
+
+        for (int i = 0; i < srvs + clients; i++) {
+            client = i >= srvs;
+
+            ccfgs = cacheConfigurations1();
+
+            startGrid(i);
+
+            checkNoCaches(i);
+        }
+
+        for (int i = 0; i < srvs + clients; i++)
+            assertFalse(ignite(i).active());
+
+        ignite(activateFrom).active(false); // Should be no-op.
+
+        ignite(activateFrom).active(true);
+
+        for (int i = 0; i < srvs + clients; i++)
+            assertTrue(ignite(i).active());
+
+        for (int i = 0; i < srvs + clients; i++) {
+            for (int c = 0; c < 2; c++)
+                checkCache(ignite(i), CACHE_NAME_PREFIX + c, true);
+
+            checkCache(ignite(i), CU.UTILITY_CACHE_NAME, true);
+        }
+
+        checkCaches(srvs + clients, CACHES);
+
+        client = false;
+
+        startGrid(srvs + clients);
+
+        for (int c = 0; c < 2; c++)
+            checkCache(ignite(srvs + clients), CACHE_NAME_PREFIX + c, true);
+
+        checkCaches(srvs + clients + 1, CACHES);
+
+        client = true;
+
+        startGrid(srvs + clients + 1);
+
+        for (int c = 0; c < 2; c++)
+            checkCache(ignite(srvs + clients + 1), CACHE_NAME_PREFIX + c, false);
+
+        checkCaches(srvs + clients + 2, CACHES);
+    }
+
+    /**
+     * @param nodes Number of nodes.
+     * @param caches Number of caches.
+     */
+    final void checkCaches(int nodes, int caches) {
+        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 = rnd.nextInt(1000);
+
+                    cache.put(key, j);
+
+                    assertEquals((Integer)j, cache.get(key));
+                }
+            }
+        }
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    public void testJoinWhileActivate1_Server() throws Exception {
+        joinWhileActivate1(false, false);
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    public void testJoinWhileActivate1_WithCache_Server() throws Exception {
+        joinWhileActivate1(false, true);
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    public void testJoinWhileActivate1_Client() throws Exception {
+        joinWhileActivate1(true, false);
+    }
+
+    /**
+     * @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(new Callable<Void>() {
+            @Override public Void call() throws Exception {
+                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 < 2; 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;
+
+        startGrid(4);
+
+        checkCaches(5, withNewCache ? 4 : 2);
+    }
+
+    /**
+     * @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 blockMsgNodes Nodes whcis block exchange messages.
+     * @return State change future.
+     * @throws Exception If failed.
+     */
+    private IgniteInternalFuture<?> startNodesAndBlockStatusChange(int srvs,
+        int clients,
+        final int stateChangeFrom,
+        final boolean initiallyActive,
+        int... blockMsgNodes) throws Exception {
+        active = initiallyActive;
+        testSpi = true;
+
+        startWithCaches1(srvs, clients);
+
+        if (initiallyActive && persistenceEnabled())
+            ignite(0).active(true);
+
+        if (blockMsgNodes.length == 0)
+            blockMsgNodes = new int[]{1};
+
+        final AffinityTopologyVersion STATE_CHANGE_TOP_VER = new AffinityTopologyVersion(srvs + clients, 1);
+
+        List<TestRecordingCommunicationSpi> spis = new ArrayList<>();
+
+        for (int idx : blockMsgNodes) {
+            TestRecordingCommunicationSpi spi = TestRecordingCommunicationSpi.spi(ignite(idx));
+
+            spis.add(spi);
+
+            blockExchangeSingleMessage(spi, STATE_CHANGE_TOP_VER);
+        }
+
+        IgniteInternalFuture<?> stateChangeFut = GridTestUtils.runAsync(new Runnable() {
+            @Override public void run() {
+                ignite(stateChangeFrom).active(!initiallyActive);
+            }
+        });
+
+        for (TestRecordingCommunicationSpi spi : spis)
+            spi.waitForBlocked();
+
+        U.sleep(500);
+
+        assertFalse(stateChangeFut.isDone());
+
+        return stateChangeFut;
+    }
+
+    /**
+     * @param spi SPI.
+     * @param topVer Exchange topology version.
+     */
+    private void blockExchangeSingleMessage(TestRecordingCommunicationSpi spi, final AffinityTopologyVersion topVer) {
+        spi.blockMessages(new IgniteBiPredicate<ClusterNode, Message>() {
+            @Override public boolean apply(ClusterNode clusterNode, Message msg) {
+                if (msg instanceof GridDhtPartitionsSingleMessage) {
+                    GridDhtPartitionsSingleMessage pMsg = (GridDhtPartitionsSingleMessage)msg;
+
+                    if (pMsg.exchangeId() != null && pMsg.exchangeId().topologyVersion().equals(topVer))
+                        return true;
+                }
+
+                return false;
+            }
+        });
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    public void testJoinWhileDeactivate1_Server() throws Exception {
+        joinWhileDeactivate1(false, false);
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    public void testJoinWhileDeactivate1_WithCache_Server() throws Exception {
+        joinWhileDeactivate1(false, true);
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    public void testJoinWhileDeactivate1_Client() throws Exception {
+        joinWhileDeactivate1(true, false);
+    }
+
+    /**
+     * @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 joinWhileDeactivate1(final boolean startClient, final boolean withNewCache) throws Exception {
+        IgniteInternalFuture<?> activeFut = startNodesAndBlockStatusChange(2, 0, 0, true);
+
+        IgniteInternalFuture<?> startFut = GridTestUtils.runAsync(new Callable<Void>() {
+            @Override public Void call() throws Exception {
+                client = startClient;
+
+                ccfgs = withNewCache ? cacheConfigurations2() : cacheConfigurations1();
+
+                startGrid(2);
+
+                return null;
+            }
+        });
+
+        TestRecordingCommunicationSpi spi1 = TestRecordingCommunicationSpi.spi(ignite(1));
+
+        spi1.stopBlock();
+
+        activeFut.get();
+        startFut.get();
+
+        checkNoCaches(3);
+
+        ignite(2).active(true);
+
+        for (int c = 0; c < 2; 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;
+
+        startGrid(4);
+
+        checkCaches(5, withNewCache ? 4 : 2);
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    public void testConcurrentJoinAndActivate() throws Exception {
+        for (int iter = 0; iter < 3; iter++) {
+            log.info("Iteration: " + iter);
+
+            active = false;
+
+            for (int i = 0; i < 3; i++) {
+                ccfgs = cacheConfigurations1();
+
+                startGrid(i);
+            }
+
+            final int START_NODES = 3;
+
+            final CyclicBarrier b = new CyclicBarrier(START_NODES + 1);
+
+            IgniteInternalFuture<?> fut1 = GridTestUtils.runAsync(new Callable<Void>() {
+                @Override public Void call() throws Exception {
+                    b.await();
+
+                    Thread.sleep(ThreadLocalRandom.current().nextLong(100) + 1);
+
+                    ignite(0).active(true);
+
+                    return null;
+                }
+            });
+
+            final AtomicInteger nodeIdx = new AtomicInteger(3);
+
+            IgniteInternalFuture<?> fut2 = GridTestUtils.runMultiThreadedAsync(new Callable<Void>() {
+                @Override public Void call() throws Exception {
+                    int idx = nodeIdx.getAndIncrement();
+
+                    b.await();
+
+                    startGrid(idx);
+
+                    return null;
+                }
+            }, START_NODES, "start-node");
+
+            fut1.get();
+            fut2.get();
+
+            checkCaches(6, 2);
+
+            afterTest();
+        }
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    public void testDeactivateSimple_SingleNode() throws Exception {
+        deactivateSimple(1, 0, 0);
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    public void testDeactivateSimple_5_Servers() throws Exception {
+        deactivateSimple(5, 0, 0);
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    public void testDeactivateSimple_5_Servers2() throws Exception {
+        deactivateSimple(5, 0, 4);
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    public void testDeactivateSimple_5_Servers_5_Clients() throws Exception {
+        deactivateSimple(5, 4, 0);
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    public void testDeactivateSimple_5_Servers_5_Clients_FromClient() throws Exception {
+        deactivateSimple(5, 4, 6);
+    }
+
+    /**
+     * @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;
+
+        final int CACHES = 2;
+
+        for (int i = 0; i < srvs + clients; i++) {
+            client = i >= srvs;
+
+            ccfgs = cacheConfigurations1();
+
+            startGrid(i);
+        }
+
+        if (persistenceEnabled())
+            ignite(deactivateFrom).active(true);
+
+        ignite(deactivateFrom).active(true); // Should be no-op.
+
+        checkCaches(srvs + clients, CACHES);
+
+        for (int i = 0; i < srvs + clients; i++)
+            assertTrue(ignite(i).active());
+
+        ignite(deactivateFrom).active(false);
+
+        for (int i = 0; i < srvs + clients; i++)
+            assertFalse(ignite(i).active());
+
+        checkNoCaches(srvs + clients);
+
+        client = false;
+
+        startGrid(srvs + clients);
+
+        checkNoCaches(srvs + clients + 1);
+
+        client = true;
+
+        startGrid(srvs + clients + 1);
+
+        checkNoCaches(srvs + clients + 2);
+
+        for (int i = 0; i < srvs + clients + 2; i++)
+            assertFalse(ignite(i).active());
+
+        ignite(deactivateFrom).active(true);
+
+        for (int i = 0; i < srvs + clients + 2; i++) {
+            assertTrue(ignite(i).active());
+
+            checkCache(ignite(i), CU.UTILITY_CACHE_NAME, true);
+        }
+
+        for (int i = 0; i < srvs; i++) {
+            for (int c = 0; c < 2; c++)
+                checkCache(ignite(i), CACHE_NAME_PREFIX + c, true);
+        }
+
+        checkCaches1(srvs + clients + 2);
+    }
+
+    /**
+     * @param srvs Number of servers.
+     * @param clients Number of clients.
+     * @throws Exception If failed.
+     */
+    private void startWithCaches1(int srvs, int clients) throws Exception {
+        for (int i = 0; i < srvs + clients; i++) {
+            ccfgs = cacheConfigurations1();
+
+            client = i >= srvs;
+
+            startGrid(i);
+        }
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    public void testClientReconnectClusterActive() throws Exception {
+        testReconnectSpi = true;
+
+        ccfgs = cacheConfigurations1();
+
+        final int SRVS = 3;
+        final int CLIENTS = 3;
+
+        startWithCaches1(SRVS, CLIENTS);
+
+        if (persistenceEnabled())
+            ignite(0).active(true);
+
+        Ignite srv = ignite(0);
+        Ignite client = ignite(SRVS);
+
+        checkCache(client, CU.UTILITY_CACHE_NAME, true);
+
+        checkCaches1(SRVS + CLIENTS);
+
+        IgniteClientReconnectAbstractTest.reconnectClientNode(log, client, srv, null);
+
+        checkCaches1(SRVS + CLIENTS);
+
+        this.client = false;
+
+        startGrid(SRVS + CLIENTS);
+
+        this.client = true;
+
+        startGrid(SRVS + CLIENTS + 1);
+
+        checkCaches1(SRVS + CLIENTS + 2);
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    public void testClientReconnectClusterInactive() throws Exception {
+        testReconnectSpi = true;
+
+        active = false;
+
+        final int SRVS = 3;
+        final int CLIENTS = 3;
+
+        startWithCaches1(SRVS, CLIENTS);
+
+        Ignite srv = ignite(0);
+        Ignite client = ignite(SRVS);
+
+        checkNoCaches(SRVS + CLIENTS);
+
+        IgniteClientReconnectAbstractTest.reconnectClientNode(log, client, srv, null);
+
+        checkNoCaches(SRVS + CLIENTS);
+
+        ignite(0).active(true);
+
+        checkCache(client, CU.UTILITY_CACHE_NAME, true);
+
+        checkCaches1(SRVS + CLIENTS);
+
+        this.client = false;
+
+        startGrid(SRVS + CLIENTS);
+
+        this.client = true;
+
+        startGrid(SRVS + CLIENTS + 1);
+
+        checkCaches1(SRVS + CLIENTS);
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    public void testClientReconnectClusterDeactivated() throws Exception {
+        clientReconnectClusterDeactivated(false);
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    public void testClientReconnectClusterDeactivateInProgress() throws Exception {
+        clientReconnectClusterDeactivated(true);
+    }
+
+    /**
+     * @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);
+        Ignite client = ignite(SRVS);
+
+        if (persistenceEnabled())
+            ignite(0).active(true);
+
+        checkCache(client, CU.UTILITY_CACHE_NAME, true);
+
+        checkCaches1(SRVS + CLIENTS);
+
+        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, new Runnable() {
+            @Override public void run() {
+                if (transition) {
+                    blockExchangeSingleMessage(spi1, STATE_CHANGE_TOP_VER);
+
+                    stateFut.set(GridTestUtils.runAsync(new Runnable() {
+                        @Override public void run() {
+                            srv.active(false);
+                        }
+                    }, "deactivate"));
+
+                    try {
+                        U.sleep(500);
+                    }
+                    catch (Exception e) {
+                        e.printStackTrace();
+                    }
+                }
+                else
+                    srv.active(false);
+            }
+        });
+
+        checkCache(client, CACHE_NAME_PREFIX + 0, false);
+
+        if (transition) {
+            assertFalse(stateFut.get().isDone());
+
+            assertFalse(client.active());
+
+            spi1.waitForBlocked();
+
+            spi1.stopBlock();
+
+            stateFut.get().get();
+        }
+
+        checkNoCaches(SRVS + CLIENTS);
+
+        ignite(0).active(true);
+
+        checkCache(client, CU.UTILITY_CACHE_NAME, true);
+
+        assertTrue(client.active());
+
+        checkCaches1(SRVS + CLIENTS);
+
+        checkCache(client, CACHE_NAME_PREFIX + 0, true);
+
+        this.client = false;
+
+        startGrid(SRVS + CLIENTS);
+
+        this.client = true;
+
+        startGrid(SRVS + CLIENTS + 1);
+
+        checkCaches1(SRVS + CLIENTS + 2);
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    public void testClientReconnectClusterActivated() throws Exception {
+        clientReconnectClusterActivated(false);
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    public void testClientReconnectClusterActivateInProgress() throws Exception {
+        clientReconnectClusterActivated(true);
+    }
+
+    /**
+     * @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 {
+        testReconnectSpi = true;
+        testSpi = transition;
+
+        active = false;
+
+        final int SRVS = 3;
+        final int CLIENTS = 3;
+
+        startWithCaches1(SRVS, CLIENTS);
+
+        final Ignite srv = ignite(0);
+        Ignite client = ignite(SRVS);
+
+        checkNoCaches(SRVS + CLIENTS);
+
+        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, new Runnable() {
+            @Override public void run() {
+                if (transition) {
+                    blockExchangeSingleMessage(spi1, STATE_CHANGE_TOP_VER);
+
+                    stateFut.set(GridTestUtils.runAsync(new Runnable() {
+                        @Override public void run() {
+                            srv.active(true);
+                        }
+                    }, "activate"));
+
+                    try {
+                        U.sleep(500);
+                    }
+                    catch (Exception e) {
+                        e.printStackTrace();
+                    }
+                }
+                else
+                    srv.active(true);
+            }
+        });
+
+        checkCache(client, CACHE_NAME_PREFIX + 0, !transition);
+
+        if (transition) {
+            assertFalse(stateFut.get().isDone());
+
+            assertFalse(client.active());
+
+            spi1.waitForBlocked();
+
+            spi1.stopBlock();
+
+            stateFut.get().get();
+        }
+
+        checkCache(client, CU.UTILITY_CACHE_NAME, true);
+
+        checkCaches1(SRVS + CLIENTS);
+
+        checkCache(client, CACHE_NAME_PREFIX + 0, true);
+
+        this.client = false;
+
+        startGrid(SRVS + CLIENTS);
+
+        this.client = true;
+
+        startGrid(SRVS + CLIENTS + 1);
+
+        checkCaches1(SRVS + CLIENTS + 2);
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    public void testInactiveTopologyChanges() throws Exception {
+        testSpi = true;
+
+        testSpiRecord = new Class[]{GridDhtPartitionsSingleMessage.class, GridDhtPartitionsFullMessage.class};
+
+        active = false;
+
+        final int SRVS = 4;
+        final int CLIENTS = 4;
+
+        startWithCaches1(SRVS, CLIENTS);
+
+        checkRecordedMessages(false);
+
+        for (int i = 0; i < 2; i++) {
+            stopGrid(i);
+
+            client = false;
+
+            startGrid(i);
+        }
+
+        checkRecordedMessages(false);
+
+        for (int i = 0; i < 2; i++) {
+            stopGrid(SRVS + i);
+
+            client = true;
+
+            startGrid(SRVS + i);
+        }
+
+        checkRecordedMessages(false);
+
+        ignite(0).active(true);
+
+        checkCaches1(SRVS + CLIENTS);
+
+        checkRecordedMessages(true);
+
+        client = false;
+
+        startGrid(SRVS + CLIENTS);
+
+        client = true;
+
+        startGrid(SRVS + CLIENTS + 1);
+
+        checkRecordedMessages(true);
+
+        checkCaches1(SRVS + CLIENTS + 2);
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    public void testActivateFailover1() throws Exception {
+        stateChangeFailover1(true);
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    public void testDeactivateFailover1() throws Exception {
+        stateChangeFailover1(false);
+    }
+
+    /**
+     * @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 startFut = GridTestUtils.runAsync(new Callable() {
+            @Override public Object call() throws Exception {
+                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);
+
+        if (!activate) {
+            checkNoCaches(9);
+
+            ignite(0).active(true);
+        }
+
+        checkCaches1(9);
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    public void testActivateFailover2() throws Exception {
+        stateChangeFailover2(true);
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    public void testDeactivateFailover2() throws Exception {
+        stateChangeFailover2(false);
+    }
+
+    /**
+     * @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 one more nodes while transition is in progress.
+        IgniteInternalFuture startFut1 = GridTestUtils.runAsync(new Callable() {
+            @Override public Object call() throws Exception {
+                startGrid(8);
+
+                return null;
+            }
+        }, "start-node1");
+        IgniteInternalFuture startFut2 = GridTestUtils.runAsync(new Callable() {
+            @Override public Object call() throws Exception {
+                startGrid(9);
+
+                return null;
+            }
+        }, "start-node2");
+
+        U.sleep(500);
+
+        // Stop coordinator.
+        stopGrid(0);
+
+        stopGrid(getTestIgniteInstanceName(1), true, false);
+        stopGrid(getTestIgniteInstanceName(4), true, false);
+
+        fut.get();
+
+        startFut1.get();
+        startFut2.get();
+
+        client = false;
+
+        startGrid(0);
+        startGrid(1);
+
+        client = true;
+
+        startGrid(4);
+
+        if (!activate) {
+            checkNoCaches(10);
+
+            ignite(0).active(true);
+        }
+
+        checkCaches1(10);
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    public void testActivateFailover3() throws Exception {
+        stateChangeFailover3(true);
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    public void testDeactivateFailover3() throws Exception {
+        stateChangeFailover3(false);
+    }
+
+    /**
+     * @param activate If {@code true} tests activation, otherwise deactivation.
+     * @throws Exception If failed.
+     */
+    private void stateChangeFailover3(boolean activate) throws Exception {
+        testDiscoSpi = true;
+
+        startNodesAndBlockStatusChange(4, 0, 0, !activate);
+
+        client = false;
+
+        IgniteInternalFuture startFut1 = GridTestUtils.runAsync(new Callable() {
+            @Override public Object call() throws Exception {
+                startGrid(4);
+
+                return null;
+            }
+        }, "start-node1");
+
+        IgniteInternalFuture startFut2 = GridTestUtils.runAsync(new Callable() {
+            @Override public Object call() throws Exception {
+                startGrid(5);
+
+                return null;
+            }
+        }, "start-node2");
+
+        U.sleep(1000);
+
+        // Stop all nodes participating in state change and not allow last node to finish exchange.
+        for (int i = 0; i < 4; i++)
+            ((TestTcpDiscoverySpi)ignite(i).configuration().getDiscoverySpi()).simulateNodeFailure();
+
+        for (int i = 0; i < 4; i++)
+            stopGrid(getTestIgniteInstanceName(i), true, false);
+
+        startFut1.get();
+        startFut2.get();
+
+        assertFalse(ignite(4).active());
+        assertFalse(ignite(5).active());
+
+        ignite(4).active(true);
+
+        for (int i = 0; i < 4; i++)
+            startGrid(i);
+
+        checkCaches1(6);
+    }
+
+    /**
+     * @param exp If {@code true} there should be recorded messages.
+     */
+    private void checkRecordedMessages(boolean exp) {
+        for (Ignite node : G.allGrids()) {
+            List<Object> recorded =
+                TestRecordingCommunicationSpi.spi(node).recordedMessages(false);
+
+            if (exp)
+                assertFalse(F.isEmpty(recorded));
+            else
+                assertTrue(F.isEmpty(recorded));
+        }
+    }
+
+    /**
+     * @param nodes Expected nodes number.
+     */
+    private void checkCaches1(int nodes) {
+        checkCaches(nodes, 2);
+    }
+
+    /**
+     * @return Cache configurations.
+     */
+    final CacheConfiguration[] cacheConfigurations1() {
+        CacheConfiguration[] ccfgs = new CacheConfiguration[2];
+
+        ccfgs[0] = cacheConfiguration(CACHE_NAME_PREFIX + 0, ATOMIC);
+        ccfgs[1] = cacheConfiguration(CACHE_NAME_PREFIX + 1, TRANSACTIONAL);
+
+        return ccfgs;
+    }
+
+    /**
+     * @return Cache configurations.
+     */
+    final CacheConfiguration[] cacheConfigurations2() {
+        CacheConfiguration[] ccfgs = new CacheConfiguration[4];
+
+        ccfgs[0] = cacheConfiguration(CACHE_NAME_PREFIX + 0, ATOMIC);
+        ccfgs[1] = cacheConfiguration(CACHE_NAME_PREFIX + 1, TRANSACTIONAL);
+        ccfgs[2] = cacheConfiguration(CACHE_NAME_PREFIX + 2, ATOMIC);
+        ccfgs[3] = cacheConfiguration(CACHE_NAME_PREFIX + 3, TRANSACTIONAL);
+
+        return ccfgs;
+    }
+
+    /**
+     * @param name Cache name.
+     * @param atomicityMode Atomicity mode.
+     * @return Cache configuration.
+     */
+    protected final CacheConfiguration cacheConfiguration(String name, CacheAtomicityMode atomicityMode) {
+        CacheConfiguration ccfg = new CacheConfiguration(name);
+
+        ccfg.setWriteSynchronizationMode(FULL_SYNC);
+        ccfg.setAtomicityMode(atomicityMode);
+        ccfg.setBackups(1);
+
+        return ccfg;
+    }
+
+    /**
+     * @param cacheName Cache name.
+     * @param node Node.
+     * @param exp {@code True} if expect that cache is started on node.
+     */
+    void checkCache(Ignite node, String cacheName, boolean exp) {
+        GridCacheAdapter cache = ((IgniteKernal)node).context().cache().internalCache(cacheName);
+
+        if (exp)
+            assertNotNull("Cache not found [cache=" + cacheName + ", node=" + node.name() + ']', cache);
+        else
+            assertNull("Unexpected cache found [cache=" + cacheName + ", node=" + node.name() + ']', cache);
+    }
+
+    /**
+     * @param nodes Number of nodes.
+     */
+    final void checkNoCaches(int nodes) {
+        for (int i = 0; i < nodes; i++) {
+            GridCacheProcessor cache = ((IgniteKernal)ignite(i)).context().cache();
+
+            assertTrue(cache.caches().isEmpty());
+            assertTrue(cache.internalCaches().isEmpty());
+        }
+    }
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/1337901f/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/IgniteClusterActivateDeactivateTestWithPersistence.java
----------------------------------------------------------------------
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
new file mode 100644
index 0000000..4a19aa8
--- /dev/null
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/IgniteClusterActivateDeactivateTestWithPersistence.java
@@ -0,0 +1,197 @@
+/*
+ * 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.Arrays;
+import java.util.LinkedHashMap;
+import java.util.Map;
+import org.apache.ignite.Ignite;
+import org.apache.ignite.IgniteException;
+import org.apache.ignite.configuration.CacheConfiguration;
+import org.apache.ignite.testframework.GridTestUtils;
+
+/**
+ *
+ */
+public class IgniteClusterActivateDeactivateTestWithPersistence extends IgniteClusterActivateDeactivateTest {
+    /** {@inheritDoc} */
+    @Override protected boolean persistenceEnabled() {
+        return true;
+    }
+
+    /** {@inheritDoc} */
+    @Override protected void beforeTest() throws Exception {
+        super.beforeTest();
+
+        GridTestUtils.deleteDbFiles();
+    }
+
+    /** {@inheritDoc} */
+    @Override protected void afterTest() throws Exception {
+        super.afterTest();
+
+        GridTestUtils.deleteDbFiles();
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    public void testActivateCachesRestore_SingleNode() throws Exception {
+        activateCachesRestore(1, false);
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    public void testActivateCachesRestore_SingleNode_WithNewCaches() throws Exception {
+        activateCachesRestore(1, true);
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    public void testActivateCachesRestore_5_Servers() throws Exception {
+        activateCachesRestore(5, false);
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    public void testActivateCachesRestore_5_Servers_WithNewCaches() throws Exception {
+        activateCachesRestore(5, false);
+    }
+
+    /**
+     * @param srvs Number of server nodes.
+     * @param withNewCaches If {@code true} then after restart has new caches in configuration.
+     * @throws Exception If failed.
+     */
+    private void activateCachesRestore(int srvs, boolean withNewCaches) throws Exception {
+        Ignite srv = startGrids(srvs);
+
+        srv.active(true);
+
+        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);
+
+                cacheData.put(-i, i);
+            }
+        }
+
+        stopAllGrids();
+
+        for (int i = 0; i < srvs; i++) {
+            if (withNewCaches)
+                ccfgs = cacheConfigurations2();
+
+            startGrid(i);
+        }
+
+        srv = ignite(0);
+
+        checkNoCaches(srvs);
+
+        srv.active(true);
+
+        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 (CacheConfiguration ccfg : cacheConfigurations1())
+            checkCacheData(cacheData, ccfg.getName());
+
+        checkCaches(srvs, CACHES);
+
+        int nodes = srvs;
+
+        client = false;
+
+        startGrid(nodes++);
+
+        for (int i = 0; i < nodes; i++) {
+            for (int c = 0; c < CACHES; c++)
+                checkCache(ignite(i), CACHE_NAME_PREFIX + c, true);
+        }
+
+        checkCaches(nodes, CACHES);
+
+        client = true;
+
+        startGrid(nodes++);
+
+        for (int c = 0; c < CACHES; c++)
+            checkCache(ignite(nodes - 1), CACHE_NAME_PREFIX + c, 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 (CacheConfiguration ccfg : cacheConfigurations1())
+            checkCacheData(cacheData, ccfg.getName());
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    public void testActivateCacheRestoreConfigurationConflict() throws Exception {
+        final int SRVS = 3;
+
+        Ignite srv = startGrids(SRVS);
+
+        srv.active(true);
+
+        CacheConfiguration ccfg = new CacheConfiguration(DEFAULT_CACHE_NAME);
+
+        srv.createCache(ccfg);
+
+        stopAllGrids();
+
+        ccfg = new CacheConfiguration(DEFAULT_CACHE_NAME + 1);
+
+        ccfg.setGroupName(DEFAULT_CACHE_NAME);
+
+        ccfgs = new CacheConfiguration[]{ccfg};
+
+        startGrids(SRVS);
+
+        try {
+            ignite(0).active(true);
+
+            fail();
+        }
+        catch (IgniteException e) {
+            // Expected error.
+        }
+
+        for (int i = 0; i < SRVS; i++)
+            assertFalse(ignite(i).active());
+
+        checkNoCaches(SRVS);
+    }
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/1337901f/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/IgniteDaemonNodeMarshallerCacheTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/IgniteDaemonNodeMarshallerCacheTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/IgniteDaemonNodeMarshallerCacheTest.java
index fdf5350..566860d 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/IgniteDaemonNodeMarshallerCacheTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/IgniteDaemonNodeMarshallerCacheTest.java
@@ -22,9 +22,6 @@ import org.apache.ignite.Ignite;
 import org.apache.ignite.IgniteCache;
 import org.apache.ignite.configuration.CacheConfiguration;
 import org.apache.ignite.configuration.IgniteConfiguration;
-import org.apache.ignite.internal.IgniteInternalFuture;
-import org.apache.ignite.internal.IgniteKernal;
-import org.apache.ignite.internal.processors.affinity.AffinityTopologyVersion;
 import org.apache.ignite.lang.IgniteCallable;
 import org.apache.ignite.resources.IgniteInstanceResource;
 import org.apache.ignite.spi.discovery.tcp.TcpDiscoverySpi;
@@ -107,13 +104,6 @@ public class IgniteDaemonNodeMarshallerCacheTest extends GridCommonAbstractTest
 
         awaitPartitionMapExchange();
 
-        // Workaround for IGNITE-1365.
-        IgniteInternalFuture<?> fut = ((IgniteKernal) daemonNode).context().cache().context().exchange().
-            affinityReadyFuture(new AffinityTopologyVersion(2, 0));
-
-        if (fut != null)
-            fut.get();
-
         TestClass1 res1 = daemonNode.compute(daemonNode.cluster().forRemotes()).call(new TestCallable1());
 
         assertNotNull(res1);

http://git-wip-us.apache.org/repos/asf/ignite/blob/1337901f/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/pagemem/NoOpPageStoreManager.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/pagemem/NoOpPageStoreManager.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/pagemem/NoOpPageStoreManager.java
index 4dfe69b..665bb56 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/pagemem/NoOpPageStoreManager.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/pagemem/NoOpPageStoreManager.java
@@ -152,11 +152,6 @@ public class NoOpPageStoreManager implements IgnitePageStoreManager {
     }
 
     /** {@inheritDoc} */
-    @Override public void onKernalStart(boolean reconnect) throws IgniteCheckedException {
-        // No-op.
-    }
-
-    /** {@inheritDoc} */
     @Override public void onKernalStop(boolean cancel) {
         // No-op.
     }
@@ -177,8 +172,7 @@ public class NoOpPageStoreManager implements IgnitePageStoreManager {
     }
 
     /** {@inheritDoc} */
-    @Override public void storeCacheData(CacheGroupDescriptor grpDesc,
-        StoredCacheData cacheData) throws IgniteCheckedException {
+    @Override public void storeCacheData(StoredCacheData cacheData) throws IgniteCheckedException {
         // No-op.
     }
 
@@ -189,11 +183,11 @@ public class NoOpPageStoreManager implements IgnitePageStoreManager {
 
     /** {@inheritDoc} */
     @Override public void onActivate(GridKernalContext kctx) {
-
+        // No-op.
     }
 
     /** {@inheritDoc} */
     @Override public void onDeActivate(GridKernalContext kctx) {
-
+        // No-op.
     }
 }

http://git-wip-us.apache.org/repos/asf/ignite/blob/1337901f/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/pagemem/NoOpWALManager.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/pagemem/NoOpWALManager.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/pagemem/NoOpWALManager.java
index 0ef593f..72450b8 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/pagemem/NoOpWALManager.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/pagemem/NoOpWALManager.java
@@ -43,7 +43,7 @@ public class NoOpWALManager implements IgniteWriteAheadLogManager {
 
     /** {@inheritDoc} */
     @Override public void resumeLogging(WALPointer ptr) throws IgniteCheckedException {
-
+        // No-op.
     }
 
     /** {@inheritDoc} */
@@ -83,42 +83,37 @@ public class NoOpWALManager implements IgniteWriteAheadLogManager {
 
     /** {@inheritDoc} */
     @Override public void start(GridCacheSharedContext cctx) throws IgniteCheckedException {
-
+        // No-op.
     }
 
     /** {@inheritDoc} */
     @Override public void stop(boolean cancel) {
-
-    }
-
-    /** {@inheritDoc} */
-    @Override public void onKernalStart(boolean reconnect) throws IgniteCheckedException {
-
+        // No-op.
     }
 
     /** {@inheritDoc} */
     @Override public void onKernalStop(boolean cancel) {
-
+        // No-op.
     }
 
     /** {@inheritDoc} */
     @Override public void onDisconnected(IgniteFuture reconnectFut) {
-
+        // No-op.
     }
 
     /** {@inheritDoc} */
     @Override public void printMemoryStats() {
-
+        // No-op.
     }
 
     /** {@inheritDoc} */
     @Override public void onActivate(GridKernalContext kctx) throws IgniteCheckedException {
-
+        // No-op.
     }
 
     /** {@inheritDoc} */
-    @Override public void onDeActivate(GridKernalContext kctx) throws IgniteCheckedException {
-
+    @Override public void onDeActivate(GridKernalContext kctx) {
+        // No-op.
     }
 
     /** {@inheritDoc} */

http://git-wip-us.apache.org/repos/asf/ignite/blob/1337901f/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/standbycluster/AbstractNodeJoinTemplate.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/standbycluster/AbstractNodeJoinTemplate.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/standbycluster/AbstractNodeJoinTemplate.java
index 9fa6f7c..675aca5 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/standbycluster/AbstractNodeJoinTemplate.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/standbycluster/AbstractNodeJoinTemplate.java
@@ -53,123 +53,160 @@ public abstract class AbstractNodeJoinTemplate extends GridCommonAbstractTest {
     /** Cache 2. */
     protected static final String cache2 = "cache2";
 
-    //Todo Cache with node filter.
+    /** */
     protected static final String cache3 = "cache3";
 
+    /** */
     protected static final String cache4 = "cache4";
 
-    protected static final String cache5 = "cache5";
+    /** */
+    private static final String cache5 = "cache5";
 
     /** Caches info. */
-    public static final String CACHES_INFO = "cachesInfo";
+    private static final String CACHES_INFO = "cachesInfo";
 
     /** Registered caches. */
-    public static final String REGISTERED_CACHES = "registeredCaches";
+    private static final String REGISTERED_CACHES = "registeredCaches";
 
     /** Caches. */
     public static final String CACHES = "caches";
 
     /**
-     * @param ig Ig.
+     * @param ig Node.
+     * @return Cache descriptors.
      */
     protected static Map<String, DynamicCacheDescriptor> cacheDescriptors(IgniteEx ig) {
         return field((Object)field(ig.context().cache(), CACHES_INFO), REGISTERED_CACHES);
     }
 
     /**
-     * @param ig Ig.
+     * @param ig Node.
+     * @return Node caches.
      */
     protected static Map<String, GridCacheAdapter> caches(IgniteEx ig){
         return field(ig.context().cache(), CACHES);
     }
 
     /**
-     *
+     * @return Test builder.
+     * @throws Exception If failed.
      */
     public abstract JoinNodeTestPlanBuilder withOutConfigurationTemplate() throws Exception;
 
     /**
-     *
+     * @return Test builder.
+     * @throws Exception If failed.
      */
     public abstract JoinNodeTestPlanBuilder staticCacheConfigurationOnJoinTemplate() throws Exception;
 
     /**
-     *
+     * @return Test builder.
+     * @throws Exception If failed.
      */
     public abstract JoinNodeTestPlanBuilder staticCacheConfigurationInClusterTemplate() throws Exception;
 
     /**
-     *
+     * @return Test builder.
+     * @throws Exception If failed.
      */
     public abstract JoinNodeTestPlanBuilder staticCacheConfigurationSameOnBothTemplate() throws Exception;
 
     /**
-     *
+     * @return Test builder.
+     * @throws Exception If failed.
      */
     public abstract JoinNodeTestPlanBuilder staticCacheConfigurationDifferentOnBothTemplate() throws Exception;
 
     // Client node join.
 
+    /**
+     * @return Test builder.
+     * @throws Exception If failed.
+     */
     public abstract JoinNodeTestPlanBuilder joinClientWithOutConfigurationTemplate() throws Exception;
 
+    /**
+     * @return Test builder.
+     * @throws Exception If failed.
+     */
     public abstract JoinNodeTestPlanBuilder joinClientStaticCacheConfigurationOnJoinTemplate() throws Exception;
 
+    /**
+     * @return Test builder.
+     * @throws Exception If failed.
+     */
     public abstract JoinNodeTestPlanBuilder joinClientStaticCacheConfigurationInClusterTemplate() throws Exception;
 
+    /**
+     * @return Test builder.
+     * @throws Exception If failed.
+     */
     public abstract JoinNodeTestPlanBuilder joinClientStaticCacheConfigurationSameOnBothTemplate() throws Exception;
 
+    /**
+     * @return Test builder.
+     * @throws Exception If failed.
+     */
     public abstract JoinNodeTestPlanBuilder joinClientStaticCacheConfigurationDifferentOnBothTemplate() throws Exception;
 
     /**
-     *
+     * @throws Exception If failed.
      */
     public abstract void testJoinWithOutConfiguration() throws Exception;
 
     /**
-     *
+     * @throws Exception If failed.
      */
     public abstract void testStaticCacheConfigurationOnJoin() throws Exception;
 
     /**
+     * @throws Exception If failed.
      *
      */
     public abstract void testStaticCacheConfigurationInCluster() throws Exception;
 
     /**
+     * @throws Exception If failed.
      *
      */
     public abstract void testStaticCacheConfigurationSameOnBoth() throws Exception;
 
     /**
+     * @throws Exception If failed.
      *
      */
     public abstract void testStaticCacheConfigurationDifferentOnBoth() throws Exception;
 
     /**
+     * @throws Exception If failed.
      *
      */
     public abstract void testJoinClientWithOutConfiguration() throws Exception;
 
     /**
+     * @throws Exception If failed.
      *
      */
     public abstract void testJoinClientStaticCacheConfigurationOnJoin() throws Exception;
 
     /**
+     * @throws Exception If failed.
      *
      */
     public abstract void testJoinClientStaticCacheConfigurationInCluster() throws Exception;
 
     /**
+     * @throws Exception If failed.
      *
      */
     public abstract void testJoinClientStaticCacheConfigurationSameOnBoth() throws Exception;
 
     /**
-     *
+     * @throws Exception If failed.
      */
     public abstract void testJoinClientStaticCacheConfigurationDifferentOnBoth() throws Exception;
 
+    /** {@inheritDoc} */
     @Override protected void beforeTest() throws Exception {
         super.beforeTest();
 
@@ -178,6 +215,7 @@ public abstract class AbstractNodeJoinTemplate extends GridCommonAbstractTest {
         deleteRecursively(U.resolveWorkDirectory(U.defaultWorkDirectory(), "db", false));
     }
 
+    /** {@inheritDoc} */
     @Override protected void afterTest() throws Exception {
         super.afterTest();
 
@@ -188,6 +226,7 @@ public abstract class AbstractNodeJoinTemplate extends GridCommonAbstractTest {
 
     /**
      * @param idx Index.
+     * @return Ignite instance name.
      */
     protected String name(int idx) {
         return getTestIgniteInstanceName(idx);
@@ -195,6 +234,8 @@ public abstract class AbstractNodeJoinTemplate extends GridCommonAbstractTest {
 
     /**
      * @param name Name.
+     * @return Igntie configuration.
+     * @throws Exception If failed.
      */
     protected IgniteConfiguration cfg(String name) throws Exception {
         try {
@@ -206,7 +247,7 @@ public abstract class AbstractNodeJoinTemplate extends GridCommonAbstractTest {
     }
 
     /**
-     *
+     * @return Test builder.
      */
     protected JoinNodeTestPlanBuilder builder() {
         return JoinNodeTestPlanBuilder.builder();
@@ -214,13 +255,14 @@ public abstract class AbstractNodeJoinTemplate extends GridCommonAbstractTest {
 
     /**
      * @param cfgs Cfgs.
+     * @return Configurations.
      */
-    protected static <T> T[] buildConfiguration(T... cfgs) {
+    private static <T> T[] buildConfiguration(T... cfgs) {
         return cfgs;
     }
 
     /**
-     *
+     * @return Cache configuration.
      */
     protected CacheConfiguration atomicCfg() {
         return new CacheConfiguration(cache1)
@@ -228,6 +270,7 @@ public abstract class AbstractNodeJoinTemplate extends GridCommonAbstractTest {
     }
 
     /**
+     * @return Cache configuration.
      *
      */
     protected CacheConfiguration transactionCfg() {
@@ -236,7 +279,7 @@ public abstract class AbstractNodeJoinTemplate extends GridCommonAbstractTest {
     }
 
     /**
-     *
+     * @return Cache configurations.
      */
     protected CacheConfiguration[] allCacheConfigurations() {
         return buildConfiguration(atomicCfg(), transactionCfg());
@@ -309,6 +352,7 @@ public abstract class AbstractNodeJoinTemplate extends GridCommonAbstractTest {
         /** After de activate. */
         private Runnable afterDeActivate = Noop;
 
+        /** */
         private IgniteCallable<List<CacheConfiguration>> dynamicCacheStart =
             new IgniteCallable<List<CacheConfiguration>>() {
                 @Override public List<CacheConfiguration> call() throws Exception {
@@ -316,6 +360,7 @@ public abstract class AbstractNodeJoinTemplate extends GridCommonAbstractTest {
                 }
             };
 
+        /** */
         private IgniteCallable<List<String>> dynamicCacheStop =
             new IgniteCallable<List<String>>() {
                 @Override public List<String> call() throws Exception {
@@ -323,15 +368,19 @@ public abstract class AbstractNodeJoinTemplate extends GridCommonAbstractTest {
                 }
             };
 
+        /** */
         private Runnable afterDynamicCacheStarted = Noop;
 
+        /** */
         private Runnable afterDynamicCacheStopped = Noop;
 
         /** End. */
         private Runnable end = Noop;
 
         /**
-         *
+         * @param cfgs Configurations.
+         * @return Test builder.
+         * @throws Exception If failed.
          */
         public JoinNodeTestPlanBuilder clusterConfiguration(IgniteConfiguration... cfgs) throws Exception {
             clusterCfg = cfgs;
@@ -359,7 +408,8 @@ public abstract class AbstractNodeJoinTemplate extends GridCommonAbstractTest {
         }
 
         /**
-         *
+         * @param cfg Configuration.
+         * @return Test builder.
          */
         public JoinNodeTestPlanBuilder nodeConfiguration(IgniteConfiguration cfg) {
             nodeCfg = cfg;
@@ -382,6 +432,7 @@ public abstract class AbstractNodeJoinTemplate extends GridCommonAbstractTest {
 
         /**
          * @param func Func.
+         * @return Test builder.
          */
         public JoinNodeTestPlanBuilder nodeConfiguration(
             IgniteClosure<IgniteConfiguration, IgniteConfiguration> func
@@ -393,7 +444,8 @@ public abstract class AbstractNodeJoinTemplate extends GridCommonAbstractTest {
         }
 
         /**
-         *
+         * @param r Cluster start callback.
+         * @return Test builder.
          */
         public JoinNodeTestPlanBuilder afterClusterStarted(Runnable r) {
             strPlanBuilder.append("Check after cluster start\n");
@@ -404,7 +456,8 @@ public abstract class AbstractNodeJoinTemplate extends GridCommonAbstractTest {
         }
 
         /**
-         *
+         * @param r Node join callback.
+         * @return Test builder.
          */
         public JoinNodeTestPlanBuilder afterNodeJoin(Runnable r) {
             strPlanBuilder.append("Check after node join")
@@ -416,7 +469,8 @@ public abstract class AbstractNodeJoinTemplate extends GridCommonAbstractTest {
         }
 
         /**
-         *
+         * @param state State after join.
+         * @return Test builder.
          */
         public JoinNodeTestPlanBuilder stateAfterJoin(boolean state) {
             strPlanBuilder.append("Check state on all nodes after join, must be ")
@@ -429,7 +483,8 @@ public abstract class AbstractNodeJoinTemplate extends GridCommonAbstractTest {
         }
 
         /**
-         *
+         * @param r Activate callback.
+         * @return Test builder.
          */
         public JoinNodeTestPlanBuilder afterActivate(Runnable r) {
             strPlanBuilder.append("Check after activate")
@@ -441,7 +496,8 @@ public abstract class AbstractNodeJoinTemplate extends GridCommonAbstractTest {
         }
 
         /**
-         *
+         * @param r Deactivate callback.
+         * @return Test builder.
          */
         public JoinNodeTestPlanBuilder afterDeActivate(Runnable r) {
             strPlanBuilder.append("Check after deActivate")
@@ -452,6 +508,10 @@ public abstract class AbstractNodeJoinTemplate extends GridCommonAbstractTest {
             return this;
         }
 
+        /**
+         * @param caches Callback.
+         * @return Test builder.
+         */
         public JoinNodeTestPlanBuilder dynamicCacheStart(IgniteCallable<List<CacheConfiguration>> caches){
             strPlanBuilder.append("Dynamic caches start")
                 .append("\n");
@@ -461,6 +521,10 @@ public abstract class AbstractNodeJoinTemplate extends GridCommonAbstractTest {
             return this;
         }
 
+        /**
+         * @param r Cache start callback.
+         * @return Test builder.
+         */
         public JoinNodeTestPlanBuilder afterDynamicCacheStarted(Runnable r){
             strPlanBuilder.append("Check after dynamic caches start")
                 .append("\n");
@@ -470,6 +534,10 @@ public abstract class AbstractNodeJoinTemplate extends GridCommonAbstractTest {
             return this;
         }
 
+        /**
+         * @param caches Callback.
+         * @return Test builder.
+         */
         public JoinNodeTestPlanBuilder dynamicCacheStop(IgniteCallable<List<String>> caches){
             strPlanBuilder.append("Dynamic caches stop")
                 .append("\n");
@@ -479,6 +547,10 @@ public abstract class AbstractNodeJoinTemplate extends GridCommonAbstractTest {
             return this;
         }
 
+        /**
+         * @param r Callback.
+         * @return Test builder.
+         */
         public JoinNodeTestPlanBuilder afterDynamicCacheStopped(Runnable r){
             strPlanBuilder.append("Check after dynamic caches stop")
                 .append("\n");
@@ -490,6 +562,7 @@ public abstract class AbstractNodeJoinTemplate extends GridCommonAbstractTest {
 
         /**
          * @param end End.
+         * @return Test builder.
          */
         public JoinNodeTestPlanBuilder setEnd(Runnable end) {
             strPlanBuilder.append("Check before stop")
@@ -501,7 +574,7 @@ public abstract class AbstractNodeJoinTemplate extends GridCommonAbstractTest {
         }
 
         /**
-         *
+         * @throws Exception If failed.
          */
         public void execute() throws Exception {
             try {
@@ -611,12 +684,19 @@ public abstract class AbstractNodeJoinTemplate extends GridCommonAbstractTest {
             }
         }
 
+        /**
+         * @param ig Node.
+         * @return Next minor version.
+         */
         private AffinityTopologyVersion nextMinorVersion(IgniteEx ig){
             AffinityTopologyVersion cur = ig.context().discovery().topologyVersionEx();
 
-           return new AffinityTopologyVersion(cur.topologyVersion(), cur.minorTopologyVersion() + 1);
+           return cur.nextMinorVersion();
         }
 
+        /**
+         * @param ver Version.
+         */
         private void awaitTopologyVersion(final AffinityTopologyVersion ver){
             onAllNode(new CI1<IgniteEx>() {
                 @Override public void apply(IgniteEx ig) {
@@ -643,7 +723,7 @@ public abstract class AbstractNodeJoinTemplate extends GridCommonAbstractTest {
         }
 
         /**
-         *
+         * @return Started nodes.
          */
         protected List<IgniteEx> grids() {
             List<IgniteEx> res = new ArrayList<>();
@@ -655,14 +735,14 @@ public abstract class AbstractNodeJoinTemplate extends GridCommonAbstractTest {
         }
 
         /**
-         *
+         * @return Test builder.
          */
         public static JoinNodeTestPlanBuilder builder() {
             return new JoinNodeTestPlanBuilder();
         }
 
         /**
-         *
+         * @return Callback.
          */
         public Runnable checkCacheOnlySystem() {
             return onAllNode(new IgniteInClosure<IgniteEx>() {
@@ -686,15 +766,11 @@ public abstract class AbstractNodeJoinTemplate extends GridCommonAbstractTest {
         }
 
         /**
-         *
+         * @return Callback.
          */
         public Runnable checkCacheEmpty() {
             return onAllNode(new IgniteInClosure<IgniteEx>() {
                 @Override public void apply(IgniteEx ig) {
-                    Map<String, DynamicCacheDescriptor> desc = cacheDescriptors(ig);
-
-                    Assert.assertTrue(desc.isEmpty());
-
                     Assert.assertNull(ig.context().cache().cache(cache1));
                     Assert.assertNull(ig.context().cache().cache(cache2));
 
@@ -706,7 +782,7 @@ public abstract class AbstractNodeJoinTemplate extends GridCommonAbstractTest {
         }
 
         /**
-         *
+         * @return Callback.
          */
         public Runnable checkCacheNotEmpty() {
             return onAllNode(new IgniteInClosure<IgniteEx>() {
@@ -735,6 +811,7 @@ public abstract class AbstractNodeJoinTemplate extends GridCommonAbstractTest {
 
         /**
          * @param cls Closure.
+         * @return Callback.
          */
         private Runnable onAllNode(final IgniteInClosure<IgniteEx> cls) {
             return new Runnable() {

http://git-wip-us.apache.org/repos/asf/ignite/blob/1337901f/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/standbycluster/IgniteChangeGlobalStateAbstractTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/standbycluster/IgniteChangeGlobalStateAbstractTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/standbycluster/IgniteChangeGlobalStateAbstractTest.java
index a06e0ce..4e575cc 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/standbycluster/IgniteChangeGlobalStateAbstractTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/standbycluster/IgniteChangeGlobalStateAbstractTest.java
@@ -47,13 +47,10 @@ public abstract class IgniteChangeGlobalStateAbstractTest extends GridCommonAbst
     private static final String clientSuffix = "-client";
 
     /** Primary ip finder. */
-    protected final TcpDiscoveryIpFinder primaryIpFinder = new TcpDiscoveryVmIpFinder(true);
+    private final TcpDiscoveryIpFinder primaryIpFinder = new TcpDiscoveryVmIpFinder(true);
 
     /** Back up ip finder. */
-    protected final TcpDiscoveryIpFinder backUpIpFinder = new TcpDiscoveryVmIpFinder(true);
-
-    /** Consistent id count. */
-    private int consistentIdCnt;
+    private final TcpDiscoveryIpFinder backUpIpFinder = new TcpDiscoveryVmIpFinder(true);
 
     /** Nodes. */
     protected Map<String, Ignite> nodes = new ConcurrentHashMap<>();
@@ -91,28 +88,28 @@ public abstract class IgniteChangeGlobalStateAbstractTest extends GridCommonAbst
     }
 
     /**
-     *
+     * @return Number of server nodes in primary cluster.
      */
     protected int primaryNodes() {
         return 3;
     }
 
     /**
-     *
+     * @return Number of client nodes in primary cluster.
      */
     protected int primaryClientNodes() {
         return 3;
     }
 
     /**
-     *
+     * @return Number of server nodes in backup cluster.
      */
     protected int backUpNodes() {
         return 3;
     }
 
     /**
-     *
+     * @return Number of client nodes in backup cluster.
      */
     protected int backUpClientNodes() {
         return 3;
@@ -120,6 +117,7 @@ public abstract class IgniteChangeGlobalStateAbstractTest extends GridCommonAbst
 
     /**
      * @param idx idx.
+     * @return Primary cluster node.
      */
     protected Ignite primary(int idx) {
         return nodes.get("node" + idx + primarySuffix);
@@ -127,29 +125,33 @@ public abstract class IgniteChangeGlobalStateAbstractTest extends GridCommonAbst
 
     /**
      * @param idx idx.
+     * @return Primary cluster client node.
      */
-    protected Ignite primaryClient(int idx) {
+    Ignite primaryClient(int idx) {
         return nodes.get("node" + idx + primarySuffix + clientSuffix);
     }
 
     /**
      * @param idx idx.
+     * @return Backup cluster node.
      */
-    protected Ignite backUp(int idx) {
+    Ignite backUp(int idx) {
         return nodes.get("node" + idx + backUpSuffix);
     }
 
     /**
      * @param idx idx.
+     * @return Backup cluster client node.
      */
-    protected Ignite backUpClient(int idx) {
+    Ignite backUpClient(int idx) {
         return nodes.get("node" + idx + backUpSuffix + clientSuffix);
     }
 
     /**
      * @param cnt Count.
+     * @throws Exception If failed.
      */
-    protected void startPrimaryNodes(int cnt) throws Exception {
+    private void startPrimaryNodes(int cnt) throws Exception {
         for (int i = 0; i < cnt; i++)
             startPrimary(i);
 
@@ -159,8 +161,9 @@ public abstract class IgniteChangeGlobalStateAbstractTest extends GridCommonAbst
 
     /**
      * @param idx Index.
+     * @throws Exception If failed.
      */
-    protected void startPrimary(int idx) throws Exception {
+    private void startPrimary(int idx) throws Exception {
         String node = "node" + idx;
 
         String name = node + primarySuffix;
@@ -176,22 +179,26 @@ public abstract class IgniteChangeGlobalStateAbstractTest extends GridCommonAbst
 
     /**
      * @param cnt Count.
+     * @throws Exception If failed.
      */
-    protected void startBackUpNodes(int cnt) throws Exception {
+    private void startBackUpNodes(int cnt) throws Exception {
         for (int i = 0; i < cnt; i++)
             startBackUp(i);
     }
 
     /**
      * @param idx Index.
+     * @throws Exception If failed.
      */
-    protected void startBackUp(int idx) throws Exception {
+    void startBackUp(int idx) throws Exception {
         String node = "node" + idx;
 
         String name = node + backUpSuffix;
 
         IgniteConfiguration cfg = getConfiguration(name);
+
         cfg.setConsistentId(node);
+
         ((TcpDiscoverySpi)cfg.getDiscoverySpi()).setIpFinder(backUpIpFinder);
 
         Ignite ig = startGrid(name, cfg);
@@ -201,16 +208,19 @@ public abstract class IgniteChangeGlobalStateAbstractTest extends GridCommonAbst
 
     /**
      * @param cnt Count.
+     * @throws Exception If failed.
      */
-    protected void startPrimaryClientNodes(int cnt) throws Exception {
+    void startPrimaryClientNodes(int cnt) throws Exception {
         for (int i = 0; i < cnt; i++) {
             String node = "node" + i;
 
             String name = node + primarySuffix + clientSuffix;
 
             IgniteConfiguration cfg = getConfiguration(name);
+
             cfg.setConsistentId(node);
             cfg.setClientMode(true);
+
             ((TcpDiscoverySpi)cfg.getDiscoverySpi()).setIpFinder(primaryIpFinder);
 
             Ignite ig = startGrid(name, cfg);
@@ -221,8 +231,9 @@ public abstract class IgniteChangeGlobalStateAbstractTest extends GridCommonAbst
 
     /**
      * @param cnt Count.
+     * @throws Exception If failed.
      */
-    protected void startBackUpClientNodes(int cnt) throws Exception {
+    private void startBackUpClientNodes(int cnt) throws Exception {
         for (int i = 0; i < cnt; i++) {
             String node = "node" + i;
 
@@ -241,9 +252,9 @@ public abstract class IgniteChangeGlobalStateAbstractTest extends GridCommonAbst
     }
 
     /**
-     *
+     * @return All nodes from backup cluster.
      */
-    protected Iterable<Ignite> allBackUpNodes() {
+    Iterable<Ignite> allBackUpNodes() {
         List<Ignite> r = new ArrayList<>();
 
         for (String name : this.nodes.keySet())
@@ -254,11 +265,10 @@ public abstract class IgniteChangeGlobalStateAbstractTest extends GridCommonAbst
     }
 
     /**
-     *
+     * @param includeClient If {@code true} then allow to return client.
+     * @return Random node from backup topology.
      */
-    protected Ignite randomBackUp(boolean includeClient) {
-        int nodes = 0;
-
+    Ignite randomBackUp(boolean includeClient) {
         List<Ignite> igs = new ArrayList<>();
 
         for (String name : this.nodes.keySet())
@@ -281,7 +291,7 @@ public abstract class IgniteChangeGlobalStateAbstractTest extends GridCommonAbst
     /**
      * @param i Idx.
      */
-    protected void stopPrimary(int i) {
+    void stopPrimary(int i) {
         String name = "node" + i + primarySuffix;
 
         nodes.get(name).close();
@@ -292,7 +302,7 @@ public abstract class IgniteChangeGlobalStateAbstractTest extends GridCommonAbst
     /**
      *
      */
-    protected void stopAllPrimary() {
+    void stopAllPrimary() {
         stopAll(primarySuffix);
     }
 
@@ -357,10 +367,9 @@ public abstract class IgniteChangeGlobalStateAbstractTest extends GridCommonAbst
     }
 
     /**
-     *
+     * @return Test class name.
      */
     protected String testName() {
         return getClass().getSimpleName();
     }
-
 }

http://git-wip-us.apache.org/repos/asf/ignite/blob/1337901f/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/standbycluster/IgniteChangeGlobalStateCacheTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/standbycluster/IgniteChangeGlobalStateCacheTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/standbycluster/IgniteChangeGlobalStateCacheTest.java
index 2f2385d..938b3c8 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/standbycluster/IgniteChangeGlobalStateCacheTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/standbycluster/IgniteChangeGlobalStateCacheTest.java
@@ -62,7 +62,7 @@ public class IgniteChangeGlobalStateCacheTest extends IgniteChangeGlobalStateAbs
     }
 
     /**
-     *
+     * @throws Exception If failed.
      */
     public void testMoreKeyValueAfterActivate() throws Exception {
         String cacheName = "my-cache";

http://git-wip-us.apache.org/repos/asf/ignite/blob/1337901f/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/standbycluster/IgniteChangeGlobalStateDataStreamerTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/standbycluster/IgniteChangeGlobalStateDataStreamerTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/standbycluster/IgniteChangeGlobalStateDataStreamerTest.java
index d3e8298..16be316 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/standbycluster/IgniteChangeGlobalStateDataStreamerTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/standbycluster/IgniteChangeGlobalStateDataStreamerTest.java
@@ -36,10 +36,9 @@ public class IgniteChangeGlobalStateDataStreamerTest extends IgniteChangeGlobalS
     }
 
     /**
-     *
+     * @throws Exception If failed.
      */
-    public void testDeActivateAndActivateDataStreamer() throws InterruptedException {
-
+    public void testDeActivateAndActivateDataStreamer() throws Exception {
         Ignite ig1 = primary(0);
         Ignite ig2 = primary(1);
         Ignite ig3 = primary(2);

http://git-wip-us.apache.org/repos/asf/ignite/blob/1337901f/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/standbycluster/IgniteChangeGlobalStateDataStructureTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/standbycluster/IgniteChangeGlobalStateDataStructureTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/standbycluster/IgniteChangeGlobalStateDataStructureTest.java
index 9013226..8902a36 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/standbycluster/IgniteChangeGlobalStateDataStructureTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/standbycluster/IgniteChangeGlobalStateDataStructureTest.java
@@ -35,9 +35,9 @@ import static org.apache.ignite.testframework.GridTestUtils.runAsync;
  */
 public class IgniteChangeGlobalStateDataStructureTest extends IgniteChangeGlobalStateAbstractTest {
     /**
-     *
+     * @throws Exception If failed.
      */
-    public void testDeActivateAndActivateAtomicLong() throws Exception{
+    public void testDeActivateAndActivateAtomicLong() throws Exception {
         String lName = "myLong";
 
         Ignite ig1 = primary(0);
@@ -106,7 +106,7 @@ public class IgniteChangeGlobalStateDataStructureTest extends IgniteChangeGlobal
     }
 
     /**
-     *
+     * @throws Exception If failed.
      */
     public void testDeActivateAndActivateCountDownLatch() throws Exception {
         final AtomicInteger cnt = new AtomicInteger();


[33/33] ignite git commit: Merge.

Posted by sb...@apache.org.
Merge.


Project: http://git-wip-us.apache.org/repos/asf/ignite/repo
Commit: http://git-wip-us.apache.org/repos/asf/ignite/commit/5b2400ab
Tree: http://git-wip-us.apache.org/repos/asf/ignite/tree/5b2400ab
Diff: http://git-wip-us.apache.org/repos/asf/ignite/diff/5b2400ab

Branch: refs/heads/ignite-2.1.2-exchange
Commit: 5b2400ab4a0c43a1b8597fd674d5c6a373bd3c43
Parents: 377cc9d aa21a9b
Author: sboikov <sb...@gridgain.com>
Authored: Wed Jul 5 14:22:31 2017 +0300
Committer: sboikov <sb...@gridgain.com>
Committed: Wed Jul 5 14:22:31 2017 +0300

----------------------------------------------------------------------
 .../jdbc2/JdbcPreparedStatementSelfTest.java    |   35 +
 .../jdbc/JdbcPreparedStatementSelfTest.java     |   35 +
 .../thin/JdbcThinPreparedStatementSelfTest.java |   35 +
 .../PersistentStoreConfiguration.java           |   39 +-
 .../org/apache/ignite/events/EventType.java     |   12 +
 .../ignite/events/WalSegmentArchivedEvent.java  |   62 +
 .../apache/ignite/internal/GridComponent.java   |    4 +-
 .../ignite/internal/GridPluginComponent.java    |    2 +-
 .../apache/ignite/internal/IgniteKernal.java    |   33 +-
 .../internal/jdbc/JdbcPreparedStatement.java    |    6 +-
 .../internal/jdbc/thin/JdbcThinConnection.java  |    7 +-
 .../jdbc/thin/JdbcThinPreparedStatement.java    |    2 -
 .../internal/jdbc2/JdbcPreparedStatement.java   |   12 +-
 .../internal/managers/GridManagerAdapter.java   |    2 +-
 .../checkpoint/GridCheckpointManager.java       |    2 +-
 .../managers/communication/GridIoManager.java   |   10 +-
 .../communication/GridMessageListener.java      |    3 +-
 .../deployment/GridDeploymentCommunication.java |    4 +-
 .../internal/managers/discovery/DiscoCache.java |   17 +-
 .../discovery/DiscoveryLocalJoinData.java       |  104 ++
 .../discovery/GridDiscoveryManager.java         |  128 +-
 .../eventstorage/GridEventStorageManager.java   |    4 +-
 .../pagemem/store/IgnitePageStoreManager.java   |    3 +-
 .../internal/pagemem/wal/record/WALRecord.java  |   11 +-
 .../processors/GridProcessorAdapter.java        |    2 +-
 .../cache/CacheAffinitySharedManager.java       |   67 +-
 .../processors/cache/CacheGroupContext.java     |    4 +-
 .../processors/cache/CacheGroupData.java        |    4 +-
 .../cache/ChangeGlobalStateMessage.java         |  120 --
 .../processors/cache/ClusterCachesInfo.java     |  493 +++++--
 .../internal/processors/cache/ClusterState.java |   38 -
 .../cache/DynamicCacheChangeRequest.java        |   52 +-
 .../processors/cache/ExchangeActions.java       |   37 +-
 .../processors/cache/GridCacheEventManager.java |    2 -
 .../cache/GridCacheEvictionManager.java         |    1 -
 .../processors/cache/GridCacheIoManager.java    |   98 +-
 .../processors/cache/GridCacheMvccManager.java  |    9 +-
 .../GridCachePartitionExchangeManager.java      |  439 +++---
 .../processors/cache/GridCacheProcessor.java    |  193 +--
 .../cache/GridCacheSharedContext.java           |   60 +-
 .../cache/GridCacheSharedManager.java           |    6 -
 .../cache/GridCacheSharedManagerAdapter.java    |   16 -
 .../cache/GridDeferredAckMessageSender.java     |    3 +-
 .../processors/cache/PendingDiscoveryEvent.java |   61 +
 .../processors/cache/StateChangeRequest.java    |   77 ++
 .../cache/binary/BinaryMetadataTransport.java   |    4 +-
 .../binary/CacheObjectBinaryProcessorImpl.java  |    4 +-
 .../distributed/GridCacheTxRecoveryFuture.java  |    1 -
 .../distributed/dht/GridDhtCacheAdapter.java    |    1 -
 .../cache/distributed/dht/GridDhtGetFuture.java |    1 -
 .../distributed/dht/GridDhtGetSingleFuture.java |    2 -
 .../dht/GridDhtPartitionTopologyImpl.java       |   13 +-
 .../dht/GridDhtTopologyFutureAdapter.java       |    2 +-
 .../dht/GridPartitionedSingleGetFuture.java     |    3 -
 .../GridNearAtomicAbstractUpdateFuture.java     |    1 -
 .../dht/preloader/GridDhtForceKeysFuture.java   |    1 -
 .../dht/preloader/GridDhtPartitionDemander.java |    2 +
 .../GridDhtPartitionsExchangeFuture.java        |  240 +++-
 .../preloader/GridDhtPartitionsFullMessage.java |   44 +-
 .../GridDhtPartitionsSingleMessage.java         |   38 +-
 .../dht/preloader/GridDhtPreloader.java         |    2 +-
 .../distributed/near/GridNearGetFuture.java     |    2 -
 .../near/GridNearTxPrepareRequest.java          |    1 -
 .../GridCacheDatabaseSharedManager.java         |  111 +-
 .../persistence/GridCacheOffheapManager.java    |    5 +-
 .../IgniteCacheDatabaseSharedManager.java       |   74 +-
 .../persistence/IgniteCacheSnapshotManager.java |   20 +-
 .../persistence/file/FilePageStoreManager.java  |   14 +-
 .../persistence/tree/io/TrackingPageIO.java     |   12 +-
 .../wal/AbstractWalRecordsIterator.java         |  289 ++++
 .../cache/persistence/wal/FileInput.java        |   16 +-
 .../cache/persistence/wal/FileWALPointer.java   |    4 +-
 .../wal/FileWriteAheadLogManager.java           |  594 ++++----
 .../cache/persistence/wal/RecordSerializer.java |    5 +
 .../persistence/wal/SegmentArchiveResult.java   |   61 +
 .../persistence/wal/SegmentEofException.java    |    3 +-
 .../wal/reader/IgniteWalIteratorFactory.java    |  102 ++
 .../wal/reader/StandaloneGridKernalContext.java |  499 +++++++
 ...ndaloneIgniteCacheDatabaseSharedManager.java |   30 +
 .../reader/StandaloneWalRecordsIterator.java    |  258 ++++
 .../wal/serializer/RecordV1Serializer.java      |   45 +-
 .../query/GridCacheDistributedQueryManager.java |    4 +-
 .../store/GridCacheStoreManagerAdapter.java     |    1 -
 .../cache/transactions/IgniteTxManager.java     |    2 +-
 .../cache/version/GridCacheVersionManager.java  |    6 -
 .../cacheobject/IgniteCacheObjectProcessor.java |    5 -
 .../IgniteCacheObjectProcessorImpl.java         |    5 -
 .../cluster/ChangeGlobalStateFinishMessage.java |   86 ++
 .../cluster/ChangeGlobalStateMessage.java       |  140 ++
 .../processors/cluster/ClusterProcessor.java    |    5 +-
 .../cluster/DiscoveryDataClusterState.java      |  157 +++
 .../cluster/GridClusterStateProcessor.java      | 1129 ++++++---------
 .../cluster/IgniteChangeGlobalStateSupport.java |    3 +-
 .../continuous/GridContinuousProcessor.java     |    4 +-
 .../datastreamer/DataStreamProcessor.java       |    2 +-
 .../datastreamer/DataStreamerImpl.java          |    2 +-
 .../datastructures/DataStructuresProcessor.java |    6 +-
 .../datastructures/GridCacheAtomicLongImpl.java |    2 +-
 .../GridCacheAtomicReferenceImpl.java           |    2 +-
 .../GridCacheAtomicSequenceImpl.java            |    2 +-
 .../GridCacheAtomicStampedImpl.java             |    2 +-
 .../GridCacheCountDownLatchImpl.java            |    2 +-
 .../datastructures/GridCacheLockImpl.java       |    4 +-
 .../datastructures/GridCacheQueueAdapter.java   |    1 -
 .../datastructures/GridCacheSemaphoreImpl.java  |    2 +-
 .../datastructures/GridCacheSetImpl.java        |    1 -
 .../processors/igfs/IgfsDataManager.java        |    2 +-
 .../igfs/IgfsFragmentizerManager.java           |    4 +-
 .../internal/processors/igfs/IgfsImpl.java      |    2 -
 .../internal/processors/igfs/IgfsProcessor.java |    2 +-
 .../processors/job/GridJobProcessor.java        |    8 +-
 .../GridMarshallerMappingProcessor.java         |    4 +-
 .../utils/PlatformConfigurationUtils.java       |    8 +-
 .../processors/query/GridQueryProcessor.java    |    6 +-
 .../processors/rest/GridRestProcessor.java      |    2 +-
 .../cluster/GridChangeStateCommandHandler.java  |    2 +-
 .../handlers/task/GridTaskCommandHandler.java   |    4 +-
 .../service/GridServiceProcessor.java           |    6 +-
 .../processors/task/GridTaskProcessor.java      |    8 +-
 .../visor/cache/VisorCacheConfiguration.java    |   11 +-
 .../visor/node/VisorBasicConfiguration.java     |    2 +-
 .../node/VisorMemoryPolicyConfiguration.java    |    2 +-
 .../visor/node/VisorNodeDataCollectorJob.java   |   32 +-
 .../node/VisorNodeDataCollectorJobResult.java   |   73 +-
 .../visor/node/VisorNodeDataCollectorTask.java  |   14 +-
 .../node/VisorNodeDataCollectorTaskResult.java  |   28 +-
 .../visor/node/VisorPersistenceMetrics.java     |  214 +++
 .../visor/service/VisorServiceTask.java         |   10 +-
 .../jobstealing/JobStealingCollisionSpi.java    |    2 +-
 .../ignite/spi/discovery/tcp/ClientImpl.java    |   12 +-
 .../ignite/spi/discovery/tcp/ServerImpl.java    |   10 +-
 .../internal/TestRecordingCommunicationSpi.java |   10 +
 ...GridManagerLocalMessageListenerSelfTest.java |    4 +-
 ...idCommunicationManagerListenersSelfTest.java |    2 +-
 .../GridCommunicationSendMessageSelfTest.java   |    2 +-
 ...unicationBalanceMultipleConnectionsTest.java |    5 +
 .../cache/GridCacheAbstractMetricsSelfTest.java |   24 -
 .../cache/GridCachePartitionedGetSelfTest.java  |    2 +-
 .../cache/IgniteActiveClusterTest.java          |  182 ---
 .../IgniteClusterActivateDeactivateTest.java    | 1284 ++++++++++++++++++
 ...erActivateDeactivateTestWithPersistence.java |  197 +++
 .../IgniteDaemonNodeMarshallerCacheTest.java    |   10 -
 ...lerCacheClientRequestsMappingOnMissTest.java |    6 +-
 ...naryObjectMetadataExchangeMultinodeTest.java |    6 +-
 .../IgniteSemaphoreAbstractSelfTest.java        |   17 +-
 ...IgnitePersistentStoreDataStructuresTest.java |    2 +
 .../wal/IgniteWalHistoryReservationsTest.java   |    2 +-
 .../db/wal/reader/IgniteWalReaderTest.java      |  385 ++++++
 .../db/wal/reader/MockWalIteratorFactory.java   |  114 ++
 .../pagemem/NoOpPageStoreManager.java           |   12 +-
 .../persistence/pagemem/NoOpWALManager.java     |   23 +-
 .../AbstractNodeJoinTemplate.java               |  149 +-
 .../IgniteChangeGlobalStateAbstractTest.java    |   65 +-
 .../IgniteChangeGlobalStateCacheTest.java       |    2 +-
 ...IgniteChangeGlobalStateDataStreamerTest.java |    5 +-
 ...gniteChangeGlobalStateDataStructureTest.java |    6 +-
 .../IgniteChangeGlobalStateFailOverTest.java    |   26 +-
 .../IgniteChangeGlobalStateTest.java            |  158 +--
 .../IgniteStandByClusterTest.java               |   17 +-
 .../join/JoinActiveNodeToActiveCluster.java     |   62 +-
 ...ctiveNodeToActiveClusterWithPersistence.java |   17 +
 .../IgniteStandByClientReconnectTest.java       |   13 +-
 ...eStandByClientReconnectToNewClusterTest.java |   13 +-
 ...DeadlockDetectionMessageMarshallingTest.java |    2 +-
 .../communication/GridIoManagerBenchmark.java   |    4 +-
 .../communication/GridIoManagerBenchmark0.java  |   12 +-
 .../communication/GridCacheMessageSelfTest.java |    2 +-
 ...cpCommunicationSpiMultithreadedSelfTest.java |    2 +-
 .../testframework/GridSpiTestContext.java       |    5 +-
 .../testframework/junits/GridAbstractTest.java  |    4 +-
 .../junits/common/GridCommonAbstractTest.java   |    3 +
 .../ignite/testsuites/IgnitePdsTestSuite2.java  |    9 +-
 .../testsuites/IgniteStandByClusterSuite.java   |    5 +-
 .../processors/hadoop/HadoopProcessor.java      |    4 +-
 .../hadoop/shuffle/HadoopShuffle.java           |    2 +-
 ...ileSystemShmemExternalDualAsyncSelfTest.java |    5 +
 .../query/h2/opt/GridH2IndexBase.java           |    2 +-
 .../query/h2/twostep/GridMapQueryExecutor.java  |    2 +-
 .../h2/twostep/GridReduceQueryExecutor.java     |    2 +-
 .../cache/IgniteCacheAbstractQuerySelfTest.java |    3 +-
 .../Cache/CacheConfigurationTest.cs             |   21 +
 .../Apache.Ignite.Core.Tests/EventsTest.cs      |   19 +-
 .../Cache/Configuration/CacheConfiguration.cs   |   75 +-
 .../Apache.Ignite.Core/IgniteConfiguration.cs   |    7 +-
 .../Impl/Events/RemoteListenEventFilter.cs      |    3 +
 .../Apache.Ignite.Core/Impl/IgniteUtils.cs      |   12 +-
 .../spark/JavaEmbeddedIgniteRDDSelfTest.java    |    5 +
 .../frontend/app/filters/duration.filter.js     |    2 +-
 .../generator/ConfigurationGenerator.js         |    9 +-
 .../states/configuration/caches/affinity.pug    |   18 +-
 .../states/configuration/caches/general.pug     |   50 +-
 191 files changed, 7182 insertions(+), 2830 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/5b2400ab/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/CacheAffinitySharedManager.java
----------------------------------------------------------------------
diff --cc modules/core/src/main/java/org/apache/ignite/internal/processors/cache/CacheAffinitySharedManager.java
index 6301497,8d08c3f..a8af753
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/CacheAffinitySharedManager.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/CacheAffinitySharedManager.java
@@@ -53,7 -53,8 +54,9 @@@ import org.apache.ignite.internal.proce
  import org.apache.ignite.internal.processors.cache.distributed.dht.GridDhtPartitionTopology;
  import org.apache.ignite.internal.processors.cache.distributed.dht.preloader.GridDhtPartitionFullMap;
  import org.apache.ignite.internal.processors.cache.distributed.dht.preloader.GridDhtPartitionsExchangeFuture;
 +import org.apache.ignite.internal.util.GridAtomicLong;
+ import org.apache.ignite.internal.processors.cluster.ChangeGlobalStateFinishMessage;
+ import org.apache.ignite.internal.processors.cluster.DiscoveryDataClusterState;
  import org.apache.ignite.internal.util.future.GridCompoundFuture;
  import org.apache.ignite.internal.util.future.GridFinishedFuture;
  import org.apache.ignite.internal.util.future.GridFutureAdapter;

http://git-wip-us.apache.org/repos/asf/ignite/blob/5b2400ab/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCachePartitionExchangeManager.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/ignite/blob/5b2400ab/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheProcessor.java
----------------------------------------------------------------------
diff --cc modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheProcessor.java
index e238b11,624dec0..dab678a
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheProcessor.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheProcessor.java
@@@ -2170,9 -2148,10 +2150,12 @@@ public class GridCacheProcessor extend
  
              if (!sharedCtx.kernalContext().clientNode())
                  sharedCtx.database().onCacheGroupsStopped(stoppedGroups);
+ 
+             if (exchActions.deactivate())
+                 sharedCtx.deactivate();
          }
 +
 +        log.info("processor onExchangeDone [topVer=" + topVer + ", time=" + (U.currentTimeMillis() - start) + ']');
      }
  
      /**

http://git-wip-us.apache.org/repos/asf/ignite/blob/5b2400ab/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/preloader/GridDhtPartitionsExchangeFuture.java
----------------------------------------------------------------------
diff --cc modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/preloader/GridDhtPartitionsExchangeFuture.java
index 4e46daa,cea758a..3479b4e
--- 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
@@@ -77,9 -73,9 +77,10 @@@ import org.apache.ignite.internal.proce
  import org.apache.ignite.internal.processors.cache.distributed.dht.GridDhtPartitionTopology;
  import org.apache.ignite.internal.processors.cache.transactions.IgniteTxKey;
  import org.apache.ignite.internal.processors.cache.version.GridCacheVersion;
- import org.apache.ignite.internal.processors.cluster.GridClusterStateProcessor;
+ import org.apache.ignite.internal.processors.cluster.ChangeGlobalStateFinishMessage;
+ import org.apache.ignite.internal.processors.cluster.ChangeGlobalStateMessage;
  import org.apache.ignite.internal.processors.timeout.GridTimeoutObjectAdapter;
 +import org.apache.ignite.internal.util.GridAtomicLong;
  import org.apache.ignite.internal.util.future.GridCompoundFuture;
  import org.apache.ignite.internal.util.future.GridFutureAdapter;
  import org.apache.ignite.internal.util.tostring.GridToStringExclude;
@@@ -578,9 -571,12 +592,13 @@@ public class GridDhtPartitionsExchangeF
              if (discoEvt.type() == EVT_DISCOVERY_CUSTOM_EVT) {
                  DiscoveryCustomMessage msg = ((DiscoveryCustomEvent)discoEvt).customMessage();
  
-                 if (msg instanceof DynamicCacheChangeBatch) {
-                     long start = U.currentTimeMillis();
+                 if (msg instanceof ChangeGlobalStateMessage) {
+                     assert exchActions != null && !exchActions.empty();
  
+                     exchange = onClusterStateChangeRequest(crdNode);
+                 }
+                 else if (msg instanceof DynamicCacheChangeBatch) {
++                    long start = U.currentTimeMillis();
                      assert exchActions != null && !exchActions.empty();
  
                      exchange = onCacheChangeRequest(crdNode);
@@@ -609,12 -601,28 +627,30 @@@
  
                          cctx.affinity().initStartedCaches(crdNode, this, receivedCaches);
                      }
-                     else
-                         cctx.cache().startCachesOnLocalJoin(topVer);
+                     else {
+                         cctx.activate();
+ 
+                         List<T2<DynamicCacheDescriptor, NearCacheConfiguration>> caches =
+                             cctx.cache().cachesToStartOnLocalJoin();
+ 
+                         if (cctx.database().persistenceEnabled() &&
+                             !cctx.kernalContext().clientNode()) {
+                             List<DynamicCacheDescriptor> startDescs = new ArrayList<>();
+ 
+                             if (caches != null) {
+                                 for (T2<DynamicCacheDescriptor, NearCacheConfiguration> c : caches)
+                                     startDescs.add(c.get1());
+                             }
+ 
+                             cctx.database().readCheckpointAndRestoreMemory(startDescs);
+                         }
+ 
+                         cctx.cache().startCachesOnLocalJoin(caches, topVer);
+                     }
                  }
  
 +                exchLog.info("Caches start time [topVer=" + topologyVersion() + ", time=" + (U.currentTimeMillis() - start) + ']');
 +
                  exchange = CU.clientNode(discoEvt.eventNode()) ?
                      onClientNodeEvent(crdNode) :
                      onServerNodeEvent(crdNode);
@@@ -1227,11 -1285,10 +1333,12 @@@
  
          assert !nodes.contains(cctx.localNode());
  
 +        sndTs = U.currentTimeMillis();
 +
-         if (log.isDebugEnabled())
+         if (log.isDebugEnabled()) {
              log.debug("Sending full partition map [nodeIds=" + F.viewReadOnly(nodes, F.node2id()) +
                  ", exchId=" + exchId + ", msg=" + m + ']');
+         }
  
          for (ClusterNode node : nodes) {
              try {
@@@ -1586,18 -1568,9 +1694,18 @@@
  
          if (updateSingleMap) {
              try {
 -                // Do not update partition map, in case cluster transitioning to inactive state.
 +                cnt.incrementAndGet();
 +
 +                long start = U.currentTimeMillis();
 +
-                 if (!exchangeOnChangeGlobalState || exchActions.newClusterState() != ClusterState.INACTIVE)
+                 if (!deactivateCluster())
                      updatePartitionSingleMap(node, msg);
 +
 +                long time = U.currentTimeMillis() - start;
 +
 +                minTime.setIfLess(time);
 +                maxTime.setIfGreater(time);
 +                totTime.addAndGet(time);
              }
              finally {
                  synchronized (this) {
@@@ -2068,14 -2048,17 +2195,19 @@@
              }
          }
  
 +        rcvTs = U.currentTimeMillis();
 +
          updatePartitionFullMap(msg);
  
-         if (exchangeOnChangeGlobalState && !F.isEmpty(msg.getExceptionsMap()))
-             cctx.kernalContext().state().onFullResponseMessage(msg.getExceptionsMap());
+         IgniteCheckedException err = null;
  
-         onDone(exchId.topologyVersion());
+         if (stateChangeExchange() && !F.isEmpty(msg.getErrorsMap())) {
+             err = new IgniteCheckedException("Cluster state change failed");
+ 
+             cctx.kernalContext().state().onStateChangeError(msg.getErrorsMap(), exchActions.stateChangeRequest());
+         }
+ 
+         onDone(exchId.topologyVersion(), err);
      }
  
      /**


[23/33] ignite git commit: IGNITE-5426: JdbcThinConnection.readOnly state propagation. This closes #2200.

Posted by sb...@apache.org.
IGNITE-5426: JdbcThinConnection.readOnly state propagation. This closes #2200.


Project: http://git-wip-us.apache.org/repos/asf/ignite/repo
Commit: http://git-wip-us.apache.org/repos/asf/ignite/commit/44fad244
Tree: http://git-wip-us.apache.org/repos/asf/ignite/tree/44fad244
Diff: http://git-wip-us.apache.org/repos/asf/ignite/diff/44fad244

Branch: refs/heads/ignite-2.1.2-exchange
Commit: 44fad244aa40f552159af153fefdf25d47c7375e
Parents: 54572c3
Author: tledkov-gridgain <tl...@gridgain.com>
Authored: Wed Jul 5 11:18:48 2017 +0300
Committer: devozerov <vo...@gridgain.com>
Committed: Wed Jul 5 11:18:48 2017 +0300

----------------------------------------------------------------------
 .../apache/ignite/internal/jdbc/thin/JdbcThinConnection.java  | 7 ++++++-
 1 file changed, 6 insertions(+), 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/44fad244/modules/core/src/main/java/org/apache/ignite/internal/jdbc/thin/JdbcThinConnection.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/jdbc/thin/JdbcThinConnection.java b/modules/core/src/main/java/org/apache/ignite/internal/jdbc/thin/JdbcThinConnection.java
index 14c34ee..89ef2fc 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/jdbc/thin/JdbcThinConnection.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/jdbc/thin/JdbcThinConnection.java
@@ -76,6 +76,9 @@ public class JdbcThinConnection implements Connection {
     /** Auto commit flag. */
     private boolean autoCommit;
 
+    /** Read-only flag. */
+    private boolean readOnly;
+
     /** Current transaction holdability. */
     private int holdability;
 
@@ -277,13 +280,15 @@ public class JdbcThinConnection implements Connection {
     /** {@inheritDoc} */
     @Override public void setReadOnly(boolean readOnly) throws SQLException {
         ensureNotClosed();
+
+        this.readOnly = readOnly;
     }
 
     /** {@inheritDoc} */
     @Override public boolean isReadOnly() throws SQLException {
         ensureNotClosed();
 
-        return true;
+        return readOnly;
     }
 
     /** {@inheritDoc} */